1
0
mirror of https://github.com/redis/go-redis.git synced 2025-11-30 18:01:23 +03:00
Files
go-redis/osscluster_router.go
ofekshenawa f711eb0f62 feat(cluster): Implement Request and Response Policy Based Routing in Cluster Mode (#3422)
* Add search module builders and tests (#1)

* Add search module builders and tests

* Add tests

* Use builders and Actions in more clean way

* Update search_builders.go

Co-authored-by: Nedyalko Dyakov <1547186+ndyakov@users.noreply.github.com>

* Update search_builders.go

Co-authored-by: Nedyalko Dyakov <1547186+ndyakov@users.noreply.github.com>

* Apply suggestions from code review

Co-authored-by: Nedyalko Dyakov <1547186+ndyakov@users.noreply.github.com>

* feat(routing): add internal request/response policy enums

* feat: load the policy table in cluster client (#4)

* feat: load the policy table in cluster client

* Remove comments

* modify Tips and command pplicy in commandInfo (#5)

* centralize cluster command routing in osscluster_router.go and refactor osscluster.go (#6)

* centralize cluster command routing in osscluster_router.go and refactor osscluster.go

* enalbe ci on all branches

* Add debug prints

* Add debug prints

* FIX: deal with nil policy

* FIX: fixing clusterClient process

* chore(osscluster): simplify switch case

* wip(command): ai generated clone method for commands

* feat: implement response aggregator for Redis cluster commands

* feat: implement response aggregator for Redis cluster commands

* fix: solve concurrency errors

* fix: solve concurrency errors

* return MaxRedirects settings

* remove locks from getCommandPolicy

* Handle MOVED errors more robustly, remove cluster reloading at exectutions, ennsure better routing

* Fix: supports Process hook test

* Fix: remove response aggregation for single shard commands

* Add more preformant type conversion for Cmd type

* Add router logic into processPipeline

---------

Co-authored-by: Nedyalko Dyakov <nedyalko.dyakov@gmail.com>

* remove thread debugging code

* remove thread debugging code && reject commands with policy that cannot be used in pipeline

* refactor processPipline and cmdType enum

* remove FDescribe from cluster tests

* Add tests

* fix aggregation test

* fix mget test

* fix mget test

* remove aggregateKeyedResponses

* added scaffolding for the req-resp manager

* added default policies for the search commands

* split command map into module->command

* cleanup, added logic to refresh the cache

* added reactive cache refresh

* revert cluster refresh

* fixed lint

* addresed first batch of comments

* rewrote aggregator implementations with atomic for native or nearnative primitives

* addressed more comments, fixed lint

* added batch aggregator operations

* fixed lint

* updated batch aggregator, fixed extractcommandvalue

* fixed lint

* added batching to aggregateResponses

* fixed deadlocks

* changed aggregator logic, added error params

* added preemptive return to the aggregators

* more work on the aggregators

* updated and and or aggregators

* fixed lint

* added configurable policy resolvers

* slight refactor

* removed the interface, slight refactor

* change func signature from cmdName to cmder

* added nil safety assertions

* few small refactors

* added read only policies

* removed leftover prints

* Rebased to master, resolved comnflicts

* fixed lint

* updated gha

* fixed tests, minor consistency refactor

* preallocated simple errors

* changed numeric aggregators to use float64

* speculative test fix

* Update command.go

Co-authored-by: Nedyalko Dyakov <1547186+ndyakov@users.noreply.github.com>

* Update main_test.go

Co-authored-by: Nedyalko Dyakov <1547186+ndyakov@users.noreply.github.com>

* Add static shard picker

* Fix nil value handling in command aggregation

* Modify the Clone method to return a shallow copy

* Add clone method to digest command

* Optimize keyless command routing to respect ShardPicker policy

* Remove MGET references

* Fix MGET aggregation to map individual values to keys across shards

* Add clone method to hybrid search commands

* Undo changes in route keyless test

* remove comments

* Add test for DisableRoutingPolicies option

* Add Routing Policies Comprehensive Test Suite and Fix multi keyed aggregation for different step

---------

Co-authored-by: Nedyalko Dyakov <1547186+ndyakov@users.noreply.github.com>
Co-authored-by: Nedyalko Dyakov <nedyalko.dyakov@gmail.com>
Co-authored-by: Hristo Temelski <hristo.temelski@redis.com>
2025-11-28 11:46:23 +02:00

993 lines
26 KiB
Go

package redis
import (
"context"
"errors"
"fmt"
"reflect"
"sync"
"time"
"github.com/redis/go-redis/v9/internal/hashtag"
"github.com/redis/go-redis/v9/internal/routing"
)
var (
errInvalidCmdPointer = errors.New("redis: invalid command pointer")
errNoCmdsToAggregate = errors.New("redis: no commands to aggregate")
errNoResToAggregate = errors.New("redis: no results to aggregate")
errInvalidCursorCmdArgsCount = errors.New("redis: FT.CURSOR command requires at least 3 arguments")
errInvalidCursorIdType = errors.New("redis: invalid cursor ID type")
)
// slotResult represents the result of executing a command on a specific slot
type slotResult struct {
cmd Cmder
keys []string
err error
}
// routeAndRun routes a command to the appropriate cluster nodes and executes it
func (c *ClusterClient) routeAndRun(ctx context.Context, cmd Cmder, node *clusterNode) error {
var policy *routing.CommandPolicy
if c.cmdInfoResolver != nil {
policy = c.cmdInfoResolver.GetCommandPolicy(ctx, cmd)
}
// Set stepCount from cmdInfo if not already set
if cmd.stepCount() == 0 {
if cmdInfo := c.cmdInfo(ctx, cmd.Name()); cmdInfo != nil && cmdInfo.StepCount > 0 {
cmd.SetStepCount(cmdInfo.StepCount)
}
}
if policy == nil {
return c.executeDefault(ctx, cmd, policy, node)
}
switch policy.Request {
case routing.ReqAllNodes:
return c.executeOnAllNodes(ctx, cmd, policy)
case routing.ReqAllShards:
return c.executeOnAllShards(ctx, cmd, policy)
case routing.ReqMultiShard:
return c.executeMultiShard(ctx, cmd, policy)
case routing.ReqSpecial:
return c.executeSpecialCommand(ctx, cmd, policy, node)
default:
return c.executeDefault(ctx, cmd, policy, node)
}
}
// executeDefault handles standard command routing based on keys
func (c *ClusterClient) executeDefault(ctx context.Context, cmd Cmder, policy *routing.CommandPolicy, node *clusterNode) error {
if policy != nil && !c.hasKeys(cmd) {
if c.readOnlyEnabled() && policy.IsReadOnly() {
return c.executeOnArbitraryNode(ctx, cmd)
}
}
return node.Client.Process(ctx, cmd)
}
// executeOnArbitraryNode routes command to an arbitrary node
func (c *ClusterClient) executeOnArbitraryNode(ctx context.Context, cmd Cmder) error {
node := c.pickArbitraryNode(ctx)
if node == nil {
return errClusterNoNodes
}
return node.Client.Process(ctx, cmd)
}
// executeOnAllNodes executes command on all nodes (masters and replicas)
func (c *ClusterClient) executeOnAllNodes(ctx context.Context, cmd Cmder, policy *routing.CommandPolicy) error {
state, err := c.state.Get(ctx)
if err != nil {
return err
}
nodes := append(state.Masters, state.Slaves...)
if len(nodes) == 0 {
return errClusterNoNodes
}
return c.executeParallel(ctx, cmd, nodes, policy)
}
// executeOnAllShards executes command on all master shards
func (c *ClusterClient) executeOnAllShards(ctx context.Context, cmd Cmder, policy *routing.CommandPolicy) error {
state, err := c.state.Get(ctx)
if err != nil {
return err
}
if len(state.Masters) == 0 {
return errClusterNoNodes
}
return c.executeParallel(ctx, cmd, state.Masters, policy)
}
// executeMultiShard handles commands that operate on multiple keys across shards
func (c *ClusterClient) executeMultiShard(ctx context.Context, cmd Cmder, policy *routing.CommandPolicy) error {
args := cmd.Args()
firstKeyPos := int(cmdFirstKeyPos(cmd))
stepCount := int(cmd.stepCount())
if stepCount == 0 {
stepCount = 1 // Default to 1 if not set
}
if firstKeyPos == 0 || firstKeyPos >= len(args) {
return fmt.Errorf("redis: multi-shard command %s has no key arguments", cmd.Name())
}
// Group keys by slot
slotMap := make(map[int][]string)
keyOrder := make([]string, 0)
for i := firstKeyPos; i < len(args); i += stepCount {
key, ok := args[i].(string)
if !ok {
return fmt.Errorf("redis: non-string key at position %d: %v", i, args[i])
}
slot := hashtag.Slot(key)
slotMap[slot] = append(slotMap[slot], key)
for j := 1; j < stepCount; j++ {
if i+j >= len(args) {
break
}
slotMap[slot] = append(slotMap[slot], args[i+j].(string))
}
keyOrder = append(keyOrder, key)
}
return c.executeMultiSlot(ctx, cmd, slotMap, keyOrder, policy)
}
// executeMultiSlot executes commands across multiple slots concurrently
func (c *ClusterClient) executeMultiSlot(ctx context.Context, cmd Cmder, slotMap map[int][]string, keyOrder []string, policy *routing.CommandPolicy) error {
results := make(chan slotResult, len(slotMap))
var wg sync.WaitGroup
// Execute on each slot concurrently
for slot, keys := range slotMap {
wg.Add(1)
go func(slot int, keys []string) {
defer wg.Done()
node, err := c.cmdNodeWithShardPicker(ctx, cmd.Name(), slot, c.opt.ShardPicker)
if err != nil {
results <- slotResult{nil, keys, err}
return
}
// Create a command for this specific slot's keys
subCmd := c.createSlotSpecificCommand(ctx, cmd, keys)
err = node.Client.Process(ctx, subCmd)
results <- slotResult{subCmd, keys, err}
}(slot, keys)
}
go func() {
wg.Wait()
close(results)
}()
return c.aggregateMultiSlotResults(ctx, cmd, results, keyOrder, policy)
}
// createSlotSpecificCommand creates a new command for a specific slot's keys
func (c *ClusterClient) createSlotSpecificCommand(ctx context.Context, originalCmd Cmder, keys []string) Cmder {
originalArgs := originalCmd.Args()
firstKeyPos := int(cmdFirstKeyPos(originalCmd))
// Build new args with only the specified keys
newArgs := make([]interface{}, 0, firstKeyPos+len(keys))
// Copy command name and arguments before the keys
newArgs = append(newArgs, originalArgs[:firstKeyPos]...)
// Add the slot-specific keys
for _, key := range keys {
newArgs = append(newArgs, key)
}
// Create a new command of the same type using the helper function
return createCommandByType(ctx, originalCmd.GetCmdType(), newArgs...)
}
// createCommandByType creates a new command of the specified type with the given arguments
func createCommandByType(ctx context.Context, cmdType CmdType, args ...interface{}) Cmder {
switch cmdType {
case CmdTypeString:
return NewStringCmd(ctx, args...)
case CmdTypeInt:
return NewIntCmd(ctx, args...)
case CmdTypeBool:
return NewBoolCmd(ctx, args...)
case CmdTypeFloat:
return NewFloatCmd(ctx, args...)
case CmdTypeStringSlice:
return NewStringSliceCmd(ctx, args...)
case CmdTypeIntSlice:
return NewIntSliceCmd(ctx, args...)
case CmdTypeFloatSlice:
return NewFloatSliceCmd(ctx, args...)
case CmdTypeBoolSlice:
return NewBoolSliceCmd(ctx, args...)
case CmdTypeStatus:
return NewStatusCmd(ctx, args...)
case CmdTypeTime:
return NewTimeCmd(ctx, args...)
case CmdTypeMapStringString:
return NewMapStringStringCmd(ctx, args...)
case CmdTypeMapStringInt:
return NewMapStringIntCmd(ctx, args...)
case CmdTypeMapStringInterface:
return NewMapStringInterfaceCmd(ctx, args...)
case CmdTypeMapStringInterfaceSlice:
return NewMapStringInterfaceSliceCmd(ctx, args...)
case CmdTypeSlice:
return NewSliceCmd(ctx, args...)
case CmdTypeStringStructMap:
return NewStringStructMapCmd(ctx, args...)
case CmdTypeXMessageSlice:
return NewXMessageSliceCmd(ctx, args...)
case CmdTypeXStreamSlice:
return NewXStreamSliceCmd(ctx, args...)
case CmdTypeXPending:
return NewXPendingCmd(ctx, args...)
case CmdTypeXPendingExt:
return NewXPendingExtCmd(ctx, args...)
case CmdTypeXAutoClaim:
return NewXAutoClaimCmd(ctx, args...)
case CmdTypeXAutoClaimJustID:
return NewXAutoClaimJustIDCmd(ctx, args...)
case CmdTypeXInfoStreamFull:
return NewXInfoStreamFullCmd(ctx, args...)
case CmdTypeZSlice:
return NewZSliceCmd(ctx, args...)
case CmdTypeZWithKey:
return NewZWithKeyCmd(ctx, args...)
case CmdTypeClusterSlots:
return NewClusterSlotsCmd(ctx, args...)
case CmdTypeGeoPos:
return NewGeoPosCmd(ctx, args...)
case CmdTypeCommandsInfo:
return NewCommandsInfoCmd(ctx, args...)
case CmdTypeSlowLog:
return NewSlowLogCmd(ctx, args...)
case CmdTypeKeyValues:
return NewKeyValuesCmd(ctx, args...)
case CmdTypeZSliceWithKey:
return NewZSliceWithKeyCmd(ctx, args...)
case CmdTypeFunctionList:
return NewFunctionListCmd(ctx, args...)
case CmdTypeFunctionStats:
return NewFunctionStatsCmd(ctx, args...)
case CmdTypeKeyFlags:
return NewKeyFlagsCmd(ctx, args...)
case CmdTypeDuration:
return NewDurationCmd(ctx, time.Millisecond, args...)
}
return NewCmd(ctx, args...)
}
// executeSpecialCommand handles commands with special routing requirements
func (c *ClusterClient) executeSpecialCommand(ctx context.Context, cmd Cmder, policy *routing.CommandPolicy, node *clusterNode) error {
switch cmd.Name() {
case "ft.cursor":
return c.executeCursorCommand(ctx, cmd)
default:
return c.executeDefault(ctx, cmd, policy, node)
}
}
// executeCursorCommand handles FT.CURSOR commands with sticky routing
func (c *ClusterClient) executeCursorCommand(ctx context.Context, cmd Cmder) error {
args := cmd.Args()
if len(args) < 4 {
return errInvalidCursorCmdArgsCount
}
cursorID, ok := args[3].(string)
if !ok {
return errInvalidCursorIdType
}
// Route based on cursor ID to maintain stickiness
slot := hashtag.Slot(cursorID)
node, err := c.cmdNodeWithShardPicker(ctx, cmd.Name(), slot, c.opt.ShardPicker)
if err != nil {
return err
}
return node.Client.Process(ctx, cmd)
}
// executeParallel executes a command on multiple nodes concurrently
func (c *ClusterClient) executeParallel(ctx context.Context, cmd Cmder, nodes []*clusterNode, policy *routing.CommandPolicy) error {
if len(nodes) == 0 {
return errClusterNoNodes
}
if len(nodes) == 1 {
return nodes[0].Client.Process(ctx, cmd)
}
type nodeResult struct {
cmd Cmder
err error
}
results := make(chan nodeResult, len(nodes))
var wg sync.WaitGroup
for _, node := range nodes {
wg.Add(1)
go func(n *clusterNode) {
defer wg.Done()
cmdCopy := cmd.Clone()
err := n.Client.Process(ctx, cmdCopy)
results <- nodeResult{cmdCopy, err}
}(node)
}
go func() {
wg.Wait()
close(results)
}()
// Collect results and check for errors
cmds := make([]Cmder, 0, len(nodes))
var firstErr error
for result := range results {
if result.err != nil && firstErr == nil {
firstErr = result.err
}
cmds = append(cmds, result.cmd)
}
// If there was an error and no policy specified, fail fast
if firstErr != nil && (policy == nil || policy.Response == routing.RespDefaultKeyless) {
cmd.SetErr(firstErr)
return firstErr
}
return c.aggregateResponses(cmd, cmds, policy)
}
// aggregateMultiSlotResults aggregates results from multi-slot execution
func (c *ClusterClient) aggregateMultiSlotResults(ctx context.Context, cmd Cmder, results <-chan slotResult, keyOrder []string, policy *routing.CommandPolicy) error {
keyedResults := make(map[string]routing.AggregatorResErr)
var firstErr error
for result := range results {
if result.err != nil && firstErr == nil {
firstErr = result.err
}
if result.cmd != nil && result.err == nil {
value, err := ExtractCommandValue(result.cmd)
// Check if the result is a slice (e.g., from MGET)
if sliceValue, ok := value.([]interface{}); ok {
// Map each element to its corresponding key
for i, key := range result.keys {
if i < len(sliceValue) {
keyedResults[key] = routing.AggregatorResErr{Result: sliceValue[i], Err: err}
} else {
keyedResults[key] = routing.AggregatorResErr{Result: nil, Err: err}
}
}
} else {
// For non-slice results, map the entire result to each key
for _, key := range result.keys {
keyedResults[key] = routing.AggregatorResErr{Result: value, Err: err}
}
}
}
// TODO: return multiple errors by order when we will implement multiple errors returning
if result.err != nil {
firstErr = result.err
}
}
return c.aggregateKeyedValues(cmd, keyedResults, keyOrder, policy)
}
// aggregateKeyedValues aggregates individual key-value pairs while preserving key order
func (c *ClusterClient) aggregateKeyedValues(cmd Cmder, keyedResults map[string]routing.AggregatorResErr, keyOrder []string, policy *routing.CommandPolicy) error {
if len(keyedResults) == 0 {
return errNoResToAggregate
}
aggregator := c.createAggregator(policy, cmd, true)
// Set key order for keyed aggregators
var keyedAgg *routing.DefaultKeyedAggregator
var isKeyedAgg bool
var err error
if keyedAgg, isKeyedAgg = aggregator.(*routing.DefaultKeyedAggregator); isKeyedAgg {
err = keyedAgg.BatchAddWithKeyOrder(keyedResults, keyOrder)
} else {
err = aggregator.BatchAdd(keyedResults)
}
if err != nil {
return err
}
return c.finishAggregation(cmd, aggregator)
}
// aggregateResponses aggregates multiple shard responses
func (c *ClusterClient) aggregateResponses(cmd Cmder, cmds []Cmder, policy *routing.CommandPolicy) error {
if len(cmds) == 0 {
return errNoCmdsToAggregate
}
if len(cmds) == 1 {
shardCmd := cmds[0]
if err := shardCmd.Err(); err != nil {
cmd.SetErr(err)
return err
}
value, _ := ExtractCommandValue(shardCmd)
return c.setCommandValue(cmd, value)
}
aggregator := c.createAggregator(policy, cmd, false)
batchWithErrs := []routing.AggregatorResErr{}
// Add all results to aggregator
for _, shardCmd := range cmds {
value, err := ExtractCommandValue(shardCmd)
batchWithErrs = append(batchWithErrs, routing.AggregatorResErr{
Result: value,
Err: err,
})
}
err := aggregator.BatchSlice(batchWithErrs)
if err != nil {
return err
}
return c.finishAggregation(cmd, aggregator)
}
// createAggregator creates the appropriate response aggregator
func (c *ClusterClient) createAggregator(policy *routing.CommandPolicy, cmd Cmder, isKeyed bool) routing.ResponseAggregator {
if policy != nil {
return routing.NewResponseAggregator(policy.Response, cmd.Name())
}
if !isKeyed {
firstKeyPos := cmdFirstKeyPos(cmd)
isKeyed = firstKeyPos > 0
}
return routing.NewDefaultAggregator(isKeyed)
}
// finishAggregation completes the aggregation process and sets the result
func (c *ClusterClient) finishAggregation(cmd Cmder, aggregator routing.ResponseAggregator) error {
finalValue, finalErr := aggregator.Result()
if finalErr != nil {
cmd.SetErr(finalErr)
return finalErr
}
return c.setCommandValue(cmd, finalValue)
}
// pickArbitraryNode selects a master or slave shard using the configured ShardPicker
func (c *ClusterClient) pickArbitraryNode(ctx context.Context) *clusterNode {
state, err := c.state.Get(ctx)
if err != nil || len(state.Masters) == 0 {
return nil
}
allNodes := append(state.Masters, state.Slaves...)
idx := c.opt.ShardPicker.Next(len(allNodes))
return allNodes[idx]
}
// hasKeys checks if a command operates on keys
func (c *ClusterClient) hasKeys(cmd Cmder) bool {
firstKeyPos := cmdFirstKeyPos(cmd)
return firstKeyPos > 0
}
func (c *ClusterClient) readOnlyEnabled() bool {
return c.opt.ReadOnly
}
// setCommandValue sets the aggregated value on a command using the enum-based approach
func (c *ClusterClient) setCommandValue(cmd Cmder, value interface{}) error {
// If value is nil, it might mean ExtractCommandValue couldn't extract the value
// but the command might have executed successfully. In this case, don't set an error.
if value == nil {
// ExtractCommandValue returned nil - this means the command type is not supported
// in the aggregation flow. This is a programming error, not a runtime error.
if cmd.Err() != nil {
// Command already has an error, preserve it
return cmd.Err()
}
// Command executed successfully but we can't extract/set the aggregated value
// This indicates the command type needs to be added to ExtractCommandValue
return fmt.Errorf("redis: cannot aggregate command %s: unsupported command type %d",
cmd.Name(), cmd.GetCmdType())
}
switch cmd.GetCmdType() {
case CmdTypeGeneric:
if c, ok := cmd.(*Cmd); ok {
c.SetVal(value)
}
case CmdTypeString:
if c, ok := cmd.(*StringCmd); ok {
if v, ok := value.(string); ok {
c.SetVal(v)
}
}
case CmdTypeInt:
if c, ok := cmd.(*IntCmd); ok {
if v, ok := value.(int64); ok {
c.SetVal(v)
} else if v, ok := value.(float64); ok {
c.SetVal(int64(v))
}
}
case CmdTypeBool:
if c, ok := cmd.(*BoolCmd); ok {
if v, ok := value.(bool); ok {
c.SetVal(v)
}
}
case CmdTypeFloat:
if c, ok := cmd.(*FloatCmd); ok {
if v, ok := value.(float64); ok {
c.SetVal(v)
}
}
case CmdTypeStringSlice:
if c, ok := cmd.(*StringSliceCmd); ok {
if v, ok := value.([]string); ok {
c.SetVal(v)
}
}
case CmdTypeIntSlice:
if c, ok := cmd.(*IntSliceCmd); ok {
if v, ok := value.([]int64); ok {
c.SetVal(v)
} else if v, ok := value.([]float64); ok {
els := len(v)
intSlc := make([]int, els)
for i := range v {
intSlc[i] = int(v[i])
}
}
}
case CmdTypeFloatSlice:
if c, ok := cmd.(*FloatSliceCmd); ok {
if v, ok := value.([]float64); ok {
c.SetVal(v)
}
}
case CmdTypeBoolSlice:
if c, ok := cmd.(*BoolSliceCmd); ok {
if v, ok := value.([]bool); ok {
c.SetVal(v)
}
}
case CmdTypeMapStringString:
if c, ok := cmd.(*MapStringStringCmd); ok {
if v, ok := value.(map[string]string); ok {
c.SetVal(v)
}
}
case CmdTypeMapStringInt:
if c, ok := cmd.(*MapStringIntCmd); ok {
if v, ok := value.(map[string]int64); ok {
c.SetVal(v)
}
}
case CmdTypeMapStringInterface:
if c, ok := cmd.(*MapStringInterfaceCmd); ok {
if v, ok := value.(map[string]interface{}); ok {
c.SetVal(v)
}
}
case CmdTypeSlice:
if c, ok := cmd.(*SliceCmd); ok {
if v, ok := value.([]interface{}); ok {
c.SetVal(v)
}
}
case CmdTypeStatus:
if c, ok := cmd.(*StatusCmd); ok {
if v, ok := value.(string); ok {
c.SetVal(v)
}
}
case CmdTypeDuration:
if c, ok := cmd.(*DurationCmd); ok {
if v, ok := value.(time.Duration); ok {
c.SetVal(v)
}
}
case CmdTypeTime:
if c, ok := cmd.(*TimeCmd); ok {
if v, ok := value.(time.Time); ok {
c.SetVal(v)
}
}
case CmdTypeKeyValueSlice:
if c, ok := cmd.(*KeyValueSliceCmd); ok {
if v, ok := value.([]KeyValue); ok {
c.SetVal(v)
}
}
case CmdTypeStringStructMap:
if c, ok := cmd.(*StringStructMapCmd); ok {
if v, ok := value.(map[string]struct{}); ok {
c.SetVal(v)
}
}
case CmdTypeXMessageSlice:
if c, ok := cmd.(*XMessageSliceCmd); ok {
if v, ok := value.([]XMessage); ok {
c.SetVal(v)
}
}
case CmdTypeXStreamSlice:
if c, ok := cmd.(*XStreamSliceCmd); ok {
if v, ok := value.([]XStream); ok {
c.SetVal(v)
}
}
case CmdTypeXPending:
if c, ok := cmd.(*XPendingCmd); ok {
if v, ok := value.(*XPending); ok {
c.SetVal(v)
}
}
case CmdTypeXPendingExt:
if c, ok := cmd.(*XPendingExtCmd); ok {
if v, ok := value.([]XPendingExt); ok {
c.SetVal(v)
}
}
case CmdTypeXAutoClaim:
if c, ok := cmd.(*XAutoClaimCmd); ok {
if v, ok := value.(CmdTypeXAutoClaimValue); ok {
c.SetVal(v.messages, v.start)
}
}
case CmdTypeXAutoClaimJustID:
if c, ok := cmd.(*XAutoClaimJustIDCmd); ok {
if v, ok := value.(CmdTypeXAutoClaimJustIDValue); ok {
c.SetVal(v.ids, v.start)
}
}
case CmdTypeXInfoConsumers:
if c, ok := cmd.(*XInfoConsumersCmd); ok {
if v, ok := value.([]XInfoConsumer); ok {
c.SetVal(v)
}
}
case CmdTypeXInfoGroups:
if c, ok := cmd.(*XInfoGroupsCmd); ok {
if v, ok := value.([]XInfoGroup); ok {
c.SetVal(v)
}
}
case CmdTypeXInfoStream:
if c, ok := cmd.(*XInfoStreamCmd); ok {
if v, ok := value.(*XInfoStream); ok {
c.SetVal(v)
}
}
case CmdTypeXInfoStreamFull:
if c, ok := cmd.(*XInfoStreamFullCmd); ok {
if v, ok := value.(*XInfoStreamFull); ok {
c.SetVal(v)
}
}
case CmdTypeZSlice:
if c, ok := cmd.(*ZSliceCmd); ok {
if v, ok := value.([]Z); ok {
c.SetVal(v)
}
}
case CmdTypeZWithKey:
if c, ok := cmd.(*ZWithKeyCmd); ok {
if v, ok := value.(*ZWithKey); ok {
c.SetVal(v)
}
}
case CmdTypeScan:
if c, ok := cmd.(*ScanCmd); ok {
if v, ok := value.(CmdTypeScanValue); ok {
c.SetVal(v.keys, v.cursor)
}
}
case CmdTypeClusterSlots:
if c, ok := cmd.(*ClusterSlotsCmd); ok {
if v, ok := value.([]ClusterSlot); ok {
c.SetVal(v)
}
}
case CmdTypeGeoLocation:
if c, ok := cmd.(*GeoLocationCmd); ok {
if v, ok := value.([]GeoLocation); ok {
c.SetVal(v)
}
}
case CmdTypeGeoSearchLocation:
if c, ok := cmd.(*GeoSearchLocationCmd); ok {
if v, ok := value.([]GeoLocation); ok {
c.SetVal(v)
}
}
case CmdTypeGeoPos:
if c, ok := cmd.(*GeoPosCmd); ok {
if v, ok := value.([]*GeoPos); ok {
c.SetVal(v)
}
}
case CmdTypeCommandsInfo:
if c, ok := cmd.(*CommandsInfoCmd); ok {
if v, ok := value.(map[string]*CommandInfo); ok {
c.SetVal(v)
}
}
case CmdTypeSlowLog:
if c, ok := cmd.(*SlowLogCmd); ok {
if v, ok := value.([]SlowLog); ok {
c.SetVal(v)
}
}
case CmdTypeMapStringStringSlice:
if c, ok := cmd.(*MapStringStringSliceCmd); ok {
if v, ok := value.([]map[string]string); ok {
c.SetVal(v)
}
}
case CmdTypeMapMapStringInterface:
if c, ok := cmd.(*MapMapStringInterfaceCmd); ok {
if v, ok := value.(map[string]interface{}); ok {
c.SetVal(v)
}
}
case CmdTypeMapStringInterfaceSlice:
if c, ok := cmd.(*MapStringInterfaceSliceCmd); ok {
if v, ok := value.([]map[string]interface{}); ok {
c.SetVal(v)
}
}
case CmdTypeKeyValues:
if c, ok := cmd.(*KeyValuesCmd); ok {
// KeyValuesCmd needs a key string and values slice
if v, ok := value.(CmdTypeKeyValuesValue); ok {
c.SetVal(v.key, v.values)
}
}
case CmdTypeZSliceWithKey:
if c, ok := cmd.(*ZSliceWithKeyCmd); ok {
// ZSliceWithKeyCmd needs a key string and Z slice
if v, ok := value.(CmdTypeZSliceWithKeyValue); ok {
c.SetVal(v.key, v.zSlice)
}
}
case CmdTypeFunctionList:
if c, ok := cmd.(*FunctionListCmd); ok {
if v, ok := value.([]Library); ok {
c.SetVal(v)
}
}
case CmdTypeFunctionStats:
if c, ok := cmd.(*FunctionStatsCmd); ok {
if v, ok := value.(FunctionStats); ok {
c.SetVal(v)
}
}
case CmdTypeLCS:
if c, ok := cmd.(*LCSCmd); ok {
if v, ok := value.(*LCSMatch); ok {
c.SetVal(v)
}
}
case CmdTypeKeyFlags:
if c, ok := cmd.(*KeyFlagsCmd); ok {
if v, ok := value.([]KeyFlags); ok {
c.SetVal(v)
}
}
case CmdTypeClusterLinks:
if c, ok := cmd.(*ClusterLinksCmd); ok {
if v, ok := value.([]ClusterLink); ok {
c.SetVal(v)
}
}
case CmdTypeClusterShards:
if c, ok := cmd.(*ClusterShardsCmd); ok {
if v, ok := value.([]ClusterShard); ok {
c.SetVal(v)
}
}
case CmdTypeRankWithScore:
if c, ok := cmd.(*RankWithScoreCmd); ok {
if v, ok := value.(RankScore); ok {
c.SetVal(v)
}
}
case CmdTypeClientInfo:
if c, ok := cmd.(*ClientInfoCmd); ok {
if v, ok := value.(*ClientInfo); ok {
c.SetVal(v)
}
}
case CmdTypeACLLog:
if c, ok := cmd.(*ACLLogCmd); ok {
if v, ok := value.([]*ACLLogEntry); ok {
c.SetVal(v)
}
}
case CmdTypeInfo:
if c, ok := cmd.(*InfoCmd); ok {
if v, ok := value.(map[string]map[string]string); ok {
c.SetVal(v)
}
}
case CmdTypeMonitor:
// MonitorCmd doesn't have SetVal method
// Skip setting value for MonitorCmd
case CmdTypeJSON:
if c, ok := cmd.(*JSONCmd); ok {
if v, ok := value.(string); ok {
c.SetVal(v)
}
}
case CmdTypeJSONSlice:
if c, ok := cmd.(*JSONSliceCmd); ok {
if v, ok := value.([]interface{}); ok {
c.SetVal(v)
}
}
case CmdTypeIntPointerSlice:
if c, ok := cmd.(*IntPointerSliceCmd); ok {
if v, ok := value.([]*int64); ok {
c.SetVal(v)
}
}
case CmdTypeScanDump:
if c, ok := cmd.(*ScanDumpCmd); ok {
if v, ok := value.(ScanDump); ok {
c.SetVal(v)
}
}
case CmdTypeBFInfo:
if c, ok := cmd.(*BFInfoCmd); ok {
if v, ok := value.(BFInfo); ok {
c.SetVal(v)
}
}
case CmdTypeCFInfo:
if c, ok := cmd.(*CFInfoCmd); ok {
if v, ok := value.(CFInfo); ok {
c.SetVal(v)
}
}
case CmdTypeCMSInfo:
if c, ok := cmd.(*CMSInfoCmd); ok {
if v, ok := value.(CMSInfo); ok {
c.SetVal(v)
}
}
case CmdTypeTopKInfo:
if c, ok := cmd.(*TopKInfoCmd); ok {
if v, ok := value.(TopKInfo); ok {
c.SetVal(v)
}
}
case CmdTypeTDigestInfo:
if c, ok := cmd.(*TDigestInfoCmd); ok {
if v, ok := value.(TDigestInfo); ok {
c.SetVal(v)
}
}
case CmdTypeFTSynDump:
if c, ok := cmd.(*FTSynDumpCmd); ok {
if v, ok := value.([]FTSynDumpResult); ok {
c.SetVal(v)
}
}
case CmdTypeAggregate:
if c, ok := cmd.(*AggregateCmd); ok {
if v, ok := value.(*FTAggregateResult); ok {
c.SetVal(v)
}
}
case CmdTypeFTInfo:
if c, ok := cmd.(*FTInfoCmd); ok {
if v, ok := value.(FTInfoResult); ok {
c.SetVal(v)
}
}
case CmdTypeFTSpellCheck:
if c, ok := cmd.(*FTSpellCheckCmd); ok {
if v, ok := value.([]SpellCheckResult); ok {
c.SetVal(v)
}
}
case CmdTypeFTSearch:
if c, ok := cmd.(*FTSearchCmd); ok {
if v, ok := value.(FTSearchResult); ok {
c.SetVal(v)
}
}
case CmdTypeTSTimestampValue:
if c, ok := cmd.(*TSTimestampValueCmd); ok {
if v, ok := value.(TSTimestampValue); ok {
c.SetVal(v)
}
}
case CmdTypeTSTimestampValueSlice:
if c, ok := cmd.(*TSTimestampValueSliceCmd); ok {
if v, ok := value.([]TSTimestampValue); ok {
c.SetVal(v)
}
}
default:
// Fallback to reflection for unknown types
return c.setCommandValueReflection(cmd, value)
}
return nil
}
// setCommandValueReflection is a fallback function that uses reflection
func (c *ClusterClient) setCommandValueReflection(cmd Cmder, value interface{}) error {
cmdValue := reflect.ValueOf(cmd)
if cmdValue.Kind() != reflect.Ptr || cmdValue.IsNil() {
return errInvalidCmdPointer
}
setValMethod := cmdValue.MethodByName("SetVal")
if !setValMethod.IsValid() {
return fmt.Errorf("redis: command %T does not have SetVal method", cmd)
}
args := []reflect.Value{reflect.ValueOf(value)}
switch cmd.(type) {
case *XAutoClaimCmd, *XAutoClaimJustIDCmd:
args = append(args, reflect.ValueOf(""))
case *ScanCmd:
args = append(args, reflect.ValueOf(uint64(0)))
case *KeyValuesCmd, *ZSliceWithKeyCmd:
if key, ok := value.(string); ok {
args = []reflect.Value{reflect.ValueOf(key)}
if _, ok := cmd.(*ZSliceWithKeyCmd); ok {
args = append(args, reflect.ValueOf([]Z{}))
} else {
args = append(args, reflect.ValueOf([]string{}))
}
}
}
defer func() {
if r := recover(); r != nil {
cmd.SetErr(fmt.Errorf("redis: failed to set command value: %v", r))
}
}()
setValMethod.Call(args)
return nil
}