From f92dde0e413804dcd79167efa5b9729ad200b60a Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 18 Nov 2024 16:45:14 +0100 Subject: [PATCH 01/53] feat(blooms): Apply task timeout in bloom builder (#14988) --- pkg/bloombuild/builder/builder.go | 10 ++ pkg/bloombuild/builder/builder_test.go | 183 ++++++++++++++++++++----- pkg/bloombuild/builder/config.go | 2 + 3 files changed, 162 insertions(+), 33 deletions(-) diff --git a/pkg/bloombuild/builder/builder.go b/pkg/bloombuild/builder/builder.go index 1b0a81a24725..716b26ea8987 100644 --- a/pkg/bloombuild/builder/builder.go +++ b/pkg/bloombuild/builder/builder.go @@ -368,6 +368,12 @@ func (b *Builder) processTask( logger := task.GetLogger(b.logger) level.Debug(logger).Log("msg", "task started") + if timeout := b.limits.BuilderResponseTimeout(task.Tenant); timeout > 0 { + var cancel context.CancelFunc + ctx, cancel = context.WithDeadline(ctx, time.Now().Add(timeout)) + defer cancel() + } + client, err := b.bloomStore.Client(task.Table.ModelTime()) if err != nil { level.Error(logger).Log("msg", "failed to get client", "err", err) @@ -390,6 +396,10 @@ func (b *Builder) processTask( ) for i := range task.Gaps { + if ctx.Err() != nil { + return nil, ctx.Err() + } + gap := task.Gaps[i] logger := log.With(logger, "gap", gap.Bounds.String()) diff --git a/pkg/bloombuild/builder/builder_test.go b/pkg/bloombuild/builder/builder_test.go index 6197c6209974..fcd19179c142 100644 --- a/pkg/bloombuild/builder/builder_test.go +++ b/pkg/bloombuild/builder/builder_test.go @@ -18,6 +18,7 @@ import ( "go.uber.org/atomic" "google.golang.org/grpc" + "github.com/grafana/loki/v3/pkg/bloombuild/planner/plannertest" "github.com/grafana/loki/v3/pkg/bloombuild/protos" "github.com/grafana/loki/v3/pkg/storage" v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" @@ -28,10 +29,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/types" ) -func Test_BuilderLoop(t *testing.T) { - logger := log.NewNopLogger() - //logger := log.NewLogfmtLogger(os.Stdout) - +func setupBuilder(t *testing.T, plannerAddr string, limits Limits, logger log.Logger) *Builder { schemaCfg := config.SchemaConfig{ Configs: []config.PeriodConfig{ { @@ -64,22 +62,8 @@ func Test_BuilderLoop(t *testing.T) { }, } - tasks := make([]*protos.ProtoTask, 256) - for i := range tasks { - tasks[i] = &protos.ProtoTask{ - Id: fmt.Sprintf("task-%d", i), - } - } - - server, err := newFakePlannerServer(tasks) - require.NoError(t, err) - - // Start the server so the builder can connect and receive tasks. - server.Start() - - limits := fakeLimits{} cfg := Config{ - PlannerAddress: server.Addr(), + PlannerAddress: plannerAddr, BackoffConfig: backoff.Config{ MinBackoff: 1 * time.Second, MaxBackoff: 10 * time.Second, @@ -88,8 +72,48 @@ func Test_BuilderLoop(t *testing.T) { } flagext.DefaultValues(&cfg.GrpcConfig) - builder, err := New(cfg, limits, schemaCfg, storageCfg, storage.NewClientMetrics(), nil, fakeBloomStore{}, logger, prometheus.DefaultRegisterer, nil) + metrics := storage.NewClientMetrics() + metrics.Unregister() + + builder, err := New(cfg, limits, schemaCfg, storageCfg, metrics, nil, fakeBloomStore{}, logger, prometheus.NewPedanticRegistry(), nil) + require.NoError(t, err) + + return builder +} + +func createTasks(n int) []*protos.ProtoTask { + tasks := make([]*protos.ProtoTask, n) + for i := range tasks { + tasks[i] = protos.NewTask( + plannertest.TestTable, + "fake", + v1.NewBounds(model.Fingerprint(i), model.Fingerprint(i+10)), + plannertest.TsdbID(1), + []protos.Gap{ + { + Bounds: v1.NewBounds(model.Fingerprint(i+1), model.Fingerprint(i+2)), + }, + { + Bounds: v1.NewBounds(model.Fingerprint(i+3), model.Fingerprint(i+9)), + }, + }, + ).ToProtoTask() + } + return tasks +} + +func Test_BuilderLoop(t *testing.T) { + logger := log.NewNopLogger() + //logger := log.NewLogfmtLogger(os.Stdout) + + tasks := createTasks(256) + server, err := newFakePlannerServer(tasks) require.NoError(t, err) + + // Start the server so the builder can connect and receive tasks. + server.Start() + + builder := setupBuilder(t, server.Addr(), fakeLimits{}, logger) t.Cleanup(func() { err = services.StopAndAwaitTerminated(context.Background(), builder) require.NoError(t, err) @@ -128,9 +152,71 @@ func Test_BuilderLoop(t *testing.T) { require.True(t, server.shutdownCalled) } +func Test_BuilderLoop_Timeout(t *testing.T) { + for _, tc := range []struct { + name string + timeout time.Duration + allTasksSucceed bool + }{ + { + name: "no timeout configured", + timeout: 0, + allTasksSucceed: true, + }, + { + name: "long enough timeout", + timeout: 15 * time.Minute, + allTasksSucceed: true, + }, + { + name: "task times out", + timeout: 1 * time.Nanosecond, // Pretty much immediately. + allTasksSucceed: false, + }, + } { + t.Run(tc.name, func(t *testing.T) { + logger := log.NewNopLogger() + //logger := log.NewLogfmtLogger(os.Stdout) + + tasks := createTasks(256) + server, err := newFakePlannerServer(tasks) + require.NoError(t, err) + + // Start the server so the builder can connect and receive tasks. + server.Start() + + limits := fakeLimits{ + taskTimout: tc.timeout, + } + builder := setupBuilder(t, server.Addr(), limits, logger) + t.Cleanup(func() { + err = services.StopAndAwaitTerminated(context.Background(), builder) + require.NoError(t, err) + + server.Stop() + }) + + err = services.StartAndAwaitRunning(context.Background(), builder) + require.NoError(t, err) + + require.Eventually(t, func() bool { + return server.CompletedTasks() >= len(tasks) + }, 30*time.Second, 500*time.Millisecond) + + erroredTasks := server.ErroredTasks() + if tc.allTasksSucceed { + require.Equal(t, 0, erroredTasks) + } else { + require.Equal(t, len(tasks), erroredTasks) + } + }) + } +} + type fakePlannerServer struct { tasks []*protos.ProtoTask completedTasks atomic.Int64 + erroredTasks atomic.Int64 shutdownCalled bool listenAddr string @@ -198,11 +284,18 @@ func (f *fakePlannerServer) BuilderLoop(srv protos.PlannerForBuilder_BuilderLoop if err := srv.Send(&protos.PlannerToBuilder{Task: task}); err != nil { return fmt.Errorf("failed to send task: %w", err) } - if _, err := srv.Recv(); err != nil { + + result, err := srv.Recv() + if err != nil { return fmt.Errorf("failed to receive task response: %w", err) } - time.Sleep(10 * time.Millisecond) // Simulate task processing time to add some latency. + f.completedTasks.Inc() + if result.Result.Error != "" { + f.erroredTasks.Inc() + } + + time.Sleep(10 * time.Millisecond) // Simulate task processing time to add some latency. } // No more tasks. Wait until shutdown. @@ -214,32 +307,36 @@ func (f *fakePlannerServer) CompletedTasks() int { return int(f.completedTasks.Load()) } +func (f *fakePlannerServer) ErroredTasks() int { + return int(f.erroredTasks.Load()) +} + func (f *fakePlannerServer) NotifyBuilderShutdown(_ context.Context, _ *protos.NotifyBuilderShutdownRequest) (*protos.NotifyBuilderShutdownResponse, error) { f.shutdownCalled = true return &protos.NotifyBuilderShutdownResponse{}, nil } type fakeLimits struct { + Limits + taskTimout time.Duration } -func (f fakeLimits) BloomBlockEncoding(_ string) string { - panic("implement me") -} - -func (f fakeLimits) BloomNGramLength(_ string) int { - panic("implement me") -} +var _ Limits = fakeLimits{} -func (f fakeLimits) BloomNGramSkip(_ string) int { - panic("implement me") +func (f fakeLimits) BloomBlockEncoding(_ string) string { + return "none" } func (f fakeLimits) BloomMaxBlockSize(_ string) int { - panic("implement me") + return 0 } func (f fakeLimits) BloomMaxBloomSize(_ string) int { - panic("implement me") + return 0 +} + +func (f fakeLimits) BuilderResponseTimeout(_ string) time.Duration { + return f.taskTimout } type fakeBloomStore struct { @@ -250,6 +347,26 @@ func (f fakeBloomStore) BloomMetrics() *v1.Metrics { return nil } +func (f fakeBloomStore) Client(_ model.Time) (bloomshipper.Client, error) { + return fakeBloomClient{}, nil +} + +func (f fakeBloomStore) Fetcher(_ model.Time) (*bloomshipper.Fetcher, error) { + return &bloomshipper.Fetcher{}, nil +} + +type fakeBloomClient struct { + bloomshipper.Client +} + +func (f fakeBloomClient) PutBlock(_ context.Context, _ bloomshipper.Block) error { + return nil +} + +func (f fakeBloomClient) PutMeta(_ context.Context, _ bloomshipper.Meta) error { + return nil +} + func parseDayTime(s string) config.DayTime { t, err := time.Parse("2006-01-02", s) if err != nil { diff --git a/pkg/bloombuild/builder/config.go b/pkg/bloombuild/builder/config.go index dcb44c55b5f3..0a6444fa24cc 100644 --- a/pkg/bloombuild/builder/config.go +++ b/pkg/bloombuild/builder/config.go @@ -3,6 +3,7 @@ package builder import ( "flag" "fmt" + "time" "github.com/grafana/dskit/backoff" "github.com/grafana/dskit/grpcclient" @@ -40,4 +41,5 @@ type Limits interface { BloomBlockEncoding(tenantID string) string BloomMaxBlockSize(tenantID string) int BloomMaxBloomSize(tenantID string) int + BuilderResponseTimeout(tenantID string) time.Duration } From 3abb3b116d157ce26ecbb02e6c74b562e31f732d Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 18 Nov 2024 11:44:25 -0500 Subject: [PATCH 02/53] fix(deps): update module github.com/hashicorp/golang-lru to v2 (#14979) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: Paul Rogers Co-authored-by: Paul Rogers <129207811+paul1r@users.noreply.github.com> --- clients/pkg/logentry/stages/timestamp.go | 10 +- go.mod | 4 +- pkg/distributor/distributor.go | 9 +- pkg/kafka/encoding.go | 8 +- .../hashicorp/golang-lru/.golangci.yml | 30 -- .../github.com/hashicorp/golang-lru/README.md | 25 -- vendor/github.com/hashicorp/golang-lru/arc.go | 256 ------------------ vendor/github.com/hashicorp/golang-lru/doc.go | 21 -- .../hashicorp/golang-lru/testing.go | 16 -- .../hashicorp/golang-lru/{ => v2}/.gitignore | 0 .../hashicorp/golang-lru/v2/.golangci.yml | 46 ++++ .../hashicorp/golang-lru/{ => v2}/2q.go | 103 +++++-- .../hashicorp/golang-lru/v2/README.md | 79 ++++++ .../github.com/hashicorp/golang-lru/v2/doc.go | 24 ++ .../hashicorp/golang-lru/{ => v2}/lru.go | 89 +++--- vendor/modules.txt | 2 +- 16 files changed, 293 insertions(+), 429 deletions(-) delete mode 100644 vendor/github.com/hashicorp/golang-lru/.golangci.yml delete mode 100644 vendor/github.com/hashicorp/golang-lru/README.md delete mode 100644 vendor/github.com/hashicorp/golang-lru/arc.go delete mode 100644 vendor/github.com/hashicorp/golang-lru/doc.go delete mode 100644 vendor/github.com/hashicorp/golang-lru/testing.go rename vendor/github.com/hashicorp/golang-lru/{ => v2}/.gitignore (100%) create mode 100644 vendor/github.com/hashicorp/golang-lru/v2/.golangci.yml rename vendor/github.com/hashicorp/golang-lru/{ => v2}/2q.go (64%) create mode 100644 vendor/github.com/hashicorp/golang-lru/v2/README.md create mode 100644 vendor/github.com/hashicorp/golang-lru/v2/doc.go rename vendor/github.com/hashicorp/golang-lru/{ => v2}/lru.go (72%) diff --git a/clients/pkg/logentry/stages/timestamp.go b/clients/pkg/logentry/stages/timestamp.go index fb1fb8a27c3b..3e2ce9d730a0 100644 --- a/clients/pkg/logentry/stages/timestamp.go +++ b/clients/pkg/logentry/stages/timestamp.go @@ -8,7 +8,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" - lru "github.com/hashicorp/golang-lru" + lru "github.com/hashicorp/golang-lru/v2" "github.com/mitchellh/mapstructure" "github.com/prometheus/common/model" @@ -114,9 +114,9 @@ func newTimestampStage(logger log.Logger, config interface{}) (Stage, error) { return nil, err } - var lastKnownTimestamps *lru.Cache + var lastKnownTimestamps *lru.Cache[string, time.Time] if *cfg.ActionOnFailure == TimestampActionOnFailureFudge { - lastKnownTimestamps, err = lru.New(maxLastKnownTimestampsCacheSize) + lastKnownTimestamps, err = lru.New[string, time.Time](maxLastKnownTimestampsCacheSize) if err != nil { return nil, err } @@ -138,7 +138,7 @@ type timestampStage struct { // Stores the last known timestamp for a given "stream id" (guessed, since at this stage // there's no reliable way to know it). - lastKnownTimestamps *lru.Cache + lastKnownTimestamps *lru.Cache[string, time.Time] } // Name implements Stage @@ -222,7 +222,7 @@ func (ts *timestampStage) processActionOnFailureFudge(labels model.LabelSet, t * } // Fudge the timestamp - *t = lastTimestamp.(time.Time).Add(1 * time.Nanosecond) + *t = lastTimestamp.Add(1 * time.Nanosecond) // Store the fudged timestamp, so that a subsequent fudged timestamp will be 1ns after it ts.lastKnownTimestamps.Add(labelsStr, *t) diff --git a/go.mod b/go.mod index bab9fd32ca61..526cd6f53d07 100644 --- a/go.mod +++ b/go.mod @@ -58,7 +58,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 github.com/grpc-ecosystem/grpc-opentracing v0.0.0-20180507213350-8e809c8a8645 github.com/hashicorp/consul/api v1.30.0 - github.com/hashicorp/golang-lru v0.6.0 + github.com/hashicorp/golang-lru/v2 v2.0.7 github.com/imdario/mergo v0.3.16 github.com/influxdata/telegraf v1.16.3 github.com/jmespath/go-jmespath v0.4.0 @@ -126,7 +126,6 @@ require ( github.com/gogo/googleapis v1.4.1 github.com/grafana/jsonparser v0.0.0-20241004153430-023329977675 github.com/grafana/loki/pkg/push v0.0.0-20240924133635-758364c7775f - github.com/hashicorp/golang-lru/v2 v2.0.7 github.com/heroku/x v0.4.0 github.com/influxdata/tdigest v0.0.2-0.20210216194612-fc98d27c9e8b github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db @@ -173,6 +172,7 @@ require ( github.com/go-ole/go-ole v1.2.6 // indirect github.com/goccy/go-json v0.10.3 // indirect github.com/gorilla/handlers v1.5.2 // indirect + github.com/hashicorp/golang-lru v0.6.0 // indirect github.com/lufia/plan9stats v0.0.0-20220913051719-115f729f3c8c // indirect github.com/moby/docker-image-spec v1.3.1 // indirect github.com/moby/sys/userns v0.1.0 // indirect diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index e4b92c8f585c..f776e6778b43 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -28,7 +28,7 @@ import ( "github.com/grafana/dskit/services" "github.com/grafana/dskit/tenant" "github.com/grafana/dskit/user" - lru "github.com/hashicorp/golang-lru" + lru "github.com/hashicorp/golang-lru/v2" "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -148,7 +148,7 @@ type Distributor struct { subservicesWatcher *services.FailureWatcher // Per-user rate limiter. ingestionRateLimiter *limiter.RateLimiter - labelCache *lru.Cache + labelCache *lru.Cache[string, labelData] // Push failures rate limiter. writeFailuresManager *writefailures.Manager @@ -217,7 +217,7 @@ func New( var servs []services.Service rateLimitStrat := validation.LocalIngestionRateStrategy - labelCache, err := lru.New(maxLabelCacheSize) + labelCache, err := lru.New[string, labelData](maxLabelCacheSize) if err != nil { return nil, err } @@ -1086,8 +1086,7 @@ type labelData struct { func (d *Distributor) parseStreamLabels(vContext validationContext, key string, stream logproto.Stream) (labels.Labels, string, uint64, error) { if val, ok := d.labelCache.Get(key); ok { - labelVal := val.(labelData) - return labelVal.ls, labelVal.ls.String(), labelVal.hash, nil + return val.ls, val.ls.String(), val.hash, nil } ls, err := syntax.ParseLabels(key) diff --git a/pkg/kafka/encoding.go b/pkg/kafka/encoding.go index 65daf59c25e7..15479336de67 100644 --- a/pkg/kafka/encoding.go +++ b/pkg/kafka/encoding.go @@ -9,7 +9,7 @@ import ( "github.com/twmb/franz-go/pkg/kgo" - lru "github.com/hashicorp/golang-lru" + lru "github.com/hashicorp/golang-lru/v2" "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/v3/pkg/logproto" @@ -126,11 +126,11 @@ func marshalWriteRequestToRecord(partitionID int32, tenantID string, stream logp // It caches parsed labels for efficiency. type Decoder struct { stream *logproto.Stream - cache *lru.Cache + cache *lru.Cache[string, labels.Labels] } func NewDecoder() (*Decoder, error) { - cache, err := lru.New(5000) // Set LRU size to 5000, adjust as needed + cache, err := lru.New[string, labels.Labels](5000) if err != nil { return nil, fmt.Errorf("failed to create LRU cache: %w", err) } @@ -154,7 +154,7 @@ func (d *Decoder) Decode(data []byte) (logproto.Stream, labels.Labels, error) { var ls labels.Labels if cachedLabels, ok := d.cache.Get(d.stream.Labels); ok { - ls = cachedLabels.(labels.Labels) + ls = cachedLabels } else { var err error ls, err = syntax.ParseLabels(d.stream.Labels) diff --git a/vendor/github.com/hashicorp/golang-lru/.golangci.yml b/vendor/github.com/hashicorp/golang-lru/.golangci.yml deleted file mode 100644 index 49202fc41e64..000000000000 --- a/vendor/github.com/hashicorp/golang-lru/.golangci.yml +++ /dev/null @@ -1,30 +0,0 @@ -linters: - enable: - - megacheck - - revive - - govet - - unconvert - - megacheck - - gas - - gocyclo - - dupl - - misspell - - unparam - - unused - - typecheck - - ineffassign - - stylecheck - - exportloopref - - gocritic - - nakedret - - gosimple - - prealloc - fast: false - disable-all: true - -issues: - exclude-rules: - - path: _test\.go - linters: - - dupl - exclude-use-default: false diff --git a/vendor/github.com/hashicorp/golang-lru/README.md b/vendor/github.com/hashicorp/golang-lru/README.md deleted file mode 100644 index 063bb16056ef..000000000000 --- a/vendor/github.com/hashicorp/golang-lru/README.md +++ /dev/null @@ -1,25 +0,0 @@ -golang-lru -========== - -This provides the `lru` package which implements a fixed-size -thread safe LRU cache. It is based on the cache in Groupcache. - -Documentation -============= - -Full docs are available on [Godoc](https://pkg.go.dev/github.com/hashicorp/golang-lru) - -Example -======= - -Using the LRU is very simple: - -```go -l, _ := New(128) -for i := 0; i < 256; i++ { - l.Add(i, nil) -} -if l.Len() != 128 { - panic(fmt.Sprintf("bad len: %v", l.Len())) -} -``` diff --git a/vendor/github.com/hashicorp/golang-lru/arc.go b/vendor/github.com/hashicorp/golang-lru/arc.go deleted file mode 100644 index e396f8428aa3..000000000000 --- a/vendor/github.com/hashicorp/golang-lru/arc.go +++ /dev/null @@ -1,256 +0,0 @@ -package lru - -import ( - "sync" - - "github.com/hashicorp/golang-lru/simplelru" -) - -// ARCCache is a thread-safe fixed size Adaptive Replacement Cache (ARC). -// ARC is an enhancement over the standard LRU cache in that tracks both -// frequency and recency of use. This avoids a burst in access to new -// entries from evicting the frequently used older entries. It adds some -// additional tracking overhead to a standard LRU cache, computationally -// it is roughly 2x the cost, and the extra memory overhead is linear -// with the size of the cache. ARC has been patented by IBM, but is -// similar to the TwoQueueCache (2Q) which requires setting parameters. -type ARCCache struct { - size int // Size is the total capacity of the cache - p int // P is the dynamic preference towards T1 or T2 - - t1 simplelru.LRUCache // T1 is the LRU for recently accessed items - b1 simplelru.LRUCache // B1 is the LRU for evictions from t1 - - t2 simplelru.LRUCache // T2 is the LRU for frequently accessed items - b2 simplelru.LRUCache // B2 is the LRU for evictions from t2 - - lock sync.RWMutex -} - -// NewARC creates an ARC of the given size -func NewARC(size int) (*ARCCache, error) { - // Create the sub LRUs - b1, err := simplelru.NewLRU(size, nil) - if err != nil { - return nil, err - } - b2, err := simplelru.NewLRU(size, nil) - if err != nil { - return nil, err - } - t1, err := simplelru.NewLRU(size, nil) - if err != nil { - return nil, err - } - t2, err := simplelru.NewLRU(size, nil) - if err != nil { - return nil, err - } - - // Initialize the ARC - c := &ARCCache{ - size: size, - p: 0, - t1: t1, - b1: b1, - t2: t2, - b2: b2, - } - return c, nil -} - -// Get looks up a key's value from the cache. -func (c *ARCCache) Get(key interface{}) (value interface{}, ok bool) { - c.lock.Lock() - defer c.lock.Unlock() - - // If the value is contained in T1 (recent), then - // promote it to T2 (frequent) - if val, ok := c.t1.Peek(key); ok { - c.t1.Remove(key) - c.t2.Add(key, val) - return val, ok - } - - // Check if the value is contained in T2 (frequent) - if val, ok := c.t2.Get(key); ok { - return val, ok - } - - // No hit - return nil, false -} - -// Add adds a value to the cache. -func (c *ARCCache) Add(key, value interface{}) { - c.lock.Lock() - defer c.lock.Unlock() - - // Check if the value is contained in T1 (recent), and potentially - // promote it to frequent T2 - if c.t1.Contains(key) { - c.t1.Remove(key) - c.t2.Add(key, value) - return - } - - // Check if the value is already in T2 (frequent) and update it - if c.t2.Contains(key) { - c.t2.Add(key, value) - return - } - - // Check if this value was recently evicted as part of the - // recently used list - if c.b1.Contains(key) { - // T1 set is too small, increase P appropriately - delta := 1 - b1Len := c.b1.Len() - b2Len := c.b2.Len() - if b2Len > b1Len { - delta = b2Len / b1Len - } - if c.p+delta >= c.size { - c.p = c.size - } else { - c.p += delta - } - - // Potentially need to make room in the cache - if c.t1.Len()+c.t2.Len() >= c.size { - c.replace(false) - } - - // Remove from B1 - c.b1.Remove(key) - - // Add the key to the frequently used list - c.t2.Add(key, value) - return - } - - // Check if this value was recently evicted as part of the - // frequently used list - if c.b2.Contains(key) { - // T2 set is too small, decrease P appropriately - delta := 1 - b1Len := c.b1.Len() - b2Len := c.b2.Len() - if b1Len > b2Len { - delta = b1Len / b2Len - } - if delta >= c.p { - c.p = 0 - } else { - c.p -= delta - } - - // Potentially need to make room in the cache - if c.t1.Len()+c.t2.Len() >= c.size { - c.replace(true) - } - - // Remove from B2 - c.b2.Remove(key) - - // Add the key to the frequently used list - c.t2.Add(key, value) - return - } - - // Potentially need to make room in the cache - if c.t1.Len()+c.t2.Len() >= c.size { - c.replace(false) - } - - // Keep the size of the ghost buffers trim - if c.b1.Len() > c.size-c.p { - c.b1.RemoveOldest() - } - if c.b2.Len() > c.p { - c.b2.RemoveOldest() - } - - // Add to the recently seen list - c.t1.Add(key, value) -} - -// replace is used to adaptively evict from either T1 or T2 -// based on the current learned value of P -func (c *ARCCache) replace(b2ContainsKey bool) { - t1Len := c.t1.Len() - if t1Len > 0 && (t1Len > c.p || (t1Len == c.p && b2ContainsKey)) { - k, _, ok := c.t1.RemoveOldest() - if ok { - c.b1.Add(k, nil) - } - } else { - k, _, ok := c.t2.RemoveOldest() - if ok { - c.b2.Add(k, nil) - } - } -} - -// Len returns the number of cached entries -func (c *ARCCache) Len() int { - c.lock.RLock() - defer c.lock.RUnlock() - return c.t1.Len() + c.t2.Len() -} - -// Keys returns all the cached keys -func (c *ARCCache) Keys() []interface{} { - c.lock.RLock() - defer c.lock.RUnlock() - k1 := c.t1.Keys() - k2 := c.t2.Keys() - return append(k1, k2...) -} - -// Remove is used to purge a key from the cache -func (c *ARCCache) Remove(key interface{}) { - c.lock.Lock() - defer c.lock.Unlock() - if c.t1.Remove(key) { - return - } - if c.t2.Remove(key) { - return - } - if c.b1.Remove(key) { - return - } - if c.b2.Remove(key) { - return - } -} - -// Purge is used to clear the cache -func (c *ARCCache) Purge() { - c.lock.Lock() - defer c.lock.Unlock() - c.t1.Purge() - c.t2.Purge() - c.b1.Purge() - c.b2.Purge() -} - -// Contains is used to check if the cache contains a key -// without updating recency or frequency. -func (c *ARCCache) Contains(key interface{}) bool { - c.lock.RLock() - defer c.lock.RUnlock() - return c.t1.Contains(key) || c.t2.Contains(key) -} - -// Peek is used to inspect the cache value of a key -// without updating recency or frequency. -func (c *ARCCache) Peek(key interface{}) (value interface{}, ok bool) { - c.lock.RLock() - defer c.lock.RUnlock() - if val, ok := c.t1.Peek(key); ok { - return val, ok - } - return c.t2.Peek(key) -} diff --git a/vendor/github.com/hashicorp/golang-lru/doc.go b/vendor/github.com/hashicorp/golang-lru/doc.go deleted file mode 100644 index 2547df979d0b..000000000000 --- a/vendor/github.com/hashicorp/golang-lru/doc.go +++ /dev/null @@ -1,21 +0,0 @@ -// Package lru provides three different LRU caches of varying sophistication. -// -// Cache is a simple LRU cache. It is based on the -// LRU implementation in groupcache: -// https://github.com/golang/groupcache/tree/master/lru -// -// TwoQueueCache tracks frequently used and recently used entries separately. -// This avoids a burst of accesses from taking out frequently used entries, -// at the cost of about 2x computational overhead and some extra bookkeeping. -// -// ARCCache is an adaptive replacement cache. It tracks recent evictions as -// well as recent usage in both the frequent and recent caches. Its -// computational overhead is comparable to TwoQueueCache, but the memory -// overhead is linear with the size of the cache. -// -// ARC has been patented by IBM, so do not use it if that is problematic for -// your program. -// -// All caches in this package take locks while operating, and are therefore -// thread-safe for consumers. -package lru diff --git a/vendor/github.com/hashicorp/golang-lru/testing.go b/vendor/github.com/hashicorp/golang-lru/testing.go deleted file mode 100644 index 492760782c5e..000000000000 --- a/vendor/github.com/hashicorp/golang-lru/testing.go +++ /dev/null @@ -1,16 +0,0 @@ -package lru - -import ( - "crypto/rand" - "math" - "math/big" - "testing" -) - -func getRand(tb testing.TB) int64 { - out, err := rand.Int(rand.Reader, big.NewInt(math.MaxInt64)) - if err != nil { - tb.Fatal(err) - } - return out.Int64() -} diff --git a/vendor/github.com/hashicorp/golang-lru/.gitignore b/vendor/github.com/hashicorp/golang-lru/v2/.gitignore similarity index 100% rename from vendor/github.com/hashicorp/golang-lru/.gitignore rename to vendor/github.com/hashicorp/golang-lru/v2/.gitignore diff --git a/vendor/github.com/hashicorp/golang-lru/v2/.golangci.yml b/vendor/github.com/hashicorp/golang-lru/v2/.golangci.yml new file mode 100644 index 000000000000..7e7b8a96275a --- /dev/null +++ b/vendor/github.com/hashicorp/golang-lru/v2/.golangci.yml @@ -0,0 +1,46 @@ +# Copyright (c) HashiCorp, Inc. +# SPDX-License-Identifier: MPL-2.0 + +linters: + fast: false + disable-all: true + enable: + - revive + - megacheck + - govet + - unconvert + - gas + - gocyclo + - dupl + - misspell + - unparam + - unused + - typecheck + - ineffassign + # - stylecheck + - exportloopref + - gocritic + - nakedret + - gosimple + - prealloc + +# golangci-lint configuration file +linters-settings: + revive: + ignore-generated-header: true + severity: warning + rules: + - name: package-comments + severity: warning + disabled: true + - name: exported + severity: warning + disabled: false + arguments: ["checkPrivateReceivers", "disableStutteringCheck"] + +issues: + exclude-use-default: false + exclude-rules: + - path: _test\.go + linters: + - dupl diff --git a/vendor/github.com/hashicorp/golang-lru/2q.go b/vendor/github.com/hashicorp/golang-lru/v2/2q.go similarity index 64% rename from vendor/github.com/hashicorp/golang-lru/2q.go rename to vendor/github.com/hashicorp/golang-lru/v2/2q.go index 15fcad0306e3..8c95252b6f27 100644 --- a/vendor/github.com/hashicorp/golang-lru/2q.go +++ b/vendor/github.com/hashicorp/golang-lru/v2/2q.go @@ -1,10 +1,13 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: MPL-2.0 + package lru import ( - "fmt" + "errors" "sync" - "github.com/hashicorp/golang-lru/simplelru" + "github.com/hashicorp/golang-lru/v2/simplelru" ) const ( @@ -26,33 +29,35 @@ const ( // computationally about 2x the cost, and adds some metadata over // head. The ARCCache is similar, but does not require setting any // parameters. -type TwoQueueCache struct { - size int - recentSize int +type TwoQueueCache[K comparable, V any] struct { + size int + recentSize int + recentRatio float64 + ghostRatio float64 - recent simplelru.LRUCache - frequent simplelru.LRUCache - recentEvict simplelru.LRUCache + recent simplelru.LRUCache[K, V] + frequent simplelru.LRUCache[K, V] + recentEvict simplelru.LRUCache[K, struct{}] lock sync.RWMutex } // New2Q creates a new TwoQueueCache using the default // values for the parameters. -func New2Q(size int) (*TwoQueueCache, error) { - return New2QParams(size, Default2QRecentRatio, Default2QGhostEntries) +func New2Q[K comparable, V any](size int) (*TwoQueueCache[K, V], error) { + return New2QParams[K, V](size, Default2QRecentRatio, Default2QGhostEntries) } // New2QParams creates a new TwoQueueCache using the provided // parameter values. -func New2QParams(size int, recentRatio, ghostRatio float64) (*TwoQueueCache, error) { +func New2QParams[K comparable, V any](size int, recentRatio, ghostRatio float64) (*TwoQueueCache[K, V], error) { if size <= 0 { - return nil, fmt.Errorf("invalid size") + return nil, errors.New("invalid size") } if recentRatio < 0.0 || recentRatio > 1.0 { - return nil, fmt.Errorf("invalid recent ratio") + return nil, errors.New("invalid recent ratio") } if ghostRatio < 0.0 || ghostRatio > 1.0 { - return nil, fmt.Errorf("invalid ghost ratio") + return nil, errors.New("invalid ghost ratio") } // Determine the sub-sizes @@ -60,23 +65,25 @@ func New2QParams(size int, recentRatio, ghostRatio float64) (*TwoQueueCache, err evictSize := int(float64(size) * ghostRatio) // Allocate the LRUs - recent, err := simplelru.NewLRU(size, nil) + recent, err := simplelru.NewLRU[K, V](size, nil) if err != nil { return nil, err } - frequent, err := simplelru.NewLRU(size, nil) + frequent, err := simplelru.NewLRU[K, V](size, nil) if err != nil { return nil, err } - recentEvict, err := simplelru.NewLRU(evictSize, nil) + recentEvict, err := simplelru.NewLRU[K, struct{}](evictSize, nil) if err != nil { return nil, err } // Initialize the cache - c := &TwoQueueCache{ + c := &TwoQueueCache[K, V]{ size: size, recentSize: recentSize, + recentRatio: recentRatio, + ghostRatio: ghostRatio, recent: recent, frequent: frequent, recentEvict: recentEvict, @@ -85,7 +92,7 @@ func New2QParams(size int, recentRatio, ghostRatio float64) (*TwoQueueCache, err } // Get looks up a key's value from the cache. -func (c *TwoQueueCache) Get(key interface{}) (value interface{}, ok bool) { +func (c *TwoQueueCache[K, V]) Get(key K) (value V, ok bool) { c.lock.Lock() defer c.lock.Unlock() @@ -103,11 +110,11 @@ func (c *TwoQueueCache) Get(key interface{}) (value interface{}, ok bool) { } // No hit - return nil, false + return } // Add adds a value to the cache. -func (c *TwoQueueCache) Add(key, value interface{}) { +func (c *TwoQueueCache[K, V]) Add(key K, value V) { c.lock.Lock() defer c.lock.Unlock() @@ -141,7 +148,7 @@ func (c *TwoQueueCache) Add(key, value interface{}) { } // ensureSpace is used to ensure we have space in the cache -func (c *TwoQueueCache) ensureSpace(recentEvict bool) { +func (c *TwoQueueCache[K, V]) ensureSpace(recentEvict bool) { // If we have space, nothing to do recentLen := c.recent.Len() freqLen := c.frequent.Len() @@ -153,7 +160,7 @@ func (c *TwoQueueCache) ensureSpace(recentEvict bool) { // the target, evict from there if recentLen > 0 && (recentLen > c.recentSize || (recentLen == c.recentSize && !recentEvict)) { k, _, _ := c.recent.RemoveOldest() - c.recentEvict.Add(k, nil) + c.recentEvict.Add(k, struct{}{}) return } @@ -162,15 +169,43 @@ func (c *TwoQueueCache) ensureSpace(recentEvict bool) { } // Len returns the number of items in the cache. -func (c *TwoQueueCache) Len() int { +func (c *TwoQueueCache[K, V]) Len() int { c.lock.RLock() defer c.lock.RUnlock() return c.recent.Len() + c.frequent.Len() } +// Resize changes the cache size. +func (c *TwoQueueCache[K, V]) Resize(size int) (evicted int) { + c.lock.Lock() + defer c.lock.Unlock() + + // Recalculate the sub-sizes + recentSize := int(float64(size) * c.recentRatio) + evictSize := int(float64(size) * c.ghostRatio) + c.size = size + c.recentSize = recentSize + + // ensureSpace + diff := c.recent.Len() + c.frequent.Len() - size + if diff < 0 { + diff = 0 + } + for i := 0; i < diff; i++ { + c.ensureSpace(true) + } + + // Reallocate the LRUs + c.recent.Resize(size) + c.frequent.Resize(size) + c.recentEvict.Resize(evictSize) + + return diff +} + // Keys returns a slice of the keys in the cache. // The frequently used keys are first in the returned slice. -func (c *TwoQueueCache) Keys() []interface{} { +func (c *TwoQueueCache[K, V]) Keys() []K { c.lock.RLock() defer c.lock.RUnlock() k1 := c.frequent.Keys() @@ -178,8 +213,18 @@ func (c *TwoQueueCache) Keys() []interface{} { return append(k1, k2...) } +// Values returns a slice of the values in the cache. +// The frequently used values are first in the returned slice. +func (c *TwoQueueCache[K, V]) Values() []V { + c.lock.RLock() + defer c.lock.RUnlock() + v1 := c.frequent.Values() + v2 := c.recent.Values() + return append(v1, v2...) +} + // Remove removes the provided key from the cache. -func (c *TwoQueueCache) Remove(key interface{}) { +func (c *TwoQueueCache[K, V]) Remove(key K) { c.lock.Lock() defer c.lock.Unlock() if c.frequent.Remove(key) { @@ -194,7 +239,7 @@ func (c *TwoQueueCache) Remove(key interface{}) { } // Purge is used to completely clear the cache. -func (c *TwoQueueCache) Purge() { +func (c *TwoQueueCache[K, V]) Purge() { c.lock.Lock() defer c.lock.Unlock() c.recent.Purge() @@ -204,7 +249,7 @@ func (c *TwoQueueCache) Purge() { // Contains is used to check if the cache contains a key // without updating recency or frequency. -func (c *TwoQueueCache) Contains(key interface{}) bool { +func (c *TwoQueueCache[K, V]) Contains(key K) bool { c.lock.RLock() defer c.lock.RUnlock() return c.frequent.Contains(key) || c.recent.Contains(key) @@ -212,7 +257,7 @@ func (c *TwoQueueCache) Contains(key interface{}) bool { // Peek is used to inspect the cache value of a key // without updating recency or frequency. -func (c *TwoQueueCache) Peek(key interface{}) (value interface{}, ok bool) { +func (c *TwoQueueCache[K, V]) Peek(key K) (value V, ok bool) { c.lock.RLock() defer c.lock.RUnlock() if val, ok := c.frequent.Peek(key); ok { diff --git a/vendor/github.com/hashicorp/golang-lru/v2/README.md b/vendor/github.com/hashicorp/golang-lru/v2/README.md new file mode 100644 index 000000000000..a942eb539700 --- /dev/null +++ b/vendor/github.com/hashicorp/golang-lru/v2/README.md @@ -0,0 +1,79 @@ +golang-lru +========== + +This provides the `lru` package which implements a fixed-size +thread safe LRU cache. It is based on the cache in Groupcache. + +Documentation +============= + +Full docs are available on [Go Packages](https://pkg.go.dev/github.com/hashicorp/golang-lru/v2) + +LRU cache example +================= + +```go +package main + +import ( + "fmt" + "github.com/hashicorp/golang-lru/v2" +) + +func main() { + l, _ := lru.New[int, any](128) + for i := 0; i < 256; i++ { + l.Add(i, nil) + } + if l.Len() != 128 { + panic(fmt.Sprintf("bad len: %v", l.Len())) + } +} +``` + +Expirable LRU cache example +=========================== + +```go +package main + +import ( + "fmt" + "time" + + "github.com/hashicorp/golang-lru/v2/expirable" +) + +func main() { + // make cache with 10ms TTL and 5 max keys + cache := expirable.NewLRU[string, string](5, nil, time.Millisecond*10) + + + // set value under key1. + cache.Add("key1", "val1") + + // get value under key1 + r, ok := cache.Get("key1") + + // check for OK value + if ok { + fmt.Printf("value before expiration is found: %v, value: %q\n", ok, r) + } + + // wait for cache to expire + time.Sleep(time.Millisecond * 12) + + // get value under key1 after key expiration + r, ok = cache.Get("key1") + fmt.Printf("value after expiration is found: %v, value: %q\n", ok, r) + + // set value under key2, would evict old entry because it is already expired. + cache.Add("key2", "val2") + + fmt.Printf("Cache len: %d\n", cache.Len()) + // Output: + // value before expiration is found: true, value: "val1" + // value after expiration is found: false, value: "" + // Cache len: 1 +} +``` diff --git a/vendor/github.com/hashicorp/golang-lru/v2/doc.go b/vendor/github.com/hashicorp/golang-lru/v2/doc.go new file mode 100644 index 000000000000..24107ee0edee --- /dev/null +++ b/vendor/github.com/hashicorp/golang-lru/v2/doc.go @@ -0,0 +1,24 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: MPL-2.0 + +// Package lru provides three different LRU caches of varying sophistication. +// +// Cache is a simple LRU cache. It is based on the LRU implementation in +// groupcache: https://github.com/golang/groupcache/tree/master/lru +// +// TwoQueueCache tracks frequently used and recently used entries separately. +// This avoids a burst of accesses from taking out frequently used entries, at +// the cost of about 2x computational overhead and some extra bookkeeping. +// +// ARCCache is an adaptive replacement cache. It tracks recent evictions as well +// as recent usage in both the frequent and recent caches. Its computational +// overhead is comparable to TwoQueueCache, but the memory overhead is linear +// with the size of the cache. +// +// ARC has been patented by IBM, so do not use it if that is problematic for +// your program. For this reason, it is in a separate go module contained within +// this repository. +// +// All caches in this package take locks while operating, and are therefore +// thread-safe for consumers. +package lru diff --git a/vendor/github.com/hashicorp/golang-lru/lru.go b/vendor/github.com/hashicorp/golang-lru/v2/lru.go similarity index 72% rename from vendor/github.com/hashicorp/golang-lru/lru.go rename to vendor/github.com/hashicorp/golang-lru/v2/lru.go index 895d8e3ea0c1..a2655f1f3109 100644 --- a/vendor/github.com/hashicorp/golang-lru/lru.go +++ b/vendor/github.com/hashicorp/golang-lru/v2/lru.go @@ -1,9 +1,12 @@ +// Copyright (c) HashiCorp, Inc. +// SPDX-License-Identifier: MPL-2.0 + package lru import ( "sync" - "github.com/hashicorp/golang-lru/simplelru" + "github.com/hashicorp/golang-lru/v2/simplelru" ) const ( @@ -12,23 +15,24 @@ const ( ) // Cache is a thread-safe fixed size LRU cache. -type Cache struct { - lru *simplelru.LRU - evictedKeys, evictedVals []interface{} - onEvictedCB func(k, v interface{}) - lock sync.RWMutex +type Cache[K comparable, V any] struct { + lru *simplelru.LRU[K, V] + evictedKeys []K + evictedVals []V + onEvictedCB func(k K, v V) + lock sync.RWMutex } // New creates an LRU of the given size. -func New(size int) (*Cache, error) { - return NewWithEvict(size, nil) +func New[K comparable, V any](size int) (*Cache[K, V], error) { + return NewWithEvict[K, V](size, nil) } // NewWithEvict constructs a fixed size cache with the given eviction // callback. -func NewWithEvict(size int, onEvicted func(key, value interface{})) (c *Cache, err error) { +func NewWithEvict[K comparable, V any](size int, onEvicted func(key K, value V)) (c *Cache[K, V], err error) { // create a cache with default settings - c = &Cache{ + c = &Cache[K, V]{ onEvictedCB: onEvicted, } if onEvicted != nil { @@ -39,21 +43,22 @@ func NewWithEvict(size int, onEvicted func(key, value interface{})) (c *Cache, e return } -func (c *Cache) initEvictBuffers() { - c.evictedKeys = make([]interface{}, 0, DefaultEvictedBufferSize) - c.evictedVals = make([]interface{}, 0, DefaultEvictedBufferSize) +func (c *Cache[K, V]) initEvictBuffers() { + c.evictedKeys = make([]K, 0, DefaultEvictedBufferSize) + c.evictedVals = make([]V, 0, DefaultEvictedBufferSize) } // onEvicted save evicted key/val and sent in externally registered callback // outside of critical section -func (c *Cache) onEvicted(k, v interface{}) { +func (c *Cache[K, V]) onEvicted(k K, v V) { c.evictedKeys = append(c.evictedKeys, k) c.evictedVals = append(c.evictedVals, v) } // Purge is used to completely clear the cache. -func (c *Cache) Purge() { - var ks, vs []interface{} +func (c *Cache[K, V]) Purge() { + var ks []K + var vs []V c.lock.Lock() c.lru.Purge() if c.onEvictedCB != nil && len(c.evictedKeys) > 0 { @@ -70,8 +75,9 @@ func (c *Cache) Purge() { } // Add adds a value to the cache. Returns true if an eviction occurred. -func (c *Cache) Add(key, value interface{}) (evicted bool) { - var k, v interface{} +func (c *Cache[K, V]) Add(key K, value V) (evicted bool) { + var k K + var v V c.lock.Lock() evicted = c.lru.Add(key, value) if c.onEvictedCB != nil && evicted { @@ -86,7 +92,7 @@ func (c *Cache) Add(key, value interface{}) (evicted bool) { } // Get looks up a key's value from the cache. -func (c *Cache) Get(key interface{}) (value interface{}, ok bool) { +func (c *Cache[K, V]) Get(key K) (value V, ok bool) { c.lock.Lock() value, ok = c.lru.Get(key) c.lock.Unlock() @@ -95,7 +101,7 @@ func (c *Cache) Get(key interface{}) (value interface{}, ok bool) { // Contains checks if a key is in the cache, without updating the // recent-ness or deleting it for being stale. -func (c *Cache) Contains(key interface{}) bool { +func (c *Cache[K, V]) Contains(key K) bool { c.lock.RLock() containKey := c.lru.Contains(key) c.lock.RUnlock() @@ -104,7 +110,7 @@ func (c *Cache) Contains(key interface{}) bool { // Peek returns the key value (or undefined if not found) without updating // the "recently used"-ness of the key. -func (c *Cache) Peek(key interface{}) (value interface{}, ok bool) { +func (c *Cache[K, V]) Peek(key K) (value V, ok bool) { c.lock.RLock() value, ok = c.lru.Peek(key) c.lock.RUnlock() @@ -114,8 +120,9 @@ func (c *Cache) Peek(key interface{}) (value interface{}, ok bool) { // ContainsOrAdd checks if a key is in the cache without updating the // recent-ness or deleting it for being stale, and if not, adds the value. // Returns whether found and whether an eviction occurred. -func (c *Cache) ContainsOrAdd(key, value interface{}) (ok, evicted bool) { - var k, v interface{} +func (c *Cache[K, V]) ContainsOrAdd(key K, value V) (ok, evicted bool) { + var k K + var v V c.lock.Lock() if c.lru.Contains(key) { c.lock.Unlock() @@ -136,8 +143,9 @@ func (c *Cache) ContainsOrAdd(key, value interface{}) (ok, evicted bool) { // PeekOrAdd checks if a key is in the cache without updating the // recent-ness or deleting it for being stale, and if not, adds the value. // Returns whether found and whether an eviction occurred. -func (c *Cache) PeekOrAdd(key, value interface{}) (previous interface{}, ok, evicted bool) { - var k, v interface{} +func (c *Cache[K, V]) PeekOrAdd(key K, value V) (previous V, ok, evicted bool) { + var k K + var v V c.lock.Lock() previous, ok = c.lru.Peek(key) if ok { @@ -153,12 +161,13 @@ func (c *Cache) PeekOrAdd(key, value interface{}) (previous interface{}, ok, evi if c.onEvictedCB != nil && evicted { c.onEvictedCB(k, v) } - return nil, false, evicted + return } // Remove removes the provided key from the cache. -func (c *Cache) Remove(key interface{}) (present bool) { - var k, v interface{} +func (c *Cache[K, V]) Remove(key K) (present bool) { + var k K + var v V c.lock.Lock() present = c.lru.Remove(key) if c.onEvictedCB != nil && present { @@ -173,8 +182,9 @@ func (c *Cache) Remove(key interface{}) (present bool) { } // Resize changes the cache size. -func (c *Cache) Resize(size int) (evicted int) { - var ks, vs []interface{} +func (c *Cache[K, V]) Resize(size int) (evicted int) { + var ks []K + var vs []V c.lock.Lock() evicted = c.lru.Resize(size) if c.onEvictedCB != nil && evicted > 0 { @@ -191,8 +201,9 @@ func (c *Cache) Resize(size int) (evicted int) { } // RemoveOldest removes the oldest item from the cache. -func (c *Cache) RemoveOldest() (key, value interface{}, ok bool) { - var k, v interface{} +func (c *Cache[K, V]) RemoveOldest() (key K, value V, ok bool) { + var k K + var v V c.lock.Lock() key, value, ok = c.lru.RemoveOldest() if c.onEvictedCB != nil && ok { @@ -207,7 +218,7 @@ func (c *Cache) RemoveOldest() (key, value interface{}, ok bool) { } // GetOldest returns the oldest entry -func (c *Cache) GetOldest() (key, value interface{}, ok bool) { +func (c *Cache[K, V]) GetOldest() (key K, value V, ok bool) { c.lock.RLock() key, value, ok = c.lru.GetOldest() c.lock.RUnlock() @@ -215,15 +226,23 @@ func (c *Cache) GetOldest() (key, value interface{}, ok bool) { } // Keys returns a slice of the keys in the cache, from oldest to newest. -func (c *Cache) Keys() []interface{} { +func (c *Cache[K, V]) Keys() []K { c.lock.RLock() keys := c.lru.Keys() c.lock.RUnlock() return keys } +// Values returns a slice of the values in the cache, from oldest to newest. +func (c *Cache[K, V]) Values() []V { + c.lock.RLock() + values := c.lru.Values() + c.lock.RUnlock() + return values +} + // Len returns the number of items in the cache. -func (c *Cache) Len() int { +func (c *Cache[K, V]) Len() int { c.lock.RLock() length := c.lru.Len() c.lock.RUnlock() diff --git a/vendor/modules.txt b/vendor/modules.txt index 01b94fbc4c17..cb3dc0fa401b 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1119,10 +1119,10 @@ github.com/hashicorp/go-sockaddr github.com/hashicorp/go-uuid # github.com/hashicorp/golang-lru v0.6.0 ## explicit; go 1.12 -github.com/hashicorp/golang-lru github.com/hashicorp/golang-lru/simplelru # github.com/hashicorp/golang-lru/v2 v2.0.7 ## explicit; go 1.18 +github.com/hashicorp/golang-lru/v2 github.com/hashicorp/golang-lru/v2/internal github.com/hashicorp/golang-lru/v2/simplelru # github.com/hashicorp/memberlist v0.5.0 => github.com/grafana/memberlist v0.3.1-0.20220714140823-09ffed8adbbe From d165161eddd8eeb0ef0eefb1b97296390b3c54f9 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 18 Nov 2024 11:10:34 -0700 Subject: [PATCH 03/53] chore(deps): update actions/setup-go action to v5 (#14971) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- .github/workflows/operator-bundle.yaml | 2 +- .github/workflows/operator-scorecard.yaml | 2 +- .github/workflows/operator.yaml | 10 +++++----- .github/workflows/promtail-windows-test.yml | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/operator-bundle.yaml b/.github/workflows/operator-bundle.yaml index 39630e45b8f8..e99fc0865612 100644 --- a/.github/workflows/operator-bundle.yaml +++ b/.github/workflows/operator-bundle.yaml @@ -19,7 +19,7 @@ jobs: go: ['1.22'] steps: - name: Set up Go 1.x - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go }} id: go diff --git a/.github/workflows/operator-scorecard.yaml b/.github/workflows/operator-scorecard.yaml index 1a067a0ea140..4fd8fe5852d3 100644 --- a/.github/workflows/operator-scorecard.yaml +++ b/.github/workflows/operator-scorecard.yaml @@ -19,7 +19,7 @@ jobs: go: ['1.22'] steps: - name: Set up Go 1.x - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go }} id: go diff --git a/.github/workflows/operator.yaml b/.github/workflows/operator.yaml index 639746aeb5ac..e751dd0fd7f6 100644 --- a/.github/workflows/operator.yaml +++ b/.github/workflows/operator.yaml @@ -21,7 +21,7 @@ jobs: - name: Install make run: sudo apt-get install make - name: Set up Go 1.x - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go }} id: go @@ -43,7 +43,7 @@ jobs: - name: Install make run: sudo apt-get install make - name: Set up Go 1.x - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go }} id: go @@ -69,7 +69,7 @@ jobs: - name: Install make run: sudo apt-get install make - name: Set up Go 1.x - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go }} id: go @@ -90,7 +90,7 @@ jobs: - name: Install make run: sudo apt-get install make - name: Set up Go 1.x - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go }} id: go @@ -111,7 +111,7 @@ jobs: - name: Install make run: sudo apt-get install make - name: Set up Go 1.x - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go }} id: go diff --git a/.github/workflows/promtail-windows-test.yml b/.github/workflows/promtail-windows-test.yml index 90ccf72ad1be..0aa131d769ab 100644 --- a/.github/workflows/promtail-windows-test.yml +++ b/.github/workflows/promtail-windows-test.yml @@ -14,7 +14,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Setup Go ${{ matrix.go-version }} - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go-version }} # You can test your matrix by printing the current Go version From 76bc184be77659d3481395f953bf50a7c8abbb64 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 18 Nov 2024 11:11:52 -0700 Subject: [PATCH 04/53] chore(deps): update logstash docker tag to v8 (#14975) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- clients/cmd/logstash/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/cmd/logstash/Dockerfile b/clients/cmd/logstash/Dockerfile index f9845e87bd17..5f7662f6ae49 100644 --- a/clients/cmd/logstash/Dockerfile +++ b/clients/cmd/logstash/Dockerfile @@ -1,4 +1,4 @@ -FROM logstash:7.17.25 +FROM logstash:8.16.0 USER logstash ENV PATH /usr/share/logstash/vendor/jruby/bin:/usr/share/logstash/vendor/bundle/jruby/2.5.0/bin:/usr/share/logstash/jdk/bin:$PATH From 2d407bd308064bf014cd5f9718119b8fc625fea6 Mon Sep 17 00:00:00 2001 From: Matt Veitas Date: Mon, 18 Nov 2024 13:21:16 -0500 Subject: [PATCH 05/53] feat: Add otlp attribute deployment.environment.name to list of default labels (#14747) --- docs/sources/send-data/otel/_index.md | 1 + docs/sources/shared/configuration.md | 2 +- pkg/loghttp/push/otlp_config.go | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/sources/send-data/otel/_index.md b/docs/sources/send-data/otel/_index.md index 176cd0afd410..ce0b1de64483 100644 --- a/docs/sources/send-data/otel/_index.md +++ b/docs/sources/send-data/otel/_index.md @@ -79,6 +79,7 @@ Since the OpenTelemetry protocol differs from the Loki storage model, here is ho - cloud.region - container.name - deployment.environment + - deployment.environment.name - k8s.cluster.name - k8s.container.name - k8s.cronjob.name diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index ec2fc098afba..203d7130232a 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -2264,7 +2264,7 @@ write_failures_logging: otlp_config: # List of default otlp resource attributes to be picked as index labels # CLI flag: -distributor.otlp.default_resource_attributes_as_index_labels - [default_resource_attributes_as_index_labels: | default = [service.name service.namespace service.instance.id deployment.environment cloud.region cloud.availability_zone k8s.cluster.name k8s.namespace.name k8s.pod.name k8s.container.name container.name k8s.replicaset.name k8s.deployment.name k8s.statefulset.name k8s.daemonset.name k8s.cronjob.name k8s.job.name]] + [default_resource_attributes_as_index_labels: | default = [service.name service.namespace service.instance.id deployment.environment deployment.environment.name cloud.region cloud.availability_zone k8s.cluster.name k8s.namespace.name k8s.pod.name k8s.container.name container.name k8s.replicaset.name k8s.deployment.name k8s.statefulset.name k8s.daemonset.name k8s.cronjob.name k8s.job.name]] # Enable writes to Kafka during Push requests. # CLI flag: -distributor.kafka-writes-enabled diff --git a/pkg/loghttp/push/otlp_config.go b/pkg/loghttp/push/otlp_config.go index f71efe8bee7d..4739e1f10dd3 100644 --- a/pkg/loghttp/push/otlp_config.go +++ b/pkg/loghttp/push/otlp_config.go @@ -56,6 +56,7 @@ func (cfg *GlobalOTLPConfig) RegisterFlags(fs *flag.FlagSet) { "service.namespace", "service.instance.id", "deployment.environment", + "deployment.environment.name", "cloud.region", "cloud.availability_zone", "k8s.cluster.name", From 321976e20f351da18044bd08c4ac5ec360ec24ad Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 18 Nov 2024 11:22:13 -0700 Subject: [PATCH 06/53] chore(deps): update docker/build-push-action action to v6 (#14981) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- .github/workflows/operator-images.yaml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/operator-images.yaml b/.github/workflows/operator-images.yaml index 768b6acf0010..a2c58a448d9a 100644 --- a/.github/workflows/operator-images.yaml +++ b/.github/workflows/operator-images.yaml @@ -48,7 +48,7 @@ jobs: echo "IMAGE_TAGS=$IMAGE_TAGS" >> $GITHUB_OUTPUT - name: Build and publish image on quay.io - uses: docker/build-push-action@v5 + uses: docker/build-push-action@v6 with: context: ./operator push: true @@ -86,7 +86,7 @@ jobs: echo "IMAGE_TAGS=$IMAGE_TAGS" >> $GITHUB_OUTPUT - name: Build and publish image on quay.io - uses: docker/build-push-action@v5 + uses: docker/build-push-action@v6 with: context: ./operator/bundle/openshift file: ./operator/bundle/openshift/bundle.Dockerfile @@ -125,7 +125,7 @@ jobs: echo "IMAGE_TAGS=$IMAGE_TAGS" >> $GITHUB_OUTPUT - name: Build and publish image on quay.io - uses: docker/build-push-action@v5 + uses: docker/build-push-action@v6 with: context: ./operator file: ./operator/calculator.Dockerfile From 556195d85b0740b670d712e373d103ff7a3034f3 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 18 Nov 2024 15:59:39 -0500 Subject: [PATCH 07/53] chore(deps): update terraform google to v6.12.0 (#14992) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- tools/gcplog/main.tf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/gcplog/main.tf b/tools/gcplog/main.tf index c4725c4ff0d0..71ccf30b9af6 100644 --- a/tools/gcplog/main.tf +++ b/tools/gcplog/main.tf @@ -2,7 +2,7 @@ terraform { required_providers { google = { source = "hashicorp/google" - version = "6.11.2" + version = "6.12.0" } } } From 1b93086a2082ec18a8f766dec7778b04e991ba10 Mon Sep 17 00:00:00 2001 From: mericks Date: Mon, 18 Nov 2024 13:27:51 -0800 Subject: [PATCH 08/53] feat: expose topologySpreadConstraints for admin-api pods (#14995) --- docs/sources/setup/install/helm/reference.md | 12 +++++++++++- .../templates/admin-api/deployment-admin-api.yaml | 8 +++++++- production/helm/loki/values.yaml | 2 ++ 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/docs/sources/setup/install/helm/reference.md b/docs/sources/setup/install/helm/reference.md index 028843f283dc..23113b35a662 100644 --- a/docs/sources/setup/install/helm/reference.md +++ b/docs/sources/setup/install/helm/reference.md @@ -82,7 +82,8 @@ This is the generated reference for the Loki Helm Chart values. "type": "RollingUpdate" }, "terminationGracePeriodSeconds": 60, - "tolerations": [] + "tolerations": [], + "topologySpreadConstraints": [] } @@ -280,6 +281,15 @@ This is the generated reference for the Loki Helm Chart values.
 []
 
+ + + + adminApi.topologySpreadConstraints + list + Topology Spread Constraints for admin-api pods +
+[]
+
diff --git a/production/helm/loki/templates/admin-api/deployment-admin-api.yaml b/production/helm/loki/templates/admin-api/deployment-admin-api.yaml index f85bbf90014b..2d356882204b 100644 --- a/production/helm/loki/templates/admin-api/deployment-admin-api.yaml +++ b/production/helm/loki/templates/admin-api/deployment-admin-api.yaml @@ -38,6 +38,12 @@ spec: {{- toYaml . | nindent 8 }} {{- end }} spec: + {{- if semverCompare ">=1.19-0" .Capabilities.KubeVersion.Version }} + {{- with .Values.adminApi.topologySpreadConstraints }} + topologySpreadConstraints: + {{- toYaml . | nindent 8 }} + {{- end }} + {{- end }} serviceAccountName: {{ template "loki.serviceAccountName" . }} {{- if .Values.adminApi.priorityClassName }} priorityClassName: {{ .Values.adminApi.priorityClassName }} @@ -167,4 +173,4 @@ spec: path: CAs/public.crt {{- end }} {{- end }} -{{- end }} \ No newline at end of file +{{- end }} diff --git a/production/helm/loki/values.yaml b/production/helm/loki/values.yaml index 7dbde9d9fed1..9ab4de54ba20 100644 --- a/production/helm/loki/values.yaml +++ b/production/helm/loki/values.yaml @@ -876,6 +876,8 @@ adminApi: affinity: {} # -- Node selector for admin-api Pods nodeSelector: {} + # -- Topology Spread Constraints for admin-api pods + topologySpreadConstraints: [] # -- Tolerations for admin-api Pods tolerations: [] # -- Grace period to allow the admin-api to shutdown before it is killed From 33a214880bd950a10db85c0418538261014e7fa1 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 18 Nov 2024 16:45:27 -0500 Subject: [PATCH 09/53] fix(deps): update module github.com/go-redis/redis/v8 to v9 (#14978) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: Paul Rogers --- go.mod | 2 +- go.sum | 8 +- pkg/storage/chunk/cache/redis_cache_test.go | 2 +- pkg/storage/chunk/cache/redis_client.go | 20 +- pkg/storage/chunk/cache/redis_client_test.go | 2 +- .../github.com/go-redis/redis/v8/.gitignore | 3 - .../github.com/go-redis/redis/v8/CHANGELOG.md | 177 - vendor/github.com/go-redis/redis/v8/Makefile | 35 - vendor/github.com/go-redis/redis/v8/README.md | 175 - .../github.com/go-redis/redis/v8/command.go | 3478 ----------- .../github.com/go-redis/redis/v8/commands.go | 3475 ----------- .../redis/v8/internal/proto/reader.go | 332 - .../go-redis/redis/v8/internal/safe.go | 12 - .../go-redis/redis/v8/internal/unsafe.go | 21 - .../go-redis/redis/v8/internal/util.go | 46 - .../github.com/go-redis/redis/v8/package.json | 8 - vendor/github.com/go-redis/redis/v8/redis.go | 773 --- .../github.com/redis/go-redis/v9/.gitignore | 6 + .../v8 => redis/go-redis/v9}/.golangci.yml | 0 .../v8 => redis/go-redis/v9}/.prettierrc.yml | 0 .../github.com/redis/go-redis/v9/CHANGELOG.md | 133 + .../redis/go-redis/v9/CONTRIBUTING.md | 101 + .../redis/v8 => redis/go-redis/v9}/LICENSE | 2 +- vendor/github.com/redis/go-redis/v9/Makefile | 50 + vendor/github.com/redis/go-redis/v9/README.md | 275 + .../v8 => redis/go-redis/v9}/RELEASING.md | 0 .../redis/go-redis/v9/acl_commands.go | 35 + .../redis/go-redis/v9/bitmap_commands.go | 161 + .../redis/go-redis/v9/cluster_commands.go | 192 + .../github.com/redis/go-redis/v9/command.go | 5557 +++++++++++++++++ .../github.com/redis/go-redis/v9/commands.go | 719 +++ .../redis/v8 => redis/go-redis/v9}/doc.go | 0 .../redis/v8 => redis/go-redis/v9}/error.go | 21 +- .../redis/go-redis/v9/gears_commands.go | 149 + .../redis/go-redis/v9/generic_commands.go | 384 ++ .../redis/go-redis/v9/geo_commands.go | 155 + .../redis/go-redis/v9/hash_commands.go | 450 ++ .../redis/go-redis/v9/hyperloglog_commands.go | 42 + .../v8 => redis/go-redis/v9}/internal/arg.go | 4 +- .../go-redis/v9}/internal/hashtag/hashtag.go | 2 +- .../go-redis/v9}/internal/hscan/hscan.go | 6 + .../go-redis/v9}/internal/hscan/structmap.go | 36 +- .../go-redis/v9}/internal/internal.go | 2 +- .../v8 => redis/go-redis/v9}/internal/log.go | 0 .../v8 => redis/go-redis/v9}/internal/once.go | 7 +- .../go-redis/v9}/internal/pool/conn.go | 18 +- .../go-redis/v9/internal/pool/conn_check.go | 49 + .../v9/internal/pool/conn_check_dummy.go | 9 + .../go-redis/v9}/internal/pool/pool.go | 227 +- .../go-redis/v9}/internal/pool/pool_single.go | 0 .../go-redis/v9}/internal/pool/pool_sticky.go | 0 .../go-redis/v9/internal/proto/reader.go | 552 ++ .../go-redis/v9}/internal/proto/scan.go | 7 +- .../go-redis/v9}/internal/proto/writer.go | 42 +- .../go-redis/v9}/internal/rand/rand.go | 0 .../redis/go-redis/v9/internal/util.go | 128 + .../go-redis/v9}/internal/util/safe.go | 1 - .../go-redis/v9}/internal/util/strconv.go | 0 .../redis/go-redis/v9/internal/util/type.go | 5 + .../go-redis/v9}/internal/util/unsafe.go | 1 - .../v8 => redis/go-redis/v9}/iterator.go | 13 +- vendor/github.com/redis/go-redis/v9/json.go | 599 ++ .../redis/go-redis/v9/list_commands.go | 289 + .../redis/v8 => redis/go-redis/v9}/options.go | 282 +- .../go-redis/v9/osscluster.go} | 631 +- .../go-redis/v9/osscluster_commands.go} | 12 +- .../v8 => redis/go-redis/v9}/pipeline.go | 74 +- .../redis/go-redis/v9/probabilistic.go | 1429 +++++ .../redis/v8 => redis/go-redis/v9}/pubsub.go | 97 +- .../redis/go-redis/v9/pubsub_commands.go | 76 + vendor/github.com/redis/go-redis/v9/redis.go | 874 +++ .../redis/v8 => redis/go-redis/v9}/result.go | 22 +- .../redis/v8 => redis/go-redis/v9}/ring.go | 483 +- .../redis/v8 => redis/go-redis/v9}/script.go | 23 +- .../redis/go-redis/v9/scripting_commands.go | 215 + .../redis/go-redis/v9/search_commands.go | 2240 +++++++ .../v8 => redis/go-redis/v9}/sentinel.go | 371 +- .../redis/go-redis/v9/set_commands.go | 217 + .../redis/go-redis/v9/sortedset_commands.go | 772 +++ .../redis/go-redis/v9/stream_commands.go | 450 ++ .../redis/go-redis/v9/string_commands.go | 303 + .../redis/go-redis/v9/timeseries_commands.go | 950 +++ .../redis/v8 => redis/go-redis/v9}/tx.go | 56 +- .../v8 => redis/go-redis/v9}/universal.go | 127 +- .../redis/v8 => redis/go-redis/v9}/version.go | 2 +- vendor/modules.txt | 20 +- 86 files changed, 19169 insertions(+), 9555 deletions(-) delete mode 100644 vendor/github.com/go-redis/redis/v8/.gitignore delete mode 100644 vendor/github.com/go-redis/redis/v8/CHANGELOG.md delete mode 100644 vendor/github.com/go-redis/redis/v8/Makefile delete mode 100644 vendor/github.com/go-redis/redis/v8/README.md delete mode 100644 vendor/github.com/go-redis/redis/v8/command.go delete mode 100644 vendor/github.com/go-redis/redis/v8/commands.go delete mode 100644 vendor/github.com/go-redis/redis/v8/internal/proto/reader.go delete mode 100644 vendor/github.com/go-redis/redis/v8/internal/safe.go delete mode 100644 vendor/github.com/go-redis/redis/v8/internal/unsafe.go delete mode 100644 vendor/github.com/go-redis/redis/v8/internal/util.go delete mode 100644 vendor/github.com/go-redis/redis/v8/package.json delete mode 100644 vendor/github.com/go-redis/redis/v8/redis.go create mode 100644 vendor/github.com/redis/go-redis/v9/.gitignore rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/.golangci.yml (100%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/.prettierrc.yml (100%) create mode 100644 vendor/github.com/redis/go-redis/v9/CHANGELOG.md create mode 100644 vendor/github.com/redis/go-redis/v9/CONTRIBUTING.md rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/LICENSE (95%) create mode 100644 vendor/github.com/redis/go-redis/v9/Makefile create mode 100644 vendor/github.com/redis/go-redis/v9/README.md rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/RELEASING.md (100%) create mode 100644 vendor/github.com/redis/go-redis/v9/acl_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/bitmap_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/cluster_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/command.go create mode 100644 vendor/github.com/redis/go-redis/v9/commands.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/doc.go (100%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/error.go (84%) create mode 100644 vendor/github.com/redis/go-redis/v9/gears_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/generic_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/geo_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/hash_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/hyperloglog_commands.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/arg.go (92%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/hashtag/hashtag.go (98%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/hscan/hscan.go (96%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/hscan/structmap.go (70%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/internal.go (89%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/log.go (100%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/once.go (94%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/pool/conn.go (82%) create mode 100644 vendor/github.com/redis/go-redis/v9/internal/pool/conn_check.go create mode 100644 vendor/github.com/redis/go-redis/v9/internal/pool/conn_check_dummy.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/pool/pool.go (71%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/pool/pool_single.go (100%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/pool/pool_sticky.go (100%) create mode 100644 vendor/github.com/redis/go-redis/v9/internal/proto/reader.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/proto/scan.go (97%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/proto/writer.go (76%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/rand/rand.go (100%) create mode 100644 vendor/github.com/redis/go-redis/v9/internal/util.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/util/safe.go (88%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/util/strconv.go (100%) create mode 100644 vendor/github.com/redis/go-redis/v9/internal/util/type.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/internal/util/unsafe.go (94%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/iterator.go (84%) create mode 100644 vendor/github.com/redis/go-redis/v9/json.go create mode 100644 vendor/github.com/redis/go-redis/v9/list_commands.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/options.go (53%) rename vendor/github.com/{go-redis/redis/v8/cluster.go => redis/go-redis/v9/osscluster.go} (69%) rename vendor/github.com/{go-redis/redis/v8/cluster_commands.go => redis/go-redis/v9/osscluster_commands.go} (85%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/pipeline.go (74%) create mode 100644 vendor/github.com/redis/go-redis/v9/probabilistic.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/pubsub.go (86%) create mode 100644 vendor/github.com/redis/go-redis/v9/pubsub_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/redis.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/result.go (87%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/ring.go (58%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/script.go (64%) create mode 100644 vendor/github.com/redis/go-redis/v9/scripting_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/search_commands.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/sentinel.go (66%) create mode 100644 vendor/github.com/redis/go-redis/v9/set_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/sortedset_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/stream_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/string_commands.go create mode 100644 vendor/github.com/redis/go-redis/v9/timeseries_commands.go rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/tx.go (79%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/universal.go (61%) rename vendor/github.com/{go-redis/redis/v8 => redis/go-redis/v9}/version.go (83%) diff --git a/go.mod b/go.mod index 526cd6f53d07..97b3897d2eb0 100644 --- a/go.mod +++ b/go.mod @@ -38,7 +38,6 @@ require ( github.com/fsouza/fake-gcs-server v1.50.2 github.com/go-kit/log v0.2.1 github.com/go-logfmt/logfmt v0.6.0 - github.com/go-redis/redis/v8 v8.11.5 github.com/gocql/gocql v0.0.0-20200526081602-cd04bd7f22a7 github.com/gogo/protobuf v1.3.2 // remember to update loki-build-image/Dockerfile too github.com/gogo/status v1.1.1 @@ -86,6 +85,7 @@ require ( github.com/prometheus/client_model v0.6.1 github.com/prometheus/common v0.60.1 github.com/prometheus/prometheus v0.53.2-0.20240726125539-d4f098ae80fb + github.com/redis/go-redis/v9 v9.7.0 github.com/segmentio/fasthash v1.0.3 github.com/shurcooL/httpfs v0.0.0-20230704072500-f1e31cf0ba5c github.com/shurcooL/vfsgen v0.0.0-20200824052919-0d455de96546 diff --git a/go.sum b/go.sum index e84a71327fa2..696163c61767 100644 --- a/go.sum +++ b/go.sum @@ -1032,6 +1032,10 @@ github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dR github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/boombuler/barcode v1.0.1/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= +github.com/bsm/ginkgo/v2 v2.12.0 h1:Ny8MWAHyOepLGlLKYmXG4IEkioBysk6GpaRTLC8zwWs= +github.com/bsm/ginkgo/v2 v2.12.0/go.mod h1:SwYbGRRDovPVboqFv0tPTcG1sN61LM1Z4ARdbAV9g4c= +github.com/bsm/gomega v1.27.10 h1:yeMWxP2pV2fG3FgAODIY8EiRE3dy0aeFYt4l7wh6yKA= +github.com/bsm/gomega v1.27.10/go.mod h1:JyEr/xRbxbtgWNi8tIEVPUYZ5Dzef52k01W3YH0H+O0= 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-20231215233829-aa82cc1e6500 h1:6lhrsTEnloDPXyeZBvSYvQf8u86jbKehZPVDDlkgDl4= @@ -1494,8 +1498,6 @@ github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91 github.com/go-playground/validator/v10 v10.19.0 h1:ol+5Fu+cSq9JD7SoSqe04GMI92cbn0+wvQ3bZ8b/AU4= github.com/go-playground/validator/v10 v10.19.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= github.com/go-redis/redis v6.15.9+incompatible/go.mod h1:NAIEuMOZ/fxfXJIrKDQDz8wamY7mA7PouImQ2Jvg6kA= -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.13.1 h1:x+LHXBI2nMB1vqndymf26quycC4aggYJ7DECYbiz03g= github.com/go-resty/resty/v2 v2.13.1/go.mod h1:GznXlLxkq6Nh4sU59rPmUw3VtgpO3aS96ORAI6Q7d+0= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= @@ -2448,6 +2450,8 @@ github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563/go.mod h1:bCqn github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/redis/go-redis/v9 v9.7.0 h1:HhLSs+B6O021gwzl+locl0zEDnyNkxMtf/Z3NNBMa9E= +github.com/redis/go-redis/v9 v9.7.0/go.mod h1:f6zhXITC7JUJIlPEiBOTXxJgPLdZcA93GewI7inzyWw= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/renier/xmlrpc v0.0.0-20170708154548-ce4a1a486c03/go.mod h1:gRAiPF5C5Nd0eyyRdqIu9qTiFSoZzpTq727b5B8fkkU= diff --git a/pkg/storage/chunk/cache/redis_cache_test.go b/pkg/storage/chunk/cache/redis_cache_test.go index 0215b5702e74..91ee7f98362d 100644 --- a/pkg/storage/chunk/cache/redis_cache_test.go +++ b/pkg/storage/chunk/cache/redis_cache_test.go @@ -7,7 +7,7 @@ import ( "github.com/alicebob/miniredis/v2" "github.com/go-kit/log" - "github.com/go-redis/redis/v8" + "github.com/redis/go-redis/v9" "github.com/stretchr/testify/require" ) diff --git a/pkg/storage/chunk/cache/redis_client.go b/pkg/storage/chunk/cache/redis_client.go index 52a718027658..55731c7b11a4 100644 --- a/pkg/storage/chunk/cache/redis_client.go +++ b/pkg/storage/chunk/cache/redis_client.go @@ -12,7 +12,7 @@ import ( "github.com/grafana/dskit/flagext" - "github.com/go-redis/redis/v8" + "github.com/redis/go-redis/v9" ) // RedisConfig defines how a RedisCache should be constructed. @@ -63,15 +63,15 @@ func NewRedisClient(cfg *RedisConfig) (*RedisClient, error) { } opt := &redis.UniversalOptions{ - Addrs: endpoints, - MasterName: cfg.MasterName, - Username: cfg.Username, - Password: cfg.Password.String(), - DB: cfg.DB, - PoolSize: cfg.PoolSize, - IdleTimeout: cfg.IdleTimeout, - MaxConnAge: cfg.MaxConnAge, - RouteRandomly: cfg.RouteRandomly, + Addrs: endpoints, + MasterName: cfg.MasterName, + Username: cfg.Username, + Password: cfg.Password.String(), + DB: cfg.DB, + PoolSize: cfg.PoolSize, + ConnMaxIdleTime: cfg.IdleTimeout, + ConnMaxLifetime: cfg.MaxConnAge, + RouteRandomly: cfg.RouteRandomly, } if cfg.EnableTLS { opt.TLSConfig = &tls.Config{InsecureSkipVerify: cfg.InsecureSkipVerify} diff --git a/pkg/storage/chunk/cache/redis_client_test.go b/pkg/storage/chunk/cache/redis_client_test.go index 2a8b7426f56d..275eb9fcb412 100644 --- a/pkg/storage/chunk/cache/redis_client_test.go +++ b/pkg/storage/chunk/cache/redis_client_test.go @@ -7,7 +7,7 @@ import ( "time" "github.com/alicebob/miniredis/v2" - "github.com/go-redis/redis/v8" + "github.com/redis/go-redis/v9" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) diff --git a/vendor/github.com/go-redis/redis/v8/.gitignore b/vendor/github.com/go-redis/redis/v8/.gitignore deleted file mode 100644 index b975a7b4c326..000000000000 --- a/vendor/github.com/go-redis/redis/v8/.gitignore +++ /dev/null @@ -1,3 +0,0 @@ -*.rdb -testdata/*/ -.idea/ diff --git a/vendor/github.com/go-redis/redis/v8/CHANGELOG.md b/vendor/github.com/go-redis/redis/v8/CHANGELOG.md deleted file mode 100644 index 195e51933866..000000000000 --- a/vendor/github.com/go-redis/redis/v8/CHANGELOG.md +++ /dev/null @@ -1,177 +0,0 @@ -## [8.11.5](https://github.com/go-redis/redis/compare/v8.11.4...v8.11.5) (2022-03-17) - - -### Bug Fixes - -* add missing Expire methods to Cmdable ([17e3b43](https://github.com/go-redis/redis/commit/17e3b43879d516437ada71cf9c0deac6a382ed9a)) -* add whitespace for avoid unlikely colisions ([7f7c181](https://github.com/go-redis/redis/commit/7f7c1817617cfec909efb13d14ad22ef05a6ad4c)) -* example/otel compile error ([#2028](https://github.com/go-redis/redis/issues/2028)) ([187c07c](https://github.com/go-redis/redis/commit/187c07c41bf68dc3ab280bc3a925e960bbef6475)) -* **extra/redisotel:** set span.kind attribute to client ([065b200](https://github.com/go-redis/redis/commit/065b200070b41e6e949710b4f9e01b50ccc60ab2)) -* format ([96f53a0](https://github.com/go-redis/redis/commit/96f53a0159a28affa94beec1543a62234e7f8b32)) -* invalid type assert in stringArg ([de6c131](https://github.com/go-redis/redis/commit/de6c131865b8263400c8491777b295035f2408e4)) -* rename Golang to Go ([#2030](https://github.com/go-redis/redis/issues/2030)) ([b82a2d9](https://github.com/go-redis/redis/commit/b82a2d9d4d2de7b7cbe8fcd4895be62dbcacacbc)) -* set timeout for WAIT command. Fixes [#1963](https://github.com/go-redis/redis/issues/1963) ([333fee1](https://github.com/go-redis/redis/commit/333fee1a8fd98a2fbff1ab187c1b03246a7eb01f)) -* update some argument counts in pre-allocs ([f6974eb](https://github.com/go-redis/redis/commit/f6974ebb5c40a8adf90d2cacab6dc297f4eba4c2)) - - -### Features - -* Add redis v7's NX, XX, GT, LT expire variants ([e19bbb2](https://github.com/go-redis/redis/commit/e19bbb26e2e395c6e077b48d80d79e99f729a8b8)) -* add support for acl sentinel auth in universal client ([ab0ccc4](https://github.com/go-redis/redis/commit/ab0ccc47413f9b2a6eabc852fed5005a3ee1af6e)) -* add support for COPY command ([#2016](https://github.com/go-redis/redis/issues/2016)) ([730afbc](https://github.com/go-redis/redis/commit/730afbcffb93760e8a36cc06cfe55ab102b693a7)) -* add support for passing extra attributes added to spans ([39faaa1](https://github.com/go-redis/redis/commit/39faaa171523834ba527c9789710c4fde87f5a2e)) -* add support for time.Duration write and scan ([2f1b74e](https://github.com/go-redis/redis/commit/2f1b74e20cdd7719b2aecf0768d3e3ae7c3e781b)) -* **redisotel:** ability to override TracerProvider ([#1998](https://github.com/go-redis/redis/issues/1998)) ([bf8d4aa](https://github.com/go-redis/redis/commit/bf8d4aa60c00366cda2e98c3ddddc8cf68507417)) -* set net.peer.name and net.peer.port in otel example ([69bf454](https://github.com/go-redis/redis/commit/69bf454f706204211cd34835f76b2e8192d3766d)) - - - -## [8.11.4](https://github.com/go-redis/redis/compare/v8.11.3...v8.11.4) (2021-10-04) - - -### Features - -* add acl auth support for sentinels ([f66582f](https://github.com/go-redis/redis/commit/f66582f44f3dc3a4705a5260f982043fde4aa634)) -* add Cmd.{String,Int,Float,Bool}Slice helpers and an example ([5d3d293](https://github.com/go-redis/redis/commit/5d3d293cc9c60b90871e2420602001463708ce24)) -* add SetVal method for each command ([168981d](https://github.com/go-redis/redis/commit/168981da2d84ee9e07d15d3e74d738c162e264c4)) - - - -## v8.11 - -- Remove OpenTelemetry metrics. -- Supports more redis commands and options. - -## v8.10 - -- Removed extra OpenTelemetry spans from go-redis core. Now go-redis instrumentation only adds a - single span with a Redis command (instead of 4 spans). There are multiple reasons behind this - decision: - - - Traces become smaller and less noisy. - - It may be costly to process those 3 extra spans for each query. - - go-redis no longer depends on OpenTelemetry. - - Eventually we hope to replace the information that we no longer collect with OpenTelemetry - Metrics. - -## v8.9 - -- Changed `PubSub.Channel` to only rely on `Ping` result. You can now use `WithChannelSize`, - `WithChannelHealthCheckInterval`, and `WithChannelSendTimeout` to override default settings. - -## v8.8 - -- To make updating easier, extra modules now have the same version as go-redis does. That means that - you need to update your imports: - -``` -github.com/go-redis/redis/extra/redisotel -> github.com/go-redis/redis/extra/redisotel/v8 -github.com/go-redis/redis/extra/rediscensus -> github.com/go-redis/redis/extra/rediscensus/v8 -``` - -## v8.5 - -- [knadh](https://github.com/knadh) contributed long-awaited ability to scan Redis Hash into a - struct: - -```go -err := rdb.HGetAll(ctx, "hash").Scan(&data) - -err := rdb.MGet(ctx, "key1", "key2").Scan(&data) -``` - -- Please check [redismock](https://github.com/go-redis/redismock) by - [monkey92t](https://github.com/monkey92t) if you are looking for mocking Redis Client. - -## v8 - -- All commands require `context.Context` as a first argument, e.g. `rdb.Ping(ctx)`. If you are not - using `context.Context` yet, the simplest option is to define global package variable - `var ctx = context.TODO()` and use it when `ctx` is required. - -- Full support for `context.Context` canceling. - -- Added `redis.NewFailoverClusterClient` that supports routing read-only commands to a slave node. - -- Added `redisext.OpenTemetryHook` that adds - [Redis OpenTelemetry instrumentation](https://redis.uptrace.dev/tracing/). - -- Redis slow log support. - -- Ring uses Rendezvous Hashing by default which provides better distribution. You need to move - existing keys to a new location or keys will be inaccessible / lost. To use old hashing scheme: - -```go -import "github.com/golang/groupcache/consistenthash" - -ring := redis.NewRing(&redis.RingOptions{ - NewConsistentHash: func() { - return consistenthash.New(100, crc32.ChecksumIEEE) - }, -}) -``` - -- `ClusterOptions.MaxRedirects` default value is changed from 8 to 3. -- `Options.MaxRetries` default value is changed from 0 to 3. - -- `Cluster.ForEachNode` is renamed to `ForEachShard` for consistency with `Ring`. - -## v7.3 - -- New option `Options.Username` which causes client to use `AuthACL`. Be aware if your connection - URL contains username. - -## v7.2 - -- Existing `HMSet` is renamed to `HSet` and old deprecated `HMSet` is restored for Redis 3 users. - -## v7.1 - -- Existing `Cmd.String` is renamed to `Cmd.Text`. New `Cmd.String` implements `fmt.Stringer` - interface. - -## v7 - -- _Important_. Tx.Pipeline now returns a non-transactional pipeline. Use Tx.TxPipeline for a - transactional pipeline. -- WrapProcess is replaced with more convenient AddHook that has access to context.Context. -- WithContext now can not be used to create a shallow copy of the client. -- New methods ProcessContext, DoContext, and ExecContext. -- Client respects Context.Deadline when setting net.Conn deadline. -- Client listens on Context.Done while waiting for a connection from the pool and returns an error - when context context is cancelled. -- Add PubSub.ChannelWithSubscriptions that sends `*Subscription` in addition to `*Message` to allow - detecting reconnections. -- `time.Time` is now marshalled in RFC3339 format. `rdb.Get("foo").Time()` helper is added to parse - the time. -- `SetLimiter` is removed and added `Options.Limiter` instead. -- `HMSet` is deprecated as of Redis v4. - -## v6.15 - -- Cluster and Ring pipelines process commands for each node in its own goroutine. - -## 6.14 - -- Added Options.MinIdleConns. -- Added Options.MaxConnAge. -- PoolStats.FreeConns is renamed to PoolStats.IdleConns. -- Add Client.Do to simplify creating custom commands. -- Add Cmd.String, Cmd.Int, Cmd.Int64, Cmd.Uint64, Cmd.Float64, and Cmd.Bool helpers. -- Lower memory usage. - -## v6.13 - -- Ring got new options called `HashReplicas` and `Hash`. It is recommended to set - `HashReplicas = 1000` for better keys distribution between shards. -- Cluster client was optimized to use much less memory when reloading cluster state. -- PubSub.ReceiveMessage is re-worked to not use ReceiveTimeout so it does not lose data when timeout - occurres. In most cases it is recommended to use PubSub.Channel instead. -- Dialer.KeepAlive is set to 5 minutes by default. - -## v6.12 - -- ClusterClient got new option called `ClusterSlots` which allows to build cluster of normal Redis - Servers that don't have cluster mode enabled. See - https://godoc.org/github.com/go-redis/redis#example-NewClusterClient--ManualSetup diff --git a/vendor/github.com/go-redis/redis/v8/Makefile b/vendor/github.com/go-redis/redis/v8/Makefile deleted file mode 100644 index a4cfe0576e68..000000000000 --- a/vendor/github.com/go-redis/redis/v8/Makefile +++ /dev/null @@ -1,35 +0,0 @@ -PACKAGE_DIRS := $(shell find . -mindepth 2 -type f -name 'go.mod' -exec dirname {} \; | sort) - -test: testdeps - go test ./... - go test ./... -short -race - go test ./... -run=NONE -bench=. -benchmem - env GOOS=linux GOARCH=386 go test ./... - go vet - -testdeps: testdata/redis/src/redis-server - -bench: testdeps - go test ./... -test.run=NONE -test.bench=. -test.benchmem - -.PHONY: all test testdeps bench - -testdata/redis: - mkdir -p $@ - wget -qO- https://download.redis.io/releases/redis-6.2.5.tar.gz | tar xvz --strip-components=1 -C $@ - -testdata/redis/src/redis-server: testdata/redis - cd $< && make all - -fmt: - gofmt -w -s ./ - goimports -w -local github.com/go-redis/redis ./ - -go_mod_tidy: - go get -u && go mod tidy - set -e; for dir in $(PACKAGE_DIRS); do \ - echo "go mod tidy in $${dir}"; \ - (cd "$${dir}" && \ - go get -u && \ - go mod tidy); \ - done diff --git a/vendor/github.com/go-redis/redis/v8/README.md b/vendor/github.com/go-redis/redis/v8/README.md deleted file mode 100644 index f3b6a018cb54..000000000000 --- a/vendor/github.com/go-redis/redis/v8/README.md +++ /dev/null @@ -1,175 +0,0 @@ -# Redis client for Go - -![build workflow](https://github.com/go-redis/redis/actions/workflows/build.yml/badge.svg) -[![PkgGoDev](https://pkg.go.dev/badge/github.com/go-redis/redis/v8)](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc) -[![Documentation](https://img.shields.io/badge/redis-documentation-informational)](https://redis.uptrace.dev/) - -go-redis is brought to you by :star: [**uptrace/uptrace**](https://github.com/uptrace/uptrace). -Uptrace is an open source and blazingly fast **distributed tracing** backend powered by -OpenTelemetry and ClickHouse. Give it a star as well! - -## Resources - -- [Discussions](https://github.com/go-redis/redis/discussions) -- [Documentation](https://redis.uptrace.dev) -- [Reference](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc) -- [Examples](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#pkg-examples) -- [RealWorld example app](https://github.com/uptrace/go-treemux-realworld-example-app) - -Other projects you may like: - -- [Bun](https://bun.uptrace.dev) - fast and simple SQL client for PostgreSQL, MySQL, and SQLite. -- [BunRouter](https://bunrouter.uptrace.dev/) - fast and flexible HTTP router for Go. - -## Ecosystem - -- [Redis Mock](https://github.com/go-redis/redismock) -- [Distributed Locks](https://github.com/bsm/redislock) -- [Redis Cache](https://github.com/go-redis/cache) -- [Rate limiting](https://github.com/go-redis/redis_rate) - -## Features - -- Redis 3 commands except QUIT, MONITOR, and SYNC. -- Automatic connection pooling with - [circuit breaker](https://en.wikipedia.org/wiki/Circuit_breaker_design_pattern) support. -- [Pub/Sub](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#PubSub). -- [Transactions](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#example-Client-TxPipeline). -- [Pipeline](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#example-Client.Pipeline) and - [TxPipeline](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#example-Client.TxPipeline). -- [Scripting](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#Script). -- [Timeouts](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#Options). -- [Redis Sentinel](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#NewFailoverClient). -- [Redis Cluster](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#NewClusterClient). -- [Cluster of Redis Servers](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#example-NewClusterClient-ManualSetup) - without using cluster mode and Redis Sentinel. -- [Ring](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#NewRing). -- [Instrumentation](https://pkg.go.dev/github.com/go-redis/redis/v8?tab=doc#example-package-Instrumentation). - -## Installation - -go-redis supports 2 last Go versions and requires a Go version with -[modules](https://github.com/golang/go/wiki/Modules) support. So make sure to initialize a Go -module: - -```shell -go mod init github.com/my/repo -``` - -And then install go-redis/v8 (note _v8_ in the import; omitting it is a popular mistake): - -```shell -go get github.com/go-redis/redis/v8 -``` - -## Quickstart - -```go -import ( - "context" - "github.com/go-redis/redis/v8" - "fmt" -) - -var ctx = context.Background() - -func ExampleClient() { - rdb := redis.NewClient(&redis.Options{ - Addr: "localhost:6379", - Password: "", // no password set - DB: 0, // use default DB - }) - - err := rdb.Set(ctx, "key", "value", 0).Err() - if err != nil { - panic(err) - } - - val, err := rdb.Get(ctx, "key").Result() - if err != nil { - panic(err) - } - fmt.Println("key", val) - - val2, err := rdb.Get(ctx, "key2").Result() - if err == redis.Nil { - fmt.Println("key2 does not exist") - } else if err != nil { - panic(err) - } else { - fmt.Println("key2", val2) - } - // Output: key value - // key2 does not exist -} -``` - -## Look and feel - -Some corner cases: - -```go -// SET key value EX 10 NX -set, err := rdb.SetNX(ctx, "key", "value", 10*time.Second).Result() - -// SET key value keepttl NX -set, err := rdb.SetNX(ctx, "key", "value", redis.KeepTTL).Result() - -// SORT list LIMIT 0 2 ASC -vals, err := rdb.Sort(ctx, "list", &redis.Sort{Offset: 0, Count: 2, Order: "ASC"}).Result() - -// ZRANGEBYSCORE zset -inf +inf WITHSCORES LIMIT 0 2 -vals, err := rdb.ZRangeByScoreWithScores(ctx, "zset", &redis.ZRangeBy{ - Min: "-inf", - Max: "+inf", - Offset: 0, - Count: 2, -}).Result() - -// ZINTERSTORE out 2 zset1 zset2 WEIGHTS 2 3 AGGREGATE SUM -vals, err := rdb.ZInterStore(ctx, "out", &redis.ZStore{ - Keys: []string{"zset1", "zset2"}, - Weights: []int64{2, 3} -}).Result() - -// EVAL "return {KEYS[1],ARGV[1]}" 1 "key" "hello" -vals, err := rdb.Eval(ctx, "return {KEYS[1],ARGV[1]}", []string{"key"}, "hello").Result() - -// custom command -res, err := rdb.Do(ctx, "set", "key", "value").Result() -``` - -## Run the test - -go-redis will start a redis-server and run the test cases. - -The paths of redis-server bin file and redis config file are defined in `main_test.go`: - -``` -var ( - redisServerBin, _ = filepath.Abs(filepath.Join("testdata", "redis", "src", "redis-server")) - redisServerConf, _ = filepath.Abs(filepath.Join("testdata", "redis", "redis.conf")) -) -``` - -For local testing, you can change the variables to refer to your local files, or create a soft link -to the corresponding folder for redis-server and copy the config file to `testdata/redis/`: - -``` -ln -s /usr/bin/redis-server ./go-redis/testdata/redis/src -cp ./go-redis/testdata/redis.conf ./go-redis/testdata/redis/ -``` - -Lastly, run: - -``` -go test -``` - -## Contributors - -Thanks to all the people who already contributed! - - - - diff --git a/vendor/github.com/go-redis/redis/v8/command.go b/vendor/github.com/go-redis/redis/v8/command.go deleted file mode 100644 index 4bb12a85be43..000000000000 --- a/vendor/github.com/go-redis/redis/v8/command.go +++ /dev/null @@ -1,3478 +0,0 @@ -package redis - -import ( - "context" - "fmt" - "net" - "strconv" - "time" - - "github.com/go-redis/redis/v8/internal" - "github.com/go-redis/redis/v8/internal/hscan" - "github.com/go-redis/redis/v8/internal/proto" - "github.com/go-redis/redis/v8/internal/util" -) - -type Cmder interface { - Name() string - FullName() string - Args() []interface{} - String() string - stringArg(int) string - firstKeyPos() int8 - SetFirstKeyPos(int8) - - readTimeout() *time.Duration - readReply(rd *proto.Reader) error - - SetErr(error) - Err() error -} - -func setCmdsErr(cmds []Cmder, e error) { - for _, cmd := range cmds { - if cmd.Err() == nil { - cmd.SetErr(e) - } - } -} - -func cmdsFirstErr(cmds []Cmder) error { - for _, cmd := range cmds { - if err := cmd.Err(); err != nil { - return err - } - } - return nil -} - -func writeCmds(wr *proto.Writer, cmds []Cmder) error { - for _, cmd := range cmds { - if err := writeCmd(wr, cmd); err != nil { - return err - } - } - return nil -} - -func writeCmd(wr *proto.Writer, cmd Cmder) error { - return wr.WriteArgs(cmd.Args()) -} - -func cmdFirstKeyPos(cmd Cmder, info *CommandInfo) int { - if pos := cmd.firstKeyPos(); pos != 0 { - return int(pos) - } - - switch cmd.Name() { - case "eval", "evalsha": - if cmd.stringArg(2) != "0" { - return 3 - } - - return 0 - case "publish": - return 1 - case "memory": - // https://github.com/redis/redis/issues/7493 - if cmd.stringArg(1) == "usage" { - return 2 - } - } - - if info != nil { - return int(info.FirstKeyPos) - } - return 0 -} - -func cmdString(cmd Cmder, val interface{}) string { - b := make([]byte, 0, 64) - - for i, arg := range cmd.Args() { - if i > 0 { - b = append(b, ' ') - } - b = internal.AppendArg(b, arg) - } - - if err := cmd.Err(); err != nil { - b = append(b, ": "...) - b = append(b, err.Error()...) - } else if val != nil { - b = append(b, ": "...) - b = internal.AppendArg(b, val) - } - - return internal.String(b) -} - -//------------------------------------------------------------------------------ - -type baseCmd struct { - ctx context.Context - args []interface{} - err error - keyPos int8 - - _readTimeout *time.Duration -} - -var _ Cmder = (*Cmd)(nil) - -func (cmd *baseCmd) Name() string { - if len(cmd.args) == 0 { - return "" - } - // Cmd name must be lower cased. - return internal.ToLower(cmd.stringArg(0)) -} - -func (cmd *baseCmd) FullName() string { - switch name := cmd.Name(); name { - case "cluster", "command": - if len(cmd.args) == 1 { - return name - } - if s2, ok := cmd.args[1].(string); ok { - return name + " " + s2 - } - return name - default: - return name - } -} - -func (cmd *baseCmd) Args() []interface{} { - return cmd.args -} - -func (cmd *baseCmd) stringArg(pos int) string { - if pos < 0 || pos >= len(cmd.args) { - return "" - } - arg := cmd.args[pos] - switch v := arg.(type) { - case string: - return v - default: - // TODO: consider using appendArg - return fmt.Sprint(v) - } -} - -func (cmd *baseCmd) firstKeyPos() int8 { - return cmd.keyPos -} - -func (cmd *baseCmd) SetFirstKeyPos(keyPos int8) { - cmd.keyPos = keyPos -} - -func (cmd *baseCmd) SetErr(e error) { - cmd.err = e -} - -func (cmd *baseCmd) Err() error { - return cmd.err -} - -func (cmd *baseCmd) readTimeout() *time.Duration { - return cmd._readTimeout -} - -func (cmd *baseCmd) setReadTimeout(d time.Duration) { - cmd._readTimeout = &d -} - -//------------------------------------------------------------------------------ - -type Cmd struct { - baseCmd - - val interface{} -} - -func NewCmd(ctx context.Context, args ...interface{}) *Cmd { - return &Cmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *Cmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *Cmd) SetVal(val interface{}) { - cmd.val = val -} - -func (cmd *Cmd) Val() interface{} { - return cmd.val -} - -func (cmd *Cmd) Result() (interface{}, error) { - return cmd.val, cmd.err -} - -func (cmd *Cmd) Text() (string, error) { - if cmd.err != nil { - return "", cmd.err - } - return toString(cmd.val) -} - -func toString(val interface{}) (string, error) { - switch val := val.(type) { - case string: - return val, nil - default: - err := fmt.Errorf("redis: unexpected type=%T for String", val) - return "", err - } -} - -func (cmd *Cmd) Int() (int, error) { - if cmd.err != nil { - return 0, cmd.err - } - switch val := cmd.val.(type) { - case int64: - return int(val), nil - case string: - return strconv.Atoi(val) - default: - err := fmt.Errorf("redis: unexpected type=%T for Int", val) - return 0, err - } -} - -func (cmd *Cmd) Int64() (int64, error) { - if cmd.err != nil { - return 0, cmd.err - } - return toInt64(cmd.val) -} - -func toInt64(val interface{}) (int64, error) { - switch val := val.(type) { - case int64: - return val, nil - case string: - return strconv.ParseInt(val, 10, 64) - default: - err := fmt.Errorf("redis: unexpected type=%T for Int64", val) - return 0, err - } -} - -func (cmd *Cmd) Uint64() (uint64, error) { - if cmd.err != nil { - return 0, cmd.err - } - return toUint64(cmd.val) -} - -func toUint64(val interface{}) (uint64, error) { - switch val := val.(type) { - case int64: - return uint64(val), nil - case string: - return strconv.ParseUint(val, 10, 64) - default: - err := fmt.Errorf("redis: unexpected type=%T for Uint64", val) - return 0, err - } -} - -func (cmd *Cmd) Float32() (float32, error) { - if cmd.err != nil { - return 0, cmd.err - } - return toFloat32(cmd.val) -} - -func toFloat32(val interface{}) (float32, error) { - switch val := val.(type) { - case int64: - return float32(val), nil - case string: - f, err := strconv.ParseFloat(val, 32) - if err != nil { - return 0, err - } - return float32(f), nil - default: - err := fmt.Errorf("redis: unexpected type=%T for Float32", val) - return 0, err - } -} - -func (cmd *Cmd) Float64() (float64, error) { - if cmd.err != nil { - return 0, cmd.err - } - return toFloat64(cmd.val) -} - -func toFloat64(val interface{}) (float64, error) { - switch val := val.(type) { - case int64: - return float64(val), nil - case string: - return strconv.ParseFloat(val, 64) - default: - err := fmt.Errorf("redis: unexpected type=%T for Float64", val) - return 0, err - } -} - -func (cmd *Cmd) Bool() (bool, error) { - if cmd.err != nil { - return false, cmd.err - } - return toBool(cmd.val) -} - -func toBool(val interface{}) (bool, error) { - switch val := val.(type) { - case int64: - return val != 0, nil - case string: - return strconv.ParseBool(val) - default: - err := fmt.Errorf("redis: unexpected type=%T for Bool", val) - return false, err - } -} - -func (cmd *Cmd) Slice() ([]interface{}, error) { - if cmd.err != nil { - return nil, cmd.err - } - switch val := cmd.val.(type) { - case []interface{}: - return val, nil - default: - return nil, fmt.Errorf("redis: unexpected type=%T for Slice", val) - } -} - -func (cmd *Cmd) StringSlice() ([]string, error) { - slice, err := cmd.Slice() - if err != nil { - return nil, err - } - - ss := make([]string, len(slice)) - for i, iface := range slice { - val, err := toString(iface) - if err != nil { - return nil, err - } - ss[i] = val - } - return ss, nil -} - -func (cmd *Cmd) Int64Slice() ([]int64, error) { - slice, err := cmd.Slice() - if err != nil { - return nil, err - } - - nums := make([]int64, len(slice)) - for i, iface := range slice { - val, err := toInt64(iface) - if err != nil { - return nil, err - } - nums[i] = val - } - return nums, nil -} - -func (cmd *Cmd) Uint64Slice() ([]uint64, error) { - slice, err := cmd.Slice() - if err != nil { - return nil, err - } - - nums := make([]uint64, len(slice)) - for i, iface := range slice { - val, err := toUint64(iface) - if err != nil { - return nil, err - } - nums[i] = val - } - return nums, nil -} - -func (cmd *Cmd) Float32Slice() ([]float32, error) { - slice, err := cmd.Slice() - if err != nil { - return nil, err - } - - floats := make([]float32, len(slice)) - for i, iface := range slice { - val, err := toFloat32(iface) - if err != nil { - return nil, err - } - floats[i] = val - } - return floats, nil -} - -func (cmd *Cmd) Float64Slice() ([]float64, error) { - slice, err := cmd.Slice() - if err != nil { - return nil, err - } - - floats := make([]float64, len(slice)) - for i, iface := range slice { - val, err := toFloat64(iface) - if err != nil { - return nil, err - } - floats[i] = val - } - return floats, nil -} - -func (cmd *Cmd) BoolSlice() ([]bool, error) { - slice, err := cmd.Slice() - if err != nil { - return nil, err - } - - bools := make([]bool, len(slice)) - for i, iface := range slice { - val, err := toBool(iface) - if err != nil { - return nil, err - } - bools[i] = val - } - return bools, nil -} - -func (cmd *Cmd) readReply(rd *proto.Reader) (err error) { - cmd.val, err = rd.ReadReply(sliceParser) - return err -} - -// sliceParser implements proto.MultiBulkParse. -func sliceParser(rd *proto.Reader, n int64) (interface{}, error) { - vals := make([]interface{}, n) - for i := 0; i < len(vals); i++ { - v, err := rd.ReadReply(sliceParser) - if err != nil { - if err == Nil { - vals[i] = nil - continue - } - if err, ok := err.(proto.RedisError); ok { - vals[i] = err - continue - } - return nil, err - } - vals[i] = v - } - return vals, nil -} - -//------------------------------------------------------------------------------ - -type SliceCmd struct { - baseCmd - - val []interface{} -} - -var _ Cmder = (*SliceCmd)(nil) - -func NewSliceCmd(ctx context.Context, args ...interface{}) *SliceCmd { - return &SliceCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *SliceCmd) SetVal(val []interface{}) { - cmd.val = val -} - -func (cmd *SliceCmd) Val() []interface{} { - return cmd.val -} - -func (cmd *SliceCmd) Result() ([]interface{}, error) { - return cmd.val, cmd.err -} - -func (cmd *SliceCmd) String() string { - return cmdString(cmd, cmd.val) -} - -// Scan scans the results from the map into a destination struct. The map keys -// are matched in the Redis struct fields by the `redis:"field"` tag. -func (cmd *SliceCmd) Scan(dst interface{}) error { - if cmd.err != nil { - return cmd.err - } - - // Pass the list of keys and values. - // Skip the first two args for: HMGET key - var args []interface{} - if cmd.args[0] == "hmget" { - args = cmd.args[2:] - } else { - // Otherwise, it's: MGET field field ... - args = cmd.args[1:] - } - - return hscan.Scan(dst, args, cmd.val) -} - -func (cmd *SliceCmd) readReply(rd *proto.Reader) error { - v, err := rd.ReadArrayReply(sliceParser) - if err != nil { - return err - } - cmd.val = v.([]interface{}) - return nil -} - -//------------------------------------------------------------------------------ - -type StatusCmd struct { - baseCmd - - val string -} - -var _ Cmder = (*StatusCmd)(nil) - -func NewStatusCmd(ctx context.Context, args ...interface{}) *StatusCmd { - return &StatusCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *StatusCmd) SetVal(val string) { - cmd.val = val -} - -func (cmd *StatusCmd) Val() string { - return cmd.val -} - -func (cmd *StatusCmd) Result() (string, error) { - return cmd.val, cmd.err -} - -func (cmd *StatusCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *StatusCmd) readReply(rd *proto.Reader) (err error) { - cmd.val, err = rd.ReadString() - return err -} - -//------------------------------------------------------------------------------ - -type IntCmd struct { - baseCmd - - val int64 -} - -var _ Cmder = (*IntCmd)(nil) - -func NewIntCmd(ctx context.Context, args ...interface{}) *IntCmd { - return &IntCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *IntCmd) SetVal(val int64) { - cmd.val = val -} - -func (cmd *IntCmd) Val() int64 { - return cmd.val -} - -func (cmd *IntCmd) Result() (int64, error) { - return cmd.val, cmd.err -} - -func (cmd *IntCmd) Uint64() (uint64, error) { - return uint64(cmd.val), cmd.err -} - -func (cmd *IntCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *IntCmd) readReply(rd *proto.Reader) (err error) { - cmd.val, err = rd.ReadIntReply() - return err -} - -//------------------------------------------------------------------------------ - -type IntSliceCmd struct { - baseCmd - - val []int64 -} - -var _ Cmder = (*IntSliceCmd)(nil) - -func NewIntSliceCmd(ctx context.Context, args ...interface{}) *IntSliceCmd { - return &IntSliceCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *IntSliceCmd) SetVal(val []int64) { - cmd.val = val -} - -func (cmd *IntSliceCmd) Val() []int64 { - return cmd.val -} - -func (cmd *IntSliceCmd) Result() ([]int64, error) { - return cmd.val, cmd.err -} - -func (cmd *IntSliceCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *IntSliceCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]int64, n) - for i := 0; i < len(cmd.val); i++ { - num, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - cmd.val[i] = num - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type DurationCmd struct { - baseCmd - - val time.Duration - precision time.Duration -} - -var _ Cmder = (*DurationCmd)(nil) - -func NewDurationCmd(ctx context.Context, precision time.Duration, args ...interface{}) *DurationCmd { - return &DurationCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - precision: precision, - } -} - -func (cmd *DurationCmd) SetVal(val time.Duration) { - cmd.val = val -} - -func (cmd *DurationCmd) Val() time.Duration { - return cmd.val -} - -func (cmd *DurationCmd) Result() (time.Duration, error) { - return cmd.val, cmd.err -} - -func (cmd *DurationCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *DurationCmd) readReply(rd *proto.Reader) error { - n, err := rd.ReadIntReply() - if err != nil { - return err - } - switch n { - // -2 if the key does not exist - // -1 if the key exists but has no associated expire - case -2, -1: - cmd.val = time.Duration(n) - default: - cmd.val = time.Duration(n) * cmd.precision - } - return nil -} - -//------------------------------------------------------------------------------ - -type TimeCmd struct { - baseCmd - - val time.Time -} - -var _ Cmder = (*TimeCmd)(nil) - -func NewTimeCmd(ctx context.Context, args ...interface{}) *TimeCmd { - return &TimeCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *TimeCmd) SetVal(val time.Time) { - cmd.val = val -} - -func (cmd *TimeCmd) Val() time.Time { - return cmd.val -} - -func (cmd *TimeCmd) Result() (time.Time, error) { - return cmd.val, cmd.err -} - -func (cmd *TimeCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *TimeCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - if n != 2 { - return nil, fmt.Errorf("got %d elements, expected 2", n) - } - - sec, err := rd.ReadInt() - if err != nil { - return nil, err - } - - microsec, err := rd.ReadInt() - if err != nil { - return nil, err - } - - cmd.val = time.Unix(sec, microsec*1000) - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type BoolCmd struct { - baseCmd - - val bool -} - -var _ Cmder = (*BoolCmd)(nil) - -func NewBoolCmd(ctx context.Context, args ...interface{}) *BoolCmd { - return &BoolCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *BoolCmd) SetVal(val bool) { - cmd.val = val -} - -func (cmd *BoolCmd) Val() bool { - return cmd.val -} - -func (cmd *BoolCmd) Result() (bool, error) { - return cmd.val, cmd.err -} - -func (cmd *BoolCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *BoolCmd) readReply(rd *proto.Reader) error { - v, err := rd.ReadReply(nil) - // `SET key value NX` returns nil when key already exists. But - // `SETNX key value` returns bool (0/1). So convert nil to bool. - if err == Nil { - cmd.val = false - return nil - } - if err != nil { - return err - } - switch v := v.(type) { - case int64: - cmd.val = v == 1 - return nil - case string: - cmd.val = v == "OK" - return nil - default: - return fmt.Errorf("got %T, wanted int64 or string", v) - } -} - -//------------------------------------------------------------------------------ - -type StringCmd struct { - baseCmd - - val string -} - -var _ Cmder = (*StringCmd)(nil) - -func NewStringCmd(ctx context.Context, args ...interface{}) *StringCmd { - return &StringCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *StringCmd) SetVal(val string) { - cmd.val = val -} - -func (cmd *StringCmd) Val() string { - return cmd.val -} - -func (cmd *StringCmd) Result() (string, error) { - return cmd.Val(), cmd.err -} - -func (cmd *StringCmd) Bytes() ([]byte, error) { - return util.StringToBytes(cmd.val), cmd.err -} - -func (cmd *StringCmd) Bool() (bool, error) { - if cmd.err != nil { - return false, cmd.err - } - return strconv.ParseBool(cmd.val) -} - -func (cmd *StringCmd) Int() (int, error) { - if cmd.err != nil { - return 0, cmd.err - } - return strconv.Atoi(cmd.Val()) -} - -func (cmd *StringCmd) Int64() (int64, error) { - if cmd.err != nil { - return 0, cmd.err - } - return strconv.ParseInt(cmd.Val(), 10, 64) -} - -func (cmd *StringCmd) Uint64() (uint64, error) { - if cmd.err != nil { - return 0, cmd.err - } - return strconv.ParseUint(cmd.Val(), 10, 64) -} - -func (cmd *StringCmd) Float32() (float32, error) { - if cmd.err != nil { - return 0, cmd.err - } - f, err := strconv.ParseFloat(cmd.Val(), 32) - if err != nil { - return 0, err - } - return float32(f), nil -} - -func (cmd *StringCmd) Float64() (float64, error) { - if cmd.err != nil { - return 0, cmd.err - } - return strconv.ParseFloat(cmd.Val(), 64) -} - -func (cmd *StringCmd) Time() (time.Time, error) { - if cmd.err != nil { - return time.Time{}, cmd.err - } - return time.Parse(time.RFC3339Nano, cmd.Val()) -} - -func (cmd *StringCmd) Scan(val interface{}) error { - if cmd.err != nil { - return cmd.err - } - return proto.Scan([]byte(cmd.val), val) -} - -func (cmd *StringCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *StringCmd) readReply(rd *proto.Reader) (err error) { - cmd.val, err = rd.ReadString() - return err -} - -//------------------------------------------------------------------------------ - -type FloatCmd struct { - baseCmd - - val float64 -} - -var _ Cmder = (*FloatCmd)(nil) - -func NewFloatCmd(ctx context.Context, args ...interface{}) *FloatCmd { - return &FloatCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *FloatCmd) SetVal(val float64) { - cmd.val = val -} - -func (cmd *FloatCmd) Val() float64 { - return cmd.val -} - -func (cmd *FloatCmd) Result() (float64, error) { - return cmd.Val(), cmd.Err() -} - -func (cmd *FloatCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *FloatCmd) readReply(rd *proto.Reader) (err error) { - cmd.val, err = rd.ReadFloatReply() - return err -} - -//------------------------------------------------------------------------------ - -type FloatSliceCmd struct { - baseCmd - - val []float64 -} - -var _ Cmder = (*FloatSliceCmd)(nil) - -func NewFloatSliceCmd(ctx context.Context, args ...interface{}) *FloatSliceCmd { - return &FloatSliceCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *FloatSliceCmd) SetVal(val []float64) { - cmd.val = val -} - -func (cmd *FloatSliceCmd) Val() []float64 { - return cmd.val -} - -func (cmd *FloatSliceCmd) Result() ([]float64, error) { - return cmd.val, cmd.err -} - -func (cmd *FloatSliceCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *FloatSliceCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]float64, n) - for i := 0; i < len(cmd.val); i++ { - switch num, err := rd.ReadFloatReply(); { - case err == Nil: - cmd.val[i] = 0 - case err != nil: - return nil, err - default: - cmd.val[i] = num - } - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type StringSliceCmd struct { - baseCmd - - val []string -} - -var _ Cmder = (*StringSliceCmd)(nil) - -func NewStringSliceCmd(ctx context.Context, args ...interface{}) *StringSliceCmd { - return &StringSliceCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *StringSliceCmd) SetVal(val []string) { - cmd.val = val -} - -func (cmd *StringSliceCmd) Val() []string { - return cmd.val -} - -func (cmd *StringSliceCmd) Result() ([]string, error) { - return cmd.Val(), cmd.Err() -} - -func (cmd *StringSliceCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *StringSliceCmd) ScanSlice(container interface{}) error { - return proto.ScanSlice(cmd.Val(), container) -} - -func (cmd *StringSliceCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]string, n) - for i := 0; i < len(cmd.val); i++ { - switch s, err := rd.ReadString(); { - case err == Nil: - cmd.val[i] = "" - case err != nil: - return nil, err - default: - cmd.val[i] = s - } - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type BoolSliceCmd struct { - baseCmd - - val []bool -} - -var _ Cmder = (*BoolSliceCmd)(nil) - -func NewBoolSliceCmd(ctx context.Context, args ...interface{}) *BoolSliceCmd { - return &BoolSliceCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *BoolSliceCmd) SetVal(val []bool) { - cmd.val = val -} - -func (cmd *BoolSliceCmd) Val() []bool { - return cmd.val -} - -func (cmd *BoolSliceCmd) Result() ([]bool, error) { - return cmd.val, cmd.err -} - -func (cmd *BoolSliceCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *BoolSliceCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]bool, n) - for i := 0; i < len(cmd.val); i++ { - n, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - cmd.val[i] = n == 1 - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type StringStringMapCmd struct { - baseCmd - - val map[string]string -} - -var _ Cmder = (*StringStringMapCmd)(nil) - -func NewStringStringMapCmd(ctx context.Context, args ...interface{}) *StringStringMapCmd { - return &StringStringMapCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *StringStringMapCmd) SetVal(val map[string]string) { - cmd.val = val -} - -func (cmd *StringStringMapCmd) Val() map[string]string { - return cmd.val -} - -func (cmd *StringStringMapCmd) Result() (map[string]string, error) { - return cmd.val, cmd.err -} - -func (cmd *StringStringMapCmd) String() string { - return cmdString(cmd, cmd.val) -} - -// Scan scans the results from the map into a destination struct. The map keys -// are matched in the Redis struct fields by the `redis:"field"` tag. -func (cmd *StringStringMapCmd) Scan(dest interface{}) error { - if cmd.err != nil { - return cmd.err - } - - strct, err := hscan.Struct(dest) - if err != nil { - return err - } - - for k, v := range cmd.val { - if err := strct.Scan(k, v); err != nil { - return err - } - } - - return nil -} - -func (cmd *StringStringMapCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make(map[string]string, n/2) - for i := int64(0); i < n; i += 2 { - key, err := rd.ReadString() - if err != nil { - return nil, err - } - - value, err := rd.ReadString() - if err != nil { - return nil, err - } - - cmd.val[key] = value - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type StringIntMapCmd struct { - baseCmd - - val map[string]int64 -} - -var _ Cmder = (*StringIntMapCmd)(nil) - -func NewStringIntMapCmd(ctx context.Context, args ...interface{}) *StringIntMapCmd { - return &StringIntMapCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *StringIntMapCmd) SetVal(val map[string]int64) { - cmd.val = val -} - -func (cmd *StringIntMapCmd) Val() map[string]int64 { - return cmd.val -} - -func (cmd *StringIntMapCmd) Result() (map[string]int64, error) { - return cmd.val, cmd.err -} - -func (cmd *StringIntMapCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *StringIntMapCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make(map[string]int64, n/2) - for i := int64(0); i < n; i += 2 { - key, err := rd.ReadString() - if err != nil { - return nil, err - } - - n, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - - cmd.val[key] = n - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type StringStructMapCmd struct { - baseCmd - - val map[string]struct{} -} - -var _ Cmder = (*StringStructMapCmd)(nil) - -func NewStringStructMapCmd(ctx context.Context, args ...interface{}) *StringStructMapCmd { - return &StringStructMapCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *StringStructMapCmd) SetVal(val map[string]struct{}) { - cmd.val = val -} - -func (cmd *StringStructMapCmd) Val() map[string]struct{} { - return cmd.val -} - -func (cmd *StringStructMapCmd) Result() (map[string]struct{}, error) { - return cmd.val, cmd.err -} - -func (cmd *StringStructMapCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *StringStructMapCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make(map[string]struct{}, n) - for i := int64(0); i < n; i++ { - key, err := rd.ReadString() - if err != nil { - return nil, err - } - cmd.val[key] = struct{}{} - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type XMessage struct { - ID string - Values map[string]interface{} -} - -type XMessageSliceCmd struct { - baseCmd - - val []XMessage -} - -var _ Cmder = (*XMessageSliceCmd)(nil) - -func NewXMessageSliceCmd(ctx context.Context, args ...interface{}) *XMessageSliceCmd { - return &XMessageSliceCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *XMessageSliceCmd) SetVal(val []XMessage) { - cmd.val = val -} - -func (cmd *XMessageSliceCmd) Val() []XMessage { - return cmd.val -} - -func (cmd *XMessageSliceCmd) Result() ([]XMessage, error) { - return cmd.val, cmd.err -} - -func (cmd *XMessageSliceCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XMessageSliceCmd) readReply(rd *proto.Reader) error { - var err error - cmd.val, err = readXMessageSlice(rd) - return err -} - -func readXMessageSlice(rd *proto.Reader) ([]XMessage, error) { - n, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - - msgs := make([]XMessage, n) - for i := 0; i < n; i++ { - var err error - msgs[i], err = readXMessage(rd) - if err != nil { - return nil, err - } - } - return msgs, nil -} - -func readXMessage(rd *proto.Reader) (XMessage, error) { - n, err := rd.ReadArrayLen() - if err != nil { - return XMessage{}, err - } - if n != 2 { - return XMessage{}, fmt.Errorf("got %d, wanted 2", n) - } - - id, err := rd.ReadString() - if err != nil { - return XMessage{}, err - } - - var values map[string]interface{} - - v, err := rd.ReadArrayReply(stringInterfaceMapParser) - if err != nil { - if err != proto.Nil { - return XMessage{}, err - } - } else { - values = v.(map[string]interface{}) - } - - return XMessage{ - ID: id, - Values: values, - }, nil -} - -// stringInterfaceMapParser implements proto.MultiBulkParse. -func stringInterfaceMapParser(rd *proto.Reader, n int64) (interface{}, error) { - m := make(map[string]interface{}, n/2) - for i := int64(0); i < n; i += 2 { - key, err := rd.ReadString() - if err != nil { - return nil, err - } - - value, err := rd.ReadString() - if err != nil { - return nil, err - } - - m[key] = value - } - return m, nil -} - -//------------------------------------------------------------------------------ - -type XStream struct { - Stream string - Messages []XMessage -} - -type XStreamSliceCmd struct { - baseCmd - - val []XStream -} - -var _ Cmder = (*XStreamSliceCmd)(nil) - -func NewXStreamSliceCmd(ctx context.Context, args ...interface{}) *XStreamSliceCmd { - return &XStreamSliceCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *XStreamSliceCmd) SetVal(val []XStream) { - cmd.val = val -} - -func (cmd *XStreamSliceCmd) Val() []XStream { - return cmd.val -} - -func (cmd *XStreamSliceCmd) Result() ([]XStream, error) { - return cmd.val, cmd.err -} - -func (cmd *XStreamSliceCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XStreamSliceCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]XStream, n) - for i := 0; i < len(cmd.val); i++ { - i := i - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - if n != 2 { - return nil, fmt.Errorf("got %d, wanted 2", n) - } - - stream, err := rd.ReadString() - if err != nil { - return nil, err - } - - msgs, err := readXMessageSlice(rd) - if err != nil { - return nil, err - } - - cmd.val[i] = XStream{ - Stream: stream, - Messages: msgs, - } - return nil, nil - }) - if err != nil { - return nil, err - } - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type XPending struct { - Count int64 - Lower string - Higher string - Consumers map[string]int64 -} - -type XPendingCmd struct { - baseCmd - val *XPending -} - -var _ Cmder = (*XPendingCmd)(nil) - -func NewXPendingCmd(ctx context.Context, args ...interface{}) *XPendingCmd { - return &XPendingCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *XPendingCmd) SetVal(val *XPending) { - cmd.val = val -} - -func (cmd *XPendingCmd) Val() *XPending { - return cmd.val -} - -func (cmd *XPendingCmd) Result() (*XPending, error) { - return cmd.val, cmd.err -} - -func (cmd *XPendingCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XPendingCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - if n != 4 { - return nil, fmt.Errorf("got %d, wanted 4", n) - } - - count, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - - lower, err := rd.ReadString() - if err != nil && err != Nil { - return nil, err - } - - higher, err := rd.ReadString() - if err != nil && err != Nil { - return nil, err - } - - cmd.val = &XPending{ - Count: count, - Lower: lower, - Higher: higher, - } - _, err = rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - for i := int64(0); i < n; i++ { - _, err = rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - if n != 2 { - return nil, fmt.Errorf("got %d, wanted 2", n) - } - - consumerName, err := rd.ReadString() - if err != nil { - return nil, err - } - - consumerPending, err := rd.ReadInt() - if err != nil { - return nil, err - } - - if cmd.val.Consumers == nil { - cmd.val.Consumers = make(map[string]int64) - } - cmd.val.Consumers[consumerName] = consumerPending - - return nil, nil - }) - if err != nil { - return nil, err - } - } - return nil, nil - }) - if err != nil && err != Nil { - return nil, err - } - - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type XPendingExt struct { - ID string - Consumer string - Idle time.Duration - RetryCount int64 -} - -type XPendingExtCmd struct { - baseCmd - val []XPendingExt -} - -var _ Cmder = (*XPendingExtCmd)(nil) - -func NewXPendingExtCmd(ctx context.Context, args ...interface{}) *XPendingExtCmd { - return &XPendingExtCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *XPendingExtCmd) SetVal(val []XPendingExt) { - cmd.val = val -} - -func (cmd *XPendingExtCmd) Val() []XPendingExt { - return cmd.val -} - -func (cmd *XPendingExtCmd) Result() ([]XPendingExt, error) { - return cmd.val, cmd.err -} - -func (cmd *XPendingExtCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XPendingExtCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]XPendingExt, 0, n) - for i := int64(0); i < n; i++ { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - if n != 4 { - return nil, fmt.Errorf("got %d, wanted 4", n) - } - - id, err := rd.ReadString() - if err != nil { - return nil, err - } - - consumer, err := rd.ReadString() - if err != nil && err != Nil { - return nil, err - } - - idle, err := rd.ReadIntReply() - if err != nil && err != Nil { - return nil, err - } - - retryCount, err := rd.ReadIntReply() - if err != nil && err != Nil { - return nil, err - } - - cmd.val = append(cmd.val, XPendingExt{ - ID: id, - Consumer: consumer, - Idle: time.Duration(idle) * time.Millisecond, - RetryCount: retryCount, - }) - return nil, nil - }) - if err != nil { - return nil, err - } - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type XAutoClaimCmd struct { - baseCmd - - start string - val []XMessage -} - -var _ Cmder = (*XAutoClaimCmd)(nil) - -func NewXAutoClaimCmd(ctx context.Context, args ...interface{}) *XAutoClaimCmd { - return &XAutoClaimCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *XAutoClaimCmd) SetVal(val []XMessage, start string) { - cmd.val = val - cmd.start = start -} - -func (cmd *XAutoClaimCmd) Val() (messages []XMessage, start string) { - return cmd.val, cmd.start -} - -func (cmd *XAutoClaimCmd) Result() (messages []XMessage, start string, err error) { - return cmd.val, cmd.start, cmd.err -} - -func (cmd *XAutoClaimCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XAutoClaimCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - if n != 2 { - return nil, fmt.Errorf("got %d, wanted 2", n) - } - var err error - - cmd.start, err = rd.ReadString() - if err != nil { - return nil, err - } - - cmd.val, err = readXMessageSlice(rd) - if err != nil { - return nil, err - } - - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type XAutoClaimJustIDCmd struct { - baseCmd - - start string - val []string -} - -var _ Cmder = (*XAutoClaimJustIDCmd)(nil) - -func NewXAutoClaimJustIDCmd(ctx context.Context, args ...interface{}) *XAutoClaimJustIDCmd { - return &XAutoClaimJustIDCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *XAutoClaimJustIDCmd) SetVal(val []string, start string) { - cmd.val = val - cmd.start = start -} - -func (cmd *XAutoClaimJustIDCmd) Val() (ids []string, start string) { - return cmd.val, cmd.start -} - -func (cmd *XAutoClaimJustIDCmd) Result() (ids []string, start string, err error) { - return cmd.val, cmd.start, cmd.err -} - -func (cmd *XAutoClaimJustIDCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XAutoClaimJustIDCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - if n != 2 { - return nil, fmt.Errorf("got %d, wanted 2", n) - } - var err error - - cmd.start, err = rd.ReadString() - if err != nil { - return nil, err - } - - nn, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - - cmd.val = make([]string, nn) - for i := 0; i < nn; i++ { - cmd.val[i], err = rd.ReadString() - if err != nil { - return nil, err - } - } - - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type XInfoConsumersCmd struct { - baseCmd - val []XInfoConsumer -} - -type XInfoConsumer struct { - Name string - Pending int64 - Idle int64 -} - -var _ Cmder = (*XInfoConsumersCmd)(nil) - -func NewXInfoConsumersCmd(ctx context.Context, stream string, group string) *XInfoConsumersCmd { - return &XInfoConsumersCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: []interface{}{"xinfo", "consumers", stream, group}, - }, - } -} - -func (cmd *XInfoConsumersCmd) SetVal(val []XInfoConsumer) { - cmd.val = val -} - -func (cmd *XInfoConsumersCmd) Val() []XInfoConsumer { - return cmd.val -} - -func (cmd *XInfoConsumersCmd) Result() ([]XInfoConsumer, error) { - return cmd.val, cmd.err -} - -func (cmd *XInfoConsumersCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XInfoConsumersCmd) readReply(rd *proto.Reader) error { - n, err := rd.ReadArrayLen() - if err != nil { - return err - } - - cmd.val = make([]XInfoConsumer, n) - - for i := 0; i < n; i++ { - cmd.val[i], err = readXConsumerInfo(rd) - if err != nil { - return err - } - } - - return nil -} - -func readXConsumerInfo(rd *proto.Reader) (XInfoConsumer, error) { - var consumer XInfoConsumer - - n, err := rd.ReadArrayLen() - if err != nil { - return consumer, err - } - if n != 6 { - return consumer, fmt.Errorf("redis: got %d elements in XINFO CONSUMERS reply, wanted 6", n) - } - - for i := 0; i < 3; i++ { - key, err := rd.ReadString() - if err != nil { - return consumer, err - } - - val, err := rd.ReadString() - if err != nil { - return consumer, err - } - - switch key { - case "name": - consumer.Name = val - case "pending": - consumer.Pending, err = strconv.ParseInt(val, 0, 64) - if err != nil { - return consumer, err - } - case "idle": - consumer.Idle, err = strconv.ParseInt(val, 0, 64) - if err != nil { - return consumer, err - } - default: - return consumer, fmt.Errorf("redis: unexpected content %s in XINFO CONSUMERS reply", key) - } - } - - return consumer, nil -} - -//------------------------------------------------------------------------------ - -type XInfoGroupsCmd struct { - baseCmd - val []XInfoGroup -} - -type XInfoGroup struct { - Name string - Consumers int64 - Pending int64 - LastDeliveredID string -} - -var _ Cmder = (*XInfoGroupsCmd)(nil) - -func NewXInfoGroupsCmd(ctx context.Context, stream string) *XInfoGroupsCmd { - return &XInfoGroupsCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: []interface{}{"xinfo", "groups", stream}, - }, - } -} - -func (cmd *XInfoGroupsCmd) SetVal(val []XInfoGroup) { - cmd.val = val -} - -func (cmd *XInfoGroupsCmd) Val() []XInfoGroup { - return cmd.val -} - -func (cmd *XInfoGroupsCmd) Result() ([]XInfoGroup, error) { - return cmd.val, cmd.err -} - -func (cmd *XInfoGroupsCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XInfoGroupsCmd) readReply(rd *proto.Reader) error { - n, err := rd.ReadArrayLen() - if err != nil { - return err - } - - cmd.val = make([]XInfoGroup, n) - - for i := 0; i < n; i++ { - cmd.val[i], err = readXGroupInfo(rd) - if err != nil { - return err - } - } - - return nil -} - -func readXGroupInfo(rd *proto.Reader) (XInfoGroup, error) { - var group XInfoGroup - - n, err := rd.ReadArrayLen() - if err != nil { - return group, err - } - if n != 8 { - return group, fmt.Errorf("redis: got %d elements in XINFO GROUPS reply, wanted 8", n) - } - - for i := 0; i < 4; i++ { - key, err := rd.ReadString() - if err != nil { - return group, err - } - - val, err := rd.ReadString() - if err != nil { - return group, err - } - - switch key { - case "name": - group.Name = val - case "consumers": - group.Consumers, err = strconv.ParseInt(val, 0, 64) - if err != nil { - return group, err - } - case "pending": - group.Pending, err = strconv.ParseInt(val, 0, 64) - if err != nil { - return group, err - } - case "last-delivered-id": - group.LastDeliveredID = val - default: - return group, fmt.Errorf("redis: unexpected content %s in XINFO GROUPS reply", key) - } - } - - return group, nil -} - -//------------------------------------------------------------------------------ - -type XInfoStreamCmd struct { - baseCmd - val *XInfoStream -} - -type XInfoStream struct { - Length int64 - RadixTreeKeys int64 - RadixTreeNodes int64 - Groups int64 - LastGeneratedID string - FirstEntry XMessage - LastEntry XMessage -} - -var _ Cmder = (*XInfoStreamCmd)(nil) - -func NewXInfoStreamCmd(ctx context.Context, stream string) *XInfoStreamCmd { - return &XInfoStreamCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: []interface{}{"xinfo", "stream", stream}, - }, - } -} - -func (cmd *XInfoStreamCmd) SetVal(val *XInfoStream) { - cmd.val = val -} - -func (cmd *XInfoStreamCmd) Val() *XInfoStream { - return cmd.val -} - -func (cmd *XInfoStreamCmd) Result() (*XInfoStream, error) { - return cmd.val, cmd.err -} - -func (cmd *XInfoStreamCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XInfoStreamCmd) readReply(rd *proto.Reader) error { - v, err := rd.ReadReply(xStreamInfoParser) - if err != nil { - return err - } - cmd.val = v.(*XInfoStream) - return nil -} - -func xStreamInfoParser(rd *proto.Reader, n int64) (interface{}, error) { - if n != 14 { - return nil, fmt.Errorf("redis: got %d elements in XINFO STREAM reply,"+ - "wanted 14", n) - } - var info XInfoStream - for i := 0; i < 7; i++ { - key, err := rd.ReadString() - if err != nil { - return nil, err - } - switch key { - case "length": - info.Length, err = rd.ReadIntReply() - case "radix-tree-keys": - info.RadixTreeKeys, err = rd.ReadIntReply() - case "radix-tree-nodes": - info.RadixTreeNodes, err = rd.ReadIntReply() - case "groups": - info.Groups, err = rd.ReadIntReply() - case "last-generated-id": - info.LastGeneratedID, err = rd.ReadString() - case "first-entry": - info.FirstEntry, err = readXMessage(rd) - if err == Nil { - err = nil - } - case "last-entry": - info.LastEntry, err = readXMessage(rd) - if err == Nil { - err = nil - } - default: - return nil, fmt.Errorf("redis: unexpected content %s "+ - "in XINFO STREAM reply", key) - } - if err != nil { - return nil, err - } - } - return &info, nil -} - -//------------------------------------------------------------------------------ - -type XInfoStreamFullCmd struct { - baseCmd - val *XInfoStreamFull -} - -type XInfoStreamFull struct { - Length int64 - RadixTreeKeys int64 - RadixTreeNodes int64 - LastGeneratedID string - Entries []XMessage - Groups []XInfoStreamGroup -} - -type XInfoStreamGroup struct { - Name string - LastDeliveredID string - PelCount int64 - Pending []XInfoStreamGroupPending - Consumers []XInfoStreamConsumer -} - -type XInfoStreamGroupPending struct { - ID string - Consumer string - DeliveryTime time.Time - DeliveryCount int64 -} - -type XInfoStreamConsumer struct { - Name string - SeenTime time.Time - PelCount int64 - Pending []XInfoStreamConsumerPending -} - -type XInfoStreamConsumerPending struct { - ID string - DeliveryTime time.Time - DeliveryCount int64 -} - -var _ Cmder = (*XInfoStreamFullCmd)(nil) - -func NewXInfoStreamFullCmd(ctx context.Context, args ...interface{}) *XInfoStreamFullCmd { - return &XInfoStreamFullCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *XInfoStreamFullCmd) SetVal(val *XInfoStreamFull) { - cmd.val = val -} - -func (cmd *XInfoStreamFullCmd) Val() *XInfoStreamFull { - return cmd.val -} - -func (cmd *XInfoStreamFullCmd) Result() (*XInfoStreamFull, error) { - return cmd.val, cmd.err -} - -func (cmd *XInfoStreamFullCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *XInfoStreamFullCmd) readReply(rd *proto.Reader) error { - n, err := rd.ReadArrayLen() - if err != nil { - return err - } - if n != 12 { - return fmt.Errorf("redis: got %d elements in XINFO STREAM FULL reply,"+ - "wanted 12", n) - } - - cmd.val = &XInfoStreamFull{} - - for i := 0; i < 6; i++ { - key, err := rd.ReadString() - if err != nil { - return err - } - - switch key { - case "length": - cmd.val.Length, err = rd.ReadIntReply() - case "radix-tree-keys": - cmd.val.RadixTreeKeys, err = rd.ReadIntReply() - case "radix-tree-nodes": - cmd.val.RadixTreeNodes, err = rd.ReadIntReply() - case "last-generated-id": - cmd.val.LastGeneratedID, err = rd.ReadString() - case "entries": - cmd.val.Entries, err = readXMessageSlice(rd) - case "groups": - cmd.val.Groups, err = readStreamGroups(rd) - default: - return fmt.Errorf("redis: unexpected content %s "+ - "in XINFO STREAM reply", key) - } - if err != nil { - return err - } - } - return nil -} - -func readStreamGroups(rd *proto.Reader) ([]XInfoStreamGroup, error) { - n, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - groups := make([]XInfoStreamGroup, 0, n) - for i := 0; i < n; i++ { - nn, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if nn != 10 { - return nil, fmt.Errorf("redis: got %d elements in XINFO STREAM FULL reply,"+ - "wanted 10", nn) - } - - group := XInfoStreamGroup{} - - for f := 0; f < 5; f++ { - key, err := rd.ReadString() - if err != nil { - return nil, err - } - - switch key { - case "name": - group.Name, err = rd.ReadString() - case "last-delivered-id": - group.LastDeliveredID, err = rd.ReadString() - case "pel-count": - group.PelCount, err = rd.ReadIntReply() - case "pending": - group.Pending, err = readXInfoStreamGroupPending(rd) - case "consumers": - group.Consumers, err = readXInfoStreamConsumers(rd) - default: - return nil, fmt.Errorf("redis: unexpected content %s "+ - "in XINFO STREAM reply", key) - } - - if err != nil { - return nil, err - } - } - - groups = append(groups, group) - } - - return groups, nil -} - -func readXInfoStreamGroupPending(rd *proto.Reader) ([]XInfoStreamGroupPending, error) { - n, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - - pending := make([]XInfoStreamGroupPending, 0, n) - - for i := 0; i < n; i++ { - nn, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if nn != 4 { - return nil, fmt.Errorf("redis: got %d elements in XINFO STREAM FULL reply,"+ - "wanted 4", nn) - } - - p := XInfoStreamGroupPending{} - - p.ID, err = rd.ReadString() - if err != nil { - return nil, err - } - - p.Consumer, err = rd.ReadString() - if err != nil { - return nil, err - } - - delivery, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - p.DeliveryTime = time.Unix(delivery/1000, delivery%1000*int64(time.Millisecond)) - - p.DeliveryCount, err = rd.ReadIntReply() - if err != nil { - return nil, err - } - - pending = append(pending, p) - } - - return pending, nil -} - -func readXInfoStreamConsumers(rd *proto.Reader) ([]XInfoStreamConsumer, error) { - n, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - - consumers := make([]XInfoStreamConsumer, 0, n) - - for i := 0; i < n; i++ { - nn, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if nn != 8 { - return nil, fmt.Errorf("redis: got %d elements in XINFO STREAM FULL reply,"+ - "wanted 8", nn) - } - - c := XInfoStreamConsumer{} - - for f := 0; f < 4; f++ { - cKey, err := rd.ReadString() - if err != nil { - return nil, err - } - - switch cKey { - case "name": - c.Name, err = rd.ReadString() - case "seen-time": - seen, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - c.SeenTime = time.Unix(seen/1000, seen%1000*int64(time.Millisecond)) - case "pel-count": - c.PelCount, err = rd.ReadIntReply() - case "pending": - pendingNumber, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - - c.Pending = make([]XInfoStreamConsumerPending, 0, pendingNumber) - - for pn := 0; pn < pendingNumber; pn++ { - nn, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if nn != 3 { - return nil, fmt.Errorf("redis: got %d elements in XINFO STREAM reply,"+ - "wanted 3", nn) - } - - p := XInfoStreamConsumerPending{} - - p.ID, err = rd.ReadString() - if err != nil { - return nil, err - } - - delivery, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - p.DeliveryTime = time.Unix(delivery/1000, delivery%1000*int64(time.Millisecond)) - - p.DeliveryCount, err = rd.ReadIntReply() - if err != nil { - return nil, err - } - - c.Pending = append(c.Pending, p) - } - default: - return nil, fmt.Errorf("redis: unexpected content %s "+ - "in XINFO STREAM reply", cKey) - } - if err != nil { - return nil, err - } - } - consumers = append(consumers, c) - } - - return consumers, nil -} - -//------------------------------------------------------------------------------ - -type ZSliceCmd struct { - baseCmd - - val []Z -} - -var _ Cmder = (*ZSliceCmd)(nil) - -func NewZSliceCmd(ctx context.Context, args ...interface{}) *ZSliceCmd { - return &ZSliceCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *ZSliceCmd) SetVal(val []Z) { - cmd.val = val -} - -func (cmd *ZSliceCmd) Val() []Z { - return cmd.val -} - -func (cmd *ZSliceCmd) Result() ([]Z, error) { - return cmd.val, cmd.err -} - -func (cmd *ZSliceCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *ZSliceCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]Z, n/2) - for i := 0; i < len(cmd.val); i++ { - member, err := rd.ReadString() - if err != nil { - return nil, err - } - - score, err := rd.ReadFloatReply() - if err != nil { - return nil, err - } - - cmd.val[i] = Z{ - Member: member, - Score: score, - } - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type ZWithKeyCmd struct { - baseCmd - - val *ZWithKey -} - -var _ Cmder = (*ZWithKeyCmd)(nil) - -func NewZWithKeyCmd(ctx context.Context, args ...interface{}) *ZWithKeyCmd { - return &ZWithKeyCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *ZWithKeyCmd) SetVal(val *ZWithKey) { - cmd.val = val -} - -func (cmd *ZWithKeyCmd) Val() *ZWithKey { - return cmd.val -} - -func (cmd *ZWithKeyCmd) Result() (*ZWithKey, error) { - return cmd.Val(), cmd.Err() -} - -func (cmd *ZWithKeyCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *ZWithKeyCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - if n != 3 { - return nil, fmt.Errorf("got %d elements, expected 3", n) - } - - cmd.val = &ZWithKey{} - var err error - - cmd.val.Key, err = rd.ReadString() - if err != nil { - return nil, err - } - - cmd.val.Member, err = rd.ReadString() - if err != nil { - return nil, err - } - - cmd.val.Score, err = rd.ReadFloatReply() - if err != nil { - return nil, err - } - - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type ScanCmd struct { - baseCmd - - page []string - cursor uint64 - - process cmdable -} - -var _ Cmder = (*ScanCmd)(nil) - -func NewScanCmd(ctx context.Context, process cmdable, args ...interface{}) *ScanCmd { - return &ScanCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - process: process, - } -} - -func (cmd *ScanCmd) SetVal(page []string, cursor uint64) { - cmd.page = page - cmd.cursor = cursor -} - -func (cmd *ScanCmd) Val() (keys []string, cursor uint64) { - return cmd.page, cmd.cursor -} - -func (cmd *ScanCmd) Result() (keys []string, cursor uint64, err error) { - return cmd.page, cmd.cursor, cmd.err -} - -func (cmd *ScanCmd) String() string { - return cmdString(cmd, cmd.page) -} - -func (cmd *ScanCmd) readReply(rd *proto.Reader) (err error) { - cmd.page, cmd.cursor, err = rd.ReadScanReply() - return err -} - -// Iterator creates a new ScanIterator. -func (cmd *ScanCmd) Iterator() *ScanIterator { - return &ScanIterator{ - cmd: cmd, - } -} - -//------------------------------------------------------------------------------ - -type ClusterNode struct { - ID string - Addr string -} - -type ClusterSlot struct { - Start int - End int - Nodes []ClusterNode -} - -type ClusterSlotsCmd struct { - baseCmd - - val []ClusterSlot -} - -var _ Cmder = (*ClusterSlotsCmd)(nil) - -func NewClusterSlotsCmd(ctx context.Context, args ...interface{}) *ClusterSlotsCmd { - return &ClusterSlotsCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *ClusterSlotsCmd) SetVal(val []ClusterSlot) { - cmd.val = val -} - -func (cmd *ClusterSlotsCmd) Val() []ClusterSlot { - return cmd.val -} - -func (cmd *ClusterSlotsCmd) Result() ([]ClusterSlot, error) { - return cmd.Val(), cmd.Err() -} - -func (cmd *ClusterSlotsCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *ClusterSlotsCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]ClusterSlot, n) - for i := 0; i < len(cmd.val); i++ { - n, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if n < 2 { - err := fmt.Errorf("redis: got %d elements in cluster info, expected at least 2", n) - return nil, err - } - - start, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - - end, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - - nodes := make([]ClusterNode, n-2) - for j := 0; j < len(nodes); j++ { - n, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if n != 2 && n != 3 { - err := fmt.Errorf("got %d elements in cluster info address, expected 2 or 3", n) - return nil, err - } - - ip, err := rd.ReadString() - if err != nil { - return nil, err - } - - port, err := rd.ReadString() - if err != nil { - return nil, err - } - - nodes[j].Addr = net.JoinHostPort(ip, port) - - if n == 3 { - id, err := rd.ReadString() - if err != nil { - return nil, err - } - nodes[j].ID = id - } - } - - cmd.val[i] = ClusterSlot{ - Start: int(start), - End: int(end), - Nodes: nodes, - } - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -// GeoLocation is used with GeoAdd to add geospatial location. -type GeoLocation struct { - Name string - Longitude, Latitude, Dist float64 - GeoHash int64 -} - -// GeoRadiusQuery is used with GeoRadius to query geospatial index. -type GeoRadiusQuery struct { - Radius float64 - // Can be m, km, ft, or mi. Default is km. - Unit string - WithCoord bool - WithDist bool - WithGeoHash bool - Count int - // Can be ASC or DESC. Default is no sort order. - Sort string - Store string - StoreDist string -} - -type GeoLocationCmd struct { - baseCmd - - q *GeoRadiusQuery - locations []GeoLocation -} - -var _ Cmder = (*GeoLocationCmd)(nil) - -func NewGeoLocationCmd(ctx context.Context, q *GeoRadiusQuery, args ...interface{}) *GeoLocationCmd { - return &GeoLocationCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: geoLocationArgs(q, args...), - }, - q: q, - } -} - -func geoLocationArgs(q *GeoRadiusQuery, args ...interface{}) []interface{} { - args = append(args, q.Radius) - if q.Unit != "" { - args = append(args, q.Unit) - } else { - args = append(args, "km") - } - if q.WithCoord { - args = append(args, "withcoord") - } - if q.WithDist { - args = append(args, "withdist") - } - if q.WithGeoHash { - args = append(args, "withhash") - } - if q.Count > 0 { - args = append(args, "count", q.Count) - } - if q.Sort != "" { - args = append(args, q.Sort) - } - if q.Store != "" { - args = append(args, "store") - args = append(args, q.Store) - } - if q.StoreDist != "" { - args = append(args, "storedist") - args = append(args, q.StoreDist) - } - return args -} - -func (cmd *GeoLocationCmd) SetVal(locations []GeoLocation) { - cmd.locations = locations -} - -func (cmd *GeoLocationCmd) Val() []GeoLocation { - return cmd.locations -} - -func (cmd *GeoLocationCmd) Result() ([]GeoLocation, error) { - return cmd.locations, cmd.err -} - -func (cmd *GeoLocationCmd) String() string { - return cmdString(cmd, cmd.locations) -} - -func (cmd *GeoLocationCmd) readReply(rd *proto.Reader) error { - v, err := rd.ReadArrayReply(newGeoLocationSliceParser(cmd.q)) - if err != nil { - return err - } - cmd.locations = v.([]GeoLocation) - return nil -} - -func newGeoLocationSliceParser(q *GeoRadiusQuery) proto.MultiBulkParse { - return func(rd *proto.Reader, n int64) (interface{}, error) { - locs := make([]GeoLocation, 0, n) - for i := int64(0); i < n; i++ { - v, err := rd.ReadReply(newGeoLocationParser(q)) - if err != nil { - return nil, err - } - switch vv := v.(type) { - case string: - locs = append(locs, GeoLocation{ - Name: vv, - }) - case *GeoLocation: - // TODO: avoid copying - locs = append(locs, *vv) - default: - return nil, fmt.Errorf("got %T, expected string or *GeoLocation", v) - } - } - return locs, nil - } -} - -func newGeoLocationParser(q *GeoRadiusQuery) proto.MultiBulkParse { - return func(rd *proto.Reader, n int64) (interface{}, error) { - var loc GeoLocation - var err error - - loc.Name, err = rd.ReadString() - if err != nil { - return nil, err - } - if q.WithDist { - loc.Dist, err = rd.ReadFloatReply() - if err != nil { - return nil, err - } - } - if q.WithGeoHash { - loc.GeoHash, err = rd.ReadIntReply() - if err != nil { - return nil, err - } - } - if q.WithCoord { - n, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if n != 2 { - return nil, fmt.Errorf("got %d coordinates, expected 2", n) - } - - loc.Longitude, err = rd.ReadFloatReply() - if err != nil { - return nil, err - } - loc.Latitude, err = rd.ReadFloatReply() - if err != nil { - return nil, err - } - } - - return &loc, nil - } -} - -//------------------------------------------------------------------------------ - -// GeoSearchQuery is used for GEOSearch/GEOSearchStore command query. -type GeoSearchQuery struct { - Member string - - // Latitude and Longitude when using FromLonLat option. - Longitude float64 - Latitude float64 - - // Distance and unit when using ByRadius option. - // Can use m, km, ft, or mi. Default is km. - Radius float64 - RadiusUnit string - - // Height, width and unit when using ByBox option. - // Can be m, km, ft, or mi. Default is km. - BoxWidth float64 - BoxHeight float64 - BoxUnit string - - // Can be ASC or DESC. Default is no sort order. - Sort string - Count int - CountAny bool -} - -type GeoSearchLocationQuery struct { - GeoSearchQuery - - WithCoord bool - WithDist bool - WithHash bool -} - -type GeoSearchStoreQuery struct { - GeoSearchQuery - - // When using the StoreDist option, the command stores the items in a - // sorted set populated with their distance from the center of the circle or box, - // as a floating-point number, in the same unit specified for that shape. - StoreDist bool -} - -func geoSearchLocationArgs(q *GeoSearchLocationQuery, args []interface{}) []interface{} { - args = geoSearchArgs(&q.GeoSearchQuery, args) - - if q.WithCoord { - args = append(args, "withcoord") - } - if q.WithDist { - args = append(args, "withdist") - } - if q.WithHash { - args = append(args, "withhash") - } - - return args -} - -func geoSearchArgs(q *GeoSearchQuery, args []interface{}) []interface{} { - if q.Member != "" { - args = append(args, "frommember", q.Member) - } else { - args = append(args, "fromlonlat", q.Longitude, q.Latitude) - } - - if q.Radius > 0 { - if q.RadiusUnit == "" { - q.RadiusUnit = "km" - } - args = append(args, "byradius", q.Radius, q.RadiusUnit) - } else { - if q.BoxUnit == "" { - q.BoxUnit = "km" - } - args = append(args, "bybox", q.BoxWidth, q.BoxHeight, q.BoxUnit) - } - - if q.Sort != "" { - args = append(args, q.Sort) - } - - if q.Count > 0 { - args = append(args, "count", q.Count) - if q.CountAny { - args = append(args, "any") - } - } - - return args -} - -type GeoSearchLocationCmd struct { - baseCmd - - opt *GeoSearchLocationQuery - val []GeoLocation -} - -var _ Cmder = (*GeoSearchLocationCmd)(nil) - -func NewGeoSearchLocationCmd( - ctx context.Context, opt *GeoSearchLocationQuery, args ...interface{}, -) *GeoSearchLocationCmd { - return &GeoSearchLocationCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - opt: opt, - } -} - -func (cmd *GeoSearchLocationCmd) SetVal(val []GeoLocation) { - cmd.val = val -} - -func (cmd *GeoSearchLocationCmd) Val() []GeoLocation { - return cmd.val -} - -func (cmd *GeoSearchLocationCmd) Result() ([]GeoLocation, error) { - return cmd.val, cmd.err -} - -func (cmd *GeoSearchLocationCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *GeoSearchLocationCmd) readReply(rd *proto.Reader) error { - n, err := rd.ReadArrayLen() - if err != nil { - return err - } - - cmd.val = make([]GeoLocation, n) - for i := 0; i < n; i++ { - _, err = rd.ReadArrayLen() - if err != nil { - return err - } - - var loc GeoLocation - - loc.Name, err = rd.ReadString() - if err != nil { - return err - } - if cmd.opt.WithDist { - loc.Dist, err = rd.ReadFloatReply() - if err != nil { - return err - } - } - if cmd.opt.WithHash { - loc.GeoHash, err = rd.ReadIntReply() - if err != nil { - return err - } - } - if cmd.opt.WithCoord { - nn, err := rd.ReadArrayLen() - if err != nil { - return err - } - if nn != 2 { - return fmt.Errorf("got %d coordinates, expected 2", nn) - } - - loc.Longitude, err = rd.ReadFloatReply() - if err != nil { - return err - } - loc.Latitude, err = rd.ReadFloatReply() - if err != nil { - return err - } - } - - cmd.val[i] = loc - } - - return nil -} - -//------------------------------------------------------------------------------ - -type GeoPos struct { - Longitude, Latitude float64 -} - -type GeoPosCmd struct { - baseCmd - - val []*GeoPos -} - -var _ Cmder = (*GeoPosCmd)(nil) - -func NewGeoPosCmd(ctx context.Context, args ...interface{}) *GeoPosCmd { - return &GeoPosCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *GeoPosCmd) SetVal(val []*GeoPos) { - cmd.val = val -} - -func (cmd *GeoPosCmd) Val() []*GeoPos { - return cmd.val -} - -func (cmd *GeoPosCmd) Result() ([]*GeoPos, error) { - return cmd.Val(), cmd.Err() -} - -func (cmd *GeoPosCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *GeoPosCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]*GeoPos, n) - for i := 0; i < len(cmd.val); i++ { - i := i - _, err := rd.ReadReply(func(rd *proto.Reader, n int64) (interface{}, error) { - longitude, err := rd.ReadFloatReply() - if err != nil { - return nil, err - } - - latitude, err := rd.ReadFloatReply() - if err != nil { - return nil, err - } - - cmd.val[i] = &GeoPos{ - Longitude: longitude, - Latitude: latitude, - } - return nil, nil - }) - if err != nil { - if err == Nil { - cmd.val[i] = nil - continue - } - return nil, err - } - } - return nil, nil - }) - return err -} - -//------------------------------------------------------------------------------ - -type CommandInfo struct { - Name string - Arity int8 - Flags []string - ACLFlags []string - FirstKeyPos int8 - LastKeyPos int8 - StepCount int8 - ReadOnly bool -} - -type CommandsInfoCmd struct { - baseCmd - - val map[string]*CommandInfo -} - -var _ Cmder = (*CommandsInfoCmd)(nil) - -func NewCommandsInfoCmd(ctx context.Context, args ...interface{}) *CommandsInfoCmd { - return &CommandsInfoCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *CommandsInfoCmd) SetVal(val map[string]*CommandInfo) { - cmd.val = val -} - -func (cmd *CommandsInfoCmd) Val() map[string]*CommandInfo { - return cmd.val -} - -func (cmd *CommandsInfoCmd) Result() (map[string]*CommandInfo, error) { - return cmd.Val(), cmd.Err() -} - -func (cmd *CommandsInfoCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *CommandsInfoCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make(map[string]*CommandInfo, n) - for i := int64(0); i < n; i++ { - v, err := rd.ReadReply(commandInfoParser) - if err != nil { - return nil, err - } - vv := v.(*CommandInfo) - cmd.val[vv.Name] = vv - } - return nil, nil - }) - return err -} - -func commandInfoParser(rd *proto.Reader, n int64) (interface{}, error) { - const numArgRedis5 = 6 - const numArgRedis6 = 7 - - switch n { - case numArgRedis5, numArgRedis6: - // continue - default: - return nil, fmt.Errorf("redis: got %d elements in COMMAND reply, wanted 7", n) - } - - var cmd CommandInfo - var err error - - cmd.Name, err = rd.ReadString() - if err != nil { - return nil, err - } - - arity, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - cmd.Arity = int8(arity) - - _, err = rd.ReadReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.Flags = make([]string, n) - for i := 0; i < len(cmd.Flags); i++ { - switch s, err := rd.ReadString(); { - case err == Nil: - cmd.Flags[i] = "" - case err != nil: - return nil, err - default: - cmd.Flags[i] = s - } - } - return nil, nil - }) - if err != nil { - return nil, err - } - - firstKeyPos, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - cmd.FirstKeyPos = int8(firstKeyPos) - - lastKeyPos, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - cmd.LastKeyPos = int8(lastKeyPos) - - stepCount, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - cmd.StepCount = int8(stepCount) - - for _, flag := range cmd.Flags { - if flag == "readonly" { - cmd.ReadOnly = true - break - } - } - - if n == numArgRedis5 { - return &cmd, nil - } - - _, err = rd.ReadReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.ACLFlags = make([]string, n) - for i := 0; i < len(cmd.ACLFlags); i++ { - switch s, err := rd.ReadString(); { - case err == Nil: - cmd.ACLFlags[i] = "" - case err != nil: - return nil, err - default: - cmd.ACLFlags[i] = s - } - } - return nil, nil - }) - if err != nil { - return nil, err - } - - return &cmd, nil -} - -//------------------------------------------------------------------------------ - -type cmdsInfoCache struct { - fn func(ctx context.Context) (map[string]*CommandInfo, error) - - once internal.Once - cmds map[string]*CommandInfo -} - -func newCmdsInfoCache(fn func(ctx context.Context) (map[string]*CommandInfo, error)) *cmdsInfoCache { - return &cmdsInfoCache{ - fn: fn, - } -} - -func (c *cmdsInfoCache) Get(ctx context.Context) (map[string]*CommandInfo, error) { - err := c.once.Do(func() error { - cmds, err := c.fn(ctx) - if err != nil { - return err - } - - // Extensions have cmd names in upper case. Convert them to lower case. - for k, v := range cmds { - lower := internal.ToLower(k) - if lower != k { - cmds[lower] = v - } - } - - c.cmds = cmds - return nil - }) - return c.cmds, err -} - -//------------------------------------------------------------------------------ - -type SlowLog struct { - ID int64 - Time time.Time - Duration time.Duration - Args []string - // These are also optional fields emitted only by Redis 4.0 or greater: - // https://redis.io/commands/slowlog#output-format - ClientAddr string - ClientName string -} - -type SlowLogCmd struct { - baseCmd - - val []SlowLog -} - -var _ Cmder = (*SlowLogCmd)(nil) - -func NewSlowLogCmd(ctx context.Context, args ...interface{}) *SlowLogCmd { - return &SlowLogCmd{ - baseCmd: baseCmd{ - ctx: ctx, - args: args, - }, - } -} - -func (cmd *SlowLogCmd) SetVal(val []SlowLog) { - cmd.val = val -} - -func (cmd *SlowLogCmd) Val() []SlowLog { - return cmd.val -} - -func (cmd *SlowLogCmd) Result() ([]SlowLog, error) { - return cmd.Val(), cmd.Err() -} - -func (cmd *SlowLogCmd) String() string { - return cmdString(cmd, cmd.val) -} - -func (cmd *SlowLogCmd) readReply(rd *proto.Reader) error { - _, err := rd.ReadArrayReply(func(rd *proto.Reader, n int64) (interface{}, error) { - cmd.val = make([]SlowLog, n) - for i := 0; i < len(cmd.val); i++ { - n, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if n < 4 { - err := fmt.Errorf("redis: got %d elements in slowlog get, expected at least 4", n) - return nil, err - } - - id, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - - createdAt, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - createdAtTime := time.Unix(createdAt, 0) - - costs, err := rd.ReadIntReply() - if err != nil { - return nil, err - } - costsDuration := time.Duration(costs) * time.Microsecond - - cmdLen, err := rd.ReadArrayLen() - if err != nil { - return nil, err - } - if cmdLen < 1 { - err := fmt.Errorf("redis: got %d elements commands reply in slowlog get, expected at least 1", cmdLen) - return nil, err - } - - cmdString := make([]string, cmdLen) - for i := 0; i < cmdLen; i++ { - cmdString[i], err = rd.ReadString() - if err != nil { - return nil, err - } - } - - var address, name string - for i := 4; i < n; i++ { - str, err := rd.ReadString() - if err != nil { - return nil, err - } - if i == 4 { - address = str - } else if i == 5 { - name = str - } - } - - cmd.val[i] = SlowLog{ - ID: id, - Time: createdAtTime, - Duration: costsDuration, - Args: cmdString, - ClientAddr: address, - ClientName: name, - } - } - return nil, nil - }) - return err -} diff --git a/vendor/github.com/go-redis/redis/v8/commands.go b/vendor/github.com/go-redis/redis/v8/commands.go deleted file mode 100644 index bbfe089df166..000000000000 --- a/vendor/github.com/go-redis/redis/v8/commands.go +++ /dev/null @@ -1,3475 +0,0 @@ -package redis - -import ( - "context" - "errors" - "io" - "time" - - "github.com/go-redis/redis/v8/internal" -) - -// KeepTTL is a Redis KEEPTTL option to keep existing TTL, it requires your redis-server version >= 6.0, -// otherwise you will receive an error: (error) ERR syntax error. -// For example: -// -// rdb.Set(ctx, key, value, redis.KeepTTL) -const KeepTTL = -1 - -func usePrecise(dur time.Duration) bool { - return dur < time.Second || dur%time.Second != 0 -} - -func formatMs(ctx context.Context, dur time.Duration) int64 { - if dur > 0 && dur < time.Millisecond { - internal.Logger.Printf( - ctx, - "specified duration is %s, but minimal supported value is %s - truncating to 1ms", - dur, time.Millisecond, - ) - return 1 - } - return int64(dur / time.Millisecond) -} - -func formatSec(ctx context.Context, dur time.Duration) int64 { - if dur > 0 && dur < time.Second { - internal.Logger.Printf( - ctx, - "specified duration is %s, but minimal supported value is %s - truncating to 1s", - dur, time.Second, - ) - return 1 - } - return int64(dur / time.Second) -} - -func appendArgs(dst, src []interface{}) []interface{} { - if len(src) == 1 { - return appendArg(dst, src[0]) - } - - dst = append(dst, src...) - return dst -} - -func appendArg(dst []interface{}, arg interface{}) []interface{} { - switch arg := arg.(type) { - case []string: - for _, s := range arg { - dst = append(dst, s) - } - return dst - case []interface{}: - dst = append(dst, arg...) - return dst - case map[string]interface{}: - for k, v := range arg { - dst = append(dst, k, v) - } - return dst - case map[string]string: - for k, v := range arg { - dst = append(dst, k, v) - } - return dst - default: - return append(dst, arg) - } -} - -type Cmdable interface { - Pipeline() Pipeliner - Pipelined(ctx context.Context, fn func(Pipeliner) error) ([]Cmder, error) - - TxPipelined(ctx context.Context, fn func(Pipeliner) error) ([]Cmder, error) - TxPipeline() Pipeliner - - Command(ctx context.Context) *CommandsInfoCmd - ClientGetName(ctx context.Context) *StringCmd - Echo(ctx context.Context, message interface{}) *StringCmd - Ping(ctx context.Context) *StatusCmd - Quit(ctx context.Context) *StatusCmd - Del(ctx context.Context, keys ...string) *IntCmd - Unlink(ctx context.Context, keys ...string) *IntCmd - Dump(ctx context.Context, key string) *StringCmd - Exists(ctx context.Context, keys ...string) *IntCmd - Expire(ctx context.Context, key string, expiration time.Duration) *BoolCmd - ExpireAt(ctx context.Context, key string, tm time.Time) *BoolCmd - ExpireNX(ctx context.Context, key string, expiration time.Duration) *BoolCmd - ExpireXX(ctx context.Context, key string, expiration time.Duration) *BoolCmd - ExpireGT(ctx context.Context, key string, expiration time.Duration) *BoolCmd - ExpireLT(ctx context.Context, key string, expiration time.Duration) *BoolCmd - Keys(ctx context.Context, pattern string) *StringSliceCmd - Migrate(ctx context.Context, host, port, key string, db int, timeout time.Duration) *StatusCmd - Move(ctx context.Context, key string, db int) *BoolCmd - ObjectRefCount(ctx context.Context, key string) *IntCmd - ObjectEncoding(ctx context.Context, key string) *StringCmd - ObjectIdleTime(ctx context.Context, key string) *DurationCmd - Persist(ctx context.Context, key string) *BoolCmd - PExpire(ctx context.Context, key string, expiration time.Duration) *BoolCmd - PExpireAt(ctx context.Context, key string, tm time.Time) *BoolCmd - PTTL(ctx context.Context, key string) *DurationCmd - RandomKey(ctx context.Context) *StringCmd - Rename(ctx context.Context, key, newkey string) *StatusCmd - RenameNX(ctx context.Context, key, newkey string) *BoolCmd - Restore(ctx context.Context, key string, ttl time.Duration, value string) *StatusCmd - RestoreReplace(ctx context.Context, key string, ttl time.Duration, value string) *StatusCmd - Sort(ctx context.Context, key string, sort *Sort) *StringSliceCmd - SortStore(ctx context.Context, key, store string, sort *Sort) *IntCmd - SortInterfaces(ctx context.Context, key string, sort *Sort) *SliceCmd - Touch(ctx context.Context, keys ...string) *IntCmd - TTL(ctx context.Context, key string) *DurationCmd - Type(ctx context.Context, key string) *StatusCmd - Append(ctx context.Context, key, value string) *IntCmd - Decr(ctx context.Context, key string) *IntCmd - DecrBy(ctx context.Context, key string, decrement int64) *IntCmd - Get(ctx context.Context, key string) *StringCmd - GetRange(ctx context.Context, key string, start, end int64) *StringCmd - GetSet(ctx context.Context, key string, value interface{}) *StringCmd - GetEx(ctx context.Context, key string, expiration time.Duration) *StringCmd - GetDel(ctx context.Context, key string) *StringCmd - Incr(ctx context.Context, key string) *IntCmd - IncrBy(ctx context.Context, key string, value int64) *IntCmd - IncrByFloat(ctx context.Context, key string, value float64) *FloatCmd - MGet(ctx context.Context, keys ...string) *SliceCmd - MSet(ctx context.Context, values ...interface{}) *StatusCmd - MSetNX(ctx context.Context, values ...interface{}) *BoolCmd - Set(ctx context.Context, key string, value interface{}, expiration time.Duration) *StatusCmd - SetArgs(ctx context.Context, key string, value interface{}, a SetArgs) *StatusCmd - // TODO: rename to SetEx - SetEX(ctx context.Context, key string, value interface{}, expiration time.Duration) *StatusCmd - SetNX(ctx context.Context, key string, value interface{}, expiration time.Duration) *BoolCmd - SetXX(ctx context.Context, key string, value interface{}, expiration time.Duration) *BoolCmd - SetRange(ctx context.Context, key string, offset int64, value string) *IntCmd - StrLen(ctx context.Context, key string) *IntCmd - Copy(ctx context.Context, sourceKey string, destKey string, db int, replace bool) *IntCmd - - GetBit(ctx context.Context, key string, offset int64) *IntCmd - SetBit(ctx context.Context, key string, offset int64, value int) *IntCmd - BitCount(ctx context.Context, key string, bitCount *BitCount) *IntCmd - BitOpAnd(ctx context.Context, destKey string, keys ...string) *IntCmd - BitOpOr(ctx context.Context, destKey string, keys ...string) *IntCmd - BitOpXor(ctx context.Context, destKey string, keys ...string) *IntCmd - BitOpNot(ctx context.Context, destKey string, key string) *IntCmd - BitPos(ctx context.Context, key string, bit int64, pos ...int64) *IntCmd - BitField(ctx context.Context, key string, args ...interface{}) *IntSliceCmd - - Scan(ctx context.Context, cursor uint64, match string, count int64) *ScanCmd - ScanType(ctx context.Context, cursor uint64, match string, count int64, keyType string) *ScanCmd - SScan(ctx context.Context, key string, cursor uint64, match string, count int64) *ScanCmd - HScan(ctx context.Context, key string, cursor uint64, match string, count int64) *ScanCmd - ZScan(ctx context.Context, key string, cursor uint64, match string, count int64) *ScanCmd - - HDel(ctx context.Context, key string, fields ...string) *IntCmd - HExists(ctx context.Context, key, field string) *BoolCmd - HGet(ctx context.Context, key, field string) *StringCmd - HGetAll(ctx context.Context, key string) *StringStringMapCmd - HIncrBy(ctx context.Context, key, field string, incr int64) *IntCmd - HIncrByFloat(ctx context.Context, key, field string, incr float64) *FloatCmd - HKeys(ctx context.Context, key string) *StringSliceCmd - HLen(ctx context.Context, key string) *IntCmd - HMGet(ctx context.Context, key string, fields ...string) *SliceCmd - HSet(ctx context.Context, key string, values ...interface{}) *IntCmd - HMSet(ctx context.Context, key string, values ...interface{}) *BoolCmd - HSetNX(ctx context.Context, key, field string, value interface{}) *BoolCmd - HVals(ctx context.Context, key string) *StringSliceCmd - HRandField(ctx context.Context, key string, count int, withValues bool) *StringSliceCmd - - BLPop(ctx context.Context, timeout time.Duration, keys ...string) *StringSliceCmd - BRPop(ctx context.Context, timeout time.Duration, keys ...string) *StringSliceCmd - BRPopLPush(ctx context.Context, source, destination string, timeout time.Duration) *StringCmd - LIndex(ctx context.Context, key string, index int64) *StringCmd - LInsert(ctx context.Context, key, op string, pivot, value interface{}) *IntCmd - LInsertBefore(ctx context.Context, key string, pivot, value interface{}) *IntCmd - LInsertAfter(ctx context.Context, key string, pivot, value interface{}) *IntCmd - LLen(ctx context.Context, key string) *IntCmd - LPop(ctx context.Context, key string) *StringCmd - LPopCount(ctx context.Context, key string, count int) *StringSliceCmd - LPos(ctx context.Context, key string, value string, args LPosArgs) *IntCmd - LPosCount(ctx context.Context, key string, value string, count int64, args LPosArgs) *IntSliceCmd - LPush(ctx context.Context, key string, values ...interface{}) *IntCmd - LPushX(ctx context.Context, key string, values ...interface{}) *IntCmd - LRange(ctx context.Context, key string, start, stop int64) *StringSliceCmd - LRem(ctx context.Context, key string, count int64, value interface{}) *IntCmd - LSet(ctx context.Context, key string, index int64, value interface{}) *StatusCmd - LTrim(ctx context.Context, key string, start, stop int64) *StatusCmd - RPop(ctx context.Context, key string) *StringCmd - RPopCount(ctx context.Context, key string, count int) *StringSliceCmd - RPopLPush(ctx context.Context, source, destination string) *StringCmd - RPush(ctx context.Context, key string, values ...interface{}) *IntCmd - RPushX(ctx context.Context, key string, values ...interface{}) *IntCmd - LMove(ctx context.Context, source, destination, srcpos, destpos string) *StringCmd - BLMove(ctx context.Context, source, destination, srcpos, destpos string, timeout time.Duration) *StringCmd - - SAdd(ctx context.Context, key string, members ...interface{}) *IntCmd - SCard(ctx context.Context, key string) *IntCmd - SDiff(ctx context.Context, keys ...string) *StringSliceCmd - SDiffStore(ctx context.Context, destination string, keys ...string) *IntCmd - SInter(ctx context.Context, keys ...string) *StringSliceCmd - SInterStore(ctx context.Context, destination string, keys ...string) *IntCmd - SIsMember(ctx context.Context, key string, member interface{}) *BoolCmd - SMIsMember(ctx context.Context, key string, members ...interface{}) *BoolSliceCmd - SMembers(ctx context.Context, key string) *StringSliceCmd - SMembersMap(ctx context.Context, key string) *StringStructMapCmd - SMove(ctx context.Context, source, destination string, member interface{}) *BoolCmd - SPop(ctx context.Context, key string) *StringCmd - SPopN(ctx context.Context, key string, count int64) *StringSliceCmd - SRandMember(ctx context.Context, key string) *StringCmd - SRandMemberN(ctx context.Context, key string, count int64) *StringSliceCmd - SRem(ctx context.Context, key string, members ...interface{}) *IntCmd - SUnion(ctx context.Context, keys ...string) *StringSliceCmd - SUnionStore(ctx context.Context, destination string, keys ...string) *IntCmd - - XAdd(ctx context.Context, a *XAddArgs) *StringCmd - XDel(ctx context.Context, stream string, ids ...string) *IntCmd - XLen(ctx context.Context, stream string) *IntCmd - XRange(ctx context.Context, stream, start, stop string) *XMessageSliceCmd - XRangeN(ctx context.Context, stream, start, stop string, count int64) *XMessageSliceCmd - XRevRange(ctx context.Context, stream string, start, stop string) *XMessageSliceCmd - XRevRangeN(ctx context.Context, stream string, start, stop string, count int64) *XMessageSliceCmd - XRead(ctx context.Context, a *XReadArgs) *XStreamSliceCmd - XReadStreams(ctx context.Context, streams ...string) *XStreamSliceCmd - XGroupCreate(ctx context.Context, stream, group, start string) *StatusCmd - XGroupCreateMkStream(ctx context.Context, stream, group, start string) *StatusCmd - XGroupSetID(ctx context.Context, stream, group, start string) *StatusCmd - XGroupDestroy(ctx context.Context, stream, group string) *IntCmd - XGroupCreateConsumer(ctx context.Context, stream, group, consumer string) *IntCmd - XGroupDelConsumer(ctx context.Context, stream, group, consumer string) *IntCmd - XReadGroup(ctx context.Context, a *XReadGroupArgs) *XStreamSliceCmd - XAck(ctx context.Context, stream, group string, ids ...string) *IntCmd - XPending(ctx context.Context, stream, group string) *XPendingCmd - XPendingExt(ctx context.Context, a *XPendingExtArgs) *XPendingExtCmd - XClaim(ctx context.Context, a *XClaimArgs) *XMessageSliceCmd - XClaimJustID(ctx context.Context, a *XClaimArgs) *StringSliceCmd - XAutoClaim(ctx context.Context, a *XAutoClaimArgs) *XAutoClaimCmd - XAutoClaimJustID(ctx context.Context, a *XAutoClaimArgs) *XAutoClaimJustIDCmd - - // TODO: XTrim and XTrimApprox remove in v9. - XTrim(ctx context.Context, key string, maxLen int64) *IntCmd - XTrimApprox(ctx context.Context, key string, maxLen int64) *IntCmd - XTrimMaxLen(ctx context.Context, key string, maxLen int64) *IntCmd - XTrimMaxLenApprox(ctx context.Context, key string, maxLen, limit int64) *IntCmd - XTrimMinID(ctx context.Context, key string, minID string) *IntCmd - XTrimMinIDApprox(ctx context.Context, key string, minID string, limit int64) *IntCmd - XInfoGroups(ctx context.Context, key string) *XInfoGroupsCmd - XInfoStream(ctx context.Context, key string) *XInfoStreamCmd - XInfoStreamFull(ctx context.Context, key string, count int) *XInfoStreamFullCmd - XInfoConsumers(ctx context.Context, key string, group string) *XInfoConsumersCmd - - BZPopMax(ctx context.Context, timeout time.Duration, keys ...string) *ZWithKeyCmd - BZPopMin(ctx context.Context, timeout time.Duration, keys ...string) *ZWithKeyCmd - - // TODO: remove - // ZAddCh - // ZIncr - // ZAddNXCh - // ZAddXXCh - // ZIncrNX - // ZIncrXX - // in v9. - // use ZAddArgs and ZAddArgsIncr. - - ZAdd(ctx context.Context, key string, members ...*Z) *IntCmd - ZAddNX(ctx context.Context, key string, members ...*Z) *IntCmd - ZAddXX(ctx context.Context, key string, members ...*Z) *IntCmd - ZAddCh(ctx context.Context, key string, members ...*Z) *IntCmd - ZAddNXCh(ctx context.Context, key string, members ...*Z) *IntCmd - ZAddXXCh(ctx context.Context, key string, members ...*Z) *IntCmd - ZAddArgs(ctx context.Context, key string, args ZAddArgs) *IntCmd - ZAddArgsIncr(ctx context.Context, key string, args ZAddArgs) *FloatCmd - ZIncr(ctx context.Context, key string, member *Z) *FloatCmd - ZIncrNX(ctx context.Context, key string, member *Z) *FloatCmd - ZIncrXX(ctx context.Context, key string, member *Z) *FloatCmd - ZCard(ctx context.Context, key string) *IntCmd - ZCount(ctx context.Context, key, min, max string) *IntCmd - ZLexCount(ctx context.Context, key, min, max string) *IntCmd - ZIncrBy(ctx context.Context, key string, increment float64, member string) *FloatCmd - ZInter(ctx context.Context, store *ZStore) *StringSliceCmd - ZInterWithScores(ctx context.Context, store *ZStore) *ZSliceCmd - ZInterStore(ctx context.Context, destination string, store *ZStore) *IntCmd - ZMScore(ctx context.Context, key string, members ...string) *FloatSliceCmd - ZPopMax(ctx context.Context, key string, count ...int64) *ZSliceCmd - ZPopMin(ctx context.Context, key string, count ...int64) *ZSliceCmd - ZRange(ctx context.Context, key string, start, stop int64) *StringSliceCmd - ZRangeWithScores(ctx context.Context, key string, start, stop int64) *ZSliceCmd - ZRangeByScore(ctx context.Context, key string, opt *ZRangeBy) *StringSliceCmd - ZRangeByLex(ctx context.Context, key string, opt *ZRangeBy) *StringSliceCmd - ZRangeByScoreWithScores(ctx context.Context, key string, opt *ZRangeBy) *ZSliceCmd - ZRangeArgs(ctx context.Context, z ZRangeArgs) *StringSliceCmd - ZRangeArgsWithScores(ctx context.Context, z ZRangeArgs) *ZSliceCmd - ZRangeStore(ctx context.Context, dst string, z ZRangeArgs) *IntCmd - ZRank(ctx context.Context, key, member string) *IntCmd - ZRem(ctx context.Context, key string, members ...interface{}) *IntCmd - ZRemRangeByRank(ctx context.Context, key string, start, stop int64) *IntCmd - ZRemRangeByScore(ctx context.Context, key, min, max string) *IntCmd - ZRemRangeByLex(ctx context.Context, key, min, max string) *IntCmd - ZRevRange(ctx context.Context, key string, start, stop int64) *StringSliceCmd - ZRevRangeWithScores(ctx context.Context, key string, start, stop int64) *ZSliceCmd - ZRevRangeByScore(ctx context.Context, key string, opt *ZRangeBy) *StringSliceCmd - ZRevRangeByLex(ctx context.Context, key string, opt *ZRangeBy) *StringSliceCmd - ZRevRangeByScoreWithScores(ctx context.Context, key string, opt *ZRangeBy) *ZSliceCmd - ZRevRank(ctx context.Context, key, member string) *IntCmd - ZScore(ctx context.Context, key, member string) *FloatCmd - ZUnionStore(ctx context.Context, dest string, store *ZStore) *IntCmd - ZUnion(ctx context.Context, store ZStore) *StringSliceCmd - ZUnionWithScores(ctx context.Context, store ZStore) *ZSliceCmd - ZRandMember(ctx context.Context, key string, count int, withScores bool) *StringSliceCmd - ZDiff(ctx context.Context, keys ...string) *StringSliceCmd - ZDiffWithScores(ctx context.Context, keys ...string) *ZSliceCmd - ZDiffStore(ctx context.Context, destination string, keys ...string) *IntCmd - - PFAdd(ctx context.Context, key string, els ...interface{}) *IntCmd - PFCount(ctx context.Context, keys ...string) *IntCmd - PFMerge(ctx context.Context, dest string, keys ...string) *StatusCmd - - BgRewriteAOF(ctx context.Context) *StatusCmd - BgSave(ctx context.Context) *StatusCmd - ClientKill(ctx context.Context, ipPort string) *StatusCmd - ClientKillByFilter(ctx context.Context, keys ...string) *IntCmd - ClientList(ctx context.Context) *StringCmd - ClientPause(ctx context.Context, dur time.Duration) *BoolCmd - ClientID(ctx context.Context) *IntCmd - ConfigGet(ctx context.Context, parameter string) *SliceCmd - ConfigResetStat(ctx context.Context) *StatusCmd - ConfigSet(ctx context.Context, parameter, value string) *StatusCmd - ConfigRewrite(ctx context.Context) *StatusCmd - DBSize(ctx context.Context) *IntCmd - FlushAll(ctx context.Context) *StatusCmd - FlushAllAsync(ctx context.Context) *StatusCmd - FlushDB(ctx context.Context) *StatusCmd - FlushDBAsync(ctx context.Context) *StatusCmd - Info(ctx context.Context, section ...string) *StringCmd - LastSave(ctx context.Context) *IntCmd - Save(ctx context.Context) *StatusCmd - Shutdown(ctx context.Context) *StatusCmd - ShutdownSave(ctx context.Context) *StatusCmd - ShutdownNoSave(ctx context.Context) *StatusCmd - SlaveOf(ctx context.Context, host, port string) *StatusCmd - Time(ctx context.Context) *TimeCmd - DebugObject(ctx context.Context, key string) *StringCmd - ReadOnly(ctx context.Context) *StatusCmd - ReadWrite(ctx context.Context) *StatusCmd - MemoryUsage(ctx context.Context, key string, samples ...int) *IntCmd - - Eval(ctx context.Context, script string, keys []string, args ...interface{}) *Cmd - EvalSha(ctx context.Context, sha1 string, keys []string, args ...interface{}) *Cmd - ScriptExists(ctx context.Context, hashes ...string) *BoolSliceCmd - ScriptFlush(ctx context.Context) *StatusCmd - ScriptKill(ctx context.Context) *StatusCmd - ScriptLoad(ctx context.Context, script string) *StringCmd - - Publish(ctx context.Context, channel string, message interface{}) *IntCmd - PubSubChannels(ctx context.Context, pattern string) *StringSliceCmd - PubSubNumSub(ctx context.Context, channels ...string) *StringIntMapCmd - PubSubNumPat(ctx context.Context) *IntCmd - - ClusterSlots(ctx context.Context) *ClusterSlotsCmd - ClusterNodes(ctx context.Context) *StringCmd - ClusterMeet(ctx context.Context, host, port string) *StatusCmd - ClusterForget(ctx context.Context, nodeID string) *StatusCmd - ClusterReplicate(ctx context.Context, nodeID string) *StatusCmd - ClusterResetSoft(ctx context.Context) *StatusCmd - ClusterResetHard(ctx context.Context) *StatusCmd - ClusterInfo(ctx context.Context) *StringCmd - ClusterKeySlot(ctx context.Context, key string) *IntCmd - ClusterGetKeysInSlot(ctx context.Context, slot int, count int) *StringSliceCmd - ClusterCountFailureReports(ctx context.Context, nodeID string) *IntCmd - ClusterCountKeysInSlot(ctx context.Context, slot int) *IntCmd - ClusterDelSlots(ctx context.Context, slots ...int) *StatusCmd - ClusterDelSlotsRange(ctx context.Context, min, max int) *StatusCmd - ClusterSaveConfig(ctx context.Context) *StatusCmd - ClusterSlaves(ctx context.Context, nodeID string) *StringSliceCmd - ClusterFailover(ctx context.Context) *StatusCmd - ClusterAddSlots(ctx context.Context, slots ...int) *StatusCmd - ClusterAddSlotsRange(ctx context.Context, min, max int) *StatusCmd - - GeoAdd(ctx context.Context, key string, geoLocation ...*GeoLocation) *IntCmd - GeoPos(ctx context.Context, key string, members ...string) *GeoPosCmd - GeoRadius(ctx context.Context, key string, longitude, latitude float64, query *GeoRadiusQuery) *GeoLocationCmd - GeoRadiusStore(ctx context.Context, key string, longitude, latitude float64, query *GeoRadiusQuery) *IntCmd - GeoRadiusByMember(ctx context.Context, key, member string, query *GeoRadiusQuery) *GeoLocationCmd - GeoRadiusByMemberStore(ctx context.Context, key, member string, query *GeoRadiusQuery) *IntCmd - GeoSearch(ctx context.Context, key string, q *GeoSearchQuery) *StringSliceCmd - GeoSearchLocation(ctx context.Context, key string, q *GeoSearchLocationQuery) *GeoSearchLocationCmd - GeoSearchStore(ctx context.Context, key, store string, q *GeoSearchStoreQuery) *IntCmd - GeoDist(ctx context.Context, key string, member1, member2, unit string) *FloatCmd - GeoHash(ctx context.Context, key string, members ...string) *StringSliceCmd -} - -type StatefulCmdable interface { - Cmdable - Auth(ctx context.Context, password string) *StatusCmd - AuthACL(ctx context.Context, username, password string) *StatusCmd - Select(ctx context.Context, index int) *StatusCmd - SwapDB(ctx context.Context, index1, index2 int) *StatusCmd - ClientSetName(ctx context.Context, name string) *BoolCmd -} - -var ( - _ Cmdable = (*Client)(nil) - _ Cmdable = (*Tx)(nil) - _ Cmdable = (*Ring)(nil) - _ Cmdable = (*ClusterClient)(nil) -) - -type cmdable func(ctx context.Context, cmd Cmder) error - -type statefulCmdable func(ctx context.Context, cmd Cmder) error - -//------------------------------------------------------------------------------ - -func (c statefulCmdable) Auth(ctx context.Context, password string) *StatusCmd { - cmd := NewStatusCmd(ctx, "auth", password) - _ = c(ctx, cmd) - return cmd -} - -// AuthACL Perform an AUTH command, using the given user and pass. -// Should be used to authenticate the current connection with one of the connections defined in the ACL list -// when connecting to a Redis 6.0 instance, or greater, that is using the Redis ACL system. -func (c statefulCmdable) AuthACL(ctx context.Context, username, password string) *StatusCmd { - cmd := NewStatusCmd(ctx, "auth", username, password) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Wait(ctx context.Context, numSlaves int, timeout time.Duration) *IntCmd { - cmd := NewIntCmd(ctx, "wait", numSlaves, int(timeout/time.Millisecond)) - cmd.setReadTimeout(timeout) - _ = c(ctx, cmd) - return cmd -} - -func (c statefulCmdable) Select(ctx context.Context, index int) *StatusCmd { - cmd := NewStatusCmd(ctx, "select", index) - _ = c(ctx, cmd) - return cmd -} - -func (c statefulCmdable) SwapDB(ctx context.Context, index1, index2 int) *StatusCmd { - cmd := NewStatusCmd(ctx, "swapdb", index1, index2) - _ = c(ctx, cmd) - return cmd -} - -// ClientSetName assigns a name to the connection. -func (c statefulCmdable) ClientSetName(ctx context.Context, name string) *BoolCmd { - cmd := NewBoolCmd(ctx, "client", "setname", name) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -func (c cmdable) Command(ctx context.Context) *CommandsInfoCmd { - cmd := NewCommandsInfoCmd(ctx, "command") - _ = c(ctx, cmd) - return cmd -} - -// ClientGetName returns the name of the connection. -func (c cmdable) ClientGetName(ctx context.Context) *StringCmd { - cmd := NewStringCmd(ctx, "client", "getname") - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Echo(ctx context.Context, message interface{}) *StringCmd { - cmd := NewStringCmd(ctx, "echo", message) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Ping(ctx context.Context) *StatusCmd { - cmd := NewStatusCmd(ctx, "ping") - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Quit(_ context.Context) *StatusCmd { - panic("not implemented") -} - -func (c cmdable) Del(ctx context.Context, keys ...string) *IntCmd { - args := make([]interface{}, 1+len(keys)) - args[0] = "del" - for i, key := range keys { - args[1+i] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Unlink(ctx context.Context, keys ...string) *IntCmd { - args := make([]interface{}, 1+len(keys)) - args[0] = "unlink" - for i, key := range keys { - args[1+i] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Dump(ctx context.Context, key string) *StringCmd { - cmd := NewStringCmd(ctx, "dump", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Exists(ctx context.Context, keys ...string) *IntCmd { - args := make([]interface{}, 1+len(keys)) - args[0] = "exists" - for i, key := range keys { - args[1+i] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Expire(ctx context.Context, key string, expiration time.Duration) *BoolCmd { - return c.expire(ctx, key, expiration, "") -} - -func (c cmdable) ExpireNX(ctx context.Context, key string, expiration time.Duration) *BoolCmd { - return c.expire(ctx, key, expiration, "NX") -} - -func (c cmdable) ExpireXX(ctx context.Context, key string, expiration time.Duration) *BoolCmd { - return c.expire(ctx, key, expiration, "XX") -} - -func (c cmdable) ExpireGT(ctx context.Context, key string, expiration time.Duration) *BoolCmd { - return c.expire(ctx, key, expiration, "GT") -} - -func (c cmdable) ExpireLT(ctx context.Context, key string, expiration time.Duration) *BoolCmd { - return c.expire(ctx, key, expiration, "LT") -} - -func (c cmdable) expire( - ctx context.Context, key string, expiration time.Duration, mode string, -) *BoolCmd { - args := make([]interface{}, 3, 4) - args[0] = "expire" - args[1] = key - args[2] = formatSec(ctx, expiration) - if mode != "" { - args = append(args, mode) - } - - cmd := NewBoolCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ExpireAt(ctx context.Context, key string, tm time.Time) *BoolCmd { - cmd := NewBoolCmd(ctx, "expireat", key, tm.Unix()) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Keys(ctx context.Context, pattern string) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "keys", pattern) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Migrate(ctx context.Context, host, port, key string, db int, timeout time.Duration) *StatusCmd { - cmd := NewStatusCmd( - ctx, - "migrate", - host, - port, - key, - db, - formatMs(ctx, timeout), - ) - cmd.setReadTimeout(timeout) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Move(ctx context.Context, key string, db int) *BoolCmd { - cmd := NewBoolCmd(ctx, "move", key, db) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ObjectRefCount(ctx context.Context, key string) *IntCmd { - cmd := NewIntCmd(ctx, "object", "refcount", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ObjectEncoding(ctx context.Context, key string) *StringCmd { - cmd := NewStringCmd(ctx, "object", "encoding", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ObjectIdleTime(ctx context.Context, key string) *DurationCmd { - cmd := NewDurationCmd(ctx, time.Second, "object", "idletime", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Persist(ctx context.Context, key string) *BoolCmd { - cmd := NewBoolCmd(ctx, "persist", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) PExpire(ctx context.Context, key string, expiration time.Duration) *BoolCmd { - cmd := NewBoolCmd(ctx, "pexpire", key, formatMs(ctx, expiration)) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) PExpireAt(ctx context.Context, key string, tm time.Time) *BoolCmd { - cmd := NewBoolCmd( - ctx, - "pexpireat", - key, - tm.UnixNano()/int64(time.Millisecond), - ) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) PTTL(ctx context.Context, key string) *DurationCmd { - cmd := NewDurationCmd(ctx, time.Millisecond, "pttl", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) RandomKey(ctx context.Context) *StringCmd { - cmd := NewStringCmd(ctx, "randomkey") - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Rename(ctx context.Context, key, newkey string) *StatusCmd { - cmd := NewStatusCmd(ctx, "rename", key, newkey) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) RenameNX(ctx context.Context, key, newkey string) *BoolCmd { - cmd := NewBoolCmd(ctx, "renamenx", key, newkey) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Restore(ctx context.Context, key string, ttl time.Duration, value string) *StatusCmd { - cmd := NewStatusCmd( - ctx, - "restore", - key, - formatMs(ctx, ttl), - value, - ) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) RestoreReplace(ctx context.Context, key string, ttl time.Duration, value string) *StatusCmd { - cmd := NewStatusCmd( - ctx, - "restore", - key, - formatMs(ctx, ttl), - value, - "replace", - ) - _ = c(ctx, cmd) - return cmd -} - -type Sort struct { - By string - Offset, Count int64 - Get []string - Order string - Alpha bool -} - -func (sort *Sort) args(key string) []interface{} { - args := []interface{}{"sort", key} - if sort.By != "" { - args = append(args, "by", sort.By) - } - if sort.Offset != 0 || sort.Count != 0 { - args = append(args, "limit", sort.Offset, sort.Count) - } - for _, get := range sort.Get { - args = append(args, "get", get) - } - if sort.Order != "" { - args = append(args, sort.Order) - } - if sort.Alpha { - args = append(args, "alpha") - } - return args -} - -func (c cmdable) Sort(ctx context.Context, key string, sort *Sort) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, sort.args(key)...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SortStore(ctx context.Context, key, store string, sort *Sort) *IntCmd { - args := sort.args(key) - if store != "" { - args = append(args, "store", store) - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SortInterfaces(ctx context.Context, key string, sort *Sort) *SliceCmd { - cmd := NewSliceCmd(ctx, sort.args(key)...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Touch(ctx context.Context, keys ...string) *IntCmd { - args := make([]interface{}, len(keys)+1) - args[0] = "touch" - for i, key := range keys { - args[i+1] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) TTL(ctx context.Context, key string) *DurationCmd { - cmd := NewDurationCmd(ctx, time.Second, "ttl", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Type(ctx context.Context, key string) *StatusCmd { - cmd := NewStatusCmd(ctx, "type", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Append(ctx context.Context, key, value string) *IntCmd { - cmd := NewIntCmd(ctx, "append", key, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Decr(ctx context.Context, key string) *IntCmd { - cmd := NewIntCmd(ctx, "decr", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) DecrBy(ctx context.Context, key string, decrement int64) *IntCmd { - cmd := NewIntCmd(ctx, "decrby", key, decrement) - _ = c(ctx, cmd) - return cmd -} - -// Get Redis `GET key` command. It returns redis.Nil error when key does not exist. -func (c cmdable) Get(ctx context.Context, key string) *StringCmd { - cmd := NewStringCmd(ctx, "get", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) GetRange(ctx context.Context, key string, start, end int64) *StringCmd { - cmd := NewStringCmd(ctx, "getrange", key, start, end) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) GetSet(ctx context.Context, key string, value interface{}) *StringCmd { - cmd := NewStringCmd(ctx, "getset", key, value) - _ = c(ctx, cmd) - return cmd -} - -// GetEx An expiration of zero removes the TTL associated with the key (i.e. GETEX key persist). -// Requires Redis >= 6.2.0. -func (c cmdable) GetEx(ctx context.Context, key string, expiration time.Duration) *StringCmd { - args := make([]interface{}, 0, 4) - args = append(args, "getex", key) - if expiration > 0 { - if usePrecise(expiration) { - args = append(args, "px", formatMs(ctx, expiration)) - } else { - args = append(args, "ex", formatSec(ctx, expiration)) - } - } else if expiration == 0 { - args = append(args, "persist") - } - - cmd := NewStringCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// GetDel redis-server version >= 6.2.0. -func (c cmdable) GetDel(ctx context.Context, key string) *StringCmd { - cmd := NewStringCmd(ctx, "getdel", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Incr(ctx context.Context, key string) *IntCmd { - cmd := NewIntCmd(ctx, "incr", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) IncrBy(ctx context.Context, key string, value int64) *IntCmd { - cmd := NewIntCmd(ctx, "incrby", key, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) IncrByFloat(ctx context.Context, key string, value float64) *FloatCmd { - cmd := NewFloatCmd(ctx, "incrbyfloat", key, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) MGet(ctx context.Context, keys ...string) *SliceCmd { - args := make([]interface{}, 1+len(keys)) - args[0] = "mget" - for i, key := range keys { - args[1+i] = key - } - cmd := NewSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// MSet is like Set but accepts multiple values: -// - MSet("key1", "value1", "key2", "value2") -// - MSet([]string{"key1", "value1", "key2", "value2"}) -// - MSet(map[string]interface{}{"key1": "value1", "key2": "value2"}) -func (c cmdable) MSet(ctx context.Context, values ...interface{}) *StatusCmd { - args := make([]interface{}, 1, 1+len(values)) - args[0] = "mset" - args = appendArgs(args, values) - cmd := NewStatusCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// MSetNX is like SetNX but accepts multiple values: -// - MSetNX("key1", "value1", "key2", "value2") -// - MSetNX([]string{"key1", "value1", "key2", "value2"}) -// - MSetNX(map[string]interface{}{"key1": "value1", "key2": "value2"}) -func (c cmdable) MSetNX(ctx context.Context, values ...interface{}) *BoolCmd { - args := make([]interface{}, 1, 1+len(values)) - args[0] = "msetnx" - args = appendArgs(args, values) - cmd := NewBoolCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// Set Redis `SET key value [expiration]` command. -// Use expiration for `SETEX`-like behavior. -// -// Zero expiration means the key has no expiration time. -// KeepTTL is a Redis KEEPTTL option to keep existing TTL, it requires your redis-server version >= 6.0, -// otherwise you will receive an error: (error) ERR syntax error. -func (c cmdable) Set(ctx context.Context, key string, value interface{}, expiration time.Duration) *StatusCmd { - args := make([]interface{}, 3, 5) - args[0] = "set" - args[1] = key - args[2] = value - if expiration > 0 { - if usePrecise(expiration) { - args = append(args, "px", formatMs(ctx, expiration)) - } else { - args = append(args, "ex", formatSec(ctx, expiration)) - } - } else if expiration == KeepTTL { - args = append(args, "keepttl") - } - - cmd := NewStatusCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// SetArgs provides arguments for the SetArgs function. -type SetArgs struct { - // Mode can be `NX` or `XX` or empty. - Mode string - - // Zero `TTL` or `Expiration` means that the key has no expiration time. - TTL time.Duration - ExpireAt time.Time - - // When Get is true, the command returns the old value stored at key, or nil when key did not exist. - Get bool - - // KeepTTL is a Redis KEEPTTL option to keep existing TTL, it requires your redis-server version >= 6.0, - // otherwise you will receive an error: (error) ERR syntax error. - KeepTTL bool -} - -// SetArgs supports all the options that the SET command supports. -// It is the alternative to the Set function when you want -// to have more control over the options. -func (c cmdable) SetArgs(ctx context.Context, key string, value interface{}, a SetArgs) *StatusCmd { - args := []interface{}{"set", key, value} - - if a.KeepTTL { - args = append(args, "keepttl") - } - - if !a.ExpireAt.IsZero() { - args = append(args, "exat", a.ExpireAt.Unix()) - } - if a.TTL > 0 { - if usePrecise(a.TTL) { - args = append(args, "px", formatMs(ctx, a.TTL)) - } else { - args = append(args, "ex", formatSec(ctx, a.TTL)) - } - } - - if a.Mode != "" { - args = append(args, a.Mode) - } - - if a.Get { - args = append(args, "get") - } - - cmd := NewStatusCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// SetEX Redis `SETEX key expiration value` command. -func (c cmdable) SetEX(ctx context.Context, key string, value interface{}, expiration time.Duration) *StatusCmd { - cmd := NewStatusCmd(ctx, "setex", key, formatSec(ctx, expiration), value) - _ = c(ctx, cmd) - return cmd -} - -// SetNX Redis `SET key value [expiration] NX` command. -// -// Zero expiration means the key has no expiration time. -// KeepTTL is a Redis KEEPTTL option to keep existing TTL, it requires your redis-server version >= 6.0, -// otherwise you will receive an error: (error) ERR syntax error. -func (c cmdable) SetNX(ctx context.Context, key string, value interface{}, expiration time.Duration) *BoolCmd { - var cmd *BoolCmd - switch expiration { - case 0: - // Use old `SETNX` to support old Redis versions. - cmd = NewBoolCmd(ctx, "setnx", key, value) - case KeepTTL: - cmd = NewBoolCmd(ctx, "set", key, value, "keepttl", "nx") - default: - if usePrecise(expiration) { - cmd = NewBoolCmd(ctx, "set", key, value, "px", formatMs(ctx, expiration), "nx") - } else { - cmd = NewBoolCmd(ctx, "set", key, value, "ex", formatSec(ctx, expiration), "nx") - } - } - - _ = c(ctx, cmd) - return cmd -} - -// SetXX Redis `SET key value [expiration] XX` command. -// -// Zero expiration means the key has no expiration time. -// KeepTTL is a Redis KEEPTTL option to keep existing TTL, it requires your redis-server version >= 6.0, -// otherwise you will receive an error: (error) ERR syntax error. -func (c cmdable) SetXX(ctx context.Context, key string, value interface{}, expiration time.Duration) *BoolCmd { - var cmd *BoolCmd - switch expiration { - case 0: - cmd = NewBoolCmd(ctx, "set", key, value, "xx") - case KeepTTL: - cmd = NewBoolCmd(ctx, "set", key, value, "keepttl", "xx") - default: - if usePrecise(expiration) { - cmd = NewBoolCmd(ctx, "set", key, value, "px", formatMs(ctx, expiration), "xx") - } else { - cmd = NewBoolCmd(ctx, "set", key, value, "ex", formatSec(ctx, expiration), "xx") - } - } - - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SetRange(ctx context.Context, key string, offset int64, value string) *IntCmd { - cmd := NewIntCmd(ctx, "setrange", key, offset, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) StrLen(ctx context.Context, key string) *IntCmd { - cmd := NewIntCmd(ctx, "strlen", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) Copy(ctx context.Context, sourceKey, destKey string, db int, replace bool) *IntCmd { - args := []interface{}{"copy", sourceKey, destKey, "DB", db} - if replace { - args = append(args, "REPLACE") - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -func (c cmdable) GetBit(ctx context.Context, key string, offset int64) *IntCmd { - cmd := NewIntCmd(ctx, "getbit", key, offset) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SetBit(ctx context.Context, key string, offset int64, value int) *IntCmd { - cmd := NewIntCmd( - ctx, - "setbit", - key, - offset, - value, - ) - _ = c(ctx, cmd) - return cmd -} - -type BitCount struct { - Start, End int64 -} - -func (c cmdable) BitCount(ctx context.Context, key string, bitCount *BitCount) *IntCmd { - args := []interface{}{"bitcount", key} - if bitCount != nil { - args = append( - args, - bitCount.Start, - bitCount.End, - ) - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) bitOp(ctx context.Context, op, destKey string, keys ...string) *IntCmd { - args := make([]interface{}, 3+len(keys)) - args[0] = "bitop" - args[1] = op - args[2] = destKey - for i, key := range keys { - args[3+i] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) BitOpAnd(ctx context.Context, destKey string, keys ...string) *IntCmd { - return c.bitOp(ctx, "and", destKey, keys...) -} - -func (c cmdable) BitOpOr(ctx context.Context, destKey string, keys ...string) *IntCmd { - return c.bitOp(ctx, "or", destKey, keys...) -} - -func (c cmdable) BitOpXor(ctx context.Context, destKey string, keys ...string) *IntCmd { - return c.bitOp(ctx, "xor", destKey, keys...) -} - -func (c cmdable) BitOpNot(ctx context.Context, destKey string, key string) *IntCmd { - return c.bitOp(ctx, "not", destKey, key) -} - -func (c cmdable) BitPos(ctx context.Context, key string, bit int64, pos ...int64) *IntCmd { - args := make([]interface{}, 3+len(pos)) - args[0] = "bitpos" - args[1] = key - args[2] = bit - switch len(pos) { - case 0: - case 1: - args[3] = pos[0] - case 2: - args[3] = pos[0] - args[4] = pos[1] - default: - panic("too many arguments") - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) BitField(ctx context.Context, key string, args ...interface{}) *IntSliceCmd { - a := make([]interface{}, 0, 2+len(args)) - a = append(a, "bitfield") - a = append(a, key) - a = append(a, args...) - cmd := NewIntSliceCmd(ctx, a...) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -func (c cmdable) Scan(ctx context.Context, cursor uint64, match string, count int64) *ScanCmd { - args := []interface{}{"scan", cursor} - if match != "" { - args = append(args, "match", match) - } - if count > 0 { - args = append(args, "count", count) - } - cmd := NewScanCmd(ctx, c, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ScanType(ctx context.Context, cursor uint64, match string, count int64, keyType string) *ScanCmd { - args := []interface{}{"scan", cursor} - if match != "" { - args = append(args, "match", match) - } - if count > 0 { - args = append(args, "count", count) - } - if keyType != "" { - args = append(args, "type", keyType) - } - cmd := NewScanCmd(ctx, c, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SScan(ctx context.Context, key string, cursor uint64, match string, count int64) *ScanCmd { - args := []interface{}{"sscan", key, cursor} - if match != "" { - args = append(args, "match", match) - } - if count > 0 { - args = append(args, "count", count) - } - cmd := NewScanCmd(ctx, c, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HScan(ctx context.Context, key string, cursor uint64, match string, count int64) *ScanCmd { - args := []interface{}{"hscan", key, cursor} - if match != "" { - args = append(args, "match", match) - } - if count > 0 { - args = append(args, "count", count) - } - cmd := NewScanCmd(ctx, c, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZScan(ctx context.Context, key string, cursor uint64, match string, count int64) *ScanCmd { - args := []interface{}{"zscan", key, cursor} - if match != "" { - args = append(args, "match", match) - } - if count > 0 { - args = append(args, "count", count) - } - cmd := NewScanCmd(ctx, c, args...) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -func (c cmdable) HDel(ctx context.Context, key string, fields ...string) *IntCmd { - args := make([]interface{}, 2+len(fields)) - args[0] = "hdel" - args[1] = key - for i, field := range fields { - args[2+i] = field - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HExists(ctx context.Context, key, field string) *BoolCmd { - cmd := NewBoolCmd(ctx, "hexists", key, field) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HGet(ctx context.Context, key, field string) *StringCmd { - cmd := NewStringCmd(ctx, "hget", key, field) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HGetAll(ctx context.Context, key string) *StringStringMapCmd { - cmd := NewStringStringMapCmd(ctx, "hgetall", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HIncrBy(ctx context.Context, key, field string, incr int64) *IntCmd { - cmd := NewIntCmd(ctx, "hincrby", key, field, incr) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HIncrByFloat(ctx context.Context, key, field string, incr float64) *FloatCmd { - cmd := NewFloatCmd(ctx, "hincrbyfloat", key, field, incr) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HKeys(ctx context.Context, key string) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "hkeys", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HLen(ctx context.Context, key string) *IntCmd { - cmd := NewIntCmd(ctx, "hlen", key) - _ = c(ctx, cmd) - return cmd -} - -// HMGet returns the values for the specified fields in the hash stored at key. -// It returns an interface{} to distinguish between empty string and nil value. -func (c cmdable) HMGet(ctx context.Context, key string, fields ...string) *SliceCmd { - args := make([]interface{}, 2+len(fields)) - args[0] = "hmget" - args[1] = key - for i, field := range fields { - args[2+i] = field - } - cmd := NewSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// HSet accepts values in following formats: -// - HSet("myhash", "key1", "value1", "key2", "value2") -// - HSet("myhash", []string{"key1", "value1", "key2", "value2"}) -// - HSet("myhash", map[string]interface{}{"key1": "value1", "key2": "value2"}) -// -// Note that it requires Redis v4 for multiple field/value pairs support. -func (c cmdable) HSet(ctx context.Context, key string, values ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(values)) - args[0] = "hset" - args[1] = key - args = appendArgs(args, values) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// HMSet is a deprecated version of HSet left for compatibility with Redis 3. -func (c cmdable) HMSet(ctx context.Context, key string, values ...interface{}) *BoolCmd { - args := make([]interface{}, 2, 2+len(values)) - args[0] = "hmset" - args[1] = key - args = appendArgs(args, values) - cmd := NewBoolCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HSetNX(ctx context.Context, key, field string, value interface{}) *BoolCmd { - cmd := NewBoolCmd(ctx, "hsetnx", key, field, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) HVals(ctx context.Context, key string) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "hvals", key) - _ = c(ctx, cmd) - return cmd -} - -// HRandField redis-server version >= 6.2.0. -func (c cmdable) HRandField(ctx context.Context, key string, count int, withValues bool) *StringSliceCmd { - args := make([]interface{}, 0, 4) - - // Although count=0 is meaningless, redis accepts count=0. - args = append(args, "hrandfield", key, count) - if withValues { - args = append(args, "withvalues") - } - - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -func (c cmdable) BLPop(ctx context.Context, timeout time.Duration, keys ...string) *StringSliceCmd { - args := make([]interface{}, 1+len(keys)+1) - args[0] = "blpop" - for i, key := range keys { - args[1+i] = key - } - args[len(args)-1] = formatSec(ctx, timeout) - cmd := NewStringSliceCmd(ctx, args...) - cmd.setReadTimeout(timeout) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) BRPop(ctx context.Context, timeout time.Duration, keys ...string) *StringSliceCmd { - args := make([]interface{}, 1+len(keys)+1) - args[0] = "brpop" - for i, key := range keys { - args[1+i] = key - } - args[len(keys)+1] = formatSec(ctx, timeout) - cmd := NewStringSliceCmd(ctx, args...) - cmd.setReadTimeout(timeout) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) BRPopLPush(ctx context.Context, source, destination string, timeout time.Duration) *StringCmd { - cmd := NewStringCmd( - ctx, - "brpoplpush", - source, - destination, - formatSec(ctx, timeout), - ) - cmd.setReadTimeout(timeout) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LIndex(ctx context.Context, key string, index int64) *StringCmd { - cmd := NewStringCmd(ctx, "lindex", key, index) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LInsert(ctx context.Context, key, op string, pivot, value interface{}) *IntCmd { - cmd := NewIntCmd(ctx, "linsert", key, op, pivot, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LInsertBefore(ctx context.Context, key string, pivot, value interface{}) *IntCmd { - cmd := NewIntCmd(ctx, "linsert", key, "before", pivot, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LInsertAfter(ctx context.Context, key string, pivot, value interface{}) *IntCmd { - cmd := NewIntCmd(ctx, "linsert", key, "after", pivot, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LLen(ctx context.Context, key string) *IntCmd { - cmd := NewIntCmd(ctx, "llen", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LPop(ctx context.Context, key string) *StringCmd { - cmd := NewStringCmd(ctx, "lpop", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LPopCount(ctx context.Context, key string, count int) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "lpop", key, count) - _ = c(ctx, cmd) - return cmd -} - -type LPosArgs struct { - Rank, MaxLen int64 -} - -func (c cmdable) LPos(ctx context.Context, key string, value string, a LPosArgs) *IntCmd { - args := []interface{}{"lpos", key, value} - if a.Rank != 0 { - args = append(args, "rank", a.Rank) - } - if a.MaxLen != 0 { - args = append(args, "maxlen", a.MaxLen) - } - - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LPosCount(ctx context.Context, key string, value string, count int64, a LPosArgs) *IntSliceCmd { - args := []interface{}{"lpos", key, value, "count", count} - if a.Rank != 0 { - args = append(args, "rank", a.Rank) - } - if a.MaxLen != 0 { - args = append(args, "maxlen", a.MaxLen) - } - cmd := NewIntSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LPush(ctx context.Context, key string, values ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(values)) - args[0] = "lpush" - args[1] = key - args = appendArgs(args, values) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LPushX(ctx context.Context, key string, values ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(values)) - args[0] = "lpushx" - args[1] = key - args = appendArgs(args, values) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LRange(ctx context.Context, key string, start, stop int64) *StringSliceCmd { - cmd := NewStringSliceCmd( - ctx, - "lrange", - key, - start, - stop, - ) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LRem(ctx context.Context, key string, count int64, value interface{}) *IntCmd { - cmd := NewIntCmd(ctx, "lrem", key, count, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LSet(ctx context.Context, key string, index int64, value interface{}) *StatusCmd { - cmd := NewStatusCmd(ctx, "lset", key, index, value) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LTrim(ctx context.Context, key string, start, stop int64) *StatusCmd { - cmd := NewStatusCmd( - ctx, - "ltrim", - key, - start, - stop, - ) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) RPop(ctx context.Context, key string) *StringCmd { - cmd := NewStringCmd(ctx, "rpop", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) RPopCount(ctx context.Context, key string, count int) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "rpop", key, count) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) RPopLPush(ctx context.Context, source, destination string) *StringCmd { - cmd := NewStringCmd(ctx, "rpoplpush", source, destination) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) RPush(ctx context.Context, key string, values ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(values)) - args[0] = "rpush" - args[1] = key - args = appendArgs(args, values) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) RPushX(ctx context.Context, key string, values ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(values)) - args[0] = "rpushx" - args[1] = key - args = appendArgs(args, values) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) LMove(ctx context.Context, source, destination, srcpos, destpos string) *StringCmd { - cmd := NewStringCmd(ctx, "lmove", source, destination, srcpos, destpos) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) BLMove( - ctx context.Context, source, destination, srcpos, destpos string, timeout time.Duration, -) *StringCmd { - cmd := NewStringCmd(ctx, "blmove", source, destination, srcpos, destpos, formatSec(ctx, timeout)) - cmd.setReadTimeout(timeout) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -func (c cmdable) SAdd(ctx context.Context, key string, members ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(members)) - args[0] = "sadd" - args[1] = key - args = appendArgs(args, members) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SCard(ctx context.Context, key string) *IntCmd { - cmd := NewIntCmd(ctx, "scard", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SDiff(ctx context.Context, keys ...string) *StringSliceCmd { - args := make([]interface{}, 1+len(keys)) - args[0] = "sdiff" - for i, key := range keys { - args[1+i] = key - } - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SDiffStore(ctx context.Context, destination string, keys ...string) *IntCmd { - args := make([]interface{}, 2+len(keys)) - args[0] = "sdiffstore" - args[1] = destination - for i, key := range keys { - args[2+i] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SInter(ctx context.Context, keys ...string) *StringSliceCmd { - args := make([]interface{}, 1+len(keys)) - args[0] = "sinter" - for i, key := range keys { - args[1+i] = key - } - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SInterStore(ctx context.Context, destination string, keys ...string) *IntCmd { - args := make([]interface{}, 2+len(keys)) - args[0] = "sinterstore" - args[1] = destination - for i, key := range keys { - args[2+i] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SIsMember(ctx context.Context, key string, member interface{}) *BoolCmd { - cmd := NewBoolCmd(ctx, "sismember", key, member) - _ = c(ctx, cmd) - return cmd -} - -// SMIsMember Redis `SMISMEMBER key member [member ...]` command. -func (c cmdable) SMIsMember(ctx context.Context, key string, members ...interface{}) *BoolSliceCmd { - args := make([]interface{}, 2, 2+len(members)) - args[0] = "smismember" - args[1] = key - args = appendArgs(args, members) - cmd := NewBoolSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// SMembers Redis `SMEMBERS key` command output as a slice. -func (c cmdable) SMembers(ctx context.Context, key string) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "smembers", key) - _ = c(ctx, cmd) - return cmd -} - -// SMembersMap Redis `SMEMBERS key` command output as a map. -func (c cmdable) SMembersMap(ctx context.Context, key string) *StringStructMapCmd { - cmd := NewStringStructMapCmd(ctx, "smembers", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SMove(ctx context.Context, source, destination string, member interface{}) *BoolCmd { - cmd := NewBoolCmd(ctx, "smove", source, destination, member) - _ = c(ctx, cmd) - return cmd -} - -// SPop Redis `SPOP key` command. -func (c cmdable) SPop(ctx context.Context, key string) *StringCmd { - cmd := NewStringCmd(ctx, "spop", key) - _ = c(ctx, cmd) - return cmd -} - -// SPopN Redis `SPOP key count` command. -func (c cmdable) SPopN(ctx context.Context, key string, count int64) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "spop", key, count) - _ = c(ctx, cmd) - return cmd -} - -// SRandMember Redis `SRANDMEMBER key` command. -func (c cmdable) SRandMember(ctx context.Context, key string) *StringCmd { - cmd := NewStringCmd(ctx, "srandmember", key) - _ = c(ctx, cmd) - return cmd -} - -// SRandMemberN Redis `SRANDMEMBER key count` command. -func (c cmdable) SRandMemberN(ctx context.Context, key string, count int64) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "srandmember", key, count) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SRem(ctx context.Context, key string, members ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(members)) - args[0] = "srem" - args[1] = key - args = appendArgs(args, members) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SUnion(ctx context.Context, keys ...string) *StringSliceCmd { - args := make([]interface{}, 1+len(keys)) - args[0] = "sunion" - for i, key := range keys { - args[1+i] = key - } - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) SUnionStore(ctx context.Context, destination string, keys ...string) *IntCmd { - args := make([]interface{}, 2+len(keys)) - args[0] = "sunionstore" - args[1] = destination - for i, key := range keys { - args[2+i] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -// XAddArgs accepts values in the following formats: -// - XAddArgs.Values = []interface{}{"key1", "value1", "key2", "value2"} -// - XAddArgs.Values = []string("key1", "value1", "key2", "value2") -// - XAddArgs.Values = map[string]interface{}{"key1": "value1", "key2": "value2"} -// -// Note that map will not preserve the order of key-value pairs. -// MaxLen/MaxLenApprox and MinID are in conflict, only one of them can be used. -type XAddArgs struct { - Stream string - NoMkStream bool - MaxLen int64 // MAXLEN N - - // Deprecated: use MaxLen+Approx, remove in v9. - MaxLenApprox int64 // MAXLEN ~ N - - MinID string - // Approx causes MaxLen and MinID to use "~" matcher (instead of "="). - Approx bool - Limit int64 - ID string - Values interface{} -} - -// XAdd a.Limit has a bug, please confirm it and use it. -// issue: https://github.com/redis/redis/issues/9046 -func (c cmdable) XAdd(ctx context.Context, a *XAddArgs) *StringCmd { - args := make([]interface{}, 0, 11) - args = append(args, "xadd", a.Stream) - if a.NoMkStream { - args = append(args, "nomkstream") - } - switch { - case a.MaxLen > 0: - if a.Approx { - args = append(args, "maxlen", "~", a.MaxLen) - } else { - args = append(args, "maxlen", a.MaxLen) - } - case a.MaxLenApprox > 0: - // TODO remove in v9. - args = append(args, "maxlen", "~", a.MaxLenApprox) - case a.MinID != "": - if a.Approx { - args = append(args, "minid", "~", a.MinID) - } else { - args = append(args, "minid", a.MinID) - } - } - if a.Limit > 0 { - args = append(args, "limit", a.Limit) - } - if a.ID != "" { - args = append(args, a.ID) - } else { - args = append(args, "*") - } - args = appendArg(args, a.Values) - - cmd := NewStringCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XDel(ctx context.Context, stream string, ids ...string) *IntCmd { - args := []interface{}{"xdel", stream} - for _, id := range ids { - args = append(args, id) - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XLen(ctx context.Context, stream string) *IntCmd { - cmd := NewIntCmd(ctx, "xlen", stream) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XRange(ctx context.Context, stream, start, stop string) *XMessageSliceCmd { - cmd := NewXMessageSliceCmd(ctx, "xrange", stream, start, stop) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XRangeN(ctx context.Context, stream, start, stop string, count int64) *XMessageSliceCmd { - cmd := NewXMessageSliceCmd(ctx, "xrange", stream, start, stop, "count", count) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XRevRange(ctx context.Context, stream, start, stop string) *XMessageSliceCmd { - cmd := NewXMessageSliceCmd(ctx, "xrevrange", stream, start, stop) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XRevRangeN(ctx context.Context, stream, start, stop string, count int64) *XMessageSliceCmd { - cmd := NewXMessageSliceCmd(ctx, "xrevrange", stream, start, stop, "count", count) - _ = c(ctx, cmd) - return cmd -} - -type XReadArgs struct { - Streams []string // list of streams and ids, e.g. stream1 stream2 id1 id2 - Count int64 - Block time.Duration -} - -func (c cmdable) XRead(ctx context.Context, a *XReadArgs) *XStreamSliceCmd { - args := make([]interface{}, 0, 6+len(a.Streams)) - args = append(args, "xread") - - keyPos := int8(1) - if a.Count > 0 { - args = append(args, "count") - args = append(args, a.Count) - keyPos += 2 - } - if a.Block >= 0 { - args = append(args, "block") - args = append(args, int64(a.Block/time.Millisecond)) - keyPos += 2 - } - args = append(args, "streams") - keyPos++ - for _, s := range a.Streams { - args = append(args, s) - } - - cmd := NewXStreamSliceCmd(ctx, args...) - if a.Block >= 0 { - cmd.setReadTimeout(a.Block) - } - cmd.SetFirstKeyPos(keyPos) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XReadStreams(ctx context.Context, streams ...string) *XStreamSliceCmd { - return c.XRead(ctx, &XReadArgs{ - Streams: streams, - Block: -1, - }) -} - -func (c cmdable) XGroupCreate(ctx context.Context, stream, group, start string) *StatusCmd { - cmd := NewStatusCmd(ctx, "xgroup", "create", stream, group, start) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XGroupCreateMkStream(ctx context.Context, stream, group, start string) *StatusCmd { - cmd := NewStatusCmd(ctx, "xgroup", "create", stream, group, start, "mkstream") - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XGroupSetID(ctx context.Context, stream, group, start string) *StatusCmd { - cmd := NewStatusCmd(ctx, "xgroup", "setid", stream, group, start) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XGroupDestroy(ctx context.Context, stream, group string) *IntCmd { - cmd := NewIntCmd(ctx, "xgroup", "destroy", stream, group) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XGroupCreateConsumer(ctx context.Context, stream, group, consumer string) *IntCmd { - cmd := NewIntCmd(ctx, "xgroup", "createconsumer", stream, group, consumer) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XGroupDelConsumer(ctx context.Context, stream, group, consumer string) *IntCmd { - cmd := NewIntCmd(ctx, "xgroup", "delconsumer", stream, group, consumer) - _ = c(ctx, cmd) - return cmd -} - -type XReadGroupArgs struct { - Group string - Consumer string - Streams []string // list of streams and ids, e.g. stream1 stream2 id1 id2 - Count int64 - Block time.Duration - NoAck bool -} - -func (c cmdable) XReadGroup(ctx context.Context, a *XReadGroupArgs) *XStreamSliceCmd { - args := make([]interface{}, 0, 10+len(a.Streams)) - args = append(args, "xreadgroup", "group", a.Group, a.Consumer) - - keyPos := int8(4) - if a.Count > 0 { - args = append(args, "count", a.Count) - keyPos += 2 - } - if a.Block >= 0 { - args = append(args, "block", int64(a.Block/time.Millisecond)) - keyPos += 2 - } - if a.NoAck { - args = append(args, "noack") - keyPos++ - } - args = append(args, "streams") - keyPos++ - for _, s := range a.Streams { - args = append(args, s) - } - - cmd := NewXStreamSliceCmd(ctx, args...) - if a.Block >= 0 { - cmd.setReadTimeout(a.Block) - } - cmd.SetFirstKeyPos(keyPos) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XAck(ctx context.Context, stream, group string, ids ...string) *IntCmd { - args := []interface{}{"xack", stream, group} - for _, id := range ids { - args = append(args, id) - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XPending(ctx context.Context, stream, group string) *XPendingCmd { - cmd := NewXPendingCmd(ctx, "xpending", stream, group) - _ = c(ctx, cmd) - return cmd -} - -type XPendingExtArgs struct { - Stream string - Group string - Idle time.Duration - Start string - End string - Count int64 - Consumer string -} - -func (c cmdable) XPendingExt(ctx context.Context, a *XPendingExtArgs) *XPendingExtCmd { - args := make([]interface{}, 0, 9) - args = append(args, "xpending", a.Stream, a.Group) - if a.Idle != 0 { - args = append(args, "idle", formatMs(ctx, a.Idle)) - } - args = append(args, a.Start, a.End, a.Count) - if a.Consumer != "" { - args = append(args, a.Consumer) - } - cmd := NewXPendingExtCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -type XAutoClaimArgs struct { - Stream string - Group string - MinIdle time.Duration - Start string - Count int64 - Consumer string -} - -func (c cmdable) XAutoClaim(ctx context.Context, a *XAutoClaimArgs) *XAutoClaimCmd { - args := xAutoClaimArgs(ctx, a) - cmd := NewXAutoClaimCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XAutoClaimJustID(ctx context.Context, a *XAutoClaimArgs) *XAutoClaimJustIDCmd { - args := xAutoClaimArgs(ctx, a) - args = append(args, "justid") - cmd := NewXAutoClaimJustIDCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func xAutoClaimArgs(ctx context.Context, a *XAutoClaimArgs) []interface{} { - args := make([]interface{}, 0, 8) - args = append(args, "xautoclaim", a.Stream, a.Group, a.Consumer, formatMs(ctx, a.MinIdle), a.Start) - if a.Count > 0 { - args = append(args, "count", a.Count) - } - return args -} - -type XClaimArgs struct { - Stream string - Group string - Consumer string - MinIdle time.Duration - Messages []string -} - -func (c cmdable) XClaim(ctx context.Context, a *XClaimArgs) *XMessageSliceCmd { - args := xClaimArgs(a) - cmd := NewXMessageSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XClaimJustID(ctx context.Context, a *XClaimArgs) *StringSliceCmd { - args := xClaimArgs(a) - args = append(args, "justid") - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func xClaimArgs(a *XClaimArgs) []interface{} { - args := make([]interface{}, 0, 5+len(a.Messages)) - args = append(args, - "xclaim", - a.Stream, - a.Group, a.Consumer, - int64(a.MinIdle/time.Millisecond)) - for _, id := range a.Messages { - args = append(args, id) - } - return args -} - -// xTrim If approx is true, add the "~" parameter, otherwise it is the default "=" (redis default). -// example: -// XTRIM key MAXLEN/MINID threshold LIMIT limit. -// XTRIM key MAXLEN/MINID ~ threshold LIMIT limit. -// The redis-server version is lower than 6.2, please set limit to 0. -func (c cmdable) xTrim( - ctx context.Context, key, strategy string, - approx bool, threshold interface{}, limit int64, -) *IntCmd { - args := make([]interface{}, 0, 7) - args = append(args, "xtrim", key, strategy) - if approx { - args = append(args, "~") - } - args = append(args, threshold) - if limit > 0 { - args = append(args, "limit", limit) - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// Deprecated: use XTrimMaxLen, remove in v9. -func (c cmdable) XTrim(ctx context.Context, key string, maxLen int64) *IntCmd { - return c.xTrim(ctx, key, "maxlen", false, maxLen, 0) -} - -// Deprecated: use XTrimMaxLenApprox, remove in v9. -func (c cmdable) XTrimApprox(ctx context.Context, key string, maxLen int64) *IntCmd { - return c.xTrim(ctx, key, "maxlen", true, maxLen, 0) -} - -// XTrimMaxLen No `~` rules are used, `limit` cannot be used. -// cmd: XTRIM key MAXLEN maxLen -func (c cmdable) XTrimMaxLen(ctx context.Context, key string, maxLen int64) *IntCmd { - return c.xTrim(ctx, key, "maxlen", false, maxLen, 0) -} - -// XTrimMaxLenApprox LIMIT has a bug, please confirm it and use it. -// issue: https://github.com/redis/redis/issues/9046 -// cmd: XTRIM key MAXLEN ~ maxLen LIMIT limit -func (c cmdable) XTrimMaxLenApprox(ctx context.Context, key string, maxLen, limit int64) *IntCmd { - return c.xTrim(ctx, key, "maxlen", true, maxLen, limit) -} - -// XTrimMinID No `~` rules are used, `limit` cannot be used. -// cmd: XTRIM key MINID minID -func (c cmdable) XTrimMinID(ctx context.Context, key string, minID string) *IntCmd { - return c.xTrim(ctx, key, "minid", false, minID, 0) -} - -// XTrimMinIDApprox LIMIT has a bug, please confirm it and use it. -// issue: https://github.com/redis/redis/issues/9046 -// cmd: XTRIM key MINID ~ minID LIMIT limit -func (c cmdable) XTrimMinIDApprox(ctx context.Context, key string, minID string, limit int64) *IntCmd { - return c.xTrim(ctx, key, "minid", true, minID, limit) -} - -func (c cmdable) XInfoConsumers(ctx context.Context, key string, group string) *XInfoConsumersCmd { - cmd := NewXInfoConsumersCmd(ctx, key, group) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XInfoGroups(ctx context.Context, key string) *XInfoGroupsCmd { - cmd := NewXInfoGroupsCmd(ctx, key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) XInfoStream(ctx context.Context, key string) *XInfoStreamCmd { - cmd := NewXInfoStreamCmd(ctx, key) - _ = c(ctx, cmd) - return cmd -} - -// XInfoStreamFull XINFO STREAM FULL [COUNT count] -// redis-server >= 6.0. -func (c cmdable) XInfoStreamFull(ctx context.Context, key string, count int) *XInfoStreamFullCmd { - args := make([]interface{}, 0, 6) - args = append(args, "xinfo", "stream", key, "full") - if count > 0 { - args = append(args, "count", count) - } - cmd := NewXInfoStreamFullCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -// Z represents sorted set member. -type Z struct { - Score float64 - Member interface{} -} - -// ZWithKey represents sorted set member including the name of the key where it was popped. -type ZWithKey struct { - Z - Key string -} - -// ZStore is used as an arg to ZInter/ZInterStore and ZUnion/ZUnionStore. -type ZStore struct { - Keys []string - Weights []float64 - // Can be SUM, MIN or MAX. - Aggregate string -} - -func (z ZStore) len() (n int) { - n = len(z.Keys) - if len(z.Weights) > 0 { - n += 1 + len(z.Weights) - } - if z.Aggregate != "" { - n += 2 - } - return n -} - -func (z ZStore) appendArgs(args []interface{}) []interface{} { - for _, key := range z.Keys { - args = append(args, key) - } - if len(z.Weights) > 0 { - args = append(args, "weights") - for _, weights := range z.Weights { - args = append(args, weights) - } - } - if z.Aggregate != "" { - args = append(args, "aggregate", z.Aggregate) - } - return args -} - -// BZPopMax Redis `BZPOPMAX key [key ...] timeout` command. -func (c cmdable) BZPopMax(ctx context.Context, timeout time.Duration, keys ...string) *ZWithKeyCmd { - args := make([]interface{}, 1+len(keys)+1) - args[0] = "bzpopmax" - for i, key := range keys { - args[1+i] = key - } - args[len(args)-1] = formatSec(ctx, timeout) - cmd := NewZWithKeyCmd(ctx, args...) - cmd.setReadTimeout(timeout) - _ = c(ctx, cmd) - return cmd -} - -// BZPopMin Redis `BZPOPMIN key [key ...] timeout` command. -func (c cmdable) BZPopMin(ctx context.Context, timeout time.Duration, keys ...string) *ZWithKeyCmd { - args := make([]interface{}, 1+len(keys)+1) - args[0] = "bzpopmin" - for i, key := range keys { - args[1+i] = key - } - args[len(args)-1] = formatSec(ctx, timeout) - cmd := NewZWithKeyCmd(ctx, args...) - cmd.setReadTimeout(timeout) - _ = c(ctx, cmd) - return cmd -} - -// ZAddArgs WARN: The GT, LT and NX options are mutually exclusive. -type ZAddArgs struct { - NX bool - XX bool - LT bool - GT bool - Ch bool - Members []Z -} - -func (c cmdable) zAddArgs(key string, args ZAddArgs, incr bool) []interface{} { - a := make([]interface{}, 0, 6+2*len(args.Members)) - a = append(a, "zadd", key) - - // The GT, LT and NX options are mutually exclusive. - if args.NX { - a = append(a, "nx") - } else { - if args.XX { - a = append(a, "xx") - } - if args.GT { - a = append(a, "gt") - } else if args.LT { - a = append(a, "lt") - } - } - if args.Ch { - a = append(a, "ch") - } - if incr { - a = append(a, "incr") - } - for _, m := range args.Members { - a = append(a, m.Score) - a = append(a, m.Member) - } - return a -} - -func (c cmdable) ZAddArgs(ctx context.Context, key string, args ZAddArgs) *IntCmd { - cmd := NewIntCmd(ctx, c.zAddArgs(key, args, false)...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZAddArgsIncr(ctx context.Context, key string, args ZAddArgs) *FloatCmd { - cmd := NewFloatCmd(ctx, c.zAddArgs(key, args, true)...) - _ = c(ctx, cmd) - return cmd -} - -// TODO: Compatible with v8 api, will be removed in v9. -func (c cmdable) zAdd(ctx context.Context, key string, args ZAddArgs, members ...*Z) *IntCmd { - args.Members = make([]Z, len(members)) - for i, m := range members { - args.Members[i] = *m - } - cmd := NewIntCmd(ctx, c.zAddArgs(key, args, false)...) - _ = c(ctx, cmd) - return cmd -} - -// ZAdd Redis `ZADD key score member [score member ...]` command. -func (c cmdable) ZAdd(ctx context.Context, key string, members ...*Z) *IntCmd { - return c.zAdd(ctx, key, ZAddArgs{}, members...) -} - -// ZAddNX Redis `ZADD key NX score member [score member ...]` command. -func (c cmdable) ZAddNX(ctx context.Context, key string, members ...*Z) *IntCmd { - return c.zAdd(ctx, key, ZAddArgs{ - NX: true, - }, members...) -} - -// ZAddXX Redis `ZADD key XX score member [score member ...]` command. -func (c cmdable) ZAddXX(ctx context.Context, key string, members ...*Z) *IntCmd { - return c.zAdd(ctx, key, ZAddArgs{ - XX: true, - }, members...) -} - -// ZAddCh Redis `ZADD key CH score member [score member ...]` command. -// Deprecated: Use -// client.ZAddArgs(ctx, ZAddArgs{ -// Ch: true, -// Members: []Z, -// }) -// remove in v9. -func (c cmdable) ZAddCh(ctx context.Context, key string, members ...*Z) *IntCmd { - return c.zAdd(ctx, key, ZAddArgs{ - Ch: true, - }, members...) -} - -// ZAddNXCh Redis `ZADD key NX CH score member [score member ...]` command. -// Deprecated: Use -// client.ZAddArgs(ctx, ZAddArgs{ -// NX: true, -// Ch: true, -// Members: []Z, -// }) -// remove in v9. -func (c cmdable) ZAddNXCh(ctx context.Context, key string, members ...*Z) *IntCmd { - return c.zAdd(ctx, key, ZAddArgs{ - NX: true, - Ch: true, - }, members...) -} - -// ZAddXXCh Redis `ZADD key XX CH score member [score member ...]` command. -// Deprecated: Use -// client.ZAddArgs(ctx, ZAddArgs{ -// XX: true, -// Ch: true, -// Members: []Z, -// }) -// remove in v9. -func (c cmdable) ZAddXXCh(ctx context.Context, key string, members ...*Z) *IntCmd { - return c.zAdd(ctx, key, ZAddArgs{ - XX: true, - Ch: true, - }, members...) -} - -// ZIncr Redis `ZADD key INCR score member` command. -// Deprecated: Use -// client.ZAddArgsIncr(ctx, ZAddArgs{ -// Members: []Z, -// }) -// remove in v9. -func (c cmdable) ZIncr(ctx context.Context, key string, member *Z) *FloatCmd { - return c.ZAddArgsIncr(ctx, key, ZAddArgs{ - Members: []Z{*member}, - }) -} - -// ZIncrNX Redis `ZADD key NX INCR score member` command. -// Deprecated: Use -// client.ZAddArgsIncr(ctx, ZAddArgs{ -// NX: true, -// Members: []Z, -// }) -// remove in v9. -func (c cmdable) ZIncrNX(ctx context.Context, key string, member *Z) *FloatCmd { - return c.ZAddArgsIncr(ctx, key, ZAddArgs{ - NX: true, - Members: []Z{*member}, - }) -} - -// ZIncrXX Redis `ZADD key XX INCR score member` command. -// Deprecated: Use -// client.ZAddArgsIncr(ctx, ZAddArgs{ -// XX: true, -// Members: []Z, -// }) -// remove in v9. -func (c cmdable) ZIncrXX(ctx context.Context, key string, member *Z) *FloatCmd { - return c.ZAddArgsIncr(ctx, key, ZAddArgs{ - XX: true, - Members: []Z{*member}, - }) -} - -func (c cmdable) ZCard(ctx context.Context, key string) *IntCmd { - cmd := NewIntCmd(ctx, "zcard", key) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZCount(ctx context.Context, key, min, max string) *IntCmd { - cmd := NewIntCmd(ctx, "zcount", key, min, max) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZLexCount(ctx context.Context, key, min, max string) *IntCmd { - cmd := NewIntCmd(ctx, "zlexcount", key, min, max) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZIncrBy(ctx context.Context, key string, increment float64, member string) *FloatCmd { - cmd := NewFloatCmd(ctx, "zincrby", key, increment, member) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZInterStore(ctx context.Context, destination string, store *ZStore) *IntCmd { - args := make([]interface{}, 0, 3+store.len()) - args = append(args, "zinterstore", destination, len(store.Keys)) - args = store.appendArgs(args) - cmd := NewIntCmd(ctx, args...) - cmd.SetFirstKeyPos(3) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZInter(ctx context.Context, store *ZStore) *StringSliceCmd { - args := make([]interface{}, 0, 2+store.len()) - args = append(args, "zinter", len(store.Keys)) - args = store.appendArgs(args) - cmd := NewStringSliceCmd(ctx, args...) - cmd.SetFirstKeyPos(2) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZInterWithScores(ctx context.Context, store *ZStore) *ZSliceCmd { - args := make([]interface{}, 0, 3+store.len()) - args = append(args, "zinter", len(store.Keys)) - args = store.appendArgs(args) - args = append(args, "withscores") - cmd := NewZSliceCmd(ctx, args...) - cmd.SetFirstKeyPos(2) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZMScore(ctx context.Context, key string, members ...string) *FloatSliceCmd { - args := make([]interface{}, 2+len(members)) - args[0] = "zmscore" - args[1] = key - for i, member := range members { - args[2+i] = member - } - cmd := NewFloatSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZPopMax(ctx context.Context, key string, count ...int64) *ZSliceCmd { - args := []interface{}{ - "zpopmax", - key, - } - - switch len(count) { - case 0: - break - case 1: - args = append(args, count[0]) - default: - panic("too many arguments") - } - - cmd := NewZSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZPopMin(ctx context.Context, key string, count ...int64) *ZSliceCmd { - args := []interface{}{ - "zpopmin", - key, - } - - switch len(count) { - case 0: - break - case 1: - args = append(args, count[0]) - default: - panic("too many arguments") - } - - cmd := NewZSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// ZRangeArgs is all the options of the ZRange command. -// In version> 6.2.0, you can replace the(cmd): -// ZREVRANGE, -// ZRANGEBYSCORE, -// ZREVRANGEBYSCORE, -// ZRANGEBYLEX, -// ZREVRANGEBYLEX. -// Please pay attention to your redis-server version. -// -// Rev, ByScore, ByLex and Offset+Count options require redis-server 6.2.0 and higher. -type ZRangeArgs struct { - Key string - - // When the ByScore option is provided, the open interval(exclusive) can be set. - // By default, the score intervals specified by and are closed (inclusive). - // It is similar to the deprecated(6.2.0+) ZRangeByScore command. - // For example: - // ZRangeArgs{ - // Key: "example-key", - // Start: "(3", - // Stop: 8, - // ByScore: true, - // } - // cmd: "ZRange example-key (3 8 ByScore" (3 < score <= 8). - // - // For the ByLex option, it is similar to the deprecated(6.2.0+) ZRangeByLex command. - // You can set the and options as follows: - // ZRangeArgs{ - // Key: "example-key", - // Start: "[abc", - // Stop: "(def", - // ByLex: true, - // } - // cmd: "ZRange example-key [abc (def ByLex" - // - // For normal cases (ByScore==false && ByLex==false), and should be set to the index range (int). - // You can read the documentation for more information: https://redis.io/commands/zrange - Start interface{} - Stop interface{} - - // The ByScore and ByLex options are mutually exclusive. - ByScore bool - ByLex bool - - Rev bool - - // limit offset count. - Offset int64 - Count int64 -} - -func (z ZRangeArgs) appendArgs(args []interface{}) []interface{} { - // For Rev+ByScore/ByLex, we need to adjust the position of and . - if z.Rev && (z.ByScore || z.ByLex) { - args = append(args, z.Key, z.Stop, z.Start) - } else { - args = append(args, z.Key, z.Start, z.Stop) - } - - if z.ByScore { - args = append(args, "byscore") - } else if z.ByLex { - args = append(args, "bylex") - } - if z.Rev { - args = append(args, "rev") - } - if z.Offset != 0 || z.Count != 0 { - args = append(args, "limit", z.Offset, z.Count) - } - return args -} - -func (c cmdable) ZRangeArgs(ctx context.Context, z ZRangeArgs) *StringSliceCmd { - args := make([]interface{}, 0, 9) - args = append(args, "zrange") - args = z.appendArgs(args) - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRangeArgsWithScores(ctx context.Context, z ZRangeArgs) *ZSliceCmd { - args := make([]interface{}, 0, 10) - args = append(args, "zrange") - args = z.appendArgs(args) - args = append(args, "withscores") - cmd := NewZSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRange(ctx context.Context, key string, start, stop int64) *StringSliceCmd { - return c.ZRangeArgs(ctx, ZRangeArgs{ - Key: key, - Start: start, - Stop: stop, - }) -} - -func (c cmdable) ZRangeWithScores(ctx context.Context, key string, start, stop int64) *ZSliceCmd { - return c.ZRangeArgsWithScores(ctx, ZRangeArgs{ - Key: key, - Start: start, - Stop: stop, - }) -} - -type ZRangeBy struct { - Min, Max string - Offset, Count int64 -} - -func (c cmdable) zRangeBy(ctx context.Context, zcmd, key string, opt *ZRangeBy, withScores bool) *StringSliceCmd { - args := []interface{}{zcmd, key, opt.Min, opt.Max} - if withScores { - args = append(args, "withscores") - } - if opt.Offset != 0 || opt.Count != 0 { - args = append( - args, - "limit", - opt.Offset, - opt.Count, - ) - } - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRangeByScore(ctx context.Context, key string, opt *ZRangeBy) *StringSliceCmd { - return c.zRangeBy(ctx, "zrangebyscore", key, opt, false) -} - -func (c cmdable) ZRangeByLex(ctx context.Context, key string, opt *ZRangeBy) *StringSliceCmd { - return c.zRangeBy(ctx, "zrangebylex", key, opt, false) -} - -func (c cmdable) ZRangeByScoreWithScores(ctx context.Context, key string, opt *ZRangeBy) *ZSliceCmd { - args := []interface{}{"zrangebyscore", key, opt.Min, opt.Max, "withscores"} - if opt.Offset != 0 || opt.Count != 0 { - args = append( - args, - "limit", - opt.Offset, - opt.Count, - ) - } - cmd := NewZSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRangeStore(ctx context.Context, dst string, z ZRangeArgs) *IntCmd { - args := make([]interface{}, 0, 10) - args = append(args, "zrangestore", dst) - args = z.appendArgs(args) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRank(ctx context.Context, key, member string) *IntCmd { - cmd := NewIntCmd(ctx, "zrank", key, member) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRem(ctx context.Context, key string, members ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(members)) - args[0] = "zrem" - args[1] = key - args = appendArgs(args, members) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRemRangeByRank(ctx context.Context, key string, start, stop int64) *IntCmd { - cmd := NewIntCmd( - ctx, - "zremrangebyrank", - key, - start, - stop, - ) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRemRangeByScore(ctx context.Context, key, min, max string) *IntCmd { - cmd := NewIntCmd(ctx, "zremrangebyscore", key, min, max) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRemRangeByLex(ctx context.Context, key, min, max string) *IntCmd { - cmd := NewIntCmd(ctx, "zremrangebylex", key, min, max) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRevRange(ctx context.Context, key string, start, stop int64) *StringSliceCmd { - cmd := NewStringSliceCmd(ctx, "zrevrange", key, start, stop) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRevRangeWithScores(ctx context.Context, key string, start, stop int64) *ZSliceCmd { - cmd := NewZSliceCmd(ctx, "zrevrange", key, start, stop, "withscores") - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) zRevRangeBy(ctx context.Context, zcmd, key string, opt *ZRangeBy) *StringSliceCmd { - args := []interface{}{zcmd, key, opt.Max, opt.Min} - if opt.Offset != 0 || opt.Count != 0 { - args = append( - args, - "limit", - opt.Offset, - opt.Count, - ) - } - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRevRangeByScore(ctx context.Context, key string, opt *ZRangeBy) *StringSliceCmd { - return c.zRevRangeBy(ctx, "zrevrangebyscore", key, opt) -} - -func (c cmdable) ZRevRangeByLex(ctx context.Context, key string, opt *ZRangeBy) *StringSliceCmd { - return c.zRevRangeBy(ctx, "zrevrangebylex", key, opt) -} - -func (c cmdable) ZRevRangeByScoreWithScores(ctx context.Context, key string, opt *ZRangeBy) *ZSliceCmd { - args := []interface{}{"zrevrangebyscore", key, opt.Max, opt.Min, "withscores"} - if opt.Offset != 0 || opt.Count != 0 { - args = append( - args, - "limit", - opt.Offset, - opt.Count, - ) - } - cmd := NewZSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZRevRank(ctx context.Context, key, member string) *IntCmd { - cmd := NewIntCmd(ctx, "zrevrank", key, member) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZScore(ctx context.Context, key, member string) *FloatCmd { - cmd := NewFloatCmd(ctx, "zscore", key, member) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZUnion(ctx context.Context, store ZStore) *StringSliceCmd { - args := make([]interface{}, 0, 2+store.len()) - args = append(args, "zunion", len(store.Keys)) - args = store.appendArgs(args) - cmd := NewStringSliceCmd(ctx, args...) - cmd.SetFirstKeyPos(2) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZUnionWithScores(ctx context.Context, store ZStore) *ZSliceCmd { - args := make([]interface{}, 0, 3+store.len()) - args = append(args, "zunion", len(store.Keys)) - args = store.appendArgs(args) - args = append(args, "withscores") - cmd := NewZSliceCmd(ctx, args...) - cmd.SetFirstKeyPos(2) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ZUnionStore(ctx context.Context, dest string, store *ZStore) *IntCmd { - args := make([]interface{}, 0, 3+store.len()) - args = append(args, "zunionstore", dest, len(store.Keys)) - args = store.appendArgs(args) - cmd := NewIntCmd(ctx, args...) - cmd.SetFirstKeyPos(3) - _ = c(ctx, cmd) - return cmd -} - -// ZRandMember redis-server version >= 6.2.0. -func (c cmdable) ZRandMember(ctx context.Context, key string, count int, withScores bool) *StringSliceCmd { - args := make([]interface{}, 0, 4) - - // Although count=0 is meaningless, redis accepts count=0. - args = append(args, "zrandmember", key, count) - if withScores { - args = append(args, "withscores") - } - - cmd := NewStringSliceCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -// ZDiff redis-server version >= 6.2.0. -func (c cmdable) ZDiff(ctx context.Context, keys ...string) *StringSliceCmd { - args := make([]interface{}, 2+len(keys)) - args[0] = "zdiff" - args[1] = len(keys) - for i, key := range keys { - args[i+2] = key - } - - cmd := NewStringSliceCmd(ctx, args...) - cmd.SetFirstKeyPos(2) - _ = c(ctx, cmd) - return cmd -} - -// ZDiffWithScores redis-server version >= 6.2.0. -func (c cmdable) ZDiffWithScores(ctx context.Context, keys ...string) *ZSliceCmd { - args := make([]interface{}, 3+len(keys)) - args[0] = "zdiff" - args[1] = len(keys) - for i, key := range keys { - args[i+2] = key - } - args[len(keys)+2] = "withscores" - - cmd := NewZSliceCmd(ctx, args...) - cmd.SetFirstKeyPos(2) - _ = c(ctx, cmd) - return cmd -} - -// ZDiffStore redis-server version >=6.2.0. -func (c cmdable) ZDiffStore(ctx context.Context, destination string, keys ...string) *IntCmd { - args := make([]interface{}, 0, 3+len(keys)) - args = append(args, "zdiffstore", destination, len(keys)) - for _, key := range keys { - args = append(args, key) - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -func (c cmdable) PFAdd(ctx context.Context, key string, els ...interface{}) *IntCmd { - args := make([]interface{}, 2, 2+len(els)) - args[0] = "pfadd" - args[1] = key - args = appendArgs(args, els) - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) PFCount(ctx context.Context, keys ...string) *IntCmd { - args := make([]interface{}, 1+len(keys)) - args[0] = "pfcount" - for i, key := range keys { - args[1+i] = key - } - cmd := NewIntCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) PFMerge(ctx context.Context, dest string, keys ...string) *StatusCmd { - args := make([]interface{}, 2+len(keys)) - args[0] = "pfmerge" - args[1] = dest - for i, key := range keys { - args[2+i] = key - } - cmd := NewStatusCmd(ctx, args...) - _ = c(ctx, cmd) - return cmd -} - -//------------------------------------------------------------------------------ - -func (c cmdable) BgRewriteAOF(ctx context.Context) *StatusCmd { - cmd := NewStatusCmd(ctx, "bgrewriteaof") - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) BgSave(ctx context.Context) *StatusCmd { - cmd := NewStatusCmd(ctx, "bgsave") - _ = c(ctx, cmd) - return cmd -} - -func (c cmdable) ClientKill(ctx context.Context, ipPort string) *StatusCmd { - cmd := NewStatusCmd(ctx, "client", "kill", ipPort) - _ = c(ctx, cmd) - return cmd -} - -// ClientKillByFilter is new style syntax, while the ClientKill is old -// -// CLIENT KILL