diff --git a/CHANGELOG.md b/CHANGELOG.md
index 19645661a..ec9c46088 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,6 +1,11 @@
# Changelog
-## Unreleased
+## v7 WIP
+
+- WrapProcess is replaced with more convenient AddHook that has access to context.Context.
+- WithContext no longer creates shallow copy.
+
+## v6.15
- Cluster and Ring pipelines process commands for each node in its own goroutine.
diff --git a/bench_test.go b/bench_test.go
index 3af2e24ea..12cd0241b 100644
--- a/bench_test.go
+++ b/bench_test.go
@@ -2,6 +2,7 @@ package redis_test
import (
"bytes"
+ "context"
"fmt"
"strings"
"testing"
@@ -198,3 +199,140 @@ func BenchmarkZAdd(b *testing.B) {
}
})
}
+
+var clientSink *redis.Client
+
+func BenchmarkWithContext(b *testing.B) {
+ rdb := benchmarkRedisClient(10)
+ defer rdb.Close()
+
+ ctx := context.Background()
+
+ b.ResetTimer()
+ b.ReportAllocs()
+
+ for i := 0; i < b.N; i++ {
+ clientSink = rdb.WithContext(ctx)
+ }
+}
+
+var ringSink *redis.Ring
+
+func BenchmarkRingWithContext(b *testing.B) {
+ rdb := redis.NewRing(&redis.RingOptions{})
+ defer rdb.Close()
+
+ ctx := context.Background()
+
+ b.ResetTimer()
+ b.ReportAllocs()
+
+ for i := 0; i < b.N; i++ {
+ ringSink = rdb.WithContext(ctx)
+ }
+}
+
+//------------------------------------------------------------------------------
+
+func newClusterScenario() *clusterScenario {
+ return &clusterScenario{
+ ports: []string{"8220", "8221", "8222", "8223", "8224", "8225"},
+ nodeIds: make([]string, 6),
+ processes: make(map[string]*redisProcess, 6),
+ clients: make(map[string]*redis.Client, 6),
+ }
+}
+
+func BenchmarkClusterPing(b *testing.B) {
+ if testing.Short() {
+ b.Skip("skipping in short mode")
+ }
+
+ cluster := newClusterScenario()
+ if err := startCluster(cluster); err != nil {
+ b.Fatal(err)
+ }
+ defer stopCluster(cluster)
+
+ client := cluster.clusterClient(redisClusterOptions())
+ defer client.Close()
+
+ b.ResetTimer()
+
+ b.RunParallel(func(pb *testing.PB) {
+ for pb.Next() {
+ err := client.Ping().Err()
+ if err != nil {
+ b.Fatal(err)
+ }
+ }
+ })
+}
+
+func BenchmarkClusterSetString(b *testing.B) {
+ if testing.Short() {
+ b.Skip("skipping in short mode")
+ }
+
+ cluster := newClusterScenario()
+ if err := startCluster(cluster); err != nil {
+ b.Fatal(err)
+ }
+ defer stopCluster(cluster)
+
+ client := cluster.clusterClient(redisClusterOptions())
+ defer client.Close()
+
+ value := string(bytes.Repeat([]byte{'1'}, 10000))
+
+ b.ResetTimer()
+
+ b.RunParallel(func(pb *testing.PB) {
+ for pb.Next() {
+ err := client.Set("key", value, 0).Err()
+ if err != nil {
+ b.Fatal(err)
+ }
+ }
+ })
+}
+
+func BenchmarkClusterReloadState(b *testing.B) {
+ if testing.Short() {
+ b.Skip("skipping in short mode")
+ }
+
+ cluster := newClusterScenario()
+ if err := startCluster(cluster); err != nil {
+ b.Fatal(err)
+ }
+ defer stopCluster(cluster)
+
+ client := cluster.clusterClient(redisClusterOptions())
+ defer client.Close()
+
+ b.ResetTimer()
+
+ for i := 0; i < b.N; i++ {
+ err := client.ReloadState()
+ if err != nil {
+ b.Fatal(err)
+ }
+ }
+}
+
+var clusterSink *redis.ClusterClient
+
+func BenchmarkClusterWithContext(b *testing.B) {
+ rdb := redis.NewClusterClient(&redis.ClusterOptions{})
+ defer rdb.Close()
+
+ ctx := context.Background()
+
+ b.ResetTimer()
+ b.ReportAllocs()
+
+ for i := 0; i < b.N; i++ {
+ clusterSink = rdb.WithContext(ctx)
+ }
+}
diff --git a/cluster.go b/cluster.go
index 68194c8d4..cf96035c5 100644
--- a/cluster.go
+++ b/cluster.go
@@ -639,22 +639,22 @@ func (c *clusterStateHolder) ReloadOrGet() (*clusterState, error) {
//------------------------------------------------------------------------------
-// ClusterClient is a Redis Cluster client representing a pool of zero
-// or more underlying connections. It's safe for concurrent use by
-// multiple goroutines.
-type ClusterClient struct {
+type clusterClient struct {
cmdable
-
- ctx context.Context
+ hooks
opt *ClusterOptions
nodes *clusterNodes
state *clusterStateHolder
cmdsInfoCache *cmdsInfoCache
+}
- process func(Cmder) error
- processPipeline func([]Cmder) error
- processTxPipeline func([]Cmder) error
+// ClusterClient is a Redis Cluster client representing a pool of zero
+// or more underlying connections. It's safe for concurrent use by
+// multiple goroutines.
+type ClusterClient struct {
+ *clusterClient
+ ctx context.Context
}
// NewClusterClient returns a Redis Cluster client as described in
@@ -663,16 +663,14 @@ func NewClusterClient(opt *ClusterOptions) *ClusterClient {
opt.init()
c := &ClusterClient{
- opt: opt,
- nodes: newClusterNodes(opt),
+ clusterClient: &clusterClient{
+ opt: opt,
+ nodes: newClusterNodes(opt),
+ },
}
c.state = newClusterStateHolder(c.loadState)
c.cmdsInfoCache = newCmdsInfoCache(c.cmdsInfo)
- c.process = c.defaultProcess
- c.processPipeline = c.defaultProcessPipeline
- c.processTxPipeline = c.defaultProcessTxPipeline
-
c.init()
if opt.IdleCheckFrequency > 0 {
go c.reaper(opt.IdleCheckFrequency)
@@ -682,14 +680,7 @@ func NewClusterClient(opt *ClusterOptions) *ClusterClient {
}
func (c *ClusterClient) init() {
- c.cmdable.setProcessor(c.Process)
-}
-
-// ReloadState reloads cluster state. If available it calls ClusterSlots func
-// to get cluster slots information.
-func (c *ClusterClient) ReloadState() error {
- _, err := c.state.Reload()
- return err
+ c.cmdable = c.Process
}
func (c *ClusterClient) Context() context.Context {
@@ -703,15 +694,9 @@ func (c *ClusterClient) WithContext(ctx context.Context) *ClusterClient {
if ctx == nil {
panic("nil context")
}
- c2 := c.clone()
- c2.ctx = ctx
- return c2
-}
-
-func (c *ClusterClient) clone() *ClusterClient {
- cp := *c
- cp.init()
- return &cp
+ clone := *c
+ clone.ctx = ctx
+ return &clone
}
// Options returns read-only Options that were used to create the client.
@@ -719,164 +704,10 @@ func (c *ClusterClient) Options() *ClusterOptions {
return c.opt
}
-func (c *ClusterClient) retryBackoff(attempt int) time.Duration {
- return internal.RetryBackoff(attempt, c.opt.MinRetryBackoff, c.opt.MaxRetryBackoff)
-}
-
-func (c *ClusterClient) cmdsInfo() (map[string]*CommandInfo, error) {
- addrs, err := c.nodes.Addrs()
- if err != nil {
- return nil, err
- }
-
- var firstErr error
- for _, addr := range addrs {
- node, err := c.nodes.Get(addr)
- if err != nil {
- return nil, err
- }
- if node == nil {
- continue
- }
-
- info, err := node.Client.Command().Result()
- if err == nil {
- return info, nil
- }
- if firstErr == nil {
- firstErr = err
- }
- }
- return nil, firstErr
-}
-
-func (c *ClusterClient) cmdInfo(name string) *CommandInfo {
- cmdsInfo, err := c.cmdsInfoCache.Get()
- if err != nil {
- return nil
- }
-
- info := cmdsInfo[name]
- if info == nil {
- internal.Logf("info for cmd=%s not found", name)
- }
- return info
-}
-
-func cmdSlot(cmd Cmder, pos int) int {
- if pos == 0 {
- return hashtag.RandomSlot()
- }
- firstKey := cmd.stringArg(pos)
- return hashtag.Slot(firstKey)
-}
-
-func (c *ClusterClient) cmdSlot(cmd Cmder) int {
- args := cmd.Args()
- if args[0] == "cluster" && args[1] == "getkeysinslot" {
- return args[2].(int)
- }
-
- cmdInfo := c.cmdInfo(cmd.Name())
- return cmdSlot(cmd, cmdFirstKeyPos(cmd, cmdInfo))
-}
-
-func (c *ClusterClient) cmdSlotAndNode(cmd Cmder) (int, *clusterNode, error) {
- state, err := c.state.Get()
- if err != nil {
- return 0, nil, err
- }
-
- cmdInfo := c.cmdInfo(cmd.Name())
- slot := c.cmdSlot(cmd)
-
- if c.opt.ReadOnly && cmdInfo != nil && cmdInfo.ReadOnly {
- if c.opt.RouteByLatency {
- node, err := state.slotClosestNode(slot)
- return slot, node, err
- }
-
- if c.opt.RouteRandomly {
- node := state.slotRandomNode(slot)
- return slot, node, nil
- }
-
- node, err := state.slotSlaveNode(slot)
- return slot, node, err
- }
-
- node, err := state.slotMasterNode(slot)
- return slot, node, err
-}
-
-func (c *ClusterClient) slotMasterNode(slot int) (*clusterNode, error) {
- state, err := c.state.Get()
- if err != nil {
- return nil, err
- }
-
- nodes := state.slotNodes(slot)
- if len(nodes) > 0 {
- return nodes[0], nil
- }
- return c.nodes.Random()
-}
-
-func (c *ClusterClient) Watch(fn func(*Tx) error, keys ...string) error {
- if len(keys) == 0 {
- return fmt.Errorf("redis: Watch requires at least one key")
- }
-
- slot := hashtag.Slot(keys[0])
- for _, key := range keys[1:] {
- if hashtag.Slot(key) != slot {
- err := fmt.Errorf("redis: Watch requires all keys to be in the same slot")
- return err
- }
- }
-
- node, err := c.slotMasterNode(slot)
- if err != nil {
- return err
- }
-
- for attempt := 0; attempt <= c.opt.MaxRedirects; attempt++ {
- if attempt > 0 {
- time.Sleep(c.retryBackoff(attempt))
- }
-
- err = node.Client.Watch(fn, keys...)
- if err == nil {
- break
- }
- if err != Nil {
- c.state.LazyReload()
- }
-
- moved, ask, addr := internal.IsMovedError(err)
- if moved || ask {
- node, err = c.nodes.GetOrCreate(addr)
- if err != nil {
- return err
- }
- continue
- }
-
- if err == pool.ErrClosed || internal.IsReadOnlyError(err) {
- node, err = c.slotMasterNode(slot)
- if err != nil {
- return err
- }
- continue
- }
-
- if internal.IsRetryableError(err, true) {
- continue
- }
-
- return err
- }
-
+// ReloadState reloads cluster state. If available it calls ClusterSlots func
+// to get cluster slots information.
+func (c *ClusterClient) ReloadState() error {
+ _, err := c.state.Reload()
return err
}
@@ -895,17 +726,11 @@ func (c *ClusterClient) Do(args ...interface{}) *Cmd {
return cmd
}
-func (c *ClusterClient) WrapProcess(
- fn func(oldProcess func(Cmder) error) func(Cmder) error,
-) {
- c.process = fn(c.process)
-}
-
func (c *ClusterClient) Process(cmd Cmder) error {
- return c.process(cmd)
+ return c.hooks.process(c.ctx, cmd, c.process)
}
-func (c *ClusterClient) defaultProcess(cmd Cmder) error {
+func (c *ClusterClient) process(cmd Cmder) error {
var node *clusterNode
var ask bool
for attempt := 0; attempt <= c.opt.MaxRedirects; attempt++ {
@@ -1186,7 +1011,7 @@ func (c *ClusterClient) Pipeline() Pipeliner {
pipe := Pipeline{
exec: c.processPipeline,
}
- pipe.statefulCmdable.setProcessor(pipe.Process)
+ pipe.init()
return &pipe
}
@@ -1194,14 +1019,11 @@ func (c *ClusterClient) Pipelined(fn func(Pipeliner) error) ([]Cmder, error) {
return c.Pipeline().Pipelined(fn)
}
-func (c *ClusterClient) WrapProcessPipeline(
- fn func(oldProcess func([]Cmder) error) func([]Cmder) error,
-) {
- c.processPipeline = fn(c.processPipeline)
- c.processTxPipeline = fn(c.processTxPipeline)
+func (c *ClusterClient) processPipeline(cmds []Cmder) error {
+ return c.hooks.processPipeline(c.ctx, cmds, c._processPipeline)
}
-func (c *ClusterClient) defaultProcessPipeline(cmds []Cmder) error {
+func (c *ClusterClient) _processPipeline(cmds []Cmder) error {
cmdsMap := newCmdsMap()
err := c.mapCmdsByNode(cmds, cmdsMap)
if err != nil {
@@ -1383,7 +1205,7 @@ func (c *ClusterClient) TxPipeline() Pipeliner {
pipe := Pipeline{
exec: c.processTxPipeline,
}
- pipe.statefulCmdable.setProcessor(pipe.Process)
+ pipe.init()
return &pipe
}
@@ -1391,7 +1213,11 @@ func (c *ClusterClient) TxPipelined(fn func(Pipeliner) error) ([]Cmder, error) {
return c.TxPipeline().Pipelined(fn)
}
-func (c *ClusterClient) defaultProcessTxPipeline(cmds []Cmder) error {
+func (c *ClusterClient) processTxPipeline(cmds []Cmder) error {
+ return c.hooks.processPipeline(c.ctx, cmds, c._processTxPipeline)
+}
+
+func (c *ClusterClient) _processTxPipeline(cmds []Cmder) error {
state, err := c.state.Get()
if err != nil {
return err
@@ -1529,6 +1355,64 @@ func (c *ClusterClient) txPipelineReadQueued(
return nil
}
+func (c *ClusterClient) Watch(fn func(*Tx) error, keys ...string) error {
+ if len(keys) == 0 {
+ return fmt.Errorf("redis: Watch requires at least one key")
+ }
+
+ slot := hashtag.Slot(keys[0])
+ for _, key := range keys[1:] {
+ if hashtag.Slot(key) != slot {
+ err := fmt.Errorf("redis: Watch requires all keys to be in the same slot")
+ return err
+ }
+ }
+
+ node, err := c.slotMasterNode(slot)
+ if err != nil {
+ return err
+ }
+
+ for attempt := 0; attempt <= c.opt.MaxRedirects; attempt++ {
+ if attempt > 0 {
+ time.Sleep(c.retryBackoff(attempt))
+ }
+
+ err = node.Client.Watch(fn, keys...)
+ if err == nil {
+ break
+ }
+ if err != Nil {
+ c.state.LazyReload()
+ }
+
+ moved, ask, addr := internal.IsMovedError(err)
+ if moved || ask {
+ node, err = c.nodes.GetOrCreate(addr)
+ if err != nil {
+ return err
+ }
+ continue
+ }
+
+ if err == pool.ErrClosed || internal.IsReadOnlyError(err) {
+ node, err = c.slotMasterNode(slot)
+ if err != nil {
+ return err
+ }
+ continue
+ }
+
+ if internal.IsRetryableError(err, true) {
+ continue
+ }
+
+ return err
+ }
+
+ return err
+}
+
func (c *ClusterClient) pubSub() *PubSub {
var node *clusterNode
pubsub := &PubSub{
@@ -1590,6 +1474,109 @@ func (c *ClusterClient) PSubscribe(channels ...string) *PubSub {
return pubsub
}
+func (c *ClusterClient) retryBackoff(attempt int) time.Duration {
+ return internal.RetryBackoff(attempt, c.opt.MinRetryBackoff, c.opt.MaxRetryBackoff)
+}
+
+func (c *ClusterClient) cmdsInfo() (map[string]*CommandInfo, error) {
+ addrs, err := c.nodes.Addrs()
+ if err != nil {
+ return nil, err
+ }
+
+ var firstErr error
+ for _, addr := range addrs {
+ node, err := c.nodes.Get(addr)
+ if err != nil {
+ return nil, err
+ }
+ if node == nil {
+ continue
+ }
+
+ info, err := node.Client.Command().Result()
+ if err == nil {
+ return info, nil
+ }
+ if firstErr == nil {
+ firstErr = err
+ }
+ }
+ return nil, firstErr
+}
+
+func (c *ClusterClient) cmdInfo(name string) *CommandInfo {
+ cmdsInfo, err := c.cmdsInfoCache.Get()
+ if err != nil {
+ return nil
+ }
+
+ info := cmdsInfo[name]
+ if info == nil {
+ internal.Logf("info for cmd=%s not found", name)
+ }
+ return info
+}
+
+func cmdSlot(cmd Cmder, pos int) int {
+ if pos == 0 {
+ return hashtag.RandomSlot()
+ }
+ firstKey := cmd.stringArg(pos)
+ return hashtag.Slot(firstKey)
+}
+
+func (c *ClusterClient) cmdSlot(cmd Cmder) int {
+ args := cmd.Args()
+ if args[0] == "cluster" && args[1] == "getkeysinslot" {
+ return args[2].(int)
+ }
+
+ cmdInfo := c.cmdInfo(cmd.Name())
+ return cmdSlot(cmd, cmdFirstKeyPos(cmd, cmdInfo))
+}
+
+func (c *ClusterClient) cmdSlotAndNode(cmd Cmder) (int, *clusterNode, error) {
+ state, err := c.state.Get()
+ if err != nil {
+ return 0, nil, err
+ }
+
+ cmdInfo := c.cmdInfo(cmd.Name())
+ slot := c.cmdSlot(cmd)
+
+ if c.opt.ReadOnly && cmdInfo != nil && cmdInfo.ReadOnly {
+ if c.opt.RouteByLatency {
+ node, err := state.slotClosestNode(slot)
+ return slot, node, err
+ }
+
+ if c.opt.RouteRandomly {
+ node := state.slotRandomNode(slot)
+ return slot, node, nil
+ }
+
+ node, err := state.slotSlaveNode(slot)
+ return slot, node, err
+ }
+
+ node, err := state.slotMasterNode(slot)
+ return slot, node, err
+}
+
+func (c *ClusterClient) slotMasterNode(slot int) (*clusterNode, error) {
+ state, err := c.state.Get()
+ if err != nil {
+ return nil, err
+ }
+
+ nodes := state.slotNodes(slot)
+ if len(nodes) > 0 {
+ return nodes[0], nil
+ }
+ return c.nodes.Random()
+}
+
func appendUniqueNode(nodes []*clusterNode, node *clusterNode) []*clusterNode {
for _, n := range nodes {
if n == node {
diff --git a/cluster_test.go b/cluster_test.go
index f128a8a83..ad3a6be1c 100644
--- a/cluster_test.go
+++ b/cluster_test.go
@@ -1,13 +1,11 @@
package redis_test
import (
- "bytes"
"fmt"
"net"
"strconv"
"strings"
"sync"
- "testing"
"time"
"github.com/go-redis/redis"
@@ -545,18 +543,6 @@ var _ = Describe("ClusterClient", func() {
Expect(stats).To(BeAssignableToTypeOf(&redis.PoolStats{}))
})
- It("removes idle connections", func() {
- stats := client.PoolStats()
- Expect(stats.TotalConns).NotTo(BeZero())
- Expect(stats.IdleConns).NotTo(BeZero())
-
- time.Sleep(2 * time.Second)
-
- stats = client.PoolStats()
- Expect(stats.TotalConns).To(BeZero())
- Expect(stats.IdleConns).To(BeZero())
- })
-
It("returns an error when there are no attempts left", func() {
opt := redisClusterOptions()
opt.MaxRedirects = -1
@@ -1054,92 +1040,3 @@ var _ = Describe("ClusterClient timeout", func() {
testTimeout()
})
})
-
-//------------------------------------------------------------------------------
-
-func newClusterScenario() *clusterScenario {
- return &clusterScenario{
- ports: []string{"8220", "8221", "8222", "8223", "8224", "8225"},
- nodeIds: make([]string, 6),
- processes: make(map[string]*redisProcess, 6),
- clients: make(map[string]*redis.Client, 6),
- }
-}
-
-func BenchmarkClusterPing(b *testing.B) {
- if testing.Short() {
- b.Skip("skipping in short mode")
- }
-
- cluster := newClusterScenario()
- if err := startCluster(cluster); err != nil {
- b.Fatal(err)
- }
- defer stopCluster(cluster)
-
- client := cluster.clusterClient(redisClusterOptions())
- defer client.Close()
-
- b.ResetTimer()
-
- b.RunParallel(func(pb *testing.PB) {
- for pb.Next() {
- err := client.Ping().Err()
- if err != nil {
- b.Fatal(err)
- }
- }
- })
-}
-
-func BenchmarkClusterSetString(b *testing.B) {
- if testing.Short() {
- b.Skip("skipping in short mode")
- }
-
- cluster := newClusterScenario()
- if err := startCluster(cluster); err != nil {
- b.Fatal(err)
- }
- defer stopCluster(cluster)
-
- client := cluster.clusterClient(redisClusterOptions())
- defer client.Close()
-
- value := string(bytes.Repeat([]byte{'1'}, 10000))
-
- b.ResetTimer()
-
- b.RunParallel(func(pb *testing.PB) {
- for pb.Next() {
- err := client.Set("key", value, 0).Err()
- if err != nil {
- b.Fatal(err)
- }
- }
- })
-}
-
-func BenchmarkClusterReloadState(b *testing.B) {
- if testing.Short() {
- b.Skip("skipping in short mode")
- }
-
- cluster := newClusterScenario()
- if err := startCluster(cluster); err != nil {
- b.Fatal(err)
- }
- defer stopCluster(cluster)
-
- client := cluster.clusterClient(redisClusterOptions())
- defer client.Close()
-
- b.ResetTimer()
-
- for i := 0; i < b.N; i++ {
- err := client.ReloadState()
- if err != nil {
- b.Fatal(err)
- }
- }
-}
diff --git a/command.go b/command.go
index 6f0589591..177459ed8 100644
--- a/command.go
+++ b/command.go
@@ -100,8 +100,14 @@ type baseCmd struct {
var _ Cmder = (*Cmd)(nil)
-func (cmd *baseCmd) Err() error {
- return cmd.err
+func (cmd *baseCmd) Name() string {
+ if len(cmd._args) > 0 {
+ // Cmd name must be lower cased.
+ s := internal.ToLower(cmd.stringArg(0))
+ cmd._args[0] = s
+ return s
+ }
+ return ""
}
func (cmd *baseCmd) Args() []interface{} {
@@ -116,14 +122,8 @@ func (cmd *baseCmd) stringArg(pos int) string {
return s
}
-func (cmd *baseCmd) Name() string {
- if len(cmd._args) > 0 {
- // Cmd name must be lower cased.
- s := internal.ToLower(cmd.stringArg(0))
- cmd._args[0] = s
- return s
- }
- return ""
+func (cmd *baseCmd) Err() error {
+ return cmd.err
}
func (cmd *baseCmd) readTimeout() *time.Duration {
diff --git a/commands.go b/commands.go
index ffba8fd0b..aa224b57b 100644
--- a/commands.go
+++ b/commands.go
@@ -306,132 +306,118 @@ var _ Cmdable = (*Tx)(nil)
var _ Cmdable = (*Ring)(nil)
var _ Cmdable = (*ClusterClient)(nil)
-type cmdable struct {
- process func(cmd Cmder) error
-}
-
-func (c *cmdable) setProcessor(fn func(Cmder) error) {
- c.process = fn
-}
+type cmdable func(cmd Cmder) error
-type statefulCmdable struct {
- cmdable
- process func(cmd Cmder) error
-}
-
-func (c *statefulCmdable) setProcessor(fn func(Cmder) error) {
- c.process = fn
- c.cmdable.setProcessor(fn)
-}
+type statefulCmdable func(cmd Cmder) error
//------------------------------------------------------------------------------
-func (c *statefulCmdable) Auth(password string) *StatusCmd {
+func (c statefulCmdable) Auth(password string) *StatusCmd {
cmd := NewStatusCmd("auth", password)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Echo(message interface{}) *StringCmd {
+func (c cmdable) Echo(message interface{}) *StringCmd {
cmd := NewStringCmd("echo", message)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Ping() *StatusCmd {
+func (c cmdable) Ping() *StatusCmd {
cmd := NewStatusCmd("ping")
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Wait(numSlaves int, timeout time.Duration) *IntCmd {
+func (c cmdable) Wait(numSlaves int, timeout time.Duration) *IntCmd {
cmd := NewIntCmd("wait", numSlaves, int(timeout/time.Millisecond))
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Quit() *StatusCmd {
+func (c cmdable) Quit() *StatusCmd {
panic("not implemented")
}
-func (c *statefulCmdable) Select(index int) *StatusCmd {
+func (c statefulCmdable) Select(index int) *StatusCmd {
cmd := NewStatusCmd("select", index)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *statefulCmdable) SwapDB(index1, index2 int) *StatusCmd {
+func (c statefulCmdable) SwapDB(index1, index2 int) *StatusCmd {
cmd := NewStatusCmd("swapdb", index1, index2)
- c.process(cmd)
+ c(cmd)
return cmd
}
//------------------------------------------------------------------------------
-func (c *cmdable) Command() *CommandsInfoCmd {
+func (c cmdable) Command() *CommandsInfoCmd {
cmd := NewCommandsInfoCmd("command")
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Del(keys ...string) *IntCmd {
+func (c cmdable) Del(keys ...string) *IntCmd {
args := make([]interface{}, 1+len(keys))
args[0] = "del"
for i, key := range keys {
args[1+i] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Unlink(keys ...string) *IntCmd {
+func (c cmdable) Unlink(keys ...string) *IntCmd {
args := make([]interface{}, 1+len(keys))
args[0] = "unlink"
for i, key := range keys {
args[1+i] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Dump(key string) *StringCmd {
+func (c cmdable) Dump(key string) *StringCmd {
cmd := NewStringCmd("dump", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Exists(keys ...string) *IntCmd {
+func (c cmdable) Exists(keys ...string) *IntCmd {
args := make([]interface{}, 1+len(keys))
args[0] = "exists"
for i, key := range keys {
args[1+i] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Expire(key string, expiration time.Duration) *BoolCmd {
+func (c cmdable) Expire(key string, expiration time.Duration) *BoolCmd {
cmd := NewBoolCmd("expire", key, formatSec(expiration))
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ExpireAt(key string, tm time.Time) *BoolCmd {
+func (c cmdable) ExpireAt(key string, tm time.Time) *BoolCmd {
cmd := NewBoolCmd("expireat", key, tm.Unix())
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Keys(pattern string) *StringSliceCmd {
+func (c cmdable) Keys(pattern string) *StringSliceCmd {
cmd := NewStringSliceCmd("keys", pattern)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Migrate(host, port, key string, db int, timeout time.Duration) *StatusCmd {
+func (c cmdable) Migrate(host, port, key string, db int, timeout time.Duration) *StatusCmd {
cmd := NewStatusCmd(
"migrate",
host,
@@ -441,92 +427,92 @@ func (c *cmdable) Migrate(host, port, key string, db int, timeout time.Duration)
formatMs(timeout),
)
cmd.setReadTimeout(timeout)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Move(key string, db int) *BoolCmd {
+func (c cmdable) Move(key string, db int) *BoolCmd {
cmd := NewBoolCmd("move", key, db)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ObjectRefCount(key string) *IntCmd {
+func (c cmdable) ObjectRefCount(key string) *IntCmd {
cmd := NewIntCmd("object", "refcount", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ObjectEncoding(key string) *StringCmd {
+func (c cmdable) ObjectEncoding(key string) *StringCmd {
cmd := NewStringCmd("object", "encoding", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ObjectIdleTime(key string) *DurationCmd {
+func (c cmdable) ObjectIdleTime(key string) *DurationCmd {
cmd := NewDurationCmd(time.Second, "object", "idletime", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Persist(key string) *BoolCmd {
+func (c cmdable) Persist(key string) *BoolCmd {
cmd := NewBoolCmd("persist", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) PExpire(key string, expiration time.Duration) *BoolCmd {
+func (c cmdable) PExpire(key string, expiration time.Duration) *BoolCmd {
cmd := NewBoolCmd("pexpire", key, formatMs(expiration))
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) PExpireAt(key string, tm time.Time) *BoolCmd {
+func (c cmdable) PExpireAt(key string, tm time.Time) *BoolCmd {
cmd := NewBoolCmd(
"pexpireat",
key,
tm.UnixNano()/int64(time.Millisecond),
)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) PTTL(key string) *DurationCmd {
+func (c cmdable) PTTL(key string) *DurationCmd {
cmd := NewDurationCmd(time.Millisecond, "pttl", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) RandomKey() *StringCmd {
+func (c cmdable) RandomKey() *StringCmd {
cmd := NewStringCmd("randomkey")
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Rename(key, newkey string) *StatusCmd {
+func (c cmdable) Rename(key, newkey string) *StatusCmd {
cmd := NewStatusCmd("rename", key, newkey)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) RenameNX(key, newkey string) *BoolCmd {
+func (c cmdable) RenameNX(key, newkey string) *BoolCmd {
cmd := NewBoolCmd("renamenx", key, newkey)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Restore(key string, ttl time.Duration, value string) *StatusCmd {
+func (c cmdable) Restore(key string, ttl time.Duration, value string) *StatusCmd {
cmd := NewStatusCmd(
"restore",
key,
formatMs(ttl),
value,
)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) RestoreReplace(key string, ttl time.Duration, value string) *StatusCmd {
+func (c cmdable) RestoreReplace(key string, ttl time.Duration, value string) *StatusCmd {
cmd := NewStatusCmd(
"restore",
key,
@@ -534,7 +520,7 @@ func (c *cmdable) RestoreReplace(key string, ttl time.Duration, value string) *S
value,
"replace",
)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -566,52 +552,52 @@ func (sort *Sort) args(key string) []interface{} {
return args
}
-func (c *cmdable) Sort(key string, sort *Sort) *StringSliceCmd {
+func (c cmdable) Sort(key string, sort *Sort) *StringSliceCmd {
cmd := NewStringSliceCmd(sort.args(key)...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SortStore(key, store string, sort *Sort) *IntCmd {
+func (c cmdable) SortStore(key, store string, sort *Sort) *IntCmd {
args := sort.args(key)
if store != "" {
args = append(args, "store", store)
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SortInterfaces(key string, sort *Sort) *SliceCmd {
+func (c cmdable) SortInterfaces(key string, sort *Sort) *SliceCmd {
cmd := NewSliceCmd(sort.args(key)...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Touch(keys ...string) *IntCmd {
+func (c cmdable) Touch(keys ...string) *IntCmd {
args := make([]interface{}, len(keys)+1)
args[0] = "touch"
for i, key := range keys {
args[i+1] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) TTL(key string) *DurationCmd {
+func (c cmdable) TTL(key string) *DurationCmd {
cmd := NewDurationCmd(time.Second, "ttl", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Type(key string) *StatusCmd {
+func (c cmdable) Type(key string) *StatusCmd {
cmd := NewStatusCmd("type", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Scan(cursor uint64, match string, count int64) *ScanCmd {
+func (c cmdable) Scan(cursor uint64, match string, count int64) *ScanCmd {
args := []interface{}{"scan", cursor}
if match != "" {
args = append(args, "match", match)
@@ -619,12 +605,12 @@ func (c *cmdable) Scan(cursor uint64, match string, count int64) *ScanCmd {
if count > 0 {
args = append(args, "count", count)
}
- cmd := NewScanCmd(c.process, args...)
- c.process(cmd)
+ cmd := NewScanCmd(c, args...)
+ c(cmd)
return cmd
}
-func (c *cmdable) SScan(key string, cursor uint64, match string, count int64) *ScanCmd {
+func (c cmdable) SScan(key string, cursor uint64, match string, count int64) *ScanCmd {
args := []interface{}{"sscan", key, cursor}
if match != "" {
args = append(args, "match", match)
@@ -632,12 +618,12 @@ func (c *cmdable) SScan(key string, cursor uint64, match string, count int64) *S
if count > 0 {
args = append(args, "count", count)
}
- cmd := NewScanCmd(c.process, args...)
- c.process(cmd)
+ cmd := NewScanCmd(c, args...)
+ c(cmd)
return cmd
}
-func (c *cmdable) HScan(key string, cursor uint64, match string, count int64) *ScanCmd {
+func (c cmdable) HScan(key string, cursor uint64, match string, count int64) *ScanCmd {
args := []interface{}{"hscan", key, cursor}
if match != "" {
args = append(args, "match", match)
@@ -645,12 +631,12 @@ func (c *cmdable) HScan(key string, cursor uint64, match string, count int64) *S
if count > 0 {
args = append(args, "count", count)
}
- cmd := NewScanCmd(c.process, args...)
- c.process(cmd)
+ cmd := NewScanCmd(c, args...)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZScan(key string, cursor uint64, match string, count int64) *ScanCmd {
+func (c cmdable) ZScan(key string, cursor uint64, match string, count int64) *ScanCmd {
args := []interface{}{"zscan", key, cursor}
if match != "" {
args = append(args, "match", match)
@@ -658,16 +644,16 @@ func (c *cmdable) ZScan(key string, cursor uint64, match string, count int64) *S
if count > 0 {
args = append(args, "count", count)
}
- cmd := NewScanCmd(c.process, args...)
- c.process(cmd)
+ cmd := NewScanCmd(c, args...)
+ c(cmd)
return cmd
}
//------------------------------------------------------------------------------
-func (c *cmdable) Append(key, value string) *IntCmd {
+func (c cmdable) Append(key, value string) *IntCmd {
cmd := NewIntCmd("append", key, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -675,7 +661,7 @@ type BitCount struct {
Start, End int64
}
-func (c *cmdable) BitCount(key string, bitCount *BitCount) *IntCmd {
+func (c cmdable) BitCount(key string, bitCount *BitCount) *IntCmd {
args := []interface{}{"bitcount", key}
if bitCount != nil {
args = append(
@@ -685,11 +671,11 @@ func (c *cmdable) BitCount(key string, bitCount *BitCount) *IntCmd {
)
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) bitOp(op, destKey string, keys ...string) *IntCmd {
+func (c cmdable) bitOp(op, destKey string, keys ...string) *IntCmd {
args := make([]interface{}, 3+len(keys))
args[0] = "bitop"
args[1] = op
@@ -698,27 +684,27 @@ func (c *cmdable) bitOp(op, destKey string, keys ...string) *IntCmd {
args[3+i] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) BitOpAnd(destKey string, keys ...string) *IntCmd {
+func (c cmdable) BitOpAnd(destKey string, keys ...string) *IntCmd {
return c.bitOp("and", destKey, keys...)
}
-func (c *cmdable) BitOpOr(destKey string, keys ...string) *IntCmd {
+func (c cmdable) BitOpOr(destKey string, keys ...string) *IntCmd {
return c.bitOp("or", destKey, keys...)
}
-func (c *cmdable) BitOpXor(destKey string, keys ...string) *IntCmd {
+func (c cmdable) BitOpXor(destKey string, keys ...string) *IntCmd {
return c.bitOp("xor", destKey, keys...)
}
-func (c *cmdable) BitOpNot(destKey string, key string) *IntCmd {
+func (c cmdable) BitOpNot(destKey string, key string) *IntCmd {
return c.bitOp("not", destKey, key)
}
-func (c *cmdable) BitPos(key string, bit int64, pos ...int64) *IntCmd {
+func (c cmdable) BitPos(key string, bit int64, pos ...int64) *IntCmd {
args := make([]interface{}, 3+len(pos))
args[0] = "bitpos"
args[1] = key
@@ -734,91 +720,91 @@ func (c *cmdable) BitPos(key string, bit int64, pos ...int64) *IntCmd {
panic("too many arguments")
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Decr(key string) *IntCmd {
+func (c cmdable) Decr(key string) *IntCmd {
cmd := NewIntCmd("decr", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) DecrBy(key string, decrement int64) *IntCmd {
+func (c cmdable) DecrBy(key string, decrement int64) *IntCmd {
cmd := NewIntCmd("decrby", key, decrement)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `GET key` command. It returns redis.Nil error when key does not exist.
-func (c *cmdable) Get(key string) *StringCmd {
+func (c cmdable) Get(key string) *StringCmd {
cmd := NewStringCmd("get", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) GetBit(key string, offset int64) *IntCmd {
+func (c cmdable) GetBit(key string, offset int64) *IntCmd {
cmd := NewIntCmd("getbit", key, offset)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) GetRange(key string, start, end int64) *StringCmd {
+func (c cmdable) GetRange(key string, start, end int64) *StringCmd {
cmd := NewStringCmd("getrange", key, start, end)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) GetSet(key string, value interface{}) *StringCmd {
+func (c cmdable) GetSet(key string, value interface{}) *StringCmd {
cmd := NewStringCmd("getset", key, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) Incr(key string) *IntCmd {
+func (c cmdable) Incr(key string) *IntCmd {
cmd := NewIntCmd("incr", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) IncrBy(key string, value int64) *IntCmd {
+func (c cmdable) IncrBy(key string, value int64) *IntCmd {
cmd := NewIntCmd("incrby", key, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) IncrByFloat(key string, value float64) *FloatCmd {
+func (c cmdable) IncrByFloat(key string, value float64) *FloatCmd {
cmd := NewFloatCmd("incrbyfloat", key, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) MGet(keys ...string) *SliceCmd {
+func (c cmdable) MGet(keys ...string) *SliceCmd {
args := make([]interface{}, 1+len(keys))
args[0] = "mget"
for i, key := range keys {
args[1+i] = key
}
cmd := NewSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) MSet(pairs ...interface{}) *StatusCmd {
+func (c cmdable) MSet(pairs ...interface{}) *StatusCmd {
args := make([]interface{}, 1, 1+len(pairs))
args[0] = "mset"
args = appendArgs(args, pairs)
cmd := NewStatusCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) MSetNX(pairs ...interface{}) *BoolCmd {
+func (c cmdable) MSetNX(pairs ...interface{}) *BoolCmd {
args := make([]interface{}, 1, 1+len(pairs))
args[0] = "msetnx"
args = appendArgs(args, pairs)
cmd := NewBoolCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -826,7 +812,7 @@ func (c *cmdable) MSetNX(pairs ...interface{}) *BoolCmd {
//
// Use expiration for `SETEX`-like behavior.
// Zero expiration means the key has no expiration time.
-func (c *cmdable) Set(key string, value interface{}, expiration time.Duration) *StatusCmd {
+func (c cmdable) Set(key string, value interface{}, expiration time.Duration) *StatusCmd {
args := make([]interface{}, 3, 4)
args[0] = "set"
args[1] = key
@@ -839,25 +825,25 @@ func (c *cmdable) Set(key string, value interface{}, expiration time.Duration) *
}
}
cmd := NewStatusCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SetBit(key string, offset int64, value int) *IntCmd {
+func (c cmdable) SetBit(key string, offset int64, value int) *IntCmd {
cmd := NewIntCmd(
"setbit",
key,
offset,
value,
)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `SET key value [expiration] NX` command.
//
// Zero expiration means the key has no expiration time.
-func (c *cmdable) SetNX(key string, value interface{}, expiration time.Duration) *BoolCmd {
+func (c cmdable) SetNX(key string, value interface{}, expiration time.Duration) *BoolCmd {
var cmd *BoolCmd
if expiration == 0 {
// Use old `SETNX` to support old Redis versions.
@@ -869,14 +855,14 @@ func (c *cmdable) SetNX(key string, value interface{}, expiration time.Duration)
cmd = NewBoolCmd("set", key, value, "ex", formatSec(expiration), "nx")
}
}
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `SET key value [expiration] XX` command.
//
// Zero expiration means the key has no expiration time.
-func (c *cmdable) SetXX(key string, value interface{}, expiration time.Duration) *BoolCmd {
+func (c cmdable) SetXX(key string, value interface{}, expiration time.Duration) *BoolCmd {
var cmd *BoolCmd
if expiration == 0 {
cmd = NewBoolCmd("set", key, value, "xx")
@@ -887,25 +873,25 @@ func (c *cmdable) SetXX(key string, value interface{}, expiration time.Duration)
cmd = NewBoolCmd("set", key, value, "ex", formatSec(expiration), "xx")
}
}
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SetRange(key string, offset int64, value string) *IntCmd {
+func (c cmdable) SetRange(key string, offset int64, value string) *IntCmd {
cmd := NewIntCmd("setrange", key, offset, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) StrLen(key string) *IntCmd {
+func (c cmdable) StrLen(key string) *IntCmd {
cmd := NewIntCmd("strlen", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
//------------------------------------------------------------------------------
-func (c *cmdable) HDel(key string, fields ...string) *IntCmd {
+func (c cmdable) HDel(key string, fields ...string) *IntCmd {
args := make([]interface{}, 2+len(fields))
args[0] = "hdel"
args[1] = key
@@ -913,53 +899,53 @@ func (c *cmdable) HDel(key string, fields ...string) *IntCmd {
args[2+i] = field
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HExists(key, field string) *BoolCmd {
+func (c cmdable) HExists(key, field string) *BoolCmd {
cmd := NewBoolCmd("hexists", key, field)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HGet(key, field string) *StringCmd {
+func (c cmdable) HGet(key, field string) *StringCmd {
cmd := NewStringCmd("hget", key, field)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HGetAll(key string) *StringStringMapCmd {
+func (c cmdable) HGetAll(key string) *StringStringMapCmd {
cmd := NewStringStringMapCmd("hgetall", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HIncrBy(key, field string, incr int64) *IntCmd {
+func (c cmdable) HIncrBy(key, field string, incr int64) *IntCmd {
cmd := NewIntCmd("hincrby", key, field, incr)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HIncrByFloat(key, field string, incr float64) *FloatCmd {
+func (c cmdable) HIncrByFloat(key, field string, incr float64) *FloatCmd {
cmd := NewFloatCmd("hincrbyfloat", key, field, incr)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HKeys(key string) *StringSliceCmd {
+func (c cmdable) HKeys(key string) *StringSliceCmd {
cmd := NewStringSliceCmd("hkeys", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HLen(key string) *IntCmd {
+func (c cmdable) HLen(key string) *IntCmd {
cmd := NewIntCmd("hlen", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HMGet(key string, fields ...string) *SliceCmd {
+func (c cmdable) HMGet(key string, fields ...string) *SliceCmd {
args := make([]interface{}, 2+len(fields))
args[0] = "hmget"
args[1] = key
@@ -967,11 +953,11 @@ func (c *cmdable) HMGet(key string, fields ...string) *SliceCmd {
args[2+i] = field
}
cmd := NewSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HMSet(key string, fields map[string]interface{}) *StatusCmd {
+func (c cmdable) HMSet(key string, fields map[string]interface{}) *StatusCmd {
args := make([]interface{}, 2+len(fields)*2)
args[0] = "hmset"
args[1] = key
@@ -982,31 +968,31 @@ func (c *cmdable) HMSet(key string, fields map[string]interface{}) *StatusCmd {
i += 2
}
cmd := NewStatusCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HSet(key, field string, value interface{}) *BoolCmd {
+func (c cmdable) HSet(key, field string, value interface{}) *BoolCmd {
cmd := NewBoolCmd("hset", key, field, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HSetNX(key, field string, value interface{}) *BoolCmd {
+func (c cmdable) HSetNX(key, field string, value interface{}) *BoolCmd {
cmd := NewBoolCmd("hsetnx", key, field, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) HVals(key string) *StringSliceCmd {
+func (c cmdable) HVals(key string) *StringSliceCmd {
cmd := NewStringSliceCmd("hvals", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
//------------------------------------------------------------------------------
-func (c *cmdable) BLPop(timeout time.Duration, keys ...string) *StringSliceCmd {
+func (c cmdable) BLPop(timeout time.Duration, keys ...string) *StringSliceCmd {
args := make([]interface{}, 1+len(keys)+1)
args[0] = "blpop"
for i, key := range keys {
@@ -1015,11 +1001,11 @@ func (c *cmdable) BLPop(timeout time.Duration, keys ...string) *StringSliceCmd {
args[len(args)-1] = formatSec(timeout)
cmd := NewStringSliceCmd(args...)
cmd.setReadTimeout(timeout)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) BRPop(timeout time.Duration, keys ...string) *StringSliceCmd {
+func (c cmdable) BRPop(timeout time.Duration, keys ...string) *StringSliceCmd {
args := make([]interface{}, 1+len(keys)+1)
args[0] = "brpop"
for i, key := range keys {
@@ -1028,11 +1014,11 @@ func (c *cmdable) BRPop(timeout time.Duration, keys ...string) *StringSliceCmd {
args[len(keys)+1] = formatSec(timeout)
cmd := NewStringSliceCmd(args...)
cmd.setReadTimeout(timeout)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) BRPopLPush(source, destination string, timeout time.Duration) *StringCmd {
+func (c cmdable) BRPopLPush(source, destination string, timeout time.Duration) *StringCmd {
cmd := NewStringCmd(
"brpoplpush",
source,
@@ -1040,154 +1026,154 @@ func (c *cmdable) BRPopLPush(source, destination string, timeout time.Duration)
formatSec(timeout),
)
cmd.setReadTimeout(timeout)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LIndex(key string, index int64) *StringCmd {
+func (c cmdable) LIndex(key string, index int64) *StringCmd {
cmd := NewStringCmd("lindex", key, index)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LInsert(key, op string, pivot, value interface{}) *IntCmd {
+func (c cmdable) LInsert(key, op string, pivot, value interface{}) *IntCmd {
cmd := NewIntCmd("linsert", key, op, pivot, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LInsertBefore(key string, pivot, value interface{}) *IntCmd {
+func (c cmdable) LInsertBefore(key string, pivot, value interface{}) *IntCmd {
cmd := NewIntCmd("linsert", key, "before", pivot, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LInsertAfter(key string, pivot, value interface{}) *IntCmd {
+func (c cmdable) LInsertAfter(key string, pivot, value interface{}) *IntCmd {
cmd := NewIntCmd("linsert", key, "after", pivot, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LLen(key string) *IntCmd {
+func (c cmdable) LLen(key string) *IntCmd {
cmd := NewIntCmd("llen", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LPop(key string) *StringCmd {
+func (c cmdable) LPop(key string) *StringCmd {
cmd := NewStringCmd("lpop", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LPush(key string, values ...interface{}) *IntCmd {
+func (c cmdable) LPush(key string, values ...interface{}) *IntCmd {
args := make([]interface{}, 2, 2+len(values))
args[0] = "lpush"
args[1] = key
args = appendArgs(args, values)
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LPushX(key string, value interface{}) *IntCmd {
+func (c cmdable) LPushX(key string, value interface{}) *IntCmd {
cmd := NewIntCmd("lpushx", key, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LRange(key string, start, stop int64) *StringSliceCmd {
+func (c cmdable) LRange(key string, start, stop int64) *StringSliceCmd {
cmd := NewStringSliceCmd(
"lrange",
key,
start,
stop,
)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LRem(key string, count int64, value interface{}) *IntCmd {
+func (c cmdable) LRem(key string, count int64, value interface{}) *IntCmd {
cmd := NewIntCmd("lrem", key, count, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LSet(key string, index int64, value interface{}) *StatusCmd {
+func (c cmdable) LSet(key string, index int64, value interface{}) *StatusCmd {
cmd := NewStatusCmd("lset", key, index, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) LTrim(key string, start, stop int64) *StatusCmd {
+func (c cmdable) LTrim(key string, start, stop int64) *StatusCmd {
cmd := NewStatusCmd(
"ltrim",
key,
start,
stop,
)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) RPop(key string) *StringCmd {
+func (c cmdable) RPop(key string) *StringCmd {
cmd := NewStringCmd("rpop", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) RPopLPush(source, destination string) *StringCmd {
+func (c cmdable) RPopLPush(source, destination string) *StringCmd {
cmd := NewStringCmd("rpoplpush", source, destination)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) RPush(key string, values ...interface{}) *IntCmd {
+func (c cmdable) RPush(key string, values ...interface{}) *IntCmd {
args := make([]interface{}, 2, 2+len(values))
args[0] = "rpush"
args[1] = key
args = appendArgs(args, values)
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) RPushX(key string, value interface{}) *IntCmd {
+func (c cmdable) RPushX(key string, value interface{}) *IntCmd {
cmd := NewIntCmd("rpushx", key, value)
- c.process(cmd)
+ c(cmd)
return cmd
}
//------------------------------------------------------------------------------
-func (c *cmdable) SAdd(key string, members ...interface{}) *IntCmd {
+func (c cmdable) SAdd(key string, members ...interface{}) *IntCmd {
args := make([]interface{}, 2, 2+len(members))
args[0] = "sadd"
args[1] = key
args = appendArgs(args, members)
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SCard(key string) *IntCmd {
+func (c cmdable) SCard(key string) *IntCmd {
cmd := NewIntCmd("scard", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SDiff(keys ...string) *StringSliceCmd {
+func (c cmdable) SDiff(keys ...string) *StringSliceCmd {
args := make([]interface{}, 1+len(keys))
args[0] = "sdiff"
for i, key := range keys {
args[1+i] = key
}
cmd := NewStringSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SDiffStore(destination string, keys ...string) *IntCmd {
+func (c cmdable) SDiffStore(destination string, keys ...string) *IntCmd {
args := make([]interface{}, 2+len(keys))
args[0] = "sdiffstore"
args[1] = destination
@@ -1195,22 +1181,22 @@ func (c *cmdable) SDiffStore(destination string, keys ...string) *IntCmd {
args[2+i] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SInter(keys ...string) *StringSliceCmd {
+func (c cmdable) SInter(keys ...string) *StringSliceCmd {
args := make([]interface{}, 1+len(keys))
args[0] = "sinter"
for i, key := range keys {
args[1+i] = key
}
cmd := NewStringSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SInterStore(destination string, keys ...string) *IntCmd {
+func (c cmdable) SInterStore(destination string, keys ...string) *IntCmd {
args := make([]interface{}, 2+len(keys))
args[0] = "sinterstore"
args[1] = destination
@@ -1218,86 +1204,86 @@ func (c *cmdable) SInterStore(destination string, keys ...string) *IntCmd {
args[2+i] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SIsMember(key string, member interface{}) *BoolCmd {
+func (c cmdable) SIsMember(key string, member interface{}) *BoolCmd {
cmd := NewBoolCmd("sismember", key, member)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `SMEMBERS key` command output as a slice
-func (c *cmdable) SMembers(key string) *StringSliceCmd {
+func (c cmdable) SMembers(key string) *StringSliceCmd {
cmd := NewStringSliceCmd("smembers", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `SMEMBERS key` command output as a map
-func (c *cmdable) SMembersMap(key string) *StringStructMapCmd {
+func (c cmdable) SMembersMap(key string) *StringStructMapCmd {
cmd := NewStringStructMapCmd("smembers", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SMove(source, destination string, member interface{}) *BoolCmd {
+func (c cmdable) SMove(source, destination string, member interface{}) *BoolCmd {
cmd := NewBoolCmd("smove", source, destination, member)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `SPOP key` command.
-func (c *cmdable) SPop(key string) *StringCmd {
+func (c cmdable) SPop(key string) *StringCmd {
cmd := NewStringCmd("spop", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `SPOP key count` command.
-func (c *cmdable) SPopN(key string, count int64) *StringSliceCmd {
+func (c cmdable) SPopN(key string, count int64) *StringSliceCmd {
cmd := NewStringSliceCmd("spop", key, count)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `SRANDMEMBER key` command.
-func (c *cmdable) SRandMember(key string) *StringCmd {
+func (c cmdable) SRandMember(key string) *StringCmd {
cmd := NewStringCmd("srandmember", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `SRANDMEMBER key count` command.
-func (c *cmdable) SRandMemberN(key string, count int64) *StringSliceCmd {
+func (c cmdable) SRandMemberN(key string, count int64) *StringSliceCmd {
cmd := NewStringSliceCmd("srandmember", key, count)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SRem(key string, members ...interface{}) *IntCmd {
+func (c cmdable) SRem(key string, members ...interface{}) *IntCmd {
args := make([]interface{}, 2, 2+len(members))
args[0] = "srem"
args[1] = key
args = appendArgs(args, members)
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SUnion(keys ...string) *StringSliceCmd {
+func (c cmdable) SUnion(keys ...string) *StringSliceCmd {
args := make([]interface{}, 1+len(keys))
args[0] = "sunion"
for i, key := range keys {
args[1+i] = key
}
cmd := NewStringSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) SUnionStore(destination string, keys ...string) *IntCmd {
+func (c cmdable) SUnionStore(destination string, keys ...string) *IntCmd {
args := make([]interface{}, 2+len(keys))
args[0] = "sunionstore"
args[1] = destination
@@ -1305,7 +1291,7 @@ func (c *cmdable) SUnionStore(destination string, keys ...string) *IntCmd {
args[2+i] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -1319,7 +1305,7 @@ type XAddArgs struct {
Values map[string]interface{}
}
-func (c *cmdable) XAdd(a *XAddArgs) *StringCmd {
+func (c cmdable) XAdd(a *XAddArgs) *StringCmd {
args := make([]interface{}, 0, 6+len(a.Values)*2)
args = append(args, "xadd")
args = append(args, a.Stream)
@@ -1339,47 +1325,47 @@ func (c *cmdable) XAdd(a *XAddArgs) *StringCmd {
}
cmd := NewStringCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XDel(stream string, ids ...string) *IntCmd {
+func (c cmdable) XDel(stream string, ids ...string) *IntCmd {
args := []interface{}{"xdel", stream}
for _, id := range ids {
args = append(args, id)
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XLen(stream string) *IntCmd {
+func (c cmdable) XLen(stream string) *IntCmd {
cmd := NewIntCmd("xlen", stream)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XRange(stream, start, stop string) *XMessageSliceCmd {
+func (c cmdable) XRange(stream, start, stop string) *XMessageSliceCmd {
cmd := NewXMessageSliceCmd("xrange", stream, start, stop)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XRangeN(stream, start, stop string, count int64) *XMessageSliceCmd {
+func (c cmdable) XRangeN(stream, start, stop string, count int64) *XMessageSliceCmd {
cmd := NewXMessageSliceCmd("xrange", stream, start, stop, "count", count)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XRevRange(stream, start, stop string) *XMessageSliceCmd {
+func (c cmdable) XRevRange(stream, start, stop string) *XMessageSliceCmd {
cmd := NewXMessageSliceCmd("xrevrange", stream, start, stop)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XRevRangeN(stream, start, stop string, count int64) *XMessageSliceCmd {
+func (c cmdable) XRevRangeN(stream, start, stop string, count int64) *XMessageSliceCmd {
cmd := NewXMessageSliceCmd("xrevrange", stream, start, stop, "count", count)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -1389,7 +1375,7 @@ type XReadArgs struct {
Block time.Duration
}
-func (c *cmdable) XRead(a *XReadArgs) *XStreamSliceCmd {
+func (c cmdable) XRead(a *XReadArgs) *XStreamSliceCmd {
args := make([]interface{}, 0, 5+len(a.Streams))
args = append(args, "xread")
if a.Count > 0 {
@@ -1409,44 +1395,44 @@ func (c *cmdable) XRead(a *XReadArgs) *XStreamSliceCmd {
if a.Block >= 0 {
cmd.setReadTimeout(a.Block)
}
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XReadStreams(streams ...string) *XStreamSliceCmd {
+func (c cmdable) XReadStreams(streams ...string) *XStreamSliceCmd {
return c.XRead(&XReadArgs{
Streams: streams,
Block: -1,
})
}
-func (c *cmdable) XGroupCreate(stream, group, start string) *StatusCmd {
+func (c cmdable) XGroupCreate(stream, group, start string) *StatusCmd {
cmd := NewStatusCmd("xgroup", "create", stream, group, start)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XGroupCreateMkStream(stream, group, start string) *StatusCmd {
+func (c cmdable) XGroupCreateMkStream(stream, group, start string) *StatusCmd {
cmd := NewStatusCmd("xgroup", "create", stream, group, start, "mkstream")
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XGroupSetID(stream, group, start string) *StatusCmd {
+func (c cmdable) XGroupSetID(stream, group, start string) *StatusCmd {
cmd := NewStatusCmd("xgroup", "setid", stream, group, start)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XGroupDestroy(stream, group string) *IntCmd {
+func (c cmdable) XGroupDestroy(stream, group string) *IntCmd {
cmd := NewIntCmd("xgroup", "destroy", stream, group)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XGroupDelConsumer(stream, group, consumer string) *IntCmd {
+func (c cmdable) XGroupDelConsumer(stream, group, consumer string) *IntCmd {
cmd := NewIntCmd("xgroup", "delconsumer", stream, group, consumer)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -1459,7 +1445,7 @@ type XReadGroupArgs struct {
NoAck bool
}
-func (c *cmdable) XReadGroup(a *XReadGroupArgs) *XStreamSliceCmd {
+func (c cmdable) XReadGroup(a *XReadGroupArgs) *XStreamSliceCmd {
args := make([]interface{}, 0, 8+len(a.Streams))
args = append(args, "xreadgroup", "group", a.Group, a.Consumer)
if a.Count > 0 {
@@ -1480,23 +1466,23 @@ func (c *cmdable) XReadGroup(a *XReadGroupArgs) *XStreamSliceCmd {
if a.Block >= 0 {
cmd.setReadTimeout(a.Block)
}
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XAck(stream, group string, ids ...string) *IntCmd {
+func (c cmdable) XAck(stream, group string, ids ...string) *IntCmd {
args := []interface{}{"xack", stream, group}
for _, id := range ids {
args = append(args, id)
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XPending(stream, group string) *XPendingCmd {
+func (c cmdable) XPending(stream, group string) *XPendingCmd {
cmd := NewXPendingCmd("xpending", stream, group)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -1509,14 +1495,14 @@ type XPendingExtArgs struct {
Consumer string
}
-func (c *cmdable) XPendingExt(a *XPendingExtArgs) *XPendingExtCmd {
+func (c cmdable) XPendingExt(a *XPendingExtArgs) *XPendingExtCmd {
args := make([]interface{}, 0, 7)
args = append(args, "xpending", a.Stream, a.Group, a.Start, a.End, a.Count)
if a.Consumer != "" {
args = append(args, a.Consumer)
}
cmd := NewXPendingExtCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -1528,18 +1514,18 @@ type XClaimArgs struct {
Messages []string
}
-func (c *cmdable) XClaim(a *XClaimArgs) *XMessageSliceCmd {
+func (c cmdable) XClaim(a *XClaimArgs) *XMessageSliceCmd {
args := xClaimArgs(a)
cmd := NewXMessageSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XClaimJustID(a *XClaimArgs) *StringSliceCmd {
+func (c cmdable) XClaimJustID(a *XClaimArgs) *StringSliceCmd {
args := xClaimArgs(a)
args = append(args, "justid")
cmd := NewStringSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -1556,15 +1542,15 @@ func xClaimArgs(a *XClaimArgs) []interface{} {
return args
}
-func (c *cmdable) XTrim(key string, maxLen int64) *IntCmd {
+func (c cmdable) XTrim(key string, maxLen int64) *IntCmd {
cmd := NewIntCmd("xtrim", key, "maxlen", maxLen)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) XTrimApprox(key string, maxLen int64) *IntCmd {
+func (c cmdable) XTrimApprox(key string, maxLen int64) *IntCmd {
cmd := NewIntCmd("xtrim", key, "maxlen", "~", maxLen)
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -1590,7 +1576,7 @@ type ZStore struct {
}
// Redis `BZPOPMAX key [key ...] timeout` command.
-func (c *cmdable) BZPopMax(timeout time.Duration, keys ...string) *ZWithKeyCmd {
+func (c cmdable) BZPopMax(timeout time.Duration, keys ...string) *ZWithKeyCmd {
args := make([]interface{}, 1+len(keys)+1)
args[0] = "bzpopmax"
for i, key := range keys {
@@ -1599,12 +1585,12 @@ func (c *cmdable) BZPopMax(timeout time.Duration, keys ...string) *ZWithKeyCmd {
args[len(args)-1] = formatSec(timeout)
cmd := NewZWithKeyCmd(args...)
cmd.setReadTimeout(timeout)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `BZPOPMIN key [key ...] timeout` command.
-func (c *cmdable) BZPopMin(timeout time.Duration, keys ...string) *ZWithKeyCmd {
+func (c cmdable) BZPopMin(timeout time.Duration, keys ...string) *ZWithKeyCmd {
args := make([]interface{}, 1+len(keys)+1)
args[0] = "bzpopmin"
for i, key := range keys {
@@ -1613,22 +1599,22 @@ func (c *cmdable) BZPopMin(timeout time.Duration, keys ...string) *ZWithKeyCmd {
args[len(args)-1] = formatSec(timeout)
cmd := NewZWithKeyCmd(args...)
cmd.setReadTimeout(timeout)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) zAdd(a []interface{}, n int, members ...*Z) *IntCmd {
+func (c cmdable) zAdd(a []interface{}, n int, members ...*Z) *IntCmd {
for i, m := range members {
a[n+2*i] = m.Score
a[n+2*i+1] = m.Member
}
cmd := NewIntCmd(a...)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `ZADD key score member [score member ...]` command.
-func (c *cmdable) ZAdd(key string, members ...*Z) *IntCmd {
+func (c cmdable) ZAdd(key string, members ...*Z) *IntCmd {
const n = 2
a := make([]interface{}, n+2*len(members))
a[0], a[1] = "zadd", key
@@ -1636,7 +1622,7 @@ func (c *cmdable) ZAdd(key string, members ...*Z) *IntCmd {
}
// Redis `ZADD key NX score member [score member ...]` command.
-func (c *cmdable) ZAddNX(key string, members ...*Z) *IntCmd {
+func (c cmdable) ZAddNX(key string, members ...*Z) *IntCmd {
const n = 3
a := make([]interface{}, n+2*len(members))
a[0], a[1], a[2] = "zadd", key, "nx"
@@ -1644,7 +1630,7 @@ func (c *cmdable) ZAddNX(key string, members ...*Z) *IntCmd {
}
// Redis `ZADD key XX score member [score member ...]` command.
-func (c *cmdable) ZAddXX(key string, members ...*Z) *IntCmd {
+func (c cmdable) ZAddXX(key string, members ...*Z) *IntCmd {
const n = 3
a := make([]interface{}, n+2*len(members))
a[0], a[1], a[2] = "zadd", key, "xx"
@@ -1652,7 +1638,7 @@ func (c *cmdable) ZAddXX(key string, members ...*Z) *IntCmd {
}
// Redis `ZADD key CH score member [score member ...]` command.
-func (c *cmdable) ZAddCh(key string, members ...*Z) *IntCmd {
+func (c cmdable) ZAddCh(key string, members ...*Z) *IntCmd {
const n = 3
a := make([]interface{}, n+2*len(members))
a[0], a[1], a[2] = "zadd", key, "ch"
@@ -1660,7 +1646,7 @@ func (c *cmdable) ZAddCh(key string, members ...*Z) *IntCmd {
}
// Redis `ZADD key NX CH score member [score member ...]` command.
-func (c *cmdable) ZAddNXCh(key string, members ...*Z) *IntCmd {
+func (c cmdable) ZAddNXCh(key string, members ...*Z) *IntCmd {
const n = 4
a := make([]interface{}, n+2*len(members))
a[0], a[1], a[2], a[3] = "zadd", key, "nx", "ch"
@@ -1668,25 +1654,25 @@ func (c *cmdable) ZAddNXCh(key string, members ...*Z) *IntCmd {
}
// Redis `ZADD key XX CH score member [score member ...]` command.
-func (c *cmdable) ZAddXXCh(key string, members ...*Z) *IntCmd {
+func (c cmdable) ZAddXXCh(key string, members ...*Z) *IntCmd {
const n = 4
a := make([]interface{}, n+2*len(members))
a[0], a[1], a[2], a[3] = "zadd", key, "xx", "ch"
return c.zAdd(a, n, members...)
}
-func (c *cmdable) zIncr(a []interface{}, n int, members ...*Z) *FloatCmd {
+func (c cmdable) zIncr(a []interface{}, n int, members ...*Z) *FloatCmd {
for i, m := range members {
a[n+2*i] = m.Score
a[n+2*i+1] = m.Member
}
cmd := NewFloatCmd(a...)
- c.process(cmd)
+ c(cmd)
return cmd
}
// Redis `ZADD key INCR score member` command.
-func (c *cmdable) ZIncr(key string, member *Z) *FloatCmd {
+func (c cmdable) ZIncr(key string, member *Z) *FloatCmd {
const n = 3
a := make([]interface{}, n+2)
a[0], a[1], a[2] = "zadd", key, "incr"
@@ -1694,7 +1680,7 @@ func (c *cmdable) ZIncr(key string, member *Z) *FloatCmd {
}
// Redis `ZADD key NX INCR score member` command.
-func (c *cmdable) ZIncrNX(key string, member *Z) *FloatCmd {
+func (c cmdable) ZIncrNX(key string, member *Z) *FloatCmd {
const n = 4
a := make([]interface{}, n+2)
a[0], a[1], a[2], a[3] = "zadd", key, "incr", "nx"
@@ -1702,38 +1688,38 @@ func (c *cmdable) ZIncrNX(key string, member *Z) *FloatCmd {
}
// Redis `ZADD key XX INCR score member` command.
-func (c *cmdable) ZIncrXX(key string, member *Z) *FloatCmd {
+func (c cmdable) ZIncrXX(key string, member *Z) *FloatCmd {
const n = 4
a := make([]interface{}, n+2)
a[0], a[1], a[2], a[3] = "zadd", key, "incr", "xx"
return c.zIncr(a, n, member)
}
-func (c *cmdable) ZCard(key string) *IntCmd {
+func (c cmdable) ZCard(key string) *IntCmd {
cmd := NewIntCmd("zcard", key)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZCount(key, min, max string) *IntCmd {
+func (c cmdable) ZCount(key, min, max string) *IntCmd {
cmd := NewIntCmd("zcount", key, min, max)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZLexCount(key, min, max string) *IntCmd {
+func (c cmdable) ZLexCount(key, min, max string) *IntCmd {
cmd := NewIntCmd("zlexcount", key, min, max)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZIncrBy(key string, increment float64, member string) *FloatCmd {
+func (c cmdable) ZIncrBy(key string, increment float64, member string) *FloatCmd {
cmd := NewFloatCmd("zincrby", key, increment, member)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZInterStore(destination string, store *ZStore, keys ...string) *IntCmd {
+func (c cmdable) ZInterStore(destination string, store *ZStore, keys ...string) *IntCmd {
args := make([]interface{}, 3+len(keys))
args[0] = "zinterstore"
args[1] = destination
@@ -1751,11 +1737,11 @@ func (c *cmdable) ZInterStore(destination string, store *ZStore, keys ...string)
args = append(args, "aggregate", store.Aggregate)
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZPopMax(key string, count ...int64) *ZSliceCmd {
+func (c cmdable) ZPopMax(key string, count ...int64) *ZSliceCmd {
args := []interface{}{
"zpopmax",
key,
@@ -1771,11 +1757,11 @@ func (c *cmdable) ZPopMax(key string, count ...int64) *ZSliceCmd {
}
cmd := NewZSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZPopMin(key string, count ...int64) *ZSliceCmd {
+func (c cmdable) ZPopMin(key string, count ...int64) *ZSliceCmd {
args := []interface{}{
"zpopmin",
key,
@@ -1791,11 +1777,11 @@ func (c *cmdable) ZPopMin(key string, count ...int64) *ZSliceCmd {
}
cmd := NewZSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) zRange(key string, start, stop int64, withScores bool) *StringSliceCmd {
+func (c cmdable) zRange(key string, start, stop int64, withScores bool) *StringSliceCmd {
args := []interface{}{
"zrange",
key,
@@ -1806,17 +1792,17 @@ func (c *cmdable) zRange(key string, start, stop int64, withScores bool) *String
args = append(args, "withscores")
}
cmd := NewStringSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRange(key string, start, stop int64) *StringSliceCmd {
+func (c cmdable) ZRange(key string, start, stop int64) *StringSliceCmd {
return c.zRange(key, start, stop, false)
}
-func (c *cmdable) ZRangeWithScores(key string, start, stop int64) *ZSliceCmd {
+func (c cmdable) ZRangeWithScores(key string, start, stop int64) *ZSliceCmd {
cmd := NewZSliceCmd("zrange", key, start, stop, "withscores")
- c.process(cmd)
+ c(cmd)
return cmd
}
@@ -1825,7 +1811,7 @@ type ZRangeBy struct {
Offset, Count int64
}
-func (c *cmdable) zRangeBy(zcmd, key string, opt *ZRangeBy, withScores bool) *StringSliceCmd {
+func (c cmdable) zRangeBy(zcmd, key string, opt *ZRangeBy, withScores bool) *StringSliceCmd {
args := []interface{}{zcmd, key, opt.Min, opt.Max}
if withScores {
args = append(args, "withscores")
@@ -1839,19 +1825,19 @@ func (c *cmdable) zRangeBy(zcmd, key string, opt *ZRangeBy, withScores bool) *St
)
}
cmd := NewStringSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRangeByScore(key string, opt *ZRangeBy) *StringSliceCmd {
+func (c cmdable) ZRangeByScore(key string, opt *ZRangeBy) *StringSliceCmd {
return c.zRangeBy("zrangebyscore", key, opt, false)
}
-func (c *cmdable) ZRangeByLex(key string, opt *ZRangeBy) *StringSliceCmd {
+func (c cmdable) ZRangeByLex(key string, opt *ZRangeBy) *StringSliceCmd {
return c.zRangeBy("zrangebylex", key, opt, false)
}
-func (c *cmdable) ZRangeByScoreWithScores(key string, opt *ZRangeBy) *ZSliceCmd {
+func (c cmdable) ZRangeByScoreWithScores(key string, opt *ZRangeBy) *ZSliceCmd {
args := []interface{}{"zrangebyscore", key, opt.Min, opt.Max, "withscores"}
if opt.Offset != 0 || opt.Count != 0 {
args = append(
@@ -1862,62 +1848,62 @@ func (c *cmdable) ZRangeByScoreWithScores(key string, opt *ZRangeBy) *ZSliceCmd
)
}
cmd := NewZSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRank(key, member string) *IntCmd {
+func (c cmdable) ZRank(key, member string) *IntCmd {
cmd := NewIntCmd("zrank", key, member)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRem(key string, members ...interface{}) *IntCmd {
+func (c cmdable) ZRem(key string, members ...interface{}) *IntCmd {
args := make([]interface{}, 2, 2+len(members))
args[0] = "zrem"
args[1] = key
args = appendArgs(args, members)
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRemRangeByRank(key string, start, stop int64) *IntCmd {
+func (c cmdable) ZRemRangeByRank(key string, start, stop int64) *IntCmd {
cmd := NewIntCmd(
"zremrangebyrank",
key,
start,
stop,
)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRemRangeByScore(key, min, max string) *IntCmd {
+func (c cmdable) ZRemRangeByScore(key, min, max string) *IntCmd {
cmd := NewIntCmd("zremrangebyscore", key, min, max)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRemRangeByLex(key, min, max string) *IntCmd {
+func (c cmdable) ZRemRangeByLex(key, min, max string) *IntCmd {
cmd := NewIntCmd("zremrangebylex", key, min, max)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRevRange(key string, start, stop int64) *StringSliceCmd {
+func (c cmdable) ZRevRange(key string, start, stop int64) *StringSliceCmd {
cmd := NewStringSliceCmd("zrevrange", key, start, stop)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRevRangeWithScores(key string, start, stop int64) *ZSliceCmd {
+func (c cmdable) ZRevRangeWithScores(key string, start, stop int64) *ZSliceCmd {
cmd := NewZSliceCmd("zrevrange", key, start, stop, "withscores")
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) zRevRangeBy(zcmd, key string, opt *ZRangeBy) *StringSliceCmd {
+func (c cmdable) zRevRangeBy(zcmd, key string, opt *ZRangeBy) *StringSliceCmd {
args := []interface{}{zcmd, key, opt.Max, opt.Min}
if opt.Offset != 0 || opt.Count != 0 {
args = append(
@@ -1928,19 +1914,19 @@ func (c *cmdable) zRevRangeBy(zcmd, key string, opt *ZRangeBy) *StringSliceCmd {
)
}
cmd := NewStringSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRevRangeByScore(key string, opt *ZRangeBy) *StringSliceCmd {
+func (c cmdable) ZRevRangeByScore(key string, opt *ZRangeBy) *StringSliceCmd {
return c.zRevRangeBy("zrevrangebyscore", key, opt)
}
-func (c *cmdable) ZRevRangeByLex(key string, opt *ZRangeBy) *StringSliceCmd {
+func (c cmdable) ZRevRangeByLex(key string, opt *ZRangeBy) *StringSliceCmd {
return c.zRevRangeBy("zrevrangebylex", key, opt)
}
-func (c *cmdable) ZRevRangeByScoreWithScores(key string, opt *ZRangeBy) *ZSliceCmd {
+func (c cmdable) ZRevRangeByScoreWithScores(key string, opt *ZRangeBy) *ZSliceCmd {
args := []interface{}{"zrevrangebyscore", key, opt.Max, opt.Min, "withscores"}
if opt.Offset != 0 || opt.Count != 0 {
args = append(
@@ -1951,24 +1937,24 @@ func (c *cmdable) ZRevRangeByScoreWithScores(key string, opt *ZRangeBy) *ZSliceC
)
}
cmd := NewZSliceCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZRevRank(key, member string) *IntCmd {
+func (c cmdable) ZRevRank(key, member string) *IntCmd {
cmd := NewIntCmd("zrevrank", key, member)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ZScore(key, member string) *FloatCmd {
+func (c cmdable) ZScore(key, member string) *FloatCmd {
cmd := NewFloatCmd("zscore", key, member)
- c.process(cmd)
+ c(cmd)
return cmd
}
// TODO: move keys to ZStore?
-func (c *cmdable) ZUnionStore(dest string, store *ZStore, keys ...string) *IntCmd {
+func (c cmdable) ZUnionStore(dest string, store *ZStore, keys ...string) *IntCmd {
args := make([]interface{}, 3+len(keys))
args[0] = "zunionstore"
args[1] = dest
@@ -1986,34 +1972,34 @@ func (c *cmdable) ZUnionStore(dest string, store *ZStore, keys ...string) *IntCm
args = append(args, "aggregate", store.Aggregate)
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
//------------------------------------------------------------------------------
-func (c *cmdable) PFAdd(key string, els ...interface{}) *IntCmd {
+func (c cmdable) PFAdd(key string, els ...interface{}) *IntCmd {
args := make([]interface{}, 2, 2+len(els))
args[0] = "pfadd"
args[1] = key
args = appendArgs(args, els)
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) PFCount(keys ...string) *IntCmd {
+func (c cmdable) PFCount(keys ...string) *IntCmd {
args := make([]interface{}, 1+len(keys))
args[0] = "pfcount"
for i, key := range keys {
args[1+i] = key
}
cmd := NewIntCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) PFMerge(dest string, keys ...string) *StatusCmd {
+func (c cmdable) PFMerge(dest string, keys ...string) *StatusCmd {
args := make([]interface{}, 2+len(keys))
args[0] = "pfmerge"
args[1] = dest
@@ -2021,33 +2007,33 @@ func (c *cmdable) PFMerge(dest string, keys ...string) *StatusCmd {
args[2+i] = key
}
cmd := NewStatusCmd(args...)
- c.process(cmd)
+ c(cmd)
return cmd
}
//------------------------------------------------------------------------------
-func (c *cmdable) BgRewriteAOF() *StatusCmd {
+func (c cmdable) BgRewriteAOF() *StatusCmd {
cmd := NewStatusCmd("bgrewriteaof")
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) BgSave() *StatusCmd {
+func (c cmdable) BgSave() *StatusCmd {
cmd := NewStatusCmd("bgsave")
- c.process(cmd)
+ c(cmd)
return cmd
}
-func (c *cmdable) ClientKill(ipPort string) *StatusCmd {
+func (c cmdable) ClientKill(ipPort string) *StatusCmd {
cmd := NewStatusCmd("client", "kill", ipPort)
- c.process(cmd)
+ c(cmd)
return cmd
}
// ClientKillByFilter is new style synx, while the ClientKill is old
// CLIENT KILL