From 8c0f0452d2955ad58e140a48e5ce66396a73a4ec Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Wed, 10 Apr 2024 13:07:42 -0700 Subject: [PATCH 01/26] Sketching out new tree queue with shuffle shard nodes --- pkg/scheduler/queue/tree_queue_sketch.go | 335 ++++++++++++++++++ pkg/scheduler/queue/tree_queue_sketch_test.go | 326 +++++++++++++++++ 2 files changed, 661 insertions(+) create mode 100644 pkg/scheduler/queue/tree_queue_sketch.go create mode 100644 pkg/scheduler/queue/tree_queue_sketch_test.go diff --git a/pkg/scheduler/queue/tree_queue_sketch.go b/pkg/scheduler/queue/tree_queue_sketch.go new file mode 100644 index 00000000000..8709e1245b2 --- /dev/null +++ b/pkg/scheduler/queue/tree_queue_sketch.go @@ -0,0 +1,335 @@ +package queue + +import ( + "container/list" + "fmt" +) + +type TreeNodeIFace interface { + //EnqueueFrontByPath(QueuePath, any) error + enqueueBackByPath(*Tree, QueuePath, any) error + dequeue() (QueuePath, any) // Dequeue returns the dequeued value, and the QueuePath (starting with root) to the node which was dequeued from + IsEmpty() bool + Name() string + getOrAddNode(*Tree, QueuePath) (TreeNodeIFace, error) + getLocalQueue() *list.List +} + +type NodeType string + +const roundRobin = NodeType("round-robin") + +// shuffleShard node uses an externally maintained shuffle sharding mapping to select the next item to dequeue +const shuffleShard = NodeType("shuffle-shard") + +type Tree struct { + nodeTypesByDepth []NodeType + maxDepth int + rootNode TreeNodeIFace + *ShuffleShardState +} + +type ShuffleShardState struct { + tenantQuerierMap map[TenantID]map[QuerierID]struct{} + currentQuerier *QuerierID +} + +func NewTree(nodeTypeByDepth []NodeType, shuffleShardState *ShuffleShardState) (*Tree, error) { + if len(nodeTypeByDepth) <= 0 { + return nil, fmt.Errorf("no node types provided") + } + + shuffleShardCount := 0 + for _, nt := range nodeTypeByDepth { + if nt == shuffleShard { + shuffleShardCount++ + } + } + if shuffleShardCount > 1 { + return nil, fmt.Errorf("more than one tree layer with type shuffle-shard not currently supported") + } + + if shuffleShardCount > 0 && shuffleShardState.tenantQuerierMap == nil { + return nil, fmt.Errorf("cannot create tree with shuffle-shard nodes, without reference to tenant-querier map") + } + + t := &Tree{ + nodeTypesByDepth: nodeTypeByDepth, + maxDepth: len(nodeTypeByDepth) - 1, + rootNode: nil, + ShuffleShardState: shuffleShardState, + } + t.rootNode = t.newNode("root", 0) + return t, nil +} + +// EnqueueBackByPath takes a QueuePath (excluding the root node), and +func (t *Tree) EnqueueBackByPath(path QueuePath, v any) error { + // nodes need *Tree in order to know what type of node to create if necessary, and + if len(path) > t.maxDepth { + return fmt.Errorf("cannot create path with depth: %v; greater than max depth %v", len(path), t.maxDepth) + } + return t.rootNode.enqueueBackByPath(t, path, v) +} + +func (t *Tree) Dequeue() (QueuePath, any) { + return t.rootNode.dequeue() +} + +func (t *Tree) newNode(name string, depth int) TreeNodeIFace { + switch t.nodeTypesByDepth[depth] { + case roundRobin: + return &RoundRobinNode{ + name: name, + localQueue: list.New(), + queuePosition: localQueueIndex, + queueOrder: make([]string, 0), + queueMap: make(map[string]TreeNodeIFace, 1), + depth: depth, + } + case shuffleShard: + return &ShuffleShardNode{ + name: name, + localQueue: list.New(), + queueOrder: make([]string, 0), + queueMap: make(map[string]TreeNodeIFace, 1), + depth: depth, + stateUpdateInfo: t.ShuffleShardState, + } + default: + panic("no defined node type at provided depth") + } +} + +type RoundRobinNode struct { + name string + localQueue *list.List + queuePosition int // position for dequeueing from queueOrder + queueOrder []string // order for dequeueing from self/children + queueMap map[string]TreeNodeIFace + depth int +} + +func (rrn *RoundRobinNode) enqueueBackByPath(t *Tree, pathFromNode QueuePath, v any) error { + childNode, err := rrn.getOrAddNode(t, pathFromNode) + if err != nil { + return err + } + // TODO (casie): Create localQueue on node creation; why not? + childNode.getLocalQueue().PushBack(v) + return nil +} + +func (rrn *RoundRobinNode) getLocalQueue() *list.List { + return rrn.localQueue +} + +func (rrn *RoundRobinNode) getOrAddNode(t *Tree, pathFromNode QueuePath) (TreeNodeIFace, error) { + if len(pathFromNode) == 0 { + return rrn, nil + } + + // TODO (casie): make rrn.queueMap on node creation; why not? + var childNode TreeNodeIFace + var ok bool + if childNode, ok = rrn.queueMap[pathFromNode[0]]; !ok { + if rrn.depth+1 > t.maxDepth { + return nil, fmt.Errorf("cannot create a node at depth %v; greater than max depth %v", rrn.depth+1, len(t.nodeTypesByDepth)-1) + } + // child does not exist, create it + childNode = t.newNode(pathFromNode[0], rrn.depth+1) + + // add childNode to rrn.queueMap + rrn.queueMap[childNode.Name()] = childNode + // add childNode to rrn.queueOrder before the current position, update rrn.queuePosition to current element + if rrn.queuePosition == localQueueIndex { + rrn.queueOrder = append(rrn.queueOrder, childNode.Name()) + } else { + rrn.queueOrder = append(rrn.queueOrder[:rrn.queuePosition], append([]string{childNode.Name()}, rrn.queueOrder[rrn.queuePosition:]...)...) + rrn.queuePosition++ + } + + } + return childNode.getOrAddNode(t, pathFromNode[1:]) +} + +func (rrn *RoundRobinNode) Name() string { + return rrn.name +} + +func (rrn *RoundRobinNode) dequeue() (QueuePath, any) { + var v any + + initialLen := len(rrn.queueOrder) + path := QueuePath{rrn.name} + + // iterate over all elements in rrn.queueOrder. If we ever find a valid value v, we'll return; + // otherwise, return nil if we iterate over all rrn.queueOrder and find nothing + for i := 0; i <= initialLen && v == nil; i++ { + if rrn.queuePosition == localQueueIndex { + // The next item should be dequeued from the local queue. + if rrn.localQueue.Len() > 0 { + if elt := rrn.localQueue.Front(); elt != nil { + rrn.localQueue.Remove(elt) + v = elt.Value + } + } + rrn.queuePosition++ + if rrn.queuePosition >= len(rrn.queueOrder) { + rrn.queuePosition = localQueueIndex + } + } else { + // dequeue from a child node + childNodeName := rrn.queueOrder[rrn.queuePosition] + childPath := QueuePath{} + childNode := rrn.queueMap[childNodeName] + childPath, v = childNode.dequeue() + path = append(path, childPath...) + + if childNode.IsEmpty() { + // - delete child node from rrn.queueOrder + // - delete child node from rrn.queueMap + delete(rrn.queueMap, childNodeName) + // queueOrder may have been updated by another process? Make sure to only remove a node of matching name + for idx, name := range rrn.queueOrder { + if name == childNodeName { + rrn.queueOrder = append(rrn.queueOrder[:idx], rrn.queueOrder[idx+1:]...) + } + } + // rrn.queuePosition only needs updated if it was originally at the end of the slice + if rrn.queuePosition >= len(rrn.queueOrder) { + rrn.queuePosition = localQueueIndex + } + + } else { // the child node is either dequeued from or not, but is non-empty so we leave it be + rrn.queuePosition++ + if rrn.queuePosition >= len(rrn.queueOrder) { + rrn.queuePosition = localQueueIndex + } + } + } + } + return path, v +} + +func (rrn *RoundRobinNode) IsEmpty() bool { + // node has nothing in local queue, and no child nodes + return rrn.localQueue.Len() == 0 && len(rrn.queueMap) == 0 +} + +type ShuffleShardNode struct { + name string + localQueue *list.List // should never be populated + queuePosition int + queueOrder []string // will be a slice of tenants (+ self?) + queueMap map[string]TreeNodeIFace + depth int + stateUpdateInfo *ShuffleShardState +} + +func (ssn *ShuffleShardNode) Name() string { + return ssn.name +} + +func (ssn *ShuffleShardNode) dequeue() (path QueuePath, v any) { + // start from ssn.queuePosition + // check that tenant for querierID against availabilityMap + // if exists, move element to "back" of queue + // if doesn't exist, check next child + // nothing here to dequeue + path = QueuePath{ssn.name} + if ssn.IsEmpty() { + return path, nil + } + + // no children, but has a non-zero length local queue -- should probably never happen? + if len(ssn.queueOrder) == 0 { + if elt := ssn.localQueue.Front(); elt != nil { + ssn.localQueue.Remove(elt) + return path, elt.Value + } + } + + // node has children, dequeue from them first + // get the value + nodeName, dequeueNode := ssn.dequeueGetNode() + + // no node for this querier + if dequeueNode == nil { + return path, v + } + childPath, v := dequeueNode.dequeue() + ssn.dequeueMoveToBack(nodeName) + + // move the node in our order if appropriate + + return append(path, childPath...), v +} + +// return a function which selects the tree node from which to dequeue; return fn passed to tree +func (ssn *ShuffleShardNode) dequeueGetNode() (string, TreeNodeIFace) { + checkIndex := ssn.queuePosition + for iters := 0; iters < len(ssn.queueOrder); iters++ { + if checkIndex > len(ssn.queueOrder) { + checkIndex = 0 + } + tenantName := ssn.queueOrder[checkIndex] + if tenantQuerierSet, ok := ssn.stateUpdateInfo.tenantQuerierMap[TenantID(tenantName)]; ok { + if _, ok := tenantQuerierSet[*ssn.stateUpdateInfo.currentQuerier]; ok { + return tenantName, ssn.queueMap[tenantName] + } + } + checkIndex++ + } + return "", nil +} + +func (ssn *ShuffleShardNode) dequeueMoveToBack(name string) { + for i, child := range ssn.queueOrder { + if child == name { + ssn.queueOrder = append(ssn.queueOrder[:i], append(ssn.queueOrder[i+1:], child)...) // ugly + } + } +} + +func (ssn *ShuffleShardNode) enqueueBackByPath(t *Tree, path QueuePath, v any) error { + nodeToQueueTo, err := ssn.getOrAddNode(t, path) + if err != nil { + return err + } + nodeToQueueTo.getLocalQueue().PushBack(v) + return nil +} + +func (ssn *ShuffleShardNode) IsEmpty() bool { + return ssn.localQueue.Len() == 0 && len(ssn.queueOrder) == 0 +} + +func (ssn *ShuffleShardNode) getOrAddNode(t *Tree, path QueuePath) (TreeNodeIFace, error) { + if len(path) == 0 { + return ssn, nil + } + + var childNode TreeNodeIFace + var ok bool + if childNode, ok = ssn.queueMap[path[0]]; !ok { + if ssn.depth+1 > t.maxDepth { + return nil, fmt.Errorf("cannot create a node at depth %v; greater than max depth %v", ssn.depth+1, len(t.nodeTypesByDepth)-1) + } + childNode = t.newNode(path[0], ssn.depth+1) + // add childNode to rrn.queueMap + ssn.queueMap[childNode.Name()] = childNode + // update rrn.queueOrder to place childNode behind rrn.queuePosition + if ssn.queuePosition <= 0 { + ssn.queueOrder = append(ssn.queueOrder, childNode.Name()) + } else { + ssn.queueOrder = append(ssn.queueOrder[:ssn.queuePosition], append([]string{childNode.Name()}, ssn.queueOrder[ssn.queuePosition:]...)...) + ssn.queuePosition++ + } + } + return childNode.getOrAddNode(t, path[1:]) +} + +func (ssn *ShuffleShardNode) getLocalQueue() *list.List { + return ssn.localQueue +} diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go new file mode 100644 index 00000000000..05772fb7765 --- /dev/null +++ b/pkg/scheduler/queue/tree_queue_sketch_test.go @@ -0,0 +1,326 @@ +package queue + +import ( + "fmt" + "github.com/stretchr/testify/require" + "testing" +) + +func Test_NewTree(t *testing.T) { + tests := []struct { + name string + args []NodeType + expectErr bool + }{ + { + name: "create round-robin-only tree", + args: []NodeType{roundRobin, roundRobin}, + }, + { + name: "create shuffle-shard-only tree", + args: []NodeType{shuffleShard}, + }, + { + name: "create empty tree", + args: []NodeType{}, + expectErr: true, + }, + { + name: "create tree with multiple shuffle-shard depths", + args: []NodeType{shuffleShard, roundRobin, shuffleShard}, + expectErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + _, err := NewTree(tt.args, &ShuffleShardState{map[TenantID]map[QuerierID]struct{}{}, nil}) + if tt.expectErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + + }) + } +} + +func Test_EnqueueCreatingNode(t *testing.T) { + tests := []struct { + name string + nodeTypeByDepth []NodeType + enqueueNodeName string + expectErr bool + }{ + { + name: "enqueue over max-depth to round-robin node", + nodeTypeByDepth: []NodeType{roundRobin}, + enqueueNodeName: "bad-child-1", + expectErr: true, + }, + { + name: "enqueue over max-depth to shuffle-shard node", + nodeTypeByDepth: []NodeType{shuffleShard}, + enqueueNodeName: "bad-child-1", + expectErr: true, + }, + { + name: "enqueue round-robin node to round-robin node", + nodeTypeByDepth: []NodeType{roundRobin, roundRobin}, + enqueueNodeName: "round-robin-child-1", + }, + { + name: "enqueue shuffle-shard node to round-robin node", + nodeTypeByDepth: []NodeType{roundRobin, shuffleShard}, + enqueueNodeName: "shuffle-shard-child-1", + }, + { + name: "enqueue round-robin node to shuffle-shard node", + nodeTypeByDepth: []NodeType{shuffleShard, roundRobin}, + enqueueNodeName: "round-robin-child-1", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.nodeTypeByDepth, &ShuffleShardState{map[TenantID]map[QuerierID]struct{}{}, nil}) + require.NoError(t, err) + err = tree.EnqueueBackByPath(QueuePath{tt.enqueueNodeName}, "some-object") + if tt.expectErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + }) + } +} + +func Test_Dequeue_RootNode(t *testing.T) { + tests := []struct { + name string + rootNodeType NodeType + enqueueToRoot any + }{ + { + name: "dequeue from empty round-robin root node", + rootNodeType: roundRobin, + }, + { + name: "dequeue from empty shuffle-shard root node", + rootNodeType: shuffleShard, + }, + { + name: "dequeue from non-empty round-robin root node", + rootNodeType: roundRobin, + enqueueToRoot: "something-in-root", + }, + { + name: "dequeue from non-empty shuffle-shard root node", + rootNodeType: shuffleShard, + enqueueToRoot: "something-else-in-root", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree([]NodeType{tt.rootNodeType}, &ShuffleShardState{map[TenantID]map[QuerierID]struct{}{}, nil}) + require.NoError(t, err) + if tt.enqueueToRoot != nil { + err = tree.EnqueueBackByPath(QueuePath{}, tt.enqueueToRoot) + require.NoError(t, err) + } + path, v := tree.Dequeue() + require.Equal(t, QueuePath{"root"}, path) + require.Equal(t, tt.enqueueToRoot, v) + + }) + } +} + +func Test_RoundRobinDequeue(t *testing.T) { + tests := []struct { + name string + nodesByDepth []NodeType + selfQueueObjects []string + childQueueObjects map[string][]any + grandchildren map[string]struct { + path QueuePath + objs []any + } + numDequeuesToExpected int + expected string + }{ + { + name: "dequeue from round-robin child when local queue empty", + nodesByDepth: []NodeType{roundRobin, roundRobin}, + childQueueObjects: map[string][]any{"child-1": {"child-1:some-object"}}, + numDequeuesToExpected: 1, + expected: "child-1:some-object", + }, + { + name: "dequeue from round-robin root when on node's turn", + nodesByDepth: []NodeType{roundRobin, roundRobin}, + selfQueueObjects: []string{"root:object-1", "root:object-2"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:object-1"}}, + numDequeuesToExpected: 2, + expected: "child-1:object-1", + }, + { + name: "dequeue from second round-robin child when first child is empty", + nodesByDepth: []NodeType{roundRobin, roundRobin}, + childQueueObjects: map[string][]any{"child-1": {nil}, "child-2": {"child-2:some-object"}}, + numDequeuesToExpected: 1, + expected: "child-2:some-object", + }, + { + name: "dequeue from round-robin grandchild when non-empty", + nodesByDepth: []NodeType{roundRobin, roundRobin, roundRobin}, + childQueueObjects: map[string][]any{"child-1": {"child-1:object-1", "child-1:object-2"}, "child-2": {"child-2:object-1"}}, + numDequeuesToExpected: 3, + expected: "grandchild-1:object-1", + grandchildren: map[string]struct { + path QueuePath + objs []any + }{"grandchild-1": {path: QueuePath{"child-1"}, objs: []any{"grandchild-1:object-1"}}}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.nodesByDepth, &ShuffleShardState{map[TenantID]map[QuerierID]struct{}{}, nil}) + require.NoError(t, err) + + for _, sqo := range tt.selfQueueObjects { + err = tree.EnqueueBackByPath(QueuePath{}, sqo) + require.NoError(t, err) + } + for childName, objs := range tt.childQueueObjects { + for _, obj := range objs { + err = tree.EnqueueBackByPath(QueuePath{childName}, obj) + } + } + + for grandchildName, grandchildData := range tt.grandchildren { + for _, obj := range grandchildData.objs { + err = tree.EnqueueBackByPath(append(grandchildData.path, grandchildName), obj) + require.NoError(t, err) + } + } + + var finalValue any + for i := 0; i < tt.numDequeuesToExpected; i++ { + _, finalValue = tree.Dequeue() + fmt.Println("v: ", finalValue) + } + v, ok := finalValue.(string) + require.True(t, ok) + require.Equal(t, tt.expected, v) + }) + } +} + +func Test_ShuffleShardDequeue(t *testing.T) { + type enqueueObj struct { + obj any + path QueuePath + } + tests := []struct { + name string + nodesByDepth []NodeType + tqMap map[TenantID]map[QuerierID]struct{} + currQuerier []QuerierID + enqueueObjs []enqueueObj + expected []any + expectErr bool + }{ + { + name: "happy path - tenant found in tenant-querier map under first child", + nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{{obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}}, + expected: []any{"query-1"}, + }, + { + name: "tenant exists, but not for querier", + nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{{obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}}, + expected: []any{nil}, + }, + { + name: "1 of 3 tenants exist for querier", + nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{{obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, {obj: "query-3", path: QueuePath{"query-component-1", "tenant-3"}}}, + expected: []any{"query-3"}, + }, + { + name: "tenant exists for querier on next parent node", + nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, + {obj: "query-4", path: QueuePath{"query-component-1", "tenant-3"}}, + }, + expected: []any{"query-4", "query-3", nil}, + }, + { + name: "2 of 3 tenants exist for querier", + nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, + }, + expected: []any{"query-1", "query-3", nil}, + }, + { + name: "root node is shuffle-shard node", + nodesByDepth: []NodeType{shuffleShard, roundRobin, roundRobin}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-2", "query-component-1"}}, + {obj: "query-2", path: QueuePath{"tenant-1", "query-component-2"}}, + {obj: "query-3", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{"query-2", "query-3", nil}, + }, + { + name: "dequeueing for one querier returns nil, but does return for a different querier", + nodesByDepth: []NodeType{shuffleShard, roundRobin, roundRobin}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-2", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{nil, "query-1"}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + currentQuerier := QuerierID("") + tree, err := NewTree(tt.nodesByDepth, &ShuffleShardState{tt.tqMap, ¤tQuerier}) + require.NoError(t, err) + + for _, o := range tt.enqueueObjs { + err = tree.EnqueueBackByPath(o.path, o.obj) + require.NoError(t, err) + } + // currQuerier at position i is used to dequeue the expected result at position i + require.Equal(t, len(tt.currQuerier), len(tt.expected)) + for i := 0; i < len(tt.expected); i++ { + tree.currentQuerier = &tt.currQuerier[i] + _, v := tree.Dequeue() + require.Equal(t, tt.expected[i], v) + } + }) + } + +} From ee2a8046368a40b06801dc4fabf25fecec9e6707 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Wed, 10 Apr 2024 14:13:30 -0700 Subject: [PATCH 02/26] Add test to illustrate updating tenantQuerierAssignments outside of tree --- pkg/scheduler/queue/tree_queue_sketch.go | 35 ++++++++------- pkg/scheduler/queue/tree_queue_sketch_test.go | 43 ++++++++++++++++++- 2 files changed, 61 insertions(+), 17 deletions(-) diff --git a/pkg/scheduler/queue/tree_queue_sketch.go b/pkg/scheduler/queue/tree_queue_sketch.go index 8709e1245b2..e169a1f26ab 100644 --- a/pkg/scheduler/queue/tree_queue_sketch.go +++ b/pkg/scheduler/queue/tree_queue_sketch.go @@ -89,12 +89,12 @@ func (t *Tree) newNode(name string, depth int) TreeNodeIFace { } case shuffleShard: return &ShuffleShardNode{ - name: name, - localQueue: list.New(), - queueOrder: make([]string, 0), - queueMap: make(map[string]TreeNodeIFace, 1), - depth: depth, - stateUpdateInfo: t.ShuffleShardState, + name: name, + localQueue: list.New(), + queueOrder: make([]string, 0), + queueMap: make(map[string]TreeNodeIFace, 1), + depth: depth, + ShuffleShardState: t.ShuffleShardState, } default: panic("no defined node type at provided depth") @@ -218,13 +218,13 @@ func (rrn *RoundRobinNode) IsEmpty() bool { } type ShuffleShardNode struct { - name string - localQueue *list.List // should never be populated - queuePosition int - queueOrder []string // will be a slice of tenants (+ self?) - queueMap map[string]TreeNodeIFace - depth int - stateUpdateInfo *ShuffleShardState + name string + localQueue *list.List // should never be populated + queuePosition int + queueOrder []string // will be a slice of tenants (+ self?) + queueMap map[string]TreeNodeIFace + depth int + *ShuffleShardState } func (ssn *ShuffleShardNode) Name() string { @@ -242,6 +242,11 @@ func (ssn *ShuffleShardNode) dequeue() (path QueuePath, v any) { return path, nil } + // can't get a tenant if no querier set + if ssn.currentQuerier == nil { + return path, nil + } + // no children, but has a non-zero length local queue -- should probably never happen? if len(ssn.queueOrder) == 0 { if elt := ssn.localQueue.Front(); elt != nil { @@ -274,8 +279,8 @@ func (ssn *ShuffleShardNode) dequeueGetNode() (string, TreeNodeIFace) { checkIndex = 0 } tenantName := ssn.queueOrder[checkIndex] - if tenantQuerierSet, ok := ssn.stateUpdateInfo.tenantQuerierMap[TenantID(tenantName)]; ok { - if _, ok := tenantQuerierSet[*ssn.stateUpdateInfo.currentQuerier]; ok { + if tenantQuerierSet, ok := ssn.tenantQuerierMap[TenantID(tenantName)]; ok { + if _, ok := tenantQuerierSet[*ssn.currentQuerier]; ok { return tenantName, ssn.queueMap[tenantName] } } diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go index 05772fb7765..7da94eb90b5 100644 --- a/pkg/scheduler/queue/tree_queue_sketch_test.go +++ b/pkg/scheduler/queue/tree_queue_sketch_test.go @@ -1,7 +1,6 @@ package queue import ( - "fmt" "github.com/stretchr/testify/require" "testing" ) @@ -207,7 +206,6 @@ func Test_RoundRobinDequeue(t *testing.T) { var finalValue any for i := 0; i < tt.numDequeuesToExpected; i++ { _, finalValue = tree.Dequeue() - fmt.Println("v: ", finalValue) } v, ok := finalValue.(string) require.True(t, ok) @@ -324,3 +322,44 @@ func Test_ShuffleShardDequeue(t *testing.T) { } } + +// This test is a little messy; I can clean it up, but it's meant to illustrate that we can update a state +// in tenantQuerierAssignments, and the tree dequeue behavior will adjust accordingly. +func Test_ChangeShuffleShardState(t *testing.T) { + tqa := tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}}, + } + + state := &ShuffleShardState{ + tenantQuerierMap: tqa.tenantQuerierIDs, + currentQuerier: nil, + } + + tree, err := NewTree([]NodeType{shuffleShard, roundRobin, roundRobin}, state) + require.NoError(t, err) + err = tree.EnqueueBackByPath(QueuePath{"tenant-1", "query-component-1"}, "query-1") + err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-2") + err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-3") + require.NoError(t, err) + + querier1 := QuerierID("querier-1") + querier2 := QuerierID("querier-2") + querier3 := QuerierID("querier-3") + + // set state to querier-2 should dequeue query-2 + state.currentQuerier = &querier2 + _, v := tree.Dequeue() + require.Equal(t, "query-2", v) + + // set state to querier-1 should dequeue query-1 + state.currentQuerier = &querier1 + _, v = tree.Dequeue() + require.Equal(t, "query-1", v) + + // update tqa map to assign querier-3 to tenant-2, then set state to querier-3 should dequeue query-3 + tqa.tenantQuerierIDs["tenant-2"]["querier-3"] = struct{}{} + state.currentQuerier = &querier3 + _, v = tree.Dequeue() + require.Equal(t, "query-3", v) + +} From fadc1be2fb46271f353df56d79896399c79c60b9 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Fri, 26 Apr 2024 16:14:16 -0700 Subject: [PATCH 03/26] pull out dequeue ops --- pkg/scheduler/queue/tree_queue_sketch.go | 211 ++++++++++++------ pkg/scheduler/queue/tree_queue_sketch_test.go | 59 +++-- 2 files changed, 183 insertions(+), 87 deletions(-) diff --git a/pkg/scheduler/queue/tree_queue_sketch.go b/pkg/scheduler/queue/tree_queue_sketch.go index e169a1f26ab..a5a2f9116da 100644 --- a/pkg/scheduler/queue/tree_queue_sketch.go +++ b/pkg/scheduler/queue/tree_queue_sketch.go @@ -32,6 +32,7 @@ type Tree struct { type ShuffleShardState struct { tenantQuerierMap map[TenantID]map[QuerierID]struct{} currentQuerier *QuerierID + nodesChecked int } func NewTree(nodeTypeByDepth []NodeType, shuffleShardState *ShuffleShardState) (*Tree, error) { @@ -108,6 +109,11 @@ type RoundRobinNode struct { queueOrder []string // order for dequeueing from self/children queueMap map[string]TreeNodeIFace depth int + roundRobinState +} + +type roundRobinState struct { + nodesChecked int } func (rrn *RoundRobinNode) enqueueBackByPath(t *Tree, pathFromNode QueuePath, v any) error { @@ -129,7 +135,6 @@ func (rrn *RoundRobinNode) getOrAddNode(t *Tree, pathFromNode QueuePath) (TreeNo return rrn, nil } - // TODO (casie): make rrn.queueMap on node creation; why not? var childNode TreeNodeIFace var ok bool if childNode, ok = rrn.queueMap[pathFromNode[0]]; !ok { @@ -159,57 +164,79 @@ func (rrn *RoundRobinNode) Name() string { func (rrn *RoundRobinNode) dequeue() (QueuePath, any) { var v any + var childPath QueuePath - initialLen := len(rrn.queueOrder) path := QueuePath{rrn.name} - // iterate over all elements in rrn.queueOrder. If we ever find a valid value v, we'll return; - // otherwise, return nil if we iterate over all rrn.queueOrder and find nothing - for i := 0; i <= initialLen && v == nil; i++ { - if rrn.queuePosition == localQueueIndex { - // The next item should be dequeued from the local queue. + if rrn.IsEmpty() { + return path, nil + } + + var checkedAllNodes bool + var dequeueNode TreeNodeIFace + // continue until we've found a value or checked all nodes that need checking + for v == nil && !checkedAllNodes { + dequeueNode, checkedAllNodes = rrn.dequeueGetNode() + // dequeueing from local queue + if dequeueNode == rrn { if rrn.localQueue.Len() > 0 { + // dequeueNode is self, local queue non-empty if elt := rrn.localQueue.Front(); elt != nil { rrn.localQueue.Remove(elt) v = elt.Value } } - rrn.queuePosition++ - if rrn.queuePosition >= len(rrn.queueOrder) { - rrn.queuePosition = localQueueIndex - } + } else if dequeueNode == nil { + // no dequeue-able node found + return path, v } else { - // dequeue from a child node - childNodeName := rrn.queueOrder[rrn.queuePosition] - childPath := QueuePath{} - childNode := rrn.queueMap[childNodeName] - childPath, v = childNode.dequeue() - path = append(path, childPath...) - - if childNode.IsEmpty() { - // - delete child node from rrn.queueOrder - // - delete child node from rrn.queueMap - delete(rrn.queueMap, childNodeName) - // queueOrder may have been updated by another process? Make sure to only remove a node of matching name + // dequeue from a child + childPath, v = dequeueNode.dequeue() + if dequeueNode.IsEmpty() { + deleteNodeName := dequeueNode.Name() + delete(rrn.queueMap, deleteNodeName) for idx, name := range rrn.queueOrder { - if name == childNodeName { + if name == deleteNodeName { rrn.queueOrder = append(rrn.queueOrder[:idx], rrn.queueOrder[idx+1:]...) } } - // rrn.queuePosition only needs updated if it was originally at the end of the slice - if rrn.queuePosition >= len(rrn.queueOrder) { - rrn.queuePosition = localQueueIndex - } - - } else { // the child node is either dequeued from or not, but is non-empty so we leave it be - rrn.queuePosition++ - if rrn.queuePosition >= len(rrn.queueOrder) { - rrn.queuePosition = localQueueIndex - } + // removing an element sets our position one step forward; reset it to original queuePosition + rrn.queuePosition-- } } + rrn.dequeueUpdateState(v) + } + return append(path, childPath...), v +} + +func (rrn *RoundRobinNode) dequeueGetNode() (TreeNodeIFace, bool) { + checkedAllNodes := rrn.nodesChecked == len(rrn.queueOrder)+1 // must check local queue as well + if rrn.queuePosition == localQueueIndex { + return rrn, checkedAllNodes + } + currentNodeName := rrn.queueOrder[rrn.queuePosition] + // if the node is in queueMap, return it + if node, ok := rrn.queueMap[currentNodeName]; ok { + return node, checkedAllNodes + } + // if not in queueMap, return nil + return nil, checkedAllNodes +} + +func (rrn *RoundRobinNode) dequeueUpdateState(v any) { + // non-nil value dequeued; reset node check counter + if v != nil { + rrn.nodesChecked = 0 + } else { + // v is still nil, increment node check counter + rrn.nodesChecked++ + } + + // either way, advance queue position + rrn.queuePosition++ + if rrn.queuePosition >= len(rrn.queueOrder) { + rrn.queuePosition = localQueueIndex } - return path, v } func (rrn *RoundRobinNode) IsEmpty() bool { @@ -231,62 +258,110 @@ func (ssn *ShuffleShardNode) Name() string { return ssn.name } -func (ssn *ShuffleShardNode) dequeue() (path QueuePath, v any) { - // start from ssn.queuePosition - // check that tenant for querierID against availabilityMap - // if exists, move element to "back" of queue - // if doesn't exist, check next child - // nothing here to dequeue - path = QueuePath{ssn.name} +// start from ssn.queuePosition +// check that tenant for querierID against availabilityMap +// if exists, move element to "back" of queue +// if doesn't exist, check next child +// nothing here to dequeue +func (ssn *ShuffleShardNode) dequeue() (QueuePath, any) { + var v any + var childPath QueuePath + + path := QueuePath{ssn.name} + if ssn.IsEmpty() { return path, nil } - // can't get a tenant if no querier set - if ssn.currentQuerier == nil { - return path, nil - } + var checkedAllNodes bool + var dequeueNode TreeNodeIFace - // no children, but has a non-zero length local queue -- should probably never happen? - if len(ssn.queueOrder) == 0 { - if elt := ssn.localQueue.Front(); elt != nil { - ssn.localQueue.Remove(elt) - return path, elt.Value + for v == nil && !checkedAllNodes { + dequeueNode, checkedAllNodes = ssn.dequeueGetNode() + if dequeueNode == ssn { + if ssn.localQueue.Len() > 0 { + if elt := ssn.localQueue.Front(); elt != nil { + ssn.localQueue.Remove(elt) + v = elt.Value + } + } + } else if dequeueNode == nil { + // no node found for querier + return path, v + } else { + childPath, v = dequeueNode.dequeue() + if dequeueNode.IsEmpty() { + deleteNodeName := dequeueNode.Name() + delete(ssn.queueMap, deleteNodeName) + for idx, name := range ssn.queueOrder { + if name == deleteNodeName { + ssn.queueOrder = append(ssn.queueOrder[:idx], ssn.queueOrder[idx+1:]...) + } + } + // removing an element sets our position one step forward; reset to original + ssn.queuePosition-- + } } + ssn.dequeueUpdateState(v, dequeueNode.Name()) } + return append(path, childPath...), v +} - // node has children, dequeue from them first - // get the value - nodeName, dequeueNode := ssn.dequeueGetNode() - - // no node for this querier - if dequeueNode == nil { - return path, v +// move dequeueNode to the back of queueOrder, regardless of whether a value was found? +func (ssn *ShuffleShardNode) dequeueUpdateState(v any, nodeName string) { + if v != nil { + ssn.nodesChecked = 0 + ssn.queuePosition++ + return } - childPath, v := dequeueNode.dequeue() - ssn.dequeueMoveToBack(nodeName) - // move the node in our order if appropriate + ssn.nodesChecked++ - return append(path, childPath...), v + // if dequeueNode was self, advance queue position no matter what, and return + if ssn.queuePosition == localQueueIndex { + ssn.queuePosition++ + return + } + + // otherwise, don't change queuePosition, only move element to back + for i, child := range ssn.queueOrder { + if child == nodeName { + ssn.queueOrder = append(ssn.queueOrder[:i], append(ssn.queueOrder[i+1:], child)...) // ugly + } + } } // return a function which selects the tree node from which to dequeue; return fn passed to tree -func (ssn *ShuffleShardNode) dequeueGetNode() (string, TreeNodeIFace) { +func (ssn *ShuffleShardNode) dequeueGetNode() (TreeNodeIFace, bool) { + // can't get a tenant if no querier set + if ssn.currentQuerier == nil { + return nil, true + } + + checkedAllNodes := ssn.nodesChecked == len(ssn.queueOrder)+1 // must check local queue as well + // no children + if len(ssn.queueOrder) == 0 || ssn.queuePosition == localQueueIndex { + return ssn, checkedAllNodes + } + checkIndex := ssn.queuePosition + for iters := 0; iters < len(ssn.queueOrder); iters++ { - if checkIndex > len(ssn.queueOrder) { + if checkIndex >= len(ssn.queueOrder) { checkIndex = 0 } tenantName := ssn.queueOrder[checkIndex] + // increment nodes checked even if not in tenant-querier map + ssn.nodesChecked++ + checkedAllNodes = ssn.nodesChecked == len(ssn.queueOrder) if tenantQuerierSet, ok := ssn.tenantQuerierMap[TenantID(tenantName)]; ok { if _, ok := tenantQuerierSet[*ssn.currentQuerier]; ok { - return tenantName, ssn.queueMap[tenantName] + return ssn.queueMap[tenantName], checkedAllNodes } } checkIndex++ } - return "", nil + return nil, checkedAllNodes } func (ssn *ShuffleShardNode) dequeueMoveToBack(name string) { @@ -322,9 +397,9 @@ func (ssn *ShuffleShardNode) getOrAddNode(t *Tree, path QueuePath) (TreeNodeIFac return nil, fmt.Errorf("cannot create a node at depth %v; greater than max depth %v", ssn.depth+1, len(t.nodeTypesByDepth)-1) } childNode = t.newNode(path[0], ssn.depth+1) - // add childNode to rrn.queueMap + // add childNode to ssn.queueMap ssn.queueMap[childNode.Name()] = childNode - // update rrn.queueOrder to place childNode behind rrn.queuePosition + // update ssn.queueOrder to place childNode behind rrn.queuePosition if ssn.queuePosition <= 0 { ssn.queueOrder = append(ssn.queueOrder, childNode.Name()) } else { diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go index 7da94eb90b5..d88da9baf1a 100644 --- a/pkg/scheduler/queue/tree_queue_sketch_test.go +++ b/pkg/scheduler/queue/tree_queue_sketch_test.go @@ -1,38 +1,47 @@ package queue import ( + "fmt" "github.com/stretchr/testify/require" "testing" ) func Test_NewTree(t *testing.T) { tests := []struct { - name string - args []NodeType - expectErr bool + name string + nodesByDepth []NodeType + state *ShuffleShardState + expectErr bool }{ { - name: "create round-robin-only tree", - args: []NodeType{roundRobin, roundRobin}, + name: "create round-robin-only tree", + nodesByDepth: []NodeType{roundRobin, roundRobin}, }, { - name: "create shuffle-shard-only tree", - args: []NodeType{shuffleShard}, + name: "create shuffle-shard-only tree", + nodesByDepth: []NodeType{shuffleShard}, + state: &ShuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, }, { - name: "create empty tree", - args: []NodeType{}, - expectErr: true, + name: "create empty tree", + nodesByDepth: []NodeType{}, + expectErr: true, }, { - name: "create tree with multiple shuffle-shard depths", - args: []NodeType{shuffleShard, roundRobin, shuffleShard}, - expectErr: true, + name: "create shuffle-shard tree with no tenant-querier map", + nodesByDepth: []NodeType{shuffleShard}, + state: &ShuffleShardState{}, + expectErr: true, + }, + { + name: "create tree with multiple shuffle-shard depths", + nodesByDepth: []NodeType{shuffleShard, roundRobin, shuffleShard}, + expectErr: true, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - _, err := NewTree(tt.args, &ShuffleShardState{map[TenantID]map[QuerierID]struct{}{}, nil}) + _, err := NewTree(tt.nodesByDepth, tt.state) if tt.expectErr { require.Error(t, err) } else { @@ -80,7 +89,7 @@ func Test_EnqueueCreatingNode(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.nodeTypeByDepth, &ShuffleShardState{map[TenantID]map[QuerierID]struct{}{}, nil}) + tree, err := NewTree(tt.nodeTypeByDepth, &ShuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}) require.NoError(t, err) err = tree.EnqueueBackByPath(QueuePath{tt.enqueueNodeName}, "some-object") if tt.expectErr { @@ -119,7 +128,8 @@ func Test_Dequeue_RootNode(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree([]NodeType{tt.rootNodeType}, &ShuffleShardState{map[TenantID]map[QuerierID]struct{}{}, nil}) + querierID := QuerierID("placeholder") + tree, err := NewTree([]NodeType{tt.rootNodeType}, &ShuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, currentQuerier: &querierID}) require.NoError(t, err) if tt.enqueueToRoot != nil { err = tree.EnqueueBackByPath(QueuePath{}, tt.enqueueToRoot) @@ -183,7 +193,7 @@ func Test_RoundRobinDequeue(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.nodesByDepth, &ShuffleShardState{map[TenantID]map[QuerierID]struct{}{}, nil}) + tree, err := NewTree(tt.nodesByDepth, &ShuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}) require.NoError(t, err) for _, sqo := range tt.selfQueueObjects { @@ -299,12 +309,22 @@ func Test_ShuffleShardDequeue(t *testing.T) { }, expected: []any{nil, "query-1"}, }, + { + name: "no querier set in state", + nodesByDepth: []NodeType{shuffleShard, roundRobin, roundRobin}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + currQuerier: []QuerierID{""}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{nil}, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { currentQuerier := QuerierID("") - tree, err := NewTree(tt.nodesByDepth, &ShuffleShardState{tt.tqMap, ¤tQuerier}) + tree, err := NewTree(tt.nodesByDepth, &ShuffleShardState{tenantQuerierMap: tt.tqMap, currentQuerier: ¤tQuerier}) require.NoError(t, err) for _, o := range tt.enqueueObjs { @@ -315,7 +335,8 @@ func Test_ShuffleShardDequeue(t *testing.T) { require.Equal(t, len(tt.currQuerier), len(tt.expected)) for i := 0; i < len(tt.expected); i++ { tree.currentQuerier = &tt.currQuerier[i] - _, v := tree.Dequeue() + path, v := tree.Dequeue() + fmt.Println(path, v) require.Equal(t, tt.expected[i], v) } }) From 3cd24810efeea9ce477882acf304ab68542cdc23 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Wed, 1 May 2024 12:36:28 -0700 Subject: [PATCH 04/26] Deduplicate node logic and extract node state types --- pkg/scheduler/queue/tree_queue_sketch.go | 483 ++++++---------- pkg/scheduler/queue/tree_queue_sketch_test.go | 524 ++++++++++++------ 2 files changed, 520 insertions(+), 487 deletions(-) diff --git a/pkg/scheduler/queue/tree_queue_sketch.go b/pkg/scheduler/queue/tree_queue_sketch.go index a5a2f9116da..bcb709a01e9 100644 --- a/pkg/scheduler/queue/tree_queue_sketch.go +++ b/pkg/scheduler/queue/tree_queue_sketch.go @@ -5,119 +5,82 @@ import ( "fmt" ) +// TODO (casie): We might not need this interface anymore. type TreeNodeIFace interface { //EnqueueFrontByPath(QueuePath, any) error - enqueueBackByPath(*Tree, QueuePath, any) error + enqueueBackByPath(OpsPath, any) error dequeue() (QueuePath, any) // Dequeue returns the dequeued value, and the QueuePath (starting with root) to the node which was dequeued from IsEmpty() bool Name() string - getOrAddNode(*Tree, QueuePath) (TreeNodeIFace, error) getLocalQueue() *list.List + getOrAddNode(OpsPath) (TreeNodeIFace, error) } -type NodeType string - -const roundRobin = NodeType("round-robin") - -// shuffleShard node uses an externally maintained shuffle sharding mapping to select the next item to dequeue -const shuffleShard = NodeType("shuffle-shard") - -type Tree struct { - nodeTypesByDepth []NodeType - maxDepth int - rootNode TreeNodeIFace - *ShuffleShardState +type DequeueAlgorithm interface { + ops() dequeueOps } -type ShuffleShardState struct { - tenantQuerierMap map[TenantID]map[QuerierID]struct{} - currentQuerier *QuerierID - nodesChecked int +type OpsPath []struct { + name string + dequeueAlgorithm DequeueAlgorithm } -func NewTree(nodeTypeByDepth []NodeType, shuffleShardState *ShuffleShardState) (*Tree, error) { - if len(nodeTypeByDepth) <= 0 { - return nil, fmt.Errorf("no node types provided") - } +type dequeueGetNodeFunc func(*Node) (TreeNodeIFace, bool) +type dequeueUpdateStateFunc func(*Node, any, string) - shuffleShardCount := 0 - for _, nt := range nodeTypeByDepth { - if nt == shuffleShard { - shuffleShardCount++ - } - } - if shuffleShardCount > 1 { - return nil, fmt.Errorf("more than one tree layer with type shuffle-shard not currently supported") - } - - if shuffleShardCount > 0 && shuffleShardState.tenantQuerierMap == nil { - return nil, fmt.Errorf("cannot create tree with shuffle-shard nodes, without reference to tenant-querier map") - } - - t := &Tree{ - nodeTypesByDepth: nodeTypeByDepth, - maxDepth: len(nodeTypeByDepth) - 1, - rootNode: nil, - ShuffleShardState: shuffleShardState, - } - t.rootNode = t.newNode("root", 0) - return t, nil +type dequeueOps struct { + getNode dequeueGetNodeFunc + updateState dequeueUpdateStateFunc } -// EnqueueBackByPath takes a QueuePath (excluding the root node), and -func (t *Tree) EnqueueBackByPath(path QueuePath, v any) error { - // nodes need *Tree in order to know what type of node to create if necessary, and - if len(path) > t.maxDepth { - return fmt.Errorf("cannot create path with depth: %v; greater than max depth %v", len(path), t.maxDepth) - } - return t.rootNode.enqueueBackByPath(t, path, v) +type Node struct { + name string + localQueue *list.List + queuePosition int // position for dequeueing from queueOrder + queueOrder []string // order for dequeueing from self/children + queueMap map[string]TreeNodeIFace + depth int + dequeueAlgorithm DequeueAlgorithm } -func (t *Tree) Dequeue() (QueuePath, any) { - return t.rootNode.dequeue() -} +// TODO (casie): Do we want some concept of a max-depth? -func (t *Tree) newNode(name string, depth int) TreeNodeIFace { - switch t.nodeTypesByDepth[depth] { - case roundRobin: - return &RoundRobinNode{ - name: name, - localQueue: list.New(), - queuePosition: localQueueIndex, - queueOrder: make([]string, 0), - queueMap: make(map[string]TreeNodeIFace, 1), - depth: depth, - } - case shuffleShard: - return &ShuffleShardNode{ - name: name, - localQueue: list.New(), - queueOrder: make([]string, 0), - queueMap: make(map[string]TreeNodeIFace, 1), - depth: depth, - ShuffleShardState: t.ShuffleShardState, +func NewNode(name string, da DequeueAlgorithm) (TreeNodeIFace, error) { + // TODO (casie): write a unit test + if da == nil { + return nil, fmt.Errorf("cannot create a node without a defined dequeueing algorithm") + } + switch da.(type) { + case *shuffleShardState: + if da.(*shuffleShardState).tenantQuerierMap == nil { + return nil, fmt.Errorf("cannot create a shufffle-shard node with nil tenant-querier map") } - default: - panic("no defined node type at provided depth") } + return &Node{ + name: name, + localQueue: list.New(), + queuePosition: localQueueIndex, + queueOrder: make([]string, 0), + queueMap: make(map[string]TreeNodeIFace, 1), + dequeueAlgorithm: da, + }, nil +} + +func (n *Node) IsEmpty() bool { + // node has nothing in local queue, and no child nodes + return n.localQueue.Len() == 0 && len(n.queueMap) == 0 } -type RoundRobinNode struct { - name string - localQueue *list.List - queuePosition int // position for dequeueing from queueOrder - queueOrder []string // order for dequeueing from self/children - queueMap map[string]TreeNodeIFace - depth int - roundRobinState +func (n *Node) Name() string { + return n.name } -type roundRobinState struct { - nodesChecked int +func (n *Node) getLocalQueue() *list.List { + return n.localQueue } -func (rrn *RoundRobinNode) enqueueBackByPath(t *Tree, pathFromNode QueuePath, v any) error { - childNode, err := rrn.getOrAddNode(t, pathFromNode) +func (n *Node) enqueueBackByPath(pathFromNode OpsPath, v any) error { + childNode, err := n.getOrAddNode(pathFromNode) if err != nil { return err } @@ -126,49 +89,13 @@ func (rrn *RoundRobinNode) enqueueBackByPath(t *Tree, pathFromNode QueuePath, v return nil } -func (rrn *RoundRobinNode) getLocalQueue() *list.List { - return rrn.localQueue -} - -func (rrn *RoundRobinNode) getOrAddNode(t *Tree, pathFromNode QueuePath) (TreeNodeIFace, error) { - if len(pathFromNode) == 0 { - return rrn, nil - } - - var childNode TreeNodeIFace - var ok bool - if childNode, ok = rrn.queueMap[pathFromNode[0]]; !ok { - if rrn.depth+1 > t.maxDepth { - return nil, fmt.Errorf("cannot create a node at depth %v; greater than max depth %v", rrn.depth+1, len(t.nodeTypesByDepth)-1) - } - // child does not exist, create it - childNode = t.newNode(pathFromNode[0], rrn.depth+1) - - // add childNode to rrn.queueMap - rrn.queueMap[childNode.Name()] = childNode - // add childNode to rrn.queueOrder before the current position, update rrn.queuePosition to current element - if rrn.queuePosition == localQueueIndex { - rrn.queueOrder = append(rrn.queueOrder, childNode.Name()) - } else { - rrn.queueOrder = append(rrn.queueOrder[:rrn.queuePosition], append([]string{childNode.Name()}, rrn.queueOrder[rrn.queuePosition:]...)...) - rrn.queuePosition++ - } - - } - return childNode.getOrAddNode(t, pathFromNode[1:]) -} - -func (rrn *RoundRobinNode) Name() string { - return rrn.name -} - -func (rrn *RoundRobinNode) dequeue() (QueuePath, any) { +func (n *Node) dequeue() (QueuePath, any) { var v any var childPath QueuePath - path := QueuePath{rrn.name} + path := QueuePath{n.name} - if rrn.IsEmpty() { + if n.IsEmpty() { return path, nil } @@ -176,13 +103,13 @@ func (rrn *RoundRobinNode) dequeue() (QueuePath, any) { var dequeueNode TreeNodeIFace // continue until we've found a value or checked all nodes that need checking for v == nil && !checkedAllNodes { - dequeueNode, checkedAllNodes = rrn.dequeueGetNode() + dequeueNode, checkedAllNodes = n.dequeueAlgorithm.ops().getNode(n) // dequeueing from local queue - if dequeueNode == rrn { - if rrn.localQueue.Len() > 0 { + if dequeueNode == n { + if n.localQueue.Len() > 0 { // dequeueNode is self, local queue non-empty - if elt := rrn.localQueue.Front(); elt != nil { - rrn.localQueue.Remove(elt) + if elt := n.localQueue.Front(); elt != nil { + n.localQueue.Remove(elt) v = elt.Value } } @@ -192,224 +119,170 @@ func (rrn *RoundRobinNode) dequeue() (QueuePath, any) { } else { // dequeue from a child childPath, v = dequeueNode.dequeue() + // if the dequeue node is empty _after_ dequeueing, delete it from children if dequeueNode.IsEmpty() { deleteNodeName := dequeueNode.Name() - delete(rrn.queueMap, deleteNodeName) - for idx, name := range rrn.queueOrder { + delete(n.queueMap, deleteNodeName) + for idx, name := range n.queueOrder { if name == deleteNodeName { - rrn.queueOrder = append(rrn.queueOrder[:idx], rrn.queueOrder[idx+1:]...) + n.queueOrder = append(n.queueOrder[:idx], n.queueOrder[idx+1:]...) } } // removing an element sets our position one step forward; reset it to original queuePosition - rrn.queuePosition-- + n.queuePosition-- } } - rrn.dequeueUpdateState(v) + n.dequeueAlgorithm.ops().updateState(n, v, dequeueNode.Name()) } return append(path, childPath...), v } -func (rrn *RoundRobinNode) dequeueGetNode() (TreeNodeIFace, bool) { - checkedAllNodes := rrn.nodesChecked == len(rrn.queueOrder)+1 // must check local queue as well - if rrn.queuePosition == localQueueIndex { - return rrn, checkedAllNodes - } - currentNodeName := rrn.queueOrder[rrn.queuePosition] - // if the node is in queueMap, return it - if node, ok := rrn.queueMap[currentNodeName]; ok { - return node, checkedAllNodes - } - // if not in queueMap, return nil - return nil, checkedAllNodes -} - -func (rrn *RoundRobinNode) dequeueUpdateState(v any) { - // non-nil value dequeued; reset node check counter - if v != nil { - rrn.nodesChecked = 0 - } else { - // v is still nil, increment node check counter - rrn.nodesChecked++ +// TODO (casie): Ugly -- if a matching node name is found, the dequeueAlgorithm will be ignored, which +// +// could lead to some unexpected behavior. +// +// getOrAddNode checks whether the first node name in pathFromNode exists in a node's children; +// if no node exists, one is created and added to the "end" of the node's children, from the perspective +// of node.queuePosition (e.g., if node.queuePosition is 4, the new node will be created at position 3). +// This follows queue behavior, but if, for some reason, we ever want to create nodes in a way that violates +// queueing behavior, this will require a refactor, as will, well, the entire "tree queue" data structure. +func (n *Node) getOrAddNode(pathFromNode OpsPath) (TreeNodeIFace, error) { + if len(pathFromNode) == 0 { + return n, nil } - // either way, advance queue position - rrn.queuePosition++ - if rrn.queuePosition >= len(rrn.queueOrder) { - rrn.queuePosition = localQueueIndex - } -} + var childNode TreeNodeIFace + var ok bool + var err error + if childNode, ok = n.queueMap[pathFromNode[0].name]; !ok { + // child does not exist, create it + childNode, err = NewNode(pathFromNode[0].name, pathFromNode[0].dequeueAlgorithm) + if err != nil { + return nil, err + } -func (rrn *RoundRobinNode) IsEmpty() bool { - // node has nothing in local queue, and no child nodes - return rrn.localQueue.Len() == 0 && len(rrn.queueMap) == 0 -} + // add childNode to rrn.queueMap + n.queueMap[childNode.Name()] = childNode + // add childNode to rrn.queueOrder before the current position, update rrn.queuePosition to current element + if n.queuePosition == localQueueIndex { + n.queueOrder = append(n.queueOrder, childNode.Name()) + } else { + n.queueOrder = append(n.queueOrder[:n.queuePosition], append([]string{childNode.Name()}, n.queueOrder[n.queuePosition:]...)...) + n.queuePosition++ + } -type ShuffleShardNode struct { - name string - localQueue *list.List // should never be populated - queuePosition int - queueOrder []string // will be a slice of tenants (+ self?) - queueMap map[string]TreeNodeIFace - depth int - *ShuffleShardState + } + // TODO (casie): How to determine what dequeueAlgo to pass? + return childNode.getOrAddNode(pathFromNode[1:]) } -func (ssn *ShuffleShardNode) Name() string { - return ssn.name +type roundRobinState struct { + subtreeNodesChecked int } -// start from ssn.queuePosition -// check that tenant for querierID against availabilityMap -// if exists, move element to "back" of queue -// if doesn't exist, check next child -// nothing here to dequeue -func (ssn *ShuffleShardNode) dequeue() (QueuePath, any) { - var v any - var childPath QueuePath - - path := QueuePath{ssn.name} - - if ssn.IsEmpty() { - return path, nil +func (rrs *roundRobinState) ops() dequeueOps { + getNode := func(n *Node) (TreeNodeIFace, bool) { + checkedAllNodes := rrs.subtreeNodesChecked == len(n.queueOrder)+1 + if n.queuePosition == localQueueIndex { + return n, checkedAllNodes + } + currentNodeName := n.queueOrder[n.queuePosition] + if node, ok := n.queueMap[currentNodeName]; ok { + return node, checkedAllNodes + } + return nil, checkedAllNodes } - var checkedAllNodes bool - var dequeueNode TreeNodeIFace - - for v == nil && !checkedAllNodes { - dequeueNode, checkedAllNodes = ssn.dequeueGetNode() - if dequeueNode == ssn { - if ssn.localQueue.Len() > 0 { - if elt := ssn.localQueue.Front(); elt != nil { - ssn.localQueue.Remove(elt) - v = elt.Value - } - } - } else if dequeueNode == nil { - // no node found for querier - return path, v + // TODO (casie): Ignoring nodeName here because it doesn't matter...ugly + updateState := func(n *Node, v any, _ string) { + if v != nil { + rrs.subtreeNodesChecked = 0 } else { - childPath, v = dequeueNode.dequeue() - if dequeueNode.IsEmpty() { - deleteNodeName := dequeueNode.Name() - delete(ssn.queueMap, deleteNodeName) - for idx, name := range ssn.queueOrder { - if name == deleteNodeName { - ssn.queueOrder = append(ssn.queueOrder[:idx], ssn.queueOrder[idx+1:]...) - } - } - // removing an element sets our position one step forward; reset to original - ssn.queuePosition-- - } + rrs.subtreeNodesChecked++ } - ssn.dequeueUpdateState(v, dequeueNode.Name()) - } - return append(path, childPath...), v -} -// move dequeueNode to the back of queueOrder, regardless of whether a value was found? -func (ssn *ShuffleShardNode) dequeueUpdateState(v any, nodeName string) { - if v != nil { - ssn.nodesChecked = 0 - ssn.queuePosition++ - return + n.queuePosition++ + if n.queuePosition >= len(n.queueOrder) { + n.queuePosition = localQueueIndex + } } - ssn.nodesChecked++ - - // if dequeueNode was self, advance queue position no matter what, and return - if ssn.queuePosition == localQueueIndex { - ssn.queuePosition++ - return + return dequeueOps{ + getNode: getNode, + updateState: updateState, } +} - // otherwise, don't change queuePosition, only move element to back - for i, child := range ssn.queueOrder { - if child == nodeName { - ssn.queueOrder = append(ssn.queueOrder[:i], append(ssn.queueOrder[i+1:], child)...) // ugly - } - } +type shuffleShardState struct { + tenantQuerierMap map[TenantID]map[QuerierID]struct{} + currentQuerier *QuerierID + nodesChecked int } -// return a function which selects the tree node from which to dequeue; return fn passed to tree -func (ssn *ShuffleShardNode) dequeueGetNode() (TreeNodeIFace, bool) { - // can't get a tenant if no querier set - if ssn.currentQuerier == nil { - return nil, true - } +func (sss *shuffleShardState) ops() dequeueOps { + // start from ssn.queuePosition + // check that tenant for querierID against availabilityMap + // if exists, move element to "back" of queue + // if doesn't exist, check next child + // nothing here to dequeue + getNode := func(n *Node) (TreeNodeIFace, bool) { + // can't get a tenant if no querier set + if sss.currentQuerier == nil { + return nil, true + } - checkedAllNodes := ssn.nodesChecked == len(ssn.queueOrder)+1 // must check local queue as well - // no children - if len(ssn.queueOrder) == 0 || ssn.queuePosition == localQueueIndex { - return ssn, checkedAllNodes - } + checkedAllNodes := sss.nodesChecked == len(n.queueOrder)+1 // must check local queue as well + // no children + if len(n.queueOrder) == 0 || n.queuePosition == localQueueIndex { + return n, checkedAllNodes + } - checkIndex := ssn.queuePosition + checkIndex := n.queuePosition - for iters := 0; iters < len(ssn.queueOrder); iters++ { - if checkIndex >= len(ssn.queueOrder) { - checkIndex = 0 - } - tenantName := ssn.queueOrder[checkIndex] - // increment nodes checked even if not in tenant-querier map - ssn.nodesChecked++ - checkedAllNodes = ssn.nodesChecked == len(ssn.queueOrder) - if tenantQuerierSet, ok := ssn.tenantQuerierMap[TenantID(tenantName)]; ok { - if _, ok := tenantQuerierSet[*ssn.currentQuerier]; ok { - return ssn.queueMap[tenantName], checkedAllNodes + for iters := 0; iters < len(n.queueOrder); iters++ { + if checkIndex >= len(n.queueOrder) { + checkIndex = 0 + } + tenantName := n.queueOrder[checkIndex] + // increment nodes checked even if not in tenant-querier map + sss.nodesChecked++ + checkedAllNodes = sss.nodesChecked == len(n.queueOrder) + if tenantQuerierSet, ok := sss.tenantQuerierMap[TenantID(tenantName)]; ok { + if _, ok := tenantQuerierSet[*sss.currentQuerier]; ok { + return n.queueMap[tenantName], checkedAllNodes + } } + checkIndex++ } - checkIndex++ + return nil, checkedAllNodes } - return nil, checkedAllNodes -} -func (ssn *ShuffleShardNode) dequeueMoveToBack(name string) { - for i, child := range ssn.queueOrder { - if child == name { - ssn.queueOrder = append(ssn.queueOrder[:i], append(ssn.queueOrder[i+1:], child)...) // ugly + updateState := func(n *Node, v any, nodeName string) { + if v != nil { + sss.nodesChecked = 0 + n.queuePosition++ + return } - } -} -func (ssn *ShuffleShardNode) enqueueBackByPath(t *Tree, path QueuePath, v any) error { - nodeToQueueTo, err := ssn.getOrAddNode(t, path) - if err != nil { - return err - } - nodeToQueueTo.getLocalQueue().PushBack(v) - return nil -} + sss.nodesChecked++ -func (ssn *ShuffleShardNode) IsEmpty() bool { - return ssn.localQueue.Len() == 0 && len(ssn.queueOrder) == 0 -} + // if dequeueNode was self, advance queue position no matter what, and return + if n.queuePosition == localQueueIndex { + n.queuePosition++ + return + } -func (ssn *ShuffleShardNode) getOrAddNode(t *Tree, path QueuePath) (TreeNodeIFace, error) { - if len(path) == 0 { - return ssn, nil + // otherwise, don't change queuePosition, only move element to back + for i, child := range n.queueOrder { + if child == nodeName { + n.queueOrder = append(n.queueOrder[:i], append(n.queueOrder[i+1:], child)...) // ugly + } + } } - var childNode TreeNodeIFace - var ok bool - if childNode, ok = ssn.queueMap[path[0]]; !ok { - if ssn.depth+1 > t.maxDepth { - return nil, fmt.Errorf("cannot create a node at depth %v; greater than max depth %v", ssn.depth+1, len(t.nodeTypesByDepth)-1) - } - childNode = t.newNode(path[0], ssn.depth+1) - // add childNode to ssn.queueMap - ssn.queueMap[childNode.Name()] = childNode - // update ssn.queueOrder to place childNode behind rrn.queuePosition - if ssn.queuePosition <= 0 { - ssn.queueOrder = append(ssn.queueOrder, childNode.Name()) - } else { - ssn.queueOrder = append(ssn.queueOrder[:ssn.queuePosition], append([]string{childNode.Name()}, ssn.queueOrder[ssn.queuePosition:]...)...) - ssn.queuePosition++ - } + return dequeueOps{ + getNode: getNode, + updateState: updateState, } - return childNode.getOrAddNode(t, path[1:]) -} -func (ssn *ShuffleShardNode) getLocalQueue() *list.List { - return ssn.localQueue } diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go index d88da9baf1a..9c4cf91e6fc 100644 --- a/pkg/scheduler/queue/tree_queue_sketch_test.go +++ b/pkg/scheduler/queue/tree_queue_sketch_test.go @@ -6,42 +6,33 @@ import ( "testing" ) -func Test_NewTree(t *testing.T) { +// TODO (casie): Add tests for dequeuing from multiple different node children states + +func Test_NewNode(t *testing.T) { tests := []struct { - name string - nodesByDepth []NodeType - state *ShuffleShardState - expectErr bool + name string + rootAlgo DequeueAlgorithm + childPath OpsPath + state *shuffleShardState + expectErr bool }{ { - name: "create round-robin-only tree", - nodesByDepth: []NodeType{roundRobin, roundRobin}, - }, - { - name: "create shuffle-shard-only tree", - nodesByDepth: []NodeType{shuffleShard}, - state: &ShuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - }, - { - name: "create empty tree", - nodesByDepth: []NodeType{}, - expectErr: true, + name: "create round-robin node", + rootAlgo: &roundRobinState{}, }, { - name: "create shuffle-shard tree with no tenant-querier map", - nodesByDepth: []NodeType{shuffleShard}, - state: &ShuffleShardState{}, - expectErr: true, + name: "create shuffle-shard node", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, }, { - name: "create tree with multiple shuffle-shard depths", - nodesByDepth: []NodeType{shuffleShard, roundRobin, shuffleShard}, - expectErr: true, + name: "create shuffle-shard tree with no tenant-querier map", + rootAlgo: &shuffleShardState{}, + expectErr: true, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - _, err := NewTree(tt.nodesByDepth, tt.state) + _, err := NewNode("root", tt.rootAlgo) if tt.expectErr { require.Error(t, err) } else { @@ -52,46 +43,63 @@ func Test_NewTree(t *testing.T) { } } -func Test_EnqueueCreatingNode(t *testing.T) { +func Test_EnqueueBackByPath(t *testing.T) { tests := []struct { - name string - nodeTypeByDepth []NodeType - enqueueNodeName string - expectErr bool + name string + rootAlgo DequeueAlgorithm + children []OpsPath + expectErr bool }{ { - name: "enqueue over max-depth to round-robin node", - nodeTypeByDepth: []NodeType{roundRobin}, - enqueueNodeName: "bad-child-1", - expectErr: true, + name: "enqueue round-robin node to round-robin node", + rootAlgo: &roundRobinState{}, + children: []OpsPath{{{"round-robin-child-1", &roundRobinState{}}}}, + }, + { + name: "enqueue shuffle-shard node to round-robin node", + rootAlgo: &roundRobinState{}, + children: []OpsPath{{{ + "shuffle-shard-child-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + }}}, }, { - name: "enqueue over max-depth to shuffle-shard node", - nodeTypeByDepth: []NodeType{shuffleShard}, - enqueueNodeName: "bad-child-1", - expectErr: true, + name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node", + rootAlgo: &roundRobinState{}, + children: []OpsPath{{{"shuffle-shard-child-1", &shuffleShardState{}}}}, + expectErr: true, }, { - name: "enqueue round-robin node to round-robin node", - nodeTypeByDepth: []NodeType{roundRobin, roundRobin}, - enqueueNodeName: "round-robin-child-1", + name: "enqueue round-robin node to shuffle-shard node", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + children: []OpsPath{{{"round-robin-child-1", &roundRobinState{}}}}, }, { - name: "enqueue shuffle-shard node to round-robin node", - nodeTypeByDepth: []NodeType{roundRobin, shuffleShard}, - enqueueNodeName: "shuffle-shard-child-1", + name: "create tree with multiple shuffle-shard depths", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + children: []OpsPath{ + { + {"child", &roundRobinState{}}, + {"grandchild", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + }, + }, }, { - name: "enqueue round-robin node to shuffle-shard node", - nodeTypeByDepth: []NodeType{shuffleShard, roundRobin}, - enqueueNodeName: "round-robin-child-1", + name: "enqueue different types of nodes at the same depth", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + children: []OpsPath{ + {{"child-1", &roundRobinState{}}}, + {{"child-2", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}}, + }, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.nodeTypeByDepth, &ShuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}) + root, err := NewNode("root", tt.rootAlgo) require.NoError(t, err) - err = tree.EnqueueBackByPath(QueuePath{tt.enqueueNodeName}, "some-object") + + for _, childPath := range tt.children { + err = root.enqueueBackByPath(childPath, "some-object") + } if tt.expectErr { require.Error(t, err) } else { @@ -104,40 +112,56 @@ func Test_EnqueueCreatingNode(t *testing.T) { func Test_Dequeue_RootNode(t *testing.T) { tests := []struct { name string - rootNodeType NodeType - enqueueToRoot any + rootAlgo DequeueAlgorithm + enqueueToRoot []any }{ { - name: "dequeue from empty round-robin root node", - rootNodeType: roundRobin, + name: "dequeue from empty round-robin root node", + rootAlgo: &roundRobinState{}, }, { - name: "dequeue from empty shuffle-shard root node", - rootNodeType: shuffleShard, + name: "dequeue from empty shuffle-shard root node", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, }, { name: "dequeue from non-empty round-robin root node", - rootNodeType: roundRobin, - enqueueToRoot: "something-in-root", + rootAlgo: &roundRobinState{}, + enqueueToRoot: []any{"something-in-root"}, }, { name: "dequeue from non-empty shuffle-shard root node", - rootNodeType: shuffleShard, - enqueueToRoot: "something-else-in-root", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + enqueueToRoot: []any{"something-else-in-root"}, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { + // TODO (casie): ugly querierID := QuerierID("placeholder") - tree, err := NewTree([]NodeType{tt.rootNodeType}, &ShuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, currentQuerier: &querierID}) + switch tt.rootAlgo.(type) { + case *shuffleShardState: + tt.rootAlgo.(*shuffleShardState).currentQuerier = &querierID + } + + root, err := NewNode("root", tt.rootAlgo) require.NoError(t, err) - if tt.enqueueToRoot != nil { - err = tree.EnqueueBackByPath(QueuePath{}, tt.enqueueToRoot) + + for _, elt := range tt.enqueueToRoot { + err := root.enqueueBackByPath(OpsPath{}, elt) require.NoError(t, err) } - path, v := tree.Dequeue() - require.Equal(t, QueuePath{"root"}, path) - require.Equal(t, tt.enqueueToRoot, v) + for _, elt := range tt.enqueueToRoot { + path, v := root.dequeue() + require.Equal(t, QueuePath{"root"}, path) + require.Equal(t, elt, v) + + } + + if tt.enqueueToRoot == nil { + path, v := root.dequeue() + require.Equal(t, QueuePath{"root"}, path) + require.Nil(t, v) + } }) } @@ -145,81 +169,82 @@ func Test_Dequeue_RootNode(t *testing.T) { func Test_RoundRobinDequeue(t *testing.T) { tests := []struct { - name string - nodesByDepth []NodeType - selfQueueObjects []string - childQueueObjects map[string][]any - grandchildren map[string]struct { - path QueuePath + name string + selfQueueObjects []string + children []string + childQueueObjects map[string][]any + grandchildren []string + grandchildrenQueueObjects map[string]struct { + path OpsPath objs []any } - numDequeuesToExpected int - expected string + expected []string }{ { - name: "dequeue from round-robin child when local queue empty", - nodesByDepth: []NodeType{roundRobin, roundRobin}, - childQueueObjects: map[string][]any{"child-1": {"child-1:some-object"}}, - numDequeuesToExpected: 1, - expected: "child-1:some-object", + name: "dequeue from round-robin child when local queue empty", + children: []string{"child-1"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:some-object"}}, + expected: []string{"child-1:some-object"}, }, { - name: "dequeue from round-robin root when on node's turn", - nodesByDepth: []NodeType{roundRobin, roundRobin}, - selfQueueObjects: []string{"root:object-1", "root:object-2"}, - childQueueObjects: map[string][]any{"child-1": {"child-1:object-1"}}, - numDequeuesToExpected: 2, - expected: "child-1:object-1", + name: "dequeue from round-robin root when on node's turn", + selfQueueObjects: []string{"root:object-1", "root:object-2"}, + children: []string{"child-1"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:object-1"}}, + expected: []string{"root:object-1", "child-1:object-1"}, }, { - name: "dequeue from second round-robin child when first child is empty", - nodesByDepth: []NodeType{roundRobin, roundRobin}, - childQueueObjects: map[string][]any{"child-1": {nil}, "child-2": {"child-2:some-object"}}, - numDequeuesToExpected: 1, - expected: "child-2:some-object", + name: "dequeue from second round-robin child when first child is empty", + children: []string{"child-1", "child-2"}, + childQueueObjects: map[string][]any{"child-1": {nil}, "child-2": {"child-2:some-object"}}, + expected: []string{"child-2:some-object"}, }, { - name: "dequeue from round-robin grandchild when non-empty", - nodesByDepth: []NodeType{roundRobin, roundRobin, roundRobin}, - childQueueObjects: map[string][]any{"child-1": {"child-1:object-1", "child-1:object-2"}, "child-2": {"child-2:object-1"}}, - numDequeuesToExpected: 3, - expected: "grandchild-1:object-1", - grandchildren: map[string]struct { - path QueuePath + name: "dequeue from round-robin grandchild when non-empty", + children: []string{"child-1", "child-2"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:object-1", "child-1:object-2"}, "child-2": {"child-2:object-1"}}, + expected: []string{"child-1:object-1", "child-2:object-1", "grandchild-1:object-1"}, + grandchildren: []string{"grandchild-1"}, + grandchildrenQueueObjects: map[string]struct { + path OpsPath objs []any - }{"grandchild-1": {path: QueuePath{"child-1"}, objs: []any{"grandchild-1:object-1"}}}, + }{"grandchild-1": { + path: OpsPath{{"child-1", &roundRobinState{}}, {"grandchild-1", &roundRobinState{}}}, + objs: []any{"grandchild-1:object-1"}, + }}, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.nodesByDepth, &ShuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}) + root, err := NewNode("root", &roundRobinState{}) require.NoError(t, err) for _, sqo := range tt.selfQueueObjects { - err = tree.EnqueueBackByPath(QueuePath{}, sqo) + err = root.enqueueBackByPath(OpsPath{}, sqo) require.NoError(t, err) } - for childName, objs := range tt.childQueueObjects { - for _, obj := range objs { - err = tree.EnqueueBackByPath(QueuePath{childName}, obj) + + for _, child := range tt.children { + for _, obj := range tt.childQueueObjects[child] { + err = root.enqueueBackByPath(OpsPath{{child, &roundRobinState{}}}, obj) } } - for grandchildName, grandchildData := range tt.grandchildren { - for _, obj := range grandchildData.objs { - err = tree.EnqueueBackByPath(append(grandchildData.path, grandchildName), obj) + for _, grandchild := range tt.grandchildren { + gqo := tt.grandchildrenQueueObjects[grandchild] + for _, obj := range gqo.objs { + err := root.enqueueBackByPath(gqo.path, obj) require.NoError(t, err) } } - var finalValue any - for i := 0; i < tt.numDequeuesToExpected; i++ { - _, finalValue = tree.Dequeue() + for _, expected := range tt.expected { + _, val := root.dequeue() + v, ok := val.(string) + require.True(t, ok) + require.Equal(t, expected, v) } - v, ok := finalValue.(string) - require.True(t, ok) - require.Equal(t, tt.expected, v) }) } } @@ -227,95 +252,202 @@ func Test_RoundRobinDequeue(t *testing.T) { func Test_ShuffleShardDequeue(t *testing.T) { type enqueueObj struct { obj any - path QueuePath + path OpsPath } tests := []struct { - name string - nodesByDepth []NodeType - tqMap map[TenantID]map[QuerierID]struct{} - currQuerier []QuerierID - enqueueObjs []enqueueObj - expected []any - expectErr bool + name string + rootAlgo DequeueAlgorithm + tqMap map[TenantID]map[QuerierID]struct{} + currQuerier []QuerierID + enqueueObjs []enqueueObj + expected []any + expectErr bool }{ { - name: "happy path - tenant found in tenant-querier map under first child", - nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - currQuerier: []QuerierID{"querier-1"}, - enqueueObjs: []enqueueObj{{obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}}, - expected: []any{"query-1"}, + name: "happy path - tenant found in tenant-querier map under first child", + rootAlgo: &roundRobinState{}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-1", &roundRobinState{}}, + }}, + }, + expected: []any{"query-1"}, }, { - name: "tenant exists, but not for querier", - nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, - currQuerier: []QuerierID{"querier-1"}, - enqueueObjs: []enqueueObj{{obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}}, - expected: []any{nil}, + name: "tenant exists, but not for querier", + rootAlgo: &roundRobinState{}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{ + { + obj: "query-1", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-1", &roundRobinState{}}, + }, + }, + }, + expected: []any{nil}, }, { - name: "1 of 3 tenants exist for querier", - nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - currQuerier: []QuerierID{"querier-1"}, - enqueueObjs: []enqueueObj{{obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, {obj: "query-3", path: QueuePath{"query-component-1", "tenant-3"}}}, - expected: []any{"query-3"}, + name: "1 of 3 tenants exist for querier", + rootAlgo: &roundRobinState{}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{ + { + obj: "query-1", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-1", &roundRobinState{}}, + }, + }, + { + obj: "query-2", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-2", &roundRobinState{}}, + }, + }, + { + obj: "query-3", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-3", &roundRobinState{}}, + }, + }, + }, + expected: []any{"query-3"}, }, { - name: "tenant exists for querier on next parent node", - nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + name: "tenant exists for querier on next parent node", + rootAlgo: &roundRobinState{}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, - {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, - {obj: "query-4", path: QueuePath{"query-component-1", "tenant-3"}}, + { + obj: "query-1", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-1", &roundRobinState{}}, + }, + }, + { + obj: "query-2", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-2", &roundRobinState{}}, + }, + }, + { + obj: "query-3", + path: OpsPath{ + {"query-component-2", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-3", &roundRobinState{}}, + }, + }, + { + obj: "query-4", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-3", &roundRobinState{}}, + }, + }, }, expected: []any{"query-4", "query-3", nil}, }, { - name: "2 of 3 tenants exist for querier", - nodesByDepth: []NodeType{roundRobin, shuffleShard, roundRobin}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + name: "2 of 3 tenants exist for querier", + rootAlgo: &roundRobinState{}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, - {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, + { + obj: "query-1", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-1", &roundRobinState{}}, + }, + }, + { + obj: "query-2", + path: OpsPath{ + {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-2", &roundRobinState{}}, + }, + }, + { + obj: "query-3", + path: OpsPath{ + {"query-component-2", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + {"tenant-3", &roundRobinState{}}, + }, + }, }, expected: []any{"query-1", "query-3", nil}, }, { - name: "root node is shuffle-shard node", - nodesByDepth: []NodeType{shuffleShard, roundRobin, roundRobin}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + name: "root node is shuffle-shard node", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-2", "query-component-1"}}, - {obj: "query-2", path: QueuePath{"tenant-1", "query-component-2"}}, - {obj: "query-3", path: QueuePath{"tenant-1", "query-component-1"}}, + { + obj: "query-1", + path: OpsPath{ + {"tenant-2", &roundRobinState{}}, + {"query-component-1", &roundRobinState{}}, + }, + }, + { + obj: "query-2", + path: OpsPath{ + {"tenant-1", &roundRobinState{}}, + {"query-component-2", &roundRobinState{}}, + }, + }, + { + obj: "query-3", + path: OpsPath{ + {"tenant-1", &roundRobinState{}}, + {"query-component-1", &roundRobinState{}}, + }, + }, }, expected: []any{"query-2", "query-3", nil}, }, { - name: "dequeueing for one querier returns nil, but does return for a different querier", - nodesByDepth: []NodeType{shuffleShard, roundRobin, roundRobin}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - currQuerier: []QuerierID{"querier-2", "querier-1"}, + name: "dequeueing for one querier returns nil, but does return for a different querier", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + currQuerier: []QuerierID{"querier-2", "querier-1"}, enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + { + obj: "query-1", + path: OpsPath{ + {"tenant-1", &roundRobinState{}}, + {"query-component-1", &roundRobinState{}}, + }, + }, }, expected: []any{nil, "query-1"}, }, { - name: "no querier set in state", - nodesByDepth: []NodeType{shuffleShard, roundRobin, roundRobin}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - currQuerier: []QuerierID{""}, + name: "no querier set in state", + rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + currQuerier: []QuerierID{""}, enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + { + obj: "query-1", + path: OpsPath{ + {"tenant-1", &roundRobinState{}}, + {"query-component-1", &roundRobinState{}}, + }, + }, }, expected: []any{nil}, }, @@ -323,19 +455,36 @@ func Test_ShuffleShardDequeue(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - currentQuerier := QuerierID("") - tree, err := NewTree(tt.nodesByDepth, &ShuffleShardState{tenantQuerierMap: tt.tqMap, currentQuerier: ¤tQuerier}) + currentQuerier := QuerierID("placeholder") + sss := &shuffleShardState{ + tenantQuerierMap: tt.tqMap, + currentQuerier: ¤tQuerier, + nodesChecked: 0, + } + switch tt.rootAlgo.(type) { + case *shuffleShardState: + tt.rootAlgo = sss + } + root, err := NewNode("root", tt.rootAlgo) + // TODO (casie): ugly + require.NoError(t, err) for _, o := range tt.enqueueObjs { - err = tree.EnqueueBackByPath(o.path, o.obj) + for i, elt := range o.path { + switch elt.dequeueAlgorithm.(type) { + case *shuffleShardState: + o.path[i].dequeueAlgorithm = sss + } + } + err = root.enqueueBackByPath(o.path, o.obj) require.NoError(t, err) } // currQuerier at position i is used to dequeue the expected result at position i require.Equal(t, len(tt.currQuerier), len(tt.expected)) for i := 0; i < len(tt.expected); i++ { - tree.currentQuerier = &tt.currQuerier[i] - path, v := tree.Dequeue() + sss.currentQuerier = &tt.currQuerier[i] + path, v := root.dequeue() fmt.Println(path, v) require.Equal(t, tt.expected[i], v) } @@ -351,16 +500,27 @@ func Test_ChangeShuffleShardState(t *testing.T) { tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}}, } - state := &ShuffleShardState{ + state := &shuffleShardState{ tenantQuerierMap: tqa.tenantQuerierIDs, currentQuerier: nil, } - tree, err := NewTree([]NodeType{shuffleShard, roundRobin, roundRobin}, state) + root, err := NewNode("root", state) + //tree, err := NewTree([]NodeType{shuffleShard, roundRobin, roundRobin}, state) require.NoError(t, err) - err = tree.EnqueueBackByPath(QueuePath{"tenant-1", "query-component-1"}, "query-1") - err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-2") - err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-3") + + err = root.enqueueBackByPath(OpsPath{ + {"tenant-1", &roundRobinState{}}, + {"query-component-1", &roundRobinState{}}, + }, "query-1") + err = root.enqueueBackByPath(OpsPath{ + {"tenant-2", &roundRobinState{}}, + {"query-component-1", &roundRobinState{}}, + }, "query-2") + err = root.enqueueBackByPath(OpsPath{ + {"tenant-2", &roundRobinState{}}, + {"query-component-1", &roundRobinState{}}, + }, "query-3") require.NoError(t, err) querier1 := QuerierID("querier-1") @@ -369,18 +529,18 @@ func Test_ChangeShuffleShardState(t *testing.T) { // set state to querier-2 should dequeue query-2 state.currentQuerier = &querier2 - _, v := tree.Dequeue() + _, v := root.dequeue() require.Equal(t, "query-2", v) // set state to querier-1 should dequeue query-1 state.currentQuerier = &querier1 - _, v = tree.Dequeue() + _, v = root.dequeue() require.Equal(t, "query-1", v) // update tqa map to assign querier-3 to tenant-2, then set state to querier-3 should dequeue query-3 tqa.tenantQuerierIDs["tenant-2"]["querier-3"] = struct{}{} state.currentQuerier = &querier3 - _, v = tree.Dequeue() + _, v = root.dequeue() require.Equal(t, "query-3", v) } From d2f00a04109532f060d38845aaf20f2b8a6d6c9a Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Wed, 1 May 2024 14:10:56 -0700 Subject: [PATCH 05/26] Add a test for reassigning tenantQuerierID map value --- pkg/scheduler/queue/tree_queue_sketch_test.go | 71 +++++++++++-------- 1 file changed, 42 insertions(+), 29 deletions(-) diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go index 9c4cf91e6fc..13f7ee37e36 100644 --- a/pkg/scheduler/queue/tree_queue_sketch_test.go +++ b/pkg/scheduler/queue/tree_queue_sketch_test.go @@ -254,19 +254,22 @@ func Test_ShuffleShardDequeue(t *testing.T) { obj any path OpsPath } + tests := []struct { name string rootAlgo DequeueAlgorithm - tqMap map[TenantID]map[QuerierID]struct{} + state *shuffleShardState currQuerier []QuerierID enqueueObjs []enqueueObj expected []any expectErr bool }{ { - name: "happy path - tenant found in tenant-querier map under first child", - rootAlgo: &roundRobinState{}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + name: "happy path - tenant found in tenant-querier map under first child", + rootAlgo: &roundRobinState{}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + }, currQuerier: []QuerierID{"querier-1"}, enqueueObjs: []enqueueObj{ {obj: "query-1", path: OpsPath{ @@ -277,9 +280,11 @@ func Test_ShuffleShardDequeue(t *testing.T) { expected: []any{"query-1"}, }, { - name: "tenant exists, but not for querier", - rootAlgo: &roundRobinState{}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, + name: "tenant exists, but not for querier", + rootAlgo: &roundRobinState{}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, + }, currQuerier: []QuerierID{"querier-1"}, enqueueObjs: []enqueueObj{ { @@ -293,9 +298,10 @@ func Test_ShuffleShardDequeue(t *testing.T) { expected: []any{nil}, }, { - name: "1 of 3 tenants exist for querier", - rootAlgo: &roundRobinState{}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + name: "1 of 3 tenants exist for querier", + rootAlgo: &roundRobinState{}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, currQuerier: []QuerierID{"querier-1"}, enqueueObjs: []enqueueObj{ { @@ -323,9 +329,10 @@ func Test_ShuffleShardDequeue(t *testing.T) { expected: []any{"query-3"}, }, { - name: "tenant exists for querier on next parent node", - rootAlgo: &roundRobinState{}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + name: "tenant exists for querier on next parent node", + rootAlgo: &roundRobinState{}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ { @@ -362,7 +369,7 @@ func Test_ShuffleShardDequeue(t *testing.T) { { name: "2 of 3 tenants exist for querier", rootAlgo: &roundRobinState{}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ { @@ -392,7 +399,7 @@ func Test_ShuffleShardDequeue(t *testing.T) { { name: "root node is shuffle-shard node", rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}}, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ { @@ -422,7 +429,7 @@ func Test_ShuffleShardDequeue(t *testing.T) { { name: "dequeueing for one querier returns nil, but does return for a different querier", rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}}, currQuerier: []QuerierID{"querier-2", "querier-1"}, enqueueObjs: []enqueueObj{ { @@ -438,7 +445,7 @@ func Test_ShuffleShardDequeue(t *testing.T) { { name: "no querier set in state", rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - tqMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}}, currQuerier: []QuerierID{""}, enqueueObjs: []enqueueObj{ { @@ -456,25 +463,21 @@ func Test_ShuffleShardDequeue(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { currentQuerier := QuerierID("placeholder") - sss := &shuffleShardState{ - tenantQuerierMap: tt.tqMap, - currentQuerier: ¤tQuerier, - nodesChecked: 0, - } + tt.state.currentQuerier = ¤tQuerier + + // TODO (casie): ugly switch tt.rootAlgo.(type) { case *shuffleShardState: - tt.rootAlgo = sss + tt.rootAlgo = tt.state } root, err := NewNode("root", tt.rootAlgo) - // TODO (casie): ugly - require.NoError(t, err) for _, o := range tt.enqueueObjs { for i, elt := range o.path { switch elt.dequeueAlgorithm.(type) { case *shuffleShardState: - o.path[i].dequeueAlgorithm = sss + o.path[i].dequeueAlgorithm = tt.state } } err = root.enqueueBackByPath(o.path, o.obj) @@ -483,7 +486,7 @@ func Test_ShuffleShardDequeue(t *testing.T) { // currQuerier at position i is used to dequeue the expected result at position i require.Equal(t, len(tt.currQuerier), len(tt.expected)) for i := 0; i < len(tt.expected); i++ { - sss.currentQuerier = &tt.currQuerier[i] + currentQuerier = tt.currQuerier[i] path, v := root.dequeue() fmt.Println(path, v) require.Equal(t, tt.expected[i], v) @@ -521,6 +524,10 @@ func Test_ChangeShuffleShardState(t *testing.T) { {"tenant-2", &roundRobinState{}}, {"query-component-1", &roundRobinState{}}, }, "query-3") + err = root.enqueueBackByPath(OpsPath{ + {"tenant-2", &roundRobinState{}}, + {"query-component-1", &roundRobinState{}}, + }, "query-4") require.NoError(t, err) querier1 := QuerierID("querier-1") @@ -532,15 +539,21 @@ func Test_ChangeShuffleShardState(t *testing.T) { _, v := root.dequeue() require.Equal(t, "query-2", v) - // set state to querier-1 should dequeue query-1 + // update state to querier-1 should dequeue query-1 state.currentQuerier = &querier1 _, v = root.dequeue() require.Equal(t, "query-1", v) - // update tqa map to assign querier-3 to tenant-2, then set state to querier-3 should dequeue query-3 + // update tqa map to add querier-3 as assigned to tenant-2, then set state to querier-3 should dequeue query-3 tqa.tenantQuerierIDs["tenant-2"]["querier-3"] = struct{}{} state.currentQuerier = &querier3 _, v = root.dequeue() require.Equal(t, "query-3", v) + // during reshuffle, we only ever reassign tenant values, we don't assign an entirely new map value + // to tenantQuerierIDs. Reassign tenant-2 to an empty map value , and query-4 should _not_ be dequeued + tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} + _, v = root.dequeue() + require.Nil(t, v) + } From 4b155bf662fe0c87c3414091807c8670928656dd Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Tue, 28 May 2024 16:24:50 -0700 Subject: [PATCH 06/26] wip --- pkg/scheduler/queue/queue.go | 24 +- pkg/scheduler/queue/tenant_queues.go | 94 +++- pkg/scheduler/queue/tenant_queues_test.go | 416 +++++++++++++--- pkg/scheduler/queue/tree_queue_sketch.go | 190 +++++-- pkg/scheduler/queue/tree_queue_sketch_test.go | 463 +++++++++--------- 5 files changed, 807 insertions(+), 380 deletions(-) diff --git a/pkg/scheduler/queue/queue.go b/pkg/scheduler/queue/queue.go index 4130ca7c7ac..3003009d1b1 100644 --- a/pkg/scheduler/queue/queue.go +++ b/pkg/scheduler/queue/queue.go @@ -349,7 +349,7 @@ func (q *RequestQueue) enqueueRequestInternal(r requestToEnqueue) error { // a) a query request which was successfully dequeued for the querier, or // b) an ErrShuttingDown indicating the querier has been placed in a graceful shutdown state. func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierConn) (done bool) { - req, tenant, idx, err := q.queueBroker.dequeueRequestForQuerier(waitingConn.lastTenantIndex.last, waitingConn.querierID) + req, tenant, err := q.queueBroker.dequeueRequestForQuerier(waitingConn.querierID) if err != nil { // If this querier has told us it's shutting down, terminate WaitForRequestForQuerier with an error now... waitingConn.sendError(err) @@ -357,7 +357,7 @@ func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierC return true } - waitingConn.lastTenantIndex.last = idx + //waitingConn.lastTenantIndex.last = idx if req == nil { // Nothing available for this querier, try again next time. return false @@ -394,15 +394,17 @@ func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierC requestSent := waitingConn.send(reqForQuerier) if requestSent { q.queueLength.WithLabelValues(string(tenant.tenantID)).Dec() - } else { - // should never error; any item previously in the queue already passed validation - err := q.queueBroker.enqueueRequestFront(req, tenant.maxQueriers) - if err != nil { - level.Error(q.log).Log( - "msg", "failed to re-enqueue query request after dequeue", - "err", err, "tenant", tenant.tenantID, "querier", waitingConn.querierID, - ) - } + // TODO (casie): Handle this, though I don't think we should ever have to re-enqueue to front anymore? + // Nvm we do; when queriers fail while dequeuing + //} else { + // // should never error; any item previously in the queue already passed validation + // err := q.queueBroker.enqueueRequestFront(req, tenant.maxQueriers) + // if err != nil { + // level.Error(q.log).Log( + // "msg", "failed to re-enqueue query request after dequeue", + // "err", err, "tenant", tenant.tenantID, "querier", waitingConn.querierID, + // ) + // } } return true } diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index a7439f6a5b6..f672f87e9ad 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -24,31 +24,46 @@ type tenantRequest struct { // and maintains consistency with the tenant-querier assignments type queueBroker struct { tenantQueuesTree *TreeQueue + queueTree *Tree tenantQuerierAssignments tenantQuerierAssignments maxTenantQueueSize int additionalQueueDimensionsEnabled bool + currentQuerier QuerierID } func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled bool, forgetDelay time.Duration) *queueBroker { - return &queueBroker{ - tenantQueuesTree: NewTreeQueue("root"), - tenantQuerierAssignments: tenantQuerierAssignments{ - queriersByID: map[QuerierID]*querierConn{}, - querierIDsSorted: nil, - querierForgetDelay: forgetDelay, - tenantIDOrder: nil, - tenantsByID: map[TenantID]*queueTenant{}, - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, - }, + currentQuerier := QuerierID("") + tqas := tenantQuerierAssignments{ + queriersByID: map[QuerierID]*querierConn{}, + querierIDsSorted: nil, + querierForgetDelay: forgetDelay, + tenantIDOrder: nil, + tenantsByID: map[TenantID]*queueTenant{}, + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, + } + // TODO (casie): Maybe set this using a flag, so we can also change how we build queue path accordingly + tree := NewTree( + &shuffleShardState{tenantQuerierMap: tqas.tenantQuerierIDs, currentQuerier: ¤tQuerier}, // root + &roundRobinState{}, // tenants + &roundRobinState{}, // query components + ) + qb := &queueBroker{ + tenantQueuesTree: NewTreeQueue("root"), + queueTree: tree, + tenantQuerierAssignments: tqas, maxTenantQueueSize: maxTenantQueueSize, additionalQueueDimensionsEnabled: additionalQueueDimensionsEnabled, + currentQuerier: currentQuerier, } + + return qb } func (qb *queueBroker) isEmpty() bool { - return qb.tenantQueuesTree.IsEmpty() + //return qb.tenantQueuesTree.IsEmpty() + return qb.queueTree.rootNode.IsEmpty() } // enqueueRequestBack is the standard interface to enqueue requests for dispatch to queriers. @@ -64,13 +79,21 @@ func (qb *queueBroker) enqueueRequestBack(request *tenantRequest, tenantMaxQueri if err != nil { return err } - if tenantQueueNode := qb.tenantQueuesTree.getNode(queuePath[:1]); tenantQueueNode != nil { + + if tenantQueueNode := qb.queueTree.rootNode.getNode(queuePath[:1]); tenantQueueNode != nil { if tenantQueueNode.ItemCount()+1 > qb.maxTenantQueueSize { return ErrTooManyRequests } } - err = qb.tenantQueuesTree.EnqueueBackByPath(queuePath, request) + //if tenantQueueNode := qb.tenantQueuesTree.getNode(queuePath[:1]); tenantQueueNode != nil { + // if tenantQueueNode.ItemCount()+1 > qb.maxTenantQueueSize { + // return ErrTooManyRequests + // } + //} + + //err = qb.tenantQueuesTree.EnqueueBackByPath(queuePath, request) + err = qb.queueTree.rootNode.enqueueBackByPath(qb.queueTree, queuePath, request) return err } @@ -103,19 +126,43 @@ func (qb *queueBroker) makeQueuePath(request *tenantRequest) (QueuePath, error) return QueuePath{string(request.tenantID)}, nil } -func (qb *queueBroker) dequeueRequestForQuerier(lastTenantIndex int, querierID QuerierID) (*tenantRequest, *queueTenant, int, error) { - tenant, tenantIndex, err := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndex, querierID) - if tenant == nil || err != nil { - return nil, tenant, tenantIndex, err +func (qb *queueBroker) dequeueRequestForQuerier( + //lastTenantIndex int, + querierID QuerierID, +) ( + *tenantRequest, + *queueTenant, + //int, + error, +) { + //tenant, tenantIndex, err := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndex, querierID) + // check if querier is registered and is not shutting down + if q := qb.tenantQuerierAssignments.queriersByID[querierID]; q == nil || q.shuttingDown { + return nil, nil, ErrQuerierShuttingDown } + //if tenant == nil || err != nil { + // return nil, tenant, tenantIndex, err + //} + // + //queuePath := QueuePath{string(tenant.tenantID)} + // + //queueElement := qb.tenantQueuesTree.DequeueByPath(queuePath) + + qb.currentQuerier = querierID + queuePath, queueElement := qb.queueTree.rootNode.dequeue() + + // TODO (casie): hacky - hardcoding second elt in queuePath to tenant ID + var tenantID TenantID + if len(queuePath) > 1 { + tenantID = TenantID(queuePath[1]) + } + tenant := qb.tenantQuerierAssignments.tenantsByID[tenantID] - queuePath := QueuePath{string(tenant.tenantID)} - queueElement := qb.tenantQueuesTree.DequeueByPath(queuePath) - - queueNodeAfterDequeue := qb.tenantQueuesTree.getNode(queuePath) + // dequeue returns the full path including root, but getNode expects the path _from_ root + queueNodeAfterDequeue := qb.queueTree.rootNode.getNode(queuePath[1:]) if queueNodeAfterDequeue == nil { // queue node was deleted due to being empty after dequeue - qb.tenantQuerierAssignments.removeTenant(tenant.tenantID) + qb.tenantQuerierAssignments.removeTenant(tenantID) } var request *tenantRequest @@ -124,7 +171,8 @@ func (qb *queueBroker) dequeueRequestForQuerier(lastTenantIndex int, querierID Q request = queueElement.(*tenantRequest) } - return request, tenant, tenantIndex, nil + //return request, tenant, tenantIndex, nil + return request, tenant, nil } func (qb *queueBroker) addQuerierConnection(querierID QuerierID) (resharded bool) { diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index f6b09fedd37..14389454499 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -9,16 +9,39 @@ import ( "container/list" "context" "fmt" + "github.com/grafana/dskit/httpgrpc" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "math" "math/rand" "testing" "time" - - "github.com/grafana/dskit/httpgrpc" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) +// TODO (casie): Add a test case where, if a tenant is queued in the queueTree, but doesn't exist from the queueBroker perspective, +// it should never be dequeued. + +// todo (casie): maybe implement a way to fetch an entire node that _would_ be dequeued from, rather than just one elt +func (qb *queueBroker) enqueueObjectsForTests(tenantID TenantID, numObjects int) error { + for i := 0; i < numObjects; i++ { + err := qb.queueTree.rootNode.enqueueBackByPath(qb.queueTree, QueuePath{string(tenantID)}, &tenantRequest{ + tenantID: tenantID, + req: fmt.Sprintf("%v: object-%v", tenantID, i), + }) + if err != nil { + return err + } + } + return nil +} + +func buildExpectedObject(tenantID TenantID, num int) *tenantRequest { + return &tenantRequest{ + tenantID: tenantID, + req: fmt.Sprintf("%v: object-%v", tenantID, num), + } +} + func TestQueues(t *testing.T) { qb := newQueueBroker(0, true, 0) assert.NotNil(t, qb) @@ -27,43 +50,165 @@ func TestQueues(t *testing.T) { qb.addQuerierConnection("querier-1") qb.addQuerierConnection("querier-2") - req, tenant, lastTenantIndex, err := qb.dequeueRequestForQuerier(-1, "querier-1") + type dequeueVal struct { + req *tenantRequest + tenant *queueTenant + } + + req, tenant, err := qb.dequeueRequestForQuerier("querier-1") assert.Nil(t, req) assert.Nil(t, tenant) assert.NoError(t, err) - // Add queues: [one] - qOne := getOrAdd(t, qb, "one", 0) - lastTenantIndex = confirmOrderForQuerier(t, qb, "querier-1", lastTenantIndex, qOne, qOne) + // Add tenant queues: tenant "one" + err = qb.tenantQuerierAssignments.createOrUpdateTenant("one", 0) + assert.NoError(t, err) + + tenantOne := qb.tenantQuerierAssignments.tenantsByID["one"] + + // Queue enough objects for tenant "one" + err = qb.enqueueObjectsForTests(tenantOne.tenantID, 10) + assert.NoError(t, err) + assert.NoError(t, isConsistent(qb)) + + //queuePathOne := QueuePath{"root", "one"} + expectedDequeueVals := []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, + {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, + } + + for _, expected := range expectedDequeueVals { + req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + + } + + // Add tenant two + err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) + assert.NoError(t, err) + + tenantTwo := qb.tenantQuerierAssignments.tenantsByID["two"] + + err = qb.enqueueObjectsForTests(tenantTwo.tenantID, 10) + assert.NoError(t, err) + assert.NoError(t, isConsistent(qb)) - // [one two] - qTwo := getOrAdd(t, qb, "two", 0) + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantTwo.tenantID, 0), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 2), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 1), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, + } - lastTenantIndex = confirmOrderForQuerier(t, qb, "querier-1", lastTenantIndex, qTwo, qOne, qTwo, qOne) - confirmOrderForQuerier(t, qb, "querier-2", -1, qOne, qTwo, qOne) + for _, expected := range expectedDequeueVals { + req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } + // TODO (casie): This block relies on being able to pass a lastTenantIndex; I would like to get rid of this concept + // and allow the tree to manage tenant dequeuing entirely. Do we still want this behavior in that case? + // ====== + //confirmOrderForQuerier(t, qb, "querier-2", -1, qOne, qTwo, qOne) + + //expectedDequeueVals = []dequeueVal{ + // {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, + // {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, + // {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, + //} + //for _, expected := range expectedDequeueVals { + // req, tenant, err = qb.dequeueRequestForQuerier("querier-2") + // assert.Equal(t, expected.req, req) + // assert.Equal(t, expected.tenant, tenant) + // assert.NoError(t, err) + //} + // // [one two three] - // confirm fifo by adding a third queue and iterating to it - qThree := getOrAdd(t, qb, "three", 0) + // ======== + + // confirm fifo by adding a third tenant queue and iterating to it + err = qb.tenantQuerierAssignments.createOrUpdateTenant("three", 0) + assert.NoError(t, err) + + tenantThree := qb.tenantQuerierAssignments.tenantsByID["three"] + + err = qb.enqueueObjectsForTests(tenantThree.tenantID, 10) + assert.NoError(t, err) + assert.NoError(t, isConsistent(qb)) - lastTenantIndex = confirmOrderForQuerier(t, qb, "querier-1", lastTenantIndex, qTwo, qThree, qOne) + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, + {buildExpectedObject(tenantThree.tenantID, 0), tenantThree}, + {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, + } - // Remove one: ["" two three] + for _, expected := range expectedDequeueVals { + req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } + + // Remove one: [two three] qb.removeTenantQueue("one") assert.NoError(t, isConsistent(qb)) - lastTenantIndex = confirmOrderForQuerier(t, qb, "querier-1", lastTenantIndex, qTwo, qThree, qTwo) + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, + {buildExpectedObject(tenantThree.tenantID, 1), tenantThree}, + {buildExpectedObject(tenantTwo.tenantID, 4), tenantTwo}, + } + + for _, expected := range expectedDequeueVals { + req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } // "four" is added at the beginning of the list: [four two three] - qFour := getOrAdd(t, qb, "four", 0) + err = qb.tenantQuerierAssignments.createOrUpdateTenant("four", 0) + assert.NoError(t, err) + + tenantFour := qb.tenantQuerierAssignments.tenantsByID["four"] + + err = qb.enqueueObjectsForTests(tenantFour.tenantID, 10) + assert.NoError(t, err) + assert.NoError(t, isConsistent(qb)) + + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantThree.tenantID, 2), tenantThree}, + {buildExpectedObject(tenantFour.tenantID, 0), tenantFour}, + {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, + {buildExpectedObject(tenantThree.tenantID, 3), tenantThree}, + } - lastTenantIndex = confirmOrderForQuerier(t, qb, "querier-1", lastTenantIndex, qThree, qFour, qTwo, qThree) + for _, expected := range expectedDequeueVals { + req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } - // Remove two: [four "" three] + // Remove two: [four three] qb.removeTenantQueue("two") assert.NoError(t, isConsistent(qb)) + // + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantFour.tenantID, 1), tenantFour}, + {buildExpectedObject(tenantThree.tenantID, 4), tenantThree}, + {buildExpectedObject(tenantFour.tenantID, 2), tenantFour}, + } - lastTenantIndex = confirmOrderForQuerier(t, qb, "querier-1", lastTenantIndex, qFour, qThree, qFour) + for _, expected := range expectedDequeueVals { + req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } // Remove three: [four] qb.removeTenantQueue("three") @@ -73,8 +218,9 @@ func TestQueues(t *testing.T) { qb.removeTenantQueue("four") assert.NoError(t, isConsistent(qb)) - req, _, _, err = qb.dequeueRequestForQuerier(lastTenantIndex, "querier-1") + req, tenant, err = qb.dequeueRequestForQuerier("querier-1") assert.Nil(t, req) + assert.Nil(t, tenant) assert.NoError(t, err) } @@ -105,12 +251,12 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { } // assert item count of tenant node and its subnodes queuePath := QueuePath{"tenant-1"} - assert.Equal(t, maxTenantQueueSize, qb.tenantQueuesTree.getNode(queuePath).ItemCount()) + assert.Equal(t, maxTenantQueueSize, qb.queueTree.rootNode.getNode(queuePath).ItemCount()) // assert equal distribution of queue items between tenant node and 3 subnodes for _, v := range additionalQueueDimensions { queuePath := append(QueuePath{"tenant-1"}, v...) - assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), qb.tenantQueuesTree.getNode(queuePath).LocalQueueLen()) + assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), qb.queueTree.rootNode.getNode(queuePath).getLocalQueue().Len()) } // assert error received when hitting a tenant's enqueue limit, @@ -126,7 +272,7 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { // dequeue a request qb.addQuerierConnection("querier-1") - dequeuedTenantReq, _, _, err := qb.dequeueRequestForQuerier(-1, "querier-1") + dequeuedTenantReq, _, err := qb.dequeueRequestForQuerier("querier-1") assert.NoError(t, err) assert.NotNil(t, dequeuedTenantReq) @@ -149,25 +295,82 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { qb.addQuerierConnection("querier-2") // Add queues: [one, two] - qOne := getOrAdd(t, qb, "one", 0) - qTwo := getOrAdd(t, qb, "two", 0) - confirmOrderForQuerier(t, qb, "querier-1", -1, qOne, qTwo, qOne, qTwo) - confirmOrderForQuerier(t, qb, "querier-2", -1, qOne, qTwo, qOne, qTwo) + err := qb.tenantQuerierAssignments.createOrUpdateTenant("one", 0) + assert.NoError(t, err) + err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) + assert.NoError(t, err) + + err = qb.enqueueObjectsForTests("one", 10) + assert.NoError(t, err) + tenantOne := qb.tenantQuerierAssignments.tenantsByID["one"] + + err = qb.enqueueObjectsForTests("two", 10) + assert.NoError(t, err) + tenantTwo := qb.tenantQuerierAssignments.tenantsByID["two"] + + type dequeueVal struct { + req *tenantRequest + tenant *queueTenant + } + expectedDequeueVals := []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 0), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 1), tenantTwo}, + } + + for _, expected := range expectedDequeueVals { + req, tenant, err := qb.dequeueRequestForQuerier("querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } + + // TODO (casie): This is consistent with the previous test only because the previous test + // checked n % numTenants == 0 dequeues. + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 2), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, + } + + for _, expected := range expectedDequeueVals { + req, tenant, err := qb.dequeueRequestForQuerier("querier-2") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } // After notify shutdown for querier-2, it's expected to own no queue. qb.notifyQuerierShutdown("querier-2") - tenant, _, err := qb.tenantQuerierAssignments.getNextTenantForQuerier(-1, "querier-2") + req, tenant, err := qb.dequeueRequestForQuerier("querier-2") + assert.Nil(t, req) assert.Nil(t, tenant) assert.Equal(t, ErrQuerierShuttingDown, err) // However, querier-1 still get queues because it's still running. - confirmOrderForQuerier(t, qb, "querier-1", -1, qOne, qTwo, qOne, qTwo) + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 4), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, + } + + for _, expected := range expectedDequeueVals { + req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } // After disconnecting querier-2, it's expected to own no queue. qb.tenantQuerierAssignments.removeQuerier("querier-2") - tenant, _, err = qb.tenantQuerierAssignments.getNextTenantForQuerier(-1, "querier-2") + req, tenant, err = qb.dequeueRequestForQuerier("querier-2") + assert.Nil(t, req) assert.Nil(t, tenant) assert.Equal(t, ErrQuerierShuttingDown, err) + } func TestQueuesWithQueriers(t *testing.T) { @@ -185,7 +388,7 @@ func TestQueuesWithQueriers(t *testing.T) { qb.addQuerierConnection(qid) // No querier has any queues yet. - req, tenant, _, err := qb.dequeueRequestForQuerier(-1, qid) + req, tenant, err := qb.dequeueRequestForQuerier(qid) assert.Nil(t, req) assert.Nil(t, tenant) assert.NoError(t, err) @@ -196,7 +399,13 @@ func TestQueuesWithQueriers(t *testing.T) { // Add tenant queues. for i := 0; i < numTenants; i++ { uid := TenantID(fmt.Sprintf("tenant-%d", i)) - getOrAdd(t, qb, uid, maxQueriersPerTenant) + //getOrAdd(t, qb, uid, maxQueriersPerTenant) + err := qb.tenantQuerierAssignments.createOrUpdateTenant(uid, maxQueriersPerTenant) + assert.NoError(t, err) + + //Enqueue some stuff so that the tree queue node exists + err = qb.enqueueObjectsForTests(uid, 1) + assert.NoError(t, err) // Verify it has maxQueriersPerTenant queriers assigned now. qs := qb.tenantQuerierAssignments.tenantQuerierIDs[uid] @@ -207,18 +416,9 @@ func TestQueuesWithQueriers(t *testing.T) { // and compute mean and stdDev. queriersMap := make(map[QuerierID]int) - for q := 0; q < queriers; q++ { - qid := QuerierID(fmt.Sprintf("querier-%d", q)) - - lastTenantIndex := -1 - for { - _, newIx, err := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndex, qid) - assert.NoError(t, err) - if newIx < lastTenantIndex { - break - } - lastTenantIndex = newIx - queriersMap[qid]++ + for _, querierSet := range qb.tenantQuerierAssignments.tenantQuerierIDs { + for querierID := range querierSet { + queriersMap[querierID]++ } } @@ -314,7 +514,12 @@ func TestQueues_ForgetDelay(t *testing.T) { // Add tenant queues. for i := 0; i < numTenants; i++ { tenantID := TenantID(fmt.Sprintf("tenant-%d", i)) - getOrAdd(t, qb, tenantID, maxQueriersPerTenant) + err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriersPerTenant) + assert.NoError(t, err) + + //Enqueue some stuff so that the tree queue node exists + err = qb.enqueueObjectsForTests(tenantID, 1) + assert.NoError(t, err) } // We expect querier-1 to have some tenants. @@ -406,7 +611,13 @@ func TestQueues_ForgetDelay_ShouldCorrectlyHandleQuerierReconnectingBeforeForget // Add tenant queues. for i := 0; i < numTenants; i++ { tenantID := TenantID(fmt.Sprintf("tenant-%d", i)) - getOrAdd(t, qb, tenantID, maxQueriersPerTenant) + err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriersPerTenant) + assert.NoError(t, err) + + //Enqueue some stuff so that the tree queue node exists + err = qb.enqueueObjectsForTests(tenantID, 1) + assert.NoError(t, err) + //getOrAdd(t, qb, tenantID, maxQueriersPerTenant) } // We expect querier-1 to have some tenants. @@ -466,17 +677,6 @@ func generateQuerier(r *rand.Rand) QuerierID { return QuerierID(fmt.Sprint("querier-", r.Int()%5)) } -func getOrAdd(t *testing.T, qb *queueBroker, tenantID TenantID, maxQueriers int) *list.List { - addedQueue, err := qb.getOrAddTenantQueue(tenantID, maxQueriers) - assert.Nil(t, err) - assert.NotNil(t, addedQueue) - assert.NoError(t, isConsistent(qb)) - reAddedQueue, err := qb.getOrAddTenantQueue(tenantID, maxQueriers) - assert.Nil(t, err) - assert.Equal(t, addedQueue, reAddedQueue) - return addedQueue.localQueue -} - // getOrAddTenantQueue is a test utility, not intended for use by consumers of queueBroker func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) (*TreeQueue, error) { err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriers) @@ -519,19 +719,90 @@ func confirmOrderForQuerier(t *testing.T, qb *queueBroker, querier QuerierID, la return lastTenantIndex } +// getTenantsByQuerier returns the list of tenants handled by the provided QuerierID. +func getTenantsByQuerier(broker *queueBroker, querierID QuerierID) []TenantID { + var tenantIDs []TenantID + for _, tenantID := range broker.tenantQuerierAssignments.tenantIDOrder { + querierSet := broker.tenantQuerierAssignments.tenantQuerierIDs[tenantID] + if querierSet == nil { + // If it's nil then all queriers can handle this tenant. + tenantIDs = append(tenantIDs, tenantID) + continue + } + if _, ok := querierSet[querierID]; ok { + tenantIDs = append(tenantIDs, tenantID) + } + } + return tenantIDs +} + +//// Used in test to get a pointer to the local queue that we want to check against +//func getOrAdd(t *testing.T, qb *queueBroker, tenantID TenantID, maxQueriers int) error { +// err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriers) +// if err != nil { +// return err +// } +// //assert.NoError(t, err) +// //assert.NoError(t, isConsistent(qb)) +// +// node, err := qb.getOrAddTenantQueue(tenantID) +// if err != nil { +// return err +// } +// //assert.NoError(t, err) +// //assert.NotNil(t, node) +// +// reAddedNode, err := qb.getOrAddTenantQueue(tenantID) +// assert.Nil(t, err) +// assert.Equal(t, node, reAddedNode) +// //return node.localQueue +//} + +// getOrAddTenantQueue is a test utility, not intended for use by consumers of queueBroker +func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) (*Node, error) { + err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriers) + if err != nil { + return nil, err + } + + node, err := qb.queueTree.rootNode.getOrAddNode(QueuePath{string(tenantID)}, qb.queueTree) + if err != nil { + return nil, err + } + return node, nil +} + +// removeTenantQueue is a test utility, not intended for use by consumers of queueBroker +func (qb *queueBroker) removeTenantQueue(tenantID TenantID) bool { + qb.tenantQuerierAssignments.removeTenant(tenantID) + queuePath := QueuePath{string(tenantID)} + //return qb.tenantQueuesTree.deleteNode(queuePath) + return qb.queueTree.rootNode.deleteNode(queuePath) +} + +func (qb *queueBroker) makeQueuePathForTests(tenantID TenantID) QueuePath { + return QueuePath{string(tenantID)} +} + func isConsistent(qb *queueBroker) error { if len(qb.tenantQuerierAssignments.querierIDsSorted) != len(qb.tenantQuerierAssignments.queriersByID) { return fmt.Errorf("inconsistent number of sorted queriers and querier connections") } tenantCount := 0 + for ix, tenantID := range qb.tenantQuerierAssignments.tenantIDOrder { - if tenantID != "" && qb.getQueue(tenantID) == nil { + path := qb.makeQueuePathForTests(tenantID) + node := qb.queueTree.rootNode.getNode(path) + + if tenantID != "" && node == nil { return fmt.Errorf("tenant %s doesn't have queue", tenantID) } - if tenantID == "" && qb.getQueue(tenantID) != nil { + + if tenantID == "" && node != nil { return fmt.Errorf("tenant %s shouldn't have queue", tenantID) } + if tenantID == "" { continue } @@ -557,27 +828,18 @@ func isConsistent(qb *queueBroker) error { } } - tenantQueueCount := qb.tenantQueuesTree.NodeCount() - 1 // exclude root node - if tenantCount != tenantQueueCount { + tenantQueueCount := qb.queueTree.rootNode.nodeCount() - 1 + if tenantQueueCount != tenantCount { return fmt.Errorf("inconsistent number of tenants list and tenant queues") } return nil } -// getTenantsByQuerier returns the list of tenants handled by the provided QuerierID. -func getTenantsByQuerier(broker *queueBroker, querierID QuerierID) []TenantID { - var tenantIDs []TenantID - for _, tenantID := range broker.tenantQuerierAssignments.tenantIDOrder { - querierSet := broker.tenantQuerierAssignments.tenantQuerierIDs[tenantID] - if querierSet == nil { - // If it's nil then all queriers can handle this tenant. - tenantIDs = append(tenantIDs, tenantID) - continue - } - if _, ok := querierSet[querierID]; ok { - tenantIDs = append(tenantIDs, tenantID) - } +func (n *Node) nodeCount() int { + count := 1 + for _, child := range n.queueMap { + count += child.nodeCount() } - return tenantIDs + return count } diff --git a/pkg/scheduler/queue/tree_queue_sketch.go b/pkg/scheduler/queue/tree_queue_sketch.go index bcb709a01e9..f20df6f6dc2 100644 --- a/pkg/scheduler/queue/tree_queue_sketch.go +++ b/pkg/scheduler/queue/tree_queue_sketch.go @@ -8,16 +8,16 @@ import ( // TODO (casie): We might not need this interface anymore. type TreeNodeIFace interface { //EnqueueFrontByPath(QueuePath, any) error - enqueueBackByPath(OpsPath, any) error + enqueueBackByPath(*Tree, QueuePath, any) error dequeue() (QueuePath, any) // Dequeue returns the dequeued value, and the QueuePath (starting with root) to the node which was dequeued from IsEmpty() bool Name() string getLocalQueue() *list.List - getOrAddNode(OpsPath) (TreeNodeIFace, error) + getOrAddNode(QueuePath, *Tree) (*Node, error) } type DequeueAlgorithm interface { - ops() dequeueOps + dequeueOps() dequeueOps } type OpsPath []struct { @@ -26,26 +26,47 @@ type OpsPath []struct { } type dequeueGetNodeFunc func(*Node) (TreeNodeIFace, bool) -type dequeueUpdateStateFunc func(*Node, any, string) +type dequeueUpdateStateFunc func(*Node, any, string, bool) type dequeueOps struct { getNode dequeueGetNodeFunc updateState dequeueUpdateStateFunc } +type Tree struct { + rootNode *Node + algosByDepth []DequeueAlgorithm +} + +// TODO (casie): Implement maxQueueSize to mirror qb.maxTenantQueueSize? type Node struct { name string localQueue *list.List - queuePosition int // position for dequeueing from queueOrder + queuePosition int // last queue position in queueOrder that was dequeued from queueOrder []string // order for dequeueing from self/children - queueMap map[string]TreeNodeIFace + queueMap map[string]*Node depth int dequeueAlgorithm DequeueAlgorithm } -// TODO (casie): Do we want some concept of a max-depth? +// TODO (casie): Instead of passing in actual state objects, just pass a string that maps to state +func NewTree(dequeueAlgorithms ...DequeueAlgorithm) *Tree { + if len(dequeueAlgorithms) == 0 { + return nil + } + root, err := NewNode("root", 0, dequeueAlgorithms[0]) + if err != nil { + // TODO (casie): Make NewNode private, remove err return + fmt.Print(err) + } + root.depth = 0 + return &Tree{ + rootNode: root, + algosByDepth: dequeueAlgorithms, + } +} -func NewNode(name string, da DequeueAlgorithm) (TreeNodeIFace, error) { +func NewNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { // TODO (casie): write a unit test if da == nil { return nil, fmt.Errorf("cannot create a node without a defined dequeueing algorithm") @@ -59,9 +80,10 @@ func NewNode(name string, da DequeueAlgorithm) (TreeNodeIFace, error) { return &Node{ name: name, localQueue: list.New(), - queuePosition: localQueueIndex, + queuePosition: localQueueIndex - 1, // start from -2 so that on first dequeue, localQueueIndex is dequeued from queueOrder: make([]string, 0), - queueMap: make(map[string]TreeNodeIFace, 1), + queueMap: make(map[string]*Node, 1), + depth: depth, dequeueAlgorithm: da, }, nil } @@ -71,6 +93,14 @@ func (n *Node) IsEmpty() bool { return n.localQueue.Len() == 0 && len(n.queueMap) == 0 } +func (n *Node) ItemCount() int { + items := n.localQueue.Len() + for _, child := range n.queueMap { + items += child.ItemCount() + } + return items +} + func (n *Node) Name() string { return n.name } @@ -79,8 +109,8 @@ func (n *Node) getLocalQueue() *list.List { return n.localQueue } -func (n *Node) enqueueBackByPath(pathFromNode OpsPath, v any) error { - childNode, err := n.getOrAddNode(pathFromNode) +func (n *Node) enqueueBackByPath(tree *Tree, pathFromNode QueuePath, v any) error { + childNode, err := n.getOrAddNode(pathFromNode, tree) if err != nil { return err } @@ -103,7 +133,8 @@ func (n *Node) dequeue() (QueuePath, any) { var dequeueNode TreeNodeIFace // continue until we've found a value or checked all nodes that need checking for v == nil && !checkedAllNodes { - dequeueNode, checkedAllNodes = n.dequeueAlgorithm.ops().getNode(n) + dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueOps().getNode(n) + var deletedNode bool // dequeueing from local queue if dequeueNode == n { if n.localQueue.Len() > 0 { @@ -128,64 +159,118 @@ func (n *Node) dequeue() (QueuePath, any) { n.queueOrder = append(n.queueOrder[:idx], n.queueOrder[idx+1:]...) } } - // removing an element sets our position one step forward; reset it to original queuePosition - n.queuePosition-- + // removing an element sets our position one step forward; + // tell state to reset it to original queuePosition + deletedNode = true } } - n.dequeueAlgorithm.ops().updateState(n, v, dequeueNode.Name()) + n.dequeueAlgorithm.dequeueOps().updateState(n, v, dequeueNode.Name(), deletedNode) } return append(path, childPath...), v } -// TODO (casie): Ugly -- if a matching node name is found, the dequeueAlgorithm will be ignored, which -// -// could lead to some unexpected behavior. -// +func (n *Node) getNode(pathFromNode QueuePath) *Node { + if len(pathFromNode) == 0 { + return n + } + + if n.queueMap == nil { + return nil + } + + if childQueue, ok := n.queueMap[pathFromNode[0]]; ok { + return childQueue.getNode(pathFromNode[1:]) + } + + // no child node matches next path segment + return nil +} + +func (n *Node) deleteNode(pathFromNode QueuePath) bool { + if len(pathFromNode) == 0 { + // node cannot delete itself + return false + } + + // parentPath is everything except the last node, childNode is the last in path + parentPath, deleteNodeName := pathFromNode[:len(pathFromNode)-1], pathFromNode[len(pathFromNode)-1] + + parentNode := n.getNode(parentPath) + if parentNode == nil { + // not found + return false + } + + delete(parentNode.queueMap, deleteNodeName) + for i, name := range parentNode.queueOrder { + if name == deleteNodeName { + parentNode.queueOrder = append(n.queueOrder[:i], n.queueOrder[i+1:]...) + if parentNode.queuePosition >= len(parentNode.queueOrder) { + // set _last_ dequeue position to last elt in order + parentNode.queuePosition = len(parentNode.queueOrder) - 1 + } + break + } + } + return true +} + // getOrAddNode checks whether the first node name in pathFromNode exists in a node's children; // if no node exists, one is created and added to the "end" of the node's children, from the perspective // of node.queuePosition (e.g., if node.queuePosition is 4, the new node will be created at position 3). // This follows queue behavior, but if, for some reason, we ever want to create nodes in a way that violates // queueing behavior, this will require a refactor, as will, well, the entire "tree queue" data structure. -func (n *Node) getOrAddNode(pathFromNode OpsPath) (TreeNodeIFace, error) { +func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *Tree) (*Node, error) { if len(pathFromNode) == 0 { return n, nil } - var childNode TreeNodeIFace + var childNode *Node var ok bool var err error - if childNode, ok = n.queueMap[pathFromNode[0].name]; !ok { + if childNode, ok = n.queueMap[pathFromNode[0]]; !ok { // child does not exist, create it - childNode, err = NewNode(pathFromNode[0].name, pathFromNode[0].dequeueAlgorithm) + if n.depth+1 >= len(tree.algosByDepth) { + return nil, fmt.Errorf("cannot add a node beyond max tree depth: %v", len(tree.algosByDepth)) + } + childNode, err = NewNode(pathFromNode[0], n.depth+1, tree.algosByDepth[n.depth+1]) if err != nil { return nil, err } - // add childNode to rrn.queueMap + // add childNode to n.queueMap n.queueMap[childNode.Name()] = childNode - // add childNode to rrn.queueOrder before the current position, update rrn.queuePosition to current element - if n.queuePosition == localQueueIndex { + + // add childNode to n.queueOrder before the current position, update n.queuePosition to current element + if n.queuePosition <= localQueueIndex { n.queueOrder = append(n.queueOrder, childNode.Name()) } else { n.queueOrder = append(n.queueOrder[:n.queuePosition], append([]string{childNode.Name()}, n.queueOrder[n.queuePosition:]...)...) - n.queuePosition++ + n.queuePosition++ // keep position pointed at same element } } // TODO (casie): How to determine what dequeueAlgo to pass? - return childNode.getOrAddNode(pathFromNode[1:]) + return childNode.getOrAddNode(pathFromNode[1:], tree) } type roundRobinState struct { subtreeNodesChecked int } -func (rrs *roundRobinState) ops() dequeueOps { +func (rrs *roundRobinState) dequeueOps() dequeueOps { getNode := func(n *Node) (TreeNodeIFace, bool) { + // advance the queue position for this dequeue + n.queuePosition++ + if n.queuePosition >= len(n.queueOrder) { + n.queuePosition = localQueueIndex + } + checkedAllNodes := rrs.subtreeNodesChecked == len(n.queueOrder)+1 if n.queuePosition == localQueueIndex { return n, checkedAllNodes } + currentNodeName := n.queueOrder[n.queuePosition] if node, ok := n.queueMap[currentNodeName]; ok { return node, checkedAllNodes @@ -194,16 +279,20 @@ func (rrs *roundRobinState) ops() dequeueOps { } // TODO (casie): Ignoring nodeName here because it doesn't matter...ugly - updateState := func(n *Node, v any, _ string) { + updateState := func(n *Node, v any, _ string, deletedNode bool) { if v != nil { rrs.subtreeNodesChecked = 0 } else { rrs.subtreeNodesChecked++ } - - n.queuePosition++ - if n.queuePosition >= len(n.queueOrder) { - n.queuePosition = localQueueIndex + if deletedNode { + n.queuePosition-- + // if we try to go beyond something that would increment to + // the localQueueIndex or in queueOrder, we should wrap around to the + // back of queueOrder. + if n.queuePosition < localQueueIndex-1 { + n.queuePosition = len(n.queueOrder) - 1 + } } } @@ -219,7 +308,7 @@ type shuffleShardState struct { nodesChecked int } -func (sss *shuffleShardState) ops() dequeueOps { +func (sss *shuffleShardState) dequeueOps() dequeueOps { // start from ssn.queuePosition // check that tenant for querierID against availabilityMap // if exists, move element to "back" of queue @@ -232,6 +321,13 @@ func (sss *shuffleShardState) ops() dequeueOps { } checkedAllNodes := sss.nodesChecked == len(n.queueOrder)+1 // must check local queue as well + + // advance queue position for dequeue + n.queuePosition++ + if n.queuePosition >= len(n.queueOrder) { + n.queuePosition = 0 + } + // no children if len(n.queueOrder) == 0 || n.queuePosition == localQueueIndex { return n, checkedAllNodes @@ -247,6 +343,10 @@ func (sss *shuffleShardState) ops() dequeueOps { // increment nodes checked even if not in tenant-querier map sss.nodesChecked++ checkedAllNodes = sss.nodesChecked == len(n.queueOrder) + // if the tenant-querier set is nil, any querier can serve this tenant + if sss.tenantQuerierMap[TenantID(tenantName)] == nil { + return n.queueMap[tenantName], checkedAllNodes + } if tenantQuerierSet, ok := sss.tenantQuerierMap[TenantID(tenantName)]; ok { if _, ok := tenantQuerierSet[*sss.currentQuerier]; ok { return n.queueMap[tenantName], checkedAllNodes @@ -257,10 +357,17 @@ func (sss *shuffleShardState) ops() dequeueOps { return nil, checkedAllNodes } - updateState := func(n *Node, v any, nodeName string) { + updateState := func(n *Node, v any, nodeName string, deletedNode bool) { + if deletedNode { + // when we delete a child node, we want to step back in the queue so the + // previously-next tenant will still be next (e.g., if queueOrder is + // [childA, childB, childC] and childB (position 1) is deleted, we want to + // step back to position 0 so that in the next dequeue, position 1 (childC) + // is dequeued + n.queuePosition-- + } if v != nil { sss.nodesChecked = 0 - n.queuePosition++ return } @@ -268,7 +375,6 @@ func (sss *shuffleShardState) ops() dequeueOps { // if dequeueNode was self, advance queue position no matter what, and return if n.queuePosition == localQueueIndex { - n.queuePosition++ return } @@ -276,6 +382,12 @@ func (sss *shuffleShardState) ops() dequeueOps { for i, child := range n.queueOrder { if child == nodeName { n.queueOrder = append(n.queueOrder[:i], append(n.queueOrder[i+1:], child)...) // ugly + n.queuePosition-- + // wrap around to back of order if we go past the first element in n.queueOrder. + // In the next dequeue, we will step forward again. + if n.queuePosition <= localQueueIndex { + n.queuePosition = len(n.queueOrder) - 1 + } } } } diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go index 13f7ee37e36..9aa627ec7af 100644 --- a/pkg/scheduler/queue/tree_queue_sketch_test.go +++ b/pkg/scheduler/queue/tree_queue_sketch_test.go @@ -2,11 +2,13 @@ package queue import ( "fmt" + //"fmt" "github.com/stretchr/testify/require" "testing" ) -// TODO (casie): Add tests for dequeuing from multiple different node children states +// TODO (casie): Write tests for NewTree +// TODO (casie): Write a test for dequeuing from sss childA, enqueue to new childB, expect to dequeue next from childB func Test_NewNode(t *testing.T) { tests := []struct { @@ -32,7 +34,7 @@ func Test_NewNode(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - _, err := NewNode("root", tt.rootAlgo) + _, err := NewNode("root", 0, tt.rootAlgo) if tt.expectErr { require.Error(t, err) } else { @@ -45,60 +47,60 @@ func Test_NewNode(t *testing.T) { func Test_EnqueueBackByPath(t *testing.T) { tests := []struct { - name string - rootAlgo DequeueAlgorithm - children []OpsPath - expectErr bool + name string + treeAlgosByDepth []DequeueAlgorithm + rootAlgo DequeueAlgorithm + children []QueuePath + expectErr bool }{ { - name: "enqueue round-robin node to round-robin node", - rootAlgo: &roundRobinState{}, - children: []OpsPath{{{"round-robin-child-1", &roundRobinState{}}}}, + name: "enqueue round-robin node to round-robin node", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, + children: []QueuePath{{"round-robin-child-1"}}, }, { - name: "enqueue shuffle-shard node to round-robin node", - rootAlgo: &roundRobinState{}, - children: []OpsPath{{{ - "shuffle-shard-child-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - }}}, + name: "enqueue shuffle-shard node to round-robin node", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + children: []QueuePath{{"shuffle-shard-child-1"}}, }, { - name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node", - rootAlgo: &roundRobinState{}, - children: []OpsPath{{{"shuffle-shard-child-1", &shuffleShardState{}}}}, - expectErr: true, + name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &shuffleShardState{}}, + children: []QueuePath{{"shuffle-shard-child-1"}}, + expectErr: true, }, { - name: "enqueue round-robin node to shuffle-shard node", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - children: []OpsPath{{{"round-robin-child-1", &roundRobinState{}}}}, + name: "enqueue round-robin node to shuffle-shard node", + treeAlgosByDepth: []DequeueAlgorithm{ + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + }, + children: []QueuePath{{"round-robin-child-1"}}, }, { - name: "create tree with multiple shuffle-shard depths", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - children: []OpsPath{ - { - {"child", &roundRobinState{}}, - {"grandchild", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - }, + name: "create tree with multiple shuffle-shard depths", + treeAlgosByDepth: []DequeueAlgorithm{ + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, }, + children: []QueuePath{{"child"}, {"grandchild"}}, }, { - name: "enqueue different types of nodes at the same depth", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - children: []OpsPath{ - {{"child-1", &roundRobinState{}}}, - {{"child-2", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}}, - }, + name: "enqueue beyond max-depth", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}}, + children: []QueuePath{{"child"}, {"child, grandchild"}}, + expectErr: true, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - root, err := NewNode("root", tt.rootAlgo) - require.NoError(t, err) + tree := NewTree(tt.treeAlgosByDepth...) + root := tree.rootNode + var err error for _, childPath := range tt.children { - err = root.enqueueBackByPath(childPath, "some-object") + err = root.enqueueBackByPath(tree, childPath, "some-object") } if tt.expectErr { require.Error(t, err) @@ -142,26 +144,24 @@ func Test_Dequeue_RootNode(t *testing.T) { case *shuffleShardState: tt.rootAlgo.(*shuffleShardState).currentQuerier = &querierID } - - root, err := NewNode("root", tt.rootAlgo) - require.NoError(t, err) + tree := NewTree(tt.rootAlgo) + root := tree.rootNode for _, elt := range tt.enqueueToRoot { - err := root.enqueueBackByPath(OpsPath{}, elt) + err := root.enqueueBackByPath(tree, QueuePath{}, elt) require.NoError(t, err) } + rootPath := QueuePath{"root"} for _, elt := range tt.enqueueToRoot { path, v := root.dequeue() - require.Equal(t, QueuePath{"root"}, path) + require.Equal(t, rootPath, path) require.Equal(t, elt, v) } - if tt.enqueueToRoot == nil { - path, v := root.dequeue() - require.Equal(t, QueuePath{"root"}, path) - require.Nil(t, v) - } + path, v := root.dequeue() + require.Equal(t, rootPath, path) + require.Nil(t, v) }) } @@ -175,7 +175,7 @@ func Test_RoundRobinDequeue(t *testing.T) { childQueueObjects map[string][]any grandchildren []string grandchildrenQueueObjects map[string]struct { - path OpsPath + path QueuePath objs []any } expected []string @@ -206,10 +206,10 @@ func Test_RoundRobinDequeue(t *testing.T) { expected: []string{"child-1:object-1", "child-2:object-1", "grandchild-1:object-1"}, grandchildren: []string{"grandchild-1"}, grandchildrenQueueObjects: map[string]struct { - path OpsPath + path QueuePath objs []any }{"grandchild-1": { - path: OpsPath{{"child-1", &roundRobinState{}}, {"grandchild-1", &roundRobinState{}}}, + path: QueuePath{"child-1", "grandchild-1"}, objs: []any{"grandchild-1:object-1"}, }}, }, @@ -217,24 +217,24 @@ func Test_RoundRobinDequeue(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - root, err := NewNode("root", &roundRobinState{}) - require.NoError(t, err) + tree := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + root := tree.rootNode for _, sqo := range tt.selfQueueObjects { - err = root.enqueueBackByPath(OpsPath{}, sqo) + err := root.enqueueBackByPath(tree, QueuePath{}, sqo) require.NoError(t, err) } for _, child := range tt.children { for _, obj := range tt.childQueueObjects[child] { - err = root.enqueueBackByPath(OpsPath{{child, &roundRobinState{}}}, obj) + _ = root.enqueueBackByPath(tree, QueuePath{child}, obj) } } for _, grandchild := range tt.grandchildren { gqo := tt.grandchildrenQueueObjects[grandchild] for _, obj := range gqo.objs { - err := root.enqueueBackByPath(gqo.path, obj) + err := root.enqueueBackByPath(tree, gqo.path, obj) require.NoError(t, err) } } @@ -249,215 +249,233 @@ func Test_RoundRobinDequeue(t *testing.T) { } } +func Test_DequeueOrderAfterEnqueue(t *testing.T) { + type opType string + enqueue := opType("enqueue") + dequeue := opType("dequeue") + placeholderQuerier := QuerierID("") + + type op struct { + kind opType + path QueuePath + obj any + } + + tests := []struct { + name string + treeAlgosByDepth []DequeueAlgorithm + operationOrder []op + }{ + { + name: "should dequeue from new (next-in-queue) child immediately after it is added", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, + operationOrder: []op{ + {enqueue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{"root", "child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{"root", "child-2"}, "obj-3"}, + {dequeue, QueuePath{"root", "child-1"}, "obj-2"}, + {dequeue, QueuePath{"root"}, nil}, + }, + }, + { + name: "should dequeue from new (next-in-queue) shuffle-shard child immediately after it is added", + treeAlgosByDepth: []DequeueAlgorithm{ + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, currentQuerier: &placeholderQuerier}, + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, currentQuerier: &placeholderQuerier}, + }, + operationOrder: []op{ + {enqueue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{"root", "child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{"root", "child-2"}, "obj-3"}, + {dequeue, QueuePath{"root", "child-1"}, "obj-2"}, + {dequeue, QueuePath{"root"}, nil}, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree := NewTree(tt.treeAlgosByDepth...) + root := tree.rootNode + + for _, operation := range tt.operationOrder { + if operation.kind == enqueue { + err := root.enqueueBackByPath(tree, operation.path, operation.obj) + require.NoError(t, err) + } + if operation.kind == dequeue { + path, obj := root.dequeue() + require.Equal(t, operation.path, path) + require.Equal(t, operation.obj, obj) + } + } + }) + } +} + func Test_ShuffleShardDequeue(t *testing.T) { type enqueueObj struct { obj any - path OpsPath + path QueuePath } tests := []struct { - name string - rootAlgo DequeueAlgorithm - state *shuffleShardState - currQuerier []QuerierID - enqueueObjs []enqueueObj - expected []any - expectErr bool + name string + treeAlgosByDepth []DequeueAlgorithm + state *shuffleShardState + currQuerier []QuerierID + enqueueObjs []enqueueObj + expected []any + expectErr bool }{ { - name: "happy path - tenant found in tenant-querier map under first child", - rootAlgo: &roundRobinState{}, + name: "happy path - tenant found in tenant-querier map under first child", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + }, state: &shuffleShardState{ tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, }, currQuerier: []QuerierID{"querier-1"}, enqueueObjs: []enqueueObj{ - {obj: "query-1", path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-1", &roundRobinState{}}, - }}, + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, }, expected: []any{"query-1"}, }, { - name: "tenant exists, but not for querier", - rootAlgo: &roundRobinState{}, + name: "tenant exists, but not for querier", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + }, state: &shuffleShardState{ tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, }, currQuerier: []QuerierID{"querier-1"}, enqueueObjs: []enqueueObj{ - { - obj: "query-1", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-1", &roundRobinState{}}, - }, - }, + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, }, expected: []any{nil}, }, { - name: "1 of 3 tenants exist for querier", - rootAlgo: &roundRobinState{}, + name: "1 of 3 tenants exist for querier", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + }, state: &shuffleShardState{ tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, - currQuerier: []QuerierID{"querier-1"}, + currQuerier: []QuerierID{"querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ - { - obj: "query-1", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-1", &roundRobinState{}}, - }, - }, - { - obj: "query-2", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-2", &roundRobinState{}}, - }, - }, - { - obj: "query-3", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-3", &roundRobinState{}}, - }, - }, + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-1", "tenant-3"}}, }, - expected: []any{"query-3"}, + expected: []any{"query-3", nil}, }, { - name: "tenant exists for querier on next parent node", - rootAlgo: &roundRobinState{}, + name: "tenant exists for querier on next parent node", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + }, state: &shuffleShardState{ tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ - { - obj: "query-1", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-1", &roundRobinState{}}, - }, - }, - { - obj: "query-2", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-2", &roundRobinState{}}, - }, - }, - { - obj: "query-3", - path: OpsPath{ - {"query-component-2", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-3", &roundRobinState{}}, - }, - }, - { - obj: "query-4", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-3", &roundRobinState{}}, - }, - }, + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, + {obj: "query-4", path: QueuePath{"query-component-1", "tenant-3"}}, }, expected: []any{"query-4", "query-3", nil}, }, { - name: "2 of 3 tenants exist for querier", - rootAlgo: &roundRobinState{}, + name: "2 of 3 tenants exist for querier", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + }, state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ - { - obj: "query-1", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-1", &roundRobinState{}}, - }, - }, - { - obj: "query-2", - path: OpsPath{ - {"query-component-1", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-2", &roundRobinState{}}, - }, - }, - { - obj: "query-3", - path: OpsPath{ - {"query-component-2", &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, - {"tenant-3", &roundRobinState{}}, - }, - }, + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, }, expected: []any{"query-1", "query-3", nil}, }, { - name: "root node is shuffle-shard node", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}}, + name: "root node is shuffle-shard node", + treeAlgosByDepth: []DequeueAlgorithm{ + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + &roundRobinState{}, + }, + state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}}, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ - { - obj: "query-1", - path: OpsPath{ - {"tenant-2", &roundRobinState{}}, - {"query-component-1", &roundRobinState{}}, - }, - }, - { - obj: "query-2", - path: OpsPath{ - {"tenant-1", &roundRobinState{}}, - {"query-component-2", &roundRobinState{}}, - }, - }, - { - obj: "query-3", - path: OpsPath{ - {"tenant-1", &roundRobinState{}}, - {"query-component-1", &roundRobinState{}}, - }, - }, + {obj: "query-1", path: QueuePath{"tenant-2", "query-component-1"}}, + {obj: "query-2", path: QueuePath{"tenant-1", "query-component-2"}}, + {obj: "query-3", path: QueuePath{"tenant-1", "query-component-1"}}, }, expected: []any{"query-2", "query-3", nil}, }, { - name: "dequeueing for one querier returns nil, but does return for a different querier", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + name: "dequeueing for one querier returns nil, but does return for a different querier", + treeAlgosByDepth: []DequeueAlgorithm{ + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + &roundRobinState{}, + }, state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}}, currQuerier: []QuerierID{"querier-2", "querier-1"}, enqueueObjs: []enqueueObj{ - { - obj: "query-1", - path: OpsPath{ - {"tenant-1", &roundRobinState{}}, - {"query-component-1", &roundRobinState{}}, - }, - }, + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, }, expected: []any{nil, "query-1"}, }, { - name: "no querier set in state", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + name: "no querier set in state", + treeAlgosByDepth: []DequeueAlgorithm{ + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + }, state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}}, currQuerier: []QuerierID{""}, enqueueObjs: []enqueueObj{ - { - obj: "query-1", - path: OpsPath{ - {"tenant-1", &roundRobinState{}}, - {"query-component-1", &roundRobinState{}}, - }, - }, + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, }, expected: []any{nil}, }, + { + // TODO (casie): also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) + name: "dequeue from a tenant with a nil tenant-querier map", + treeAlgosByDepth: []DequeueAlgorithm{ + &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + &roundRobinState{}, + }, + state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}}, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + {obj: "query-2", path: QueuePath{"tenant-2", "query-component-1"}}, + {obj: "query-3", path: QueuePath{"tenant-3", "query-component-1"}}, + }, + expected: []any{"query-1", "query-2", "query-3"}, + }, } for _, tt := range tests { @@ -465,22 +483,20 @@ func Test_ShuffleShardDequeue(t *testing.T) { currentQuerier := QuerierID("placeholder") tt.state.currentQuerier = ¤tQuerier - // TODO (casie): ugly - switch tt.rootAlgo.(type) { - case *shuffleShardState: - tt.rootAlgo = tt.state + // We need a reference to state in order to be able to + // update the state's currentQuerier. + for i, da := range tt.treeAlgosByDepth { + switch da.(type) { + case *shuffleShardState: + tt.treeAlgosByDepth[i] = tt.state + } } - root, err := NewNode("root", tt.rootAlgo) - require.NoError(t, err) + + tree := NewTree(tt.treeAlgosByDepth...) + root := tree.rootNode for _, o := range tt.enqueueObjs { - for i, elt := range o.path { - switch elt.dequeueAlgorithm.(type) { - case *shuffleShardState: - o.path[i].dequeueAlgorithm = tt.state - } - } - err = root.enqueueBackByPath(o.path, o.obj) + err := root.enqueueBackByPath(tree, o.path, o.obj) require.NoError(t, err) } // currQuerier at position i is used to dequeue the expected result at position i @@ -508,26 +524,13 @@ func Test_ChangeShuffleShardState(t *testing.T) { currentQuerier: nil, } - root, err := NewNode("root", state) - //tree, err := NewTree([]NodeType{shuffleShard, roundRobin, roundRobin}, state) - require.NoError(t, err) + tree := NewTree(state, &roundRobinState{}, &roundRobinState{}) + root := tree.rootNode - err = root.enqueueBackByPath(OpsPath{ - {"tenant-1", &roundRobinState{}}, - {"query-component-1", &roundRobinState{}}, - }, "query-1") - err = root.enqueueBackByPath(OpsPath{ - {"tenant-2", &roundRobinState{}}, - {"query-component-1", &roundRobinState{}}, - }, "query-2") - err = root.enqueueBackByPath(OpsPath{ - {"tenant-2", &roundRobinState{}}, - {"query-component-1", &roundRobinState{}}, - }, "query-3") - err = root.enqueueBackByPath(OpsPath{ - {"tenant-2", &roundRobinState{}}, - {"query-component-1", &roundRobinState{}}, - }, "query-4") + err := root.enqueueBackByPath(tree, QueuePath{"tenant-1", "query-component-1"}, "query-1") + err = root.enqueueBackByPath(tree, QueuePath{"tenant-2", "query-component-1"}, "query-2") + err = root.enqueueBackByPath(tree, QueuePath{"tenant-2", "query-component-1"}, "query-3") + err = root.enqueueBackByPath(tree, QueuePath{"tenant-2", "query-component-1"}, "query-4") require.NoError(t, err) querier1 := QuerierID("querier-1") From 26497e9d470b70f58262338c109ec4db7d881e6e Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 3 Jun 2024 13:55:33 -0700 Subject: [PATCH 07/26] added back lastTenantIndex concept, and empty tenant placeholders --- pkg/scheduler/queue/queue.go | 26 +- pkg/scheduler/queue/queue_test.go | 27 +- pkg/scheduler/queue/tenant_queues.go | 33 ++- pkg/scheduler/queue/tenant_queues_test.go | 252 ++++++------------ pkg/scheduler/queue/tree_queue_sketch.go | 250 +++-------------- pkg/scheduler/queue/tree_queue_sketch_test.go | 87 ++++-- 6 files changed, 225 insertions(+), 450 deletions(-) diff --git a/pkg/scheduler/queue/queue.go b/pkg/scheduler/queue/queue.go index 3003009d1b1..93a0fcea7fc 100644 --- a/pkg/scheduler/queue/queue.go +++ b/pkg/scheduler/queue/queue.go @@ -349,7 +349,7 @@ func (q *RequestQueue) enqueueRequestInternal(r requestToEnqueue) error { // a) a query request which was successfully dequeued for the querier, or // b) an ErrShuttingDown indicating the querier has been placed in a graceful shutdown state. func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierConn) (done bool) { - req, tenant, err := q.queueBroker.dequeueRequestForQuerier(waitingConn.querierID) + req, tenant, idx, err := q.queueBroker.dequeueRequestForQuerier(waitingConn.lastTenantIndex.last, waitingConn.querierID) if err != nil { // If this querier has told us it's shutting down, terminate WaitForRequestForQuerier with an error now... waitingConn.sendError(err) @@ -357,7 +357,7 @@ func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierC return true } - //waitingConn.lastTenantIndex.last = idx + waitingConn.lastTenantIndex.last = idx if req == nil { // Nothing available for this querier, try again next time. return false @@ -371,7 +371,7 @@ func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierC exceedsThreshold, queryComponent := q.QueryComponentUtilization.ExceedsThresholdForComponentName( queryComponentName, int(q.connectedQuerierWorkers.Load()), - q.queueBroker.tenantQueuesTree.ItemCount(), + q.queueBroker.queueTree.rootNode.ItemCount(), q.waitingQuerierConnsToDispatch.Len(), ) @@ -394,17 +394,15 @@ func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierC requestSent := waitingConn.send(reqForQuerier) if requestSent { q.queueLength.WithLabelValues(string(tenant.tenantID)).Dec() - // TODO (casie): Handle this, though I don't think we should ever have to re-enqueue to front anymore? - // Nvm we do; when queriers fail while dequeuing - //} else { - // // should never error; any item previously in the queue already passed validation - // err := q.queueBroker.enqueueRequestFront(req, tenant.maxQueriers) - // if err != nil { - // level.Error(q.log).Log( - // "msg", "failed to re-enqueue query request after dequeue", - // "err", err, "tenant", tenant.tenantID, "querier", waitingConn.querierID, - // ) - // } + } else { + // should never error; any item previously in the queue already passed validation + err := q.queueBroker.enqueueRequestFront(req, tenant.maxQueriers) + if err != nil { + level.Error(q.log).Log( + "msg", "failed to re-enqueue query request after dequeue", + "err", err, "tenant", tenant.tenantID, "querier", waitingConn.querierID, + ) + } } return true } diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index 2ecdf8676ba..da3d787de86 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -9,6 +9,7 @@ import ( "context" "errors" "fmt" + promtest "github.com/prometheus/client_golang/prometheus/testutil" "math/rand" "strconv" "strings" @@ -21,7 +22,6 @@ import ( "github.com/grafana/dskit/services" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - promtest "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "golang.org/x/sync/errgroup" @@ -110,7 +110,10 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { normalQueueDimensionFunc := func() []string { return []string{normalQueueDimension} } slowQueueDimensionFunc := func() []string { return []string{slowConsumerQueueDimension} } - additionalQueueDimensionsEnabledCases := []bool{false, true} + additionalQueueDimensionsEnabledCases := []bool{ + //false, + true, + } queueDurationTotals := map[bool]map[string]float64{ false: {normalQueueDimension: 0.0, slowConsumerQueueDimension: 0.0}, true: {normalQueueDimension: 0.0, slowConsumerQueueDimension: 0.0}, @@ -184,13 +187,18 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { for consumerIdx := 0; consumerIdx < numConsumers; consumerIdx++ { consumerIdx := consumerIdx queueConsumerErrGroup.Go(func() error { - return runConsumer(consumerIdx) + fmt.Printf("waiting for gofunc to run for consumer: %v\n", consumerIdx) + returnVal := runConsumer(consumerIdx) + fmt.Println("finished runConsumer without panicking") + return returnVal }) + fmt.Println("past first suspected panic") } close(startConsumersChan) err = queueConsumerErrGroup.Wait() require.NoError(t, err) + fmt.Println("past second suspected panic\n=============") // record total queue duration by queue dimensions and whether the queue splitting was enabled for _, queueDimension := range []string{normalQueueDimension, slowConsumerQueueDimension} { @@ -361,11 +369,13 @@ func runQueueConsumerIters( start chan struct{}, consumeFunc consumeRequest, ) func(consumerIdx int) error { - return func(consumerIdx int) error { + fmt.Println("constructing returnFunc") + returnFunc := func(consumerIdx int) error { consumerIters := queueActorIterationCount(totalIters, numConsumers, consumerIdx) lastTenantIndex := FirstTenant() querierID := fmt.Sprintf("consumer-%v", consumerIdx) queue.SubmitRegisterQuerierConnection(querierID) + fmt.Println("runQueueConsumerIters submitted register querier connection") defer queue.SubmitUnregisterQuerierConnection(querierID) <-start @@ -378,9 +388,12 @@ func runQueueConsumerIters( lastTenantIndex = idx } + fmt.Println("finished consumer iterations") return nil } + fmt.Println("finished constructing returnFunc\n=======") + return returnFunc } type consumeRequest func(request Request) error @@ -664,9 +677,9 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend req: req, } - require.Nil(t, queueBroker.tenantQueuesTree.getNode(QueuePath{"tenant-1"})) + require.Nil(t, queueBroker.queueTree.rootNode.getNode(QueuePath{"tenant-1"})) require.NoError(t, queueBroker.enqueueRequestBack(&tr, tenantMaxQueriers)) - require.False(t, queueBroker.tenantQueuesTree.getNode(QueuePath{"tenant-1"}).IsEmpty()) + require.False(t, queueBroker.queueTree.rootNode.getNode(QueuePath{"tenant-1"}).IsEmpty()) ctx, cancel := context.WithCancel(context.Background()) call := &waitingQuerierConn{ @@ -681,5 +694,5 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend // indicating not to re-submit a request for waitingQuerierConn for the querier require.True(t, queue.trySendNextRequestForQuerier(call)) // assert request was re-enqueued for tenant after failed send - require.False(t, queueBroker.tenantQueuesTree.getNode(QueuePath{"tenant-1"}).IsEmpty()) + require.False(t, queueBroker.queueTree.rootNode.getNode(QueuePath{"tenant-1"}).IsEmpty()) } diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index f672f87e9ad..5b9a092e400 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -31,11 +31,13 @@ type queueBroker struct { maxTenantQueueSize int additionalQueueDimensionsEnabled bool currentQuerier QuerierID + + treeShuffleShardState *shuffleShardState } func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled bool, forgetDelay time.Duration) *queueBroker { currentQuerier := QuerierID("") - tqas := tenantQuerierAssignments{ + tqas := &tenantQuerierAssignments{ queriersByID: map[QuerierID]*querierConn{}, querierIDsSorted: nil, querierForgetDelay: forgetDelay, @@ -43,19 +45,25 @@ func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled boo tenantsByID: map[TenantID]*queueTenant{}, tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, } + + sss := &shuffleShardState{ + tenantQuerierMap: tqas.tenantQuerierIDs, + currentQuerier: ¤tQuerier, + } // TODO (casie): Maybe set this using a flag, so we can also change how we build queue path accordingly tree := NewTree( - &shuffleShardState{tenantQuerierMap: tqas.tenantQuerierIDs, currentQuerier: ¤tQuerier}, // root + sss, // root &roundRobinState{}, // tenants &roundRobinState{}, // query components ) qb := &queueBroker{ tenantQueuesTree: NewTreeQueue("root"), queueTree: tree, - tenantQuerierAssignments: tqas, + tenantQuerierAssignments: *tqas, maxTenantQueueSize: maxTenantQueueSize, additionalQueueDimensionsEnabled: additionalQueueDimensionsEnabled, currentQuerier: currentQuerier, + treeShuffleShardState: sss, } return qb @@ -86,12 +94,6 @@ func (qb *queueBroker) enqueueRequestBack(request *tenantRequest, tenantMaxQueri } } - //if tenantQueueNode := qb.tenantQueuesTree.getNode(queuePath[:1]); tenantQueueNode != nil { - // if tenantQueueNode.ItemCount()+1 > qb.maxTenantQueueSize { - // return ErrTooManyRequests - // } - //} - //err = qb.tenantQueuesTree.EnqueueBackByPath(queuePath, request) err = qb.queueTree.rootNode.enqueueBackByPath(qb.queueTree, queuePath, request) return err @@ -112,7 +114,8 @@ func (qb *queueBroker) enqueueRequestFront(request *tenantRequest, tenantMaxQuer if err != nil { return err } - return qb.tenantQueuesTree.EnqueueFrontByPath(queuePath, request) + return qb.queueTree.rootNode.enqueueFrontByPath(qb.queueTree, queuePath, request) + //return qb.tenantQueuesTree.EnqueueFrontByPath(queuePath, request) } func (qb *queueBroker) makeQueuePath(request *tenantRequest) (QueuePath, error) { @@ -127,18 +130,18 @@ func (qb *queueBroker) makeQueuePath(request *tenantRequest) (QueuePath, error) } func (qb *queueBroker) dequeueRequestForQuerier( - //lastTenantIndex int, + lastTenantIndex int, querierID QuerierID, ) ( *tenantRequest, *queueTenant, - //int, + int, error, ) { //tenant, tenantIndex, err := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndex, querierID) // check if querier is registered and is not shutting down if q := qb.tenantQuerierAssignments.queriersByID[querierID]; q == nil || q.shuttingDown { - return nil, nil, ErrQuerierShuttingDown + return nil, nil, qb.treeShuffleShardState.sharedQueuePosition, ErrQuerierShuttingDown } //if tenant == nil || err != nil { // return nil, tenant, tenantIndex, err @@ -149,6 +152,8 @@ func (qb *queueBroker) dequeueRequestForQuerier( //queueElement := qb.tenantQueuesTree.DequeueByPath(queuePath) qb.currentQuerier = querierID + qb.treeShuffleShardState.sharedQueuePosition = lastTenantIndex + queuePath, queueElement := qb.queueTree.rootNode.dequeue() // TODO (casie): hacky - hardcoding second elt in queuePath to tenant ID @@ -172,7 +177,7 @@ func (qb *queueBroker) dequeueRequestForQuerier( } //return request, tenant, tenantIndex, nil - return request, tenant, nil + return request, tenant, qb.treeShuffleShardState.sharedQueuePosition, nil } func (qb *queueBroker) addQuerierConnection(querierID QuerierID) (resharded bool) { diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 14389454499..020df066dde 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -6,7 +6,6 @@ package queue import ( - "container/list" "context" "fmt" "github.com/grafana/dskit/httpgrpc" @@ -21,7 +20,8 @@ import ( // TODO (casie): Add a test case where, if a tenant is queued in the queueTree, but doesn't exist from the queueBroker perspective, // it should never be dequeued. -// todo (casie): maybe implement a way to fetch an entire node that _would_ be dequeued from, rather than just one elt +// TODO (casie): maybe implement a way to fetch an entire node that _would_ be dequeued from, rather than just one elt + func (qb *queueBroker) enqueueObjectsForTests(tenantID TenantID, numObjects int) error { for i := 0; i < numObjects; i++ { err := qb.queueTree.rootNode.enqueueBackByPath(qb.queueTree, QueuePath{string(tenantID)}, &tenantRequest{ @@ -42,6 +42,25 @@ func buildExpectedObject(tenantID TenantID, num int) *tenantRequest { } } +type dequeueVal struct { + req *tenantRequest + tenant *queueTenant +} + +func assertExpectedValuesOnDequeue(t *testing.T, qb *queueBroker, lastTenantIndex int, querierID QuerierID, expectedVals []dequeueVal) int { + var req *tenantRequest + var tenant *queueTenant + var err error + + for _, expected := range expectedVals { + req, tenant, lastTenantIndex, err = qb.dequeueRequestForQuerier(lastTenantIndex, querierID) + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } + return lastTenantIndex +} + func TestQueues(t *testing.T) { qb := newQueueBroker(0, true, 0) assert.NotNil(t, qb) @@ -50,12 +69,7 @@ func TestQueues(t *testing.T) { qb.addQuerierConnection("querier-1") qb.addQuerierConnection("querier-2") - type dequeueVal struct { - req *tenantRequest - tenant *queueTenant - } - - req, tenant, err := qb.dequeueRequestForQuerier("querier-1") + req, tenant, lastTenantIndex, err := qb.dequeueRequestForQuerier(-1, "querier-1") assert.Nil(t, req) assert.Nil(t, tenant) assert.NoError(t, err) @@ -76,14 +90,7 @@ func TestQueues(t *testing.T) { {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, } - - for _, expected := range expectedDequeueVals { - req, tenant, err = qb.dequeueRequestForQuerier("querier-1") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - - } + lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) // Add tenant two err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) @@ -102,33 +109,16 @@ func TestQueues(t *testing.T) { {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, } - for _, expected := range expectedDequeueVals { - req, tenant, err = qb.dequeueRequestForQuerier("querier-1") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - } + lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) - // TODO (casie): This block relies on being able to pass a lastTenantIndex; I would like to get rid of this concept - // and allow the tree to manage tenant dequeuing entirely. Do we still want this behavior in that case? - // ====== - //confirmOrderForQuerier(t, qb, "querier-2", -1, qOne, qTwo, qOne) - - //expectedDequeueVals = []dequeueVal{ - // {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, - // {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, - // {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, - //} - //for _, expected := range expectedDequeueVals { - // req, tenant, err = qb.dequeueRequestForQuerier("querier-2") - // assert.Equal(t, expected.req, req) - // assert.Equal(t, expected.tenant, tenant) - // assert.NoError(t, err) - //} - // - // [one two three] - // ======== + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, + } + lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) + //[one two three] // confirm fifo by adding a third tenant queue and iterating to it err = qb.tenantQuerierAssignments.createOrUpdateTenant("three", 0) assert.NoError(t, err) @@ -140,34 +130,23 @@ func TestQueues(t *testing.T) { assert.NoError(t, isConsistent(qb)) expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, + {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, {buildExpectedObject(tenantThree.tenantID, 0), tenantThree}, - {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, + {buildExpectedObject(tenantOne.tenantID, 6), tenantOne}, } - for _, expected := range expectedDequeueVals { - req, tenant, err = qb.dequeueRequestForQuerier("querier-1") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - } + lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) - // Remove one: [two three] + // Remove one: ["" two three] qb.removeTenantQueue("one") assert.NoError(t, isConsistent(qb)) expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, - {buildExpectedObject(tenantThree.tenantID, 1), tenantThree}, {buildExpectedObject(tenantTwo.tenantID, 4), tenantTwo}, + {buildExpectedObject(tenantThree.tenantID, 1), tenantThree}, + {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, } - - for _, expected := range expectedDequeueVals { - req, tenant, err = qb.dequeueRequestForQuerier("querier-1") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - } + lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) // "four" is added at the beginning of the list: [four two three] err = qb.tenantQuerierAssignments.createOrUpdateTenant("four", 0) @@ -182,35 +161,23 @@ func TestQueues(t *testing.T) { expectedDequeueVals = []dequeueVal{ {buildExpectedObject(tenantThree.tenantID, 2), tenantThree}, {buildExpectedObject(tenantFour.tenantID, 0), tenantFour}, - {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, + {buildExpectedObject(tenantTwo.tenantID, 6), tenantTwo}, {buildExpectedObject(tenantThree.tenantID, 3), tenantThree}, } + lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) - for _, expected := range expectedDequeueVals { - req, tenant, err = qb.dequeueRequestForQuerier("querier-1") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - } - - // Remove two: [four three] + // Remove two: [four "" three] qb.removeTenantQueue("two") assert.NoError(t, isConsistent(qb)) - // + expectedDequeueVals = []dequeueVal{ {buildExpectedObject(tenantFour.tenantID, 1), tenantFour}, {buildExpectedObject(tenantThree.tenantID, 4), tenantThree}, {buildExpectedObject(tenantFour.tenantID, 2), tenantFour}, } + lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) - for _, expected := range expectedDequeueVals { - req, tenant, err = qb.dequeueRequestForQuerier("querier-1") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - } - - // Remove three: [four] + // Remove three: [four ""] ) qb.removeTenantQueue("three") assert.NoError(t, isConsistent(qb)) @@ -218,7 +185,7 @@ func TestQueues(t *testing.T) { qb.removeTenantQueue("four") assert.NoError(t, isConsistent(qb)) - req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + req, tenant, _, err = qb.dequeueRequestForQuerier(lastTenantIndex, "querier-1") assert.Nil(t, req) assert.Nil(t, tenant) assert.NoError(t, err) @@ -272,7 +239,7 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { // dequeue a request qb.addQuerierConnection("querier-1") - dequeuedTenantReq, _, err := qb.dequeueRequestForQuerier("querier-1") + dequeuedTenantReq, _, _, err := qb.dequeueRequestForQuerier(-1, "querier-1") assert.NoError(t, err) assert.NotNil(t, dequeuedTenantReq) @@ -294,7 +261,7 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { qb.addQuerierConnection("querier-1") qb.addQuerierConnection("querier-2") - // Add queues: [one, two] + // Add queues: [one two] err := qb.tenantQuerierAssignments.createOrUpdateTenant("one", 0) assert.NoError(t, err) err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) @@ -308,23 +275,13 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { assert.NoError(t, err) tenantTwo := qb.tenantQuerierAssignments.tenantsByID["two"] - type dequeueVal struct { - req *tenantRequest - tenant *queueTenant - } expectedDequeueVals := []dequeueVal{ {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, {buildExpectedObject(tenantTwo.tenantID, 0), tenantTwo}, {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, {buildExpectedObject(tenantTwo.tenantID, 1), tenantTwo}, } - - for _, expected := range expectedDequeueVals { - req, tenant, err := qb.dequeueRequestForQuerier("querier-1") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - } + qOneLastTenantIndex := assertExpectedValuesOnDequeue(t, qb, -1, "querier-1", expectedDequeueVals) // TODO (casie): This is consistent with the previous test only because the previous test // checked n % numTenants == 0 dequeues. @@ -334,17 +291,11 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, } - - for _, expected := range expectedDequeueVals { - req, tenant, err := qb.dequeueRequestForQuerier("querier-2") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - } + qTwolastTenantIndex := assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) // After notify shutdown for querier-2, it's expected to own no queue. qb.notifyQuerierShutdown("querier-2") - req, tenant, err := qb.dequeueRequestForQuerier("querier-2") + req, tenant, qTwolastTenantIndex, err := qb.dequeueRequestForQuerier(qTwolastTenantIndex, "querier-2") assert.Nil(t, req) assert.Nil(t, tenant) assert.Equal(t, ErrQuerierShuttingDown, err) @@ -358,7 +309,7 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { } for _, expected := range expectedDequeueVals { - req, tenant, err = qb.dequeueRequestForQuerier("querier-1") + req, tenant, qOneLastTenantIndex, err = qb.dequeueRequestForQuerier(qOneLastTenantIndex, "querier-1") assert.Equal(t, expected.req, req) assert.Equal(t, expected.tenant, tenant) assert.NoError(t, err) @@ -366,7 +317,7 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { // After disconnecting querier-2, it's expected to own no queue. qb.tenantQuerierAssignments.removeQuerier("querier-2") - req, tenant, err = qb.dequeueRequestForQuerier("querier-2") + req, tenant, qTwolastTenantIndex, err = qb.dequeueRequestForQuerier(qTwolastTenantIndex, "querier-2") assert.Nil(t, req) assert.Nil(t, tenant) assert.Equal(t, ErrQuerierShuttingDown, err) @@ -388,7 +339,7 @@ func TestQueuesWithQueriers(t *testing.T) { qb.addQuerierConnection(qid) // No querier has any queues yet. - req, tenant, err := qb.dequeueRequestForQuerier(qid) + req, tenant, _, err := qb.dequeueRequestForQuerier(-1, qid) assert.Nil(t, req) assert.Nil(t, tenant) assert.NoError(t, err) @@ -487,7 +438,7 @@ func TestQueuesConsistency(t *testing.T) { qb.notifyQuerierShutdown(q) } - assert.NoErrorf(t, isConsistent(qb), "last action %d", i) + assert.NoErrorf(t, isConsistent(qb), "last action %d, rand: %d", i, r.Int()%6) } }) } @@ -668,7 +619,6 @@ func TestQueues_ForgetDelay_ShouldCorrectlyHandleQuerierReconnectingBeforeForget assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) } } - func generateTenant(r *rand.Rand) TenantID { return TenantID(fmt.Sprint("tenant-", r.Int()%5)) } @@ -677,48 +627,6 @@ func generateQuerier(r *rand.Rand) QuerierID { return QuerierID(fmt.Sprint("querier-", r.Int()%5)) } -// getOrAddTenantQueue is a test utility, not intended for use by consumers of queueBroker -func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) (*TreeQueue, error) { - err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriers) - if err != nil { - return nil, err - } - - queuePath := QueuePath{string(tenantID)} - return qb.tenantQueuesTree.getOrAddNode(queuePath) -} - -// getQueue is a test utility, not intended for use by consumers of queueBroker -func (qb *queueBroker) getQueue(tenantID TenantID) *TreeQueue { - tenant, err := qb.tenantQuerierAssignments.getTenant(tenantID) - if tenant == nil || err != nil { - return nil - } - - queuePath := QueuePath{string(tenantID)} - tenantQueue := qb.tenantQueuesTree.getNode(queuePath) - return tenantQueue -} - -// removeTenantQueue is a test utility, not intended for use by consumers of queueBroker -func (qb *queueBroker) removeTenantQueue(tenantID TenantID) bool { - qb.tenantQuerierAssignments.removeTenant(tenantID) - queuePath := QueuePath{string(tenantID)} - return qb.tenantQueuesTree.deleteNode(queuePath) -} - -func confirmOrderForQuerier(t *testing.T, qb *queueBroker, querier QuerierID, lastTenantIndex int, queues ...*list.List) int { - for _, queue := range queues { - var err error - tenant, _, err := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndex, querier) - tenantQueue := qb.getQueue(tenant.tenantID) - assert.Equal(t, queue, tenantQueue.localQueue) - assert.NoError(t, isConsistent(qb)) - assert.NoError(t, err) - } - return lastTenantIndex -} - // getTenantsByQuerier returns the list of tenants handled by the provided QuerierID. func getTenantsByQuerier(broker *queueBroker, querierID QuerierID) []TenantID { var tenantIDs []TenantID @@ -736,28 +644,6 @@ func getTenantsByQuerier(broker *queueBroker, querierID QuerierID) []TenantID { return tenantIDs } -//// Used in test to get a pointer to the local queue that we want to check against -//func getOrAdd(t *testing.T, qb *queueBroker, tenantID TenantID, maxQueriers int) error { -// err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriers) -// if err != nil { -// return err -// } -// //assert.NoError(t, err) -// //assert.NoError(t, isConsistent(qb)) -// -// node, err := qb.getOrAddTenantQueue(tenantID) -// if err != nil { -// return err -// } -// //assert.NoError(t, err) -// //assert.NotNil(t, node) -// -// reAddedNode, err := qb.getOrAddTenantQueue(tenantID) -// assert.Nil(t, err) -// assert.Equal(t, node, reAddedNode) -// //return node.localQueue -//} - // getOrAddTenantQueue is a test utility, not intended for use by consumers of queueBroker func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) (*Node, error) { err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriers) @@ -780,6 +666,28 @@ func (qb *queueBroker) removeTenantQueue(tenantID TenantID) bool { return qb.queueTree.rootNode.deleteNode(queuePath) } +func (n *Node) deleteNode(pathFromNode QueuePath) bool { + if len(pathFromNode) == 0 { + // node cannot delete itself + return false + } + + // parentPath is everything except the last node, childNode is the last in path + parentPath, deleteNodeName := pathFromNode[:len(pathFromNode)-1], pathFromNode[len(pathFromNode)-1] + + parentNode := n.getNode(parentPath) + if parentNode == nil { + // not found + return false + } + if deleteNode := parentNode.getNode(QueuePath{deleteNodeName}); deleteNode != nil { + childDeleted := parentNode.dequeueAlgorithm.deleteChildNode(parentNode, deleteNode) + parentNode.dequeueAlgorithm.dequeueOps().updateState(parentNode, nil, childDeleted) + return true + } + return false +} + func (qb *queueBroker) makeQueuePathForTests(tenantID TenantID) QueuePath { return QueuePath{string(tenantID)} } @@ -806,7 +714,10 @@ func isConsistent(qb *queueBroker) error { if tenantID == "" { continue } - tenantCount++ + + if tenantID != "" { + tenantCount++ + } tenant := qb.tenantQuerierAssignments.tenantsByID[tenantID] querierSet := qb.tenantQuerierAssignments.tenantQuerierIDs[tenantID] @@ -830,6 +741,9 @@ func isConsistent(qb *queueBroker) error { tenantQueueCount := qb.queueTree.rootNode.nodeCount() - 1 if tenantQueueCount != tenantCount { + //fmt.Printf("tenantIDOrder: %v\n", qb.tenantQuerierAssignments.tenantIDOrder) + //fmt.Printf("rootNode children: %v\n", qb.queueTree.rootNode.dequeueAlgorithm.getQueueOrder()) + fmt.Println("tenant count check: ", tenantQueueCount, tenantCount) return fmt.Errorf("inconsistent number of tenants list and tenant queues") } diff --git a/pkg/scheduler/queue/tree_queue_sketch.go b/pkg/scheduler/queue/tree_queue_sketch.go index f20df6f6dc2..a734071c16d 100644 --- a/pkg/scheduler/queue/tree_queue_sketch.go +++ b/pkg/scheduler/queue/tree_queue_sketch.go @@ -16,28 +16,13 @@ type TreeNodeIFace interface { getOrAddNode(QueuePath, *Tree) (*Node, error) } -type DequeueAlgorithm interface { - dequeueOps() dequeueOps -} - -type OpsPath []struct { - name string - dequeueAlgorithm DequeueAlgorithm -} - -type dequeueGetNodeFunc func(*Node) (TreeNodeIFace, bool) -type dequeueUpdateStateFunc func(*Node, any, string, bool) - -type dequeueOps struct { - getNode dequeueGetNodeFunc - updateState dequeueUpdateStateFunc -} - type Tree struct { rootNode *Node algosByDepth []DequeueAlgorithm } +// Nodes with *shuffleShardState dequeueAlgorithms disregard individual +// Nodes' queueOrder and queuePosition // TODO (casie): Implement maxQueueSize to mirror qb.maxTenantQueueSize? type Node struct { name string @@ -47,9 +32,9 @@ type Node struct { queueMap map[string]*Node depth int dequeueAlgorithm DequeueAlgorithm + childrenChecked int } -// TODO (casie): Instead of passing in actual state objects, just pass a string that maps to state func NewTree(dequeueAlgorithms ...DequeueAlgorithm) *Tree { if len(dequeueAlgorithms) == 0 { return nil @@ -73,15 +58,20 @@ func NewNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { } switch da.(type) { case *shuffleShardState: - if da.(*shuffleShardState).tenantQuerierMap == nil { - return nil, fmt.Errorf("cannot create a shufffle-shard node with nil tenant-querier map") + sss := da.(*shuffleShardState) + sss.sharedQueuePosition = localQueueIndex - 1 + if sss.tenantNodes == nil { + sss.tenantNodes = map[string][]*Node{} + } + if sss.tenantQuerierMap == nil { + sss.tenantQuerierMap = map[TenantID]map[QuerierID]struct{}{} } } return &Node{ - name: name, - localQueue: list.New(), - queuePosition: localQueueIndex - 1, // start from -2 so that on first dequeue, localQueueIndex is dequeued from - queueOrder: make([]string, 0), + name: name, + localQueue: list.New(), + queuePosition: localQueueIndex - 1, // start from -2 so that on first dequeue, localQueueIndex is dequeued from + //queueOrder: make([]string, 0), queueMap: make(map[string]*Node, 1), depth: depth, dequeueAlgorithm: da, @@ -109,6 +99,15 @@ func (n *Node) getLocalQueue() *list.List { return n.localQueue } +func (n *Node) enqueueFrontByPath(tree *Tree, pathFromNode QueuePath, v any) error { + childNode, err := n.getOrAddNode(pathFromNode, tree) + if err != nil { + return err + } + childNode.getLocalQueue().PushFront(v) + return nil +} + func (n *Node) enqueueBackByPath(tree *Tree, pathFromNode QueuePath, v any) error { childNode, err := n.getOrAddNode(pathFromNode, tree) if err != nil { @@ -130,7 +129,7 @@ func (n *Node) dequeue() (QueuePath, any) { } var checkedAllNodes bool - var dequeueNode TreeNodeIFace + var dequeueNode *Node // continue until we've found a value or checked all nodes that need checking for v == nil && !checkedAllNodes { dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueOps().getNode(n) @@ -145,26 +144,22 @@ func (n *Node) dequeue() (QueuePath, any) { } } } else if dequeueNode == nil { - // no dequeue-able node found + // no dequeue-able child found; reset checked children to 0, + // as we won't update state before moving on + n.childrenChecked = 0 return path, v } else { // dequeue from a child childPath, v = dequeueNode.dequeue() // if the dequeue node is empty _after_ dequeueing, delete it from children if dequeueNode.IsEmpty() { - deleteNodeName := dequeueNode.Name() - delete(n.queueMap, deleteNodeName) - for idx, name := range n.queueOrder { - if name == deleteNodeName { - n.queueOrder = append(n.queueOrder[:idx], n.queueOrder[idx+1:]...) - } - } // removing an element sets our position one step forward; // tell state to reset it to original queuePosition - deletedNode = true + delete(n.queueMap, dequeueNode.Name()) + deletedNode = n.dequeueAlgorithm.deleteChildNode(n, dequeueNode) } } - n.dequeueAlgorithm.dequeueOps().updateState(n, v, dequeueNode.Name(), deletedNode) + n.dequeueAlgorithm.dequeueOps().updateState(n, v, deletedNode) } return append(path, childPath...), v } @@ -186,35 +181,6 @@ func (n *Node) getNode(pathFromNode QueuePath) *Node { return nil } -func (n *Node) deleteNode(pathFromNode QueuePath) bool { - if len(pathFromNode) == 0 { - // node cannot delete itself - return false - } - - // parentPath is everything except the last node, childNode is the last in path - parentPath, deleteNodeName := pathFromNode[:len(pathFromNode)-1], pathFromNode[len(pathFromNode)-1] - - parentNode := n.getNode(parentPath) - if parentNode == nil { - // not found - return false - } - - delete(parentNode.queueMap, deleteNodeName) - for i, name := range parentNode.queueOrder { - if name == deleteNodeName { - parentNode.queueOrder = append(n.queueOrder[:i], n.queueOrder[i+1:]...) - if parentNode.queuePosition >= len(parentNode.queueOrder) { - // set _last_ dequeue position to last elt in order - parentNode.queuePosition = len(parentNode.queueOrder) - 1 - } - break - } - } - return true -} - // getOrAddNode checks whether the first node name in pathFromNode exists in a node's children; // if no node exists, one is created and added to the "end" of the node's children, from the perspective // of node.queuePosition (e.g., if node.queuePosition is 4, the new node will be created at position 3). @@ -238,163 +204,9 @@ func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *Tree) (*Node, error) { return nil, err } - // add childNode to n.queueMap - n.queueMap[childNode.Name()] = childNode - - // add childNode to n.queueOrder before the current position, update n.queuePosition to current element - if n.queuePosition <= localQueueIndex { - n.queueOrder = append(n.queueOrder, childNode.Name()) - } else { - n.queueOrder = append(n.queueOrder[:n.queuePosition], append([]string{childNode.Name()}, n.queueOrder[n.queuePosition:]...)...) - n.queuePosition++ // keep position pointed at same element - } + n.dequeueAlgorithm.addChildNode(n, childNode) } // TODO (casie): How to determine what dequeueAlgo to pass? return childNode.getOrAddNode(pathFromNode[1:], tree) } - -type roundRobinState struct { - subtreeNodesChecked int -} - -func (rrs *roundRobinState) dequeueOps() dequeueOps { - getNode := func(n *Node) (TreeNodeIFace, bool) { - // advance the queue position for this dequeue - n.queuePosition++ - if n.queuePosition >= len(n.queueOrder) { - n.queuePosition = localQueueIndex - } - - checkedAllNodes := rrs.subtreeNodesChecked == len(n.queueOrder)+1 - if n.queuePosition == localQueueIndex { - return n, checkedAllNodes - } - - currentNodeName := n.queueOrder[n.queuePosition] - if node, ok := n.queueMap[currentNodeName]; ok { - return node, checkedAllNodes - } - return nil, checkedAllNodes - } - - // TODO (casie): Ignoring nodeName here because it doesn't matter...ugly - updateState := func(n *Node, v any, _ string, deletedNode bool) { - if v != nil { - rrs.subtreeNodesChecked = 0 - } else { - rrs.subtreeNodesChecked++ - } - if deletedNode { - n.queuePosition-- - // if we try to go beyond something that would increment to - // the localQueueIndex or in queueOrder, we should wrap around to the - // back of queueOrder. - if n.queuePosition < localQueueIndex-1 { - n.queuePosition = len(n.queueOrder) - 1 - } - } - } - - return dequeueOps{ - getNode: getNode, - updateState: updateState, - } -} - -type shuffleShardState struct { - tenantQuerierMap map[TenantID]map[QuerierID]struct{} - currentQuerier *QuerierID - nodesChecked int -} - -func (sss *shuffleShardState) dequeueOps() dequeueOps { - // start from ssn.queuePosition - // check that tenant for querierID against availabilityMap - // if exists, move element to "back" of queue - // if doesn't exist, check next child - // nothing here to dequeue - getNode := func(n *Node) (TreeNodeIFace, bool) { - // can't get a tenant if no querier set - if sss.currentQuerier == nil { - return nil, true - } - - checkedAllNodes := sss.nodesChecked == len(n.queueOrder)+1 // must check local queue as well - - // advance queue position for dequeue - n.queuePosition++ - if n.queuePosition >= len(n.queueOrder) { - n.queuePosition = 0 - } - - // no children - if len(n.queueOrder) == 0 || n.queuePosition == localQueueIndex { - return n, checkedAllNodes - } - - checkIndex := n.queuePosition - - for iters := 0; iters < len(n.queueOrder); iters++ { - if checkIndex >= len(n.queueOrder) { - checkIndex = 0 - } - tenantName := n.queueOrder[checkIndex] - // increment nodes checked even if not in tenant-querier map - sss.nodesChecked++ - checkedAllNodes = sss.nodesChecked == len(n.queueOrder) - // if the tenant-querier set is nil, any querier can serve this tenant - if sss.tenantQuerierMap[TenantID(tenantName)] == nil { - return n.queueMap[tenantName], checkedAllNodes - } - if tenantQuerierSet, ok := sss.tenantQuerierMap[TenantID(tenantName)]; ok { - if _, ok := tenantQuerierSet[*sss.currentQuerier]; ok { - return n.queueMap[tenantName], checkedAllNodes - } - } - checkIndex++ - } - return nil, checkedAllNodes - } - - updateState := func(n *Node, v any, nodeName string, deletedNode bool) { - if deletedNode { - // when we delete a child node, we want to step back in the queue so the - // previously-next tenant will still be next (e.g., if queueOrder is - // [childA, childB, childC] and childB (position 1) is deleted, we want to - // step back to position 0 so that in the next dequeue, position 1 (childC) - // is dequeued - n.queuePosition-- - } - if v != nil { - sss.nodesChecked = 0 - return - } - - sss.nodesChecked++ - - // if dequeueNode was self, advance queue position no matter what, and return - if n.queuePosition == localQueueIndex { - return - } - - // otherwise, don't change queuePosition, only move element to back - for i, child := range n.queueOrder { - if child == nodeName { - n.queueOrder = append(n.queueOrder[:i], append(n.queueOrder[i+1:], child)...) // ugly - n.queuePosition-- - // wrap around to back of order if we go past the first element in n.queueOrder. - // In the next dequeue, we will step forward again. - if n.queuePosition <= localQueueIndex { - n.queuePosition = len(n.queueOrder) - 1 - } - } - } - } - - return dequeueOps{ - getNode: getNode, - updateState: updateState, - } - -} diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go index 9aa627ec7af..e129768bf52 100644 --- a/pkg/scheduler/queue/tree_queue_sketch_test.go +++ b/pkg/scheduler/queue/tree_queue_sketch_test.go @@ -10,11 +10,17 @@ import ( // TODO (casie): Write tests for NewTree // TODO (casie): Write a test for dequeuing from sss childA, enqueue to new childB, expect to dequeue next from childB +func newShuffleShardState() *shuffleShardState { + return &shuffleShardState{ + tenantQuerierMap: make(map[TenantID]map[QuerierID]struct{}), + tenantNodes: make(map[string][]*Node), + } +} + func Test_NewNode(t *testing.T) { tests := []struct { name string rootAlgo DequeueAlgorithm - childPath OpsPath state *shuffleShardState expectErr bool }{ @@ -24,12 +30,11 @@ func Test_NewNode(t *testing.T) { }, { name: "create shuffle-shard node", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + rootAlgo: newShuffleShardState(), }, { - name: "create shuffle-shard tree with no tenant-querier map", - rootAlgo: &shuffleShardState{}, - expectErr: true, + name: "create shuffle-shard tree with no tenant-querier map, we should create an empty one", + rootAlgo: &shuffleShardState{}, }, } for _, tt := range tests { @@ -64,15 +69,14 @@ func Test_EnqueueBackByPath(t *testing.T) { children: []QueuePath{{"shuffle-shard-child-1"}}, }, { - name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node", + name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node, we should create an empty one", treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &shuffleShardState{}}, children: []QueuePath{{"shuffle-shard-child-1"}}, - expectErr: true, }, { name: "enqueue round-robin node to shuffle-shard node", treeAlgosByDepth: []DequeueAlgorithm{ - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + newShuffleShardState(), &roundRobinState{}, }, children: []QueuePath{{"round-robin-child-1"}}, @@ -80,9 +84,9 @@ func Test_EnqueueBackByPath(t *testing.T) { { name: "create tree with multiple shuffle-shard depths", treeAlgosByDepth: []DequeueAlgorithm{ - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + newShuffleShardState(), &roundRobinState{}, - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + newShuffleShardState(), }, children: []QueuePath{{"child"}, {"grandchild"}}, }, @@ -123,7 +127,7 @@ func Test_Dequeue_RootNode(t *testing.T) { }, { name: "dequeue from empty shuffle-shard root node", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + rootAlgo: newShuffleShardState(), }, { name: "dequeue from non-empty round-robin root node", @@ -132,7 +136,7 @@ func Test_Dequeue_RootNode(t *testing.T) { }, { name: "dequeue from non-empty shuffle-shard root node", - rootAlgo: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + rootAlgo: newShuffleShardState(), enqueueToRoot: []any{"something-else-in-root"}, }, } @@ -282,8 +286,16 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { { name: "should dequeue from new (next-in-queue) shuffle-shard child immediately after it is added", treeAlgosByDepth: []DequeueAlgorithm{ - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, currentQuerier: &placeholderQuerier}, - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, currentQuerier: &placeholderQuerier}, + &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, + tenantNodes: map[string][]*Node{}, + currentQuerier: &placeholderQuerier, + }, + &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, + tenantNodes: map[string][]*Node{}, + currentQuerier: &placeholderQuerier, + }, }, operationOrder: []op{ {enqueue, QueuePath{"child-1"}, "obj-1"}, @@ -336,11 +348,12 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "happy path - tenant found in tenant-querier map under first child", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + newShuffleShardState(), &roundRobinState{}, }, state: &shuffleShardState{ tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{"querier-1"}, enqueueObjs: []enqueueObj{ @@ -352,11 +365,12 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "tenant exists, but not for querier", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + newShuffleShardState(), &roundRobinState{}, }, state: &shuffleShardState{ tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, + tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{"querier-1"}, enqueueObjs: []enqueueObj{ @@ -368,11 +382,12 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "1 of 3 tenants exist for querier", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + newShuffleShardState(), &roundRobinState{}, }, state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + }, currQuerier: []QuerierID{"querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, @@ -385,11 +400,12 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "tenant exists for querier on next parent node", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + newShuffleShardState(), &roundRobinState{}, }, state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + }, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, @@ -403,10 +419,13 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "2 of 3 tenants exist for querier", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + newShuffleShardState(), &roundRobinState{}, }, - state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, @@ -422,7 +441,10 @@ func Test_ShuffleShardDequeue(t *testing.T) { &roundRobinState{}, &roundRobinState{}, }, - state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}, + tenantNodes: map[string][]*Node{}, + }, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ {obj: "query-1", path: QueuePath{"tenant-2", "query-component-1"}}, @@ -438,7 +460,10 @@ func Test_ShuffleShardDequeue(t *testing.T) { &roundRobinState{}, &roundRobinState{}, }, - state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, currQuerier: []QuerierID{"querier-2", "querier-1"}, enqueueObjs: []enqueueObj{ {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, @@ -452,7 +477,10 @@ func Test_ShuffleShardDequeue(t *testing.T) { &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, }, - state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, currQuerier: []QuerierID{""}, enqueueObjs: []enqueueObj{ {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, @@ -467,7 +495,10 @@ func Test_ShuffleShardDequeue(t *testing.T) { &roundRobinState{}, &roundRobinState{}, }, - state: &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}}, + state: &shuffleShardState{ + tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}, + tenantNodes: map[string][]*Node{}, + }, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, @@ -488,7 +519,9 @@ func Test_ShuffleShardDequeue(t *testing.T) { for i, da := range tt.treeAlgosByDepth { switch da.(type) { case *shuffleShardState: - tt.treeAlgosByDepth[i] = tt.state + sss := tt.treeAlgosByDepth[i].(*shuffleShardState) + sss.tenantQuerierMap = tt.state.tenantQuerierMap + sss.currentQuerier = tt.state.currentQuerier } } From 10fcce37a0de1fa759e800a53a0175799cdbabff Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 3 Jun 2024 14:18:42 -0700 Subject: [PATCH 08/26] actually add queuing algorithms --- .../queue/tree_queueing_algorithms.go | 279 ++++++++++++++++++ 1 file changed, 279 insertions(+) create mode 100644 pkg/scheduler/queue/tree_queueing_algorithms.go diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go new file mode 100644 index 00000000000..758ba4ae9a2 --- /dev/null +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -0,0 +1,279 @@ +package queue + +import "fmt" + +type DequeueAlgorithm interface { + addChildNode(*Node, *Node) + deleteChildNode(*Node, *Node) bool + dequeueOps() dequeueOps + getQueueOrder(*Node) []string +} + +type dequeueGetNodeFunc func(*Node) (*Node, bool) +type dequeueUpdateStateFunc func(*Node, any, bool) + +type dequeueOps struct { + getNode dequeueGetNodeFunc + updateState dequeueUpdateStateFunc +} + +type roundRobinState struct { +} + +func (rrs *roundRobinState) addChildNode(parent, child *Node) { + // add childNode to n.queueMap + parent.queueMap[child.Name()] = child + + // add childNode to n.queueOrder before the current position, update n.queuePosition to current element + if parent.queuePosition <= localQueueIndex { + parent.queueOrder = append(parent.queueOrder, child.Name()) + } else { + parent.queueOrder = append(parent.queueOrder[:parent.queuePosition], append([]string{child.Name()}, parent.queueOrder[parent.queuePosition:]...)...) + parent.queuePosition++ // keep position pointed at same element + } +} + +func (rrs *roundRobinState) deleteChildNode(parent, child *Node) bool { + var childFound bool + childName := child.Name() + + for idx, name := range parent.queueOrder { + if name == childName { + parent.queueOrder = append(parent.queueOrder[:idx], parent.queueOrder[idx+1:]...) + childFound = true + } + } + return childFound +} + +func (rrs *roundRobinState) dequeueOps() dequeueOps { + getNode := func(n *Node) (*Node, bool) { + // advance the queue position for this dequeue + n.queuePosition++ + if n.queuePosition >= len(n.queueOrder) { + n.queuePosition = localQueueIndex + } + + checkedAllNodes := n.childrenChecked == len(n.queueOrder)+1 + if n.queuePosition == localQueueIndex { + return n, checkedAllNodes + } + + currentNodeName := n.queueOrder[n.queuePosition] + if node, ok := n.queueMap[currentNodeName]; ok { + return node, checkedAllNodes + } + return nil, checkedAllNodes + } + + // TODO (casie): Ignoring nodeName here because it doesn't matter...ugly + updateState := func(n *Node, v any, deletedNode bool) { + if v != nil { + n.childrenChecked = 0 + } else { + n.childrenChecked++ + } + if deletedNode { + n.queuePosition-- + // if we try to go beyond something that would increment to + // the localQueueIndex or in queueOrder, we should wrap around to the + // back of queueOrder. + if n.queuePosition < localQueueIndex-1 { + n.queuePosition = len(n.queueOrder) - 1 + } + } + } + + return dequeueOps{ + getNode: getNode, + updateState: updateState, + } +} + +func (rrs *roundRobinState) getQueueOrder(n *Node) []string { + return n.queueOrder +} + +type shuffleShardState struct { + tenantQuerierMap map[TenantID]map[QuerierID]struct{} + tenantNodes map[string][]*Node + currentQuerier *QuerierID + sharedQueueOrder []string + sharedQueuePosition int +} + +func (sss *shuffleShardState) addChildNode(parent, child *Node) { + childName := child.Name() + _, childInOrder := sss.tenantNodes[childName] + + // add childNode to node's queueMap, + // and to the shared tenantNodes map + parent.queueMap[childName] = child + sss.tenantNodes[childName] = append(sss.tenantNodes[childName], child) + + // if child already exists in shared queueOrder, return without altering order + if childInOrder { + return + } + + // otherwise, add childNode to n.queueOrder before the current position, and + // update n.queuePosition to current element + // TODO (casie): Mirrors tenantQuerierAssignments.createOrUpdateTenant; move logic here? + for i, elt := range sss.sharedQueueOrder { + if elt == "" { + sss.sharedQueueOrder[i] = childName + return + } + } + // if we get here, we didn't find any empty elements in sharedQueueOrder; append + sss.sharedQueueOrder = append(sss.sharedQueueOrder, childName) + +} + +func (sss *shuffleShardState) deleteChildNode(parent, child *Node) bool { + childName := child.Name() + + // delete from parent's children + delete(parent.queueMap, childName) + + // delete from shared tenantNodes + for i, tenantNode := range sss.tenantNodes[childName] { + if tenantNode == child { + sss.tenantNodes[childName] = append(sss.tenantNodes[childName][:i], sss.tenantNodes[childName][i+1:]...) + } + } + + // check tenantNodes; to mirror tenantQuerierAssignments, we only remove from sharedQueueOrder + // if all nodes with this name are gone. If the child is at the _end_ of sharedQueueOrder, we + // remove it outright; otherwise, we replace it with an empty string so as not to reindex all + // slice elements + // TODO (casie): Mirrors tenantQuerierAssignments.removeTenant. Pull logic here? + + removeFromSharedQueueOrder := len(sss.tenantNodes[childName]) == 0 + + var positionNeedsUpdate bool + if removeFromSharedQueueOrder { + for idx, name := range sss.sharedQueueOrder { + if name == childName { + sss.sharedQueueOrder[idx] = "" + } + } + // clear all sequential empty elts from sharedQueueOrder + lastElementIndex := len(sss.sharedQueueOrder) - 1 + for i := lastElementIndex; i >= 0 && sss.sharedQueueOrder[i] == ""; i-- { + sss.sharedQueueOrder = sss.sharedQueueOrder[:i] + } + } + return positionNeedsUpdate +} + +func (sss *shuffleShardState) dequeueOps() dequeueOps { + // start from sss.sharedQueuePosition + // check next tenant for querierID against availabilityMap + // if exists, move element to "back" of queue + // if doesn't exist, check next child + // nothing here to dequeue + getNode := func(n *Node) (*Node, bool) { + // can't get a tenant if no querier set + if sss.currentQuerier == nil { + return nil, true + } + + checkedAllNodes := n.childrenChecked == len(n.queueMap)+1 // must check local queue as well + + // advance queue position for dequeue + sss.sharedQueuePosition++ + if sss.sharedQueuePosition >= len(sss.sharedQueueOrder) { + sss.sharedQueuePosition = localQueueIndex + } + + // no children + if len(n.queueMap) == 0 || sss.sharedQueuePosition == localQueueIndex { + return n, checkedAllNodes + } + + checkIndex := sss.sharedQueuePosition + + for iters := 0; iters < len(sss.sharedQueueOrder); iters++ { + if checkIndex >= len(sss.sharedQueueOrder) { + checkIndex = 0 + } + tenantName := sss.sharedQueueOrder[checkIndex] + + if _, ok := n.queueMap[tenantName]; !ok { + // tenant not in _this_ node's children, move on + checkIndex++ + continue + } + + // increment nodes checked even if not in tenant-querier map + n.childrenChecked++ + checkedAllNodes = n.childrenChecked == len(n.queueMap)+1 + + // if the tenant-querier set is nil, any querier can serve this tenant + if sss.tenantQuerierMap[TenantID(tenantName)] == nil { + sss.sharedQueuePosition = checkIndex + return n.queueMap[tenantName], checkedAllNodes + } + if tenantQuerierSet, ok := sss.tenantQuerierMap[TenantID(tenantName)]; ok { + if _, ok := tenantQuerierSet[*sss.currentQuerier]; ok { + sss.sharedQueuePosition = checkIndex + return n.queueMap[tenantName], checkedAllNodes + } + } + checkIndex++ + } + return nil, checkedAllNodes + } + + updateState := func(n *Node, v any, deletedNode bool) { + if deletedNode { + // when we delete a child node, we want to step back in the queue so the + // previously-next tenant will still be next (e.g., if queueOrder is + // [childA, childB, childC] and childB (position 1) is deleted, we want to + // step back to position 0 so that in the next dequeue, position 1 (childC) + // is dequeued + fmt.Println("node deleted") + sss.sharedQueuePosition-- + fmt.Println(sss.sharedQueuePosition) + } + // we need to reset our checked nodes if we found a value to dequeue, or if we've checked all nodes + if v != nil { + n.childrenChecked = 0 + return + } + + n.childrenChecked++ + + // if dequeueNode was self, advance queue position no matter what, and return + if sss.sharedQueuePosition == localQueueIndex { + return + } + + // We never change the order of sharedQueueOrder; we expect sss.queuePosition to be updated + // to the last index used for the first dequeue, and hand it the result for the next one. + + // otherwise, don't change queuePosition, only move element to back + //for i, child := range sss.queueOrder { + // if child == nodeName { + // sss.queueOrder = append(sss.queueOrder[:i], append(sss.queueOrder[i+1:], child)...) // ugly + // n.queuePosition-- + // // wrap around to back of order if we go past the first element in n.queueOrder. + // // In the next dequeue, we will step forward again. + // if n.queuePosition <= localQueueIndex { + // n.queuePosition = len(sss.queueOrder) - 1 + // } + // } + //} + } + + return dequeueOps{ + getNode: getNode, + updateState: updateState, + } + +} + +func (sss *shuffleShardState) getQueueOrder(_ *Node) []string { + return sss.sharedQueueOrder +} From 92ef09438ab4dd7df8dc58c030ce72fd0cdea8f8 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 3 Jun 2024 14:38:10 -0700 Subject: [PATCH 09/26] better way to pull out tenantID --- pkg/scheduler/queue/tenant_queues.go | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 5b9a092e400..c86bdaaef47 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -6,6 +6,7 @@ package queue import ( + "fmt" "time" ) @@ -155,13 +156,21 @@ func (qb *queueBroker) dequeueRequestForQuerier( qb.treeShuffleShardState.sharedQueuePosition = lastTenantIndex queuePath, queueElement := qb.queueTree.rootNode.dequeue() + fmt.Println("path: ", queuePath) + fmt.Println("elt: ", queueElement) - // TODO (casie): hacky - hardcoding second elt in queuePath to tenant ID + var request *tenantRequest var tenantID TenantID - if len(queuePath) > 1 { - tenantID = TenantID(queuePath[1]) + if queueElement != nil { + // re-casting to same type it was enqueued as; panic would indicate a bug + request = queueElement.(*tenantRequest) + tenantID = request.tenantID + } + + var tenant *queueTenant + if tenantID != "" { + tenant = qb.tenantQuerierAssignments.tenantsByID[tenantID] } - tenant := qb.tenantQuerierAssignments.tenantsByID[tenantID] // dequeue returns the full path including root, but getNode expects the path _from_ root queueNodeAfterDequeue := qb.queueTree.rootNode.getNode(queuePath[1:]) @@ -170,13 +179,6 @@ func (qb *queueBroker) dequeueRequestForQuerier( qb.tenantQuerierAssignments.removeTenant(tenantID) } - var request *tenantRequest - if queueElement != nil { - // re-casting to same type it was enqueued as; panic would indicate a bug - request = queueElement.(*tenantRequest) - } - - //return request, tenant, tenantIndex, nil return request, tenant, qb.treeShuffleShardState.sharedQueuePosition, nil } From 295b9ae8d462e940c2a29281880effe1063b7f69 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 3 Jun 2024 17:32:13 -0700 Subject: [PATCH 10/26] fixing tests --- pkg/scheduler/queue/queue_test.go | 10 +- .../queue/tenant_querier_assignment.go | 207 +++++++++++---- pkg/scheduler/queue/tenant_queues.go | 46 ++-- pkg/scheduler/queue/tenant_queues_test.go | 50 +++- pkg/scheduler/queue/tree_queue_sketch.go | 58 ++-- pkg/scheduler/queue/tree_queue_sketch_test.go | 195 +++++++------- .../queue/tree_queueing_algorithms.go | 250 ++---------------- 7 files changed, 373 insertions(+), 443 deletions(-) diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index da3d787de86..e9c90b00424 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -111,7 +111,7 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { slowQueueDimensionFunc := func() []string { return []string{slowConsumerQueueDimension} } additionalQueueDimensionsEnabledCases := []bool{ - //false, + false, true, } queueDurationTotals := map[bool]map[string]float64{ @@ -187,18 +187,14 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { for consumerIdx := 0; consumerIdx < numConsumers; consumerIdx++ { consumerIdx := consumerIdx queueConsumerErrGroup.Go(func() error { - fmt.Printf("waiting for gofunc to run for consumer: %v\n", consumerIdx) returnVal := runConsumer(consumerIdx) - fmt.Println("finished runConsumer without panicking") return returnVal }) - fmt.Println("past first suspected panic") } close(startConsumersChan) err = queueConsumerErrGroup.Wait() require.NoError(t, err) - fmt.Println("past second suspected panic\n=============") // record total queue duration by queue dimensions and whether the queue splitting was enabled for _, queueDimension := range []string{normalQueueDimension, slowConsumerQueueDimension} { @@ -369,13 +365,11 @@ func runQueueConsumerIters( start chan struct{}, consumeFunc consumeRequest, ) func(consumerIdx int) error { - fmt.Println("constructing returnFunc") returnFunc := func(consumerIdx int) error { consumerIters := queueActorIterationCount(totalIters, numConsumers, consumerIdx) lastTenantIndex := FirstTenant() querierID := fmt.Sprintf("consumer-%v", consumerIdx) queue.SubmitRegisterQuerierConnection(querierID) - fmt.Println("runQueueConsumerIters submitted register querier connection") defer queue.SubmitUnregisterQuerierConnection(querierID) <-start @@ -388,11 +382,9 @@ func runQueueConsumerIters( lastTenantIndex = idx } - fmt.Println("finished consumer iterations") return nil } - fmt.Println("finished constructing returnFunc\n=======") return returnFunc } diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 9d3b1bb3440..00d402eba8d 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -80,53 +80,18 @@ type tenantQuerierAssignments struct { // List of all tenants with queues, used for iteration when searching for next queue to handle. tenantIDOrder []TenantID tenantsByID map[TenantID]*queueTenant + // tenantOrderIndex is the index of the _last_ tenant dequeued from; it is passed by + // the querier, and then updated to the index of the last tenant dequeued from, so it + // can be returned to the querier. Newly connected queries should pass -1 to start at the + // beginning of tenantIDOrder. + tenantOrderIndex int + tenantNodes map[string][]*Node // Tenant assigned querier ID set as determined by shuffle sharding. // If tenant querier ID set is not nil, only those queriers can handle the tenant's requests, // Tenant querier ID is set to nil if sharding is off or available queriers <= tenant's maxQueriers. tenantQuerierIDs map[TenantID]map[QuerierID]struct{} -} - -// getNextTenantForQuerier gets the next tenant in the tenant order assigned to a given querier. -// -// The next tenant for the querier is obtained by rotating through the global tenant order -// starting just after the last tenant the querier received a request for, until a tenant -// is found that is assigned to the given querier according to the querier shuffle sharding. -// A newly connected querier provides lastTenantIndex of -1 in order to start at the beginning. -func (tqa *tenantQuerierAssignments) getNextTenantForQuerier(lastTenantIndex int, querierID QuerierID) (*queueTenant, int, error) { - // check if querier is registered and is not shutting down - if q := tqa.queriersByID[querierID]; q == nil || q.shuttingDown { - return nil, lastTenantIndex, ErrQuerierShuttingDown - } - tenantOrderIndex := lastTenantIndex - for iters := 0; iters < len(tqa.tenantIDOrder); iters++ { - tenantOrderIndex++ - if tenantOrderIndex >= len(tqa.tenantIDOrder) { - // Do not use modulo (e.g. i = (i + 1) % len(slice)) to wrap this index. - // Tenant list can change size between calls and the querier provides its external view - // of the lastTenantIndex it received, which is not updated when this list changes. - // If the tenant list shrinks and the querier-provided lastTenantIndex exceeds the - // length of the tenant list, wrapping via modulo would skip the beginning of the list. - tenantOrderIndex = 0 - } - - tenantID := tqa.tenantIDOrder[tenantOrderIndex] - if tenantID == emptyTenantID { - continue - } - tenant := tqa.tenantsByID[tenantID] - - tenantQuerierSet := tqa.tenantQuerierIDs[tenantID] - if tenantQuerierSet == nil { - // tenant can use all queriers - return tenant, tenantOrderIndex, nil - } else if _, ok := tenantQuerierSet[querierID]; ok { - // tenant is assigned this querier - return tenant, tenantOrderIndex, nil - } - } - - return nil, lastTenantIndex, nil + currentQuerier *QuerierID } func (tqa *tenantQuerierAssignments) getTenant(tenantID TenantID) (*queueTenant, error) { @@ -212,22 +177,16 @@ func (tqa *tenantQuerierAssignments) addQuerierConnection(querierID QuerierID) ( return tqa.recomputeTenantQueriers() } +// removeTenant only manages deletion of a *queueTenant from tenantsByID. All other +// tenant deletion (e.g., from tenantIDOrder, or tenantNodes) is done during the dequeue operation, +// as we cannot remove from those things arbitrarily; we must check whether other tenant +// queues exist for the same tenant before removing. func (tqa *tenantQuerierAssignments) removeTenant(tenantID TenantID) { tenant := tqa.tenantsByID[tenantID] if tenant == nil { return } delete(tqa.tenantsByID, tenantID) - tqa.tenantIDOrder[tenant.orderIndex] = emptyTenantID - - // Shrink tenant list if possible by removing empty tenant IDs. - // We remove only from the end; removing from the middle would re-index all tenant IDs - // and skip tenants when starting iteration from a querier-provided lastTenantIndex. - // Empty tenant IDs stuck in the middle of the slice are handled - // by replacing them when a new tenant ID arrives in the queue. - for i := len(tqa.tenantIDOrder) - 1; i >= 0 && tqa.tenantIDOrder[i] == emptyTenantID; i-- { - tqa.tenantIDOrder = tqa.tenantIDOrder[:i] - } } func (tqa *tenantQuerierAssignments) removeQuerierConnection(querierID QuerierID, now time.Time) (resharded bool) { @@ -358,3 +317,147 @@ func (tqa *tenantQuerierAssignments) shuffleTenantQueriers(tenantID TenantID, sc tqa.tenantQuerierIDs[tenantID] = querierIDSet return true } + +func (tqa *tenantQuerierAssignments) dequeueGetNode(n *Node) (*Node, bool) { + // can't get a tenant if no querier set + if tqa.currentQuerier == nil { + return nil, true + } + + checkedAllNodes := n.childrenChecked == len(n.queueMap)+1 // must check local queue as well + + // advance queue position for dequeue + tqa.tenantOrderIndex++ + if tqa.tenantOrderIndex >= len(tqa.tenantIDOrder) { + tqa.tenantOrderIndex = localQueueIndex + } + + // no children + if len(n.queueMap) == 0 || tqa.tenantOrderIndex == localQueueIndex { + return n, checkedAllNodes + } + + checkIndex := tqa.tenantOrderIndex + + for iters := 0; iters < len(tqa.tenantIDOrder); iters++ { + if checkIndex >= len(tqa.tenantIDOrder) { + checkIndex = 0 + } + tenantID := tqa.tenantIDOrder[checkIndex] + tenantName := string(tenantID) + + if _, ok := n.queueMap[tenantName]; !ok { + // tenant not in _this_ node's children, move on + checkIndex++ + continue + } + + // increment nodes checked even if not in tenant-querier map + n.childrenChecked++ + checkedAllNodes = n.childrenChecked == len(n.queueMap)+1 + + // if the tenant-querier set is nil, any querier can serve this tenant + if tqa.tenantQuerierIDs[tenantID] == nil { + tqa.tenantOrderIndex = checkIndex + return n.queueMap[tenantName], checkedAllNodes + } + if tenantQuerierSet, ok := tqa.tenantQuerierIDs[tenantID]; ok { + if _, ok := tenantQuerierSet[*tqa.currentQuerier]; ok { + tqa.tenantOrderIndex = checkIndex + return n.queueMap[tenantName], checkedAllNodes + } + } + checkIndex++ + } + return nil, checkedAllNodes +} + +func (tqa *tenantQuerierAssignments) dequeueUpdateState(n *Node, v any, deletedNode bool) { + if deletedNode { + // when we delete a child node, we want to step back in the queue so the + // previously-next tenant will still be next (e.g., if queueOrder is + // [childA, childB, childC] and childB (position 1) is deleted, we want to + // step back to position 0 so that in the next dequeue, position 1 (childC) + // is dequeued + tqa.tenantOrderIndex-- + } + // we need to reset our checked nodes if we found a value to dequeue, or if we've checked all nodes + if v != nil { + n.childrenChecked = 0 + return + } + + n.childrenChecked++ + + // if dequeueNode was self, advance queue position no matter what, and return + if tqa.tenantOrderIndex == localQueueIndex { + return + } +} + +func (tqa *tenantQuerierAssignments) addChildNode(parent, child *Node) { + childName := child.Name() + _, tenantHasAnyQueue := tqa.tenantNodes[childName] + + // add childNode to node's queueMap, + // and to the shared tenantNodes map + parent.queueMap[childName] = child + tqa.tenantNodes[childName] = append(tqa.tenantNodes[childName], child) + + // if child has any queue, it'll already be in tenantIDOrder, return without altering order + if tenantHasAnyQueue { + return + } + + // otherwise, add childNode to n.queueOrder before the current position, and + // update n.queuePosition to current element + for i, elt := range tqa.tenantIDOrder { + // if there's a tenant with this name in tenantIDOrder already, move on + if elt == TenantID(childName) { + return + } + if elt == "" { + tqa.tenantIDOrder[i] = TenantID(childName) + return + } + } + // if we get here, we didn't find any empty elements in tenantIDOrder; append + tqa.tenantIDOrder = append(tqa.tenantIDOrder, TenantID(childName)) + +} + +func (tqa *tenantQuerierAssignments) deleteChildNode(parent, child *Node) bool { + childName := child.Name() + + // delete from parent's children + delete(parent.queueMap, childName) + + // delete from shared tenantNodes + for i, tenantNode := range tqa.tenantNodes[childName] { + if tenantNode == child { + tqa.tenantNodes[childName] = append(tqa.tenantNodes[childName][:i], tqa.tenantNodes[childName][i+1:]...) + } + } + + // check tenantNodes; we only remove from sharedQueueOrder + // if all nodes with this name are gone. If the child is at the _end_ of sharedQueueOrder, we + // remove it outright; otherwise, we replace it with an empty string so as not to reindex all + // slice elements + + removeFromSharedQueueOrder := len(tqa.tenantNodes[childName]) == 0 + + var positionNeedsUpdate bool + if removeFromSharedQueueOrder { + for idx, name := range tqa.tenantIDOrder { + if string(name) == childName { + tqa.tenantIDOrder[idx] = emptyTenantID + } + } + // clear all sequential empty elements from tenantIDOrder + lastElementIndex := len(tqa.tenantIDOrder) - 1 + for i := lastElementIndex; i >= 0 && tqa.tenantIDOrder[i] == ""; i-- { + tqa.tenantIDOrder = tqa.tenantIDOrder[:i] + } + } + return positionNeedsUpdate +} diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index c86bdaaef47..83095cd515f 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -27,13 +27,10 @@ type queueBroker struct { tenantQueuesTree *TreeQueue queueTree *Tree - tenantQuerierAssignments tenantQuerierAssignments + tenantQuerierAssignments *tenantQuerierAssignments maxTenantQueueSize int additionalQueueDimensionsEnabled bool - currentQuerier QuerierID - - treeShuffleShardState *shuffleShardState } func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled bool, forgetDelay time.Duration) *queueBroker { @@ -45,33 +42,32 @@ func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled boo tenantIDOrder: nil, tenantsByID: map[TenantID]*queueTenant{}, tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, + currentQuerier: ¤tQuerier, + tenantOrderIndex: localQueueIndex - 1, } - sss := &shuffleShardState{ - tenantQuerierMap: tqas.tenantQuerierIDs, - currentQuerier: ¤tQuerier, - } // TODO (casie): Maybe set this using a flag, so we can also change how we build queue path accordingly - tree := NewTree( - sss, // root + tree, err := NewTree( + tqas, // root &roundRobinState{}, // tenants &roundRobinState{}, // query components ) + // An error building the tree is fatal; we must panic + if err != nil { + panic(fmt.Sprintf("error creating the tree queue: %v", err)) + } qb := &queueBroker{ tenantQueuesTree: NewTreeQueue("root"), queueTree: tree, - tenantQuerierAssignments: *tqas, + tenantQuerierAssignments: tqas, maxTenantQueueSize: maxTenantQueueSize, additionalQueueDimensionsEnabled: additionalQueueDimensionsEnabled, - currentQuerier: currentQuerier, - treeShuffleShardState: sss, } return qb } func (qb *queueBroker) isEmpty() bool { - //return qb.tenantQueuesTree.IsEmpty() return qb.queueTree.rootNode.IsEmpty() } @@ -95,7 +91,6 @@ func (qb *queueBroker) enqueueRequestBack(request *tenantRequest, tenantMaxQueri } } - //err = qb.tenantQueuesTree.EnqueueBackByPath(queuePath, request) err = qb.queueTree.rootNode.enqueueBackByPath(qb.queueTree, queuePath, request) return err } @@ -116,7 +111,6 @@ func (qb *queueBroker) enqueueRequestFront(request *tenantRequest, tenantMaxQuer return err } return qb.queueTree.rootNode.enqueueFrontByPath(qb.queueTree, queuePath, request) - //return qb.tenantQueuesTree.EnqueueFrontByPath(queuePath, request) } func (qb *queueBroker) makeQueuePath(request *tenantRequest) (QueuePath, error) { @@ -139,25 +133,15 @@ func (qb *queueBroker) dequeueRequestForQuerier( int, error, ) { - //tenant, tenantIndex, err := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndex, querierID) // check if querier is registered and is not shutting down if q := qb.tenantQuerierAssignments.queriersByID[querierID]; q == nil || q.shuttingDown { - return nil, nil, qb.treeShuffleShardState.sharedQueuePosition, ErrQuerierShuttingDown + return nil, nil, qb.tenantQuerierAssignments.tenantOrderIndex, ErrQuerierShuttingDown } - //if tenant == nil || err != nil { - // return nil, tenant, tenantIndex, err - //} - // - //queuePath := QueuePath{string(tenant.tenantID)} - // - //queueElement := qb.tenantQueuesTree.DequeueByPath(queuePath) - qb.currentQuerier = querierID - qb.treeShuffleShardState.sharedQueuePosition = lastTenantIndex + qb.tenantQuerierAssignments.currentQuerier = &querierID + qb.tenantQuerierAssignments.tenantOrderIndex = lastTenantIndex queuePath, queueElement := qb.queueTree.rootNode.dequeue() - fmt.Println("path: ", queuePath) - fmt.Println("elt: ", queueElement) var request *tenantRequest var tenantID TenantID @@ -174,12 +158,12 @@ func (qb *queueBroker) dequeueRequestForQuerier( // dequeue returns the full path including root, but getNode expects the path _from_ root queueNodeAfterDequeue := qb.queueTree.rootNode.getNode(queuePath[1:]) - if queueNodeAfterDequeue == nil { + if queueNodeAfterDequeue == nil && len(qb.tenantQuerierAssignments.tenantNodes[string(tenantID)]) == 0 { // queue node was deleted due to being empty after dequeue qb.tenantQuerierAssignments.removeTenant(tenantID) } - return request, tenant, qb.treeShuffleShardState.sharedQueuePosition, nil + return request, tenant, qb.tenantQuerierAssignments.tenantOrderIndex, nil } func (qb *queueBroker) addQuerierConnection(querierID QuerierID) (resharded bool) { diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 020df066dde..1ce340e4772 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -644,6 +644,48 @@ func getTenantsByQuerier(broker *queueBroker, querierID QuerierID) []TenantID { return tenantIDs } +// getNextTenantForQuerier gets the next tenant in the tenant order assigned to a given querier. +// +// The next tenant for the querier is obtained by rotating through the global tenant order +// starting just after the last tenant the querier received a request for, until a tenant +// is found that is assigned to the given querier according to the querier shuffle sharding. +// A newly connected querier provides lastTenantIndex of -1 in order to start at the beginning. +func (tqa *tenantQuerierAssignments) getNextTenantForQuerier(lastTenantIndex int, querierID QuerierID) (*queueTenant, int, error) { + // check if querier is registered and is not shutting down + if q := tqa.queriersByID[querierID]; q == nil || q.shuttingDown { + return nil, lastTenantIndex, ErrQuerierShuttingDown + } + tenantOrderIndex := lastTenantIndex + for iters := 0; iters < len(tqa.tenantIDOrder); iters++ { + tenantOrderIndex++ + if tenantOrderIndex >= len(tqa.tenantIDOrder) { + // Do not use modulo (e.g. i = (i + 1) % len(slice)) to wrap this index. + // Tenant list can change size between calls and the querier provides its external view + // of the lastTenantIndex it received, which is not updated when this list changes. + // If the tenant list shrinks and the querier-provided lastTenantIndex exceeds the + // length of the tenant list, wrapping via modulo would skip the beginning of the list. + tenantOrderIndex = 0 + } + + tenantID := tqa.tenantIDOrder[tenantOrderIndex] + if tenantID == emptyTenantID { + continue + } + tenant := tqa.tenantsByID[tenantID] + + tenantQuerierSet := tqa.tenantQuerierIDs[tenantID] + if tenantQuerierSet == nil { + // tenant can use all queriers + return tenant, tenantOrderIndex, nil + } else if _, ok := tenantQuerierSet[querierID]; ok { + // tenant is assigned this querier + return tenant, tenantOrderIndex, nil + } + } + + return nil, lastTenantIndex, nil +} + // getOrAddTenantQueue is a test utility, not intended for use by consumers of queueBroker func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) (*Node, error) { err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriers) @@ -651,7 +693,7 @@ func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) ( return nil, err } - node, err := qb.queueTree.rootNode.getOrAddNode(QueuePath{string(tenantID)}, qb.queueTree) + node, err := qb.queueTree.rootNode.getOrAddNode(qb.makeQueuePathForTests(tenantID), qb.queueTree) if err != nil { return nil, err } @@ -662,7 +704,6 @@ func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) ( func (qb *queueBroker) removeTenantQueue(tenantID TenantID) bool { qb.tenantQuerierAssignments.removeTenant(tenantID) queuePath := QueuePath{string(tenantID)} - //return qb.tenantQueuesTree.deleteNode(queuePath) return qb.queueTree.rootNode.deleteNode(queuePath) } @@ -682,7 +723,7 @@ func (n *Node) deleteNode(pathFromNode QueuePath) bool { } if deleteNode := parentNode.getNode(QueuePath{deleteNodeName}); deleteNode != nil { childDeleted := parentNode.dequeueAlgorithm.deleteChildNode(parentNode, deleteNode) - parentNode.dequeueAlgorithm.dequeueOps().updateState(parentNode, nil, childDeleted) + parentNode.dequeueAlgorithm.dequeueUpdateState(parentNode, nil, childDeleted) return true } return false @@ -741,9 +782,6 @@ func isConsistent(qb *queueBroker) error { tenantQueueCount := qb.queueTree.rootNode.nodeCount() - 1 if tenantQueueCount != tenantCount { - //fmt.Printf("tenantIDOrder: %v\n", qb.tenantQuerierAssignments.tenantIDOrder) - //fmt.Printf("rootNode children: %v\n", qb.queueTree.rootNode.dequeueAlgorithm.getQueueOrder()) - fmt.Println("tenant count check: ", tenantQueueCount, tenantCount) return fmt.Errorf("inconsistent number of tenants list and tenant queues") } diff --git a/pkg/scheduler/queue/tree_queue_sketch.go b/pkg/scheduler/queue/tree_queue_sketch.go index a734071c16d..750b817d032 100644 --- a/pkg/scheduler/queue/tree_queue_sketch.go +++ b/pkg/scheduler/queue/tree_queue_sketch.go @@ -21,9 +21,9 @@ type Tree struct { algosByDepth []DequeueAlgorithm } -// Nodes with *shuffleShardState dequeueAlgorithms disregard individual -// Nodes' queueOrder and queuePosition -// TODO (casie): Implement maxQueueSize to mirror qb.maxTenantQueueSize? +// Node maintains tree-queue-node specific information, such as child order, and position in queue. +// Note that tenantQuerierAssignments largely disregards queueOrder and queuePosition, due to the +// way shuffle-sharding must be managed globally and on a per-querier basis type Node struct { name string localQueue *list.List @@ -35,43 +35,51 @@ type Node struct { childrenChecked int } -func NewTree(dequeueAlgorithms ...DequeueAlgorithm) *Tree { +func NewTree(dequeueAlgorithms ...DequeueAlgorithm) (*Tree, error) { if len(dequeueAlgorithms) == 0 { - return nil + return nil, fmt.Errorf("cannot create a tree without defined dequeueing algorithms") } - root, err := NewNode("root", 0, dequeueAlgorithms[0]) + root, err := newNode("root", 0, dequeueAlgorithms[0]) if err != nil { - // TODO (casie): Make NewNode private, remove err return - fmt.Print(err) + return nil, err } root.depth = 0 return &Tree{ rootNode: root, algosByDepth: dequeueAlgorithms, - } + }, nil +} + +func (t *Tree) Dequeue() (QueuePath, any) { + return t.rootNode.dequeue() } -func NewNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { +func (t *Tree) EnqueueBackByPath(path QueuePath, v any) error { + return t.rootNode.enqueueBackByPath(t, path, v) +} + +func (t *Tree) EnqueueFrontByPath(path QueuePath, v any) error { + return t.rootNode.enqueueFrontByPath(t, path, v) +} + +func newNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { // TODO (casie): write a unit test if da == nil { return nil, fmt.Errorf("cannot create a node without a defined dequeueing algorithm") } switch da.(type) { - case *shuffleShardState: - sss := da.(*shuffleShardState) - sss.sharedQueuePosition = localQueueIndex - 1 - if sss.tenantNodes == nil { - sss.tenantNodes = map[string][]*Node{} - } - if sss.tenantQuerierMap == nil { - sss.tenantQuerierMap = map[TenantID]map[QuerierID]struct{}{} + case *tenantQuerierAssignments: + tqa := da.(*tenantQuerierAssignments) + tqa.tenantOrderIndex = localQueueIndex - 1 + if tqa.tenantNodes == nil { + tqa.tenantNodes = map[string][]*Node{} } } return &Node{ - name: name, - localQueue: list.New(), - queuePosition: localQueueIndex - 1, // start from -2 so that on first dequeue, localQueueIndex is dequeued from - //queueOrder: make([]string, 0), + name: name, + localQueue: list.New(), + queuePosition: localQueueIndex - 1, // start from -2 so that on first dequeue, localQueueIndex is dequeued from + queueOrder: make([]string, 0), queueMap: make(map[string]*Node, 1), depth: depth, dequeueAlgorithm: da, @@ -132,7 +140,7 @@ func (n *Node) dequeue() (QueuePath, any) { var dequeueNode *Node // continue until we've found a value or checked all nodes that need checking for v == nil && !checkedAllNodes { - dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueOps().getNode(n) + dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueGetNode(n) var deletedNode bool // dequeueing from local queue if dequeueNode == n { @@ -159,7 +167,7 @@ func (n *Node) dequeue() (QueuePath, any) { deletedNode = n.dequeueAlgorithm.deleteChildNode(n, dequeueNode) } } - n.dequeueAlgorithm.dequeueOps().updateState(n, v, deletedNode) + n.dequeueAlgorithm.dequeueUpdateState(n, v, deletedNode) } return append(path, childPath...), v } @@ -199,7 +207,7 @@ func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *Tree) (*Node, error) { if n.depth+1 >= len(tree.algosByDepth) { return nil, fmt.Errorf("cannot add a node beyond max tree depth: %v", len(tree.algosByDepth)) } - childNode, err = NewNode(pathFromNode[0], n.depth+1, tree.algosByDepth[n.depth+1]) + childNode, err = newNode(pathFromNode[0], n.depth+1, tree.algosByDepth[n.depth+1]) if err != nil { return nil, err } diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go index e129768bf52..391b4ab6e19 100644 --- a/pkg/scheduler/queue/tree_queue_sketch_test.go +++ b/pkg/scheduler/queue/tree_queue_sketch_test.go @@ -1,45 +1,49 @@ package queue import ( - "fmt" //"fmt" "github.com/stretchr/testify/require" "testing" ) -// TODO (casie): Write tests for NewTree -// TODO (casie): Write a test for dequeuing from sss childA, enqueue to new childB, expect to dequeue next from childB +// TODO (casie): Write a test for dequeuing from tqa childA, enqueue to new childB, expect to dequeue next from childB +// TODO (casie): Write a test for enqueueFrontByPath() -func newShuffleShardState() *shuffleShardState { - return &shuffleShardState{ - tenantQuerierMap: make(map[TenantID]map[QuerierID]struct{}), +func newTenantQuerierAssignments() *tenantQuerierAssignments { + return &tenantQuerierAssignments{ + tenantQuerierIDs: make(map[TenantID]map[QuerierID]struct{}), tenantNodes: make(map[string][]*Node), } } -func Test_NewNode(t *testing.T) { +func Test_NewTree(t *testing.T) { tests := []struct { name string - rootAlgo DequeueAlgorithm - state *shuffleShardState + treeAlgos []DequeueAlgorithm + state *tenantQuerierAssignments expectErr bool }{ { - name: "create round-robin node", - rootAlgo: &roundRobinState{}, + name: "create round-robin tree", + treeAlgos: []DequeueAlgorithm{&roundRobinState{}}, + }, + { + name: "create shuffle-shard tree", + treeAlgos: []DequeueAlgorithm{newTenantQuerierAssignments()}, }, { - name: "create shuffle-shard node", - rootAlgo: newShuffleShardState(), + name: "create shuffle-shard tree with no tenant-querier map, we should create an empty one", + treeAlgos: []DequeueAlgorithm{&tenantQuerierAssignments{}}, }, { - name: "create shuffle-shard tree with no tenant-querier map, we should create an empty one", - rootAlgo: &shuffleShardState{}, + name: "fail to create tree without defined dequeueing algorithm", + treeAlgos: []DequeueAlgorithm{nil}, + expectErr: true, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - _, err := NewNode("root", 0, tt.rootAlgo) + _, err := NewTree(tt.treeAlgos...) if tt.expectErr { require.Error(t, err) } else { @@ -65,18 +69,18 @@ func Test_EnqueueBackByPath(t *testing.T) { }, { name: "enqueue shuffle-shard node to round-robin node", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}}, + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, children: []QueuePath{{"shuffle-shard-child-1"}}, }, { name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node, we should create an empty one", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &shuffleShardState{}}, + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{}}, children: []QueuePath{{"shuffle-shard-child-1"}}, }, { name: "enqueue round-robin node to shuffle-shard node", treeAlgosByDepth: []DequeueAlgorithm{ - newShuffleShardState(), + newTenantQuerierAssignments(), &roundRobinState{}, }, children: []QueuePath{{"round-robin-child-1"}}, @@ -84,9 +88,9 @@ func Test_EnqueueBackByPath(t *testing.T) { { name: "create tree with multiple shuffle-shard depths", treeAlgosByDepth: []DequeueAlgorithm{ - newShuffleShardState(), + newTenantQuerierAssignments(), &roundRobinState{}, - newShuffleShardState(), + newTenantQuerierAssignments(), }, children: []QueuePath{{"child"}, {"grandchild"}}, }, @@ -99,12 +103,11 @@ func Test_EnqueueBackByPath(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree := NewTree(tt.treeAlgosByDepth...) - root := tree.rootNode + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) - var err error for _, childPath := range tt.children { - err = root.enqueueBackByPath(tree, childPath, "some-object") + err = tree.EnqueueBackByPath(childPath, "some-object") } if tt.expectErr { require.Error(t, err) @@ -127,7 +130,7 @@ func Test_Dequeue_RootNode(t *testing.T) { }, { name: "dequeue from empty shuffle-shard root node", - rootAlgo: newShuffleShardState(), + rootAlgo: newTenantQuerierAssignments(), }, { name: "dequeue from non-empty round-robin root node", @@ -136,7 +139,7 @@ func Test_Dequeue_RootNode(t *testing.T) { }, { name: "dequeue from non-empty shuffle-shard root node", - rootAlgo: newShuffleShardState(), + rootAlgo: newTenantQuerierAssignments(), enqueueToRoot: []any{"something-else-in-root"}, }, } @@ -145,25 +148,25 @@ func Test_Dequeue_RootNode(t *testing.T) { // TODO (casie): ugly querierID := QuerierID("placeholder") switch tt.rootAlgo.(type) { - case *shuffleShardState: - tt.rootAlgo.(*shuffleShardState).currentQuerier = &querierID + case *tenantQuerierAssignments: + tt.rootAlgo.(*tenantQuerierAssignments).currentQuerier = &querierID } - tree := NewTree(tt.rootAlgo) - root := tree.rootNode + tree, err := NewTree(tt.rootAlgo) + require.NoError(t, err) for _, elt := range tt.enqueueToRoot { - err := root.enqueueBackByPath(tree, QueuePath{}, elt) + err = tree.EnqueueBackByPath(QueuePath{}, elt) require.NoError(t, err) } rootPath := QueuePath{"root"} for _, elt := range tt.enqueueToRoot { - path, v := root.dequeue() + path, v := tree.Dequeue() require.Equal(t, rootPath, path) require.Equal(t, elt, v) } - path, v := root.dequeue() + path, v := tree.Dequeue() require.Equal(t, rootPath, path) require.Nil(t, v) @@ -221,30 +224,30 @@ func Test_RoundRobinDequeue(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) - root := tree.rootNode + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) for _, sqo := range tt.selfQueueObjects { - err := root.enqueueBackByPath(tree, QueuePath{}, sqo) + err = tree.EnqueueBackByPath(QueuePath{}, sqo) require.NoError(t, err) } for _, child := range tt.children { for _, obj := range tt.childQueueObjects[child] { - _ = root.enqueueBackByPath(tree, QueuePath{child}, obj) + _ = tree.EnqueueBackByPath(QueuePath{child}, obj) } } for _, grandchild := range tt.grandchildren { gqo := tt.grandchildrenQueueObjects[grandchild] for _, obj := range gqo.objs { - err := root.enqueueBackByPath(tree, gqo.path, obj) + err = tree.EnqueueBackByPath(gqo.path, obj) require.NoError(t, err) } } for _, expected := range tt.expected { - _, val := root.dequeue() + _, val := tree.Dequeue() v, ok := val.(string) require.True(t, ok) require.Equal(t, expected, v) @@ -286,13 +289,13 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { { name: "should dequeue from new (next-in-queue) shuffle-shard child immediately after it is added", treeAlgosByDepth: []DequeueAlgorithm{ - &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, tenantNodes: map[string][]*Node{}, currentQuerier: &placeholderQuerier, }, - &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, tenantNodes: map[string][]*Node{}, currentQuerier: &placeholderQuerier, }, @@ -311,12 +314,13 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree := NewTree(tt.treeAlgosByDepth...) + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) root := tree.rootNode for _, operation := range tt.operationOrder { if operation.kind == enqueue { - err := root.enqueueBackByPath(tree, operation.path, operation.obj) + err = tree.EnqueueBackByPath(operation.path, operation.obj) require.NoError(t, err) } if operation.kind == dequeue { @@ -338,7 +342,7 @@ func Test_ShuffleShardDequeue(t *testing.T) { tests := []struct { name string treeAlgosByDepth []DequeueAlgorithm - state *shuffleShardState + state *tenantQuerierAssignments currQuerier []QuerierID enqueueObjs []enqueueObj expected []any @@ -348,11 +352,11 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "happy path - tenant found in tenant-querier map under first child", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - newShuffleShardState(), + newTenantQuerierAssignments(), &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{"querier-1"}, @@ -365,11 +369,11 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "tenant exists, but not for querier", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - newShuffleShardState(), + newTenantQuerierAssignments(), &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{"querier-1"}, @@ -382,11 +386,11 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "1 of 3 tenants exist for querier", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - newShuffleShardState(), + newTenantQuerierAssignments(), &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, }, currQuerier: []QuerierID{"querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ @@ -400,11 +404,11 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "tenant exists for querier on next parent node", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - newShuffleShardState(), + newTenantQuerierAssignments(), &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, }, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, enqueueObjs: []enqueueObj{ @@ -419,11 +423,11 @@ func Test_ShuffleShardDequeue(t *testing.T) { name: "2 of 3 tenants exist for querier", treeAlgosByDepth: []DequeueAlgorithm{ &roundRobinState{}, - newShuffleShardState(), + newTenantQuerierAssignments(), &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, @@ -437,12 +441,12 @@ func Test_ShuffleShardDequeue(t *testing.T) { { name: "root node is shuffle-shard node", treeAlgosByDepth: []DequeueAlgorithm{ - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}, tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, @@ -456,12 +460,12 @@ func Test_ShuffleShardDequeue(t *testing.T) { { name: "dequeueing for one querier returns nil, but does return for a different querier", treeAlgosByDepth: []DequeueAlgorithm{ - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{"querier-2", "querier-1"}, @@ -473,12 +477,12 @@ func Test_ShuffleShardDequeue(t *testing.T) { { name: "no querier set in state", treeAlgosByDepth: []DequeueAlgorithm{ - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{""}, @@ -491,12 +495,12 @@ func Test_ShuffleShardDequeue(t *testing.T) { // TODO (casie): also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) name: "dequeue from a tenant with a nil tenant-querier map", treeAlgosByDepth: []DequeueAlgorithm{ - &shuffleShardState{tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{}}, + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, &roundRobinState{}, }, - state: &shuffleShardState{ - tenantQuerierMap: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}, tenantNodes: map[string][]*Node{}, }, currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, @@ -518,26 +522,25 @@ func Test_ShuffleShardDequeue(t *testing.T) { // update the state's currentQuerier. for i, da := range tt.treeAlgosByDepth { switch da.(type) { - case *shuffleShardState: - sss := tt.treeAlgosByDepth[i].(*shuffleShardState) - sss.tenantQuerierMap = tt.state.tenantQuerierMap - sss.currentQuerier = tt.state.currentQuerier + case *tenantQuerierAssignments: + tqas := tt.treeAlgosByDepth[i].(*tenantQuerierAssignments) + tqas.tenantQuerierIDs = tt.state.tenantQuerierIDs + tqas.currentQuerier = tt.state.currentQuerier } } - tree := NewTree(tt.treeAlgosByDepth...) - root := tree.rootNode + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) for _, o := range tt.enqueueObjs { - err := root.enqueueBackByPath(tree, o.path, o.obj) + err = tree.EnqueueBackByPath(o.path, o.obj) require.NoError(t, err) } // currQuerier at position i is used to dequeue the expected result at position i require.Equal(t, len(tt.currQuerier), len(tt.expected)) for i := 0; i < len(tt.expected); i++ { currentQuerier = tt.currQuerier[i] - path, v := root.dequeue() - fmt.Println(path, v) + _, v := tree.Dequeue() require.Equal(t, tt.expected[i], v) } }) @@ -552,18 +555,18 @@ func Test_ChangeShuffleShardState(t *testing.T) { tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}}, } - state := &shuffleShardState{ - tenantQuerierMap: tqa.tenantQuerierIDs, + state := &tenantQuerierAssignments{ + tenantQuerierIDs: tqa.tenantQuerierIDs, currentQuerier: nil, } - tree := NewTree(state, &roundRobinState{}, &roundRobinState{}) - root := tree.rootNode + tree, err := NewTree(state, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) - err := root.enqueueBackByPath(tree, QueuePath{"tenant-1", "query-component-1"}, "query-1") - err = root.enqueueBackByPath(tree, QueuePath{"tenant-2", "query-component-1"}, "query-2") - err = root.enqueueBackByPath(tree, QueuePath{"tenant-2", "query-component-1"}, "query-3") - err = root.enqueueBackByPath(tree, QueuePath{"tenant-2", "query-component-1"}, "query-4") + err = tree.EnqueueBackByPath(QueuePath{"tenant-1", "query-component-1"}, "query-1") + err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-2") + err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-3") + err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-4") require.NoError(t, err) querier1 := QuerierID("querier-1") @@ -572,24 +575,24 @@ func Test_ChangeShuffleShardState(t *testing.T) { // set state to querier-2 should dequeue query-2 state.currentQuerier = &querier2 - _, v := root.dequeue() + _, v := tree.Dequeue() require.Equal(t, "query-2", v) // update state to querier-1 should dequeue query-1 state.currentQuerier = &querier1 - _, v = root.dequeue() + _, v = tree.Dequeue() require.Equal(t, "query-1", v) // update tqa map to add querier-3 as assigned to tenant-2, then set state to querier-3 should dequeue query-3 tqa.tenantQuerierIDs["tenant-2"]["querier-3"] = struct{}{} state.currentQuerier = &querier3 - _, v = root.dequeue() + _, v = tree.Dequeue() require.Equal(t, "query-3", v) // during reshuffle, we only ever reassign tenant values, we don't assign an entirely new map value // to tenantQuerierIDs. Reassign tenant-2 to an empty map value , and query-4 should _not_ be dequeued tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} - _, v = root.dequeue() + _, v = tree.Dequeue() require.Nil(t, v) } diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go index 758ba4ae9a2..c5e5baa0d33 100644 --- a/pkg/scheduler/queue/tree_queueing_algorithms.go +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -1,12 +1,10 @@ package queue -import "fmt" - type DequeueAlgorithm interface { addChildNode(*Node, *Node) deleteChildNode(*Node, *Node) bool - dequeueOps() dequeueOps - getQueueOrder(*Node) []string + dequeueGetNode(*Node) (*Node, bool) + dequeueUpdateState(*Node, any, bool) } type dequeueGetNodeFunc func(*Node) (*Node, bool) @@ -46,234 +44,38 @@ func (rrs *roundRobinState) deleteChildNode(parent, child *Node) bool { return childFound } -func (rrs *roundRobinState) dequeueOps() dequeueOps { - getNode := func(n *Node) (*Node, bool) { - // advance the queue position for this dequeue - n.queuePosition++ - if n.queuePosition >= len(n.queueOrder) { - n.queuePosition = localQueueIndex - } - - checkedAllNodes := n.childrenChecked == len(n.queueOrder)+1 - if n.queuePosition == localQueueIndex { - return n, checkedAllNodes - } - - currentNodeName := n.queueOrder[n.queuePosition] - if node, ok := n.queueMap[currentNodeName]; ok { - return node, checkedAllNodes - } - return nil, checkedAllNodes - } - - // TODO (casie): Ignoring nodeName here because it doesn't matter...ugly - updateState := func(n *Node, v any, deletedNode bool) { - if v != nil { - n.childrenChecked = 0 - } else { - n.childrenChecked++ - } - if deletedNode { - n.queuePosition-- - // if we try to go beyond something that would increment to - // the localQueueIndex or in queueOrder, we should wrap around to the - // back of queueOrder. - if n.queuePosition < localQueueIndex-1 { - n.queuePosition = len(n.queueOrder) - 1 - } - } - } - - return dequeueOps{ - getNode: getNode, - updateState: updateState, +func (rrs *roundRobinState) dequeueGetNode(n *Node) (*Node, bool) { + // advance the queue position for this dequeue + n.queuePosition++ + if n.queuePosition >= len(n.queueOrder) { + n.queuePosition = localQueueIndex } -} - -func (rrs *roundRobinState) getQueueOrder(n *Node) []string { - return n.queueOrder -} -type shuffleShardState struct { - tenantQuerierMap map[TenantID]map[QuerierID]struct{} - tenantNodes map[string][]*Node - currentQuerier *QuerierID - sharedQueueOrder []string - sharedQueuePosition int -} - -func (sss *shuffleShardState) addChildNode(parent, child *Node) { - childName := child.Name() - _, childInOrder := sss.tenantNodes[childName] - - // add childNode to node's queueMap, - // and to the shared tenantNodes map - parent.queueMap[childName] = child - sss.tenantNodes[childName] = append(sss.tenantNodes[childName], child) - - // if child already exists in shared queueOrder, return without altering order - if childInOrder { - return - } - - // otherwise, add childNode to n.queueOrder before the current position, and - // update n.queuePosition to current element - // TODO (casie): Mirrors tenantQuerierAssignments.createOrUpdateTenant; move logic here? - for i, elt := range sss.sharedQueueOrder { - if elt == "" { - sss.sharedQueueOrder[i] = childName - return - } - } - // if we get here, we didn't find any empty elements in sharedQueueOrder; append - sss.sharedQueueOrder = append(sss.sharedQueueOrder, childName) - -} - -func (sss *shuffleShardState) deleteChildNode(parent, child *Node) bool { - childName := child.Name() - - // delete from parent's children - delete(parent.queueMap, childName) - - // delete from shared tenantNodes - for i, tenantNode := range sss.tenantNodes[childName] { - if tenantNode == child { - sss.tenantNodes[childName] = append(sss.tenantNodes[childName][:i], sss.tenantNodes[childName][i+1:]...) - } + checkedAllNodes := n.childrenChecked == len(n.queueOrder)+1 + if n.queuePosition == localQueueIndex { + return n, checkedAllNodes } - // check tenantNodes; to mirror tenantQuerierAssignments, we only remove from sharedQueueOrder - // if all nodes with this name are gone. If the child is at the _end_ of sharedQueueOrder, we - // remove it outright; otherwise, we replace it with an empty string so as not to reindex all - // slice elements - // TODO (casie): Mirrors tenantQuerierAssignments.removeTenant. Pull logic here? - - removeFromSharedQueueOrder := len(sss.tenantNodes[childName]) == 0 - - var positionNeedsUpdate bool - if removeFromSharedQueueOrder { - for idx, name := range sss.sharedQueueOrder { - if name == childName { - sss.sharedQueueOrder[idx] = "" - } - } - // clear all sequential empty elts from sharedQueueOrder - lastElementIndex := len(sss.sharedQueueOrder) - 1 - for i := lastElementIndex; i >= 0 && sss.sharedQueueOrder[i] == ""; i-- { - sss.sharedQueueOrder = sss.sharedQueueOrder[:i] - } + currentNodeName := n.queueOrder[n.queuePosition] + if node, ok := n.queueMap[currentNodeName]; ok { + return node, checkedAllNodes } - return positionNeedsUpdate + return nil, checkedAllNodes } -func (sss *shuffleShardState) dequeueOps() dequeueOps { - // start from sss.sharedQueuePosition - // check next tenant for querierID against availabilityMap - // if exists, move element to "back" of queue - // if doesn't exist, check next child - // nothing here to dequeue - getNode := func(n *Node) (*Node, bool) { - // can't get a tenant if no querier set - if sss.currentQuerier == nil { - return nil, true - } - - checkedAllNodes := n.childrenChecked == len(n.queueMap)+1 // must check local queue as well - - // advance queue position for dequeue - sss.sharedQueuePosition++ - if sss.sharedQueuePosition >= len(sss.sharedQueueOrder) { - sss.sharedQueuePosition = localQueueIndex - } - - // no children - if len(n.queueMap) == 0 || sss.sharedQueuePosition == localQueueIndex { - return n, checkedAllNodes - } - - checkIndex := sss.sharedQueuePosition - - for iters := 0; iters < len(sss.sharedQueueOrder); iters++ { - if checkIndex >= len(sss.sharedQueueOrder) { - checkIndex = 0 - } - tenantName := sss.sharedQueueOrder[checkIndex] - - if _, ok := n.queueMap[tenantName]; !ok { - // tenant not in _this_ node's children, move on - checkIndex++ - continue - } - - // increment nodes checked even if not in tenant-querier map - n.childrenChecked++ - checkedAllNodes = n.childrenChecked == len(n.queueMap)+1 - - // if the tenant-querier set is nil, any querier can serve this tenant - if sss.tenantQuerierMap[TenantID(tenantName)] == nil { - sss.sharedQueuePosition = checkIndex - return n.queueMap[tenantName], checkedAllNodes - } - if tenantQuerierSet, ok := sss.tenantQuerierMap[TenantID(tenantName)]; ok { - if _, ok := tenantQuerierSet[*sss.currentQuerier]; ok { - sss.sharedQueuePosition = checkIndex - return n.queueMap[tenantName], checkedAllNodes - } - } - checkIndex++ - } - return nil, checkedAllNodes - } - - updateState := func(n *Node, v any, deletedNode bool) { - if deletedNode { - // when we delete a child node, we want to step back in the queue so the - // previously-next tenant will still be next (e.g., if queueOrder is - // [childA, childB, childC] and childB (position 1) is deleted, we want to - // step back to position 0 so that in the next dequeue, position 1 (childC) - // is dequeued - fmt.Println("node deleted") - sss.sharedQueuePosition-- - fmt.Println(sss.sharedQueuePosition) - } - // we need to reset our checked nodes if we found a value to dequeue, or if we've checked all nodes - if v != nil { - n.childrenChecked = 0 - return - } - +func (rrs *roundRobinState) dequeueUpdateState(n *Node, v any, deletedNode bool) { + if v != nil { + n.childrenChecked = 0 + } else { n.childrenChecked++ - - // if dequeueNode was self, advance queue position no matter what, and return - if sss.sharedQueuePosition == localQueueIndex { - return - } - - // We never change the order of sharedQueueOrder; we expect sss.queuePosition to be updated - // to the last index used for the first dequeue, and hand it the result for the next one. - - // otherwise, don't change queuePosition, only move element to back - //for i, child := range sss.queueOrder { - // if child == nodeName { - // sss.queueOrder = append(sss.queueOrder[:i], append(sss.queueOrder[i+1:], child)...) // ugly - // n.queuePosition-- - // // wrap around to back of order if we go past the first element in n.queueOrder. - // // In the next dequeue, we will step forward again. - // if n.queuePosition <= localQueueIndex { - // n.queuePosition = len(sss.queueOrder) - 1 - // } - // } - //} } - - return dequeueOps{ - getNode: getNode, - updateState: updateState, + if deletedNode { + n.queuePosition-- + // if we try to go beyond something that would increment to + // the localQueueIndex or in queueOrder, we should wrap around to the + // back of queueOrder. + if n.queuePosition < localQueueIndex-1 { + n.queuePosition = len(n.queueOrder) - 1 + } } - -} - -func (sss *shuffleShardState) getQueueOrder(_ *Node) []string { - return sss.sharedQueueOrder } From 461b92daa8192126ff3f0b99979bd79c281a3327 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Wed, 5 Jun 2024 23:58:04 -0700 Subject: [PATCH 11/26] Port extant, relevant tree queue tests to new Tree, and finish ripping previous tree queue iteration --- pkg/scheduler/queue/queue_test.go | 4 +- .../queue/tenant_querier_assignment.go | 67 +- pkg/scheduler/queue/tenant_queues.go | 10 +- pkg/scheduler/queue/tenant_queues_test.go | 3 +- pkg/scheduler/queue/tree_queue.go | 416 ++++--- pkg/scheduler/queue/tree_queue_sketch.go | 220 ---- pkg/scheduler/queue/tree_queue_sketch_test.go | 598 ---------- pkg/scheduler/queue/tree_queue_test.go | 1012 ++++++++++++----- .../queue/tree_queueing_algorithms.go | 50 +- 9 files changed, 1017 insertions(+), 1363 deletions(-) delete mode 100644 pkg/scheduler/queue/tree_queue_sketch.go delete mode 100644 pkg/scheduler/queue/tree_queue_sketch_test.go diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index e9c90b00424..29dabb4ecdb 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -671,7 +671,7 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend require.Nil(t, queueBroker.queueTree.rootNode.getNode(QueuePath{"tenant-1"})) require.NoError(t, queueBroker.enqueueRequestBack(&tr, tenantMaxQueriers)) - require.False(t, queueBroker.queueTree.rootNode.getNode(QueuePath{"tenant-1"}).IsEmpty()) + require.False(t, queueBroker.queueTree.GetNode(QueuePath{"tenant-1"}).IsEmpty()) ctx, cancel := context.WithCancel(context.Background()) call := &waitingQuerierConn{ @@ -686,5 +686,5 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend // indicating not to re-submit a request for waitingQuerierConn for the querier require.True(t, queue.trySendNextRequestForQuerier(call)) // assert request was re-enqueued for tenant after failed send - require.False(t, queueBroker.queueTree.rootNode.getNode(QueuePath{"tenant-1"}).IsEmpty()) + require.False(t, queueBroker.queueTree.GetNode(QueuePath{"tenant-1"}).IsEmpty()) } diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 00d402eba8d..4f0407cd7d7 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -53,6 +53,8 @@ func (s querierIDSlice) Search(x QuerierID) int { return sort.Search(len(s), func(i int) bool { return s[i] >= x }) } +// tenantQuerierAssignments implements DequeueAlgorithm. In the context of a Tree, it maintains a mapping of +// tenants to queriers in order to support dequeueing from an appropriate tenant if shuffle-sharding is enabled. type tenantQuerierAssignments struct { // a tenant has many queriers // a tenant has *all* queriers if: @@ -318,6 +320,17 @@ func (tqa *tenantQuerierAssignments) shuffleTenantQueriers(tenantID TenantID, sc return true } +// dequeueGetNode chooses the next node to dequeue from based on tenantIDOrder and tenantOrderIndex, which are +// shared across all nodes to maintain an O(n) (where n = # tenants) time-to-dequeue for each tenant. +// If tenant order were maintained by individual nodes, we would end up with O(mn) (where m = # query components) +// time-to-dequeue for a given tenant. +// +// tenantOrderIndex is incremented, checks if the tenant at that index is a child of the current node (it may not be, +// e.g., in the case that a tenant has queries queued for one query component but not others), and if so, returns +// the tenant node if currentQuerier can handle queries for that tenant. +// +// Note that because we use the shared tenantIDOrder and tenantOrderIndex to manage the queue, we functionally +// ignore each Node's individual queueOrder and queuePosition. func (tqa *tenantQuerierAssignments) dequeueGetNode(n *Node) (*Node, bool) { // can't get a tenant if no querier set if tqa.currentQuerier == nil { @@ -372,29 +385,24 @@ func (tqa *tenantQuerierAssignments) dequeueGetNode(n *Node) (*Node, bool) { return nil, checkedAllNodes } -func (tqa *tenantQuerierAssignments) dequeueUpdateState(n *Node, v any, deletedNode bool) { - if deletedNode { - // when we delete a child node, we want to step back in the queue so the - // previously-next tenant will still be next (e.g., if queueOrder is - // [childA, childB, childC] and childB (position 1) is deleted, we want to - // step back to position 0 so that in the next dequeue, position 1 (childC) - // is dequeued - tqa.tenantOrderIndex-- - } - // we need to reset our checked nodes if we found a value to dequeue, or if we've checked all nodes +// dequeueUpdateState updates the node state to reflect the number of children that have been checked +// for dequeueable elements, so that dequeueGetNode will stop checking if it has checked all possible nodes. +func (tqa *tenantQuerierAssignments) dequeueUpdateState(n *Node, v any, _ bool) { + // we need to reset our checked nodes if we found a value to dequeue if v != nil { n.childrenChecked = 0 return } n.childrenChecked++ - - // if dequeueNode was self, advance queue position no matter what, and return - if tqa.tenantOrderIndex == localQueueIndex { - return - } } +// addChildNode adds a child to: +// - the node's own queueMap +// - tenantNodes, which maintains a slice of all nodes with the same name. tenantNodes is checked on node deletion +// to ensure that we only remove a tenant from tenantIDOrder if _all_ nodes with the same name have been removed. +// - tenantIDOrder iff the node did not already exist in tenantNodes or tenantIDOrder. addChildNode will place +// a new tenant in the first empty ("") element it finds in tenantIDOrder, or at the end if no empty elements exist. func (tqa *tenantQuerierAssignments) addChildNode(parent, child *Node) { childName := child.Name() _, tenantHasAnyQueue := tqa.tenantNodes[childName] @@ -404,15 +412,17 @@ func (tqa *tenantQuerierAssignments) addChildNode(parent, child *Node) { parent.queueMap[childName] = child tqa.tenantNodes[childName] = append(tqa.tenantNodes[childName], child) - // if child has any queue, it'll already be in tenantIDOrder, return without altering order + // if child has any queue, it should already be in tenantIDOrder, return without altering order if tenantHasAnyQueue { return } - // otherwise, add childNode to n.queueOrder before the current position, and - // update n.queuePosition to current element + // otherwise, replace the first empty element in n.tenantIDOrder with childName, or append to the end for i, elt := range tqa.tenantIDOrder { - // if there's a tenant with this name in tenantIDOrder already, move on + // if we encounter a tenant with this name in tenantIDOrder already, return without altering order. + // This is a weak check (childName could exist farther down tenantIDOrder, but be inserted here as well), + // but should be fine, since the only time we hit this case is when createOrUpdateTenant is called, which + // only happens immediately before enqueueing. if elt == TenantID(childName) { return } @@ -426,6 +436,11 @@ func (tqa *tenantQuerierAssignments) addChildNode(parent, child *Node) { } +// deleteChildNode removes a child from: +// - parent's queueMap, +// - tenantNodes +// - tenantIDOrder iff there are no other nodes by the same name in tenantNodes. If the child is at the end of +// tenantIDOrder, it is removed outright; otherwise, it is replaced with an empty ("") element func (tqa *tenantQuerierAssignments) deleteChildNode(parent, child *Node) bool { childName := child.Name() @@ -439,14 +454,11 @@ func (tqa *tenantQuerierAssignments) deleteChildNode(parent, child *Node) bool { } } - // check tenantNodes; we only remove from sharedQueueOrder - // if all nodes with this name are gone. If the child is at the _end_ of sharedQueueOrder, we - // remove it outright; otherwise, we replace it with an empty string so as not to reindex all - // slice elements - + // check tenantNodes; we only remove from tenantIDOrder if all nodes with this name are gone. + // If the removed child is at the _end_ of tenantIDOrder, we remove it outright; otherwise, + // we replace it with an empty string so as not to reindex all slice elements removeFromSharedQueueOrder := len(tqa.tenantNodes[childName]) == 0 - var positionNeedsUpdate bool if removeFromSharedQueueOrder { for idx, name := range tqa.tenantIDOrder { if string(name) == childName { @@ -459,5 +471,8 @@ func (tqa *tenantQuerierAssignments) deleteChildNode(parent, child *Node) bool { tqa.tenantIDOrder = tqa.tenantIDOrder[:i] } } - return positionNeedsUpdate + // The only time we remove elements from tenantIDOrder is when they are empty nodes, _and_ at the end of + // tenantIDOrder, so we will never remove a tenant from tenantIDOrder at a position before tenantOrderIndex. + // Thus, we will never need to decrement tenantOrderIndex. + return false } diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 83095cd515f..681f2e67aaf 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -24,8 +24,7 @@ type tenantRequest struct { // queueBroker encapsulates access to tenant queues for pending requests // and maintains consistency with the tenant-querier assignments type queueBroker struct { - tenantQueuesTree *TreeQueue - queueTree *Tree + queueTree *Tree tenantQuerierAssignments *tenantQuerierAssignments @@ -57,7 +56,6 @@ func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled boo panic(fmt.Sprintf("error creating the tree queue: %v", err)) } qb := &queueBroker{ - tenantQueuesTree: NewTreeQueue("root"), queueTree: tree, tenantQuerierAssignments: tqas, maxTenantQueueSize: maxTenantQueueSize, @@ -68,7 +66,7 @@ func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled boo } func (qb *queueBroker) isEmpty() bool { - return qb.queueTree.rootNode.IsEmpty() + return qb.queueTree.IsEmpty() } // enqueueRequestBack is the standard interface to enqueue requests for dispatch to queriers. @@ -141,7 +139,7 @@ func (qb *queueBroker) dequeueRequestForQuerier( qb.tenantQuerierAssignments.currentQuerier = &querierID qb.tenantQuerierAssignments.tenantOrderIndex = lastTenantIndex - queuePath, queueElement := qb.queueTree.rootNode.dequeue() + queuePath, queueElement := qb.queueTree.Dequeue() var request *tenantRequest var tenantID TenantID @@ -157,7 +155,7 @@ func (qb *queueBroker) dequeueRequestForQuerier( } // dequeue returns the full path including root, but getNode expects the path _from_ root - queueNodeAfterDequeue := qb.queueTree.rootNode.getNode(queuePath[1:]) + queueNodeAfterDequeue := qb.queueTree.rootNode.getNode(queuePath) if queueNodeAfterDequeue == nil && len(qb.tenantQuerierAssignments.tenantNodes[string(tenantID)]) == 0 { // queue node was deleted due to being empty after dequeue qb.tenantQuerierAssignments.removeTenant(tenantID) diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 1ce340e4772..064fe0d97a3 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -28,6 +28,7 @@ func (qb *queueBroker) enqueueObjectsForTests(tenantID TenantID, numObjects int) tenantID: tenantID, req: fmt.Sprintf("%v: object-%v", tenantID, i), }) + fmt.Println(i) if err != nil { return err } @@ -223,7 +224,7 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { // assert equal distribution of queue items between tenant node and 3 subnodes for _, v := range additionalQueueDimensions { queuePath := append(QueuePath{"tenant-1"}, v...) - assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), qb.queueTree.rootNode.getNode(queuePath).getLocalQueue().Len()) + assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), qb.queueTree.GetNode(queuePath).getLocalQueue().Len()) } // assert error received when hitting a tenant's enqueue limit, diff --git a/pkg/scheduler/queue/tree_queue.go b/pkg/scheduler/queue/tree_queue.go index 1cf136bdc08..c8465f97354 100644 --- a/pkg/scheduler/queue/tree_queue.go +++ b/pkg/scheduler/queue/tree_queue.go @@ -4,6 +4,7 @@ package queue import ( "container/list" + "fmt" ) type QueuePath []string //nolint:revive // disallows types beginning with package name @@ -11,282 +12,269 @@ type QueueIndex int //nolint:revive // disallows types beginning with packag const localQueueIndex = -1 -// TreeQueue is a hierarchical queue implementation with an arbitrary amount of child queues. +// Tree holds metadata and a pointer to the root node of a hierarchical queue implementation. +// The root Node maintains a localQueue and an arbitrary number of child nodes (which themselves +// may have local queues and children). Each Node in Tree uses a DequeueAlgorithm (determined by +// node depth) to determine dequeue order of that Node's subtree. // -// TreeQueue internally maintains round-robin fair queuing across all of its queue dimensions. // Each queuing dimension is modeled as a node in the tree, internally reachable through a QueuePath. // -// The QueuePath is an ordered array of strings describing the path through the tree to the node, -// which contains the FIFO local queue of all items enqueued for that queuing dimension. +// The QueuePath is an ordered array of strings describing the path from the tree root to a Node. +// In addition to child Nodes, each Node contains a local queue (FIFO) of items. // -// When dequeuing from a given node, the node will round-robin equally between dequeuing directly -// from its own local queue and dequeuing recursively from its list of child TreeQueues. -// No queue at a given level of the tree is dequeued from consecutively unless all others -// at the same level of the tree are empty down to the leaf node. -type TreeQueue struct { - // name of the tree node will be set to its segment of the queue path - name string - localQueue *list.List - currentChildQueueIndex int - childQueueOrder []string - childQueueMap map[string]*TreeQueue +// When dequeuing from a given node, a Node will use its DequeueAlgorithm to choose either itself +// or a child node to dequeue from recursively (i.e., a child Node will use its own DequeueAlgorithm +// to determine how to proceed). Tree will not dequeue from two different Nodes at the same depth +// consecutively, unless the previously-checked Node was empty down to the leaf node. +type Tree struct { + rootNode *Node + algosByDepth []DequeueAlgorithm } -func NewTreeQueue(name string) *TreeQueue { - return &TreeQueue{ - name: name, - localQueue: nil, - currentChildQueueIndex: localQueueIndex, - childQueueMap: nil, - childQueueOrder: nil, +func NewTree(dequeueAlgorithms ...DequeueAlgorithm) (*Tree, error) { + if len(dequeueAlgorithms) == 0 { + return nil, fmt.Errorf("cannot create a tree without defined dequeueing algorithms") } + root, err := newNode("root", 0, dequeueAlgorithms[0]) + if err != nil { + return nil, err + } + root.depth = 0 + return &Tree{ + rootNode: root, + algosByDepth: dequeueAlgorithms, + }, nil +} + +func (t *Tree) IsEmpty() bool { + return t.rootNode.IsEmpty() } -func (q *TreeQueue) IsEmpty() bool { +// Dequeue removes and returns an item from the front of the next appropriate Node in the Tree, as +// well as the path to the Node which that item was dequeued from. +// +// Either the root/self node or a child node is chosen according to the Node's DequeueAlgorithm. If +// the root node is chosen, an item will be dequeued from the front of its localQueue. If a child +// node is chosen, it is recursively dequeued from until a node selects its localQueue. +// +// Nodes that empty down to the leaf after being dequeued from (or which are found to be empty leaf +// nodes during the dequeue operation) are deleted as the recursion returns up the stack. This +// maintains structural guarantees relied upon to make IsEmpty() non-recursive. +func (t *Tree) Dequeue() (QueuePath, any) { + path, v := t.rootNode.dequeue() + // The returned node dequeue path includes the root node; exclude + // this so that the return path can be used if needed to enqueue. + return path[1:], v +} + +// EnqueueBackByPath enqueues an item in the back of the local queue of the node +// located at a given path through the tree; nodes for the path are created as needed. +// +// path is relative to the root node; providing a QueuePath beginning with "root" +// will create a child node of the root node which is also named "root." +func (t *Tree) EnqueueBackByPath(path QueuePath, v any) error { + return t.rootNode.enqueueBackByPath(t, path, v) +} + +// EnqueueFrontByPath enqueues an item in the front of the local queue of the Node +// located at a given path through the Tree; nodes for the path are created as needed. +// +// Enqueueing to the front is intended only for items which were first enqueued to the back +// and then dequeued after reaching the front. +// +// Re-enqueueing to the front is only intended for use in cases where a queue consumer +// fails to complete operations on the dequeued item, but failure is not yet final, and the +// operations should be retried by a subsequent queue consumer. A concrete example is when +// a queue consumer fails or disconnects for unrelated reasons while we are in the process +// of dequeueing a request for it. +// +// path must be relative to the root node; providing a QueuePath beginning with "root" +// will create a child node of root which is also named "root." +func (t *Tree) EnqueueFrontByPath(path QueuePath, v any) error { + return t.rootNode.enqueueFrontByPath(t, path, v) +} + +func (t *Tree) GetNode(path QueuePath) *Node { + return t.rootNode.getNode(path) +} + +// Node maintains node-specific information used to enqueue and dequeue to itself, such as a local +// queue, node depth, references to its children, and position in queue. +// Note that the tenantQuerierAssignments DequeueAlgorithm largely disregards Node's queueOrder and +// queuePosition, managing analogous state instead, because shuffle-sharding + fairness requirements +// necessitate input from the querier. +type Node struct { + name string + localQueue *list.List + queuePosition int // next index in queueOrder to dequeue from + queueOrder []string // order for dequeueing from self/children + queueMap map[string]*Node + depth int + dequeueAlgorithm DequeueAlgorithm + childrenChecked int +} + +func newNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { + if da == nil { + return nil, fmt.Errorf("cannot create a node without a defined dequeueing algorithm") + } + switch da.(type) { + case *tenantQuerierAssignments: + tqa := da.(*tenantQuerierAssignments) + tqa.tenantOrderIndex = localQueueIndex - 1 // start from -2 so that we first check local queue + if tqa.tenantNodes == nil { + tqa.tenantNodes = map[string][]*Node{} + } + } + return &Node{ + name: name, + localQueue: list.New(), + queuePosition: localQueueIndex, + queueOrder: make([]string, 0), + queueMap: make(map[string]*Node, 1), + depth: depth, + dequeueAlgorithm: da, + }, nil +} + +func (n *Node) IsEmpty() bool { // avoid recursion to make this a cheap operation // - // Because we dereference empty child nodes during dequeuing, + // Because we dereference empty child nodes during dequeueing, // we assume that emptiness means there are no child nodes // and nothing in this tree node's local queue. // // In reality a package member could attach empty child queues with getOrAddNode // in order to get a functionally-empty tree that would report false for IsEmpty. // We assume this does not occur or is not relevant during normal operation. - return q.LocalQueueLen() == 0 && len(q.childQueueMap) == 0 + return n.localQueue.Len() == 0 && len(n.queueMap) == 0 } -// NodeCount counts the TreeQueue node and all its children, recursively. -func (q *TreeQueue) NodeCount() int { - count := 1 // count self - for _, childQueue := range q.childQueueMap { - count += childQueue.NodeCount() +// ItemCount counts the queue items in the Node and in all its children, recursively. +func (n *Node) ItemCount() int { + items := n.localQueue.Len() + for _, child := range n.queueMap { + items += child.ItemCount() } - return count + return items } -// ItemCount counts the queue items in the TreeQueue node and in all its children, recursively. -func (q *TreeQueue) ItemCount() int { - count := q.LocalQueueLen() // count self - for _, childQueue := range q.childQueueMap { - count += childQueue.ItemCount() - } - return count +func (n *Node) Name() string { + return n.name } -func (q *TreeQueue) LocalQueueLen() int { - localQueueLen := 0 - if q.localQueue != nil { - localQueueLen = q.localQueue.Len() - } - return localQueueLen +func (n *Node) getLocalQueue() *list.List { + return n.localQueue } -// EnqueueBackByPath enqueues an item in the back of the local queue of the node -// located at a given path through the tree; nodes for the path are created as needed. -// -// childPath must be relative to the receiver node; providing a QueuePath beginning with -// the receiver/parent node name will create a child node of the same name as the parent. -func (q *TreeQueue) EnqueueBackByPath(childPath QueuePath, v any) error { - childQueue, err := q.getOrAddNode(childPath) +func (n *Node) enqueueFrontByPath(tree *Tree, pathFromNode QueuePath, v any) error { + childNode, err := n.getOrAddNode(pathFromNode, tree) if err != nil { return err } - - if childQueue.localQueue == nil { - childQueue.localQueue = list.New() - } - childQueue.localQueue.PushBack(v) + childNode.localQueue.PushFront(v) return nil } -// EnqueueFrontByPath enqueues an item in the front of the local queue of the node -// located at a given path through the tree; nodes for the path are created as needed. -// -// Max queue length check is skipped; enqueueing to the front is intended only for items -// which were first enqueued to the back and then dequeued after reaching the front. -// -// Re-enqueueing to the front is intended for cases where a queue consumer fails to -// complete operations on the dequeued item, but failure is not yet final, and the -// operations should be retried by a subsequent queue consumer. -// -// childPath must be relative to the receiver node; providing a QueuePath beginning with -// the receiver/parent node name will create a child node of the same name as the parent. -func (q *TreeQueue) EnqueueFrontByPath(childPath QueuePath, v any) error { - childQueue, err := q.getOrAddNode(childPath) +func (n *Node) enqueueBackByPath(tree *Tree, pathFromNode QueuePath, v any) error { + childNode, err := n.getOrAddNode(pathFromNode, tree) if err != nil { return err } - if childQueue.localQueue == nil { - childQueue.localQueue = list.New() - } - childQueue.localQueue.PushFront(v) + // TODO (casie): Create localQueue on node creation; why not? + childNode.localQueue.PushBack(v) return nil } -// getOrAddNode recursively adds tree queue nodes based on given relative child path. -// -// childPath must be relative to the receiver node; providing a QueuePath beginning with -// the receiver/parent node name will create a child node of the same name as the parent. -func (q *TreeQueue) getOrAddNode(childPath QueuePath) (*TreeQueue, error) { - if len(childPath) == 0 { - return q, nil - } +func (n *Node) dequeue() (QueuePath, any) { + var v any + var childPath QueuePath - if q.childQueueMap == nil { - q.childQueueMap = make(map[string]*TreeQueue, 1) - } + path := QueuePath{n.name} - var childQueue *TreeQueue - var ok bool - if childQueue, ok = q.childQueueMap[childPath[0]]; !ok { - // no child node matches next path segment - // create next child before recurring - childQueue = NewTreeQueue(childPath[0]) + if n.IsEmpty() { + return path, nil + } - // add new child queue to ordered list for round-robining; - // in order to maintain round-robin order as nodes are created and deleted, - // the new child queue should be inserted directly before the current child - // queue index, essentially placing the new node at the end of the line - if q.currentChildQueueIndex == localQueueIndex { - // special case; cannot slice into childQueueOrder with index -1 - // place at end of slice, which is the last slot before the local queue slot - q.childQueueOrder = append(q.childQueueOrder, childQueue.name) + var checkedAllNodes bool + var dequeueNode *Node + // continue until we've found a value or checked all nodes that need checking + for v == nil && !checkedAllNodes { + dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueGetNode(n) + var deletedNode bool + // dequeueing from local queue + if dequeueNode == n { + if n.localQueue.Len() > 0 { + // dequeueNode is self, local queue non-empty + if elt := n.localQueue.Front(); elt != nil { + n.localQueue.Remove(elt) + v = elt.Value + } + } + } else if dequeueNode == nil { + // no dequeue-able child found; reset checked children to 0, + // as we won't update state before moving on + n.childrenChecked = 0 + return path, v } else { - // insert into order behind current child queue index - q.childQueueOrder = append( - q.childQueueOrder[:q.currentChildQueueIndex], - append( - []string{childQueue.name}, - q.childQueueOrder[q.currentChildQueueIndex:]..., - )..., - ) - // update current child queue index to its new place in the expanded slice - q.currentChildQueueIndex++ + // dequeue from a child + childPath, v = dequeueNode.dequeue() + // if the dequeue node is empty _after_ dequeueing, delete it from children + if dequeueNode.IsEmpty() { + // removing an element sets our position one step forward; + // tell state to reset it to original queuePosition + delete(n.queueMap, dequeueNode.Name()) + deletedNode = n.dequeueAlgorithm.deleteChildNode(n, dequeueNode) + } } - - // attach new child queue to lookup map - q.childQueueMap[childPath[0]] = childQueue + n.dequeueAlgorithm.dequeueUpdateState(n, v, deletedNode) } - - return childQueue.getOrAddNode(childPath[1:]) + return append(path, childPath...), v } -func (q *TreeQueue) getNode(childPath QueuePath) *TreeQueue { - if len(childPath) == 0 { - return q +func (n *Node) getNode(pathFromNode QueuePath) *Node { + if len(pathFromNode) == 0 { + return n } - if q.childQueueMap == nil { + if n.queueMap == nil { return nil } - if childQueue, ok := q.childQueueMap[childPath[0]]; ok { - return childQueue.getNode(childPath[1:]) + if childQueue, ok := n.queueMap[pathFromNode[0]]; ok { + return childQueue.getNode(pathFromNode[1:]) } // no child node matches next path segment return nil } -// DequeueByPath selects a child node by a given relative child path and calls Dequeue on the node. -// -// While the child node will recursively clean up its own empty children during dequeue, -// nodes cannot delete themselves; DequeueByPath cleans up the child node as well if it is empty. -// This maintains structural guarantees relied on to make IsEmpty() non-recursive. +// getOrAddNode recursively gets or adds tree queue nodes based on given relative child path. It +// checks whether the first node in pathFromNode exists in the Node's children; if no node exists, +// one is created and added to the Node's queueOrder, according to the Node's DequeueAlgorithm. // -// childPath is relative to the receiver node; pass a zero-length path to refer to the node itself. -func (q *TreeQueue) DequeueByPath(childPath QueuePath) any { - childQueue := q.getNode(childPath) - if childQueue == nil { - return nil - } - - v := childQueue.Dequeue() - - if childQueue.IsEmpty() { - // child node will recursively clean up its own empty children during dequeue, - // but nodes cannot delete themselves; delete the empty child in order to - // maintain structural guarantees relied on to make IsEmpty() non-recursive - q.deleteNode(childPath) +// pathFromNode must be relative to the receiver node; providing a QueuePath beginning with +// the receiver/parent node name will create a child node of the same name as the parent. +func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *Tree) (*Node, error) { + if len(pathFromNode) == 0 { + return n, nil } - return v -} - -// Dequeue removes and returns an item from the front of the next nonempty queue node in the tree. -// -// Dequeuing from a node follows the round-robin order of the node's childQueueOrder, -// dequeuing either from the node's localQueue or selecting the next child node in the order -// and recursively calling Dequeue on the child nodes until a nonempty queue is found. -// -// Nodes that empty down to the leaf after being dequeued from are deleted as the recursion returns -// up the stack. This maintains structural guarantees relied on to make IsEmpty() non-recursive. -func (q *TreeQueue) Dequeue() any { - var v any - initialLen := len(q.childQueueOrder) - - for iters := 0; iters <= initialLen && v == nil; iters++ { - if q.currentChildQueueIndex == localQueueIndex { - // dequeuing from local queue; either we have: - // 1. reached a leaf node, or - // 2. reached an inner node when it is the local queue's turn - if q.localQueue != nil { - if elem := q.localQueue.Front(); elem != nil { - q.localQueue.Remove(elem) - v = elem.Value - } - } - q.wrapIndex(true) - } else { - // dequeuing from child queue node; - // pick the child node whose turn it is and recur - childQueueName := q.childQueueOrder[q.currentChildQueueIndex] - childQueue := q.childQueueMap[childQueueName] - v = childQueue.Dequeue() - - // perform cleanup if child node is empty after dequeuing recursively - if childQueue.IsEmpty() { - // deleteNode wraps index for us - q.deleteNode(QueuePath{childQueueName}) - } else { - q.wrapIndex(true) - } + var childNode *Node + var ok bool + var err error + if childNode, ok = n.queueMap[pathFromNode[0]]; !ok { + // child does not exist, create it + if n.depth+1 >= len(tree.algosByDepth) { + return nil, fmt.Errorf("cannot add a node beyond max tree depth: %v", len(tree.algosByDepth)) } - } - return v -} - -// deleteNode removes a child node from the tree and the childQueueOrder and corrects the indices. -func (q *TreeQueue) deleteNode(childPath QueuePath) bool { - if len(childPath) == 0 { - // node cannot delete itself - return false - } - - parentPath, childQueueName := childPath[:len(childPath)-1], childPath[len(childPath)-1] - - parentNode := q.getNode(parentPath) - if parentNode == nil { - // not found - return false - } - - delete(parentNode.childQueueMap, childQueueName) - for i, name := range parentNode.childQueueOrder { - if name == childQueueName { - parentNode.childQueueOrder = append(q.childQueueOrder[:i], q.childQueueOrder[i+1:]...) - parentNode.wrapIndex(false) - break + childNode, err = newNode(pathFromNode[0], n.depth+1, tree.algosByDepth[n.depth+1]) + if err != nil { + return nil, err } - } - return true -} + // add the newly created child to the node + n.dequeueAlgorithm.addChildNode(n, childNode) -func (q *TreeQueue) wrapIndex(increment bool) { - if increment { - q.currentChildQueueIndex++ - } - if q.currentChildQueueIndex >= len(q.childQueueOrder) { - q.currentChildQueueIndex = localQueueIndex } + return childNode.getOrAddNode(pathFromNode[1:], tree) } diff --git a/pkg/scheduler/queue/tree_queue_sketch.go b/pkg/scheduler/queue/tree_queue_sketch.go deleted file mode 100644 index 750b817d032..00000000000 --- a/pkg/scheduler/queue/tree_queue_sketch.go +++ /dev/null @@ -1,220 +0,0 @@ -package queue - -import ( - "container/list" - "fmt" -) - -// TODO (casie): We might not need this interface anymore. -type TreeNodeIFace interface { - //EnqueueFrontByPath(QueuePath, any) error - enqueueBackByPath(*Tree, QueuePath, any) error - dequeue() (QueuePath, any) // Dequeue returns the dequeued value, and the QueuePath (starting with root) to the node which was dequeued from - IsEmpty() bool - Name() string - getLocalQueue() *list.List - getOrAddNode(QueuePath, *Tree) (*Node, error) -} - -type Tree struct { - rootNode *Node - algosByDepth []DequeueAlgorithm -} - -// Node maintains tree-queue-node specific information, such as child order, and position in queue. -// Note that tenantQuerierAssignments largely disregards queueOrder and queuePosition, due to the -// way shuffle-sharding must be managed globally and on a per-querier basis -type Node struct { - name string - localQueue *list.List - queuePosition int // last queue position in queueOrder that was dequeued from - queueOrder []string // order for dequeueing from self/children - queueMap map[string]*Node - depth int - dequeueAlgorithm DequeueAlgorithm - childrenChecked int -} - -func NewTree(dequeueAlgorithms ...DequeueAlgorithm) (*Tree, error) { - if len(dequeueAlgorithms) == 0 { - return nil, fmt.Errorf("cannot create a tree without defined dequeueing algorithms") - } - root, err := newNode("root", 0, dequeueAlgorithms[0]) - if err != nil { - return nil, err - } - root.depth = 0 - return &Tree{ - rootNode: root, - algosByDepth: dequeueAlgorithms, - }, nil -} - -func (t *Tree) Dequeue() (QueuePath, any) { - return t.rootNode.dequeue() -} - -func (t *Tree) EnqueueBackByPath(path QueuePath, v any) error { - return t.rootNode.enqueueBackByPath(t, path, v) -} - -func (t *Tree) EnqueueFrontByPath(path QueuePath, v any) error { - return t.rootNode.enqueueFrontByPath(t, path, v) -} - -func newNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { - // TODO (casie): write a unit test - if da == nil { - return nil, fmt.Errorf("cannot create a node without a defined dequeueing algorithm") - } - switch da.(type) { - case *tenantQuerierAssignments: - tqa := da.(*tenantQuerierAssignments) - tqa.tenantOrderIndex = localQueueIndex - 1 - if tqa.tenantNodes == nil { - tqa.tenantNodes = map[string][]*Node{} - } - } - return &Node{ - name: name, - localQueue: list.New(), - queuePosition: localQueueIndex - 1, // start from -2 so that on first dequeue, localQueueIndex is dequeued from - queueOrder: make([]string, 0), - queueMap: make(map[string]*Node, 1), - depth: depth, - dequeueAlgorithm: da, - }, nil -} - -func (n *Node) IsEmpty() bool { - // node has nothing in local queue, and no child nodes - return n.localQueue.Len() == 0 && len(n.queueMap) == 0 -} - -func (n *Node) ItemCount() int { - items := n.localQueue.Len() - for _, child := range n.queueMap { - items += child.ItemCount() - } - return items -} - -func (n *Node) Name() string { - return n.name -} - -func (n *Node) getLocalQueue() *list.List { - return n.localQueue -} - -func (n *Node) enqueueFrontByPath(tree *Tree, pathFromNode QueuePath, v any) error { - childNode, err := n.getOrAddNode(pathFromNode, tree) - if err != nil { - return err - } - childNode.getLocalQueue().PushFront(v) - return nil -} - -func (n *Node) enqueueBackByPath(tree *Tree, pathFromNode QueuePath, v any) error { - childNode, err := n.getOrAddNode(pathFromNode, tree) - if err != nil { - return err - } - // TODO (casie): Create localQueue on node creation; why not? - childNode.getLocalQueue().PushBack(v) - return nil -} - -func (n *Node) dequeue() (QueuePath, any) { - var v any - var childPath QueuePath - - path := QueuePath{n.name} - - if n.IsEmpty() { - return path, nil - } - - var checkedAllNodes bool - var dequeueNode *Node - // continue until we've found a value or checked all nodes that need checking - for v == nil && !checkedAllNodes { - dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueGetNode(n) - var deletedNode bool - // dequeueing from local queue - if dequeueNode == n { - if n.localQueue.Len() > 0 { - // dequeueNode is self, local queue non-empty - if elt := n.localQueue.Front(); elt != nil { - n.localQueue.Remove(elt) - v = elt.Value - } - } - } else if dequeueNode == nil { - // no dequeue-able child found; reset checked children to 0, - // as we won't update state before moving on - n.childrenChecked = 0 - return path, v - } else { - // dequeue from a child - childPath, v = dequeueNode.dequeue() - // if the dequeue node is empty _after_ dequeueing, delete it from children - if dequeueNode.IsEmpty() { - // removing an element sets our position one step forward; - // tell state to reset it to original queuePosition - delete(n.queueMap, dequeueNode.Name()) - deletedNode = n.dequeueAlgorithm.deleteChildNode(n, dequeueNode) - } - } - n.dequeueAlgorithm.dequeueUpdateState(n, v, deletedNode) - } - return append(path, childPath...), v -} - -func (n *Node) getNode(pathFromNode QueuePath) *Node { - if len(pathFromNode) == 0 { - return n - } - - if n.queueMap == nil { - return nil - } - - if childQueue, ok := n.queueMap[pathFromNode[0]]; ok { - return childQueue.getNode(pathFromNode[1:]) - } - - // no child node matches next path segment - return nil -} - -// getOrAddNode checks whether the first node name in pathFromNode exists in a node's children; -// if no node exists, one is created and added to the "end" of the node's children, from the perspective -// of node.queuePosition (e.g., if node.queuePosition is 4, the new node will be created at position 3). -// This follows queue behavior, but if, for some reason, we ever want to create nodes in a way that violates -// queueing behavior, this will require a refactor, as will, well, the entire "tree queue" data structure. -func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *Tree) (*Node, error) { - if len(pathFromNode) == 0 { - return n, nil - } - - var childNode *Node - var ok bool - var err error - if childNode, ok = n.queueMap[pathFromNode[0]]; !ok { - // child does not exist, create it - if n.depth+1 >= len(tree.algosByDepth) { - return nil, fmt.Errorf("cannot add a node beyond max tree depth: %v", len(tree.algosByDepth)) - } - childNode, err = newNode(pathFromNode[0], n.depth+1, tree.algosByDepth[n.depth+1]) - if err != nil { - return nil, err - } - - n.dequeueAlgorithm.addChildNode(n, childNode) - - } - // TODO (casie): How to determine what dequeueAlgo to pass? - return childNode.getOrAddNode(pathFromNode[1:], tree) -} diff --git a/pkg/scheduler/queue/tree_queue_sketch_test.go b/pkg/scheduler/queue/tree_queue_sketch_test.go deleted file mode 100644 index 391b4ab6e19..00000000000 --- a/pkg/scheduler/queue/tree_queue_sketch_test.go +++ /dev/null @@ -1,598 +0,0 @@ -package queue - -import ( - //"fmt" - "github.com/stretchr/testify/require" - "testing" -) - -// TODO (casie): Write a test for dequeuing from tqa childA, enqueue to new childB, expect to dequeue next from childB -// TODO (casie): Write a test for enqueueFrontByPath() - -func newTenantQuerierAssignments() *tenantQuerierAssignments { - return &tenantQuerierAssignments{ - tenantQuerierIDs: make(map[TenantID]map[QuerierID]struct{}), - tenantNodes: make(map[string][]*Node), - } -} - -func Test_NewTree(t *testing.T) { - tests := []struct { - name string - treeAlgos []DequeueAlgorithm - state *tenantQuerierAssignments - expectErr bool - }{ - { - name: "create round-robin tree", - treeAlgos: []DequeueAlgorithm{&roundRobinState{}}, - }, - { - name: "create shuffle-shard tree", - treeAlgos: []DequeueAlgorithm{newTenantQuerierAssignments()}, - }, - { - name: "create shuffle-shard tree with no tenant-querier map, we should create an empty one", - treeAlgos: []DequeueAlgorithm{&tenantQuerierAssignments{}}, - }, - { - name: "fail to create tree without defined dequeueing algorithm", - treeAlgos: []DequeueAlgorithm{nil}, - expectErr: true, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - _, err := NewTree(tt.treeAlgos...) - if tt.expectErr { - require.Error(t, err) - } else { - require.NoError(t, err) - } - - }) - } -} - -func Test_EnqueueBackByPath(t *testing.T) { - tests := []struct { - name string - treeAlgosByDepth []DequeueAlgorithm - rootAlgo DequeueAlgorithm - children []QueuePath - expectErr bool - }{ - { - name: "enqueue round-robin node to round-robin node", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, - children: []QueuePath{{"round-robin-child-1"}}, - }, - { - name: "enqueue shuffle-shard node to round-robin node", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, - children: []QueuePath{{"shuffle-shard-child-1"}}, - }, - { - name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node, we should create an empty one", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{}}, - children: []QueuePath{{"shuffle-shard-child-1"}}, - }, - { - name: "enqueue round-robin node to shuffle-shard node", - treeAlgosByDepth: []DequeueAlgorithm{ - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - children: []QueuePath{{"round-robin-child-1"}}, - }, - { - name: "create tree with multiple shuffle-shard depths", - treeAlgosByDepth: []DequeueAlgorithm{ - newTenantQuerierAssignments(), - &roundRobinState{}, - newTenantQuerierAssignments(), - }, - children: []QueuePath{{"child"}, {"grandchild"}}, - }, - { - name: "enqueue beyond max-depth", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}}, - children: []QueuePath{{"child"}, {"child, grandchild"}}, - expectErr: true, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.treeAlgosByDepth...) - require.NoError(t, err) - - for _, childPath := range tt.children { - err = tree.EnqueueBackByPath(childPath, "some-object") - } - if tt.expectErr { - require.Error(t, err) - } else { - require.NoError(t, err) - } - }) - } -} - -func Test_Dequeue_RootNode(t *testing.T) { - tests := []struct { - name string - rootAlgo DequeueAlgorithm - enqueueToRoot []any - }{ - { - name: "dequeue from empty round-robin root node", - rootAlgo: &roundRobinState{}, - }, - { - name: "dequeue from empty shuffle-shard root node", - rootAlgo: newTenantQuerierAssignments(), - }, - { - name: "dequeue from non-empty round-robin root node", - rootAlgo: &roundRobinState{}, - enqueueToRoot: []any{"something-in-root"}, - }, - { - name: "dequeue from non-empty shuffle-shard root node", - rootAlgo: newTenantQuerierAssignments(), - enqueueToRoot: []any{"something-else-in-root"}, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - // TODO (casie): ugly - querierID := QuerierID("placeholder") - switch tt.rootAlgo.(type) { - case *tenantQuerierAssignments: - tt.rootAlgo.(*tenantQuerierAssignments).currentQuerier = &querierID - } - tree, err := NewTree(tt.rootAlgo) - require.NoError(t, err) - - for _, elt := range tt.enqueueToRoot { - err = tree.EnqueueBackByPath(QueuePath{}, elt) - require.NoError(t, err) - } - rootPath := QueuePath{"root"} - for _, elt := range tt.enqueueToRoot { - path, v := tree.Dequeue() - require.Equal(t, rootPath, path) - require.Equal(t, elt, v) - - } - - path, v := tree.Dequeue() - require.Equal(t, rootPath, path) - require.Nil(t, v) - - }) - } -} - -func Test_RoundRobinDequeue(t *testing.T) { - tests := []struct { - name string - selfQueueObjects []string - children []string - childQueueObjects map[string][]any - grandchildren []string - grandchildrenQueueObjects map[string]struct { - path QueuePath - objs []any - } - expected []string - }{ - { - name: "dequeue from round-robin child when local queue empty", - children: []string{"child-1"}, - childQueueObjects: map[string][]any{"child-1": {"child-1:some-object"}}, - expected: []string{"child-1:some-object"}, - }, - { - name: "dequeue from round-robin root when on node's turn", - selfQueueObjects: []string{"root:object-1", "root:object-2"}, - children: []string{"child-1"}, - childQueueObjects: map[string][]any{"child-1": {"child-1:object-1"}}, - expected: []string{"root:object-1", "child-1:object-1"}, - }, - { - name: "dequeue from second round-robin child when first child is empty", - children: []string{"child-1", "child-2"}, - childQueueObjects: map[string][]any{"child-1": {nil}, "child-2": {"child-2:some-object"}}, - expected: []string{"child-2:some-object"}, - }, - { - name: "dequeue from round-robin grandchild when non-empty", - children: []string{"child-1", "child-2"}, - childQueueObjects: map[string][]any{"child-1": {"child-1:object-1", "child-1:object-2"}, "child-2": {"child-2:object-1"}}, - expected: []string{"child-1:object-1", "child-2:object-1", "grandchild-1:object-1"}, - grandchildren: []string{"grandchild-1"}, - grandchildrenQueueObjects: map[string]struct { - path QueuePath - objs []any - }{"grandchild-1": { - path: QueuePath{"child-1", "grandchild-1"}, - objs: []any{"grandchild-1:object-1"}, - }}, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) - require.NoError(t, err) - - for _, sqo := range tt.selfQueueObjects { - err = tree.EnqueueBackByPath(QueuePath{}, sqo) - require.NoError(t, err) - } - - for _, child := range tt.children { - for _, obj := range tt.childQueueObjects[child] { - _ = tree.EnqueueBackByPath(QueuePath{child}, obj) - } - } - - for _, grandchild := range tt.grandchildren { - gqo := tt.grandchildrenQueueObjects[grandchild] - for _, obj := range gqo.objs { - err = tree.EnqueueBackByPath(gqo.path, obj) - require.NoError(t, err) - } - } - - for _, expected := range tt.expected { - _, val := tree.Dequeue() - v, ok := val.(string) - require.True(t, ok) - require.Equal(t, expected, v) - } - }) - } -} - -func Test_DequeueOrderAfterEnqueue(t *testing.T) { - type opType string - enqueue := opType("enqueue") - dequeue := opType("dequeue") - placeholderQuerier := QuerierID("") - - type op struct { - kind opType - path QueuePath - obj any - } - - tests := []struct { - name string - treeAlgosByDepth []DequeueAlgorithm - operationOrder []op - }{ - { - name: "should dequeue from new (next-in-queue) child immediately after it is added", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, - operationOrder: []op{ - {enqueue, QueuePath{"child-1"}, "obj-1"}, - {enqueue, QueuePath{"child-1"}, "obj-2"}, - {dequeue, QueuePath{"root", "child-1"}, "obj-1"}, - {enqueue, QueuePath{"child-2"}, "obj-3"}, - {dequeue, QueuePath{"root", "child-2"}, "obj-3"}, - {dequeue, QueuePath{"root", "child-1"}, "obj-2"}, - {dequeue, QueuePath{"root"}, nil}, - }, - }, - { - name: "should dequeue from new (next-in-queue) shuffle-shard child immediately after it is added", - treeAlgosByDepth: []DequeueAlgorithm{ - &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, - tenantNodes: map[string][]*Node{}, - currentQuerier: &placeholderQuerier, - }, - &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, - tenantNodes: map[string][]*Node{}, - currentQuerier: &placeholderQuerier, - }, - }, - operationOrder: []op{ - {enqueue, QueuePath{"child-1"}, "obj-1"}, - {enqueue, QueuePath{"child-1"}, "obj-2"}, - {dequeue, QueuePath{"root", "child-1"}, "obj-1"}, - {enqueue, QueuePath{"child-2"}, "obj-3"}, - {dequeue, QueuePath{"root", "child-2"}, "obj-3"}, - {dequeue, QueuePath{"root", "child-1"}, "obj-2"}, - {dequeue, QueuePath{"root"}, nil}, - }, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.treeAlgosByDepth...) - require.NoError(t, err) - root := tree.rootNode - - for _, operation := range tt.operationOrder { - if operation.kind == enqueue { - err = tree.EnqueueBackByPath(operation.path, operation.obj) - require.NoError(t, err) - } - if operation.kind == dequeue { - path, obj := root.dequeue() - require.Equal(t, operation.path, path) - require.Equal(t, operation.obj, obj) - } - } - }) - } -} - -func Test_ShuffleShardDequeue(t *testing.T) { - type enqueueObj struct { - obj any - path QueuePath - } - - tests := []struct { - name string - treeAlgosByDepth []DequeueAlgorithm - state *tenantQuerierAssignments - currQuerier []QuerierID - enqueueObjs []enqueueObj - expected []any - expectErr bool - }{ - { - name: "happy path - tenant found in tenant-querier map under first child", - treeAlgosByDepth: []DequeueAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - }, - expected: []any{"query-1"}, - }, - { - name: "tenant exists, but not for querier", - treeAlgosByDepth: []DequeueAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - }, - expected: []any{nil}, - }, - { - name: "1 of 3 tenants exist for querier", - treeAlgosByDepth: []DequeueAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, - {obj: "query-3", path: QueuePath{"query-component-1", "tenant-3"}}, - }, - expected: []any{"query-3", nil}, - }, - { - name: "tenant exists for querier on next parent node", - treeAlgosByDepth: []DequeueAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, - {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, - {obj: "query-4", path: QueuePath{"query-component-1", "tenant-3"}}, - }, - expected: []any{"query-4", "query-3", nil}, - }, - { - name: "2 of 3 tenants exist for querier", - treeAlgosByDepth: []DequeueAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, - {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, - }, - expected: []any{"query-1", "query-3", nil}, - }, - { - name: "root node is shuffle-shard node", - treeAlgosByDepth: []DequeueAlgorithm{ - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &roundRobinState{}, - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-2", "query-component-1"}}, - {obj: "query-2", path: QueuePath{"tenant-1", "query-component-2"}}, - {obj: "query-3", path: QueuePath{"tenant-1", "query-component-1"}}, - }, - expected: []any{"query-2", "query-3", nil}, - }, - { - name: "dequeueing for one querier returns nil, but does return for a different querier", - treeAlgosByDepth: []DequeueAlgorithm{ - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &roundRobinState{}, - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-2", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, - }, - expected: []any{nil, "query-1"}, - }, - { - name: "no querier set in state", - treeAlgosByDepth: []DequeueAlgorithm{ - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{""}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, - }, - expected: []any{nil}, - }, - { - // TODO (casie): also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) - name: "dequeue from a tenant with a nil tenant-querier map", - treeAlgosByDepth: []DequeueAlgorithm{ - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &roundRobinState{}, - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, - {obj: "query-2", path: QueuePath{"tenant-2", "query-component-1"}}, - {obj: "query-3", path: QueuePath{"tenant-3", "query-component-1"}}, - }, - expected: []any{"query-1", "query-2", "query-3"}, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - currentQuerier := QuerierID("placeholder") - tt.state.currentQuerier = ¤tQuerier - - // We need a reference to state in order to be able to - // update the state's currentQuerier. - for i, da := range tt.treeAlgosByDepth { - switch da.(type) { - case *tenantQuerierAssignments: - tqas := tt.treeAlgosByDepth[i].(*tenantQuerierAssignments) - tqas.tenantQuerierIDs = tt.state.tenantQuerierIDs - tqas.currentQuerier = tt.state.currentQuerier - } - } - - tree, err := NewTree(tt.treeAlgosByDepth...) - require.NoError(t, err) - - for _, o := range tt.enqueueObjs { - err = tree.EnqueueBackByPath(o.path, o.obj) - require.NoError(t, err) - } - // currQuerier at position i is used to dequeue the expected result at position i - require.Equal(t, len(tt.currQuerier), len(tt.expected)) - for i := 0; i < len(tt.expected); i++ { - currentQuerier = tt.currQuerier[i] - _, v := tree.Dequeue() - require.Equal(t, tt.expected[i], v) - } - }) - } - -} - -// This test is a little messy; I can clean it up, but it's meant to illustrate that we can update a state -// in tenantQuerierAssignments, and the tree dequeue behavior will adjust accordingly. -func Test_ChangeShuffleShardState(t *testing.T) { - tqa := tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}}, - } - - state := &tenantQuerierAssignments{ - tenantQuerierIDs: tqa.tenantQuerierIDs, - currentQuerier: nil, - } - - tree, err := NewTree(state, &roundRobinState{}, &roundRobinState{}) - require.NoError(t, err) - - err = tree.EnqueueBackByPath(QueuePath{"tenant-1", "query-component-1"}, "query-1") - err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-2") - err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-3") - err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-4") - require.NoError(t, err) - - querier1 := QuerierID("querier-1") - querier2 := QuerierID("querier-2") - querier3 := QuerierID("querier-3") - - // set state to querier-2 should dequeue query-2 - state.currentQuerier = &querier2 - _, v := tree.Dequeue() - require.Equal(t, "query-2", v) - - // update state to querier-1 should dequeue query-1 - state.currentQuerier = &querier1 - _, v = tree.Dequeue() - require.Equal(t, "query-1", v) - - // update tqa map to add querier-3 as assigned to tenant-2, then set state to querier-3 should dequeue query-3 - tqa.tenantQuerierIDs["tenant-2"]["querier-3"] = struct{}{} - state.currentQuerier = &querier3 - _, v = tree.Dequeue() - require.Equal(t, "query-3", v) - - // during reshuffle, we only ever reassign tenant values, we don't assign an entirely new map value - // to tenantQuerierIDs. Reassign tenant-2 to an empty map value , and query-4 should _not_ be dequeued - tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} - _, v = tree.Dequeue() - require.Nil(t, v) - -} diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index aa0859ca46b..3b19d3e13b9 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -5,78 +5,643 @@ package queue import ( "fmt" "strings" - "testing" + //"fmt" "github.com/stretchr/testify/require" + "testing" ) -// TestDequeueBalancedTree checks dequeuing behavior from a balanced tree. -// -// Dequeuing from a balanced tree allows the test to have a simple looped structures -// while running checks to ensure that round-robin order is respected. -func TestDequeueBalancedTree(t *testing.T) { - firstDimensions := []string{"0", "1", "2"} - secondDimensions := []string{"a", "b", "c"} - itemsPerDimension := 5 - root := makeBalancedTreeQueue(t, firstDimensions, secondDimensions, itemsPerDimension) - require.NotNil(t, root) +// TODO (casie): Write a test for dequeuing from tqa childA, enqueue to new childB, expect to dequeue next from childB +// TODO (casie): Write a test for enqueueFrontByPath() - count := 0 - // tree queue will fairly dequeue from all levels of the tree - rotationsBeforeRepeat := len(firstDimensions) * len(secondDimensions) - // track dequeued paths to ensure round-robin dequeuing does not repeat before expected - dequeuedPathCache := make([]QueuePath, rotationsBeforeRepeat) +func newTenantQuerierAssignments() *tenantQuerierAssignments { + return &tenantQuerierAssignments{ + tenantQuerierIDs: make(map[TenantID]map[QuerierID]struct{}), + tenantNodes: make(map[string][]*Node), + } +} - for !root.IsEmpty() { - v := root.Dequeue() - dequeuedPath := getChildPathFromQueueItem(v) +func Test_NewTree(t *testing.T) { + tests := []struct { + name string + treeAlgos []DequeueAlgorithm + state *tenantQuerierAssignments + expectErr bool + }{ + { + name: "create round-robin tree", + treeAlgos: []DequeueAlgorithm{&roundRobinState{}}, + }, + { + name: "create shuffle-shard tree", + treeAlgos: []DequeueAlgorithm{newTenantQuerierAssignments()}, + }, + { + name: "create shuffle-shard tree with no tenant-querier map, we should create an empty one", + treeAlgos: []DequeueAlgorithm{&tenantQuerierAssignments{}}, + }, + { + name: "fail to create tree without defined dequeueing algorithm", + treeAlgos: []DequeueAlgorithm{nil}, + expectErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + _, err := NewTree(tt.treeAlgos...) + if tt.expectErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } - // require dequeued path has not repeated before the expected number of rotations - require.NotContains(t, dequeuedPathCache, dequeuedPath) + }) + } +} - dequeuedPathCache = append(dequeuedPathCache[1:], dequeuedPath) - count++ +func Test_EnqueueBackByPath(t *testing.T) { + tests := []struct { + name string + treeAlgosByDepth []DequeueAlgorithm + rootAlgo DequeueAlgorithm + children []QueuePath + expectErr bool + }{ + { + name: "enqueue round-robin node to round-robin node", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, + children: []QueuePath{{"round-robin-child-1"}}, + }, + { + name: "enqueue shuffle-shard node to round-robin node", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, + children: []QueuePath{{"shuffle-shard-child-1"}}, + }, + { + name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node, we should create an empty one", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{}}, + children: []QueuePath{{"shuffle-shard-child-1"}}, + }, + { + name: "enqueue round-robin node to shuffle-shard node", + treeAlgosByDepth: []DequeueAlgorithm{ + newTenantQuerierAssignments(), + &roundRobinState{}, + }, + children: []QueuePath{{"round-robin-child-1"}}, + }, + { + name: "create tree with multiple shuffle-shard depths", + treeAlgosByDepth: []DequeueAlgorithm{ + newTenantQuerierAssignments(), + &roundRobinState{}, + newTenantQuerierAssignments(), + }, + children: []QueuePath{{"child"}, {"grandchild"}}, + }, + { + name: "enqueue beyond max-depth", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}}, + children: []QueuePath{{"child"}, {"child, grandchild"}}, + expectErr: true, + }, } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) - // count items enqueued to nodes at depth 1 - expectedFirstDimensionCount := len(firstDimensions) * itemsPerDimension - // count items enqueued to nodes at depth 2 - expectedSecondDimensionCount := len(firstDimensions) * len(secondDimensions) * itemsPerDimension + for _, childPath := range tt.children { + err = tree.EnqueueBackByPath(childPath, "some-object") + } + if tt.expectErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + }) + } +} + +func Test_Dequeue_RootNode(t *testing.T) { + tests := []struct { + name string + rootAlgo DequeueAlgorithm + enqueueToRoot []any + }{ + { + name: "dequeue from empty round-robin root node", + rootAlgo: &roundRobinState{}, + }, + { + name: "dequeue from empty shuffle-shard root node", + rootAlgo: newTenantQuerierAssignments(), + }, + { + name: "dequeue from non-empty round-robin root node", + rootAlgo: &roundRobinState{}, + enqueueToRoot: []any{"something-in-root"}, + }, + { + name: "dequeue from non-empty shuffle-shard root node", + rootAlgo: newTenantQuerierAssignments(), + enqueueToRoot: []any{"something-else-in-root"}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + // TODO (casie): ugly + querierID := QuerierID("placeholder") + switch tt.rootAlgo.(type) { + case *tenantQuerierAssignments: + tt.rootAlgo.(*tenantQuerierAssignments).currentQuerier = &querierID + } + tree, err := NewTree(tt.rootAlgo) + require.NoError(t, err) + + path := QueuePath{} + for _, elt := range tt.enqueueToRoot { + err = tree.EnqueueBackByPath(path, elt) + require.NoError(t, err) + } + + for _, elt := range tt.enqueueToRoot { + dequeuePath, v := tree.Dequeue() + require.Equal(t, path, dequeuePath) + require.Equal(t, elt, v) + + } + + dequeuePath, v := tree.Dequeue() + require.Equal(t, path, dequeuePath) + require.Nil(t, v) + + }) + } +} + +func Test_RoundRobinDequeue(t *testing.T) { + tests := []struct { + name string + selfQueueObjects []string + children []string + childQueueObjects map[string][]any + grandchildren []string + grandchildrenQueueObjects map[string]struct { + path QueuePath + objs []any + } + expected []string + }{ + { + name: "dequeue from round-robin child when local queue empty", + children: []string{"child-1"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:some-object"}}, + expected: []string{"child-1:some-object"}, + }, + { + name: "dequeue from round-robin root when on node's turn", + selfQueueObjects: []string{"root:object-1", "root:object-2"}, + children: []string{"child-1"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:object-1"}}, + expected: []string{"root:object-1", "child-1:object-1"}, + }, + { + name: "dequeue from second round-robin child when first child is empty", + children: []string{"child-1", "child-2"}, + childQueueObjects: map[string][]any{"child-1": {nil}, "child-2": {"child-2:some-object"}}, + expected: []string{"child-2:some-object"}, + }, + { + name: "dequeue from round-robin grandchild when non-empty", + children: []string{"child-1", "child-2"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:object-1", "child-1:object-2"}, "child-2": {"child-2:object-1"}}, + expected: []string{"child-1:object-1", "child-2:object-1", "grandchild-1:object-1"}, + grandchildren: []string{"grandchild-1"}, + grandchildrenQueueObjects: map[string]struct { + path QueuePath + objs []any + }{"grandchild-1": { + path: QueuePath{"child-1", "grandchild-1"}, + objs: []any{"grandchild-1:object-1"}, + }}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + + for _, sqo := range tt.selfQueueObjects { + err = tree.EnqueueBackByPath(QueuePath{}, sqo) + require.NoError(t, err) + } + + for _, child := range tt.children { + for _, obj := range tt.childQueueObjects[child] { + _ = tree.EnqueueBackByPath(QueuePath{child}, obj) + } + } + + for _, grandchild := range tt.grandchildren { + gqo := tt.grandchildrenQueueObjects[grandchild] + for _, obj := range gqo.objs { + err = tree.EnqueueBackByPath(gqo.path, obj) + require.NoError(t, err) + } + } + + for _, expected := range tt.expected { + _, val := tree.Dequeue() + v, ok := val.(string) + require.True(t, ok) + require.Equal(t, expected, v) + } + }) + } +} + +func Test_DequeueOrderAfterEnqueue(t *testing.T) { + type opType string + enqueue := opType("enqueue") + dequeue := opType("dequeue") + placeholderQuerier := QuerierID("") + + type op struct { + kind opType + path QueuePath + obj any + } + + tests := []struct { + name string + treeAlgosByDepth []DequeueAlgorithm + operationOrder []op + }{ + { + name: "round-robin node should dequeue from first child one more time after new node added", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, + operationOrder: []op{ + {enqueue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{}, nil}, + }, + }, + { + name: "should dequeue from new (next-in-queue) shuffle-shard child immediately after it is added", + treeAlgosByDepth: []DequeueAlgorithm{ + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, + tenantNodes: map[string][]*Node{}, + currentQuerier: &placeholderQuerier, + }, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, + tenantNodes: map[string][]*Node{}, + currentQuerier: &placeholderQuerier, + }, + }, + operationOrder: []op{ + {enqueue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{}, nil}, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) + + for _, operation := range tt.operationOrder { + fmt.Println(operation) + if operation.kind == enqueue { + err = tree.EnqueueBackByPath(operation.path, operation.obj) + require.NoError(t, err) + } + if operation.kind == dequeue { + path, obj := tree.Dequeue() + require.Equal(t, operation.path, path) + require.Equal(t, operation.obj, obj) + } + } + }) + } +} + +func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { + type enqueueObj struct { + obj any + path QueuePath + } + + tests := []struct { + name string + treeAlgosByDepth []DequeueAlgorithm + state *tenantQuerierAssignments + currQuerier []QuerierID + enqueueObjs []enqueueObj + expected []any + expectErr bool + }{ + { + name: "happy path - tenant found in tenant-querier map under first child", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + newTenantQuerierAssignments(), + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + }, + expected: []any{"query-1"}, + }, + { + name: "tenant exists, but not for querier", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + newTenantQuerierAssignments(), + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, + tenantNodes: map[string][]*Node{}, + }, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + }, + expected: []any{nil}, + }, + { + name: "1 of 3 tenants exist for querier", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + newTenantQuerierAssignments(), + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-1", "tenant-3"}}, + }, + expected: []any{"query-3", nil}, + }, + { + name: "tenant exists for querier on next parent node", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + newTenantQuerierAssignments(), + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, + {obj: "query-4", path: QueuePath{"query-component-1", "tenant-3"}}, + }, + expected: []any{"query-4", "query-3", nil}, + }, + { + name: "2 of 3 tenants exist for querier", + treeAlgosByDepth: []DequeueAlgorithm{ + &roundRobinState{}, + newTenantQuerierAssignments(), + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, + }, + expected: []any{"query-1", "query-3", nil}, + }, + { + name: "root node is shuffle-shard node", + treeAlgosByDepth: []DequeueAlgorithm{ + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}, + tenantNodes: map[string][]*Node{}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-2", "query-component-1"}}, + {obj: "query-2", path: QueuePath{"tenant-1", "query-component-2"}}, + {obj: "query-3", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{"query-2", "query-3", nil}, + }, + { + name: "dequeueing for one querier returns nil, but does return for a different querier", + treeAlgosByDepth: []DequeueAlgorithm{ + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + currQuerier: []QuerierID{"querier-2", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{nil, "query-1"}, + }, + { + name: "no querier set in state", + treeAlgosByDepth: []DequeueAlgorithm{ + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + currQuerier: []QuerierID{""}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{nil}, + }, + { + // TODO (casie): also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) + name: "dequeue from a tenant with a nil tenant-querier map", + treeAlgosByDepth: []DequeueAlgorithm{ + &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, + &roundRobinState{}, + &roundRobinState{}, + }, + state: &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}, + tenantNodes: map[string][]*Node{}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + {obj: "query-2", path: QueuePath{"tenant-2", "query-component-1"}}, + {obj: "query-3", path: QueuePath{"tenant-3", "query-component-1"}}, + }, + expected: []any{"query-1", "query-2", "query-3"}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + currentQuerier := QuerierID("placeholder") + tt.state.currentQuerier = ¤tQuerier + + // We need a reference to state in order to be able to + // update the state's currentQuerier. + for i, da := range tt.treeAlgosByDepth { + switch da.(type) { + case *tenantQuerierAssignments: + tqas := tt.treeAlgosByDepth[i].(*tenantQuerierAssignments) + tqas.tenantQuerierIDs = tt.state.tenantQuerierIDs + tqas.currentQuerier = tt.state.currentQuerier + } + } + + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) + + for _, o := range tt.enqueueObjs { + err = tree.EnqueueBackByPath(o.path, o.obj) + require.NoError(t, err) + } + // currQuerier at position i is used to dequeue the expected result at position i + require.Equal(t, len(tt.currQuerier), len(tt.expected)) + for i := 0; i < len(tt.expected); i++ { + currentQuerier = tt.currQuerier[i] + _, v := tree.Dequeue() + require.Equal(t, tt.expected[i], v) + } + }) + } + +} + +// Test_ChangeTenantQuerierAssignments illustrates that we can update a state in tenantQuerierAssignments, +// and the tree dequeue behavior will adjust accordingly. +func Test_ChangeTenantQuerierAssignments(t *testing.T) { + tqa := tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{ + "tenant-1": {"querier-1": {}}, + "tenant-2": {"querier-2": {}}, + }, + } + + state := &tenantQuerierAssignments{ + tenantQuerierIDs: tqa.tenantQuerierIDs, + currentQuerier: nil, + } + + tree, err := NewTree(state, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + + err = tree.EnqueueBackByPath(QueuePath{"tenant-1", "query-component-1"}, "query-1") + err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-2") + err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-3") + err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-4") + err = tree.EnqueueBackByPath(QueuePath{"tenant-3", "query-component-1"}, "query-5") + require.NoError(t, err) + + querier1 := QuerierID("querier-1") + querier2 := QuerierID("querier-2") + querier3 := QuerierID("querier-3") + + // set state to querier-2 should dequeue query-2 + state.currentQuerier = &querier2 + state.tenantOrderIndex = -1 + _, v := tree.Dequeue() + require.Equal(t, "query-2", v) + + // update state to querier-1 should dequeue query-1 + state.currentQuerier = &querier1 + state.tenantOrderIndex = -1 + _, v = tree.Dequeue() + require.Equal(t, "query-1", v) + + // update tqa map to add querier-3 as assigned to tenant-2, then set state to querier-3 should dequeue query-3 + tqa.tenantQuerierIDs["tenant-2"]["querier-3"] = struct{}{} + state.currentQuerier = &querier3 + state.tenantOrderIndex = -1 + _, v = tree.Dequeue() + require.Equal(t, "query-3", v) + + // during reshuffle, we only ever reassign tenant values, we don't assign an entirely new map value + // to tenantQuerierIDs. Reassign tenant-2 to an empty map value, and query-5 (tenant-3), which can be handled + // by any querier, should be dequeued, + tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} + _, v = tree.Dequeue() + require.Equal(t, "query-5", v) + + // then we should not be able to dequeue query-4 + tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} + _, v = tree.Dequeue() + require.Nil(t, v) - require.Equal(t, expectedFirstDimensionCount+expectedSecondDimensionCount, count) } -// TestDequeueByPathBalancedTree checks dequeuing behavior by path from a balanced tree. +// Test_DequeueBalancedRoundRobinTree checks dequeuing behavior from a balanced round-robin tree. // // Dequeuing from a balanced tree allows the test to have a simple looped structures // while running checks to ensure that round-robin order is respected. -func TestDequeueByPathBalancedTree(t *testing.T) { +func Test_DequeueBalancedRoundRobinTree(t *testing.T) { firstDimensions := []string{"0", "1", "2"} secondDimensions := []string{"a", "b", "c"} itemsPerDimension := 5 - root := makeBalancedTreeQueue(t, firstDimensions, secondDimensions, itemsPerDimension) - require.NotNil(t, root) + tree := makeBalancedRoundRobinTree(t, firstDimensions, secondDimensions, itemsPerDimension) + require.NotNil(t, tree) count := 0 - // tree queue will fairly dequeue from all levels of the tree below the path provided; - // dequeuing by path skips the top level the tree and rotates only through the - // second-layer subtrees of the first layer node selected by the queue path - rotationsBeforeRepeat := len(secondDimensions) + + // Tree will fairly dequeue from all levels of the tree + rotationsBeforeRepeat := len(firstDimensions) * len(secondDimensions) // track dequeued paths to ensure round-robin dequeuing does not repeat before expected dequeuedPathCache := make([]QueuePath, rotationsBeforeRepeat) - for _, firstDimName := range firstDimensions { - firstDimPath := QueuePath{firstDimName} - for root.getNode(firstDimPath) != nil { - v := root.DequeueByPath(firstDimPath) - dequeuedPath := getChildPathFromQueueItem(v) + for !tree.IsEmpty() { + dequeuedPath, _ := tree.Dequeue() - // require dequeued path has not repeated before the expected number of rotations - require.NotContains(t, dequeuedPathCache, dequeuedPath) + // require dequeued path has not repeated before the expected number of rotations + require.NotContains(t, dequeuedPathCache, dequeuedPath) - dequeuedPathCache = append(dequeuedPathCache[1:], dequeuedPath) - count++ - } + dequeuedPathCache = append(dequeuedPathCache[1:], dequeuedPath) + count++ } // count items enqueued to nodes at depth 1 @@ -85,268 +650,176 @@ func TestDequeueByPathBalancedTree(t *testing.T) { expectedSecondDimensionCount := len(firstDimensions) * len(secondDimensions) * itemsPerDimension require.Equal(t, expectedFirstDimensionCount+expectedSecondDimensionCount, count) + } -// TestDequeuePathUnbalancedTree checks dequeuing behavior from an unbalanced tree. +// Test_DequeueUnbalancedRoundRobinTree checks dequeuing behavior from an unbalanced tree. // // Assertions are done one by one to illustrate and check the behaviors of dequeuing from // an unbalanced tree, where the same node will be dequeued from twice if the node remains // nonempty while its sibling nodes have been exhausted and deleted from the tree. -func TestDequeueUnbalancedTree(t *testing.T) { - root := makeUnbalancedTreeQueue(t) +func Test_DequeueUnbalancedRoundRobinTree(t *testing.T) { + tree := makeUnbalancedRoundRobinTree(t) // dequeue from root until exhausted - v := root.Dequeue() + _, v := tree.Dequeue() require.Equal(t, "root:0:val0", v) - // root:0 and any subtrees are exhausted - path := QueuePath{"0"} - v = root.DequeueByPath(path) - require.Nil(t, v) - // root:0 was deleted - require.Nil(t, root.getNode(path)) - - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:1:val0", v) - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:2:0:val0", v) - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:1:0:val0", v) - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:2:1:val0", v) - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:1:val1", v) - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:2:0:val1", v) - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:1:0:val1", v) - // root:1 and any subtrees are exhausted - path = QueuePath{"1"} - v = root.DequeueByPath(path) - require.Nil(t, v) - // root:1 was deleted - require.Nil(t, root.getNode(path)) - - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:2:1:val1", v) - v = root.Dequeue() + _, v = tree.Dequeue() require.Equal(t, "root:2:1:val2", v) - // root:2 and any subtrees are exhausted - path = QueuePath{"2"} - v = root.DequeueByPath(path) - require.Nil(t, v) - // root:2 was deleted - require.Nil(t, root.getNode(path)) - // all items have been dequeued - require.Equal(t, 0, root.ItemCount()) - require.Equal(t, 1, root.NodeCount()) + require.Equal(t, 0, tree.rootNode.ItemCount()) + require.Equal(t, 1, tree.rootNode.nodeCount()) // require nothing in local or child queues - require.True(t, root.IsEmpty()) + require.True(t, tree.IsEmpty()) } -// TestDequeueByPathUnbalancedTree checks dequeuing behavior from an unbalanced tree by path. -// -// Assertions are done one by one to illustrate and check the behaviors of dequeuing from -// an unbalanced tree, where the same node will be dequeued from twice if the node remains -// nonempty while its sibling nodes have been exhausted and deleted from the tree. -func TestDequeueByPathUnbalancedTree(t *testing.T) { - root := makeUnbalancedTreeQueue(t) - - // dequeue from root:2 until exhausted - path := QueuePath{"2"} - v := root.DequeueByPath(path) - require.Equal(t, "root:2:0:val0", v) - - v = root.DequeueByPath(path) - require.Equal(t, "root:2:1:val0", v) - - v = root.DequeueByPath(path) - require.Equal(t, "root:2:0:val1", v) - - v = root.DequeueByPath(path) - require.Equal(t, "root:2:1:val1", v) - - v = root.DequeueByPath(path) - require.Equal(t, "root:2:1:val2", v) +func Test_EnqueueDuringDequeueRespectsRoundRobin(t *testing.T) { + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + require.NotNil(t, tree) - // root:2 is exhausted; - v = root.DequeueByPath(path) - require.Nil(t, v) - // root:2 and its two children root:2:0 and root:2:1 were deleted - require.Nil(t, root.getNode(path)) - require.Equal(t, 2, len(root.childQueueMap)) - require.Equal(t, 2, len(root.childQueueOrder)) - require.Equal(t, 4, root.NodeCount()) - // 5 of 10 items were dequeued - require.Equal(t, 5, root.ItemCount()) - - // dequeue from root:1 until exhausted - path = QueuePath{"1"} - v = root.DequeueByPath(path) - require.Equal(t, "root:1:val0", v) - - v = root.DequeueByPath(path) - require.Equal(t, "root:1:0:val0", v) - - v = root.DequeueByPath(path) - require.Equal(t, "root:1:val1", v) - - v = root.DequeueByPath(path) - require.Equal(t, "root:1:0:val1", v) - - // root:1 is exhausted; - v = root.DequeueByPath(path) - require.Nil(t, v) - // root:1 and its child root:1:0 were deleted - require.Nil(t, root.getNode(path)) - require.Equal(t, 1, len(root.childQueueMap)) - require.Equal(t, 1, len(root.childQueueOrder)) - require.Equal(t, 2, root.NodeCount()) - // 9 of 10 items have been dequeued - require.Equal(t, 1, root.ItemCount()) - - // dequeue from root:0 until exhausted - path = QueuePath{"0"} - v = root.DequeueByPath(path) - require.Equal(t, "root:0:val0", v) - - // root:0 is exhausted; - v = root.DequeueByPath(path) - require.Nil(t, v) - // root:0 was deleted - require.Nil(t, root.getNode(path)) - require.Equal(t, 0, len(root.childQueueMap)) - require.Equal(t, 0, len(root.childQueueOrder)) - require.Equal(t, 1, root.NodeCount()) - // 10 of 10 items have been dequeued - require.Equal(t, 0, root.ItemCount()) - require.Equal(t, 1, root.NodeCount()) - - // require nothing in local or child queues - require.True(t, root.IsEmpty()) -} - -func TestEnqueueDuringDequeueRespectsRoundRobin(t *testing.T) { - root := NewTreeQueue("root") + root := tree.rootNode cache := map[string]struct{}{} // enqueue two items to path root:0 childPath := QueuePath{"0"} - item := makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) + item := makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item = makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) // enqueue one item to path root:1 childPath = QueuePath{"1"} - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) + item = makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) // enqueue two items to path root:2 childPath = QueuePath{"2"} - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) + item = makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item = makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, []string{"0", "1", "2"}, root.childQueueOrder) + require.Equal(t, []string{"0", "1", "2"}, root.queueOrder) // dequeue first item - v := root.Dequeue() - dequeuedPath := getChildPathFromQueueItem(v) + dequeuedPath, _ := tree.Dequeue() require.Equal(t, QueuePath{"0"}, dequeuedPath) // dequeue second item; root:1 is now exhausted and deleted - v = root.Dequeue() - dequeuedPath = getChildPathFromQueueItem(v) + dequeuedPath, _ = tree.Dequeue() require.Equal(t, QueuePath{"1"}, dequeuedPath) require.Nil(t, root.getNode(QueuePath{"1"})) - require.Equal(t, []string{"0", "2"}, root.childQueueOrder) + require.Equal(t, []string{"0", "2"}, root.queueOrder) // dequeue third item - v = root.Dequeue() - dequeuedPath = getChildPathFromQueueItem(v) + dequeuedPath, _ = tree.Dequeue() require.Equal(t, QueuePath{"2"}, dequeuedPath) // root:1 was previously exhausted; root:0, then root:2 will be next in the rotation // here we insert something new into root:1 to test that it // does not jump the line in front of root:0 or root:2 - item = makeQueueItemForChildPath(root, QueuePath{"1"}, cache) - require.NoError(t, root.EnqueueBackByPath(QueuePath{"1"}, item)) + item = makeItemForChildQueue(root, QueuePath{"1"}, cache) + require.NoError(t, tree.EnqueueBackByPath(QueuePath{"1"}, item)) require.NotNil(t, root.getNode(QueuePath{"1"})) - require.Equal(t, []string{"0", "2", "1"}, root.childQueueOrder) + require.Equal(t, []string{"0", "2", "1"}, root.queueOrder) // dequeue fourth item; the newly-enqueued root:1 item // has not jumped the line in front of root:0 - v = root.Dequeue() - dequeuedPath = getChildPathFromQueueItem(v) + dequeuedPath, _ = tree.Dequeue() require.Equal(t, QueuePath{"0"}, dequeuedPath) // dequeue fifth item; the newly-enqueued root:1 item // has not jumped the line in front of root:2 - v = root.Dequeue() - dequeuedPath = getChildPathFromQueueItem(v) + dequeuedPath, _ = tree.Dequeue() require.Equal(t, QueuePath{"2"}, dequeuedPath) // dequeue sixth item; verifying the order 0->2->1 is being followed - v = root.Dequeue() - dequeuedPath = getChildPathFromQueueItem(v) + dequeuedPath, _ = tree.Dequeue() require.Equal(t, QueuePath{"1"}, dequeuedPath) // all items have been dequeued require.Equal(t, 0, root.ItemCount()) - require.Equal(t, 1, root.NodeCount()) + require.Equal(t, 1, root.nodeCount()) // require nothing in local or child queues - require.True(t, root.IsEmpty()) + require.True(t, tree.IsEmpty()) } -func TestNodeCannotDeleteItself(t *testing.T) { - root := NewTreeQueue("root") - require.False(t, root.deleteNode(QueuePath{})) - require.NotNil(t, root) +func Test_NodeCannotDeleteItself(t *testing.T) { + tests := []struct { + name string + nodeType DequeueAlgorithm + }{ + {"round robin", &roundRobinState{}}, + {"tenant querier assignment", &tenantQuerierAssignments{}}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(&roundRobinState{}) + require.NoError(t, err) + require.NotNil(t, tree) + + require.False(t, tree.rootNode.dequeueAlgorithm.deleteChildNode(tree.rootNode, tree.rootNode)) + require.NotNil(t, tree.rootNode) + }) + } } -func makeBalancedTreeQueue(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimensions int) *TreeQueue { - root := NewTreeQueue("root") - require.Equal(t, 1, root.NodeCount()) - require.Equal(t, 0, root.ItemCount()) +func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimension int) *Tree { + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + require.Equal(t, 1, tree.rootNode.nodeCount()) + require.Equal(t, 0, tree.rootNode.ItemCount()) cache := map[string]struct{}{} for _, firstDimName := range firstDimensions { - // insert first dimension local queue items - for k := 0; k < itemsPerDimensions; k++ { + for k := 0; k < itemsPerDimension; k++ { childPath := QueuePath{firstDimName} - item := makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) + item := makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) } for _, secondDimName := range secondDimensions { - // insert second dimension local queue items - for k := 0; k < itemsPerDimensions; k++ { + for k := 0; k < itemsPerDimension; k++ { childPath := QueuePath{firstDimName, secondDimName} - item := makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) + item := makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) } } } - - return root + return tree } -func makeUnbalancedTreeQueue(t *testing.T) *TreeQueue { +func makeUnbalancedRoundRobinTree(t *testing.T) *Tree { /* root ├── child0 @@ -373,85 +846,86 @@ func makeUnbalancedTreeQueue(t *testing.T) *TreeQueue { │ └── localQueue └── localQueue */ - root := NewTreeQueue("root") - require.Equal(t, 1, root.NodeCount()) - require.Equal(t, 0, root.ItemCount()) + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + require.Equal(t, 1, tree.rootNode.nodeCount()) + require.Equal(t, 0, tree.rootNode.ItemCount()) cache := map[string]struct{}{} // enqueue one item to root:0 childPath := QueuePath{"0"} - item := makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 2, root.NodeCount()) - require.Equal(t, 1, root.ItemCount()) + item := makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 2, tree.rootNode.nodeCount()) + require.Equal(t, 1, tree.rootNode.ItemCount()) // enqueue two items to root:1 childPath = QueuePath{"1"} - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 3, root.NodeCount()) - require.Equal(t, 2, root.ItemCount()) + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 3, tree.rootNode.nodeCount()) + require.Equal(t, 2, tree.rootNode.ItemCount()) - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 3, root.NodeCount()) - require.Equal(t, 3, root.ItemCount()) + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 3, tree.rootNode.nodeCount()) + require.Equal(t, 3, tree.rootNode.ItemCount()) // enqueue two items to root:1:0 childPath = QueuePath{"1", "0"} - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 4, root.NodeCount()) - require.Equal(t, 4, root.ItemCount()) + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 4, tree.rootNode.nodeCount()) + require.Equal(t, 4, tree.rootNode.ItemCount()) - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 4, root.NodeCount()) - require.Equal(t, 5, root.ItemCount()) + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 4, tree.rootNode.nodeCount()) + require.Equal(t, 5, tree.rootNode.ItemCount()) // enqueue two items to root:2:0 childPath = QueuePath{"2", "0"} - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 6, root.NodeCount()) - require.Equal(t, 6, root.ItemCount()) + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 6, tree.rootNode.nodeCount()) + require.Equal(t, 6, tree.rootNode.ItemCount()) - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 6, root.NodeCount()) - require.Equal(t, 7, root.ItemCount()) + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 6, tree.rootNode.nodeCount()) + require.Equal(t, 7, tree.rootNode.ItemCount()) // enqueue three items to root:2:1 childPath = QueuePath{"2", "1"} - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 7, root.NodeCount()) - require.Equal(t, 8, root.ItemCount()) - - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 7, root.NodeCount()) - require.Equal(t, 9, root.ItemCount()) - - item = makeQueueItemForChildPath(root, childPath, cache) - require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, 7, root.NodeCount()) - require.Equal(t, 10, root.ItemCount()) - - return root + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, tree.rootNode.nodeCount()) + require.Equal(t, 8, tree.rootNode.ItemCount()) + + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, tree.rootNode.nodeCount()) + require.Equal(t, 9, tree.rootNode.ItemCount()) + + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, tree.rootNode.nodeCount()) + require.Equal(t, 10, tree.rootNode.ItemCount()) + + return tree } -// makeQueueItemForChildPath constructs a queue item to match its enqueued path +// makeItemForChildQueue constructs a queue item to match its enqueued path // by joining the path components and appending an incrementing value for each path. // // e.g. for a tree named "root": // - childQueuePath{"1", "0"}'s first item will be "root:1:0:val0" // - childQueuePath{"1", "0"}'s second item will be "root:1:0:val1" -func makeQueueItemForChildPath( - treeQueue *TreeQueue, childPath QueuePath, cache map[string]struct{}, +func makeItemForChildQueue( + parent *Node, childPath QueuePath, cache map[string]struct{}, ) string { - path := append(QueuePath{treeQueue.name}, childPath...) + path := append(QueuePath{parent.name}, childPath...) i := 0 for { @@ -463,11 +937,3 @@ func makeQueueItemForChildPath( i++ } } - -// getChildPathFromQueueItem enables assertions on the queue path the item was dequeued from. -// Assertions require that the queue item value was generated via makeQueueItemForChildPath. -func getChildPathFromQueueItem(v any) QueuePath { - itemPath := strings.Split(v.(string), ":") - itemPathPrefix := itemPath[1 : len(itemPath)-1] // strip value from the end - return itemPathPrefix -} diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go index c5e5baa0d33..64f510d072b 100644 --- a/pkg/scheduler/queue/tree_queueing_algorithms.go +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -1,5 +1,10 @@ +// SPDX-License-Identifier: AGPL-3.0-only + package queue +// DequeueAlgorithm represents the set of operations specific to different approaches to dequeueing. It is applied +// at the layer-level -- every Node at the same depth in a Tree shares the same DequeueAlgorithm, including state +// that may be stored in a struct that implements DequeueAlgorithm. type DequeueAlgorithm interface { addChildNode(*Node, *Node) deleteChildNode(*Node, *Node) bool @@ -7,22 +12,23 @@ type DequeueAlgorithm interface { dequeueUpdateState(*Node, any, bool) } -type dequeueGetNodeFunc func(*Node) (*Node, bool) -type dequeueUpdateStateFunc func(*Node, any, bool) - -type dequeueOps struct { - getNode dequeueGetNodeFunc - updateState dequeueUpdateStateFunc -} - +// roundRobinState is the simplest type of DequeueAlgorithm; nodes which use this DequeueAlgorithm and are at +// the same depth in a Tree do not share any state. When children are added to these nodes, they are placed at +// the "end" of the order from the perspective of the node's current queuePosition (e.g., if queuePosition is 3, +// a new child will be placed at index 2). Children are dequeued from using a simple round-robin ordering; +// queuePosition is incremented on every dequeue. type roundRobinState struct { } +// addChildNode creates a new child Node and adds it to the "end" of the parent's queueOrder from the perspective +// of the parent's queuePosition. Thus, If queuePosition is localQueueIndex, the new child is added to the end of +// queueOrder. Otherwise, the new child is added at queuePosition - 1, and queuePosition is incremented to keep +// it pointed to the same child. func (rrs *roundRobinState) addChildNode(parent, child *Node) { // add childNode to n.queueMap parent.queueMap[child.Name()] = child - // add childNode to n.queueOrder before the current position, update n.queuePosition to current element + // add childNode to n.queueOrder before the next-to-dequeue position, update n.queuePosition to current element if parent.queuePosition <= localQueueIndex { parent.queueOrder = append(parent.queueOrder, child.Name()) } else { @@ -31,6 +37,8 @@ func (rrs *roundRobinState) addChildNode(parent, child *Node) { } } +// deleteChildNode removes a child node from a parent's children, and returns whether a child was found in the +// parent's queueOrder. This allows us to adjust the parent's queuePosition if necessary. func (rrs *roundRobinState) deleteChildNode(parent, child *Node) bool { var childFound bool childName := child.Name() @@ -44,13 +52,9 @@ func (rrs *roundRobinState) deleteChildNode(parent, child *Node) bool { return childFound } +// dequeueGetNode returns the node at the node's queuePosition. queuePosition represents the position of +// the next node to dequeue from, and is incremented in dequeueUpdateState. func (rrs *roundRobinState) dequeueGetNode(n *Node) (*Node, bool) { - // advance the queue position for this dequeue - n.queuePosition++ - if n.queuePosition >= len(n.queueOrder) { - n.queuePosition = localQueueIndex - } - checkedAllNodes := n.childrenChecked == len(n.queueOrder)+1 if n.queuePosition == localQueueIndex { return n, checkedAllNodes @@ -63,19 +67,19 @@ func (rrs *roundRobinState) dequeueGetNode(n *Node) (*Node, bool) { return nil, checkedAllNodes } +// dequeueUpdateState increments queuePosition based on whether a child node was deleted during dequeue, +// and updates childrenChecked to reflect the number of children checked so that the dequeueGetNode operation +// will stop once it has checked all possible nodes. func (rrs *roundRobinState) dequeueUpdateState(n *Node, v any, deletedNode bool) { if v != nil { n.childrenChecked = 0 } else { n.childrenChecked++ } - if deletedNode { - n.queuePosition-- - // if we try to go beyond something that would increment to - // the localQueueIndex or in queueOrder, we should wrap around to the - // back of queueOrder. - if n.queuePosition < localQueueIndex-1 { - n.queuePosition = len(n.queueOrder) - 1 - } + if !deletedNode { + n.queuePosition++ + } + if n.queuePosition >= len(n.queueOrder) { + n.queuePosition = localQueueIndex } } From d8a1330fd97f73f034f9863f258d89ed223a8fc7 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Fri, 7 Jun 2024 11:45:44 -0700 Subject: [PATCH 12/26] Add config to flip tree to query component -> tenant --- pkg/frontend/v1/frontend.go | 1 + pkg/scheduler/queue/queue.go | 5 ++++- pkg/scheduler/queue/queue_test.go | 11 +++++++--- pkg/scheduler/queue/tenant_queues.go | 25 ++++++++++++++++++----- pkg/scheduler/queue/tenant_queues_test.go | 17 ++++++++------- pkg/scheduler/scheduler.go | 2 ++ 6 files changed, 43 insertions(+), 18 deletions(-) diff --git a/pkg/frontend/v1/frontend.go b/pkg/frontend/v1/frontend.go index a804b95057c..0562aadea43 100644 --- a/pkg/frontend/v1/frontend.go +++ b/pkg/frontend/v1/frontend.go @@ -128,6 +128,7 @@ func New(cfg Config, limits Limits, log log.Logger, registerer prometheus.Regist log, cfg.MaxOutstandingPerTenant, false, + false, cfg.QuerierForgetDelay, f.queueLength, f.discardedRequests, diff --git a/pkg/scheduler/queue/queue.go b/pkg/scheduler/queue/queue.go index 93a0fcea7fc..58d78211488 100644 --- a/pkg/scheduler/queue/queue.go +++ b/pkg/scheduler/queue/queue.go @@ -121,6 +121,7 @@ type RequestQueue struct { // settings maxOutstandingPerTenant int additionalQueueDimensionsEnabled bool + prioritizeQueryComponents bool forgetDelay time.Duration // metrics for reporting @@ -174,6 +175,7 @@ func NewRequestQueue( log log.Logger, maxOutstandingPerTenant int, additionalQueueDimensionsEnabled bool, + prioritizeQueryComponents bool, forgetDelay time.Duration, queueLength *prometheus.GaugeVec, discardedRequests *prometheus.CounterVec, @@ -190,6 +192,7 @@ func NewRequestQueue( log: log, maxOutstandingPerTenant: maxOutstandingPerTenant, additionalQueueDimensionsEnabled: additionalQueueDimensionsEnabled, + prioritizeQueryComponents: prioritizeQueryComponents, forgetDelay: forgetDelay, // metrics for reporting @@ -210,7 +213,7 @@ func NewRequestQueue( waitingQuerierConnsToDispatch: list.New(), QueryComponentUtilization: queryComponentCapacity, - queueBroker: newQueueBroker(maxOutstandingPerTenant, additionalQueueDimensionsEnabled, forgetDelay), + queueBroker: newQueueBroker(maxOutstandingPerTenant, additionalQueueDimensionsEnabled, prioritizeQueryComponents, forgetDelay), } q.Service = services.NewBasicService(q.starting, q.running, q.stop).WithName("request queue") diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index 29dabb4ecdb..c7a95ac3a0a 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -133,6 +133,7 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { log.NewNopLogger(), maxOutstandingRequestsPerTenant, additionalQueueDimensionsEnabled, + false, forgetQuerierDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -239,6 +240,7 @@ func BenchmarkConcurrentQueueOperations(b *testing.B) { log.NewNopLogger(), maxOutstandingRequestsPerTenant, true, + false, forgetQuerierDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -411,7 +413,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldGetRequestAfterReshardingBe queue, err := NewRequestQueue( log.NewNopLogger(), - 1, true, + 1, true, false, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -480,7 +482,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ReshardNotifiedCorrectlyForMultip queue, err := NewRequestQueue( log.NewNopLogger(), - 1, true, + 1, true, false, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -565,6 +567,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldReturnAfterContextCancelled log.NewNopLogger(), 1, true, + false, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -616,6 +619,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldReturnImmediatelyIfQuerierI log.NewNopLogger(), 1, true, + false, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -645,6 +649,7 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend log.NewNopLogger(), 1, true, + false, // TODO (casie): Write tests for when this is true forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -655,7 +660,7 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend // bypassing queue dispatcher loop for direct usage of the queueBroker and // passing a waitingQuerierConn for a canceled querier connection - queueBroker := newQueueBroker(queue.maxOutstandingPerTenant, queue.additionalQueueDimensionsEnabled, queue.forgetDelay) + queueBroker := newQueueBroker(queue.maxOutstandingPerTenant, queue.additionalQueueDimensionsEnabled, queue.prioritizeQueryComponents, queue.forgetDelay) queueBroker.addQuerierConnection(querierID) tenantMaxQueriers := 0 // no sharding diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 681f2e67aaf..15af9e420a0 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -32,7 +32,12 @@ type queueBroker struct { additionalQueueDimensionsEnabled bool } -func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled bool, forgetDelay time.Duration) *queueBroker { +func newQueueBroker( + maxTenantQueueSize int, + additionalQueueDimensionsEnabled bool, + prioritizeQueryComponents bool, + forgetDelay time.Duration, +) *queueBroker { currentQuerier := QuerierID("") tqas := &tenantQuerierAssignments{ queriersByID: map[QuerierID]*querierConn{}, @@ -45,12 +50,22 @@ func newQueueBroker(maxTenantQueueSize int, additionalQueueDimensionsEnabled boo tenantOrderIndex: localQueueIndex - 1, } - // TODO (casie): Maybe set this using a flag, so we can also change how we build queue path accordingly + // If prioritizeQueryComponents is true, the queueTree will be created + // with query components at one level above tenants; if it is false, + // tenant nodes will each maintain their own query component subtree. tree, err := NewTree( tqas, // root - &roundRobinState{}, // tenants + &roundRobinState{}, // tenant queues &roundRobinState{}, // query components ) + if prioritizeQueryComponents { + tree, err = NewTree( + &roundRobinState{}, // root + tqas, // query components + &roundRobinState{}, // tenant queues + ) + } + // An error building the tree is fatal; we must panic if err != nil { panic(fmt.Sprintf("error creating the tree queue: %v", err)) @@ -89,7 +104,7 @@ func (qb *queueBroker) enqueueRequestBack(request *tenantRequest, tenantMaxQueri } } - err = qb.queueTree.rootNode.enqueueBackByPath(qb.queueTree, queuePath, request) + err = qb.queueTree.EnqueueBackByPath(queuePath, request) return err } @@ -108,7 +123,7 @@ func (qb *queueBroker) enqueueRequestFront(request *tenantRequest, tenantMaxQuer if err != nil { return err } - return qb.queueTree.rootNode.enqueueFrontByPath(qb.queueTree, queuePath, request) + return qb.queueTree.EnqueueFrontByPath(queuePath, request) } func (qb *queueBroker) makeQueuePath(request *tenantRequest) (QueuePath, error) { diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 064fe0d97a3..0f9ed773147 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -24,11 +24,10 @@ import ( func (qb *queueBroker) enqueueObjectsForTests(tenantID TenantID, numObjects int) error { for i := 0; i < numObjects; i++ { - err := qb.queueTree.rootNode.enqueueBackByPath(qb.queueTree, QueuePath{string(tenantID)}, &tenantRequest{ + err := qb.queueTree.EnqueueBackByPath(QueuePath{string(tenantID)}, &tenantRequest{ tenantID: tenantID, req: fmt.Sprintf("%v: object-%v", tenantID, i), }) - fmt.Println(i) if err != nil { return err } @@ -63,7 +62,7 @@ func assertExpectedValuesOnDequeue(t *testing.T, qb *queueBroker, lastTenantInde } func TestQueues(t *testing.T) { - qb := newQueueBroker(0, true, 0) + qb := newQueueBroker(0, true, false, 0) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -194,7 +193,7 @@ func TestQueues(t *testing.T) { func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { maxTenantQueueSize := 100 - qb := newQueueBroker(maxTenantQueueSize, true, 0) + qb := newQueueBroker(maxTenantQueueSize, true, false, 0) additionalQueueDimensions := map[int][]string{ 0: nil, 1: {"ingester"}, @@ -255,7 +254,7 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { } func TestQueuesOnTerminatingQuerier(t *testing.T) { - qb := newQueueBroker(0, true, 0) + qb := newQueueBroker(0, true, false, 0) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -326,7 +325,7 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { } func TestQueuesWithQueriers(t *testing.T) { - qb := newQueueBroker(0, true, 0) + qb := newQueueBroker(0, true, false, 0) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -402,7 +401,7 @@ func TestQueuesConsistency(t *testing.T) { for testName, testData := range tests { t.Run(testName, func(t *testing.T) { - qb := newQueueBroker(0, true, testData.forgetDelay) + qb := newQueueBroker(0, true, false, testData.forgetDelay) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -453,7 +452,7 @@ func TestQueues_ForgetDelay(t *testing.T) { ) now := time.Now() - qb := newQueueBroker(0, true, forgetDelay) + qb := newQueueBroker(0, true, false, forgetDelay) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -550,7 +549,7 @@ func TestQueues_ForgetDelay_ShouldCorrectlyHandleQuerierReconnectingBeforeForget ) now := time.Now() - qb := newQueueBroker(0, true, forgetDelay) + qb := newQueueBroker(0, true, false, forgetDelay) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) diff --git a/pkg/scheduler/scheduler.go b/pkg/scheduler/scheduler.go index a8511fd432c..d31219a667a 100644 --- a/pkg/scheduler/scheduler.go +++ b/pkg/scheduler/scheduler.go @@ -95,6 +95,7 @@ type connectedFrontend struct { type Config struct { MaxOutstandingPerTenant int `yaml:"max_outstanding_requests_per_tenant"` AdditionalQueryQueueDimensionsEnabled bool `yaml:"additional_query_queue_dimensions_enabled" category:"experimental"` + PrioritizeQueryComponentsEnabled bool `yaml:"prioritize_query_components_enabled" category:"experimental"` QuerierForgetDelay time.Duration `yaml:"querier_forget_delay" category:"experimental"` GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config" doc:"description=This configures the gRPC client used to report errors back to the query-frontend."` @@ -160,6 +161,7 @@ func NewScheduler(cfg Config, limits Limits, log log.Logger, registerer promethe s.log, cfg.MaxOutstandingPerTenant, cfg.AdditionalQueryQueueDimensionsEnabled, + cfg.PrioritizeQueryComponentsEnabled, cfg.QuerierForgetDelay, s.queueLength, s.discardedRequests, From abc4ab04865f0fb06e38f3f06a7b56886e6e9d01 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 10 Jun 2024 15:10:50 -0700 Subject: [PATCH 13/26] Add EnqueueFrontByPath tests, makeQueuePath based on prioritizeQueryComponents config --- pkg/scheduler/queue/queue_test.go | 15 +++--- pkg/scheduler/queue/tenant_queues.go | 5 ++ pkg/scheduler/queue/tenant_queues_test.go | 7 --- pkg/scheduler/queue/tree_queue.go | 1 - pkg/scheduler/queue/tree_queue_test.go | 60 ++++++++++++++++++++--- 5 files changed, 65 insertions(+), 23 deletions(-) diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index c7a95ac3a0a..f222a31f159 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -29,6 +29,8 @@ import ( util_test "github.com/grafana/mimir/pkg/util/test" ) +// TODO (casie): Write tests for prioritizeQueryComponents is true + func TestMain(m *testing.M) { util_test.VerifyNoLeakTestMain(m) } @@ -110,10 +112,7 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { normalQueueDimensionFunc := func() []string { return []string{normalQueueDimension} } slowQueueDimensionFunc := func() []string { return []string{slowConsumerQueueDimension} } - additionalQueueDimensionsEnabledCases := []bool{ - false, - true, - } + additionalQueueDimensionsEnabledCases := []bool{false, true} queueDurationTotals := map[bool]map[string]float64{ false: {normalQueueDimension: 0.0, slowConsumerQueueDimension: 0.0}, true: {normalQueueDimension: 0.0, slowConsumerQueueDimension: 0.0}, @@ -188,8 +187,7 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { for consumerIdx := 0; consumerIdx < numConsumers; consumerIdx++ { consumerIdx := consumerIdx queueConsumerErrGroup.Go(func() error { - returnVal := runConsumer(consumerIdx) - return returnVal + return runConsumer(consumerIdx) }) } @@ -367,7 +365,7 @@ func runQueueConsumerIters( start chan struct{}, consumeFunc consumeRequest, ) func(consumerIdx int) error { - returnFunc := func(consumerIdx int) error { + return func(consumerIdx int) error { consumerIters := queueActorIterationCount(totalIters, numConsumers, consumerIdx) lastTenantIndex := FirstTenant() querierID := fmt.Sprintf("consumer-%v", consumerIdx) @@ -387,7 +385,6 @@ func runQueueConsumerIters( return nil } - return returnFunc } type consumeRequest func(request Request) error @@ -649,7 +646,7 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend log.NewNopLogger(), 1, true, - false, // TODO (casie): Write tests for when this is true + false, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 15af9e420a0..ad1ac7d9f6d 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -30,6 +30,7 @@ type queueBroker struct { maxTenantQueueSize int additionalQueueDimensionsEnabled bool + prioritizeQueryComponents bool } func newQueueBroker( @@ -75,6 +76,7 @@ func newQueueBroker( tenantQuerierAssignments: tqas, maxTenantQueueSize: maxTenantQueueSize, additionalQueueDimensionsEnabled: additionalQueueDimensionsEnabled, + prioritizeQueryComponents: prioritizeQueryComponents, } return qb @@ -129,6 +131,9 @@ func (qb *queueBroker) enqueueRequestFront(request *tenantRequest, tenantMaxQuer func (qb *queueBroker) makeQueuePath(request *tenantRequest) (QueuePath, error) { if qb.additionalQueueDimensionsEnabled { if schedulerRequest, ok := request.req.(*SchedulerRequest); ok { + if qb.prioritizeQueryComponents { + return append(schedulerRequest.AdditionalQueueDimensions, string(request.tenantID)), nil + } return append(QueuePath{string(request.tenantID)}, schedulerRequest.AdditionalQueueDimensions...), nil } } diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 0f9ed773147..c2b29a3bc42 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -17,11 +17,6 @@ import ( "time" ) -// TODO (casie): Add a test case where, if a tenant is queued in the queueTree, but doesn't exist from the queueBroker perspective, -// it should never be dequeued. - -// TODO (casie): maybe implement a way to fetch an entire node that _would_ be dequeued from, rather than just one elt - func (qb *queueBroker) enqueueObjectsForTests(tenantID TenantID, numObjects int) error { for i := 0; i < numObjects; i++ { err := qb.queueTree.EnqueueBackByPath(QueuePath{string(tenantID)}, &tenantRequest{ @@ -283,8 +278,6 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { } qOneLastTenantIndex := assertExpectedValuesOnDequeue(t, qb, -1, "querier-1", expectedDequeueVals) - // TODO (casie): This is consistent with the previous test only because the previous test - // checked n % numTenants == 0 dequeues. expectedDequeueVals = []dequeueVal{ {buildExpectedObject(tenantOne.tenantID, 2), tenantOne}, {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, diff --git a/pkg/scheduler/queue/tree_queue.go b/pkg/scheduler/queue/tree_queue.go index c8465f97354..1cd7fd9891f 100644 --- a/pkg/scheduler/queue/tree_queue.go +++ b/pkg/scheduler/queue/tree_queue.go @@ -181,7 +181,6 @@ func (n *Node) enqueueBackByPath(tree *Tree, pathFromNode QueuePath, v any) erro if err != nil { return err } - // TODO (casie): Create localQueue on node creation; why not? childNode.localQueue.PushBack(v) return nil } diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index 3b19d3e13b9..4ea71818e52 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -11,9 +11,6 @@ import ( "testing" ) -// TODO (casie): Write a test for dequeuing from tqa childA, enqueue to new childB, expect to dequeue next from childB -// TODO (casie): Write a test for enqueueFrontByPath() - func newTenantQuerierAssignments() *tenantQuerierAssignments { return &tenantQuerierAssignments{ tenantQuerierIDs: make(map[TenantID]map[QuerierID]struct{}), @@ -63,7 +60,6 @@ func Test_EnqueueBackByPath(t *testing.T) { tests := []struct { name string treeAlgosByDepth []DequeueAlgorithm - rootAlgo DequeueAlgorithm children []QueuePath expectErr bool }{ @@ -123,6 +119,59 @@ func Test_EnqueueBackByPath(t *testing.T) { } } +func Test_EnqueueFrontByPath(t *testing.T) { + type enqueueObj struct { + obj any + path QueuePath + } + someQuerier := QuerierID("placeholder") + tests := []struct { + name string + treeAlgosByDepth []DequeueAlgorithm + enqueueObjs []enqueueObj + expected []any + }{ + { + name: "enqueue to front of round-robin node", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}}, + enqueueObjs: []enqueueObj{ + {"query-1", QueuePath{}}, + {"query-2", QueuePath{}}, + }, + expected: []any{"query-2", "query-1"}, + }, + { + name: "enqueue to front of shuffle-shard node", + treeAlgosByDepth: []DequeueAlgorithm{&tenantQuerierAssignments{ + currentQuerier: &someQuerier, + }}, + enqueueObjs: []enqueueObj{ + {"query-1", QueuePath{}}, + {"query-2", QueuePath{}}, + }, + expected: []any{"query-2", "query-1"}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) + + for _, o := range tt.enqueueObjs { + err = tree.EnqueueFrontByPath(o.path, o.obj) + } + require.NoError(t, err) + + for _, expectedVal := range tt.expected { + _, v := tree.Dequeue() + require.Equal(t, expectedVal, v) + } + _, v := tree.Dequeue() + require.Nil(t, v) + }) + } +} + func Test_Dequeue_RootNode(t *testing.T) { tests := []struct { name string @@ -150,7 +199,6 @@ func Test_Dequeue_RootNode(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - // TODO (casie): ugly querierID := QuerierID("placeholder") switch tt.rootAlgo.(type) { case *tenantQuerierAssignments: @@ -498,7 +546,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { expected: []any{nil}, }, { - // TODO (casie): also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) + // This also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) name: "dequeue from a tenant with a nil tenant-querier map", treeAlgosByDepth: []DequeueAlgorithm{ &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, From d539206c1171614d7c5d2a499c75cccb97bd3568 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 10 Jun 2024 15:39:14 -0700 Subject: [PATCH 14/26] Rename to TreeQueue --- pkg/frontend/v1/frontend.go | 2 +- pkg/scheduler/queue/queue.go | 2 +- pkg/scheduler/queue/queue_test.go | 6 +-- .../queue/tenant_querier_assignment.go | 4 +- pkg/scheduler/queue/tenant_queues.go | 18 ++++---- pkg/scheduler/queue/tenant_queues_test.go | 14 +++--- pkg/scheduler/queue/tree_queue.go | 46 +++++++++---------- pkg/scheduler/queue/tree_queue_test.go | 10 ++-- .../queue/tree_queueing_algorithms.go | 6 +-- pkg/scheduler/scheduler.go | 2 +- 10 files changed, 55 insertions(+), 55 deletions(-) diff --git a/pkg/frontend/v1/frontend.go b/pkg/frontend/v1/frontend.go index 0562aadea43..82bfc98d1e1 100644 --- a/pkg/frontend/v1/frontend.go +++ b/pkg/frontend/v1/frontend.go @@ -252,7 +252,7 @@ func (f *Frontend) Process(server frontendv1pb.Frontend_ProcessServer) error { /* We want to dequeue the next unexpired request from the chosen tenant queue. - The chance of choosing a particular tenant for dequeueing is (1/active_tenants). + The chance of choosing a particular tenant for dequeuing is (1/active_tenants). This is problematic under load, especially with other middleware enabled such as querier.split-by-interval, where one request may fan out into many. If expired requests aren't exhausted before checking another tenant, it would take diff --git a/pkg/scheduler/queue/queue.go b/pkg/scheduler/queue/queue.go index 58d78211488..4c9543554c5 100644 --- a/pkg/scheduler/queue/queue.go +++ b/pkg/scheduler/queue/queue.go @@ -374,7 +374,7 @@ func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierC exceedsThreshold, queryComponent := q.QueryComponentUtilization.ExceedsThresholdForComponentName( queryComponentName, int(q.connectedQuerierWorkers.Load()), - q.queueBroker.queueTree.rootNode.ItemCount(), + q.queueBroker.tree.rootNode.ItemCount(), q.waitingQuerierConnsToDispatch.Len(), ) diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index f222a31f159..56abc269a87 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -671,9 +671,9 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend req: req, } - require.Nil(t, queueBroker.queueTree.rootNode.getNode(QueuePath{"tenant-1"})) + require.Nil(t, queueBroker.tree.rootNode.getNode(QueuePath{"tenant-1"})) require.NoError(t, queueBroker.enqueueRequestBack(&tr, tenantMaxQueriers)) - require.False(t, queueBroker.queueTree.GetNode(QueuePath{"tenant-1"}).IsEmpty()) + require.False(t, queueBroker.tree.GetNode(QueuePath{"tenant-1"}).IsEmpty()) ctx, cancel := context.WithCancel(context.Background()) call := &waitingQuerierConn{ @@ -688,5 +688,5 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend // indicating not to re-submit a request for waitingQuerierConn for the querier require.True(t, queue.trySendNextRequestForQuerier(call)) // assert request was re-enqueued for tenant after failed send - require.False(t, queueBroker.queueTree.GetNode(QueuePath{"tenant-1"}).IsEmpty()) + require.False(t, queueBroker.tree.GetNode(QueuePath{"tenant-1"}).IsEmpty()) } diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 4f0407cd7d7..42743e9f8e8 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -53,8 +53,8 @@ func (s querierIDSlice) Search(x QuerierID) int { return sort.Search(len(s), func(i int) bool { return s[i] >= x }) } -// tenantQuerierAssignments implements DequeueAlgorithm. In the context of a Tree, it maintains a mapping of -// tenants to queriers in order to support dequeueing from an appropriate tenant if shuffle-sharding is enabled. +// tenantQuerierAssignments implements DequeueAlgorithm. In the context of a TreeQueue, it maintains a mapping of +// tenants to queriers in order to support dequeuing from an appropriate tenant if shuffle-sharding is enabled. type tenantQuerierAssignments struct { // a tenant has many queriers // a tenant has *all* queriers if: diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index ad1ac7d9f6d..bd29893d857 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -24,7 +24,7 @@ type tenantRequest struct { // queueBroker encapsulates access to tenant queues for pending requests // and maintains consistency with the tenant-querier assignments type queueBroker struct { - queueTree *Tree + tree *TreeQueue tenantQuerierAssignments *tenantQuerierAssignments @@ -51,7 +51,7 @@ func newQueueBroker( tenantOrderIndex: localQueueIndex - 1, } - // If prioritizeQueryComponents is true, the queueTree will be created + // If prioritizeQueryComponents is true, the tree will be created // with query components at one level above tenants; if it is false, // tenant nodes will each maintain their own query component subtree. tree, err := NewTree( @@ -72,7 +72,7 @@ func newQueueBroker( panic(fmt.Sprintf("error creating the tree queue: %v", err)) } qb := &queueBroker{ - queueTree: tree, + tree: tree, tenantQuerierAssignments: tqas, maxTenantQueueSize: maxTenantQueueSize, additionalQueueDimensionsEnabled: additionalQueueDimensionsEnabled, @@ -83,7 +83,7 @@ func newQueueBroker( } func (qb *queueBroker) isEmpty() bool { - return qb.queueTree.IsEmpty() + return qb.tree.IsEmpty() } // enqueueRequestBack is the standard interface to enqueue requests for dispatch to queriers. @@ -100,13 +100,13 @@ func (qb *queueBroker) enqueueRequestBack(request *tenantRequest, tenantMaxQueri return err } - if tenantQueueNode := qb.queueTree.rootNode.getNode(queuePath[:1]); tenantQueueNode != nil { + if tenantQueueNode := qb.tree.rootNode.getNode(queuePath[:1]); tenantQueueNode != nil { if tenantQueueNode.ItemCount()+1 > qb.maxTenantQueueSize { return ErrTooManyRequests } } - err = qb.queueTree.EnqueueBackByPath(queuePath, request) + err = qb.tree.EnqueueBackByPath(queuePath, request) return err } @@ -125,7 +125,7 @@ func (qb *queueBroker) enqueueRequestFront(request *tenantRequest, tenantMaxQuer if err != nil { return err } - return qb.queueTree.EnqueueFrontByPath(queuePath, request) + return qb.tree.EnqueueFrontByPath(queuePath, request) } func (qb *queueBroker) makeQueuePath(request *tenantRequest) (QueuePath, error) { @@ -159,7 +159,7 @@ func (qb *queueBroker) dequeueRequestForQuerier( qb.tenantQuerierAssignments.currentQuerier = &querierID qb.tenantQuerierAssignments.tenantOrderIndex = lastTenantIndex - queuePath, queueElement := qb.queueTree.Dequeue() + queuePath, queueElement := qb.tree.Dequeue() var request *tenantRequest var tenantID TenantID @@ -175,7 +175,7 @@ func (qb *queueBroker) dequeueRequestForQuerier( } // dequeue returns the full path including root, but getNode expects the path _from_ root - queueNodeAfterDequeue := qb.queueTree.rootNode.getNode(queuePath) + queueNodeAfterDequeue := qb.tree.rootNode.getNode(queuePath) if queueNodeAfterDequeue == nil && len(qb.tenantQuerierAssignments.tenantNodes[string(tenantID)]) == 0 { // queue node was deleted due to being empty after dequeue qb.tenantQuerierAssignments.removeTenant(tenantID) diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index c2b29a3bc42..858e7463f2a 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -19,7 +19,7 @@ import ( func (qb *queueBroker) enqueueObjectsForTests(tenantID TenantID, numObjects int) error { for i := 0; i < numObjects; i++ { - err := qb.queueTree.EnqueueBackByPath(QueuePath{string(tenantID)}, &tenantRequest{ + err := qb.tree.EnqueueBackByPath(QueuePath{string(tenantID)}, &tenantRequest{ tenantID: tenantID, req: fmt.Sprintf("%v: object-%v", tenantID, i), }) @@ -213,12 +213,12 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { } // assert item count of tenant node and its subnodes queuePath := QueuePath{"tenant-1"} - assert.Equal(t, maxTenantQueueSize, qb.queueTree.rootNode.getNode(queuePath).ItemCount()) + assert.Equal(t, maxTenantQueueSize, qb.tree.rootNode.getNode(queuePath).ItemCount()) // assert equal distribution of queue items between tenant node and 3 subnodes for _, v := range additionalQueueDimensions { queuePath := append(QueuePath{"tenant-1"}, v...) - assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), qb.queueTree.GetNode(queuePath).getLocalQueue().Len()) + assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), qb.tree.GetNode(queuePath).getLocalQueue().Len()) } // assert error received when hitting a tenant's enqueue limit, @@ -686,7 +686,7 @@ func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) ( return nil, err } - node, err := qb.queueTree.rootNode.getOrAddNode(qb.makeQueuePathForTests(tenantID), qb.queueTree) + node, err := qb.tree.rootNode.getOrAddNode(qb.makeQueuePathForTests(tenantID), qb.tree) if err != nil { return nil, err } @@ -697,7 +697,7 @@ func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) ( func (qb *queueBroker) removeTenantQueue(tenantID TenantID) bool { qb.tenantQuerierAssignments.removeTenant(tenantID) queuePath := QueuePath{string(tenantID)} - return qb.queueTree.rootNode.deleteNode(queuePath) + return qb.tree.rootNode.deleteNode(queuePath) } func (n *Node) deleteNode(pathFromNode QueuePath) bool { @@ -735,7 +735,7 @@ func isConsistent(qb *queueBroker) error { for ix, tenantID := range qb.tenantQuerierAssignments.tenantIDOrder { path := qb.makeQueuePathForTests(tenantID) - node := qb.queueTree.rootNode.getNode(path) + node := qb.tree.rootNode.getNode(path) if tenantID != "" && node == nil { return fmt.Errorf("tenant %s doesn't have queue", tenantID) @@ -773,7 +773,7 @@ func isConsistent(qb *queueBroker) error { } } - tenantQueueCount := qb.queueTree.rootNode.nodeCount() - 1 + tenantQueueCount := qb.tree.rootNode.nodeCount() - 1 if tenantQueueCount != tenantCount { return fmt.Errorf("inconsistent number of tenants list and tenant queues") } diff --git a/pkg/scheduler/queue/tree_queue.go b/pkg/scheduler/queue/tree_queue.go index 1cd7fd9891f..fa8f067f185 100644 --- a/pkg/scheduler/queue/tree_queue.go +++ b/pkg/scheduler/queue/tree_queue.go @@ -12,9 +12,9 @@ type QueueIndex int //nolint:revive // disallows types beginning with packag const localQueueIndex = -1 -// Tree holds metadata and a pointer to the root node of a hierarchical queue implementation. +// TreeQueue holds metadata and a pointer to the root node of a hierarchical queue implementation. // The root Node maintains a localQueue and an arbitrary number of child nodes (which themselves -// may have local queues and children). Each Node in Tree uses a DequeueAlgorithm (determined by +// may have local queues and children). Each Node in TreeQueue uses a DequeueAlgorithm (determined by // node depth) to determine dequeue order of that Node's subtree. // // Each queuing dimension is modeled as a node in the tree, internally reachable through a QueuePath. @@ -24,33 +24,33 @@ const localQueueIndex = -1 // // When dequeuing from a given node, a Node will use its DequeueAlgorithm to choose either itself // or a child node to dequeue from recursively (i.e., a child Node will use its own DequeueAlgorithm -// to determine how to proceed). Tree will not dequeue from two different Nodes at the same depth +// to determine how to proceed). TreeQueue will not dequeue from two different Nodes at the same depth // consecutively, unless the previously-checked Node was empty down to the leaf node. -type Tree struct { +type TreeQueue struct { rootNode *Node algosByDepth []DequeueAlgorithm } -func NewTree(dequeueAlgorithms ...DequeueAlgorithm) (*Tree, error) { +func NewTree(dequeueAlgorithms ...DequeueAlgorithm) (*TreeQueue, error) { if len(dequeueAlgorithms) == 0 { - return nil, fmt.Errorf("cannot create a tree without defined dequeueing algorithms") + return nil, fmt.Errorf("cannot create a tree without defined DequeueAlgorithm") } root, err := newNode("root", 0, dequeueAlgorithms[0]) if err != nil { return nil, err } root.depth = 0 - return &Tree{ + return &TreeQueue{ rootNode: root, algosByDepth: dequeueAlgorithms, }, nil } -func (t *Tree) IsEmpty() bool { +func (t *TreeQueue) IsEmpty() bool { return t.rootNode.IsEmpty() } -// Dequeue removes and returns an item from the front of the next appropriate Node in the Tree, as +// Dequeue removes and returns an item from the front of the next appropriate Node in the TreeQueue, as // well as the path to the Node which that item was dequeued from. // // Either the root/self node or a child node is chosen according to the Node's DequeueAlgorithm. If @@ -60,7 +60,7 @@ func (t *Tree) IsEmpty() bool { // Nodes that empty down to the leaf after being dequeued from (or which are found to be empty leaf // nodes during the dequeue operation) are deleted as the recursion returns up the stack. This // maintains structural guarantees relied upon to make IsEmpty() non-recursive. -func (t *Tree) Dequeue() (QueuePath, any) { +func (t *TreeQueue) Dequeue() (QueuePath, any) { path, v := t.rootNode.dequeue() // The returned node dequeue path includes the root node; exclude // this so that the return path can be used if needed to enqueue. @@ -72,12 +72,12 @@ func (t *Tree) Dequeue() (QueuePath, any) { // // path is relative to the root node; providing a QueuePath beginning with "root" // will create a child node of the root node which is also named "root." -func (t *Tree) EnqueueBackByPath(path QueuePath, v any) error { +func (t *TreeQueue) EnqueueBackByPath(path QueuePath, v any) error { return t.rootNode.enqueueBackByPath(t, path, v) } // EnqueueFrontByPath enqueues an item in the front of the local queue of the Node -// located at a given path through the Tree; nodes for the path are created as needed. +// located at a given path through the TreeQueue; nodes for the path are created as needed. // // Enqueueing to the front is intended only for items which were first enqueued to the back // and then dequeued after reaching the front. @@ -86,15 +86,15 @@ func (t *Tree) EnqueueBackByPath(path QueuePath, v any) error { // fails to complete operations on the dequeued item, but failure is not yet final, and the // operations should be retried by a subsequent queue consumer. A concrete example is when // a queue consumer fails or disconnects for unrelated reasons while we are in the process -// of dequeueing a request for it. +// of dequeuing a request for it. // // path must be relative to the root node; providing a QueuePath beginning with "root" // will create a child node of root which is also named "root." -func (t *Tree) EnqueueFrontByPath(path QueuePath, v any) error { +func (t *TreeQueue) EnqueueFrontByPath(path QueuePath, v any) error { return t.rootNode.enqueueFrontByPath(t, path, v) } -func (t *Tree) GetNode(path QueuePath) *Node { +func (t *TreeQueue) GetNode(path QueuePath) *Node { return t.rootNode.getNode(path) } @@ -107,7 +107,7 @@ type Node struct { name string localQueue *list.List queuePosition int // next index in queueOrder to dequeue from - queueOrder []string // order for dequeueing from self/children + queueOrder []string // order for dequeuing from self/children queueMap map[string]*Node depth int dequeueAlgorithm DequeueAlgorithm @@ -116,7 +116,7 @@ type Node struct { func newNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { if da == nil { - return nil, fmt.Errorf("cannot create a node without a defined dequeueing algorithm") + return nil, fmt.Errorf("cannot create a node without a defined DequeueAlgorithm") } switch da.(type) { case *tenantQuerierAssignments: @@ -140,7 +140,7 @@ func newNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { func (n *Node) IsEmpty() bool { // avoid recursion to make this a cheap operation // - // Because we dereference empty child nodes during dequeueing, + // Because we dereference empty child nodes during dequeuing, // we assume that emptiness means there are no child nodes // and nothing in this tree node's local queue. // @@ -167,7 +167,7 @@ func (n *Node) getLocalQueue() *list.List { return n.localQueue } -func (n *Node) enqueueFrontByPath(tree *Tree, pathFromNode QueuePath, v any) error { +func (n *Node) enqueueFrontByPath(tree *TreeQueue, pathFromNode QueuePath, v any) error { childNode, err := n.getOrAddNode(pathFromNode, tree) if err != nil { return err @@ -176,7 +176,7 @@ func (n *Node) enqueueFrontByPath(tree *Tree, pathFromNode QueuePath, v any) err return nil } -func (n *Node) enqueueBackByPath(tree *Tree, pathFromNode QueuePath, v any) error { +func (n *Node) enqueueBackByPath(tree *TreeQueue, pathFromNode QueuePath, v any) error { childNode, err := n.getOrAddNode(pathFromNode, tree) if err != nil { return err @@ -201,7 +201,7 @@ func (n *Node) dequeue() (QueuePath, any) { for v == nil && !checkedAllNodes { dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueGetNode(n) var deletedNode bool - // dequeueing from local queue + // dequeuing from local queue if dequeueNode == n { if n.localQueue.Len() > 0 { // dequeueNode is self, local queue non-empty @@ -218,7 +218,7 @@ func (n *Node) dequeue() (QueuePath, any) { } else { // dequeue from a child childPath, v = dequeueNode.dequeue() - // if the dequeue node is empty _after_ dequeueing, delete it from children + // if the dequeue node is empty _after_ dequeuing, delete it from children if dequeueNode.IsEmpty() { // removing an element sets our position one step forward; // tell state to reset it to original queuePosition @@ -254,7 +254,7 @@ func (n *Node) getNode(pathFromNode QueuePath) *Node { // // pathFromNode must be relative to the receiver node; providing a QueuePath beginning with // the receiver/parent node name will create a child node of the same name as the parent. -func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *Tree) (*Node, error) { +func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *TreeQueue) (*Node, error) { if len(pathFromNode) == 0 { return n, nil } diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index 4ea71818e52..cd59330259d 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -38,7 +38,7 @@ func Test_NewTree(t *testing.T) { treeAlgos: []DequeueAlgorithm{&tenantQuerierAssignments{}}, }, { - name: "fail to create tree without defined dequeueing algorithm", + name: "fail to create tree without defined dequeuing algorithm", treeAlgos: []DequeueAlgorithm{nil}, expectErr: true, }, @@ -512,7 +512,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { expected: []any{"query-2", "query-3", nil}, }, { - name: "dequeueing for one querier returns nil, but does return for a different querier", + name: "dequeuing for one querier returns nil, but does return for a different querier", treeAlgosByDepth: []DequeueAlgorithm{ &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, @@ -677,7 +677,7 @@ func Test_DequeueBalancedRoundRobinTree(t *testing.T) { count := 0 - // Tree will fairly dequeue from all levels of the tree + // TreeQueue will fairly dequeue from all levels of the tree rotationsBeforeRepeat := len(firstDimensions) * len(secondDimensions) // track dequeued paths to ensure round-robin dequeuing does not repeat before expected dequeuedPathCache := make([]QueuePath, rotationsBeforeRepeat) @@ -842,7 +842,7 @@ func Test_NodeCannotDeleteItself(t *testing.T) { } } -func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimension int) *Tree { +func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimension int) *TreeQueue { tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) require.NoError(t, err) require.Equal(t, 1, tree.rootNode.nodeCount()) @@ -867,7 +867,7 @@ func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions return tree } -func makeUnbalancedRoundRobinTree(t *testing.T) *Tree { +func makeUnbalancedRoundRobinTree(t *testing.T) *TreeQueue { /* root ├── child0 diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go index 64f510d072b..995885a173a 100644 --- a/pkg/scheduler/queue/tree_queueing_algorithms.go +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -2,8 +2,8 @@ package queue -// DequeueAlgorithm represents the set of operations specific to different approaches to dequeueing. It is applied -// at the layer-level -- every Node at the same depth in a Tree shares the same DequeueAlgorithm, including state +// DequeueAlgorithm represents the set of operations specific to different approaches to dequeuing. It is applied +// at the layer-level -- every Node at the same depth in a TreeQueue shares the same DequeueAlgorithm, including state // that may be stored in a struct that implements DequeueAlgorithm. type DequeueAlgorithm interface { addChildNode(*Node, *Node) @@ -13,7 +13,7 @@ type DequeueAlgorithm interface { } // roundRobinState is the simplest type of DequeueAlgorithm; nodes which use this DequeueAlgorithm and are at -// the same depth in a Tree do not share any state. When children are added to these nodes, they are placed at +// the same depth in a TreeQueue do not share any state. When children are added to these nodes, they are placed at // the "end" of the order from the perspective of the node's current queuePosition (e.g., if queuePosition is 3, // a new child will be placed at index 2). Children are dequeued from using a simple round-robin ordering; // queuePosition is incremented on every dequeue. diff --git a/pkg/scheduler/scheduler.go b/pkg/scheduler/scheduler.go index d31219a667a..a8b5ddfb332 100644 --- a/pkg/scheduler/scheduler.go +++ b/pkg/scheduler/scheduler.go @@ -443,7 +443,7 @@ func (s *Scheduler) QuerierLoop(querier schedulerpb.SchedulerForQuerier_QuerierL /* We want to dequeue the next unexpired request from the chosen tenant queue. - The chance of choosing a particular tenant for dequeueing is (1/active_tenants). + The chance of choosing a particular tenant for dequeuing is (1/active_tenants). This is problematic under load, especially with other middleware enabled such as querier.split-by-interval, where one request may fan out into many. If expired requests aren't exhausted before checking another tenant, it would take From 3bae2d6276c4a555b6eed38da6274e3addffce73 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 10 Jun 2024 16:11:23 -0700 Subject: [PATCH 15/26] lint --- cmd/mimir/config-descriptor.json | 11 ++++++++ cmd/mimir/help-all.txt.tmpl | 2 ++ .../configuration-parameters/index.md | 5 ++++ pkg/scheduler/queue/queue_test.go | 2 +- .../queue/tenant_querier_assignment.go | 8 ------ pkg/scheduler/queue/tenant_queues_test.go | 11 ++++---- pkg/scheduler/queue/tree_queue.go | 4 +-- pkg/scheduler/queue/tree_queue_test.go | 25 +++++++++++++------ pkg/scheduler/scheduler.go | 5 ++-- 9 files changed, 46 insertions(+), 27 deletions(-) diff --git a/cmd/mimir/config-descriptor.json b/cmd/mimir/config-descriptor.json index 8da2843a345..ee321dd8c9a 100644 --- a/cmd/mimir/config-descriptor.json +++ b/cmd/mimir/config-descriptor.json @@ -15909,6 +15909,17 @@ "fieldType": "boolean", "fieldCategory": "experimental" }, + { + "kind": "field", + "name": "prioritize_query_components", + "required": false, + "desc": "Prioritize rotation through query components over rotation through tenants during dequeueing.", + "fieldValue": null, + "fieldDefaultValue": false, + "fieldFlag": "query-scheduler.prioritize-query-components", + "fieldType": "boolean", + "fieldCategory": "experimental" + }, { "kind": "field", "name": "querier_forget_delay", diff --git a/cmd/mimir/help-all.txt.tmpl b/cmd/mimir/help-all.txt.tmpl index 7a6f26ade4b..6f2b3e67ef8 100644 --- a/cmd/mimir/help-all.txt.tmpl +++ b/cmd/mimir/help-all.txt.tmpl @@ -2247,6 +2247,8 @@ Usage of ./cmd/mimir/mimir: Maximum number of outstanding requests per tenant per query-scheduler. In-flight requests above this limit will fail with HTTP response status code 429. (default 100) -query-scheduler.max-used-instances int The maximum number of query-scheduler instances to use, regardless how many replicas are running. This option can be set only when -query-scheduler.service-discovery-mode is set to 'ring'. 0 to use all available query-scheduler instances. + -query-scheduler.prioritize-query-components + [experimental] Prioritize rotation through query components over rotation through tenants during dequeueing. -query-scheduler.querier-forget-delay duration [experimental] If a querier disconnects without sending notification about graceful shutdown, the query-scheduler will keep the querier in the tenant's shard until the forget delay has passed. This feature is useful to reduce the blast radius when shuffle-sharding is enabled. -query-scheduler.ring.consul.acl-token string diff --git a/docs/sources/mimir/configure/configuration-parameters/index.md b/docs/sources/mimir/configure/configuration-parameters/index.md index 5f114817834..3c1424d5e2b 100644 --- a/docs/sources/mimir/configure/configuration-parameters/index.md +++ b/docs/sources/mimir/configure/configuration-parameters/index.md @@ -1708,6 +1708,11 @@ The `query_scheduler` block configures the query-scheduler. # CLI flag: -query-scheduler.additional-query-queue-dimensions-enabled [additional_query_queue_dimensions_enabled: | default = false] +# (experimental) Prioritize rotation through query components over rotation +# through tenants during dequeueing. +# CLI flag: -query-scheduler.prioritize-query-components +[prioritize_query_components: | default = false] + # (experimental) If a querier disconnects without sending notification about # graceful shutdown, the query-scheduler will keep the querier in the tenant's # shard until the forget delay has passed. This feature is useful to reduce the diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index 56abc269a87..1fb776087e4 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -9,7 +9,6 @@ import ( "context" "errors" "fmt" - promtest "github.com/prometheus/client_golang/prometheus/testutil" "math/rand" "strconv" "strings" @@ -22,6 +21,7 @@ import ( "github.com/grafana/dskit/services" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + promtest "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "golang.org/x/sync/errgroup" diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 42743e9f8e8..d2293ee7243 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -96,14 +96,6 @@ type tenantQuerierAssignments struct { currentQuerier *QuerierID } -func (tqa *tenantQuerierAssignments) getTenant(tenantID TenantID) (*queueTenant, error) { - if tenantID == emptyTenantID { - return nil, ErrInvalidTenantID - } - tenant := tqa.tenantsByID[tenantID] - return tenant, nil -} - // createOrUpdateTenant creates or updates a tenant into the tenant-querier assignment state. // // New tenants are added to the tenant order list and tenant-querier shards are shuffled if needed. diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 858e7463f2a..cbf6ac96005 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -8,13 +8,14 @@ package queue import ( "context" "fmt" - "github.com/grafana/dskit/httpgrpc" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" "math" "math/rand" "testing" "time" + + "github.com/grafana/dskit/httpgrpc" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func (qb *queueBroker) enqueueObjectsForTests(tenantID TenantID, numObjects int) error { @@ -111,7 +112,7 @@ func TestQueues(t *testing.T) { {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, } - lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) + _ = assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) //[one two three] // confirm fifo by adding a third tenant queue and iterating to it @@ -310,7 +311,7 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { // After disconnecting querier-2, it's expected to own no queue. qb.tenantQuerierAssignments.removeQuerier("querier-2") - req, tenant, qTwolastTenantIndex, err = qb.dequeueRequestForQuerier(qTwolastTenantIndex, "querier-2") + req, tenant, _, err = qb.dequeueRequestForQuerier(qTwolastTenantIndex, "querier-2") assert.Nil(t, req) assert.Nil(t, tenant) assert.Equal(t, ErrQuerierShuttingDown, err) diff --git a/pkg/scheduler/queue/tree_queue.go b/pkg/scheduler/queue/tree_queue.go index fa8f067f185..ddeb39e154d 100644 --- a/pkg/scheduler/queue/tree_queue.go +++ b/pkg/scheduler/queue/tree_queue.go @@ -118,9 +118,7 @@ func newNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { if da == nil { return nil, fmt.Errorf("cannot create a node without a defined DequeueAlgorithm") } - switch da.(type) { - case *tenantQuerierAssignments: - tqa := da.(*tenantQuerierAssignments) + if tqa, ok := da.(*tenantQuerierAssignments); ok { tqa.tenantOrderIndex = localQueueIndex - 1 // start from -2 so that we first check local queue if tqa.tenantNodes == nil { tqa.tenantNodes = map[string][]*Node{} diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index cd59330259d..353524f0936 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -5,10 +5,9 @@ package queue import ( "fmt" "strings" + "testing" - //"fmt" "github.com/stretchr/testify/require" - "testing" ) func newTenantQuerierAssignments() *tenantQuerierAssignments { @@ -620,12 +619,22 @@ func Test_ChangeTenantQuerierAssignments(t *testing.T) { tree, err := NewTree(state, &roundRobinState{}, &roundRobinState{}) require.NoError(t, err) - err = tree.EnqueueBackByPath(QueuePath{"tenant-1", "query-component-1"}, "query-1") - err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-2") - err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-3") - err = tree.EnqueueBackByPath(QueuePath{"tenant-2", "query-component-1"}, "query-4") - err = tree.EnqueueBackByPath(QueuePath{"tenant-3", "query-component-1"}, "query-5") - require.NoError(t, err) + type enqueueObj struct { + obj any + path QueuePath + } + enqueueObjs := []enqueueObj{ + {"query-1", QueuePath{"tenant-1", "query-component-1"}}, + {"query-2", QueuePath{"tenant-2", "query-component-1"}}, + {"query-3", QueuePath{"tenant-2", "query-component-1"}}, + {"query-4", QueuePath{"tenant-2", "query-component-1"}}, + {"query-5", QueuePath{"tenant-3", "query-component-1"}}, + } + + for _, eo := range enqueueObjs { + err = tree.EnqueueBackByPath(eo.path, eo.obj) + require.NoError(t, err) + } querier1 := QuerierID("querier-1") querier2 := QuerierID("querier-2") diff --git a/pkg/scheduler/scheduler.go b/pkg/scheduler/scheduler.go index a8b5ddfb332..be77d70ec56 100644 --- a/pkg/scheduler/scheduler.go +++ b/pkg/scheduler/scheduler.go @@ -95,7 +95,7 @@ type connectedFrontend struct { type Config struct { MaxOutstandingPerTenant int `yaml:"max_outstanding_requests_per_tenant"` AdditionalQueryQueueDimensionsEnabled bool `yaml:"additional_query_queue_dimensions_enabled" category:"experimental"` - PrioritizeQueryComponentsEnabled bool `yaml:"prioritize_query_components_enabled" category:"experimental"` + PrioritizeQueryComponents bool `yaml:"prioritize_query_components" category:"experimental"` QuerierForgetDelay time.Duration `yaml:"querier_forget_delay" category:"experimental"` GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config" doc:"description=This configures the gRPC client used to report errors back to the query-frontend."` @@ -105,6 +105,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.IntVar(&cfg.MaxOutstandingPerTenant, "query-scheduler.max-outstanding-requests-per-tenant", 100, "Maximum number of outstanding requests per tenant per query-scheduler. In-flight requests above this limit will fail with HTTP response status code 429.") f.BoolVar(&cfg.AdditionalQueryQueueDimensionsEnabled, "query-scheduler.additional-query-queue-dimensions-enabled", false, "Enqueue query requests with additional queue dimensions to split tenant request queues into subqueues. This enables separate requests to proceed from a tenant's subqueues even when other subqueues are blocked on slow query requests. Must be set on both query-frontend and scheduler to take effect. (default false)") + f.BoolVar(&cfg.PrioritizeQueryComponents, "query-scheduler.prioritize-query-components", false, "Prioritize rotation through query components over rotation through tenants during dequeueing.") f.DurationVar(&cfg.QuerierForgetDelay, "query-scheduler.querier-forget-delay", 0, "If a querier disconnects without sending notification about graceful shutdown, the query-scheduler will keep the querier in the tenant's shard until the forget delay has passed. This feature is useful to reduce the blast radius when shuffle-sharding is enabled.") cfg.GRPCClientConfig.RegisterFlagsWithPrefix("query-scheduler.grpc-client-config", f) @@ -161,7 +162,7 @@ func NewScheduler(cfg Config, limits Limits, log log.Logger, registerer promethe s.log, cfg.MaxOutstandingPerTenant, cfg.AdditionalQueryQueueDimensionsEnabled, - cfg.PrioritizeQueryComponentsEnabled, + cfg.PrioritizeQueryComponents, cfg.QuerierForgetDelay, s.queueLength, s.discardedRequests, From ce9491fa997ff6e9e1a7a9b8de514cfd9c498679 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Tue, 11 Jun 2024 15:25:56 -0700 Subject: [PATCH 16/26] PR feedback --- .../queue/tenant_querier_assignment.go | 106 ++++++++---------- pkg/scheduler/queue/tenant_queues.go | 12 +- pkg/scheduler/queue/tenant_queues_test.go | 8 +- pkg/scheduler/queue/tree_queue.go | 36 +++--- pkg/scheduler/queue/tree_queue_test.go | 10 +- .../queue/tree_queueing_algorithms.go | 76 +++++++------ 6 files changed, 125 insertions(+), 123 deletions(-) diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index d2293ee7243..0d9207673ce 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -323,13 +323,13 @@ func (tqa *tenantQuerierAssignments) shuffleTenantQueriers(tenantID TenantID, sc // // Note that because we use the shared tenantIDOrder and tenantOrderIndex to manage the queue, we functionally // ignore each Node's individual queueOrder and queuePosition. -func (tqa *tenantQuerierAssignments) dequeueGetNode(n *Node) (*Node, bool) { +func (tqa *tenantQuerierAssignments) dequeueSelectNode(dequeueFrom *Node) (*Node, bool) { // can't get a tenant if no querier set if tqa.currentQuerier == nil { return nil, true } - checkedAllNodes := n.childrenChecked == len(n.queueMap)+1 // must check local queue as well + checkedAllNodes := dequeueFrom.childrenChecked == len(dequeueFrom.queueMap)+1 // must check local queue as well // advance queue position for dequeue tqa.tenantOrderIndex++ @@ -338,8 +338,8 @@ func (tqa *tenantQuerierAssignments) dequeueGetNode(n *Node) (*Node, bool) { } // no children - if len(n.queueMap) == 0 || tqa.tenantOrderIndex == localQueueIndex { - return n, checkedAllNodes + if len(dequeueFrom.queueMap) == 0 || tqa.tenantOrderIndex == localQueueIndex { + return dequeueFrom, checkedAllNodes } checkIndex := tqa.tenantOrderIndex @@ -351,25 +351,25 @@ func (tqa *tenantQuerierAssignments) dequeueGetNode(n *Node) (*Node, bool) { tenantID := tqa.tenantIDOrder[checkIndex] tenantName := string(tenantID) - if _, ok := n.queueMap[tenantName]; !ok { + if _, ok := dequeueFrom.queueMap[tenantName]; !ok { // tenant not in _this_ node's children, move on checkIndex++ continue } // increment nodes checked even if not in tenant-querier map - n.childrenChecked++ - checkedAllNodes = n.childrenChecked == len(n.queueMap)+1 + dequeueFrom.childrenChecked++ + checkedAllNodes = dequeueFrom.childrenChecked == len(dequeueFrom.queueMap)+1 // if the tenant-querier set is nil, any querier can serve this tenant if tqa.tenantQuerierIDs[tenantID] == nil { tqa.tenantOrderIndex = checkIndex - return n.queueMap[tenantName], checkedAllNodes + return dequeueFrom.queueMap[tenantName], checkedAllNodes } if tenantQuerierSet, ok := tqa.tenantQuerierIDs[tenantID]; ok { if _, ok := tenantQuerierSet[*tqa.currentQuerier]; ok { tqa.tenantOrderIndex = checkIndex - return n.queueMap[tenantName], checkedAllNodes + return dequeueFrom.queueMap[tenantName], checkedAllNodes } } checkIndex++ @@ -377,16 +377,49 @@ func (tqa *tenantQuerierAssignments) dequeueGetNode(n *Node) (*Node, bool) { return nil, checkedAllNodes } -// dequeueUpdateState updates the node state to reflect the number of children that have been checked -// for dequeueable elements, so that dequeueGetNode will stop checking if it has checked all possible nodes. -func (tqa *tenantQuerierAssignments) dequeueUpdateState(n *Node, v any, _ bool) { - // we need to reset our checked nodes if we found a value to dequeue - if v != nil { - n.childrenChecked = 0 +// dequeueUpdateState deletes the dequeued-from node from the following locations if it is empty: +// - parent's queueMap, +// - tenantNodes +// - tenantIDOrder iff there are no other nodes by the same name in tenantNodes. If the child is at the end of +// tenantIDOrder, it is removed outright; otherwise, it is replaced with an empty ("") element +// +// dequeueUpdateState would normally also handle incrementing the queue position after performing a dequeue, but +// tenantQuerierAssignments currently expects the caller to handle this by having the querier set tenantOrderIndex. +func (tqa *tenantQuerierAssignments) dequeueUpdateState(parent *Node, dequeuedFrom *Node) { + // if dequeuedFrom is nil or is not empty, we don't need to do anything; + // position updates will be handled by the caller, and we don't need to remove any nodes. + if dequeuedFrom == nil || !dequeuedFrom.IsEmpty() { return } - n.childrenChecked++ + // delete from the node's children + childName := dequeuedFrom.Name() + delete(parent.queueMap, childName) + + // delete from shared tenantNodes + for i, tenantNode := range tqa.tenantNodes[childName] { + if tenantNode == dequeuedFrom { + tqa.tenantNodes[childName] = append(tqa.tenantNodes[childName][:i], tqa.tenantNodes[childName][i+1:]...) + } + } + + // check tenantNodes; we only remove from tenantIDOrder if all nodes with this name are gone. + // If the removed child is at the _end_ of tenantIDOrder, we remove it outright; otherwise, + // we replace it with an empty string so as not to reindex all slice elements + removeFromSharedQueueOrder := len(tqa.tenantNodes[childName]) == 0 + + if removeFromSharedQueueOrder { + for idx, name := range tqa.tenantIDOrder { + if string(name) == childName { + tqa.tenantIDOrder[idx] = emptyTenantID + } + } + // clear all sequential empty elements from tenantIDOrder + lastElementIndex := len(tqa.tenantIDOrder) - 1 + for i := lastElementIndex; i >= 0 && tqa.tenantIDOrder[i] == ""; i-- { + tqa.tenantIDOrder = tqa.tenantIDOrder[:i] + } + } } // addChildNode adds a child to: @@ -427,44 +460,3 @@ func (tqa *tenantQuerierAssignments) addChildNode(parent, child *Node) { tqa.tenantIDOrder = append(tqa.tenantIDOrder, TenantID(childName)) } - -// deleteChildNode removes a child from: -// - parent's queueMap, -// - tenantNodes -// - tenantIDOrder iff there are no other nodes by the same name in tenantNodes. If the child is at the end of -// tenantIDOrder, it is removed outright; otherwise, it is replaced with an empty ("") element -func (tqa *tenantQuerierAssignments) deleteChildNode(parent, child *Node) bool { - childName := child.Name() - - // delete from parent's children - delete(parent.queueMap, childName) - - // delete from shared tenantNodes - for i, tenantNode := range tqa.tenantNodes[childName] { - if tenantNode == child { - tqa.tenantNodes[childName] = append(tqa.tenantNodes[childName][:i], tqa.tenantNodes[childName][i+1:]...) - } - } - - // check tenantNodes; we only remove from tenantIDOrder if all nodes with this name are gone. - // If the removed child is at the _end_ of tenantIDOrder, we remove it outright; otherwise, - // we replace it with an empty string so as not to reindex all slice elements - removeFromSharedQueueOrder := len(tqa.tenantNodes[childName]) == 0 - - if removeFromSharedQueueOrder { - for idx, name := range tqa.tenantIDOrder { - if string(name) == childName { - tqa.tenantIDOrder[idx] = emptyTenantID - } - } - // clear all sequential empty elements from tenantIDOrder - lastElementIndex := len(tqa.tenantIDOrder) - 1 - for i := lastElementIndex; i >= 0 && tqa.tenantIDOrder[i] == ""; i-- { - tqa.tenantIDOrder = tqa.tenantIDOrder[:i] - } - } - // The only time we remove elements from tenantIDOrder is when they are empty nodes, _and_ at the end of - // tenantIDOrder, so we will never remove a tenant from tenantIDOrder at a position before tenantOrderIndex. - // Thus, we will never need to decrement tenantOrderIndex. - return false -} diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index bd29893d857..10a7dded3a9 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -55,15 +55,15 @@ func newQueueBroker( // with query components at one level above tenants; if it is false, // tenant nodes will each maintain their own query component subtree. tree, err := NewTree( - tqas, // root - &roundRobinState{}, // tenant queues - &roundRobinState{}, // query components + tqas, // root; DequeueAlgorithm selects tenants + &roundRobinState{}, // tenant queues; DequeueAlgorithm selects query component + &roundRobinState{}, // query components; DequeueAlgorithm selects query from local queue ) if prioritizeQueryComponents { tree, err = NewTree( - &roundRobinState{}, // root - tqas, // query components - &roundRobinState{}, // tenant queues + &roundRobinState{}, // root; DequeueAlgorithm selects query component + tqas, // query components; DequeueAlgorithm selects tenant + &roundRobinState{}, // tenant queues; DequeueAlgorithm selects query from local queue ) } diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index cbf6ac96005..eb1418aa854 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -6,6 +6,7 @@ package queue import ( + "container/list" "context" "fmt" "math" @@ -701,6 +702,7 @@ func (qb *queueBroker) removeTenantQueue(tenantID TenantID) bool { return qb.tree.rootNode.deleteNode(queuePath) } +// deleteNode is a test utility, not intended for use by consumers of Node func (n *Node) deleteNode(pathFromNode QueuePath) bool { if len(pathFromNode) == 0 { // node cannot delete itself @@ -716,8 +718,10 @@ func (n *Node) deleteNode(pathFromNode QueuePath) bool { return false } if deleteNode := parentNode.getNode(QueuePath{deleteNodeName}); deleteNode != nil { - childDeleted := parentNode.dequeueAlgorithm.deleteChildNode(parentNode, deleteNode) - parentNode.dequeueAlgorithm.dequeueUpdateState(parentNode, nil, childDeleted) + // empty the node so that update state properly deletes it + deleteNode.queueMap = map[string]*Node{} + deleteNode.localQueue = list.New() + parentNode.dequeueAlgorithm.dequeueUpdateState(parentNode, deleteNode) return true } return false diff --git a/pkg/scheduler/queue/tree_queue.go b/pkg/scheduler/queue/tree_queue.go index ddeb39e154d..4cf2292130a 100644 --- a/pkg/scheduler/queue/tree_queue.go +++ b/pkg/scheduler/queue/tree_queue.go @@ -197,10 +197,10 @@ func (n *Node) dequeue() (QueuePath, any) { var dequeueNode *Node // continue until we've found a value or checked all nodes that need checking for v == nil && !checkedAllNodes { - dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueGetNode(n) - var deletedNode bool + dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueSelectNode(n) + switch dequeueNode { // dequeuing from local queue - if dequeueNode == n { + case n: if n.localQueue.Len() > 0 { // dequeueNode is self, local queue non-empty if elt := n.localQueue.Front(); elt != nil { @@ -208,24 +208,24 @@ func (n *Node) dequeue() (QueuePath, any) { v = elt.Value } } - } else if dequeueNode == nil { - // no dequeue-able child found; reset checked children to 0, - // as we won't update state before moving on - n.childrenChecked = 0 - return path, v - } else { - // dequeue from a child + // no dequeue-able child found; break out of the loop, + // since we won't find anything to dequeue if we don't + // have a node to dequeue from now + case nil: + checkedAllNodes = true + // dequeue from a child + default: childPath, v = dequeueNode.dequeue() - // if the dequeue node is empty _after_ dequeuing, delete it from children - if dequeueNode.IsEmpty() { - // removing an element sets our position one step forward; - // tell state to reset it to original queuePosition - delete(n.queueMap, dequeueNode.Name()) - deletedNode = n.dequeueAlgorithm.deleteChildNode(n, dequeueNode) - } } - n.dequeueAlgorithm.dequeueUpdateState(n, v, deletedNode) + + if v == nil { + n.childrenChecked++ + } + + n.dequeueAlgorithm.dequeueUpdateState(n, dequeueNode) } + // reset children checked to 0 before completing this dequeue + n.childrenChecked = 0 return append(path, childPath...), v } diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index 353524f0936..d8c732c6389 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -371,7 +371,6 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { require.NoError(t, err) for _, operation := range tt.operationOrder { - fmt.Println(operation) if operation.kind == enqueue { err = tree.EnqueueBackByPath(operation.path, operation.obj) require.NoError(t, err) @@ -831,6 +830,8 @@ func Test_EnqueueDuringDequeueRespectsRoundRobin(t *testing.T) { require.True(t, tree.IsEmpty()) } +// Test_NodeCannotDeleteItself creates an empty node, dequeues from it, and ensures that the node still exists +// and has not deleted itself. func Test_NodeCannotDeleteItself(t *testing.T) { tests := []struct { name string @@ -841,12 +842,15 @@ func Test_NodeCannotDeleteItself(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(&roundRobinState{}) + tree, err := NewTree(tt.nodeType) require.NoError(t, err) require.NotNil(t, tree) - require.False(t, tree.rootNode.dequeueAlgorithm.deleteChildNode(tree.rootNode, tree.rootNode)) + _, _ = tree.Dequeue() + require.NotNil(t, tree.rootNode) + require.Zero(t, tree.rootNode.getLocalQueue().Len()) + require.Empty(t, tree.rootNode.queueMap) }) } } diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go index 995885a173a..14800c1ab1e 100644 --- a/pkg/scheduler/queue/tree_queueing_algorithms.go +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -6,10 +6,9 @@ package queue // at the layer-level -- every Node at the same depth in a TreeQueue shares the same DequeueAlgorithm, including state // that may be stored in a struct that implements DequeueAlgorithm. type DequeueAlgorithm interface { - addChildNode(*Node, *Node) - deleteChildNode(*Node, *Node) bool - dequeueGetNode(*Node) (*Node, bool) - dequeueUpdateState(*Node, any, bool) + addChildNode(parent, child *Node) + dequeueSelectNode(dequeueFrom *Node) (*Node, bool) + dequeueUpdateState(dequeueParent *Node, dequeuedFrom *Node) } // roundRobinState is the simplest type of DequeueAlgorithm; nodes which use this DequeueAlgorithm and are at @@ -37,49 +36,52 @@ func (rrs *roundRobinState) addChildNode(parent, child *Node) { } } -// deleteChildNode removes a child node from a parent's children, and returns whether a child was found in the -// parent's queueOrder. This allows us to adjust the parent's queuePosition if necessary. -func (rrs *roundRobinState) deleteChildNode(parent, child *Node) bool { - var childFound bool - childName := child.Name() - - for idx, name := range parent.queueOrder { - if name == childName { - parent.queueOrder = append(parent.queueOrder[:idx], parent.queueOrder[idx+1:]...) - childFound = true - } - } - return childFound -} - // dequeueGetNode returns the node at the node's queuePosition. queuePosition represents the position of // the next node to dequeue from, and is incremented in dequeueUpdateState. -func (rrs *roundRobinState) dequeueGetNode(n *Node) (*Node, bool) { - checkedAllNodes := n.childrenChecked == len(n.queueOrder)+1 - if n.queuePosition == localQueueIndex { - return n, checkedAllNodes +func (rrs *roundRobinState) dequeueSelectNode(dequeueFrom *Node) (*Node, bool) { + checkedAllNodes := dequeueFrom.childrenChecked == len(dequeueFrom.queueOrder)+1 + if dequeueFrom.queuePosition == localQueueIndex { + return dequeueFrom, checkedAllNodes } - currentNodeName := n.queueOrder[n.queuePosition] - if node, ok := n.queueMap[currentNodeName]; ok { + currentNodeName := dequeueFrom.queueOrder[dequeueFrom.queuePosition] + if node, ok := dequeueFrom.queueMap[currentNodeName]; ok { return node, checkedAllNodes } return nil, checkedAllNodes } -// dequeueUpdateState increments queuePosition based on whether a child node was deleted during dequeue, -// and updates childrenChecked to reflect the number of children checked so that the dequeueGetNode operation -// will stop once it has checked all possible nodes. -func (rrs *roundRobinState) dequeueUpdateState(n *Node, v any, deletedNode bool) { - if v != nil { - n.childrenChecked = 0 - } else { - n.childrenChecked++ +// dequeueUpdateState does the following: +// - deletes the dequeued-from child node if it is empty after the dequeue operation +// - increments queuePosition if no child was deleted +// - updates childrenChecked to reflect the number of children checked so that the dequeueGetNode operation +// will stop once it has checked all possible nodes. +func (rrs *roundRobinState) dequeueUpdateState(parent *Node, dequeuedFrom *Node) { + // if the child node is nil, we haven't done anything to the tree; return early + if dequeuedFrom == nil { + return } - if !deletedNode { - n.queuePosition++ + + // corner case: if parent == dequeuedFrom and is empty, we will try to delete a "child" (no-op), + // and won't increment position. This is fine, because if the parent is empty, there's nothing to + // increment to. + childIsEmpty := dequeuedFrom.IsEmpty() + + // if the child is empty, we should delete it, but not increment queue position, since removing an element + // from queueOrder sets our position to the next element already. + if childIsEmpty { + childName := dequeuedFrom.Name() + delete(parent.queueMap, childName) + for idx, name := range parent.queueOrder { + if name == childName { + parent.queueOrder = append(parent.queueOrder[:idx], parent.queueOrder[idx+1:]...) + } + } + + } else { + parent.queuePosition++ } - if n.queuePosition >= len(n.queueOrder) { - n.queuePosition = localQueueIndex + if parent.queuePosition >= len(parent.queueOrder) { + parent.queuePosition = localQueueIndex } } From 57fadd62e97d8c0e9df8625f999964cc0025f387 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Wed, 12 Jun 2024 14:18:17 -0700 Subject: [PATCH 17/26] renaming args and removing config flag re: PR feedback --- cmd/mimir/config-descriptor.json | 11 ------ cmd/mimir/help-all.txt.tmpl | 2 -- .../configuration-parameters/index.md | 5 --- pkg/frontend/v1/frontend.go | 1 - pkg/scheduler/queue/queue.go | 5 +-- pkg/scheduler/queue/queue_test.go | 11 ++---- .../queue/tenant_querier_assignment.go | 22 ++++++------ .../queue/tree_queueing_algorithms.go | 34 +++++++++---------- pkg/scheduler/scheduler.go | 3 -- 9 files changed, 32 insertions(+), 62 deletions(-) diff --git a/cmd/mimir/config-descriptor.json b/cmd/mimir/config-descriptor.json index ee321dd8c9a..8da2843a345 100644 --- a/cmd/mimir/config-descriptor.json +++ b/cmd/mimir/config-descriptor.json @@ -15909,17 +15909,6 @@ "fieldType": "boolean", "fieldCategory": "experimental" }, - { - "kind": "field", - "name": "prioritize_query_components", - "required": false, - "desc": "Prioritize rotation through query components over rotation through tenants during dequeueing.", - "fieldValue": null, - "fieldDefaultValue": false, - "fieldFlag": "query-scheduler.prioritize-query-components", - "fieldType": "boolean", - "fieldCategory": "experimental" - }, { "kind": "field", "name": "querier_forget_delay", diff --git a/cmd/mimir/help-all.txt.tmpl b/cmd/mimir/help-all.txt.tmpl index 6f2b3e67ef8..7a6f26ade4b 100644 --- a/cmd/mimir/help-all.txt.tmpl +++ b/cmd/mimir/help-all.txt.tmpl @@ -2247,8 +2247,6 @@ Usage of ./cmd/mimir/mimir: Maximum number of outstanding requests per tenant per query-scheduler. In-flight requests above this limit will fail with HTTP response status code 429. (default 100) -query-scheduler.max-used-instances int The maximum number of query-scheduler instances to use, regardless how many replicas are running. This option can be set only when -query-scheduler.service-discovery-mode is set to 'ring'. 0 to use all available query-scheduler instances. - -query-scheduler.prioritize-query-components - [experimental] Prioritize rotation through query components over rotation through tenants during dequeueing. -query-scheduler.querier-forget-delay duration [experimental] If a querier disconnects without sending notification about graceful shutdown, the query-scheduler will keep the querier in the tenant's shard until the forget delay has passed. This feature is useful to reduce the blast radius when shuffle-sharding is enabled. -query-scheduler.ring.consul.acl-token string diff --git a/docs/sources/mimir/configure/configuration-parameters/index.md b/docs/sources/mimir/configure/configuration-parameters/index.md index 3c1424d5e2b..5f114817834 100644 --- a/docs/sources/mimir/configure/configuration-parameters/index.md +++ b/docs/sources/mimir/configure/configuration-parameters/index.md @@ -1708,11 +1708,6 @@ The `query_scheduler` block configures the query-scheduler. # CLI flag: -query-scheduler.additional-query-queue-dimensions-enabled [additional_query_queue_dimensions_enabled: | default = false] -# (experimental) Prioritize rotation through query components over rotation -# through tenants during dequeueing. -# CLI flag: -query-scheduler.prioritize-query-components -[prioritize_query_components: | default = false] - # (experimental) If a querier disconnects without sending notification about # graceful shutdown, the query-scheduler will keep the querier in the tenant's # shard until the forget delay has passed. This feature is useful to reduce the diff --git a/pkg/frontend/v1/frontend.go b/pkg/frontend/v1/frontend.go index 82bfc98d1e1..9f7d6929eeb 100644 --- a/pkg/frontend/v1/frontend.go +++ b/pkg/frontend/v1/frontend.go @@ -128,7 +128,6 @@ func New(cfg Config, limits Limits, log log.Logger, registerer prometheus.Regist log, cfg.MaxOutstandingPerTenant, false, - false, cfg.QuerierForgetDelay, f.queueLength, f.discardedRequests, diff --git a/pkg/scheduler/queue/queue.go b/pkg/scheduler/queue/queue.go index 4c9543554c5..8578a21656b 100644 --- a/pkg/scheduler/queue/queue.go +++ b/pkg/scheduler/queue/queue.go @@ -121,7 +121,6 @@ type RequestQueue struct { // settings maxOutstandingPerTenant int additionalQueueDimensionsEnabled bool - prioritizeQueryComponents bool forgetDelay time.Duration // metrics for reporting @@ -175,7 +174,6 @@ func NewRequestQueue( log log.Logger, maxOutstandingPerTenant int, additionalQueueDimensionsEnabled bool, - prioritizeQueryComponents bool, forgetDelay time.Duration, queueLength *prometheus.GaugeVec, discardedRequests *prometheus.CounterVec, @@ -192,7 +190,6 @@ func NewRequestQueue( log: log, maxOutstandingPerTenant: maxOutstandingPerTenant, additionalQueueDimensionsEnabled: additionalQueueDimensionsEnabled, - prioritizeQueryComponents: prioritizeQueryComponents, forgetDelay: forgetDelay, // metrics for reporting @@ -213,7 +210,7 @@ func NewRequestQueue( waitingQuerierConnsToDispatch: list.New(), QueryComponentUtilization: queryComponentCapacity, - queueBroker: newQueueBroker(maxOutstandingPerTenant, additionalQueueDimensionsEnabled, prioritizeQueryComponents, forgetDelay), + queueBroker: newQueueBroker(maxOutstandingPerTenant, additionalQueueDimensionsEnabled, false, forgetDelay), } q.Service = services.NewBasicService(q.starting, q.running, q.stop).WithName("request queue") diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index 1fb776087e4..c9c3a070f43 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -132,7 +132,6 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { log.NewNopLogger(), maxOutstandingRequestsPerTenant, additionalQueueDimensionsEnabled, - false, forgetQuerierDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -238,7 +237,6 @@ func BenchmarkConcurrentQueueOperations(b *testing.B) { log.NewNopLogger(), maxOutstandingRequestsPerTenant, true, - false, forgetQuerierDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -410,7 +408,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldGetRequestAfterReshardingBe queue, err := NewRequestQueue( log.NewNopLogger(), - 1, true, false, + 1, true, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -479,7 +477,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ReshardNotifiedCorrectlyForMultip queue, err := NewRequestQueue( log.NewNopLogger(), - 1, true, false, + 1, true, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -564,7 +562,6 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldReturnAfterContextCancelled log.NewNopLogger(), 1, true, - false, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -616,7 +613,6 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldReturnImmediatelyIfQuerierI log.NewNopLogger(), 1, true, - false, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -646,7 +642,6 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend log.NewNopLogger(), 1, true, - false, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -657,7 +652,7 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend // bypassing queue dispatcher loop for direct usage of the queueBroker and // passing a waitingQuerierConn for a canceled querier connection - queueBroker := newQueueBroker(queue.maxOutstandingPerTenant, queue.additionalQueueDimensionsEnabled, queue.prioritizeQueryComponents, queue.forgetDelay) + queueBroker := newQueueBroker(queue.maxOutstandingPerTenant, queue.additionalQueueDimensionsEnabled, false, queue.forgetDelay) queueBroker.addQuerierConnection(querierID) tenantMaxQueriers := 0 // no sharding diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 0d9207673ce..ab92df3293f 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -323,13 +323,13 @@ func (tqa *tenantQuerierAssignments) shuffleTenantQueriers(tenantID TenantID, sc // // Note that because we use the shared tenantIDOrder and tenantOrderIndex to manage the queue, we functionally // ignore each Node's individual queueOrder and queuePosition. -func (tqa *tenantQuerierAssignments) dequeueSelectNode(dequeueFrom *Node) (*Node, bool) { +func (tqa *tenantQuerierAssignments) dequeueSelectNode(node *Node) (*Node, bool) { // can't get a tenant if no querier set if tqa.currentQuerier == nil { return nil, true } - checkedAllNodes := dequeueFrom.childrenChecked == len(dequeueFrom.queueMap)+1 // must check local queue as well + checkedAllNodes := node.childrenChecked == len(node.queueMap)+1 // must check local queue as well // advance queue position for dequeue tqa.tenantOrderIndex++ @@ -338,8 +338,8 @@ func (tqa *tenantQuerierAssignments) dequeueSelectNode(dequeueFrom *Node) (*Node } // no children - if len(dequeueFrom.queueMap) == 0 || tqa.tenantOrderIndex == localQueueIndex { - return dequeueFrom, checkedAllNodes + if len(node.queueMap) == 0 || tqa.tenantOrderIndex == localQueueIndex { + return node, checkedAllNodes } checkIndex := tqa.tenantOrderIndex @@ -351,25 +351,25 @@ func (tqa *tenantQuerierAssignments) dequeueSelectNode(dequeueFrom *Node) (*Node tenantID := tqa.tenantIDOrder[checkIndex] tenantName := string(tenantID) - if _, ok := dequeueFrom.queueMap[tenantName]; !ok { + if _, ok := node.queueMap[tenantName]; !ok { // tenant not in _this_ node's children, move on checkIndex++ continue } // increment nodes checked even if not in tenant-querier map - dequeueFrom.childrenChecked++ - checkedAllNodes = dequeueFrom.childrenChecked == len(dequeueFrom.queueMap)+1 + node.childrenChecked++ + checkedAllNodes = node.childrenChecked == len(node.queueMap)+1 // if the tenant-querier set is nil, any querier can serve this tenant if tqa.tenantQuerierIDs[tenantID] == nil { tqa.tenantOrderIndex = checkIndex - return dequeueFrom.queueMap[tenantName], checkedAllNodes + return node.queueMap[tenantName], checkedAllNodes } if tenantQuerierSet, ok := tqa.tenantQuerierIDs[tenantID]; ok { if _, ok := tenantQuerierSet[*tqa.currentQuerier]; ok { tqa.tenantOrderIndex = checkIndex - return dequeueFrom.queueMap[tenantName], checkedAllNodes + return node.queueMap[tenantName], checkedAllNodes } } checkIndex++ @@ -385,7 +385,7 @@ func (tqa *tenantQuerierAssignments) dequeueSelectNode(dequeueFrom *Node) (*Node // // dequeueUpdateState would normally also handle incrementing the queue position after performing a dequeue, but // tenantQuerierAssignments currently expects the caller to handle this by having the querier set tenantOrderIndex. -func (tqa *tenantQuerierAssignments) dequeueUpdateState(parent *Node, dequeuedFrom *Node) { +func (tqa *tenantQuerierAssignments) dequeueUpdateState(node *Node, dequeuedFrom *Node) { // if dequeuedFrom is nil or is not empty, we don't need to do anything; // position updates will be handled by the caller, and we don't need to remove any nodes. if dequeuedFrom == nil || !dequeuedFrom.IsEmpty() { @@ -394,7 +394,7 @@ func (tqa *tenantQuerierAssignments) dequeueUpdateState(parent *Node, dequeuedFr // delete from the node's children childName := dequeuedFrom.Name() - delete(parent.queueMap, childName) + delete(node.queueMap, childName) // delete from shared tenantNodes for i, tenantNode := range tqa.tenantNodes[childName] { diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go index 14800c1ab1e..982e45415d4 100644 --- a/pkg/scheduler/queue/tree_queueing_algorithms.go +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -7,8 +7,8 @@ package queue // that may be stored in a struct that implements DequeueAlgorithm. type DequeueAlgorithm interface { addChildNode(parent, child *Node) - dequeueSelectNode(dequeueFrom *Node) (*Node, bool) - dequeueUpdateState(dequeueParent *Node, dequeuedFrom *Node) + dequeueSelectNode(node *Node) (*Node, bool) + dequeueUpdateState(node *Node, dequeuedFrom *Node) } // roundRobinState is the simplest type of DequeueAlgorithm; nodes which use this DequeueAlgorithm and are at @@ -38,14 +38,14 @@ func (rrs *roundRobinState) addChildNode(parent, child *Node) { // dequeueGetNode returns the node at the node's queuePosition. queuePosition represents the position of // the next node to dequeue from, and is incremented in dequeueUpdateState. -func (rrs *roundRobinState) dequeueSelectNode(dequeueFrom *Node) (*Node, bool) { - checkedAllNodes := dequeueFrom.childrenChecked == len(dequeueFrom.queueOrder)+1 - if dequeueFrom.queuePosition == localQueueIndex { - return dequeueFrom, checkedAllNodes +func (rrs *roundRobinState) dequeueSelectNode(node *Node) (*Node, bool) { + checkedAllNodes := node.childrenChecked == len(node.queueOrder)+1 + if node.queuePosition == localQueueIndex { + return node, checkedAllNodes } - currentNodeName := dequeueFrom.queueOrder[dequeueFrom.queuePosition] - if node, ok := dequeueFrom.queueMap[currentNodeName]; ok { + currentNodeName := node.queueOrder[node.queuePosition] + if node, ok := node.queueMap[currentNodeName]; ok { return node, checkedAllNodes } return nil, checkedAllNodes @@ -56,14 +56,14 @@ func (rrs *roundRobinState) dequeueSelectNode(dequeueFrom *Node) (*Node, bool) { // - increments queuePosition if no child was deleted // - updates childrenChecked to reflect the number of children checked so that the dequeueGetNode operation // will stop once it has checked all possible nodes. -func (rrs *roundRobinState) dequeueUpdateState(parent *Node, dequeuedFrom *Node) { +func (rrs *roundRobinState) dequeueUpdateState(node *Node, dequeuedFrom *Node) { // if the child node is nil, we haven't done anything to the tree; return early if dequeuedFrom == nil { return } - // corner case: if parent == dequeuedFrom and is empty, we will try to delete a "child" (no-op), - // and won't increment position. This is fine, because if the parent is empty, there's nothing to + // corner case: if node == dequeuedFrom and is empty, we will try to delete a "child" (no-op), + // and won't increment position. This is fine, because if the node is empty, there's nothing to // increment to. childIsEmpty := dequeuedFrom.IsEmpty() @@ -71,17 +71,17 @@ func (rrs *roundRobinState) dequeueUpdateState(parent *Node, dequeuedFrom *Node) // from queueOrder sets our position to the next element already. if childIsEmpty { childName := dequeuedFrom.Name() - delete(parent.queueMap, childName) - for idx, name := range parent.queueOrder { + delete(node.queueMap, childName) + for idx, name := range node.queueOrder { if name == childName { - parent.queueOrder = append(parent.queueOrder[:idx], parent.queueOrder[idx+1:]...) + node.queueOrder = append(node.queueOrder[:idx], node.queueOrder[idx+1:]...) } } } else { - parent.queuePosition++ + node.queuePosition++ } - if parent.queuePosition >= len(parent.queueOrder) { - parent.queuePosition = localQueueIndex + if node.queuePosition >= len(node.queueOrder) { + node.queuePosition = localQueueIndex } } diff --git a/pkg/scheduler/scheduler.go b/pkg/scheduler/scheduler.go index be77d70ec56..83764816411 100644 --- a/pkg/scheduler/scheduler.go +++ b/pkg/scheduler/scheduler.go @@ -95,7 +95,6 @@ type connectedFrontend struct { type Config struct { MaxOutstandingPerTenant int `yaml:"max_outstanding_requests_per_tenant"` AdditionalQueryQueueDimensionsEnabled bool `yaml:"additional_query_queue_dimensions_enabled" category:"experimental"` - PrioritizeQueryComponents bool `yaml:"prioritize_query_components" category:"experimental"` QuerierForgetDelay time.Duration `yaml:"querier_forget_delay" category:"experimental"` GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config" doc:"description=This configures the gRPC client used to report errors back to the query-frontend."` @@ -105,7 +104,6 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.IntVar(&cfg.MaxOutstandingPerTenant, "query-scheduler.max-outstanding-requests-per-tenant", 100, "Maximum number of outstanding requests per tenant per query-scheduler. In-flight requests above this limit will fail with HTTP response status code 429.") f.BoolVar(&cfg.AdditionalQueryQueueDimensionsEnabled, "query-scheduler.additional-query-queue-dimensions-enabled", false, "Enqueue query requests with additional queue dimensions to split tenant request queues into subqueues. This enables separate requests to proceed from a tenant's subqueues even when other subqueues are blocked on slow query requests. Must be set on both query-frontend and scheduler to take effect. (default false)") - f.BoolVar(&cfg.PrioritizeQueryComponents, "query-scheduler.prioritize-query-components", false, "Prioritize rotation through query components over rotation through tenants during dequeueing.") f.DurationVar(&cfg.QuerierForgetDelay, "query-scheduler.querier-forget-delay", 0, "If a querier disconnects without sending notification about graceful shutdown, the query-scheduler will keep the querier in the tenant's shard until the forget delay has passed. This feature is useful to reduce the blast radius when shuffle-sharding is enabled.") cfg.GRPCClientConfig.RegisterFlagsWithPrefix("query-scheduler.grpc-client-config", f) @@ -162,7 +160,6 @@ func NewScheduler(cfg Config, limits Limits, log log.Logger, registerer promethe s.log, cfg.MaxOutstandingPerTenant, cfg.AdditionalQueryQueueDimensionsEnabled, - cfg.PrioritizeQueryComponents, cfg.QuerierForgetDelay, s.queueLength, s.discardedRequests, From 9be157abf5471766412e56160c0be1e4bb069a85 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Wed, 12 Jun 2024 15:07:49 -0700 Subject: [PATCH 18/26] some test cleanup --- pkg/scheduler/queue/tree_queue_test.go | 73 ++++++++++++-------------- 1 file changed, 33 insertions(+), 40 deletions(-) diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index d8c732c6389..719c82335f0 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -10,6 +10,13 @@ import ( "github.com/stretchr/testify/require" ) +// enqueueObj is intended for use in tests; it represents an obj to enqueue to path, or an expected path and obj +// for a given dequeue +type enqueueObj struct { + obj any + path QueuePath +} + func newTenantQuerierAssignments() *tenantQuerierAssignments { return &tenantQuerierAssignments{ tenantQuerierIDs: make(map[TenantID]map[QuerierID]struct{}), @@ -30,10 +37,6 @@ func Test_NewTree(t *testing.T) { }, { name: "create shuffle-shard tree", - treeAlgos: []DequeueAlgorithm{newTenantQuerierAssignments()}, - }, - { - name: "create shuffle-shard tree with no tenant-querier map, we should create an empty one", treeAlgos: []DequeueAlgorithm{&tenantQuerierAssignments{}}, }, { @@ -57,48 +60,51 @@ func Test_NewTree(t *testing.T) { func Test_EnqueueBackByPath(t *testing.T) { tests := []struct { - name string - treeAlgosByDepth []DequeueAlgorithm - children []QueuePath - expectErr bool + name string + treeAlgosByDepth []DequeueAlgorithm + childPathsToEnqueue []QueuePath + expectErr bool }{ { - name: "enqueue round-robin node to round-robin node", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, - children: []QueuePath{{"round-robin-child-1"}}, + name: "enqueue tenant-querier node to round-robin node", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, + childPathsToEnqueue: []QueuePath{{"child-1"}}, }, { - name: "enqueue shuffle-shard node to round-robin node", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, - children: []QueuePath{{"shuffle-shard-child-1"}}, + name: "enqueue tenant-querier node to round-robin node", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, + childPathsToEnqueue: []QueuePath{{"child-1"}, {"child-2"}}, }, { - name: "enqueue shuffle-shard node with no tenant-querier map to round-robin node, we should create an empty one", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{}}, - children: []QueuePath{{"shuffle-shard-child-1"}}, + name: "enqueue round-robin node to tenant-querier node", + treeAlgosByDepth: []DequeueAlgorithm{ + newTenantQuerierAssignments(), + &roundRobinState{}, + }, + childPathsToEnqueue: []QueuePath{{"child-1"}, {"child-2"}}, }, { - name: "enqueue round-robin node to shuffle-shard node", + name: "enqueue tenant-querier node to tenant-querier node", treeAlgosByDepth: []DequeueAlgorithm{ newTenantQuerierAssignments(), - &roundRobinState{}, + newTenantQuerierAssignments(), }, - children: []QueuePath{{"round-robin-child-1"}}, + childPathsToEnqueue: []QueuePath{{"child-1"}}, }, { - name: "create tree with multiple shuffle-shard depths", + name: "enqueue grandchildren to a tree with max-depth of 2", treeAlgosByDepth: []DequeueAlgorithm{ newTenantQuerierAssignments(), &roundRobinState{}, newTenantQuerierAssignments(), }, - children: []QueuePath{{"child"}, {"grandchild"}}, + childPathsToEnqueue: []QueuePath{{"child"}, {"grandchild"}}, }, { - name: "enqueue beyond max-depth", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}}, - children: []QueuePath{{"child"}, {"child, grandchild"}}, - expectErr: true, + name: "enqueue beyond max-depth", + treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}}, + childPathsToEnqueue: []QueuePath{{"child"}, {"child, grandchild"}}, + expectErr: true, }, } for _, tt := range tests { @@ -106,7 +112,7 @@ func Test_EnqueueBackByPath(t *testing.T) { tree, err := NewTree(tt.treeAlgosByDepth...) require.NoError(t, err) - for _, childPath := range tt.children { + for _, childPath := range tt.childPathsToEnqueue { err = tree.EnqueueBackByPath(childPath, "some-object") } if tt.expectErr { @@ -119,10 +125,6 @@ func Test_EnqueueBackByPath(t *testing.T) { } func Test_EnqueueFrontByPath(t *testing.T) { - type enqueueObj struct { - obj any - path QueuePath - } someQuerier := QuerierID("placeholder") tests := []struct { name string @@ -386,11 +388,6 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { } func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { - type enqueueObj struct { - obj any - path QueuePath - } - tests := []struct { name string treeAlgosByDepth []DequeueAlgorithm @@ -618,10 +615,6 @@ func Test_ChangeTenantQuerierAssignments(t *testing.T) { tree, err := NewTree(state, &roundRobinState{}, &roundRobinState{}) require.NoError(t, err) - type enqueueObj struct { - obj any - path QueuePath - } enqueueObjs := []enqueueObj{ {"query-1", QueuePath{"tenant-1", "query-component-1"}}, {"query-2", QueuePath{"tenant-2", "query-component-1"}}, From 2b457137ae898769934b251bc2d94d3b67f270e1 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 17 Jun 2024 14:08:01 -0700 Subject: [PATCH 19/26] PR feedback on tests, mostly --- .../queue/tenant_querier_assignment.go | 12 +++++-- pkg/scheduler/queue/tenant_queues_test.go | 35 ++++++++++--------- 2 files changed, 28 insertions(+), 19 deletions(-) diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index ab92df3293f..18e681e5017 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -337,15 +337,20 @@ func (tqa *tenantQuerierAssignments) dequeueSelectNode(node *Node) (*Node, bool) tqa.tenantOrderIndex = localQueueIndex } - // no children + // no children or local queue reached if len(node.queueMap) == 0 || tqa.tenantOrderIndex == localQueueIndex { return node, checkedAllNodes } checkIndex := tqa.tenantOrderIndex + // iterate through the tenant order until we find a tenant that is assigned to the current querier, or + // have checked the entire tenantIDOrder, whichever comes first for iters := 0; iters < len(tqa.tenantIDOrder); iters++ { if checkIndex >= len(tqa.tenantIDOrder) { + // do not use modulo to wrap this index; tenantOrderIndex is provided from an outer process + // which does not know if the tenant list has changed since the last dequeue + // wrapping with modulo after shrinking the list could cause us to skip tenants checkIndex = 0 } tenantID := tqa.tenantIDOrder[checkIndex] @@ -357,7 +362,9 @@ func (tqa *tenantQuerierAssignments) dequeueSelectNode(node *Node) (*Node, bool) continue } - // increment nodes checked even if not in tenant-querier map + // childrenChecked is used to determine when we can stop looking for a dequeueable node. At this point, + // tenantID _must_ exist in the node's children, even if currentQuerier isn't assigned to it, so we increment + // childrenChecked regardless of whether currentQuerier exists in tenantQuerierIDs for tenantID. node.childrenChecked++ checkedAllNodes = node.childrenChecked == len(node.queueMap)+1 @@ -366,6 +373,7 @@ func (tqa *tenantQuerierAssignments) dequeueSelectNode(node *Node) (*Node, bool) tqa.tenantOrderIndex = checkIndex return node.queueMap[tenantName], checkedAllNodes } + // otherwise, check if the querier is assigned to this tenant if tenantQuerierSet, ok := tqa.tenantQuerierIDs[tenantID]; ok { if _, ok := tenantQuerierSet[*tqa.currentQuerier]; ok { tqa.tenantOrderIndex = checkIndex diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index eb1418aa854..10b9d040b51 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -58,7 +58,9 @@ func assertExpectedValuesOnDequeue(t *testing.T, qb *queueBroker, lastTenantInde return lastTenantIndex } -func TestQueues(t *testing.T) { +// TestQueues_NoShuffleSharding tests dequeueing of objects for different queriers, where any querier can +// handle queries for any tenant, as tenant queues are added and removed +func TestQueues_NoShuffleSharding(t *testing.T) { qb := newQueueBroker(0, true, false, 0) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -66,7 +68,7 @@ func TestQueues(t *testing.T) { qb.addQuerierConnection("querier-1") qb.addQuerierConnection("querier-2") - req, tenant, lastTenantIndex, err := qb.dequeueRequestForQuerier(-1, "querier-1") + req, tenant, lastTenantIndexQuerierOne, err := qb.dequeueRequestForQuerier(-1, "querier-1") assert.Nil(t, req) assert.Nil(t, tenant) assert.NoError(t, err) @@ -87,7 +89,7 @@ func TestQueues(t *testing.T) { {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, } - lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) // Add tenant two err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) @@ -106,16 +108,16 @@ func TestQueues(t *testing.T) { {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, } - lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) expectedDequeueVals = []dequeueVal{ {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, } - _ = assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) + lastTenantIndexQuerierTwo := assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) - //[one two three] + // [one two three] // confirm fifo by adding a third tenant queue and iterating to it err = qb.tenantQuerierAssignments.createOrUpdateTenant("three", 0) assert.NoError(t, err) @@ -126,13 +128,14 @@ func TestQueues(t *testing.T) { assert.NoError(t, err) assert.NoError(t, isConsistent(qb)) + // lastTenantIndexQuerierOne was 0 (tenantOne) for querier-1; appending expectedDequeueVals = []dequeueVal{ {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, {buildExpectedObject(tenantThree.tenantID, 0), tenantThree}, {buildExpectedObject(tenantOne.tenantID, 6), tenantOne}, } - lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) // Remove one: ["" two three] qb.removeTenantQueue("one") @@ -143,7 +146,7 @@ func TestQueues(t *testing.T) { {buildExpectedObject(tenantThree.tenantID, 1), tenantThree}, {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, } - lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) // "four" is added at the beginning of the list: [four two three] err = qb.tenantQuerierAssignments.createOrUpdateTenant("four", 0) @@ -156,23 +159,23 @@ func TestQueues(t *testing.T) { assert.NoError(t, isConsistent(qb)) expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantTwo.tenantID, 6), tenantTwo}, {buildExpectedObject(tenantThree.tenantID, 2), tenantThree}, {buildExpectedObject(tenantFour.tenantID, 0), tenantFour}, - {buildExpectedObject(tenantTwo.tenantID, 6), tenantTwo}, - {buildExpectedObject(tenantThree.tenantID, 3), tenantThree}, + {buildExpectedObject(tenantTwo.tenantID, 7), tenantTwo}, } - lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) + lastTenantIndexQuerierTwo = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierTwo, "querier-2", expectedDequeueVals) // Remove two: [four "" three] qb.removeTenantQueue("two") assert.NoError(t, isConsistent(qb)) expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantThree.tenantID, 3), tenantThree}, {buildExpectedObject(tenantFour.tenantID, 1), tenantFour}, {buildExpectedObject(tenantThree.tenantID, 4), tenantThree}, - {buildExpectedObject(tenantFour.tenantID, 2), tenantFour}, } - lastTenantIndex = assertExpectedValuesOnDequeue(t, qb, lastTenantIndex, "querier-1", expectedDequeueVals) + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) // Remove three: [four ""] ) qb.removeTenantQueue("three") @@ -182,7 +185,7 @@ func TestQueues(t *testing.T) { qb.removeTenantQueue("four") assert.NoError(t, isConsistent(qb)) - req, tenant, _, err = qb.dequeueRequestForQuerier(lastTenantIndex, "querier-1") + req, tenant, _, err = qb.dequeueRequestForQuerier(lastTenantIndexQuerierOne, "querier-1") assert.Nil(t, req) assert.Nil(t, tenant) assert.NoError(t, err) @@ -319,7 +322,7 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { } -func TestQueuesWithQueriers(t *testing.T) { +func TestQueues_QuerierDistribution(t *testing.T) { qb := newQueueBroker(0, true, false, 0) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -345,7 +348,6 @@ func TestQueuesWithQueriers(t *testing.T) { // Add tenant queues. for i := 0; i < numTenants; i++ { uid := TenantID(fmt.Sprintf("tenant-%d", i)) - //getOrAdd(t, qb, uid, maxQueriersPerTenant) err := qb.tenantQuerierAssignments.createOrUpdateTenant(uid, maxQueriersPerTenant) assert.NoError(t, err) @@ -563,7 +565,6 @@ func TestQueues_ForgetDelay_ShouldCorrectlyHandleQuerierReconnectingBeforeForget //Enqueue some stuff so that the tree queue node exists err = qb.enqueueObjectsForTests(tenantID, 1) assert.NoError(t, err) - //getOrAdd(t, qb, tenantID, maxQueriersPerTenant) } // We expect querier-1 to have some tenants. From 872a80d7b3deb05659a5e7ebe3852d1e5fd50781 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Tue, 18 Jun 2024 16:50:26 -0700 Subject: [PATCH 20/26] Patrick PR feedback: Comment and naming updates --- .../queue/tenant_querier_assignment.go | 8 +-- pkg/scheduler/queue/tenant_queues.go | 12 ++-- pkg/scheduler/queue/tenant_queues_test.go | 2 +- pkg/scheduler/queue/tree_queue.go | 40 +++++------ pkg/scheduler/queue/tree_queue_test.go | 70 +++++++++---------- .../queue/tree_queueing_algorithms.go | 30 +++++--- 6 files changed, 85 insertions(+), 77 deletions(-) diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 18e681e5017..87e4d1261bf 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -53,7 +53,7 @@ func (s querierIDSlice) Search(x QuerierID) int { return sort.Search(len(s), func(i int) bool { return s[i] >= x }) } -// tenantQuerierAssignments implements DequeueAlgorithm. In the context of a TreeQueue, it maintains a mapping of +// tenantQuerierAssignments implements QueuingAlgorithm. In the context of a TreeQueue, it maintains a mapping of // tenants to queriers in order to support dequeuing from an appropriate tenant if shuffle-sharding is enabled. type tenantQuerierAssignments struct { // a tenant has many queriers @@ -312,7 +312,7 @@ func (tqa *tenantQuerierAssignments) shuffleTenantQueriers(tenantID TenantID, sc return true } -// dequeueGetNode chooses the next node to dequeue from based on tenantIDOrder and tenantOrderIndex, which are +// dequeueSelectNode chooses the next node to dequeue from based on tenantIDOrder and tenantOrderIndex, which are // shared across all nodes to maintain an O(n) (where n = # tenants) time-to-dequeue for each tenant. // If tenant order were maintained by individual nodes, we would end up with O(mn) (where m = # query components) // time-to-dequeue for a given tenant. @@ -362,10 +362,6 @@ func (tqa *tenantQuerierAssignments) dequeueSelectNode(node *Node) (*Node, bool) continue } - // childrenChecked is used to determine when we can stop looking for a dequeueable node. At this point, - // tenantID _must_ exist in the node's children, even if currentQuerier isn't assigned to it, so we increment - // childrenChecked regardless of whether currentQuerier exists in tenantQuerierIDs for tenantID. - node.childrenChecked++ checkedAllNodes = node.childrenChecked == len(node.queueMap)+1 // if the tenant-querier set is nil, any querier can serve this tenant diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 10a7dded3a9..7b923ae1de5 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -55,15 +55,15 @@ func newQueueBroker( // with query components at one level above tenants; if it is false, // tenant nodes will each maintain their own query component subtree. tree, err := NewTree( - tqas, // root; DequeueAlgorithm selects tenants - &roundRobinState{}, // tenant queues; DequeueAlgorithm selects query component - &roundRobinState{}, // query components; DequeueAlgorithm selects query from local queue + tqas, // root; QueuingAlgorithm selects tenants + &roundRobinState{}, // tenant queues; QueuingAlgorithm selects query component + &roundRobinState{}, // query components; QueuingAlgorithm selects query from local queue ) if prioritizeQueryComponents { tree, err = NewTree( - &roundRobinState{}, // root; DequeueAlgorithm selects query component - tqas, // query components; DequeueAlgorithm selects tenant - &roundRobinState{}, // tenant queues; DequeueAlgorithm selects query from local queue + &roundRobinState{}, // root; QueuingAlgorithm selects query component + tqas, // query components; QueuingAlgorithm selects tenant + &roundRobinState{}, // tenant queues; QueuingAlgorithm selects query from local queue ) } diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 10b9d040b51..92c3f62a90c 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -722,7 +722,7 @@ func (n *Node) deleteNode(pathFromNode QueuePath) bool { // empty the node so that update state properly deletes it deleteNode.queueMap = map[string]*Node{} deleteNode.localQueue = list.New() - parentNode.dequeueAlgorithm.dequeueUpdateState(parentNode, deleteNode) + parentNode.queuingAlgorithm.dequeueUpdateState(parentNode, deleteNode) return true } return false diff --git a/pkg/scheduler/queue/tree_queue.go b/pkg/scheduler/queue/tree_queue.go index 4cf2292130a..cdda2b6420e 100644 --- a/pkg/scheduler/queue/tree_queue.go +++ b/pkg/scheduler/queue/tree_queue.go @@ -14,7 +14,7 @@ const localQueueIndex = -1 // TreeQueue holds metadata and a pointer to the root node of a hierarchical queue implementation. // The root Node maintains a localQueue and an arbitrary number of child nodes (which themselves -// may have local queues and children). Each Node in TreeQueue uses a DequeueAlgorithm (determined by +// may have local queues and children). Each Node in TreeQueue uses a QueuingAlgorithm (determined by // node depth) to determine dequeue order of that Node's subtree. // // Each queuing dimension is modeled as a node in the tree, internally reachable through a QueuePath. @@ -22,27 +22,27 @@ const localQueueIndex = -1 // The QueuePath is an ordered array of strings describing the path from the tree root to a Node. // In addition to child Nodes, each Node contains a local queue (FIFO) of items. // -// When dequeuing from a given node, a Node will use its DequeueAlgorithm to choose either itself -// or a child node to dequeue from recursively (i.e., a child Node will use its own DequeueAlgorithm +// When dequeuing from a given node, a Node will use its QueuingAlgorithm to choose either itself +// or a child node to dequeue from recursively (i.e., a child Node will use its own QueuingAlgorithm // to determine how to proceed). TreeQueue will not dequeue from two different Nodes at the same depth // consecutively, unless the previously-checked Node was empty down to the leaf node. type TreeQueue struct { rootNode *Node - algosByDepth []DequeueAlgorithm + algosByDepth []QueuingAlgorithm } -func NewTree(dequeueAlgorithms ...DequeueAlgorithm) (*TreeQueue, error) { - if len(dequeueAlgorithms) == 0 { - return nil, fmt.Errorf("cannot create a tree without defined DequeueAlgorithm") +func NewTree(queuingAlgorithms ...QueuingAlgorithm) (*TreeQueue, error) { + if len(queuingAlgorithms) == 0 { + return nil, fmt.Errorf("cannot create a tree without defined QueuingAlgorithm") } - root, err := newNode("root", 0, dequeueAlgorithms[0]) + root, err := newNode("root", 0, queuingAlgorithms[0]) if err != nil { return nil, err } root.depth = 0 return &TreeQueue{ rootNode: root, - algosByDepth: dequeueAlgorithms, + algosByDepth: queuingAlgorithms, }, nil } @@ -53,7 +53,7 @@ func (t *TreeQueue) IsEmpty() bool { // Dequeue removes and returns an item from the front of the next appropriate Node in the TreeQueue, as // well as the path to the Node which that item was dequeued from. // -// Either the root/self node or a child node is chosen according to the Node's DequeueAlgorithm. If +// Either the root/self node or a child node is chosen according to the Node's QueuingAlgorithm. If // the root node is chosen, an item will be dequeued from the front of its localQueue. If a child // node is chosen, it is recursively dequeued from until a node selects its localQueue. // @@ -100,7 +100,7 @@ func (t *TreeQueue) GetNode(path QueuePath) *Node { // Node maintains node-specific information used to enqueue and dequeue to itself, such as a local // queue, node depth, references to its children, and position in queue. -// Note that the tenantQuerierAssignments DequeueAlgorithm largely disregards Node's queueOrder and +// Note that the tenantQuerierAssignments QueuingAlgorithm largely disregards Node's queueOrder and // queuePosition, managing analogous state instead, because shuffle-sharding + fairness requirements // necessitate input from the querier. type Node struct { @@ -110,13 +110,13 @@ type Node struct { queueOrder []string // order for dequeuing from self/children queueMap map[string]*Node depth int - dequeueAlgorithm DequeueAlgorithm + queuingAlgorithm QueuingAlgorithm childrenChecked int } -func newNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { +func newNode(name string, depth int, da QueuingAlgorithm) (*Node, error) { if da == nil { - return nil, fmt.Errorf("cannot create a node without a defined DequeueAlgorithm") + return nil, fmt.Errorf("cannot create a node without a defined QueuingAlgorithm") } if tqa, ok := da.(*tenantQuerierAssignments); ok { tqa.tenantOrderIndex = localQueueIndex - 1 // start from -2 so that we first check local queue @@ -131,7 +131,7 @@ func newNode(name string, depth int, da DequeueAlgorithm) (*Node, error) { queueOrder: make([]string, 0), queueMap: make(map[string]*Node, 1), depth: depth, - dequeueAlgorithm: da, + queuingAlgorithm: da, }, nil } @@ -197,7 +197,7 @@ func (n *Node) dequeue() (QueuePath, any) { var dequeueNode *Node // continue until we've found a value or checked all nodes that need checking for v == nil && !checkedAllNodes { - dequeueNode, checkedAllNodes = n.dequeueAlgorithm.dequeueSelectNode(n) + dequeueNode, checkedAllNodes = n.queuingAlgorithm.dequeueSelectNode(n) switch dequeueNode { // dequeuing from local queue case n: @@ -222,9 +222,9 @@ func (n *Node) dequeue() (QueuePath, any) { n.childrenChecked++ } - n.dequeueAlgorithm.dequeueUpdateState(n, dequeueNode) + n.queuingAlgorithm.dequeueUpdateState(n, dequeueNode) } - // reset children checked to 0 before completing this dequeue + // reset childrenChecked to 0 before completing this dequeue n.childrenChecked = 0 return append(path, childPath...), v } @@ -248,7 +248,7 @@ func (n *Node) getNode(pathFromNode QueuePath) *Node { // getOrAddNode recursively gets or adds tree queue nodes based on given relative child path. It // checks whether the first node in pathFromNode exists in the Node's children; if no node exists, -// one is created and added to the Node's queueOrder, according to the Node's DequeueAlgorithm. +// one is created and added to the Node's queueOrder, according to the Node's QueuingAlgorithm. // // pathFromNode must be relative to the receiver node; providing a QueuePath beginning with // the receiver/parent node name will create a child node of the same name as the parent. @@ -270,7 +270,7 @@ func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *TreeQueue) (*Node, err return nil, err } // add the newly created child to the node - n.dequeueAlgorithm.addChildNode(n, childNode) + n.queuingAlgorithm.addChildNode(n, childNode) } return childNode.getOrAddNode(pathFromNode[1:], tree) diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index 719c82335f0..682f1917426 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -27,21 +27,21 @@ func newTenantQuerierAssignments() *tenantQuerierAssignments { func Test_NewTree(t *testing.T) { tests := []struct { name string - treeAlgos []DequeueAlgorithm + treeAlgos []QueuingAlgorithm state *tenantQuerierAssignments expectErr bool }{ { name: "create round-robin tree", - treeAlgos: []DequeueAlgorithm{&roundRobinState{}}, + treeAlgos: []QueuingAlgorithm{&roundRobinState{}}, }, { - name: "create shuffle-shard tree", - treeAlgos: []DequeueAlgorithm{&tenantQuerierAssignments{}}, + name: "create tenant-querier tree", + treeAlgos: []QueuingAlgorithm{&tenantQuerierAssignments{}}, }, { name: "fail to create tree without defined dequeuing algorithm", - treeAlgos: []DequeueAlgorithm{nil}, + treeAlgos: []QueuingAlgorithm{nil}, expectErr: true, }, } @@ -61,23 +61,23 @@ func Test_NewTree(t *testing.T) { func Test_EnqueueBackByPath(t *testing.T) { tests := []struct { name string - treeAlgosByDepth []DequeueAlgorithm + treeAlgosByDepth []QueuingAlgorithm childPathsToEnqueue []QueuePath expectErr bool }{ { name: "enqueue tenant-querier node to round-robin node", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &roundRobinState{}}, childPathsToEnqueue: []QueuePath{{"child-1"}}, }, { name: "enqueue tenant-querier node to round-robin node", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, childPathsToEnqueue: []QueuePath{{"child-1"}, {"child-2"}}, }, { name: "enqueue round-robin node to tenant-querier node", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ newTenantQuerierAssignments(), &roundRobinState{}, }, @@ -85,7 +85,7 @@ func Test_EnqueueBackByPath(t *testing.T) { }, { name: "enqueue tenant-querier node to tenant-querier node", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ newTenantQuerierAssignments(), newTenantQuerierAssignments(), }, @@ -93,7 +93,7 @@ func Test_EnqueueBackByPath(t *testing.T) { }, { name: "enqueue grandchildren to a tree with max-depth of 2", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ newTenantQuerierAssignments(), &roundRobinState{}, newTenantQuerierAssignments(), @@ -102,7 +102,7 @@ func Test_EnqueueBackByPath(t *testing.T) { }, { name: "enqueue beyond max-depth", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}}, + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}}, childPathsToEnqueue: []QueuePath{{"child"}, {"child, grandchild"}}, expectErr: true, }, @@ -128,13 +128,13 @@ func Test_EnqueueFrontByPath(t *testing.T) { someQuerier := QuerierID("placeholder") tests := []struct { name string - treeAlgosByDepth []DequeueAlgorithm + treeAlgosByDepth []QueuingAlgorithm enqueueObjs []enqueueObj expected []any }{ { name: "enqueue to front of round-robin node", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}}, + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}}, enqueueObjs: []enqueueObj{ {"query-1", QueuePath{}}, {"query-2", QueuePath{}}, @@ -142,8 +142,8 @@ func Test_EnqueueFrontByPath(t *testing.T) { expected: []any{"query-2", "query-1"}, }, { - name: "enqueue to front of shuffle-shard node", - treeAlgosByDepth: []DequeueAlgorithm{&tenantQuerierAssignments{ + name: "enqueue to front of tenant-querier node", + treeAlgosByDepth: []QueuingAlgorithm{&tenantQuerierAssignments{ currentQuerier: &someQuerier, }}, enqueueObjs: []enqueueObj{ @@ -176,7 +176,7 @@ func Test_EnqueueFrontByPath(t *testing.T) { func Test_Dequeue_RootNode(t *testing.T) { tests := []struct { name string - rootAlgo DequeueAlgorithm + rootAlgo QueuingAlgorithm enqueueToRoot []any }{ { @@ -184,7 +184,7 @@ func Test_Dequeue_RootNode(t *testing.T) { rootAlgo: &roundRobinState{}, }, { - name: "dequeue from empty shuffle-shard root node", + name: "dequeue from empty tenant-querier root node", rootAlgo: newTenantQuerierAssignments(), }, { @@ -193,7 +193,7 @@ func Test_Dequeue_RootNode(t *testing.T) { enqueueToRoot: []any{"something-in-root"}, }, { - name: "dequeue from non-empty shuffle-shard root node", + name: "dequeue from non-empty tenant-querier root node", rootAlgo: newTenantQuerierAssignments(), enqueueToRoot: []any{"something-else-in-root"}, }, @@ -325,12 +325,12 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { tests := []struct { name string - treeAlgosByDepth []DequeueAlgorithm + treeAlgosByDepth []QueuingAlgorithm operationOrder []op }{ { name: "round-robin node should dequeue from first child one more time after new node added", - treeAlgosByDepth: []DequeueAlgorithm{&roundRobinState{}, &roundRobinState{}}, + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &roundRobinState{}}, operationOrder: []op{ {enqueue, QueuePath{"child-1"}, "obj-1"}, {enqueue, QueuePath{"child-1"}, "obj-2"}, @@ -342,8 +342,8 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { }, }, { - name: "should dequeue from new (next-in-queue) shuffle-shard child immediately after it is added", - treeAlgosByDepth: []DequeueAlgorithm{ + name: "should dequeue from new (next-in-queue) tenant-querier child immediately after it is added", + treeAlgosByDepth: []QueuingAlgorithm{ &tenantQuerierAssignments{ tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, tenantNodes: map[string][]*Node{}, @@ -390,7 +390,7 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { tests := []struct { name string - treeAlgosByDepth []DequeueAlgorithm + treeAlgosByDepth []QueuingAlgorithm state *tenantQuerierAssignments currQuerier []QuerierID enqueueObjs []enqueueObj @@ -399,7 +399,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { }{ { name: "happy path - tenant found in tenant-querier map under first child", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ &roundRobinState{}, newTenantQuerierAssignments(), &roundRobinState{}, @@ -416,7 +416,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { }, { name: "tenant exists, but not for querier", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ &roundRobinState{}, newTenantQuerierAssignments(), &roundRobinState{}, @@ -433,7 +433,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { }, { name: "1 of 3 tenants exist for querier", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ &roundRobinState{}, newTenantQuerierAssignments(), &roundRobinState{}, @@ -451,7 +451,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { }, { name: "tenant exists for querier on next parent node", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ &roundRobinState{}, newTenantQuerierAssignments(), &roundRobinState{}, @@ -470,7 +470,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { }, { name: "2 of 3 tenants exist for querier", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ &roundRobinState{}, newTenantQuerierAssignments(), &roundRobinState{}, @@ -488,8 +488,8 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { expected: []any{"query-1", "query-3", nil}, }, { - name: "root node is shuffle-shard node", - treeAlgosByDepth: []DequeueAlgorithm{ + name: "root node is tenant-querier node", + treeAlgosByDepth: []QueuingAlgorithm{ &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, &roundRobinState{}, @@ -508,7 +508,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { }, { name: "dequeuing for one querier returns nil, but does return for a different querier", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, &roundRobinState{}, @@ -525,7 +525,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { }, { name: "no querier set in state", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, @@ -543,7 +543,7 @@ func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { { // This also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) name: "dequeue from a tenant with a nil tenant-querier map", - treeAlgosByDepth: []DequeueAlgorithm{ + treeAlgosByDepth: []QueuingAlgorithm{ &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, &roundRobinState{}, &roundRobinState{}, @@ -828,7 +828,7 @@ func Test_EnqueueDuringDequeueRespectsRoundRobin(t *testing.T) { func Test_NodeCannotDeleteItself(t *testing.T) { tests := []struct { name string - nodeType DequeueAlgorithm + nodeType QueuingAlgorithm }{ {"round robin", &roundRobinState{}}, {"tenant querier assignment", &tenantQuerierAssignments{}}, diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go index 982e45415d4..a9a7a4419f3 100644 --- a/pkg/scheduler/queue/tree_queueing_algorithms.go +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -2,16 +2,30 @@ package queue -// DequeueAlgorithm represents the set of operations specific to different approaches to dequeuing. It is applied -// at the layer-level -- every Node at the same depth in a TreeQueue shares the same DequeueAlgorithm, including state -// that may be stored in a struct that implements DequeueAlgorithm. -type DequeueAlgorithm interface { +// QueuingAlgorithm represents the set of operations specific to different approaches to queuing/dequeuing. It is +// applied at the layer-level -- every Node at the same depth in a TreeQueue shares the same QueuingAlgorithm, +// including any state in structs that implement QueuingAlgorithm. +type QueuingAlgorithm interface { + // addChildNode creates a child Node of parent. QueuePaths passed to enqueue functions are allowed to contain + // node names for nodes which do not yet exist; in those cases, we create the missing nodes. Implementers are + // responsible for adding the newly-created node to whatever state(s) keep track of nodes in the subtree, as well + // as updating any positional pointers that have been shifted by the addition of a new node. addChildNode(parent, child *Node) + + // dequeueSelectNode, given a node, returns the node which an element will be dequeued from next, as well + // as a bool representing whether the passed node and all its children have been checked for dequeue-ability. + // The returned node may be the same as the node passed in; this triggers the base case for the recursive + // Node.dequeue, and results in dequeuing from the node's local queue. dequeueSelectNode does *not* + // update any Node fields. dequeueSelectNode(node *Node) (*Node, bool) + + // dequeueUpdateState is called after we have finished dequeuing from a node. When a child is left empty after the + // dequeue operation, dequeueUpdateState performs cleanup by deleting that child from the node and all other + // necessary state, and updates the relevant position to point to the next node to dequeue from. dequeueUpdateState(node *Node, dequeuedFrom *Node) } -// roundRobinState is the simplest type of DequeueAlgorithm; nodes which use this DequeueAlgorithm and are at +// roundRobinState is the simplest type of QueuingAlgorithm; nodes which use this QueuingAlgorithm and are at // the same depth in a TreeQueue do not share any state. When children are added to these nodes, they are placed at // the "end" of the order from the perspective of the node's current queuePosition (e.g., if queuePosition is 3, // a new child will be placed at index 2). Children are dequeued from using a simple round-robin ordering; @@ -36,10 +50,10 @@ func (rrs *roundRobinState) addChildNode(parent, child *Node) { } } -// dequeueGetNode returns the node at the node's queuePosition. queuePosition represents the position of +// dequeueSelectNode returns the node at the node's queuePosition. queuePosition represents the position of // the next node to dequeue from, and is incremented in dequeueUpdateState. func (rrs *roundRobinState) dequeueSelectNode(node *Node) (*Node, bool) { - checkedAllNodes := node.childrenChecked == len(node.queueOrder)+1 + checkedAllNodes := node.childrenChecked == len(node.queueOrder)+1 // must check local queue as well if node.queuePosition == localQueueIndex { return node, checkedAllNodes } @@ -54,8 +68,6 @@ func (rrs *roundRobinState) dequeueSelectNode(node *Node) (*Node, bool) { // dequeueUpdateState does the following: // - deletes the dequeued-from child node if it is empty after the dequeue operation // - increments queuePosition if no child was deleted -// - updates childrenChecked to reflect the number of children checked so that the dequeueGetNode operation -// will stop once it has checked all possible nodes. func (rrs *roundRobinState) dequeueUpdateState(node *Node, dequeuedFrom *Node) { // if the child node is nil, we haven't done anything to the tree; return early if dequeuedFrom == nil { From 6d7b9a6a3eecf4b71afa85aa5047235d88b4c4fc Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Mon, 24 Jun 2024 16:38:53 -0700 Subject: [PATCH 21/26] Re-include original tree queue with a config switch --- pkg/scheduler/queue/integrated_tree_queue.go | 289 +++++ pkg/scheduler/queue/queue.go | 6 +- pkg/scheduler/queue/queue_test.go | 881 +++++++------- .../queue/tenant_querier_assignment.go | 44 +- pkg/scheduler/queue/tenant_queues.go | 91 +- pkg/scheduler/queue/tenant_queues_test.go | 1034 +++++++++-------- pkg/scheduler/queue/tree_queue.go | 414 +++---- pkg/scheduler/queue/tree_queue_test.go | 8 +- .../queue/tree_queueing_algorithms.go | 4 +- pkg/scheduler/scheduler.go | 3 + 10 files changed, 1646 insertions(+), 1128 deletions(-) create mode 100644 pkg/scheduler/queue/integrated_tree_queue.go diff --git a/pkg/scheduler/queue/integrated_tree_queue.go b/pkg/scheduler/queue/integrated_tree_queue.go new file mode 100644 index 00000000000..67b5a5a83ff --- /dev/null +++ b/pkg/scheduler/queue/integrated_tree_queue.go @@ -0,0 +1,289 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package queue + +import ( + "container/list" + "fmt" +) + +type QueuePath []string //nolint:revive // disallows types beginning with package name +type QueueIndex int //nolint:revive // disallows types beginning with package name + +const localQueueIndex = -1 + +type Tree interface { + EnqueueFrontByPath(QueuePath, any) error + EnqueueBackByPath(QueuePath, any) error + Dequeue() (QueuePath, any) + ItemCount() int + IsEmpty() bool +} + +// IntegratedTreeQueue holds metadata and a pointer to the root node of a hierarchical queue implementation. +// The root Node maintains a localQueue and an arbitrary number of child nodes (which themselves +// may have local queues and children). Each Node in IntegratedTreeQueue uses a QueuingAlgorithm (determined by +// node depth) to determine dequeue order of that Node's subtree. +// +// Each queuing dimension is modeled as a node in the tree, internally reachable through a QueuePath. +// +// The QueuePath is an ordered array of strings describing the path from the tree root to a Node. +// In addition to child Nodes, each Node contains a local queue (FIFO) of items. +// +// When dequeuing from a given node, a Node will use its QueuingAlgorithm to choose either itself +// or a child node to dequeue from recursively (i.e., a child Node will use its own QueuingAlgorithm +// to determine how to proceed). IntegratedTreeQueue will not dequeue from two different Nodes at the same depth +// consecutively, unless the previously-checked Node was empty down to the leaf node. +type IntegratedTreeQueue struct { + rootNode *Node + algosByDepth []QueuingAlgorithm +} + +func NewTree(queuingAlgorithms ...QueuingAlgorithm) (*IntegratedTreeQueue, error) { + if len(queuingAlgorithms) == 0 { + return nil, fmt.Errorf("cannot create a tree without defined QueuingAlgorithm") + } + root, err := newNode("root", 0, queuingAlgorithms[0]) + if err != nil { + return nil, err + } + root.depth = 0 + return &IntegratedTreeQueue{ + rootNode: root, + algosByDepth: queuingAlgorithms, + }, nil +} + +func (t *IntegratedTreeQueue) ItemCount() int { + return t.rootNode.ItemCount() +} + +func (t *IntegratedTreeQueue) IsEmpty() bool { + return t.rootNode.IsEmpty() +} + +// Dequeue removes and returns an item from the front of the next appropriate Node in the IntegratedTreeQueue, as +// well as the path to the Node which that item was dequeued from. +// +// Either the root/self node or a child node is chosen according to the Node's QueuingAlgorithm. If +// the root node is chosen, an item will be dequeued from the front of its localQueue. If a child +// node is chosen, it is recursively dequeued from until a node selects its localQueue. +// +// Nodes that empty down to the leaf after being dequeued from (or which are found to be empty leaf +// nodes during the dequeue operation) are deleted as the recursion returns up the stack. This +// maintains structural guarantees relied upon to make IsEmpty() non-recursive. +func (t *IntegratedTreeQueue) Dequeue() (QueuePath, any) { + path, v := t.rootNode.dequeue() + // The returned node dequeue path includes the root node; exclude + // this so that the return path can be used if needed to enqueue. + return path[1:], v +} + +// EnqueueBackByPath enqueues an item in the back of the local queue of the node +// located at a given path through the tree; nodes for the path are created as needed. +// +// path is relative to the root node; providing a QueuePath beginning with "root" +// will create a child node of the root node which is also named "root." +func (t *IntegratedTreeQueue) EnqueueBackByPath(path QueuePath, v any) error { + return t.rootNode.enqueueBackByPath(t, path, v) +} + +// EnqueueFrontByPath enqueues an item in the front of the local queue of the Node +// located at a given path through the IntegratedTreeQueue; nodes for the path are created as needed. +// +// Enqueueing to the front is intended only for items which were first enqueued to the back +// and then dequeued after reaching the front. +// +// Re-enqueueing to the front is only intended for use in cases where a queue consumer +// fails to complete operations on the dequeued item, but failure is not yet final, and the +// operations should be retried by a subsequent queue consumer. A concrete example is when +// a queue consumer fails or disconnects for unrelated reasons while we are in the process +// of dequeuing a request for it. +// +// path must be relative to the root node; providing a QueuePath beginning with "root" +// will create a child node of root which is also named "root." +func (t *IntegratedTreeQueue) EnqueueFrontByPath(path QueuePath, v any) error { + return t.rootNode.enqueueFrontByPath(t, path, v) +} + +func (t *IntegratedTreeQueue) GetNode(path QueuePath) *Node { + return t.rootNode.getNode(path) +} + +// Node maintains node-specific information used to enqueue and dequeue to itself, such as a local +// queue, node depth, references to its children, and position in queue. +// Note that the tenantQuerierAssignments QueuingAlgorithm largely disregards Node's queueOrder and +// queuePosition, managing analogous state instead, because shuffle-sharding + fairness requirements +// necessitate input from the querier. +type Node struct { + name string + localQueue *list.List + queuePosition int // next index in queueOrder to dequeue from + queueOrder []string // order for dequeuing from self/children + queueMap map[string]*Node + depth int + queuingAlgorithm QueuingAlgorithm + childrenChecked int +} + +func newNode(name string, depth int, da QueuingAlgorithm) (*Node, error) { + if da == nil { + return nil, fmt.Errorf("cannot create a node without a defined QueuingAlgorithm") + } + if tqa, ok := da.(*tenantQuerierAssignments); ok { + tqa.tenantOrderIndex = localQueueIndex - 1 // start from -2 so that we first check local queue + if tqa.tenantNodes == nil { + tqa.tenantNodes = map[string][]*Node{} + } + } + return &Node{ + name: name, + localQueue: list.New(), + queuePosition: localQueueIndex, + queueOrder: make([]string, 0), + queueMap: make(map[string]*Node, 1), + depth: depth, + queuingAlgorithm: da, + }, nil +} + +func (n *Node) IsEmpty() bool { + // avoid recursion to make this a cheap operation + // + // Because we dereference empty child nodes during dequeuing, + // we assume that emptiness means there are no child nodes + // and nothing in this tree node's local queue. + // + // In reality a package member could attach empty child queues with getOrAddNode + // in order to get a functionally-empty tree that would report false for IsEmpty. + // We assume this does not occur or is not relevant during normal operation. + return n.localQueue.Len() == 0 && len(n.queueMap) == 0 +} + +// ItemCount counts the queue items in the Node and in all its children, recursively. +func (n *Node) ItemCount() int { + items := n.localQueue.Len() + for _, child := range n.queueMap { + items += child.ItemCount() + } + return items +} + +func (n *Node) Name() string { + return n.name +} + +func (n *Node) getLocalQueue() *list.List { + return n.localQueue +} + +func (n *Node) enqueueFrontByPath(tree *IntegratedTreeQueue, pathFromNode QueuePath, v any) error { + childNode, err := n.getOrAddNode(pathFromNode, tree) + if err != nil { + return err + } + childNode.localQueue.PushFront(v) + return nil +} + +func (n *Node) enqueueBackByPath(tree *IntegratedTreeQueue, pathFromNode QueuePath, v any) error { + childNode, err := n.getOrAddNode(pathFromNode, tree) + if err != nil { + return err + } + childNode.localQueue.PushBack(v) + return nil +} + +func (n *Node) dequeue() (QueuePath, any) { + var v any + var childPath QueuePath + + path := QueuePath{n.name} + + if n.IsEmpty() { + return path, nil + } + + var checkedAllNodes bool + var dequeueNode *Node + // continue until we've found a value or checked all nodes that need checking + for v == nil && !checkedAllNodes { + dequeueNode, checkedAllNodes = n.queuingAlgorithm.dequeueSelectNode(n) + switch dequeueNode { + // dequeuing from local queue + case n: + if n.localQueue.Len() > 0 { + // dequeueNode is self, local queue non-empty + if elt := n.localQueue.Front(); elt != nil { + n.localQueue.Remove(elt) + v = elt.Value + } + } + // no dequeue-able child found; break out of the loop, + // since we won't find anything to dequeue if we don't + // have a node to dequeue from now + case nil: + checkedAllNodes = true + // dequeue from a child + default: + childPath, v = dequeueNode.dequeue() + } + + if v == nil { + n.childrenChecked++ + } + + n.queuingAlgorithm.dequeueUpdateState(n, dequeueNode) + } + // reset childrenChecked to 0 before completing this dequeue + n.childrenChecked = 0 + return append(path, childPath...), v +} + +func (n *Node) getNode(pathFromNode QueuePath) *Node { + if len(pathFromNode) == 0 { + return n + } + + if n.queueMap == nil { + return nil + } + + if childQueue, ok := n.queueMap[pathFromNode[0]]; ok { + return childQueue.getNode(pathFromNode[1:]) + } + + // no child node matches next path segment + return nil +} + +// getOrAddNode recursively gets or adds tree queue nodes based on given relative child path. It +// checks whether the first node in pathFromNode exists in the Node's children; if no node exists, +// one is created and added to the Node's queueOrder, according to the Node's QueuingAlgorithm. +// +// pathFromNode must be relative to the receiver node; providing a QueuePath beginning with +// the receiver/parent node name will create a child node of the same name as the parent. +func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *IntegratedTreeQueue) (*Node, error) { + if len(pathFromNode) == 0 { + return n, nil + } + + var childNode *Node + var ok bool + var err error + if childNode, ok = n.queueMap[pathFromNode[0]]; !ok { + // child does not exist, create it + if n.depth+1 >= len(tree.algosByDepth) { + return nil, fmt.Errorf("cannot add a node beyond max tree depth: %v", len(tree.algosByDepth)) + } + childNode, err = newNode(pathFromNode[0], n.depth+1, tree.algosByDepth[n.depth+1]) + if err != nil { + return nil, err + } + // add the newly created child to the node + n.queuingAlgorithm.addChildNode(n, childNode) + + } + return childNode.getOrAddNode(pathFromNode[1:], tree) +} diff --git a/pkg/scheduler/queue/queue.go b/pkg/scheduler/queue/queue.go index 8578a21656b..21f04bd182c 100644 --- a/pkg/scheduler/queue/queue.go +++ b/pkg/scheduler/queue/queue.go @@ -121,6 +121,7 @@ type RequestQueue struct { // settings maxOutstandingPerTenant int additionalQueueDimensionsEnabled bool + useIntegratedQueue bool forgetDelay time.Duration // metrics for reporting @@ -174,6 +175,7 @@ func NewRequestQueue( log log.Logger, maxOutstandingPerTenant int, additionalQueueDimensionsEnabled bool, + useIntegratedQueue bool, forgetDelay time.Duration, queueLength *prometheus.GaugeVec, discardedRequests *prometheus.CounterVec, @@ -210,7 +212,7 @@ func NewRequestQueue( waitingQuerierConnsToDispatch: list.New(), QueryComponentUtilization: queryComponentCapacity, - queueBroker: newQueueBroker(maxOutstandingPerTenant, additionalQueueDimensionsEnabled, false, forgetDelay), + queueBroker: newQueueBroker(maxOutstandingPerTenant, additionalQueueDimensionsEnabled, useIntegratedQueue, forgetDelay), } q.Service = services.NewBasicService(q.starting, q.running, q.stop).WithName("request queue") @@ -371,7 +373,7 @@ func (q *RequestQueue) trySendNextRequestForQuerier(waitingConn *waitingQuerierC exceedsThreshold, queryComponent := q.QueryComponentUtilization.ExceedsThresholdForComponentName( queryComponentName, int(q.connectedQuerierWorkers.Load()), - q.queueBroker.tree.rootNode.ItemCount(), + q.queueBroker.tree.ItemCount(), q.waitingQuerierConnsToDispatch.Len(), ) diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index c9c3a070f43..219fe0fac97 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -31,6 +31,19 @@ import ( // TODO (casie): Write tests for prioritizeQueryComponents is true +func buildTreeTestsStruct() []struct { + name string + useIntegratedTreeQueue bool +} { + return []struct { + name string + useIntegratedTreeQueue bool + }{ + {"legacy tree queue", false}, + {"integrated tree queue", true}, + } +} + func TestMain(m *testing.M) { util_test.VerifyNoLeakTestMain(m) } @@ -95,140 +108,151 @@ func makeSchedulerRequest(tenantID string, additionalQueueDimensions []string) * // The queue broker then round-robins between the split queues, which has the effect of alternating between // dequeuing the slow queries and normal queries rather than blocking normal queries behind slow queries. func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { - promRegistry := prometheus.NewPedanticRegistry() - - maxQueriersPerTenant := 0 // disable shuffle sharding - forgetQuerierDelay := time.Duration(0) - maxOutstandingRequestsPerTenant := 1000 - - totalRequests := 100 - numTenants := 1 - numProducers := 10 - numConsumers := 1 - - normalQueueDimension := "normal-request" - slowConsumerLatency := 20 * time.Millisecond - slowConsumerQueueDimension := "slow-request" - normalQueueDimensionFunc := func() []string { return []string{normalQueueDimension} } - slowQueueDimensionFunc := func() []string { return []string{slowConsumerQueueDimension} } - - additionalQueueDimensionsEnabledCases := []bool{false, true} - queueDurationTotals := map[bool]map[string]float64{ - false: {normalQueueDimension: 0.0, slowConsumerQueueDimension: 0.0}, - true: {normalQueueDimension: 0.0, slowConsumerQueueDimension: 0.0}, - } - - for _, additionalQueueDimensionsEnabled := range additionalQueueDimensionsEnabledCases { - - // Scheduler code uses a histogram for queue duration, but a counter is a more direct metric - // for this test, as we are concerned with the total or average wait time for all queue items. - // Prometheus histograms also lack support for test assertions via prometheus/testutil. - queueDuration := promauto.With(promRegistry).NewCounterVec(prometheus.CounterOpts{ - Name: "test_query_scheduler_queue_duration_total_seconds", - Help: "[test] total time spent by items in queue before getting picked up by a consumer", - }, []string{"additional_queue_dimensions"}) - - queue, err := NewRequestQueue( - log.NewNopLogger(), - maxOutstandingRequestsPerTenant, - additionalQueueDimensionsEnabled, - forgetQuerierDelay, - promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), - promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), - promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), - promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), - ) - require.NoError(t, err) - - ctx := context.Background() - require.NoError(t, queue.starting(ctx)) - t.Cleanup(func() { - require.NoError(t, queue.stop(nil)) - }) - - // fill queue first with the slow queries, then the normal queries - for _, queueDimensionFunc := range []func() []string{slowQueueDimensionFunc, normalQueueDimensionFunc} { - startProducersChan := make(chan struct{}) - producersErrGroup, _ := errgroup.WithContext(ctx) - - runProducer := runQueueProducerIters( - queue, maxQueriersPerTenant, totalRequests/2, numProducers, numTenants, startProducersChan, queueDimensionFunc, - ) - for producerIdx := 0; producerIdx < numProducers; producerIdx++ { - producerIdx := producerIdx - producersErrGroup.Go(func() error { - return runProducer(producerIdx) - }) + treeTypes := buildTreeTestsStruct() + + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + promRegistry := prometheus.NewPedanticRegistry() + + maxQueriersPerTenant := 0 // disable shuffle sharding + forgetQuerierDelay := time.Duration(0) + maxOutstandingRequestsPerTenant := 1000 + + totalRequests := 100 + numTenants := 1 + numProducers := 10 + numConsumers := 1 + + normalQueueDimension := "normal-request" + slowConsumerLatency := 20 * time.Millisecond + slowConsumerQueueDimension := "slow-request" + normalQueueDimensionFunc := func() []string { return []string{normalQueueDimension} } + slowQueueDimensionFunc := func() []string { return []string{slowConsumerQueueDimension} } + + additionalQueueDimensionsEnabledCases := []bool{false, true} + queueDurationTotals := map[bool]map[string]float64{ + false: {normalQueueDimension: 0.0, slowConsumerQueueDimension: 0.0}, + true: {normalQueueDimension: 0.0, slowConsumerQueueDimension: 0.0}, } - close(startProducersChan) - err := producersErrGroup.Wait() - require.NoError(t, err) - } - // emulate delay when consuming the slow queries - consumeFunc := func(request Request) error { - schedulerRequest := request.(*SchedulerRequest) - if schedulerRequest.AdditionalQueueDimensions[0] == slowConsumerQueueDimension { - time.Sleep(slowConsumerLatency) - } - - queueTime := time.Since(schedulerRequest.EnqueueTime) - additionalQueueDimensionLabels := strings.Join(schedulerRequest.AdditionalQueueDimensions, ":") - queueDuration.With(prometheus.Labels{"additional_queue_dimensions": additionalQueueDimensionLabels}).Add(queueTime.Seconds()) - return nil - } - - // consume queries - queueConsumerErrGroup, ctx := errgroup.WithContext(ctx) - startConsumersChan := make(chan struct{}) - runConsumer := runQueueConsumerIters(ctx, queue, totalRequests, numConsumers, startConsumersChan, consumeFunc) - - for consumerIdx := 0; consumerIdx < numConsumers; consumerIdx++ { - consumerIdx := consumerIdx - queueConsumerErrGroup.Go(func() error { - return runConsumer(consumerIdx) - }) - } + for _, additionalQueueDimensionsEnabled := range additionalQueueDimensionsEnabledCases { + + // Scheduler code uses a histogram for queue duration, but a counter is a more direct metric + // for this test, as we are concerned with the total or average wait time for all queue items. + // Prometheus histograms also lack support for test assertions via prometheus/testutil. + queueDuration := promauto.With(promRegistry).NewCounterVec(prometheus.CounterOpts{ + Name: "test_query_scheduler_queue_duration_total_seconds", + Help: "[test] total time spent by items in queue before getting picked up by a consumer", + }, []string{"additional_queue_dimensions"}) + + queue, err := NewRequestQueue( + log.NewNopLogger(), + maxOutstandingRequestsPerTenant, + additionalQueueDimensionsEnabled, + tt.useIntegratedTreeQueue, + forgetQuerierDelay, + promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), + promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), + promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), + promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), + ) + require.NoError(t, err) + + ctx := context.Background() + require.NoError(t, queue.starting(ctx)) + t.Cleanup(func() { + require.NoError(t, queue.stop(nil)) + }) - close(startConsumersChan) - err = queueConsumerErrGroup.Wait() - require.NoError(t, err) + // fill queue first with the slow queries, then the normal queries + for _, queueDimensionFunc := range []func() []string{slowQueueDimensionFunc, normalQueueDimensionFunc} { + startProducersChan := make(chan struct{}) + producersErrGroup, _ := errgroup.WithContext(ctx) + + runProducer := runQueueProducerIters( + queue, maxQueriersPerTenant, totalRequests/2, numProducers, numTenants, startProducersChan, queueDimensionFunc, + ) + for producerIdx := 0; producerIdx < numProducers; producerIdx++ { + producerIdx := producerIdx + producersErrGroup.Go(func() error { + return runProducer(producerIdx) + }) + } + close(startProducersChan) + err := producersErrGroup.Wait() + require.NoError(t, err) + } + + // emulate delay when consuming the slow queries + consumeFunc := func(request Request) error { + schedulerRequest := request.(*SchedulerRequest) + if schedulerRequest.AdditionalQueueDimensions[0] == slowConsumerQueueDimension { + time.Sleep(slowConsumerLatency) + } - // record total queue duration by queue dimensions and whether the queue splitting was enabled - for _, queueDimension := range []string{normalQueueDimension, slowConsumerQueueDimension} { - queueDurationTotals[additionalQueueDimensionsEnabled][queueDimension] = promtest.ToFloat64( - queueDuration.With(prometheus.Labels{"additional_queue_dimensions": queueDimension}), - ) - } + queueTime := time.Since(schedulerRequest.EnqueueTime) + additionalQueueDimensionLabels := strings.Join(schedulerRequest.AdditionalQueueDimensions, ":") + queueDuration.With(prometheus.Labels{"additional_queue_dimensions": additionalQueueDimensionLabels}).Add(queueTime.Seconds()) + return nil + } + + // consume queries + queueConsumerErrGroup, ctx := errgroup.WithContext(ctx) + startConsumersChan := make(chan struct{}) + runConsumer := runQueueConsumerIters(ctx, queue, totalRequests, numConsumers, startConsumersChan, consumeFunc) + + for consumerIdx := 0; consumerIdx < numConsumers; consumerIdx++ { + consumerIdx := consumerIdx + queueConsumerErrGroup.Go(func() error { + return runConsumer(consumerIdx) + }) + } + + close(startConsumersChan) + err = queueConsumerErrGroup.Wait() + require.NoError(t, err) + + // record total queue duration by queue dimensions and whether the queue splitting was enabled + for _, queueDimension := range []string{normalQueueDimension, slowConsumerQueueDimension} { + queueDurationTotals[additionalQueueDimensionsEnabled][queueDimension] = promtest.ToFloat64( + queueDuration.With(prometheus.Labels{"additional_queue_dimensions": queueDimension}), + ) + } + + promRegistry.Unregister(queueDuration) + } - promRegistry.Unregister(queueDuration) + // total or average time in queue for a normal queue item should be roughly cut in half + // when queue splitting is enabled, as the average normal queue item waits behind + // half of the slow queue items, instead of waiting behind all the slow queue items. + expected := queueDurationTotals[false][normalQueueDimension] / 2 + actual := queueDurationTotals[true][normalQueueDimension] + // some variance allowed due to actual time processing needed beyond the slow consumer delay; + // variance is also a function of the number of consumers and the consumer delay chosen. + // variance can be tighter if the test runs longer but there is a tradeoff for testing and CI speed + delta := expected * 0.10 + require.InDelta(t, expected, actual, delta) + }) } - // total or average time in queue for a normal queue item should be roughly cut in half - // when queue splitting is enabled, as the average normal queue item waits behind - // half of the slow queue items, instead of waiting behind all the slow queue items. - expected := queueDurationTotals[false][normalQueueDimension] / 2 - actual := queueDurationTotals[true][normalQueueDimension] - // some variance allowed due to actual time processing needed beyond the slow consumer delay; - // variance is also a function of the number of consumers and the consumer delay chosen. - // variance can be tighter if the test runs longer but there is a tradeoff for testing and CI speed - delta := expected * 0.10 - require.InDelta(t, expected, actual, delta) - } func BenchmarkConcurrentQueueOperations(b *testing.B) { - maxQueriersPerTenant := 0 // disable shuffle sharding - forgetQuerierDelay := time.Duration(0) - maxOutstandingRequestsPerTenant := 100 + treeTypes := buildTreeTestsStruct() + + for _, t := range treeTypes { + b.Run(t.name, func(b *testing.B) { + maxQueriersPerTenant := 0 // disable shuffle sharding + forgetQuerierDelay := time.Duration(0) + maxOutstandingRequestsPerTenant := 100 - for _, numTenants := range []int{1, 10, 1000} { - b.Run(fmt.Sprintf("%v tenants", numTenants), func(b *testing.B) { + for _, numTenants := range []int{1, 10, 1000} { + b.Run(fmt.Sprintf("%v tenants", numTenants), func(b *testing.B) { - // Query-frontends run 5 parallel streams per scheduler by default, - // and we typically see 2-5 frontends running at any one time. - for _, numProducers := range []int{10, 25} { - b.Run(fmt.Sprintf("%v concurrent producers", numProducers), func(b *testing.B) { + // Query-frontends run 5 parallel streams per scheduler by default, + // and we typically see 2-5 frontends running at any one time. + for _, numProducers := range []int{10, 25} { + b.Run(fmt.Sprintf("%v concurrent producers", numProducers), func(b *testing.B) { // Queriers run with parallelism of 16 when query sharding is enabled. for _, numConsumers := range []int{16, 160, 1600} { @@ -237,6 +261,7 @@ func BenchmarkConcurrentQueueOperations(b *testing.B) { log.NewNopLogger(), maxOutstandingRequestsPerTenant, true, + t.useIntegratedTreeQueue, forgetQuerierDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -245,44 +270,47 @@ func BenchmarkConcurrentQueueOperations(b *testing.B) { ) require.NoError(b, err) - startSignalChan := make(chan struct{}) - queueActorsErrGroup, ctx := errgroup.WithContext(context.Background()) - - require.NoError(b, queue.starting(ctx)) - b.Cleanup(func() { - require.NoError(b, queue.stop(nil)) - }) - - runProducer := runQueueProducerIters( - queue, maxQueriersPerTenant, b.N, numProducers, numTenants, startSignalChan, nil, - ) - - for producerIdx := 0; producerIdx < numProducers; producerIdx++ { - producerIdx := producerIdx - queueActorsErrGroup.Go(func() error { - return runProducer(producerIdx) - }) - } - - runConsumer := runQueueConsumerIters(ctx, queue, b.N, numConsumers, startSignalChan, nil) - - for consumerIdx := 0; consumerIdx < numConsumers; consumerIdx++ { - consumerIdx := consumerIdx - queueActorsErrGroup.Go(func() error { - return runConsumer(consumerIdx) + startSignalChan := make(chan struct{}) + queueActorsErrGroup, ctx := errgroup.WithContext(context.Background()) + + require.NoError(b, queue.starting(ctx)) + b.Cleanup(func() { + require.NoError(b, queue.stop(nil)) + }) + + runProducer := runQueueProducerIters( + queue, maxQueriersPerTenant, b.N, numProducers, numTenants, startSignalChan, nil, + ) + + for producerIdx := 0; producerIdx < numProducers; producerIdx++ { + producerIdx := producerIdx + queueActorsErrGroup.Go(func() error { + return runProducer(producerIdx) + }) + } + + runConsumer := runQueueConsumerIters(ctx, queue, b.N, numConsumers, startSignalChan, nil) + + for consumerIdx := 0; consumerIdx < numConsumers; consumerIdx++ { + consumerIdx := consumerIdx + queueActorsErrGroup.Go(func() error { + return runConsumer(consumerIdx) + }) + } + + b.ResetTimer() + close(startSignalChan) + err = queueActorsErrGroup.Wait() + if err != nil { + require.NoError(b, err) + } }) } - - b.ResetTimer() - close(startSignalChan) - err = queueActorsErrGroup.Wait() - if err != nil { - require.NoError(b, err) - } }) } }) } + }) } } @@ -406,282 +434,333 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldGetRequestAfterReshardingBe const forgetDelay = 3 * time.Second const testTimeout = 10 * time.Second - queue, err := NewRequestQueue( - log.NewNopLogger(), - 1, true, - forgetDelay, - promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), - promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), - promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), - promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), - ) - require.NoError(t, err) - - // Start the queue service. - ctx := context.Background() - require.NoError(t, services.StartAndAwaitRunning(ctx, queue)) - t.Cleanup(func() { - // if the test has failed and the queue does not get cleared, - // we must send a shutdown signal for the remaining connected querier - // or else StopAndAwaitTerminated will never complete. - queue.SubmitUnregisterQuerierConnection("querier-2") - require.NoError(t, services.StopAndAwaitTerminated(ctx, queue)) - }) - - // Two queriers connect. - queue.SubmitRegisterQuerierConnection("querier-1") - queue.SubmitRegisterQuerierConnection("querier-2") - - // Querier-2 waits for a new request. - querier2wg := sync.WaitGroup{} - querier2wg.Add(1) - go func() { - defer querier2wg.Done() - _, _, err := queue.WaitForRequestForQuerier(ctx, FirstTenant(), "querier-2") - require.NoError(t, err) - }() - - // Querier-1 crashes (no graceful shutdown notification). - queue.SubmitUnregisterQuerierConnection("querier-1") - - // Enqueue a request from an user which would be assigned to querier-1. - // NOTE: "user-1" shuffle shard always chooses the first querier ("querier-1" in this case) - // when there are only one or two queriers in the sorted list of connected queriers - req := &SchedulerRequest{ - Ctx: context.Background(), - Request: &httpgrpc.HTTPRequest{Method: "GET", Url: "/hello"}, - AdditionalQueueDimensions: randAdditionalQueueDimension(true), - } - require.NoError(t, queue.SubmitRequestToEnqueue("user-1", req, 1, nil)) - - startTime := time.Now() - done := make(chan struct{}) - go func() { - querier2wg.Wait() - close(done) - }() - - select { - case <-done: - waitTime := time.Since(startTime) - // We expect that querier-2 got the request only after forget delay is passed. - assert.GreaterOrEqual(t, waitTime.Milliseconds(), forgetDelay.Milliseconds()) - case <-time.After(testTimeout): - t.Fatal("timeout: querier-2 did not receive the request expected to be resharded to querier-2") + treeTypes := buildTreeTestsStruct() + + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + queue, err := NewRequestQueue( + log.NewNopLogger(), + 1, true, + tt.useIntegratedTreeQueue, + forgetDelay, + promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), + promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), + promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), + promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), + ) + require.NoError(t, err) + + // Start the queue service. + ctx := context.Background() + require.NoError(t, services.StartAndAwaitRunning(ctx, queue)) + t.Cleanup(func() { + // if the test has failed and the queue does not get cleared, + // we must send a shutdown signal for the remaining connected querier + // or else StopAndAwaitTerminated will never complete. + queue.SubmitUnregisterQuerierConnection("querier-2") + require.NoError(t, services.StopAndAwaitTerminated(ctx, queue)) + }) + + // Two queriers connect. + queue.SubmitRegisterQuerierConnection("querier-1") + queue.SubmitRegisterQuerierConnection("querier-2") + + // Querier-2 waits for a new request. + querier2wg := sync.WaitGroup{} + querier2wg.Add(1) + go func() { + defer querier2wg.Done() + _, _, err := queue.WaitForRequestForQuerier(ctx, FirstTenant(), "querier-2") + require.NoError(t, err) + }() + + // Querier-1 crashes (no graceful shutdown notification). + queue.SubmitUnregisterQuerierConnection("querier-1") + + // Enqueue a request from an user which would be assigned to querier-1. + // NOTE: "user-1" shuffle shard always chooses the first querier ("querier-1" in this case) + // when there are only one or two queriers in the sorted list of connected queriers + req := &SchedulerRequest{ + Ctx: context.Background(), + Request: &httpgrpc.HTTPRequest{Method: "GET", Url: "/hello"}, + AdditionalQueueDimensions: randAdditionalQueueDimension(true), + } + require.NoError(t, queue.SubmitRequestToEnqueue("user-1", req, 1, nil)) + + startTime := time.Now() + done := make(chan struct{}) + go func() { + querier2wg.Wait() + close(done) + }() + + select { + case <-done: + waitTime := time.Since(startTime) + // We expect that querier-2 got the request only after forget delay is passed. + assert.GreaterOrEqual(t, waitTime.Milliseconds(), forgetDelay.Milliseconds()) + case <-time.After(testTimeout): + t.Fatal("timeout: querier-2 did not receive the request expected to be resharded to querier-2") + } + + }) } } func TestRequestQueue_GetNextRequestForQuerier_ReshardNotifiedCorrectlyForMultipleQuerierForget(t *testing.T) { const forgetDelay = 3 * time.Second const testTimeout = 10 * time.Second + treeTypes := buildTreeTestsStruct() + + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + queue, err := NewRequestQueue( + log.NewNopLogger(), + 1, true, + tt.useIntegratedTreeQueue, + forgetDelay, + promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), + promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), + promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), + promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), + ) + require.NoError(t, err) - queue, err := NewRequestQueue( - log.NewNopLogger(), - 1, true, - forgetDelay, - promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), - promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), - promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), - promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), - ) - require.NoError(t, err) - - // Start the queue service. - ctx := context.Background() - require.NoError(t, services.StartAndAwaitRunning(ctx, queue)) - t.Cleanup(func() { - // if the test has failed and the queue does not get cleared, - // we must send a shutdown signal for the remaining connected querier - // or else StopAndAwaitTerminated will never complete. - queue.SubmitUnregisterQuerierConnection("querier-2") - require.NoError(t, services.StopAndAwaitTerminated(ctx, queue)) - }) - - // Three queriers connect. - // We will submit the enqueue request with maxQueriers: 2. - // - // Whenever forgetDisconnectedQueriers runs, all queriers which reached zero connections since the last - // run of forgetDisconnectedQueriers will all be removed in from the shuffle shard in the same run. - // - // In this case two queriers are forgotten in the same run, but only the first forgotten querier triggers a reshard. - // In the first reshard, the tenant goes from a shuffled subset of queriers to a state of - // "tenant can use all queriers", as connected queriers is now <= tenant.maxQueriers. - // The second forgotten querier won't trigger a reshard, as connected queriers is already <= tenant.maxQueriers. - // - // We are testing that the occurrence of a reshard is reported correctly - // when not all querier forget operations in a single run of forgetDisconnectedQueriers caused a reshard. - queue.SubmitRegisterQuerierConnection("querier-1") - queue.SubmitRegisterQuerierConnection("querier-2") - queue.SubmitRegisterQuerierConnection("querier-3") - - // querier-2 waits for a new request. - querier2wg := sync.WaitGroup{} - querier2wg.Add(1) - go func() { - defer querier2wg.Done() - _, _, err := queue.WaitForRequestForQuerier(ctx, FirstTenant(), "querier-2") - require.NoError(t, err) - }() - - // querier-1 and querier-3 crash (no graceful shutdown notification). - queue.SubmitUnregisterQuerierConnection("querier-1") - queue.SubmitUnregisterQuerierConnection("querier-3") - - // Enqueue a request from a tenant which would be assigned to querier-1. - // NOTE: "user-1" shuffle shard always chooses the first querier ("querier-1" in this case) - // when there are only one or two queriers in the sorted list of connected queriers - req := &SchedulerRequest{ - Ctx: context.Background(), - Request: &httpgrpc.HTTPRequest{Method: "GET", Url: "/hello"}, - AdditionalQueueDimensions: randAdditionalQueueDimension(true), - } - require.NoError(t, queue.SubmitRequestToEnqueue("user-1", req, 2, nil)) - - startTime := time.Now() - done := make(chan struct{}) - go func() { - querier2wg.Wait() - close(done) - }() - - select { - case <-done: - waitTime := time.Since(startTime) - // We expect that querier-2 got the request only after forget delay is passed. - assert.GreaterOrEqual(t, waitTime.Milliseconds(), forgetDelay.Milliseconds()) - case <-time.After(testTimeout): - t.Fatal("timeout: querier-2 did not receive the request expected to be resharded to querier-2") + // Start the queue service. + ctx := context.Background() + require.NoError(t, services.StartAndAwaitRunning(ctx, queue)) + t.Cleanup(func() { + // if the test has failed and the queue does not get cleared, + // we must send a shutdown signal for the remaining connected querier + // or else StopAndAwaitTerminated will never complete. + queue.SubmitUnregisterQuerierConnection("querier-2") + require.NoError(t, services.StopAndAwaitTerminated(ctx, queue)) + }) + + // Three queriers connect. + // We will submit the enqueue request with maxQueriers: 2. + // + // Whenever forgetDisconnectedQueriers runs, all queriers which reached zero connections since the last + // run of forgetDisconnectedQueriers will all be removed in from the shuffle shard in the same run. + // + // In this case two queriers are forgotten in the same run, but only the first forgotten querier triggers a reshard. + // In the first reshard, the tenant goes from a shuffled subset of queriers to a state of + // "tenant can use all queriers", as connected queriers is now <= tenant.maxQueriers. + // The second forgotten querier won't trigger a reshard, as connected queriers is already <= tenant.maxQueriers. + // + // We are testing that the occurrence of a reshard is reported correctly + // when not all querier forget operations in a single run of forgetDisconnectedQueriers caused a reshard. + queue.SubmitRegisterQuerierConnection("querier-1") + queue.SubmitRegisterQuerierConnection("querier-2") + queue.SubmitRegisterQuerierConnection("querier-3") + + // querier-2 waits for a new request. + querier2wg := sync.WaitGroup{} + querier2wg.Add(1) + go func() { + defer querier2wg.Done() + _, _, err := queue.WaitForRequestForQuerier(ctx, FirstTenant(), "querier-2") + require.NoError(t, err) + }() + + // querier-1 and querier-3 crash (no graceful shutdown notification). + queue.SubmitUnregisterQuerierConnection("querier-1") + queue.SubmitUnregisterQuerierConnection("querier-3") + + // Enqueue a request from a tenant which would be assigned to querier-1. + // NOTE: "user-1" shuffle shard always chooses the first querier ("querier-1" in this case) + // when there are only one or two queriers in the sorted list of connected queriers + req := &SchedulerRequest{ + Ctx: context.Background(), + Request: &httpgrpc.HTTPRequest{Method: "GET", Url: "/hello"}, + AdditionalQueueDimensions: randAdditionalQueueDimension(true), + } + require.NoError(t, queue.SubmitRequestToEnqueue("user-1", req, 2, nil)) + + startTime := time.Now() + done := make(chan struct{}) + go func() { + querier2wg.Wait() + close(done) + }() + + select { + case <-done: + waitTime := time.Since(startTime) + // We expect that querier-2 got the request only after forget delay is passed. + assert.GreaterOrEqual(t, waitTime.Milliseconds(), forgetDelay.Milliseconds()) + case <-time.After(testTimeout): + t.Fatal("timeout: querier-2 did not receive the request expected to be resharded to querier-2") + } + + }) } } func TestRequestQueue_GetNextRequestForQuerier_ShouldReturnAfterContextCancelled(t *testing.T) { const forgetDelay = 3 * time.Second const querierID = "querier-1" + treeTypes := buildTreeTestsStruct() + + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + queue, err := NewRequestQueue( + log.NewNopLogger(), + 1, + true, + tt.useIntegratedTreeQueue, + forgetDelay, + promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), + promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), + promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), + promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), + ) + require.NoError(t, err) - queue, err := NewRequestQueue( - log.NewNopLogger(), - 1, - true, - forgetDelay, - promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), - promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), - promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), - promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), - ) - require.NoError(t, err) - - require.NoError(t, services.StartAndAwaitRunning(context.Background(), queue)) - t.Cleanup(func() { - require.NoError(t, services.StopAndAwaitTerminated(context.Background(), queue)) - }) - - queue.SubmitRegisterQuerierConnection(querierID) - - // Calling WaitForRequestForQuerier with a context that is already cancelled should fail immediately. - deadCtx, cancel := context.WithCancel(context.Background()) - cancel() - r, tenant, err := queue.WaitForRequestForQuerier(deadCtx, FirstTenant(), querierID) - assert.Nil(t, r) - assert.Equal(t, FirstTenant(), tenant) - assert.ErrorIs(t, err, context.Canceled) - - // Further, a context canceled after WaitForRequestForQuerier publishes a request should also fail. - errChan := make(chan error) - ctx, cancel := context.WithCancel(context.Background()) - - go func() { - _, _, err := queue.WaitForRequestForQuerier(ctx, FirstTenant(), querierID) - errChan <- err - }() - - time.Sleep(20 * time.Millisecond) // Wait for WaitForRequestForQuerier to be waiting for a query. - cancel() - - select { - case err := <-errChan: - require.Equal(t, context.Canceled, err) - case <-time.After(time.Second): - require.Fail(t, "gave up waiting for GetNextRequestForQuerierToReturn") + require.NoError(t, services.StartAndAwaitRunning(context.Background(), queue)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), queue)) + }) + + queue.SubmitRegisterQuerierConnection(querierID) + + // Calling WaitForRequestForQuerier with a context that is already cancelled should fail immediately. + deadCtx, cancel := context.WithCancel(context.Background()) + cancel() + r, tenant, err := queue.WaitForRequestForQuerier(deadCtx, FirstTenant(), querierID) + assert.Nil(t, r) + assert.Equal(t, FirstTenant(), tenant) + assert.ErrorIs(t, err, context.Canceled) + + // Further, a context canceled after WaitForRequestForQuerier publishes a request should also fail. + errChan := make(chan error) + ctx, cancel := context.WithCancel(context.Background()) + + go func() { + _, _, err := queue.WaitForRequestForQuerier(ctx, FirstTenant(), querierID) + errChan <- err + }() + + time.Sleep(20 * time.Millisecond) // Wait for WaitForRequestForQuerier to be waiting for a query. + cancel() + + select { + case err := <-errChan: + require.Equal(t, context.Canceled, err) + case <-time.After(time.Second): + require.Fail(t, "gave up waiting for GetNextRequestForQuerierToReturn") + } + + }) } + } func TestRequestQueue_GetNextRequestForQuerier_ShouldReturnImmediatelyIfQuerierIsAlreadyShuttingDown(t *testing.T) { const forgetDelay = 3 * time.Second const querierID = "querier-1" - queue, err := NewRequestQueue( - log.NewNopLogger(), - 1, - true, - forgetDelay, - promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), - promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), - promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), - promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), - ) - require.NoError(t, err) - - ctx := context.Background() - require.NoError(t, services.StartAndAwaitRunning(ctx, queue)) - t.Cleanup(func() { - require.NoError(t, services.StopAndAwaitTerminated(ctx, queue)) - }) - - queue.SubmitRegisterQuerierConnection(querierID) - queue.SubmitNotifyQuerierShutdown(querierID) - - _, _, err = queue.WaitForRequestForQuerier(context.Background(), FirstTenant(), querierID) - require.EqualError(t, err, "querier has informed the scheduler it is shutting down") + treeTypes := buildTreeTestsStruct() + + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + queue, err := NewRequestQueue( + log.NewNopLogger(), + 1, + true, + tt.useIntegratedTreeQueue, + forgetDelay, + promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), + promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), + promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), + promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), + ) + require.NoError(t, err) + + ctx := context.Background() + require.NoError(t, services.StartAndAwaitRunning(ctx, queue)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, queue)) + }) + + queue.SubmitRegisterQuerierConnection(querierID) + queue.SubmitNotifyQuerierShutdown(querierID) + + _, _, err = queue.WaitForRequestForQuerier(context.Background(), FirstTenant(), querierID) + require.EqualError(t, err, "querier has informed the scheduler it is shutting down") + }) + } + } func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSendToQuerier(t *testing.T) { const forgetDelay = 3 * time.Second const querierID = "querier-1" - queue, err := NewRequestQueue( - log.NewNopLogger(), - 1, - true, - forgetDelay, - promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), - promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), - promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), - promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), - ) - require.NoError(t, err) - - // bypassing queue dispatcher loop for direct usage of the queueBroker and - // passing a waitingQuerierConn for a canceled querier connection - queueBroker := newQueueBroker(queue.maxOutstandingPerTenant, queue.additionalQueueDimensionsEnabled, false, queue.forgetDelay) - queueBroker.addQuerierConnection(querierID) - - tenantMaxQueriers := 0 // no sharding - req := &SchedulerRequest{ - Ctx: context.Background(), - Request: &httpgrpc.HTTPRequest{Method: "GET", Url: "/hello"}, - AdditionalQueueDimensions: randAdditionalQueueDimension(true), - } - tr := tenantRequest{ - tenantID: TenantID("tenant-1"), - req: req, - } + treeTypes := buildTreeTestsStruct() + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + queue, err := NewRequestQueue( + log.NewNopLogger(), + 1, + true, + tt.useIntegratedTreeQueue, + forgetDelay, + promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), + promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), + promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), + promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), + ) + require.NoError(t, err) + + // bypassing queue dispatcher loop for direct usage of the queueBroker and + // passing a waitingQuerierConn for a canceled querier connection + queueBroker := newQueueBroker(queue.maxOutstandingPerTenant, queue.additionalQueueDimensionsEnabled, false, queue.forgetDelay) + queueBroker.addQuerierConnection(querierID) - require.Nil(t, queueBroker.tree.rootNode.getNode(QueuePath{"tenant-1"})) - require.NoError(t, queueBroker.enqueueRequestBack(&tr, tenantMaxQueriers)) - require.False(t, queueBroker.tree.GetNode(QueuePath{"tenant-1"}).IsEmpty()) + tenantMaxQueriers := 0 // no sharding + req := &SchedulerRequest{ + Ctx: context.Background(), + Request: &httpgrpc.HTTPRequest{Method: "GET", Url: "/hello"}, + AdditionalQueueDimensions: randAdditionalQueueDimension(true), + } + tr := tenantRequest{ + tenantID: TenantID("tenant-1"), + req: req, + } - ctx, cancel := context.WithCancel(context.Background()) - call := &waitingQuerierConn{ - querierConnCtx: ctx, - querierID: QuerierID(querierID), - lastTenantIndex: FirstTenant(), - recvChan: make(chan requestForQuerier), + // TODO (casie): Clean this up when deprecating legacy tree queue + if tq, ok := queueBroker.tree.(*TreeQueue); ok { + require.Nil(t, tq.getNode(QueuePath{"tenant-1"})) + require.NoError(t, queueBroker.enqueueRequestBack(&tr, tenantMaxQueriers)) + require.False(t, tq.getNode(QueuePath{"tenant-1"}).IsEmpty()) + } else if itq, ok := queueBroker.tree.(*IntegratedTreeQueue); ok { + require.Nil(t, itq.GetNode(QueuePath{"tenant-1"})) + require.NoError(t, queueBroker.enqueueRequestBack(&tr, tenantMaxQueriers)) + require.False(t, itq.GetNode(QueuePath{"tenant-1"}).IsEmpty()) + } + + ctx, cancel := context.WithCancel(context.Background()) + call := &waitingQuerierConn{ + querierConnCtx: ctx, + querierID: QuerierID(querierID), + lastTenantIndex: FirstTenant(), + recvChan: make(chan requestForQuerier), + } + cancel() // ensure querier context done before send is attempted + + // send to querier will fail but method returns true, + // indicating not to re-submit a request for waitingQuerierConn for the querier + require.True(t, queue.trySendNextRequestForQuerier(call)) + // assert request was re-enqueued for tenant after failed send + // TODO (casie): Clean this up when deprecating legacy tree queue + if tq, ok := queueBroker.tree.(*TreeQueue); ok { + require.False(t, tq.getNode(QueuePath{"tenant-1"}).IsEmpty()) + } else if itq, ok := queueBroker.tree.(*IntegratedTreeQueue); ok { + require.False(t, itq.GetNode(QueuePath{"tenant-1"}).IsEmpty()) + } + + }) } - cancel() // ensure querier context done before send is attempted - // send to querier will fail but method returns true, - // indicating not to re-submit a request for waitingQuerierConn for the querier - require.True(t, queue.trySendNextRequestForQuerier(call)) - // assert request was re-enqueued for tenant after failed send - require.False(t, queueBroker.tree.GetNode(QueuePath{"tenant-1"}).IsEmpty()) } diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 87e4d1261bf..3665ef038d3 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -53,7 +53,7 @@ func (s querierIDSlice) Search(x QuerierID) int { return sort.Search(len(s), func(i int) bool { return s[i] >= x }) } -// tenantQuerierAssignments implements QueuingAlgorithm. In the context of a TreeQueue, it maintains a mapping of +// tenantQuerierAssignments implements QueuingAlgorithm. In the context of a IntegratedTreeQueue, it maintains a mapping of // tenants to queriers in order to support dequeuing from an appropriate tenant if shuffle-sharding is enabled. type tenantQuerierAssignments struct { // a tenant has many queriers @@ -96,6 +96,48 @@ type tenantQuerierAssignments struct { currentQuerier *QuerierID } +// getNextTenantForQuerier gets the next tenant in the tenant order assigned to a given querier. +// It should _only_ be called by the legacy TreeQueue. +// +// The next tenant for the querier is obtained by rotating through the global tenant order +// starting just after the last tenant the querier received a request for, until a tenant +// is found that is assigned to the given querier according to the querier shuffle sharding. +// A newly connected querier provides lastTenantIndex of -1 in order to start at the beginning. +func (tqa *tenantQuerierAssignments) getNextTenantForQuerier(lastTenantIndex int, querierID QuerierID) (*queueTenant, int, error) { + // check if querier is registered and is not shutting down + if q := tqa.queriersByID[querierID]; q == nil || q.shuttingDown { + return nil, lastTenantIndex, ErrQuerierShuttingDown + } + tenantOrderIndex := lastTenantIndex + for iters := 0; iters < len(tqa.tenantIDOrder); iters++ { + tenantOrderIndex++ + if tenantOrderIndex >= len(tqa.tenantIDOrder) { + // Do not use modulo (e.g. i = (i + 1) % len(slice)) to wrap this index. + // Tenant list can change size between calls and the querier provides its external view + // of the lastTenantIndex it received, which is not updated when this list changes. + // If the tenant list shrinks and the querier-provided lastTenantIndex exceeds the + // length of the tenant list, wrapping via modulo would skip the beginning of the list. + tenantOrderIndex = 0 + } + tenantID := tqa.tenantIDOrder[tenantOrderIndex] + if tenantID == emptyTenantID { + continue + } + tenant := tqa.tenantsByID[tenantID] + + tenantQuerierSet := tqa.tenantQuerierIDs[tenantID] + if tenantQuerierSet == nil { + // tenant can use all queriers + return tenant, tenantOrderIndex, nil + } else if _, ok := tenantQuerierSet[querierID]; ok { + // tenant is assigned this querier + return tenant, tenantOrderIndex, nil + } + } + + return nil, lastTenantIndex, nil +} + // createOrUpdateTenant creates or updates a tenant into the tenant-querier assignment state. // // New tenants are added to the tenant order list and tenant-querier shards are shuffled if needed. diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 7b923ae1de5..7794bd02cd5 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -24,7 +24,7 @@ type tenantRequest struct { // queueBroker encapsulates access to tenant queues for pending requests // and maintains consistency with the tenant-querier assignments type queueBroker struct { - tree *TreeQueue + tree Tree tenantQuerierAssignments *tenantQuerierAssignments @@ -36,7 +36,7 @@ type queueBroker struct { func newQueueBroker( maxTenantQueueSize int, additionalQueueDimensionsEnabled bool, - prioritizeQueryComponents bool, + useIntegratedTreeQueue bool, forgetDelay time.Duration, ) *queueBroker { currentQuerier := QuerierID("") @@ -48,23 +48,24 @@ func newQueueBroker( tenantsByID: map[TenantID]*queueTenant{}, tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, currentQuerier: ¤tQuerier, - tenantOrderIndex: localQueueIndex - 1, + tenantOrderIndex: localQueueIndex, } - // If prioritizeQueryComponents is true, the tree will be created - // with query components at one level above tenants; if it is false, - // tenant nodes will each maintain their own query component subtree. - tree, err := NewTree( - tqas, // root; QueuingAlgorithm selects tenants - &roundRobinState{}, // tenant queues; QueuingAlgorithm selects query component - &roundRobinState{}, // query components; QueuingAlgorithm selects query from local queue - ) - if prioritizeQueryComponents { + var tree Tree + var err error + if useIntegratedTreeQueue { tree, err = NewTree( - &roundRobinState{}, // root; QueuingAlgorithm selects query component - tqas, // query components; QueuingAlgorithm selects tenant - &roundRobinState{}, // tenant queues; QueuingAlgorithm selects query from local queue + tqas, // root; QueuingAlgorithm selects tenants + &roundRobinState{}, // tenant queues; QueuingAlgorithm selects query component + &roundRobinState{}, // query components; QueuingAlgorithm selects query from local queue ) + // An error building the tree is fatal; we must panic + if err != nil { + panic(fmt.Sprintf("error creating the tree queue: %v", err)) + } + } else { + // by default, use the legacy tree queue + tree = NewTreeQueue("root") } // An error building the tree is fatal; we must panic @@ -76,7 +77,6 @@ func newQueueBroker( tenantQuerierAssignments: tqas, maxTenantQueueSize: maxTenantQueueSize, additionalQueueDimensionsEnabled: additionalQueueDimensionsEnabled, - prioritizeQueryComponents: prioritizeQueryComponents, } return qb @@ -100,8 +100,21 @@ func (qb *queueBroker) enqueueRequestBack(request *tenantRequest, tenantMaxQueri return err } - if tenantQueueNode := qb.tree.rootNode.getNode(queuePath[:1]); tenantQueueNode != nil { - if tenantQueueNode.ItemCount()+1 > qb.maxTenantQueueSize { + // TODO (casie): When deprecating TreeQueue, clean this up. + // Technically, the IntegratedTreeQueue approach is adequate for both tree types, but we are temporarily + // maintaining the legacy tree behavior as much as possible for stability reasons. + if tq, ok := qb.tree.(*TreeQueue); ok { + if tenantQueueNode := tq.getNode(queuePath[:1]); tenantQueueNode != nil { + if tenantQueueNode.ItemCount()+1 > qb.maxTenantQueueSize { + return ErrTooManyRequests + } + } + } else if _, ok := qb.tree.(*IntegratedTreeQueue); ok { + itemCount := 0 + for _, tenantNode := range qb.tenantQuerierAssignments.tenantNodes[string(request.tenantID)] { + itemCount += tenantNode.ItemCount() + } + if itemCount+1 > qb.maxTenantQueueSize { return ErrTooManyRequests } } @@ -156,10 +169,22 @@ func (qb *queueBroker) dequeueRequestForQuerier( return nil, nil, qb.tenantQuerierAssignments.tenantOrderIndex, ErrQuerierShuttingDown } - qb.tenantQuerierAssignments.currentQuerier = &querierID - qb.tenantQuerierAssignments.tenantOrderIndex = lastTenantIndex - - queuePath, queueElement := qb.tree.Dequeue() + var queuePath QueuePath + var queueElement any + if tq, ok := qb.tree.(*TreeQueue); ok { + tenant, tenantIndex, err := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndex, querierID) + if tenant == nil || err != nil { + return nil, tenant, tenantIndex, err + } + qb.tenantQuerierAssignments.tenantOrderIndex = tenantIndex + queuePath = QueuePath{string(tenant.tenantID)} + queueElement = tq.DequeueByPath(queuePath) + } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + qb.tenantQuerierAssignments.currentQuerier = &querierID + qb.tenantQuerierAssignments.tenantOrderIndex = lastTenantIndex + + queuePath, queueElement = itq.Dequeue() + } var request *tenantRequest var tenantID TenantID @@ -174,13 +199,23 @@ func (qb *queueBroker) dequeueRequestForQuerier( tenant = qb.tenantQuerierAssignments.tenantsByID[tenantID] } - // dequeue returns the full path including root, but getNode expects the path _from_ root - queueNodeAfterDequeue := qb.tree.rootNode.getNode(queuePath) - if queueNodeAfterDequeue == nil && len(qb.tenantQuerierAssignments.tenantNodes[string(tenantID)]) == 0 { - // queue node was deleted due to being empty after dequeue - qb.tenantQuerierAssignments.removeTenant(tenantID) + // TODO (casie): When deprecating TreeQueue, clean this up. + // This cannot be handled by the Tree interface without defining some other, more expansive interfaces + // between the legacy and integrated tree queues, which would be more overhead than it's worth, given + // that we will eventually retire the legacy tree queue. + if tq, ok := qb.tree.(*TreeQueue); ok { + queueNodeAfterDequeue := tq.getNode(queuePath) + if queueNodeAfterDequeue == nil { + // queue node was deleted due to being empty after dequeue + qb.tenantQuerierAssignments.removeTenant(tenant.tenantID) + } + } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + queueNodeAfterDequeue := itq.GetNode(queuePath) + if queueNodeAfterDequeue == nil && len(qb.tenantQuerierAssignments.tenantNodes[string(tenantID)]) == 0 { + // queue node was deleted due to being empty after dequeue + qb.tenantQuerierAssignments.removeTenant(tenantID) + } } - return request, tenant, qb.tenantQuerierAssignments.tenantOrderIndex, nil } diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 92c3f62a90c..d1630866e10 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -21,10 +21,15 @@ import ( func (qb *queueBroker) enqueueObjectsForTests(tenantID TenantID, numObjects int) error { for i := 0; i < numObjects; i++ { - err := qb.tree.EnqueueBackByPath(QueuePath{string(tenantID)}, &tenantRequest{ + req := &tenantRequest{ tenantID: tenantID, req: fmt.Sprintf("%v: object-%v", tenantID, i), - }) + } + path, err := qb.makeQueuePath(req) + if err != nil { + return err + } + err = qb.tree.EnqueueBackByPath(path, req) if err != nil { return err } @@ -61,334 +66,373 @@ func assertExpectedValuesOnDequeue(t *testing.T, qb *queueBroker, lastTenantInde // TestQueues_NoShuffleSharding tests dequeueing of objects for different queriers, where any querier can // handle queries for any tenant, as tenant queues are added and removed func TestQueues_NoShuffleSharding(t *testing.T) { - qb := newQueueBroker(0, true, false, 0) - assert.NotNil(t, qb) - assert.NoError(t, isConsistent(qb)) + treeTypes := buildTreeTestsStruct() + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, 0) + assert.NotNil(t, qb) + assert.NoError(t, isConsistent(qb)) - qb.addQuerierConnection("querier-1") - qb.addQuerierConnection("querier-2") + qb.addQuerierConnection("querier-1") + qb.addQuerierConnection("querier-2") - req, tenant, lastTenantIndexQuerierOne, err := qb.dequeueRequestForQuerier(-1, "querier-1") - assert.Nil(t, req) - assert.Nil(t, tenant) - assert.NoError(t, err) + req, tenant, lastTenantIndexQuerierOne, err := qb.dequeueRequestForQuerier(-1, "querier-1") + assert.Nil(t, req) + assert.Nil(t, tenant) + assert.NoError(t, err) - // Add tenant queues: tenant "one" - err = qb.tenantQuerierAssignments.createOrUpdateTenant("one", 0) - assert.NoError(t, err) + // Add tenant queues: tenant "one" + err = qb.tenantQuerierAssignments.createOrUpdateTenant("one", 0) + assert.NoError(t, err) - tenantOne := qb.tenantQuerierAssignments.tenantsByID["one"] + tenantOne := qb.tenantQuerierAssignments.tenantsByID["one"] - // Queue enough objects for tenant "one" - err = qb.enqueueObjectsForTests(tenantOne.tenantID, 10) - assert.NoError(t, err) - assert.NoError(t, isConsistent(qb)) + // Queue enough objects for tenant "one" + err = qb.enqueueObjectsForTests(tenantOne.tenantID, 10) + assert.NoError(t, err) + assert.NoError(t, isConsistent(qb)) - //queuePathOne := QueuePath{"root", "one"} - expectedDequeueVals := []dequeueVal{ - {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, - {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, - } - lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) + //queuePathOne := QueuePath{"root", "one"} + expectedDequeueVals := []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, + {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, + } + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) - // Add tenant two - err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) - assert.NoError(t, err) + // Add tenant two + err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) + assert.NoError(t, err) - tenantTwo := qb.tenantQuerierAssignments.tenantsByID["two"] + tenantTwo := qb.tenantQuerierAssignments.tenantsByID["two"] - err = qb.enqueueObjectsForTests(tenantTwo.tenantID, 10) - assert.NoError(t, err) - assert.NoError(t, isConsistent(qb)) + err = qb.enqueueObjectsForTests(tenantTwo.tenantID, 10) + assert.NoError(t, err) + assert.NoError(t, isConsistent(qb)) - expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantTwo.tenantID, 0), tenantTwo}, - {buildExpectedObject(tenantOne.tenantID, 2), tenantOne}, - {buildExpectedObject(tenantTwo.tenantID, 1), tenantTwo}, - {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, - } + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantTwo.tenantID, 0), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 2), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 1), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, + } - lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) - expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, - {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, - {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, - } - lastTenantIndexQuerierTwo := assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, + } + lastTenantIndexQuerierTwo := assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) + + // [one two three] + // confirm fifo by adding a third tenant queue and iterating to it + err = qb.tenantQuerierAssignments.createOrUpdateTenant("three", 0) + assert.NoError(t, err) + + tenantThree := qb.tenantQuerierAssignments.tenantsByID["three"] - // [one two three] - // confirm fifo by adding a third tenant queue and iterating to it - err = qb.tenantQuerierAssignments.createOrUpdateTenant("three", 0) - assert.NoError(t, err) + err = qb.enqueueObjectsForTests(tenantThree.tenantID, 10) + assert.NoError(t, err) + assert.NoError(t, isConsistent(qb)) - tenantThree := qb.tenantQuerierAssignments.tenantsByID["three"] + // lastTenantIndexQuerierOne was 0 (tenantOne) for querier-1; appending + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, + {buildExpectedObject(tenantThree.tenantID, 0), tenantThree}, + {buildExpectedObject(tenantOne.tenantID, 6), tenantOne}, + } - err = qb.enqueueObjectsForTests(tenantThree.tenantID, 10) - assert.NoError(t, err) - assert.NoError(t, isConsistent(qb)) + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) - // lastTenantIndexQuerierOne was 0 (tenantOne) for querier-1; appending - expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, - {buildExpectedObject(tenantThree.tenantID, 0), tenantThree}, - {buildExpectedObject(tenantOne.tenantID, 6), tenantOne}, - } + // Remove one: ["" two three] + qb.removeTenantQueue("one") + assert.NoError(t, isConsistent(qb)) - lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantTwo.tenantID, 4), tenantTwo}, + {buildExpectedObject(tenantThree.tenantID, 1), tenantThree}, + {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, + } + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) - // Remove one: ["" two three] - qb.removeTenantQueue("one") - assert.NoError(t, isConsistent(qb)) + // "four" is added at the beginning of the list: [four two three] + err = qb.tenantQuerierAssignments.createOrUpdateTenant("four", 0) + assert.NoError(t, err) - expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantTwo.tenantID, 4), tenantTwo}, - {buildExpectedObject(tenantThree.tenantID, 1), tenantThree}, - {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, - } - lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) + tenantFour := qb.tenantQuerierAssignments.tenantsByID["four"] - // "four" is added at the beginning of the list: [four two three] - err = qb.tenantQuerierAssignments.createOrUpdateTenant("four", 0) - assert.NoError(t, err) + err = qb.enqueueObjectsForTests(tenantFour.tenantID, 10) + assert.NoError(t, err) + assert.NoError(t, isConsistent(qb)) - tenantFour := qb.tenantQuerierAssignments.tenantsByID["four"] + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantTwo.tenantID, 6), tenantTwo}, + {buildExpectedObject(tenantThree.tenantID, 2), tenantThree}, + {buildExpectedObject(tenantFour.tenantID, 0), tenantFour}, + {buildExpectedObject(tenantTwo.tenantID, 7), tenantTwo}, + } - err = qb.enqueueObjectsForTests(tenantFour.tenantID, 10) - assert.NoError(t, err) - assert.NoError(t, isConsistent(qb)) + lastTenantIndexQuerierTwo = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierTwo, "querier-2", expectedDequeueVals) - expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantTwo.tenantID, 6), tenantTwo}, - {buildExpectedObject(tenantThree.tenantID, 2), tenantThree}, - {buildExpectedObject(tenantFour.tenantID, 0), tenantFour}, - {buildExpectedObject(tenantTwo.tenantID, 7), tenantTwo}, - } - lastTenantIndexQuerierTwo = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierTwo, "querier-2", expectedDequeueVals) + // Remove two: [four "" three] + qb.removeTenantQueue("two") + assert.NoError(t, isConsistent(qb)) - // Remove two: [four "" three] - qb.removeTenantQueue("two") - assert.NoError(t, isConsistent(qb)) + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantThree.tenantID, 3), tenantThree}, + {buildExpectedObject(tenantFour.tenantID, 1), tenantFour}, + {buildExpectedObject(tenantThree.tenantID, 4), tenantThree}, + } + lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) - expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantThree.tenantID, 3), tenantThree}, - {buildExpectedObject(tenantFour.tenantID, 1), tenantFour}, - {buildExpectedObject(tenantThree.tenantID, 4), tenantThree}, - } - lastTenantIndexQuerierOne = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierOne, "querier-1", expectedDequeueVals) + // Remove three: [four ""] ) + qb.removeTenantQueue("three") + assert.NoError(t, isConsistent(qb)) - // Remove three: [four ""] ) - qb.removeTenantQueue("three") - assert.NoError(t, isConsistent(qb)) + // Remove four: [] + qb.removeTenantQueue("four") + assert.NoError(t, isConsistent(qb)) - // Remove four: [] - qb.removeTenantQueue("four") - assert.NoError(t, isConsistent(qb)) + req, tenant, _, err = qb.dequeueRequestForQuerier(lastTenantIndexQuerierOne, "querier-1") + assert.Nil(t, req) + assert.Nil(t, tenant) + assert.NoError(t, err) - req, tenant, _, err = qb.dequeueRequestForQuerier(lastTenantIndexQuerierOne, "querier-1") - assert.Nil(t, req) - assert.Nil(t, tenant) - assert.NoError(t, err) + }) + } } func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { - maxTenantQueueSize := 100 - qb := newQueueBroker(maxTenantQueueSize, true, false, 0) - additionalQueueDimensions := map[int][]string{ - 0: nil, - 1: {"ingester"}, - 2: {"store-gateway"}, - 3: {"ingester-and-store-gateway"}, - } - req := &SchedulerRequest{ - Ctx: context.Background(), - FrontendAddr: "http://query-frontend:8007", - UserID: "tenant-1", - Request: &httpgrpc.HTTPRequest{}, - } + // TODO (casie): When implementing prioritizeQueryComponents, add tests here, since we will be able to have + // multiple nodes for a single tenant + treeTypes := buildTreeTestsStruct() + + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + maxTenantQueueSize := 100 + qb := newQueueBroker(maxTenantQueueSize, true, tt.useIntegratedTreeQueue, 0) + additionalQueueDimensions := map[int][]string{ + 0: nil, + 1: {"ingester"}, + 2: {"store-gateway"}, + 3: {"ingester-and-store-gateway"}, + } + req := &SchedulerRequest{ + Ctx: context.Background(), + FrontendAddr: "http://query-frontend:8007", + UserID: "tenant-1", + Request: &httpgrpc.HTTPRequest{}, + } + + // build queue evenly with either no additional queue dimension or one of 3 additional dimensions + for i := 0; i < len(additionalQueueDimensions); i++ { + for j := 0; j < maxTenantQueueSize/len(additionalQueueDimensions); j++ { + req.AdditionalQueueDimensions = additionalQueueDimensions[i] + tenantReq := &tenantRequest{tenantID: "tenant-1", req: req} + err := qb.enqueueRequestBack(tenantReq, 0) + assert.NoError(t, err) + } + } + // assert item count of tenant node and its subnodes + queuePath := QueuePath{"tenant-1"} + + // TODO (casie): After deprecating legacy tree queue, clean this up + if tq, ok := qb.tree.(*TreeQueue); ok { + assert.Equal(t, maxTenantQueueSize, tq.getNode(queuePath).ItemCount()) + } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + assert.Equal(t, maxTenantQueueSize, itq.GetNode(queuePath).ItemCount()) + } + + // assert equal distribution of queue items between tenant node and 3 subnodes + for _, v := range additionalQueueDimensions { + queuePath := append(QueuePath{"tenant-1"}, v...) + // TODO (casie): After deprecating legacy tree queue, clean this up + var itemCount int + if tq, ok := qb.tree.(*TreeQueue); ok { + itemCount = tq.getNode(queuePath).LocalQueueLen() + } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + itemCount = itq.GetNode(queuePath).getLocalQueue().Len() + } + assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), itemCount) + } + + // assert error received when hitting a tenant's enqueue limit, + // even though most of the requests are in the subqueues + for _, additionalQueueDimension := range additionalQueueDimensions { + // error should be received no matter if the enqueue attempt + // is for the tenant queue or any of its subqueues + req.AdditionalQueueDimensions = additionalQueueDimension + tenantReq := &tenantRequest{tenantID: "tenant-1", req: req} + err := qb.enqueueRequestBack(tenantReq, 0) + assert.ErrorIs(t, err, ErrTooManyRequests) + } + + // dequeue a request + qb.addQuerierConnection("querier-1") + dequeuedTenantReq, _, _, err := qb.dequeueRequestForQuerier(-1, "querier-1") + assert.NoError(t, err) + assert.NotNil(t, dequeuedTenantReq) - // build queue evenly with either no additional queue dimension or one of 3 additional dimensions - for i := 0; i < len(additionalQueueDimensions); i++ { - for j := 0; j < maxTenantQueueSize/len(additionalQueueDimensions); j++ { - req.AdditionalQueueDimensions = additionalQueueDimensions[i] tenantReq := &tenantRequest{tenantID: "tenant-1", req: req} - err := qb.enqueueRequestBack(tenantReq, 0) + // assert not hitting an error when enqueueing after dequeuing to below the limit + err = qb.enqueueRequestBack(tenantReq, 0) assert.NoError(t, err) - } - } - // assert item count of tenant node and its subnodes - queuePath := QueuePath{"tenant-1"} - assert.Equal(t, maxTenantQueueSize, qb.tree.rootNode.getNode(queuePath).ItemCount()) - - // assert equal distribution of queue items between tenant node and 3 subnodes - for _, v := range additionalQueueDimensions { - queuePath := append(QueuePath{"tenant-1"}, v...) - assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), qb.tree.GetNode(queuePath).getLocalQueue().Len()) - } - // assert error received when hitting a tenant's enqueue limit, - // even though most of the requests are in the subqueues - for _, additionalQueueDimension := range additionalQueueDimensions { - // error should be received no matter if the enqueue attempt - // is for the tenant queue or any of its subqueues - req.AdditionalQueueDimensions = additionalQueueDimension - tenantReq := &tenantRequest{tenantID: "tenant-1", req: req} - err := qb.enqueueRequestBack(tenantReq, 0) - assert.ErrorIs(t, err, ErrTooManyRequests) + // we then hit an error again, as we are back at the limit + err = qb.enqueueRequestBack(tenantReq, 0) + assert.ErrorIs(t, err, ErrTooManyRequests) + }) } +} - // dequeue a request - qb.addQuerierConnection("querier-1") - dequeuedTenantReq, _, _, err := qb.dequeueRequestForQuerier(-1, "querier-1") - assert.NoError(t, err) - assert.NotNil(t, dequeuedTenantReq) +func TestQueuesOnTerminatingQuerier(t *testing.T) { + treeTypes := buildTreeTestsStruct() + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, 0) + assert.NotNil(t, qb) + assert.NoError(t, isConsistent(qb)) - tenantReq := &tenantRequest{tenantID: "tenant-1", req: req} - // assert not hitting an error when enqueueing after dequeuing to below the limit - err = qb.enqueueRequestBack(tenantReq, 0) - assert.NoError(t, err) + qb.addQuerierConnection("querier-1") + qb.addQuerierConnection("querier-2") - // we then hit an error again, as we are back at the limit - err = qb.enqueueRequestBack(tenantReq, 0) - assert.ErrorIs(t, err, ErrTooManyRequests) -} + // Add queues: [one two] + err := qb.tenantQuerierAssignments.createOrUpdateTenant("one", 0) + assert.NoError(t, err) + err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) + assert.NoError(t, err) -func TestQueuesOnTerminatingQuerier(t *testing.T) { - qb := newQueueBroker(0, true, false, 0) - assert.NotNil(t, qb) - assert.NoError(t, isConsistent(qb)) - - qb.addQuerierConnection("querier-1") - qb.addQuerierConnection("querier-2") - - // Add queues: [one two] - err := qb.tenantQuerierAssignments.createOrUpdateTenant("one", 0) - assert.NoError(t, err) - err = qb.tenantQuerierAssignments.createOrUpdateTenant("two", 0) - assert.NoError(t, err) - - err = qb.enqueueObjectsForTests("one", 10) - assert.NoError(t, err) - tenantOne := qb.tenantQuerierAssignments.tenantsByID["one"] - - err = qb.enqueueObjectsForTests("two", 10) - assert.NoError(t, err) - tenantTwo := qb.tenantQuerierAssignments.tenantsByID["two"] - - expectedDequeueVals := []dequeueVal{ - {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, - {buildExpectedObject(tenantTwo.tenantID, 0), tenantTwo}, - {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, - {buildExpectedObject(tenantTwo.tenantID, 1), tenantTwo}, - } - qOneLastTenantIndex := assertExpectedValuesOnDequeue(t, qb, -1, "querier-1", expectedDequeueVals) + err = qb.enqueueObjectsForTests("one", 10) + assert.NoError(t, err) + tenantOne := qb.tenantQuerierAssignments.tenantsByID["one"] - expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantOne.tenantID, 2), tenantOne}, - {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, - {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, - {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, - } - qTwolastTenantIndex := assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) - - // After notify shutdown for querier-2, it's expected to own no queue. - qb.notifyQuerierShutdown("querier-2") - req, tenant, qTwolastTenantIndex, err := qb.dequeueRequestForQuerier(qTwolastTenantIndex, "querier-2") - assert.Nil(t, req) - assert.Nil(t, tenant) - assert.Equal(t, ErrQuerierShuttingDown, err) - - // However, querier-1 still get queues because it's still running. - expectedDequeueVals = []dequeueVal{ - {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, - {buildExpectedObject(tenantTwo.tenantID, 4), tenantTwo}, - {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, - {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, - } + err = qb.enqueueObjectsForTests("two", 10) + assert.NoError(t, err) + tenantTwo := qb.tenantQuerierAssignments.tenantsByID["two"] - for _, expected := range expectedDequeueVals { - req, tenant, qOneLastTenantIndex, err = qb.dequeueRequestForQuerier(qOneLastTenantIndex, "querier-1") - assert.Equal(t, expected.req, req) - assert.Equal(t, expected.tenant, tenant) - assert.NoError(t, err) - } + expectedDequeueVals := []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 0), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 0), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 1), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 1), tenantTwo}, + } + qOneLastTenantIndex := assertExpectedValuesOnDequeue(t, qb, -1, "querier-1", expectedDequeueVals) - // After disconnecting querier-2, it's expected to own no queue. - qb.tenantQuerierAssignments.removeQuerier("querier-2") - req, tenant, _, err = qb.dequeueRequestForQuerier(qTwolastTenantIndex, "querier-2") - assert.Nil(t, req) - assert.Nil(t, tenant) - assert.Equal(t, ErrQuerierShuttingDown, err) + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 2), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 2), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 3), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 3), tenantTwo}, + } + qTwolastTenantIndex := assertExpectedValuesOnDequeue(t, qb, -1, "querier-2", expectedDequeueVals) + + // After notify shutdown for querier-2, it's expected to own no queue. + qb.notifyQuerierShutdown("querier-2") + req, tenant, qTwolastTenantIndex, err := qb.dequeueRequestForQuerier(qTwolastTenantIndex, "querier-2") + assert.Nil(t, req) + assert.Nil(t, tenant) + assert.Equal(t, ErrQuerierShuttingDown, err) + + // However, querier-1 still get queues because it's still running. + expectedDequeueVals = []dequeueVal{ + {buildExpectedObject(tenantOne.tenantID, 4), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 4), tenantTwo}, + {buildExpectedObject(tenantOne.tenantID, 5), tenantOne}, + {buildExpectedObject(tenantTwo.tenantID, 5), tenantTwo}, + } + + for _, expected := range expectedDequeueVals { + req, tenant, qOneLastTenantIndex, err = qb.dequeueRequestForQuerier(qOneLastTenantIndex, "querier-1") + assert.Equal(t, expected.req, req) + assert.Equal(t, expected.tenant, tenant) + assert.NoError(t, err) + } + + // After disconnecting querier-2, it's expected to own no queue. + qb.tenantQuerierAssignments.removeQuerier("querier-2") + req, tenant, _, err = qb.dequeueRequestForQuerier(qTwolastTenantIndex, "querier-2") + assert.Nil(t, req) + assert.Nil(t, tenant) + assert.Equal(t, ErrQuerierShuttingDown, err) + }) + } } func TestQueues_QuerierDistribution(t *testing.T) { - qb := newQueueBroker(0, true, false, 0) - assert.NotNil(t, qb) - assert.NoError(t, isConsistent(qb)) - - queriers := 30 - numTenants := 1000 - maxQueriersPerTenant := 5 - - // Add some queriers. - for ix := 0; ix < queriers; ix++ { - qid := QuerierID(fmt.Sprintf("querier-%d", ix)) - qb.addQuerierConnection(qid) - - // No querier has any queues yet. - req, tenant, _, err := qb.dequeueRequestForQuerier(-1, qid) - assert.Nil(t, req) - assert.Nil(t, tenant) - assert.NoError(t, err) - } + treeTypes := buildTreeTestsStruct() + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, 0) + assert.NotNil(t, qb) + assert.NoError(t, isConsistent(qb)) - assert.NoError(t, isConsistent(qb)) + queriers := 30 + numTenants := 1000 + maxQueriersPerTenant := 5 - // Add tenant queues. - for i := 0; i < numTenants; i++ { - uid := TenantID(fmt.Sprintf("tenant-%d", i)) - err := qb.tenantQuerierAssignments.createOrUpdateTenant(uid, maxQueriersPerTenant) - assert.NoError(t, err) + // Add some queriers. + for ix := 0; ix < queriers; ix++ { + qid := QuerierID(fmt.Sprintf("querier-%d", ix)) + qb.addQuerierConnection(qid) - //Enqueue some stuff so that the tree queue node exists - err = qb.enqueueObjectsForTests(uid, 1) - assert.NoError(t, err) + // No querier has any queues yet. + req, tenant, _, err := qb.dequeueRequestForQuerier(-1, qid) + assert.Nil(t, req) + assert.Nil(t, tenant) + assert.NoError(t, err) + } - // Verify it has maxQueriersPerTenant queriers assigned now. - qs := qb.tenantQuerierAssignments.tenantQuerierIDs[uid] - assert.Equal(t, maxQueriersPerTenant, len(qs)) - } + assert.NoError(t, isConsistent(qb)) - // After adding all tenants, verify results. For each querier, find out how many different tenants it handles, - // and compute mean and stdDev. - queriersMap := make(map[QuerierID]int) + // Add tenant queues. + for i := 0; i < numTenants; i++ { + uid := TenantID(fmt.Sprintf("tenant-%d", i)) + err := qb.tenantQuerierAssignments.createOrUpdateTenant(uid, maxQueriersPerTenant) + assert.NoError(t, err) - for _, querierSet := range qb.tenantQuerierAssignments.tenantQuerierIDs { - for querierID := range querierSet { - queriersMap[querierID]++ - } - } + //Enqueue some stuff so that the tree queue node exists + err = qb.enqueueObjectsForTests(uid, 1) + assert.NoError(t, err) - mean := float64(0) - for _, c := range queriersMap { - mean += float64(c) - } - mean = mean / float64(len(queriersMap)) + // Verify it has maxQueriersPerTenant queriers assigned now. + qs := qb.tenantQuerierAssignments.tenantQuerierIDs[uid] + assert.Equal(t, maxQueriersPerTenant, len(qs)) + } - stdDev := float64(0) - for _, c := range queriersMap { - d := float64(c) - mean - stdDev += (d * d) - } - stdDev = math.Sqrt(stdDev / float64(len(queriersMap))) - t.Log("mean:", mean, "stddev:", stdDev) + // After adding all tenants, verify results. For each querier, find out how many different tenants it handles, + // and compute mean and stdDev. + queriersMap := make(map[QuerierID]int) - assert.InDelta(t, numTenants*maxQueriersPerTenant/queriers, mean, 1) - assert.InDelta(t, stdDev, 0, mean*0.2) + for _, querierSet := range qb.tenantQuerierAssignments.tenantQuerierIDs { + for querierID := range querierSet { + queriersMap[querierID]++ + } + } + + mean := float64(0) + for _, c := range queriersMap { + mean += float64(c) + } + mean = mean / float64(len(queriersMap)) + + stdDev := float64(0) + for _, c := range queriersMap { + d := float64(c) - mean + stdDev += (d * d) + } + stdDev = math.Sqrt(stdDev / float64(len(queriersMap))) + t.Log("mean:", mean, "stddev:", stdDev) + + assert.InDelta(t, numTenants*maxQueriersPerTenant/queriers, mean, 1) + assert.InDelta(t, stdDev, 0, mean*0.2) + }) + } } func TestQueuesConsistency(t *testing.T) { + treeTypes := buildTreeTestsStruct() tests := map[string]struct { forgetDelay time.Duration }{ @@ -396,46 +440,49 @@ func TestQueuesConsistency(t *testing.T) { "with forget delay": {forgetDelay: time.Minute}, } - for testName, testData := range tests { - t.Run(testName, func(t *testing.T) { - qb := newQueueBroker(0, true, false, testData.forgetDelay) - assert.NotNil(t, qb) - assert.NoError(t, isConsistent(qb)) - - r := rand.New(rand.NewSource(time.Now().Unix())) - - lastTenantIndexes := map[QuerierID]int{} - - conns := map[QuerierID]int{} - - for i := 0; i < 10000; i++ { - switch r.Int() % 6 { - case 0: - queue, err := qb.getOrAddTenantQueue(generateTenant(r), 3) - assert.Nil(t, err) - assert.NotNil(t, queue) - case 1: - querierID := generateQuerier(r) - _, tenantIndex, _ := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndexes[querierID], querierID) - lastTenantIndexes[querierID] = tenantIndex - case 2: - qb.removeTenantQueue(generateTenant(r)) - case 3: - q := generateQuerier(r) - qb.addQuerierConnection(q) - conns[q]++ - case 4: - q := generateQuerier(r) - if conns[q] > 0 { - qb.removeQuerierConnection(q, time.Now()) - conns[q]-- + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, testData.forgetDelay) + assert.NotNil(t, qb) + assert.NoError(t, isConsistent(qb)) + + r := rand.New(rand.NewSource(time.Now().Unix())) + + lastTenantIndexes := map[QuerierID]int{} + + conns := map[QuerierID]int{} + + for i := 0; i < 10000; i++ { + switch r.Int() % 6 { + case 0: + err := qb.getOrAddTenantQueue(generateTenant(r), 3) + assert.Nil(t, err) + case 1: + querierID := generateQuerier(r) + _, tenantIndex, _ := qb.tenantQuerierAssignments.getNextTenantForQuerier(lastTenantIndexes[querierID], querierID) + lastTenantIndexes[querierID] = tenantIndex + case 2: + qb.removeTenantQueue(generateTenant(r)) + case 3: + q := generateQuerier(r) + qb.addQuerierConnection(q) + conns[q]++ + case 4: + q := generateQuerier(r) + if conns[q] > 0 { + qb.removeQuerierConnection(q, time.Now()) + conns[q]-- + } + case 5: + q := generateQuerier(r) + qb.notifyQuerierShutdown(q) + } + + assert.NoErrorf(t, isConsistent(qb), "last action %d, rand: %d", i, r.Int()%6) } - case 5: - q := generateQuerier(r) - qb.notifyQuerierShutdown(q) - } - - assert.NoErrorf(t, isConsistent(qb), "last action %d, rand: %d", i, r.Int()%6) + }) } }) } @@ -448,93 +495,99 @@ func TestQueues_ForgetDelay(t *testing.T) { numTenants = 10 ) - now := time.Now() - qb := newQueueBroker(0, true, false, forgetDelay) - assert.NotNil(t, qb) - assert.NoError(t, isConsistent(qb)) + treeTypes := buildTreeTestsStruct() - // 3 queriers open 2 connections each. - for i := 1; i <= 3; i++ { - qb.addQuerierConnection(QuerierID(fmt.Sprintf("querier-%d", i))) - qb.addQuerierConnection(QuerierID(fmt.Sprintf("querier-%d", i))) - } + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + now := time.Now() + qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, forgetDelay) + assert.NotNil(t, qb) + assert.NoError(t, isConsistent(qb)) - // Add tenant queues. - for i := 0; i < numTenants; i++ { - tenantID := TenantID(fmt.Sprintf("tenant-%d", i)) - err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriersPerTenant) - assert.NoError(t, err) + // 3 queriers open 2 connections each. + for i := 1; i <= 3; i++ { + qb.addQuerierConnection(QuerierID(fmt.Sprintf("querier-%d", i))) + qb.addQuerierConnection(QuerierID(fmt.Sprintf("querier-%d", i))) + } - //Enqueue some stuff so that the tree queue node exists - err = qb.enqueueObjectsForTests(tenantID, 1) - assert.NoError(t, err) - } + // Add tenant queues. + for i := 0; i < numTenants; i++ { + tenantID := TenantID(fmt.Sprintf("tenant-%d", i)) + err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriersPerTenant) + assert.NoError(t, err) - // We expect querier-1 to have some tenants. - querier1Tenants := getTenantsByQuerier(qb, "querier-1") - require.NotEmpty(t, querier1Tenants) + //Enqueue some stuff so that the tree queue node exists + err = qb.enqueueObjectsForTests(tenantID, 1) + assert.NoError(t, err) + } - // Gracefully shutdown querier-1. - qb.removeQuerierConnection("querier-1", now.Add(20*time.Second)) - qb.removeQuerierConnection("querier-1", now.Add(21*time.Second)) - qb.notifyQuerierShutdown("querier-1") + // We expect querier-1 to have some tenants. + querier1Tenants := getTenantsByQuerier(qb, "querier-1") + require.NotEmpty(t, querier1Tenants) - // We expect querier-1 has been removed. - assert.NotContains(t, qb.tenantQuerierAssignments.queriersByID, "querier-1") - assert.NoError(t, isConsistent(qb)) + // Gracefully shutdown querier-1. + qb.removeQuerierConnection("querier-1", now.Add(20*time.Second)) + qb.removeQuerierConnection("querier-1", now.Add(21*time.Second)) + qb.notifyQuerierShutdown("querier-1") - // We expect querier-1 tenants have been shuffled to other queriers. - for _, tenantID := range querier1Tenants { - assert.Contains(t, append(getTenantsByQuerier(qb, "querier-2"), getTenantsByQuerier(qb, "querier-3")...), tenantID) - } + // We expect querier-1 has been removed. + assert.NotContains(t, qb.tenantQuerierAssignments.queriersByID, "querier-1") + assert.NoError(t, isConsistent(qb)) - // Querier-1 reconnects. - qb.addQuerierConnection("querier-1") - qb.addQuerierConnection("querier-1") + // We expect querier-1 tenants have been shuffled to other queriers. + for _, tenantID := range querier1Tenants { + assert.Contains(t, append(getTenantsByQuerier(qb, "querier-2"), getTenantsByQuerier(qb, "querier-3")...), tenantID) + } - // We expect the initial querier-1 tenants have got back to querier-1. - for _, tenantID := range querier1Tenants { - assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) - } + // Querier-1 reconnects. + qb.addQuerierConnection("querier-1") + qb.addQuerierConnection("querier-1") - // Querier-1 abruptly terminates (no shutdown notification received). - qb.removeQuerierConnection("querier-1", now.Add(40*time.Second)) - qb.removeQuerierConnection("querier-1", now.Add(41*time.Second)) + // We expect the initial querier-1 tenants have got back to querier-1. + for _, tenantID := range querier1Tenants { + assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) + } - // We expect querier-1 has NOT been removed. - assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) - assert.NoError(t, isConsistent(qb)) + // Querier-1 abruptly terminates (no shutdown notification received). + qb.removeQuerierConnection("querier-1", now.Add(40*time.Second)) + qb.removeQuerierConnection("querier-1", now.Add(41*time.Second)) - // We expect the querier-1 tenants have not been shuffled to other queriers. - for _, tenantID := range querier1Tenants { - assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) - } + // We expect querier-1 has NOT been removed. + assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) + assert.NoError(t, isConsistent(qb)) - // Try to forget disconnected queriers, but querier-1 forget delay hasn't passed yet. - qb.forgetDisconnectedQueriers(now.Add(90 * time.Second)) + // We expect the querier-1 tenants have not been shuffled to other queriers. + for _, tenantID := range querier1Tenants { + assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) + } - assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) - assert.NoError(t, isConsistent(qb)) + // Try to forget disconnected queriers, but querier-1 forget delay hasn't passed yet. + qb.forgetDisconnectedQueriers(now.Add(90 * time.Second)) - for _, tenantID := range querier1Tenants { - assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) - } + assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) + assert.NoError(t, isConsistent(qb)) - // Try to forget disconnected queriers. This time querier-1 forget delay has passed. - qb.forgetDisconnectedQueriers(now.Add(105 * time.Second)) + for _, tenantID := range querier1Tenants { + assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) + } - assert.NotContains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) - assert.NoError(t, isConsistent(qb)) + // Try to forget disconnected queriers. This time querier-1 forget delay has passed. + qb.forgetDisconnectedQueriers(now.Add(105 * time.Second)) - // We expect querier-1 tenants have been shuffled to other queriers. - for _, tenantID := range querier1Tenants { - assert.Contains(t, append(getTenantsByQuerier(qb, "querier-2"), getTenantsByQuerier(qb, "querier-3")...), tenantID) + assert.NotContains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) + assert.NoError(t, isConsistent(qb)) + + // We expect querier-1 tenants have been shuffled to other queriers. + for _, tenantID := range querier1Tenants { + assert.Contains(t, append(getTenantsByQuerier(qb, "querier-2"), getTenantsByQuerier(qb, "querier-3")...), tenantID) + } + }) } } @@ -545,76 +598,84 @@ func TestQueues_ForgetDelay_ShouldCorrectlyHandleQuerierReconnectingBeforeForget numTenants = 100 ) - now := time.Now() - qb := newQueueBroker(0, true, false, forgetDelay) - assert.NotNil(t, qb) - assert.NoError(t, isConsistent(qb)) + treeTypes := buildTreeTestsStruct() - // 3 queriers open 2 connections each. - for i := 1; i <= 3; i++ { - qb.addQuerierConnection(QuerierID(fmt.Sprintf("querier-%d", i))) - qb.addQuerierConnection(QuerierID(fmt.Sprintf("querier-%d", i))) - } + for _, tt := range treeTypes { + t.Run(tt.name, func(t *testing.T) { + now := time.Now() + qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, forgetDelay) + assert.NotNil(t, qb) + assert.NoError(t, isConsistent(qb)) - // Add tenant queues. - for i := 0; i < numTenants; i++ { - tenantID := TenantID(fmt.Sprintf("tenant-%d", i)) - err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriersPerTenant) - assert.NoError(t, err) + // 3 queriers open 2 connections each. + for i := 1; i <= 3; i++ { + qb.addQuerierConnection(QuerierID(fmt.Sprintf("querier-%d", i))) + qb.addQuerierConnection(QuerierID(fmt.Sprintf("querier-%d", i))) + } - //Enqueue some stuff so that the tree queue node exists - err = qb.enqueueObjectsForTests(tenantID, 1) - assert.NoError(t, err) - } + // Add tenant queues. + for i := 0; i < numTenants; i++ { + tenantID := TenantID(fmt.Sprintf("tenant-%d", i)) + err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriersPerTenant) + assert.NoError(t, err) - // We expect querier-1 to have some tenants. - querier1Tenants := getTenantsByQuerier(qb, "querier-1") - require.NotEmpty(t, querier1Tenants) + //Enqueue some stuff so that the tree queue node exists + err = qb.enqueueObjectsForTests(tenantID, 1) + assert.NoError(t, err) + } - // Querier-1 abruptly terminates (no shutdown notification received). - qb.removeQuerierConnection("querier-1", now.Add(40*time.Second)) - qb.removeQuerierConnection("querier-1", now.Add(41*time.Second)) + // We expect querier-1 to have some tenants. + querier1Tenants := getTenantsByQuerier(qb, "querier-1") + require.NotEmpty(t, querier1Tenants) - // We expect querier-1 has NOT been removed. - assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) - assert.NoError(t, isConsistent(qb)) + // Querier-1 abruptly terminates (no shutdown notification received). + qb.removeQuerierConnection("querier-1", now.Add(40*time.Second)) + qb.removeQuerierConnection("querier-1", now.Add(41*time.Second)) - // We expect the querier-1 tenants have not been shuffled to other queriers. - for _, tenantID := range querier1Tenants { - assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) - } + // We expect querier-1 has NOT been removed. + assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) + assert.NoError(t, isConsistent(qb)) - // Try to forget disconnected queriers, but querier-1 forget delay hasn't passed yet. - qb.forgetDisconnectedQueriers(now.Add(90 * time.Second)) + // We expect the querier-1 tenants have not been shuffled to other queriers. + for _, tenantID := range querier1Tenants { + assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) + } - // Querier-1 reconnects. - qb.addQuerierConnection("querier-1") - qb.addQuerierConnection("querier-1") + // Try to forget disconnected queriers, but querier-1 forget delay hasn't passed yet. + qb.forgetDisconnectedQueriers(now.Add(90 * time.Second)) - assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) - assert.NoError(t, isConsistent(qb)) + // Querier-1 reconnects. + qb.addQuerierConnection("querier-1") + qb.addQuerierConnection("querier-1") - // We expect the querier-1 tenants have not been shuffled to other queriers. - for _, tenantID := range querier1Tenants { - assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) - } + assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) + assert.NoError(t, isConsistent(qb)) + + // We expect the querier-1 tenants have not been shuffled to other queriers. + for _, tenantID := range querier1Tenants { + assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) + } - // Try to forget disconnected queriers far in the future, but there's no disconnected querier. - qb.forgetDisconnectedQueriers(now.Add(200 * time.Second)) + // Try to forget disconnected queriers far in the future, but there's no disconnected querier. + qb.forgetDisconnectedQueriers(now.Add(200 * time.Second)) - assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) - assert.NoError(t, isConsistent(qb)) + assert.Contains(t, qb.tenantQuerierAssignments.queriersByID, QuerierID("querier-1")) + assert.NoError(t, isConsistent(qb)) - for _, tenantID := range querier1Tenants { - assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) - assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) + for _, tenantID := range querier1Tenants { + assert.Contains(t, getTenantsByQuerier(qb, "querier-1"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-2"), tenantID) + assert.NotContains(t, getTenantsByQuerier(qb, "querier-3"), tenantID) + } + + }) } } + func generateTenant(r *rand.Rand) TenantID { return TenantID(fmt.Sprint("tenant-", r.Int()%5)) } @@ -640,67 +701,45 @@ func getTenantsByQuerier(broker *queueBroker, querierID QuerierID) []TenantID { return tenantIDs } -// getNextTenantForQuerier gets the next tenant in the tenant order assigned to a given querier. -// -// The next tenant for the querier is obtained by rotating through the global tenant order -// starting just after the last tenant the querier received a request for, until a tenant -// is found that is assigned to the given querier according to the querier shuffle sharding. -// A newly connected querier provides lastTenantIndex of -1 in order to start at the beginning. -func (tqa *tenantQuerierAssignments) getNextTenantForQuerier(lastTenantIndex int, querierID QuerierID) (*queueTenant, int, error) { - // check if querier is registered and is not shutting down - if q := tqa.queriersByID[querierID]; q == nil || q.shuttingDown { - return nil, lastTenantIndex, ErrQuerierShuttingDown - } - tenantOrderIndex := lastTenantIndex - for iters := 0; iters < len(tqa.tenantIDOrder); iters++ { - tenantOrderIndex++ - if tenantOrderIndex >= len(tqa.tenantIDOrder) { - // Do not use modulo (e.g. i = (i + 1) % len(slice)) to wrap this index. - // Tenant list can change size between calls and the querier provides its external view - // of the lastTenantIndex it received, which is not updated when this list changes. - // If the tenant list shrinks and the querier-provided lastTenantIndex exceeds the - // length of the tenant list, wrapping via modulo would skip the beginning of the list. - tenantOrderIndex = 0 - } - - tenantID := tqa.tenantIDOrder[tenantOrderIndex] - if tenantID == emptyTenantID { - continue - } - tenant := tqa.tenantsByID[tenantID] - - tenantQuerierSet := tqa.tenantQuerierIDs[tenantID] - if tenantQuerierSet == nil { - // tenant can use all queriers - return tenant, tenantOrderIndex, nil - } else if _, ok := tenantQuerierSet[querierID]; ok { - // tenant is assigned this querier - return tenant, tenantOrderIndex, nil - } - } - - return nil, lastTenantIndex, nil -} - // getOrAddTenantQueue is a test utility, not intended for use by consumers of queueBroker -func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) (*Node, error) { +func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) error { err := qb.tenantQuerierAssignments.createOrUpdateTenant(tenantID, maxQueriers) if err != nil { - return nil, err + return err } - node, err := qb.tree.rootNode.getOrAddNode(qb.makeQueuePathForTests(tenantID), qb.tree) - if err != nil { - return nil, err + // TODO (casie): When deprecating legacy tree queue, clean this up + queuePath := qb.makeQueuePathForTests(tenantID) + if tq, ok := qb.tree.(*TreeQueue); ok { + _, err = tq.getOrAddNode(queuePath) + } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + _, err = itq.rootNode.getOrAddNode(queuePath, itq) } - return node, nil + return err } // removeTenantQueue is a test utility, not intended for use by consumers of queueBroker func (qb *queueBroker) removeTenantQueue(tenantID TenantID) bool { qb.tenantQuerierAssignments.removeTenant(tenantID) queuePath := QueuePath{string(tenantID)} - return qb.tree.rootNode.deleteNode(queuePath) + + // TODO (casie): When deprecating legacy tree queue, clean this up + if tq, ok := qb.tree.(*TreeQueue); ok { + // Normally, emptying out a tenant queue would handle removal from tenantIDOrder, but + // in tests, we sometimes remove a tenant with an existing queue outright, in which case + // we need to update tenantIDOrder and tenantNodes manually. + for i, t := range qb.tenantQuerierAssignments.tenantIDOrder { + if tenantID == t { + qb.tenantQuerierAssignments.tenantIDOrder[i] = emptyTenantID + } + } + return tq.deleteNode(queuePath) + + } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + return itq.rootNode.deleteNode(queuePath) + } + + return false } // deleteNode is a test utility, not intended for use by consumers of Node @@ -741,14 +780,25 @@ func isConsistent(qb *queueBroker) error { for ix, tenantID := range qb.tenantQuerierAssignments.tenantIDOrder { path := qb.makeQueuePathForTests(tenantID) - node := qb.tree.rootNode.getNode(path) - if tenantID != "" && node == nil { - return fmt.Errorf("tenant %s doesn't have queue", tenantID) - } + // TODO (casie): After deprecating legacy tree queue, clean this up + if tq, ok := qb.tree.(*TreeQueue); ok { + node := tq.getNode(path) + if tenantID != "" && node == nil { + return fmt.Errorf("tenant %s doesn't have queue in legacy tree", tenantID) + } + if tenantID == "" && node != nil { + return fmt.Errorf("tenant %s shouldn't have queue in legacy tree", tenantID) + } + } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + node := itq.rootNode.getNode(path) + if tenantID != "" && node == nil { + return fmt.Errorf("tenant %s doesn't have queue", tenantID) + } - if tenantID == "" && node != nil { - return fmt.Errorf("tenant %s shouldn't have queue", tenantID) + if tenantID == "" && node != nil { + return fmt.Errorf("tenant %s shouldn't have queue", tenantID) + } } if tenantID == "" { @@ -779,7 +829,13 @@ func isConsistent(qb *queueBroker) error { } } - tenantQueueCount := qb.tree.rootNode.nodeCount() - 1 + var tenantQueueCount int + // TODO (casie): After deprecating legacy tree queue, clean this up + if tq, ok := qb.tree.(*TreeQueue); ok { + tenantQueueCount = tq.NodeCount() - 1 + } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + tenantQueueCount = itq.rootNode.nodeCount() - 1 + } if tenantQueueCount != tenantCount { return fmt.Errorf("inconsistent number of tenants list and tenant queues") } diff --git a/pkg/scheduler/queue/tree_queue.go b/pkg/scheduler/queue/tree_queue.go index cdda2b6420e..994dabedf78 100644 --- a/pkg/scheduler/queue/tree_queue.go +++ b/pkg/scheduler/queue/tree_queue.go @@ -4,138 +4,40 @@ package queue import ( "container/list" - "fmt" ) -type QueuePath []string //nolint:revive // disallows types beginning with package name -type QueueIndex int //nolint:revive // disallows types beginning with package name - -const localQueueIndex = -1 - -// TreeQueue holds metadata and a pointer to the root node of a hierarchical queue implementation. -// The root Node maintains a localQueue and an arbitrary number of child nodes (which themselves -// may have local queues and children). Each Node in TreeQueue uses a QueuingAlgorithm (determined by -// node depth) to determine dequeue order of that Node's subtree. +// TreeQueue is a hierarchical queue implementation with an arbitrary amount of child queues. // +// TreeQueue internally maintains round-robin fair queuing across all of its queue dimensions. // Each queuing dimension is modeled as a node in the tree, internally reachable through a QueuePath. // -// The QueuePath is an ordered array of strings describing the path from the tree root to a Node. -// In addition to child Nodes, each Node contains a local queue (FIFO) of items. +// The QueuePath is an ordered array of strings describing the path through the tree to the node, +// which contains the FIFO local queue of all items enqueued for that queuing dimension. // -// When dequeuing from a given node, a Node will use its QueuingAlgorithm to choose either itself -// or a child node to dequeue from recursively (i.e., a child Node will use its own QueuingAlgorithm -// to determine how to proceed). TreeQueue will not dequeue from two different Nodes at the same depth -// consecutively, unless the previously-checked Node was empty down to the leaf node. +// When dequeuing from a given node, the node will round-robin equally between dequeuing directly +// from its own local queue and dequeuing recursively from its list of child TreeQueues. +// No queue at a given level of the tree is dequeued from consecutively unless all others +// at the same level of the tree are empty down to the leaf node. type TreeQueue struct { - rootNode *Node - algosByDepth []QueuingAlgorithm + // name of the tree node will be set to its segment of the queue path + name string + localQueue *list.List + currentChildQueueIndex int + childQueueOrder []string + childQueueMap map[string]*TreeQueue } -func NewTree(queuingAlgorithms ...QueuingAlgorithm) (*TreeQueue, error) { - if len(queuingAlgorithms) == 0 { - return nil, fmt.Errorf("cannot create a tree without defined QueuingAlgorithm") - } - root, err := newNode("root", 0, queuingAlgorithms[0]) - if err != nil { - return nil, err - } - root.depth = 0 +func NewTreeQueue(name string) *TreeQueue { return &TreeQueue{ - rootNode: root, - algosByDepth: queuingAlgorithms, - }, nil -} - -func (t *TreeQueue) IsEmpty() bool { - return t.rootNode.IsEmpty() -} - -// Dequeue removes and returns an item from the front of the next appropriate Node in the TreeQueue, as -// well as the path to the Node which that item was dequeued from. -// -// Either the root/self node or a child node is chosen according to the Node's QueuingAlgorithm. If -// the root node is chosen, an item will be dequeued from the front of its localQueue. If a child -// node is chosen, it is recursively dequeued from until a node selects its localQueue. -// -// Nodes that empty down to the leaf after being dequeued from (or which are found to be empty leaf -// nodes during the dequeue operation) are deleted as the recursion returns up the stack. This -// maintains structural guarantees relied upon to make IsEmpty() non-recursive. -func (t *TreeQueue) Dequeue() (QueuePath, any) { - path, v := t.rootNode.dequeue() - // The returned node dequeue path includes the root node; exclude - // this so that the return path can be used if needed to enqueue. - return path[1:], v -} - -// EnqueueBackByPath enqueues an item in the back of the local queue of the node -// located at a given path through the tree; nodes for the path are created as needed. -// -// path is relative to the root node; providing a QueuePath beginning with "root" -// will create a child node of the root node which is also named "root." -func (t *TreeQueue) EnqueueBackByPath(path QueuePath, v any) error { - return t.rootNode.enqueueBackByPath(t, path, v) -} - -// EnqueueFrontByPath enqueues an item in the front of the local queue of the Node -// located at a given path through the TreeQueue; nodes for the path are created as needed. -// -// Enqueueing to the front is intended only for items which were first enqueued to the back -// and then dequeued after reaching the front. -// -// Re-enqueueing to the front is only intended for use in cases where a queue consumer -// fails to complete operations on the dequeued item, but failure is not yet final, and the -// operations should be retried by a subsequent queue consumer. A concrete example is when -// a queue consumer fails or disconnects for unrelated reasons while we are in the process -// of dequeuing a request for it. -// -// path must be relative to the root node; providing a QueuePath beginning with "root" -// will create a child node of root which is also named "root." -func (t *TreeQueue) EnqueueFrontByPath(path QueuePath, v any) error { - return t.rootNode.enqueueFrontByPath(t, path, v) -} - -func (t *TreeQueue) GetNode(path QueuePath) *Node { - return t.rootNode.getNode(path) -} - -// Node maintains node-specific information used to enqueue and dequeue to itself, such as a local -// queue, node depth, references to its children, and position in queue. -// Note that the tenantQuerierAssignments QueuingAlgorithm largely disregards Node's queueOrder and -// queuePosition, managing analogous state instead, because shuffle-sharding + fairness requirements -// necessitate input from the querier. -type Node struct { - name string - localQueue *list.List - queuePosition int // next index in queueOrder to dequeue from - queueOrder []string // order for dequeuing from self/children - queueMap map[string]*Node - depth int - queuingAlgorithm QueuingAlgorithm - childrenChecked int -} - -func newNode(name string, depth int, da QueuingAlgorithm) (*Node, error) { - if da == nil { - return nil, fmt.Errorf("cannot create a node without a defined QueuingAlgorithm") + name: name, + localQueue: nil, + currentChildQueueIndex: localQueueIndex, + childQueueMap: nil, + childQueueOrder: nil, } - if tqa, ok := da.(*tenantQuerierAssignments); ok { - tqa.tenantOrderIndex = localQueueIndex - 1 // start from -2 so that we first check local queue - if tqa.tenantNodes == nil { - tqa.tenantNodes = map[string][]*Node{} - } - } - return &Node{ - name: name, - localQueue: list.New(), - queuePosition: localQueueIndex, - queueOrder: make([]string, 0), - queueMap: make(map[string]*Node, 1), - depth: depth, - queuingAlgorithm: da, - }, nil } -func (n *Node) IsEmpty() bool { +func (q *TreeQueue) IsEmpty() bool { // avoid recursion to make this a cheap operation // // Because we dereference empty child nodes during dequeuing, @@ -145,133 +47,243 @@ func (n *Node) IsEmpty() bool { // In reality a package member could attach empty child queues with getOrAddNode // in order to get a functionally-empty tree that would report false for IsEmpty. // We assume this does not occur or is not relevant during normal operation. - return n.localQueue.Len() == 0 && len(n.queueMap) == 0 + return q.LocalQueueLen() == 0 && len(q.childQueueMap) == 0 } -// ItemCount counts the queue items in the Node and in all its children, recursively. -func (n *Node) ItemCount() int { - items := n.localQueue.Len() - for _, child := range n.queueMap { - items += child.ItemCount() +// NodeCount counts the TreeQueue node and all its children, recursively. +func (q *TreeQueue) NodeCount() int { + count := 1 // count self + for _, childQueue := range q.childQueueMap { + count += childQueue.NodeCount() } - return items + return count } -func (n *Node) Name() string { - return n.name +// ItemCount counts the queue items in the TreeQueue node and in all its children, recursively. +func (q *TreeQueue) ItemCount() int { + count := q.LocalQueueLen() // count self + for _, childQueue := range q.childQueueMap { + count += childQueue.ItemCount() + } + return count } -func (n *Node) getLocalQueue() *list.List { - return n.localQueue +func (q *TreeQueue) LocalQueueLen() int { + localQueueLen := 0 + if q.localQueue != nil { + localQueueLen = q.localQueue.Len() + } + return localQueueLen } -func (n *Node) enqueueFrontByPath(tree *TreeQueue, pathFromNode QueuePath, v any) error { - childNode, err := n.getOrAddNode(pathFromNode, tree) +// EnqueueBackByPath enqueues an item in the back of the local queue of the node +// located at a given path through the tree; nodes for the path are created as needed. +// +// childPath must be relative to the receiver node; providing a QueuePath beginning with +// the receiver/parent node name will create a child node of the same name as the parent. +func (q *TreeQueue) EnqueueBackByPath(childPath QueuePath, v any) error { + childQueue, err := q.getOrAddNode(childPath) if err != nil { return err } - childNode.localQueue.PushFront(v) + + if childQueue.localQueue == nil { + childQueue.localQueue = list.New() + } + childQueue.localQueue.PushBack(v) return nil } -func (n *Node) enqueueBackByPath(tree *TreeQueue, pathFromNode QueuePath, v any) error { - childNode, err := n.getOrAddNode(pathFromNode, tree) +// EnqueueFrontByPath enqueues an item in the front of the local queue of the node +// located at a given path through the tree; nodes for the path are created as needed. +// +// Max queue length check is skipped; enqueueing to the front is intended only for items +// which were first enqueued to the back and then dequeued after reaching the front. +// +// Re-enqueueing to the front is intended for cases where a queue consumer fails to +// complete operations on the dequeued item, but failure is not yet final, and the +// operations should be retried by a subsequent queue consumer. +// +// childPath must be relative to the receiver node; providing a QueuePath beginning with +// the receiver/parent node name will create a child node of the same name as the parent. +func (q *TreeQueue) EnqueueFrontByPath(childPath QueuePath, v any) error { + childQueue, err := q.getOrAddNode(childPath) if err != nil { return err } - childNode.localQueue.PushBack(v) + if childQueue.localQueue == nil { + childQueue.localQueue = list.New() + } + childQueue.localQueue.PushFront(v) return nil } -func (n *Node) dequeue() (QueuePath, any) { - var v any - var childPath QueuePath - - path := QueuePath{n.name} - - if n.IsEmpty() { - return path, nil +// getOrAddNode recursively adds tree queue nodes based on given relative child path. +// +// childPath must be relative to the receiver node; providing a QueuePath beginning with +// the receiver/parent node name will create a child node of the same name as the parent. +func (q *TreeQueue) getOrAddNode(childPath QueuePath) (*TreeQueue, error) { + if len(childPath) == 0 { + return q, nil } - var checkedAllNodes bool - var dequeueNode *Node - // continue until we've found a value or checked all nodes that need checking - for v == nil && !checkedAllNodes { - dequeueNode, checkedAllNodes = n.queuingAlgorithm.dequeueSelectNode(n) - switch dequeueNode { - // dequeuing from local queue - case n: - if n.localQueue.Len() > 0 { - // dequeueNode is self, local queue non-empty - if elt := n.localQueue.Front(); elt != nil { - n.localQueue.Remove(elt) - v = elt.Value - } - } - // no dequeue-able child found; break out of the loop, - // since we won't find anything to dequeue if we don't - // have a node to dequeue from now - case nil: - checkedAllNodes = true - // dequeue from a child - default: - childPath, v = dequeueNode.dequeue() - } + if q.childQueueMap == nil { + q.childQueueMap = make(map[string]*TreeQueue, 1) + } - if v == nil { - n.childrenChecked++ + var childQueue *TreeQueue + var ok bool + if childQueue, ok = q.childQueueMap[childPath[0]]; !ok { + // no child node matches next path segment + // create next child before recurring + childQueue = NewTreeQueue(childPath[0]) + + // add new child queue to ordered list for round-robining; + // in order to maintain round-robin order as nodes are created and deleted, + // the new child queue should be inserted directly before the current child + // queue index, essentially placing the new node at the end of the line + if q.currentChildQueueIndex == localQueueIndex { + // special case; cannot slice into childQueueOrder with index -1 + // place at end of slice, which is the last slot before the local queue slot + q.childQueueOrder = append(q.childQueueOrder, childQueue.name) + } else { + // insert into order behind current child queue index + q.childQueueOrder = append( + q.childQueueOrder[:q.currentChildQueueIndex], + append( + []string{childQueue.name}, + q.childQueueOrder[q.currentChildQueueIndex:]..., + )..., + ) + // update current child queue index to its new place in the expanded slice + q.currentChildQueueIndex++ } - n.queuingAlgorithm.dequeueUpdateState(n, dequeueNode) + // attach new child queue to lookup map + q.childQueueMap[childPath[0]] = childQueue } - // reset childrenChecked to 0 before completing this dequeue - n.childrenChecked = 0 - return append(path, childPath...), v + + return childQueue.getOrAddNode(childPath[1:]) } -func (n *Node) getNode(pathFromNode QueuePath) *Node { - if len(pathFromNode) == 0 { - return n +func (q *TreeQueue) getNode(childPath QueuePath) *TreeQueue { + if len(childPath) == 0 { + return q } - if n.queueMap == nil { + if q.childQueueMap == nil { return nil } - if childQueue, ok := n.queueMap[pathFromNode[0]]; ok { - return childQueue.getNode(pathFromNode[1:]) + if childQueue, ok := q.childQueueMap[childPath[0]]; ok { + return childQueue.getNode(childPath[1:]) } // no child node matches next path segment return nil } -// getOrAddNode recursively gets or adds tree queue nodes based on given relative child path. It -// checks whether the first node in pathFromNode exists in the Node's children; if no node exists, -// one is created and added to the Node's queueOrder, according to the Node's QueuingAlgorithm. +// DequeueByPath selects a child node by a given relative child path and calls Dequeue on the node. // -// pathFromNode must be relative to the receiver node; providing a QueuePath beginning with -// the receiver/parent node name will create a child node of the same name as the parent. -func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *TreeQueue) (*Node, error) { - if len(pathFromNode) == 0 { - return n, nil +// While the child node will recursively clean up its own empty children during dequeue, +// nodes cannot delete themselves; DequeueByPath cleans up the child node as well if it is empty. +// This maintains structural guarantees relied on to make IsEmpty() non-recursive. +// +// childPath is relative to the receiver node; pass a zero-length path to refer to the node itself. +func (q *TreeQueue) DequeueByPath(childPath QueuePath) any { + childQueue := q.getNode(childPath) + if childQueue == nil { + return nil } - var childNode *Node - var ok bool - var err error - if childNode, ok = n.queueMap[pathFromNode[0]]; !ok { - // child does not exist, create it - if n.depth+1 >= len(tree.algosByDepth) { - return nil, fmt.Errorf("cannot add a node beyond max tree depth: %v", len(tree.algosByDepth)) + _, v := childQueue.Dequeue() + + if childQueue.IsEmpty() { + // child node will recursively clean up its own empty children during dequeue, + // but nodes cannot delete themselves; delete the empty child in order to + // maintain structural guarantees relied on to make IsEmpty() non-recursive + q.deleteNode(childPath) + } + + return v +} + +// Dequeue removes and returns an item from the front of the next nonempty queue node in the tree. +// +// Dequeuing from a node follows the round-robin order of the node's childQueueOrder, +// dequeuing either from the node's localQueue or selecting the next child node in the order +// and recursively calling Dequeue on the child nodes until a nonempty queue is found. +// +// Nodes that empty down to the leaf after being dequeued from are deleted as the recursion returns +// up the stack. This maintains structural guarantees relied on to make IsEmpty() non-recursive. +func (q *TreeQueue) Dequeue() (QueuePath, any) { + var childPath QueuePath + var v any + initialLen := len(q.childQueueOrder) + path := QueuePath{q.name} + + for iters := 0; iters <= initialLen && v == nil; iters++ { + if q.currentChildQueueIndex == localQueueIndex { + // dequeuing from local queue; either we have: + // 1. reached a leaf node, or + // 2. reached an inner node when it is the local queue's turn + if q.localQueue != nil { + if elem := q.localQueue.Front(); elem != nil { + q.localQueue.Remove(elem) + v = elem.Value + } + } + q.wrapIndex(true) + } else { + // dequeuing from child queue node; + // pick the child node whose turn it is and recur + childQueueName := q.childQueueOrder[q.currentChildQueueIndex] + childQueue := q.childQueueMap[childQueueName] + childPath, v = childQueue.Dequeue() + + // perform cleanup if child node is empty after dequeuing recursively + if childQueue.IsEmpty() { + // deleteNode wraps index for us + q.deleteNode(QueuePath{childQueueName}) + } else { + q.wrapIndex(true) + } } - childNode, err = newNode(pathFromNode[0], n.depth+1, tree.algosByDepth[n.depth+1]) - if err != nil { - return nil, err + } + return append(path, childPath...), v +} + +// deleteNode removes a child node from the tree and the childQueueOrder and corrects the indices. +func (q *TreeQueue) deleteNode(childPath QueuePath) bool { + if len(childPath) == 0 { + // node cannot delete itself + return false + } + + parentPath, childQueueName := childPath[:len(childPath)-1], childPath[len(childPath)-1] + + parentNode := q.getNode(parentPath) + if parentNode == nil { + // not found + return false + } + + delete(parentNode.childQueueMap, childQueueName) + for i, name := range parentNode.childQueueOrder { + if name == childQueueName { + parentNode.childQueueOrder = append(q.childQueueOrder[:i], q.childQueueOrder[i+1:]...) + parentNode.wrapIndex(false) + break } - // add the newly created child to the node - n.queuingAlgorithm.addChildNode(n, childNode) + } + return true +} +func (q *TreeQueue) wrapIndex(increment bool) { + if increment { + q.currentChildQueueIndex++ + } + if q.currentChildQueueIndex >= len(q.childQueueOrder) { + q.currentChildQueueIndex = localQueueIndex } - return childNode.getOrAddNode(pathFromNode[1:], tree) } diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index 682f1917426..e539c9e8744 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -342,7 +342,7 @@ func Test_DequeueOrderAfterEnqueue(t *testing.T) { }, }, { - name: "should dequeue from new (next-in-queue) tenant-querier child immediately after it is added", + name: "should dequeue from new tenant-querier child before repeat-dequeueing", treeAlgosByDepth: []QueuingAlgorithm{ &tenantQuerierAssignments{ tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, @@ -678,7 +678,7 @@ func Test_DequeueBalancedRoundRobinTree(t *testing.T) { count := 0 - // TreeQueue will fairly dequeue from all levels of the tree + // IntegratedTreeQueue will fairly dequeue from all levels of the tree rotationsBeforeRepeat := len(firstDimensions) * len(secondDimensions) // track dequeued paths to ensure round-robin dequeuing does not repeat before expected dequeuedPathCache := make([]QueuePath, rotationsBeforeRepeat) @@ -848,7 +848,7 @@ func Test_NodeCannotDeleteItself(t *testing.T) { } } -func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimension int) *TreeQueue { +func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimension int) *IntegratedTreeQueue { tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) require.NoError(t, err) require.Equal(t, 1, tree.rootNode.nodeCount()) @@ -873,7 +873,7 @@ func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions return tree } -func makeUnbalancedRoundRobinTree(t *testing.T) *TreeQueue { +func makeUnbalancedRoundRobinTree(t *testing.T) *IntegratedTreeQueue { /* root ├── child0 diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go index a9a7a4419f3..60a30e23b8d 100644 --- a/pkg/scheduler/queue/tree_queueing_algorithms.go +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -3,7 +3,7 @@ package queue // QueuingAlgorithm represents the set of operations specific to different approaches to queuing/dequeuing. It is -// applied at the layer-level -- every Node at the same depth in a TreeQueue shares the same QueuingAlgorithm, +// applied at the layer-level -- every Node at the same depth in a IntegratedTreeQueue shares the same QueuingAlgorithm, // including any state in structs that implement QueuingAlgorithm. type QueuingAlgorithm interface { // addChildNode creates a child Node of parent. QueuePaths passed to enqueue functions are allowed to contain @@ -26,7 +26,7 @@ type QueuingAlgorithm interface { } // roundRobinState is the simplest type of QueuingAlgorithm; nodes which use this QueuingAlgorithm and are at -// the same depth in a TreeQueue do not share any state. When children are added to these nodes, they are placed at +// the same depth in a IntegratedTreeQueue do not share any state. When children are added to these nodes, they are placed at // the "end" of the order from the perspective of the node's current queuePosition (e.g., if queuePosition is 3, // a new child will be placed at index 2). Children are dequeued from using a simple round-robin ordering; // queuePosition is incremented on every dequeue. diff --git a/pkg/scheduler/scheduler.go b/pkg/scheduler/scheduler.go index 83764816411..ec79d33abf8 100644 --- a/pkg/scheduler/scheduler.go +++ b/pkg/scheduler/scheduler.go @@ -95,6 +95,7 @@ type connectedFrontend struct { type Config struct { MaxOutstandingPerTenant int `yaml:"max_outstanding_requests_per_tenant"` AdditionalQueryQueueDimensionsEnabled bool `yaml:"additional_query_queue_dimensions_enabled" category:"experimental"` + UseIntegratedQueryQueue bool `yaml:"use_integrated_query_queue" category:"experimental"` QuerierForgetDelay time.Duration `yaml:"querier_forget_delay" category:"experimental"` GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config" doc:"description=This configures the gRPC client used to report errors back to the query-frontend."` @@ -104,6 +105,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.IntVar(&cfg.MaxOutstandingPerTenant, "query-scheduler.max-outstanding-requests-per-tenant", 100, "Maximum number of outstanding requests per tenant per query-scheduler. In-flight requests above this limit will fail with HTTP response status code 429.") f.BoolVar(&cfg.AdditionalQueryQueueDimensionsEnabled, "query-scheduler.additional-query-queue-dimensions-enabled", false, "Enqueue query requests with additional queue dimensions to split tenant request queues into subqueues. This enables separate requests to proceed from a tenant's subqueues even when other subqueues are blocked on slow query requests. Must be set on both query-frontend and scheduler to take effect. (default false)") + f.BoolVar(&cfg.UseIntegratedQueryQueue, "query-scheduler.use-integrated-query-queue", false, "Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. (default false)") f.DurationVar(&cfg.QuerierForgetDelay, "query-scheduler.querier-forget-delay", 0, "If a querier disconnects without sending notification about graceful shutdown, the query-scheduler will keep the querier in the tenant's shard until the forget delay has passed. This feature is useful to reduce the blast radius when shuffle-sharding is enabled.") cfg.GRPCClientConfig.RegisterFlagsWithPrefix("query-scheduler.grpc-client-config", f) @@ -160,6 +162,7 @@ func NewScheduler(cfg Config, limits Limits, log log.Logger, registerer promethe s.log, cfg.MaxOutstandingPerTenant, cfg.AdditionalQueryQueueDimensionsEnabled, + cfg.UseIntegratedQueryQueue, cfg.QuerierForgetDelay, s.queueLength, s.discardedRequests, From 44a393daad2547bb497778a6e18ffbb265b1d03c Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Tue, 25 Jun 2024 15:42:33 -0700 Subject: [PATCH 22/26] Implement state update fn and bring back TreeQueue-specific tests --- ... => multi_queuing_algorithm_tree_queue.go} | 40 +- ...multi_queuing_algorithm_tree_queue_test.go | 982 +++++++++++++++ pkg/scheduler/queue/queue_test.go | 4 +- .../queue/tenant_querier_assignment.go | 14 +- pkg/scheduler/queue/tenant_queues.go | 15 +- pkg/scheduler/queue/tenant_queues_test.go | 12 +- pkg/scheduler/queue/tree_queue_test.go | 1065 +++++------------ .../queue/tree_queueing_algorithms.go | 4 +- 8 files changed, 1300 insertions(+), 836 deletions(-) rename pkg/scheduler/queue/{integrated_tree_queue.go => multi_queuing_algorithm_tree_queue.go} (84%) create mode 100644 pkg/scheduler/queue/multi_queuing_algorithm_tree_queue_test.go diff --git a/pkg/scheduler/queue/integrated_tree_queue.go b/pkg/scheduler/queue/multi_queuing_algorithm_tree_queue.go similarity index 84% rename from pkg/scheduler/queue/integrated_tree_queue.go rename to pkg/scheduler/queue/multi_queuing_algorithm_tree_queue.go index 67b5a5a83ff..3a14944a23d 100644 --- a/pkg/scheduler/queue/integrated_tree_queue.go +++ b/pkg/scheduler/queue/multi_queuing_algorithm_tree_queue.go @@ -20,9 +20,9 @@ type Tree interface { IsEmpty() bool } -// IntegratedTreeQueue holds metadata and a pointer to the root node of a hierarchical queue implementation. +// MultiQueuingAlgorithmTreeQueue holds metadata and a pointer to the root node of a hierarchical queue implementation. // The root Node maintains a localQueue and an arbitrary number of child nodes (which themselves -// may have local queues and children). Each Node in IntegratedTreeQueue uses a QueuingAlgorithm (determined by +// may have local queues and children). Each Node in MultiQueuingAlgorithmTreeQueue uses a QueuingAlgorithm (determined by // node depth) to determine dequeue order of that Node's subtree. // // Each queuing dimension is modeled as a node in the tree, internally reachable through a QueuePath. @@ -32,14 +32,14 @@ type Tree interface { // // When dequeuing from a given node, a Node will use its QueuingAlgorithm to choose either itself // or a child node to dequeue from recursively (i.e., a child Node will use its own QueuingAlgorithm -// to determine how to proceed). IntegratedTreeQueue will not dequeue from two different Nodes at the same depth +// to determine how to proceed). MultiQueuingAlgorithmTreeQueue will not dequeue from two different Nodes at the same depth // consecutively, unless the previously-checked Node was empty down to the leaf node. -type IntegratedTreeQueue struct { +type MultiQueuingAlgorithmTreeQueue struct { rootNode *Node algosByDepth []QueuingAlgorithm } -func NewTree(queuingAlgorithms ...QueuingAlgorithm) (*IntegratedTreeQueue, error) { +func NewTree(queuingAlgorithms ...QueuingAlgorithm) (*MultiQueuingAlgorithmTreeQueue, error) { if len(queuingAlgorithms) == 0 { return nil, fmt.Errorf("cannot create a tree without defined QueuingAlgorithm") } @@ -48,21 +48,21 @@ func NewTree(queuingAlgorithms ...QueuingAlgorithm) (*IntegratedTreeQueue, error return nil, err } root.depth = 0 - return &IntegratedTreeQueue{ + return &MultiQueuingAlgorithmTreeQueue{ rootNode: root, algosByDepth: queuingAlgorithms, }, nil } -func (t *IntegratedTreeQueue) ItemCount() int { +func (t *MultiQueuingAlgorithmTreeQueue) ItemCount() int { return t.rootNode.ItemCount() } -func (t *IntegratedTreeQueue) IsEmpty() bool { +func (t *MultiQueuingAlgorithmTreeQueue) IsEmpty() bool { return t.rootNode.IsEmpty() } -// Dequeue removes and returns an item from the front of the next appropriate Node in the IntegratedTreeQueue, as +// Dequeue removes and returns an item from the front of the next appropriate Node in the MultiQueuingAlgorithmTreeQueue, as // well as the path to the Node which that item was dequeued from. // // Either the root/self node or a child node is chosen according to the Node's QueuingAlgorithm. If @@ -72,7 +72,7 @@ func (t *IntegratedTreeQueue) IsEmpty() bool { // Nodes that empty down to the leaf after being dequeued from (or which are found to be empty leaf // nodes during the dequeue operation) are deleted as the recursion returns up the stack. This // maintains structural guarantees relied upon to make IsEmpty() non-recursive. -func (t *IntegratedTreeQueue) Dequeue() (QueuePath, any) { +func (t *MultiQueuingAlgorithmTreeQueue) Dequeue() (QueuePath, any) { path, v := t.rootNode.dequeue() // The returned node dequeue path includes the root node; exclude // this so that the return path can be used if needed to enqueue. @@ -84,12 +84,12 @@ func (t *IntegratedTreeQueue) Dequeue() (QueuePath, any) { // // path is relative to the root node; providing a QueuePath beginning with "root" // will create a child node of the root node which is also named "root." -func (t *IntegratedTreeQueue) EnqueueBackByPath(path QueuePath, v any) error { +func (t *MultiQueuingAlgorithmTreeQueue) EnqueueBackByPath(path QueuePath, v any) error { return t.rootNode.enqueueBackByPath(t, path, v) } // EnqueueFrontByPath enqueues an item in the front of the local queue of the Node -// located at a given path through the IntegratedTreeQueue; nodes for the path are created as needed. +// located at a given path through the MultiQueuingAlgorithmTreeQueue; nodes for the path are created as needed. // // Enqueueing to the front is intended only for items which were first enqueued to the back // and then dequeued after reaching the front. @@ -102,11 +102,11 @@ func (t *IntegratedTreeQueue) EnqueueBackByPath(path QueuePath, v any) error { // // path must be relative to the root node; providing a QueuePath beginning with "root" // will create a child node of root which is also named "root." -func (t *IntegratedTreeQueue) EnqueueFrontByPath(path QueuePath, v any) error { +func (t *MultiQueuingAlgorithmTreeQueue) EnqueueFrontByPath(path QueuePath, v any) error { return t.rootNode.enqueueFrontByPath(t, path, v) } -func (t *IntegratedTreeQueue) GetNode(path QueuePath) *Node { +func (t *MultiQueuingAlgorithmTreeQueue) GetNode(path QueuePath) *Node { return t.rootNode.getNode(path) } @@ -130,12 +130,6 @@ func newNode(name string, depth int, da QueuingAlgorithm) (*Node, error) { if da == nil { return nil, fmt.Errorf("cannot create a node without a defined QueuingAlgorithm") } - if tqa, ok := da.(*tenantQuerierAssignments); ok { - tqa.tenantOrderIndex = localQueueIndex - 1 // start from -2 so that we first check local queue - if tqa.tenantNodes == nil { - tqa.tenantNodes = map[string][]*Node{} - } - } return &Node{ name: name, localQueue: list.New(), @@ -177,7 +171,7 @@ func (n *Node) getLocalQueue() *list.List { return n.localQueue } -func (n *Node) enqueueFrontByPath(tree *IntegratedTreeQueue, pathFromNode QueuePath, v any) error { +func (n *Node) enqueueFrontByPath(tree *MultiQueuingAlgorithmTreeQueue, pathFromNode QueuePath, v any) error { childNode, err := n.getOrAddNode(pathFromNode, tree) if err != nil { return err @@ -186,7 +180,7 @@ func (n *Node) enqueueFrontByPath(tree *IntegratedTreeQueue, pathFromNode QueueP return nil } -func (n *Node) enqueueBackByPath(tree *IntegratedTreeQueue, pathFromNode QueuePath, v any) error { +func (n *Node) enqueueBackByPath(tree *MultiQueuingAlgorithmTreeQueue, pathFromNode QueuePath, v any) error { childNode, err := n.getOrAddNode(pathFromNode, tree) if err != nil { return err @@ -264,7 +258,7 @@ func (n *Node) getNode(pathFromNode QueuePath) *Node { // // pathFromNode must be relative to the receiver node; providing a QueuePath beginning with // the receiver/parent node name will create a child node of the same name as the parent. -func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *IntegratedTreeQueue) (*Node, error) { +func (n *Node) getOrAddNode(pathFromNode QueuePath, tree *MultiQueuingAlgorithmTreeQueue) (*Node, error) { if len(pathFromNode) == 0 { return n, nil } diff --git a/pkg/scheduler/queue/multi_queuing_algorithm_tree_queue_test.go b/pkg/scheduler/queue/multi_queuing_algorithm_tree_queue_test.go new file mode 100644 index 00000000000..d0d30a9237c --- /dev/null +++ b/pkg/scheduler/queue/multi_queuing_algorithm_tree_queue_test.go @@ -0,0 +1,982 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package queue + +import ( + "fmt" + "strings" + "testing" + + "github.com/stretchr/testify/require" +) + +// enqueueObj is intended for use in tests; it represents an obj to enqueue to path, or an expected path and obj +// for a given dequeue +type enqueueObj struct { + obj any + path QueuePath +} + +func newTenantQuerierAssignments() *tenantQuerierAssignments { + return &tenantQuerierAssignments{ + tenantQuerierIDs: make(map[TenantID]map[QuerierID]struct{}), + tenantNodes: make(map[string][]*Node), + } +} + +func Test_NewTree(t *testing.T) { + tests := []struct { + name string + treeAlgos []QueuingAlgorithm + state *tenantQuerierAssignments + expectErr bool + }{ + { + name: "create round-robin tree", + treeAlgos: []QueuingAlgorithm{&roundRobinState{}}, + }, + { + name: "create tenant-querier tree", + treeAlgos: []QueuingAlgorithm{&tenantQuerierAssignments{}}, + }, + { + name: "fail to create tree without defined dequeuing algorithm", + treeAlgos: []QueuingAlgorithm{nil}, + expectErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + _, err := NewTree(tt.treeAlgos...) + if tt.expectErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + + }) + } +} + +func Test_EnqueueBackByPath(t *testing.T) { + tests := []struct { + name string + treeAlgosByDepth []QueuingAlgorithm + childPathsToEnqueue []QueuePath + expectErr bool + }{ + { + name: "enqueue tenant-querier node to round-robin node", + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &roundRobinState{}}, + childPathsToEnqueue: []QueuePath{{"child-1"}}, + }, + { + name: "enqueue tenant-querier node to round-robin node", + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, + childPathsToEnqueue: []QueuePath{{"child-1"}, {"child-2"}}, + }, + { + name: "enqueue round-robin node to tenant-querier node", + treeAlgosByDepth: []QueuingAlgorithm{ + newTenantQuerierAssignments(), + &roundRobinState{}, + }, + childPathsToEnqueue: []QueuePath{{"child-1"}, {"child-2"}}, + }, + { + name: "enqueue tenant-querier node to tenant-querier node", + treeAlgosByDepth: []QueuingAlgorithm{ + newTenantQuerierAssignments(), + newTenantQuerierAssignments(), + }, + childPathsToEnqueue: []QueuePath{{"child-1"}}, + }, + { + name: "enqueue grandchildren to a tree with max-depth of 2", + treeAlgosByDepth: []QueuingAlgorithm{ + newTenantQuerierAssignments(), + &roundRobinState{}, + newTenantQuerierAssignments(), + }, + childPathsToEnqueue: []QueuePath{{"child"}, {"grandchild"}}, + }, + { + name: "enqueue beyond max-depth", + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}}, + childPathsToEnqueue: []QueuePath{{"child"}, {"child, grandchild"}}, + expectErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) + + for _, childPath := range tt.childPathsToEnqueue { + err = tree.EnqueueBackByPath(childPath, "some-object") + } + if tt.expectErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + }) + } +} + +func Test_EnqueueFrontByPath(t *testing.T) { + someQuerier := QuerierID("placeholder") + tests := []struct { + name string + treeAlgosByDepth []QueuingAlgorithm + enqueueObjs []enqueueObj + expected []any + }{ + { + name: "enqueue to front of round-robin node", + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}}, + enqueueObjs: []enqueueObj{ + {"query-1", QueuePath{}}, + {"query-2", QueuePath{}}, + }, + expected: []any{"query-2", "query-1"}, + }, + { + name: "enqueue to front of tenant-querier node", + treeAlgosByDepth: []QueuingAlgorithm{&tenantQuerierAssignments{ + currentQuerier: someQuerier, + }}, + enqueueObjs: []enqueueObj{ + {"query-1", QueuePath{}}, + {"query-2", QueuePath{}}, + }, + expected: []any{"query-2", "query-1"}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) + + for _, o := range tt.enqueueObjs { + err = tree.EnqueueFrontByPath(o.path, o.obj) + } + require.NoError(t, err) + + for _, expectedVal := range tt.expected { + _, v := tree.Dequeue() + require.Equal(t, expectedVal, v) + } + _, v := tree.Dequeue() + require.Nil(t, v) + }) + } +} + +func Test_Dequeue_RootNode(t *testing.T) { + tests := []struct { + name string + rootAlgo QueuingAlgorithm + enqueueToRoot []any + }{ + { + name: "dequeue from empty round-robin root node", + rootAlgo: &roundRobinState{}, + }, + { + name: "dequeue from empty tenant-querier root node", + rootAlgo: newTenantQuerierAssignments(), + }, + { + name: "dequeue from non-empty round-robin root node", + rootAlgo: &roundRobinState{}, + enqueueToRoot: []any{"something-in-root"}, + }, + { + name: "dequeue from non-empty tenant-querier root node", + rootAlgo: newTenantQuerierAssignments(), + enqueueToRoot: []any{"something-else-in-root"}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + querierID := QuerierID("placeholder") + switch tt.rootAlgo.(type) { + case *tenantQuerierAssignments: + tt.rootAlgo.(*tenantQuerierAssignments).currentQuerier = querierID + } + tree, err := NewTree(tt.rootAlgo) + require.NoError(t, err) + + path := QueuePath{} + for _, elt := range tt.enqueueToRoot { + err = tree.EnqueueBackByPath(path, elt) + require.NoError(t, err) + } + + for _, elt := range tt.enqueueToRoot { + dequeuePath, v := tree.Dequeue() + require.Equal(t, path, dequeuePath) + require.Equal(t, elt, v) + + } + + dequeuePath, v := tree.Dequeue() + require.Equal(t, path, dequeuePath) + require.Nil(t, v) + + }) + } +} + +func Test_RoundRobinDequeue(t *testing.T) { + tests := []struct { + name string + selfQueueObjects []string + children []string + childQueueObjects map[string][]any + grandchildren []string + grandchildrenQueueObjects map[string]struct { + path QueuePath + objs []any + } + expected []string + }{ + { + name: "dequeue from round-robin child when local queue empty", + children: []string{"child-1"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:some-object"}}, + expected: []string{"child-1:some-object"}, + }, + { + name: "dequeue from round-robin root when on node's turn", + selfQueueObjects: []string{"root:object-1", "root:object-2"}, + children: []string{"child-1"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:object-1"}}, + expected: []string{"root:object-1", "child-1:object-1"}, + }, + { + name: "dequeue from second round-robin child when first child is empty", + children: []string{"child-1", "child-2"}, + childQueueObjects: map[string][]any{"child-1": {nil}, "child-2": {"child-2:some-object"}}, + expected: []string{"child-2:some-object"}, + }, + { + name: "dequeue from round-robin grandchild when non-empty", + children: []string{"child-1", "child-2"}, + childQueueObjects: map[string][]any{"child-1": {"child-1:object-1", "child-1:object-2"}, "child-2": {"child-2:object-1"}}, + expected: []string{"child-1:object-1", "child-2:object-1", "grandchild-1:object-1"}, + grandchildren: []string{"grandchild-1"}, + grandchildrenQueueObjects: map[string]struct { + path QueuePath + objs []any + }{"grandchild-1": { + path: QueuePath{"child-1", "grandchild-1"}, + objs: []any{"grandchild-1:object-1"}, + }}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + + for _, sqo := range tt.selfQueueObjects { + err = tree.EnqueueBackByPath(QueuePath{}, sqo) + require.NoError(t, err) + } + + for _, child := range tt.children { + for _, obj := range tt.childQueueObjects[child] { + _ = tree.EnqueueBackByPath(QueuePath{child}, obj) + } + } + + for _, grandchild := range tt.grandchildren { + gqo := tt.grandchildrenQueueObjects[grandchild] + for _, obj := range gqo.objs { + err = tree.EnqueueBackByPath(gqo.path, obj) + require.NoError(t, err) + } + } + + for _, expected := range tt.expected { + _, val := tree.Dequeue() + v, ok := val.(string) + require.True(t, ok) + require.Equal(t, expected, v) + } + }) + } +} + +func Test_DequeueOrderAfterEnqueue(t *testing.T) { + type opType string + enqueue := opType("enqueue") + dequeue := opType("dequeue") + placeholderQuerier := QuerierID("some-querier") + + type op struct { + kind opType + path QueuePath + obj any + } + + tests := []struct { + name string + treeAlgosByDepth []QueuingAlgorithm + operationOrder []op + }{ + { + name: "round-robin node should dequeue from first child one more time after new node added", + treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &roundRobinState{}}, + operationOrder: []op{ + {enqueue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{}, nil}, + }, + }, + { + name: "should dequeue from new tenant-querier child before repeat-dequeueing", + treeAlgosByDepth: []QueuingAlgorithm{ + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, + tenantNodes: map[string][]*Node{}, + currentQuerier: placeholderQuerier, + }, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, + tenantNodes: map[string][]*Node{}, + currentQuerier: placeholderQuerier, + }, + }, + operationOrder: []op{ + {enqueue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{"child-1"}, "obj-1"}, + {enqueue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{"child-2"}, "obj-3"}, + {dequeue, QueuePath{"child-1"}, "obj-2"}, + {dequeue, QueuePath{}, nil}, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) + + for _, operation := range tt.operationOrder { + if operation.kind == enqueue { + err = tree.EnqueueBackByPath(operation.path, operation.obj) + require.NoError(t, err) + } + if operation.kind == dequeue { + path, obj := tree.Dequeue() + require.Equal(t, operation.path, path) + require.Equal(t, operation.obj, obj) + } + } + }) + } +} + +func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { + tests := []struct { + name string + treeAlgosByDepth []QueuingAlgorithm + currQuerier []QuerierID + enqueueObjs []enqueueObj + expected []any + expectErr bool + }{ + { + name: "happy path - tenant found in tenant-querier map under first child", + treeAlgosByDepth: []QueuingAlgorithm{ + &roundRobinState{}, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + }, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + }, + expected: []any{"query-1"}, + }, + { + name: "tenant exists, but not for querier", + treeAlgosByDepth: []QueuingAlgorithm{ + &roundRobinState{}, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + }, + currQuerier: []QuerierID{"querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + }, + expected: []any{nil}, + }, + { + name: "1 of 3 tenants exist for querier", + treeAlgosByDepth: []QueuingAlgorithm{ + &roundRobinState{}, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-1", "tenant-3"}}, + }, + expected: []any{"query-3", nil}, + }, + { + name: "tenant exists for querier on next parent node", + treeAlgosByDepth: []QueuingAlgorithm{ + &roundRobinState{}, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, + {obj: "query-4", path: QueuePath{"query-component-1", "tenant-3"}}, + }, + expected: []any{"query-4", "query-3", nil}, + }, + { + name: "2 of 3 tenants exist for querier", + treeAlgosByDepth: []QueuingAlgorithm{ + &roundRobinState{}, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, + {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, + {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, + }, + expected: []any{"query-1", "query-3", nil}, + }, + { + name: "root node is tenant-querier node", + treeAlgosByDepth: []QueuingAlgorithm{ + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + &roundRobinState{}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-2", "query-component-1"}}, + {obj: "query-2", path: QueuePath{"tenant-1", "query-component-2"}}, + {obj: "query-3", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{"query-2", "query-3", nil}, + }, + { + name: "dequeuing for one querier returns nil, but does return for a different querier", + treeAlgosByDepth: []QueuingAlgorithm{ + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + &roundRobinState{}, + }, + currQuerier: []QuerierID{"querier-2", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{nil, "query-1"}, + }, + { + name: "no querier set in state", + treeAlgosByDepth: []QueuingAlgorithm{ + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + }, + currQuerier: []QuerierID{""}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + }, + expected: []any{nil}, + }, + { + // This also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) + name: "dequeue from a tenant with a nil tenant-querier map", + treeAlgosByDepth: []QueuingAlgorithm{ + &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}, + tenantNodes: map[string][]*Node{}, + }, + &roundRobinState{}, + &roundRobinState{}, + }, + currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, + enqueueObjs: []enqueueObj{ + {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, + {obj: "query-2", path: QueuePath{"tenant-2", "query-component-1"}}, + {obj: "query-3", path: QueuePath{"tenant-3", "query-component-1"}}, + }, + expected: []any{"query-1", "query-2", "query-3"}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + //currentQuerier := QuerierID("placeholder") + //tt.state.currentQuerier = currentQuerier + // + //// We need a reference to state in order to be able to + //// update the state's currentQuerier. + tqas := make([]*tenantQuerierAssignments, 0) + for _, da := range tt.treeAlgosByDepth { + if tqa, ok := da.(*tenantQuerierAssignments); ok { + tqas = append(tqas, tqa) + } + } + + tree, err := NewTree(tt.treeAlgosByDepth...) + require.NoError(t, err) + + for _, o := range tt.enqueueObjs { + err = tree.EnqueueBackByPath(o.path, o.obj) + require.NoError(t, err) + } + // currQuerier at position i is used to dequeue the expected result at position i + require.Equal(t, len(tt.currQuerier), len(tt.expected)) + for i := 0; i < len(tt.expected); i++ { + for _, tqa := range tqas { + tqa.updateQueuingAlgorithmState(tt.currQuerier[i], i-1) + } + _, v := tree.Dequeue() + require.Equal(t, tt.expected[i], v) + } + }) + } + +} + +// Test_ChangeTenantQuerierAssignments illustrates that we can update a state in tenantQuerierAssignments, +// and the tree dequeue behavior will adjust accordingly. +func Test_ChangeTenantQuerierAssignments(t *testing.T) { + tqa := &tenantQuerierAssignments{ + tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{ + "tenant-1": {"querier-1": {}}, + "tenant-2": {"querier-2": {}}, + }, + tenantNodes: map[string][]*Node{}, + currentQuerier: QuerierID(""), + } + + tree, err := NewTree(tqa, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + + enqueueObjs := []enqueueObj{ + {"query-1", QueuePath{"tenant-1", "query-component-1"}}, + {"query-2", QueuePath{"tenant-2", "query-component-1"}}, + {"query-3", QueuePath{"tenant-2", "query-component-1"}}, + {"query-4", QueuePath{"tenant-2", "query-component-1"}}, + {"query-5", QueuePath{"tenant-3", "query-component-1"}}, + } + + for _, eo := range enqueueObjs { + err = tree.EnqueueBackByPath(eo.path, eo.obj) + require.NoError(t, err) + } + + querier1 := QuerierID("querier-1") + querier2 := QuerierID("querier-2") + querier3 := QuerierID("querier-3") + + // set state to querier-2 should dequeue query-2 + tqa.updateQueuingAlgorithmState(querier2, -1) + _, v := tree.Dequeue() + require.Equal(t, "query-2", v) + + // update tqa to querier-1 should dequeue query-1 + tqa.updateQueuingAlgorithmState(querier1, -1) + _, v = tree.Dequeue() + require.Equal(t, "query-1", v) + + // update tqa map to add querier-3 as assigned to tenant-2, then set tqa to querier-3 should dequeue query-3 + tqa.tenantQuerierIDs["tenant-2"]["querier-3"] = struct{}{} + tqa.updateQueuingAlgorithmState(querier3, -1) + _, v = tree.Dequeue() + require.Equal(t, "query-3", v) + + // during reshuffle, we only ever reassign tenant values, we don't assign an entirely new map value + // to tenantQuerierIDs. Reassign tenant-2 to an empty map value, and query-5 (tenant-3), which can be handled + // by any querier, should be dequeued, + tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} + _, v = tree.Dequeue() + require.Equal(t, "query-5", v) + + // then we should not be able to dequeue query-4 + tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} + _, v = tree.Dequeue() + require.Nil(t, v) + +} + +// Test_DequeueBalancedRoundRobinTree checks dequeuing behavior from a balanced round-robin tree. +// +// Dequeuing from a balanced tree allows the test to have a simple looped structures +// while running checks to ensure that round-robin order is respected. +func Test_DequeueBalancedRoundRobinTree(t *testing.T) { + firstDimensions := []string{"0", "1", "2"} + secondDimensions := []string{"a", "b", "c"} + itemsPerDimension := 5 + tree := makeBalancedRoundRobinTree(t, firstDimensions, secondDimensions, itemsPerDimension) + require.NotNil(t, tree) + + count := 0 + + // MultiQueuingAlgorithmTreeQueue will fairly dequeue from all levels of the tree + rotationsBeforeRepeat := len(firstDimensions) * len(secondDimensions) + // track dequeued paths to ensure round-robin dequeuing does not repeat before expected + dequeuedPathCache := make([]QueuePath, rotationsBeforeRepeat) + + for !tree.IsEmpty() { + dequeuedPath, _ := tree.Dequeue() + + // require dequeued path has not repeated before the expected number of rotations + require.NotContains(t, dequeuedPathCache, dequeuedPath) + + dequeuedPathCache = append(dequeuedPathCache[1:], dequeuedPath) + count++ + } + + // count items enqueued to nodes at depth 1 + expectedFirstDimensionCount := len(firstDimensions) * itemsPerDimension + // count items enqueued to nodes at depth 2 + expectedSecondDimensionCount := len(firstDimensions) * len(secondDimensions) * itemsPerDimension + + require.Equal(t, expectedFirstDimensionCount+expectedSecondDimensionCount, count) + +} + +// Test_DequeueUnbalancedRoundRobinTree checks dequeuing behavior from an unbalanced tree. +// +// Assertions are done one by one to illustrate and check the behaviors of dequeuing from +// an unbalanced tree, where the same node will be dequeued from twice if the node remains +// nonempty while its sibling nodes have been exhausted and deleted from the tree. +func Test_DequeueUnbalancedRoundRobinTree(t *testing.T) { + tree := makeUnbalancedRoundRobinTree(t) + + // dequeue from root until exhausted + _, v := tree.Dequeue() + require.Equal(t, "root:0:val0", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:1:val0", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:2:0:val0", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:1:0:val0", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:2:1:val0", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:1:val1", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:2:0:val1", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:1:0:val1", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:2:1:val1", v) + + _, v = tree.Dequeue() + require.Equal(t, "root:2:1:val2", v) + + // all items have been dequeued + require.Equal(t, 0, tree.rootNode.ItemCount()) + require.Equal(t, 1, tree.rootNode.nodeCount()) + + // require nothing in local or child queues + require.True(t, tree.IsEmpty()) +} + +func Test_EnqueueDuringDequeueRespectsRoundRobin(t *testing.T) { + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + require.NotNil(t, tree) + + root := tree.rootNode + + cache := map[string]struct{}{} + + // enqueue two items to path root:0 + childPath := QueuePath{"0"} + item := makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item = makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + + // enqueue one item to path root:1 + childPath = QueuePath{"1"} + item = makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + + // enqueue two items to path root:2 + childPath = QueuePath{"2"} + item = makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item = makeItemForChildQueue(root, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + + require.Equal(t, []string{"0", "1", "2"}, root.queueOrder) + + // dequeue first item + dequeuedPath, _ := tree.Dequeue() + require.Equal(t, QueuePath{"0"}, dequeuedPath) + + // dequeue second item; root:1 is now exhausted and deleted + dequeuedPath, _ = tree.Dequeue() + require.Equal(t, QueuePath{"1"}, dequeuedPath) + require.Nil(t, root.getNode(QueuePath{"1"})) + require.Equal(t, []string{"0", "2"}, root.queueOrder) + + // dequeue third item + dequeuedPath, _ = tree.Dequeue() + require.Equal(t, QueuePath{"2"}, dequeuedPath) + + // root:1 was previously exhausted; root:0, then root:2 will be next in the rotation + // here we insert something new into root:1 to test that it + // does not jump the line in front of root:0 or root:2 + item = makeItemForChildQueue(root, QueuePath{"1"}, cache) + require.NoError(t, tree.EnqueueBackByPath(QueuePath{"1"}, item)) + require.NotNil(t, root.getNode(QueuePath{"1"})) + require.Equal(t, []string{"0", "2", "1"}, root.queueOrder) + + // dequeue fourth item; the newly-enqueued root:1 item + // has not jumped the line in front of root:0 + dequeuedPath, _ = tree.Dequeue() + require.Equal(t, QueuePath{"0"}, dequeuedPath) + + // dequeue fifth item; the newly-enqueued root:1 item + // has not jumped the line in front of root:2 + dequeuedPath, _ = tree.Dequeue() + require.Equal(t, QueuePath{"2"}, dequeuedPath) + + // dequeue sixth item; verifying the order 0->2->1 is being followed + dequeuedPath, _ = tree.Dequeue() + require.Equal(t, QueuePath{"1"}, dequeuedPath) + + // all items have been dequeued + require.Equal(t, 0, root.ItemCount()) + require.Equal(t, 1, root.nodeCount()) + + // require nothing in local or child queues + require.True(t, tree.IsEmpty()) +} + +// Test_NodeCannotDeleteItself creates an empty node, dequeues from it, and ensures that the node still exists +// and has not deleted itself. +func Test_NodeCannotDeleteItself(t *testing.T) { + tests := []struct { + name string + nodeType QueuingAlgorithm + }{ + {"round robin", &roundRobinState{}}, + {"tenant querier assignment", &tenantQuerierAssignments{}}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tree, err := NewTree(tt.nodeType) + require.NoError(t, err) + require.NotNil(t, tree) + + _, _ = tree.Dequeue() + + require.NotNil(t, tree.rootNode) + require.Zero(t, tree.rootNode.getLocalQueue().Len()) + require.Empty(t, tree.rootNode.queueMap) + }) + } +} + +func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimension int) *MultiQueuingAlgorithmTreeQueue { + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + require.Equal(t, 1, tree.rootNode.nodeCount()) + require.Equal(t, 0, tree.rootNode.ItemCount()) + + cache := map[string]struct{}{} + + for _, firstDimName := range firstDimensions { + for k := 0; k < itemsPerDimension; k++ { + childPath := QueuePath{firstDimName} + item := makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + } + for _, secondDimName := range secondDimensions { + for k := 0; k < itemsPerDimension; k++ { + childPath := QueuePath{firstDimName, secondDimName} + item := makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + } + } + } + return tree +} + +func makeUnbalancedRoundRobinTree(t *testing.T) *MultiQueuingAlgorithmTreeQueue { + /* + root + ├── child0 + │ └── localQueue + │ └── val0 + ├── child1 + │ ├── child0 + │ │ └── localQueue + │ │ ├── val0 + │ │ └── val1 + │ └── localQueue + │ ├── val0 + │ └── val1 + ├── child2 + │ ├── child0 + │ │ └── localQueue + │ │ ├── val0 + │ │ └── val1 + │ ├── child1 + │ │ └── localQueue + │ │ ├── val0 + │ │ ├── val1 + │ │ └── val2 + │ └── localQueue + └── localQueue + */ + tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) + require.NoError(t, err) + require.Equal(t, 1, tree.rootNode.nodeCount()) + require.Equal(t, 0, tree.rootNode.ItemCount()) + + cache := map[string]struct{}{} + + // enqueue one item to root:0 + childPath := QueuePath{"0"} + item := makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 2, tree.rootNode.nodeCount()) + require.Equal(t, 1, tree.rootNode.ItemCount()) + + // enqueue two items to root:1 + childPath = QueuePath{"1"} + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 3, tree.rootNode.nodeCount()) + require.Equal(t, 2, tree.rootNode.ItemCount()) + + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 3, tree.rootNode.nodeCount()) + require.Equal(t, 3, tree.rootNode.ItemCount()) + + // enqueue two items to root:1:0 + childPath = QueuePath{"1", "0"} + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 4, tree.rootNode.nodeCount()) + require.Equal(t, 4, tree.rootNode.ItemCount()) + + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 4, tree.rootNode.nodeCount()) + require.Equal(t, 5, tree.rootNode.ItemCount()) + + // enqueue two items to root:2:0 + childPath = QueuePath{"2", "0"} + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 6, tree.rootNode.nodeCount()) + require.Equal(t, 6, tree.rootNode.ItemCount()) + + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 6, tree.rootNode.nodeCount()) + require.Equal(t, 7, tree.rootNode.ItemCount()) + + // enqueue three items to root:2:1 + childPath = QueuePath{"2", "1"} + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, tree.rootNode.nodeCount()) + require.Equal(t, 8, tree.rootNode.ItemCount()) + + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, tree.rootNode.nodeCount()) + require.Equal(t, 9, tree.rootNode.ItemCount()) + + item = makeItemForChildQueue(tree.rootNode, childPath, cache) + require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, tree.rootNode.nodeCount()) + require.Equal(t, 10, tree.rootNode.ItemCount()) + + return tree +} + +// makeItemForChildQueue constructs a queue item to match its enqueued path +// by joining the path components and appending an incrementing value for each path. +// +// e.g. for a tree named "root": +// - childQueuePath{"1", "0"}'s first item will be "root:1:0:val0" +// - childQueuePath{"1", "0"}'s second item will be "root:1:0:val1" +func makeItemForChildQueue( + parent *Node, childPath QueuePath, cache map[string]struct{}, +) string { + path := append(QueuePath{parent.name}, childPath...) + + i := 0 + for { + item := strings.Join(path, ":") + fmt.Sprintf(":val%d", i) + if _, ok := cache[item]; !ok { + cache[item] = struct{}{} + return item + } + i++ + } +} diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index 219fe0fac97..290707b46c4 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -734,7 +734,7 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend require.Nil(t, tq.getNode(QueuePath{"tenant-1"})) require.NoError(t, queueBroker.enqueueRequestBack(&tr, tenantMaxQueriers)) require.False(t, tq.getNode(QueuePath{"tenant-1"}).IsEmpty()) - } else if itq, ok := queueBroker.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := queueBroker.tree.(*MultiQueuingAlgorithmTreeQueue); ok { require.Nil(t, itq.GetNode(QueuePath{"tenant-1"})) require.NoError(t, queueBroker.enqueueRequestBack(&tr, tenantMaxQueriers)) require.False(t, itq.GetNode(QueuePath{"tenant-1"}).IsEmpty()) @@ -756,7 +756,7 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend // TODO (casie): Clean this up when deprecating legacy tree queue if tq, ok := queueBroker.tree.(*TreeQueue); ok { require.False(t, tq.getNode(QueuePath{"tenant-1"}).IsEmpty()) - } else if itq, ok := queueBroker.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := queueBroker.tree.(*MultiQueuingAlgorithmTreeQueue); ok { require.False(t, itq.GetNode(QueuePath{"tenant-1"}).IsEmpty()) } diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 3665ef038d3..7ee29266c5d 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -53,7 +53,7 @@ func (s querierIDSlice) Search(x QuerierID) int { return sort.Search(len(s), func(i int) bool { return s[i] >= x }) } -// tenantQuerierAssignments implements QueuingAlgorithm. In the context of a IntegratedTreeQueue, it maintains a mapping of +// tenantQuerierAssignments implements QueuingAlgorithm. In the context of a MultiQueuingAlgorithmTreeQueue, it maintains a mapping of // tenants to queriers in order to support dequeuing from an appropriate tenant if shuffle-sharding is enabled. type tenantQuerierAssignments struct { // a tenant has many queriers @@ -93,7 +93,7 @@ type tenantQuerierAssignments struct { // If tenant querier ID set is not nil, only those queriers can handle the tenant's requests, // Tenant querier ID is set to nil if sharding is off or available queriers <= tenant's maxQueriers. tenantQuerierIDs map[TenantID]map[QuerierID]struct{} - currentQuerier *QuerierID + currentQuerier QuerierID } // getNextTenantForQuerier gets the next tenant in the tenant order assigned to a given querier. @@ -367,7 +367,7 @@ func (tqa *tenantQuerierAssignments) shuffleTenantQueriers(tenantID TenantID, sc // ignore each Node's individual queueOrder and queuePosition. func (tqa *tenantQuerierAssignments) dequeueSelectNode(node *Node) (*Node, bool) { // can't get a tenant if no querier set - if tqa.currentQuerier == nil { + if tqa.currentQuerier == "" { return nil, true } @@ -413,7 +413,7 @@ func (tqa *tenantQuerierAssignments) dequeueSelectNode(node *Node) (*Node, bool) } // otherwise, check if the querier is assigned to this tenant if tenantQuerierSet, ok := tqa.tenantQuerierIDs[tenantID]; ok { - if _, ok := tenantQuerierSet[*tqa.currentQuerier]; ok { + if _, ok := tenantQuerierSet[tqa.currentQuerier]; ok { tqa.tenantOrderIndex = checkIndex return node.queueMap[tenantName], checkedAllNodes } @@ -504,5 +504,11 @@ func (tqa *tenantQuerierAssignments) addChildNode(parent, child *Node) { } // if we get here, we didn't find any empty elements in tenantIDOrder; append tqa.tenantIDOrder = append(tqa.tenantIDOrder, TenantID(childName)) +} +// updateQueuingAlgorithmState should be called before attempting to dequeue, and updates inputs required by this +// QueuingAlgorithm to dequeue the appropriate value for the given querier. +func (tqa *tenantQuerierAssignments) updateQueuingAlgorithmState(querierID QuerierID, tenantOrderIndex int) { + tqa.currentQuerier = querierID + tqa.tenantOrderIndex = tenantOrderIndex } diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 7794bd02cd5..86e36955085 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -47,7 +47,8 @@ func newQueueBroker( tenantIDOrder: nil, tenantsByID: map[TenantID]*queueTenant{}, tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, - currentQuerier: ¤tQuerier, + tenantNodes: map[string][]*Node{}, + currentQuerier: currentQuerier, tenantOrderIndex: localQueueIndex, } @@ -101,7 +102,7 @@ func (qb *queueBroker) enqueueRequestBack(request *tenantRequest, tenantMaxQueri } // TODO (casie): When deprecating TreeQueue, clean this up. - // Technically, the IntegratedTreeQueue approach is adequate for both tree types, but we are temporarily + // Technically, the MultiQueuingAlgorithmTreeQueue approach is adequate for both tree types, but we are temporarily // maintaining the legacy tree behavior as much as possible for stability reasons. if tq, ok := qb.tree.(*TreeQueue); ok { if tenantQueueNode := tq.getNode(queuePath[:1]); tenantQueueNode != nil { @@ -109,7 +110,7 @@ func (qb *queueBroker) enqueueRequestBack(request *tenantRequest, tenantMaxQueri return ErrTooManyRequests } } - } else if _, ok := qb.tree.(*IntegratedTreeQueue); ok { + } else if _, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { itemCount := 0 for _, tenantNode := range qb.tenantQuerierAssignments.tenantNodes[string(request.tenantID)] { itemCount += tenantNode.ItemCount() @@ -179,10 +180,8 @@ func (qb *queueBroker) dequeueRequestForQuerier( qb.tenantQuerierAssignments.tenantOrderIndex = tenantIndex queuePath = QueuePath{string(tenant.tenantID)} queueElement = tq.DequeueByPath(queuePath) - } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { - qb.tenantQuerierAssignments.currentQuerier = &querierID - qb.tenantQuerierAssignments.tenantOrderIndex = lastTenantIndex - + } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { + qb.tenantQuerierAssignments.updateQueuingAlgorithmState(querierID, lastTenantIndex) queuePath, queueElement = itq.Dequeue() } @@ -209,7 +208,7 @@ func (qb *queueBroker) dequeueRequestForQuerier( // queue node was deleted due to being empty after dequeue qb.tenantQuerierAssignments.removeTenant(tenant.tenantID) } - } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { queueNodeAfterDequeue := itq.GetNode(queuePath) if queueNodeAfterDequeue == nil && len(qb.tenantQuerierAssignments.tenantNodes[string(tenantID)]) == 0 { // queue node was deleted due to being empty after dequeue diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index d1630866e10..7ba974ebdd6 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -240,7 +240,7 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { // TODO (casie): After deprecating legacy tree queue, clean this up if tq, ok := qb.tree.(*TreeQueue); ok { assert.Equal(t, maxTenantQueueSize, tq.getNode(queuePath).ItemCount()) - } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { assert.Equal(t, maxTenantQueueSize, itq.GetNode(queuePath).ItemCount()) } @@ -251,7 +251,7 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { var itemCount int if tq, ok := qb.tree.(*TreeQueue); ok { itemCount = tq.getNode(queuePath).LocalQueueLen() - } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { itemCount = itq.GetNode(queuePath).getLocalQueue().Len() } assert.Equal(t, maxTenantQueueSize/len(additionalQueueDimensions), itemCount) @@ -712,7 +712,7 @@ func (qb *queueBroker) getOrAddTenantQueue(tenantID TenantID, maxQueriers int) e queuePath := qb.makeQueuePathForTests(tenantID) if tq, ok := qb.tree.(*TreeQueue); ok { _, err = tq.getOrAddNode(queuePath) - } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { _, err = itq.rootNode.getOrAddNode(queuePath, itq) } return err @@ -735,7 +735,7 @@ func (qb *queueBroker) removeTenantQueue(tenantID TenantID) bool { } return tq.deleteNode(queuePath) - } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { return itq.rootNode.deleteNode(queuePath) } @@ -790,7 +790,7 @@ func isConsistent(qb *queueBroker) error { if tenantID == "" && node != nil { return fmt.Errorf("tenant %s shouldn't have queue in legacy tree", tenantID) } - } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { node := itq.rootNode.getNode(path) if tenantID != "" && node == nil { return fmt.Errorf("tenant %s doesn't have queue", tenantID) @@ -833,7 +833,7 @@ func isConsistent(qb *queueBroker) error { // TODO (casie): After deprecating legacy tree queue, clean this up if tq, ok := qb.tree.(*TreeQueue); ok { tenantQueueCount = tq.NodeCount() - 1 - } else if itq, ok := qb.tree.(*IntegratedTreeQueue); ok { + } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { tenantQueueCount = itq.rootNode.nodeCount() - 1 } if tenantQueueCount != tenantCount { diff --git a/pkg/scheduler/queue/tree_queue_test.go b/pkg/scheduler/queue/tree_queue_test.go index e539c9e8744..0d333c39a62 100644 --- a/pkg/scheduler/queue/tree_queue_test.go +++ b/pkg/scheduler/queue/tree_queue_test.go @@ -10,687 +10,76 @@ import ( "github.com/stretchr/testify/require" ) -// enqueueObj is intended for use in tests; it represents an obj to enqueue to path, or an expected path and obj -// for a given dequeue -type enqueueObj struct { - obj any - path QueuePath -} - -func newTenantQuerierAssignments() *tenantQuerierAssignments { - return &tenantQuerierAssignments{ - tenantQuerierIDs: make(map[TenantID]map[QuerierID]struct{}), - tenantNodes: make(map[string][]*Node), - } -} - -func Test_NewTree(t *testing.T) { - tests := []struct { - name string - treeAlgos []QueuingAlgorithm - state *tenantQuerierAssignments - expectErr bool - }{ - { - name: "create round-robin tree", - treeAlgos: []QueuingAlgorithm{&roundRobinState{}}, - }, - { - name: "create tenant-querier tree", - treeAlgos: []QueuingAlgorithm{&tenantQuerierAssignments{}}, - }, - { - name: "fail to create tree without defined dequeuing algorithm", - treeAlgos: []QueuingAlgorithm{nil}, - expectErr: true, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - _, err := NewTree(tt.treeAlgos...) - if tt.expectErr { - require.Error(t, err) - } else { - require.NoError(t, err) - } - - }) - } -} - -func Test_EnqueueBackByPath(t *testing.T) { - tests := []struct { - name string - treeAlgosByDepth []QueuingAlgorithm - childPathsToEnqueue []QueuePath - expectErr bool - }{ - { - name: "enqueue tenant-querier node to round-robin node", - treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &roundRobinState{}}, - childPathsToEnqueue: []QueuePath{{"child-1"}}, - }, - { - name: "enqueue tenant-querier node to round-robin node", - treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}}, - childPathsToEnqueue: []QueuePath{{"child-1"}, {"child-2"}}, - }, - { - name: "enqueue round-robin node to tenant-querier node", - treeAlgosByDepth: []QueuingAlgorithm{ - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - childPathsToEnqueue: []QueuePath{{"child-1"}, {"child-2"}}, - }, - { - name: "enqueue tenant-querier node to tenant-querier node", - treeAlgosByDepth: []QueuingAlgorithm{ - newTenantQuerierAssignments(), - newTenantQuerierAssignments(), - }, - childPathsToEnqueue: []QueuePath{{"child-1"}}, - }, - { - name: "enqueue grandchildren to a tree with max-depth of 2", - treeAlgosByDepth: []QueuingAlgorithm{ - newTenantQuerierAssignments(), - &roundRobinState{}, - newTenantQuerierAssignments(), - }, - childPathsToEnqueue: []QueuePath{{"child"}, {"grandchild"}}, - }, - { - name: "enqueue beyond max-depth", - treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}}, - childPathsToEnqueue: []QueuePath{{"child"}, {"child, grandchild"}}, - expectErr: true, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.treeAlgosByDepth...) - require.NoError(t, err) - - for _, childPath := range tt.childPathsToEnqueue { - err = tree.EnqueueBackByPath(childPath, "some-object") - } - if tt.expectErr { - require.Error(t, err) - } else { - require.NoError(t, err) - } - }) - } -} - -func Test_EnqueueFrontByPath(t *testing.T) { - someQuerier := QuerierID("placeholder") - tests := []struct { - name string - treeAlgosByDepth []QueuingAlgorithm - enqueueObjs []enqueueObj - expected []any - }{ - { - name: "enqueue to front of round-robin node", - treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}}, - enqueueObjs: []enqueueObj{ - {"query-1", QueuePath{}}, - {"query-2", QueuePath{}}, - }, - expected: []any{"query-2", "query-1"}, - }, - { - name: "enqueue to front of tenant-querier node", - treeAlgosByDepth: []QueuingAlgorithm{&tenantQuerierAssignments{ - currentQuerier: &someQuerier, - }}, - enqueueObjs: []enqueueObj{ - {"query-1", QueuePath{}}, - {"query-2", QueuePath{}}, - }, - expected: []any{"query-2", "query-1"}, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.treeAlgosByDepth...) - require.NoError(t, err) +// TODO (casie): When deprecating TreeQueue, it's safe to delete this file wholesale; all relevant test +// functionality has been duplicated for MultiQueuingAlgorithmTreeQueue. - for _, o := range tt.enqueueObjs { - err = tree.EnqueueFrontByPath(o.path, o.obj) - } - require.NoError(t, err) - - for _, expectedVal := range tt.expected { - _, v := tree.Dequeue() - require.Equal(t, expectedVal, v) - } - _, v := tree.Dequeue() - require.Nil(t, v) - }) - } -} - -func Test_Dequeue_RootNode(t *testing.T) { - tests := []struct { - name string - rootAlgo QueuingAlgorithm - enqueueToRoot []any - }{ - { - name: "dequeue from empty round-robin root node", - rootAlgo: &roundRobinState{}, - }, - { - name: "dequeue from empty tenant-querier root node", - rootAlgo: newTenantQuerierAssignments(), - }, - { - name: "dequeue from non-empty round-robin root node", - rootAlgo: &roundRobinState{}, - enqueueToRoot: []any{"something-in-root"}, - }, - { - name: "dequeue from non-empty tenant-querier root node", - rootAlgo: newTenantQuerierAssignments(), - enqueueToRoot: []any{"something-else-in-root"}, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - querierID := QuerierID("placeholder") - switch tt.rootAlgo.(type) { - case *tenantQuerierAssignments: - tt.rootAlgo.(*tenantQuerierAssignments).currentQuerier = &querierID - } - tree, err := NewTree(tt.rootAlgo) - require.NoError(t, err) - - path := QueuePath{} - for _, elt := range tt.enqueueToRoot { - err = tree.EnqueueBackByPath(path, elt) - require.NoError(t, err) - } - - for _, elt := range tt.enqueueToRoot { - dequeuePath, v := tree.Dequeue() - require.Equal(t, path, dequeuePath) - require.Equal(t, elt, v) - - } - - dequeuePath, v := tree.Dequeue() - require.Equal(t, path, dequeuePath) - require.Nil(t, v) - - }) - } -} - -func Test_RoundRobinDequeue(t *testing.T) { - tests := []struct { - name string - selfQueueObjects []string - children []string - childQueueObjects map[string][]any - grandchildren []string - grandchildrenQueueObjects map[string]struct { - path QueuePath - objs []any - } - expected []string - }{ - { - name: "dequeue from round-robin child when local queue empty", - children: []string{"child-1"}, - childQueueObjects: map[string][]any{"child-1": {"child-1:some-object"}}, - expected: []string{"child-1:some-object"}, - }, - { - name: "dequeue from round-robin root when on node's turn", - selfQueueObjects: []string{"root:object-1", "root:object-2"}, - children: []string{"child-1"}, - childQueueObjects: map[string][]any{"child-1": {"child-1:object-1"}}, - expected: []string{"root:object-1", "child-1:object-1"}, - }, - { - name: "dequeue from second round-robin child when first child is empty", - children: []string{"child-1", "child-2"}, - childQueueObjects: map[string][]any{"child-1": {nil}, "child-2": {"child-2:some-object"}}, - expected: []string{"child-2:some-object"}, - }, - { - name: "dequeue from round-robin grandchild when non-empty", - children: []string{"child-1", "child-2"}, - childQueueObjects: map[string][]any{"child-1": {"child-1:object-1", "child-1:object-2"}, "child-2": {"child-2:object-1"}}, - expected: []string{"child-1:object-1", "child-2:object-1", "grandchild-1:object-1"}, - grandchildren: []string{"grandchild-1"}, - grandchildrenQueueObjects: map[string]struct { - path QueuePath - objs []any - }{"grandchild-1": { - path: QueuePath{"child-1", "grandchild-1"}, - objs: []any{"grandchild-1:object-1"}, - }}, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) - require.NoError(t, err) - - for _, sqo := range tt.selfQueueObjects { - err = tree.EnqueueBackByPath(QueuePath{}, sqo) - require.NoError(t, err) - } - - for _, child := range tt.children { - for _, obj := range tt.childQueueObjects[child] { - _ = tree.EnqueueBackByPath(QueuePath{child}, obj) - } - } - - for _, grandchild := range tt.grandchildren { - gqo := tt.grandchildrenQueueObjects[grandchild] - for _, obj := range gqo.objs { - err = tree.EnqueueBackByPath(gqo.path, obj) - require.NoError(t, err) - } - } - - for _, expected := range tt.expected { - _, val := tree.Dequeue() - v, ok := val.(string) - require.True(t, ok) - require.Equal(t, expected, v) - } - }) - } -} - -func Test_DequeueOrderAfterEnqueue(t *testing.T) { - type opType string - enqueue := opType("enqueue") - dequeue := opType("dequeue") - placeholderQuerier := QuerierID("") - - type op struct { - kind opType - path QueuePath - obj any - } - - tests := []struct { - name string - treeAlgosByDepth []QueuingAlgorithm - operationOrder []op - }{ - { - name: "round-robin node should dequeue from first child one more time after new node added", - treeAlgosByDepth: []QueuingAlgorithm{&roundRobinState{}, &roundRobinState{}}, - operationOrder: []op{ - {enqueue, QueuePath{"child-1"}, "obj-1"}, - {enqueue, QueuePath{"child-1"}, "obj-2"}, - {dequeue, QueuePath{"child-1"}, "obj-1"}, - {enqueue, QueuePath{"child-2"}, "obj-3"}, - {dequeue, QueuePath{"child-1"}, "obj-2"}, - {dequeue, QueuePath{"child-2"}, "obj-3"}, - {dequeue, QueuePath{}, nil}, - }, - }, - { - name: "should dequeue from new tenant-querier child before repeat-dequeueing", - treeAlgosByDepth: []QueuingAlgorithm{ - &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, - tenantNodes: map[string][]*Node{}, - currentQuerier: &placeholderQuerier, - }, - &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}, - tenantNodes: map[string][]*Node{}, - currentQuerier: &placeholderQuerier, - }, - }, - operationOrder: []op{ - {enqueue, QueuePath{"child-1"}, "obj-1"}, - {enqueue, QueuePath{"child-1"}, "obj-2"}, - {dequeue, QueuePath{"child-1"}, "obj-1"}, - {enqueue, QueuePath{"child-2"}, "obj-3"}, - {dequeue, QueuePath{"child-2"}, "obj-3"}, - {dequeue, QueuePath{"child-1"}, "obj-2"}, - {dequeue, QueuePath{}, nil}, - }, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.treeAlgosByDepth...) - require.NoError(t, err) - - for _, operation := range tt.operationOrder { - if operation.kind == enqueue { - err = tree.EnqueueBackByPath(operation.path, operation.obj) - require.NoError(t, err) - } - if operation.kind == dequeue { - path, obj := tree.Dequeue() - require.Equal(t, operation.path, path) - require.Equal(t, operation.obj, obj) - } - } - }) - } -} - -func Test_TenantQuerierAssignmentsDequeue(t *testing.T) { - tests := []struct { - name string - treeAlgosByDepth []QueuingAlgorithm - state *tenantQuerierAssignments - currQuerier []QuerierID - enqueueObjs []enqueueObj - expected []any - expectErr bool - }{ - { - name: "happy path - tenant found in tenant-querier map under first child", - treeAlgosByDepth: []QueuingAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - }, - expected: []any{"query-1"}, - }, - { - name: "tenant exists, but not for querier", - treeAlgosByDepth: []QueuingAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - }, - expected: []any{nil}, - }, - { - name: "1 of 3 tenants exist for querier", - treeAlgosByDepth: []QueuingAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, - {obj: "query-3", path: QueuePath{"query-component-1", "tenant-3"}}, - }, - expected: []any{"query-3", nil}, - }, - { - name: "tenant exists for querier on next parent node", - treeAlgosByDepth: []QueuingAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-2": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, - {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, - {obj: "query-4", path: QueuePath{"query-component-1", "tenant-3"}}, - }, - expected: []any{"query-4", "query-3", nil}, - }, - { - name: "2 of 3 tenants exist for querier", - treeAlgosByDepth: []QueuingAlgorithm{ - &roundRobinState{}, - newTenantQuerierAssignments(), - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {"querier-2": {}}, "tenant-3": {"querier-1": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"query-component-1", "tenant-1"}}, - {obj: "query-2", path: QueuePath{"query-component-1", "tenant-2"}}, - {obj: "query-3", path: QueuePath{"query-component-2", "tenant-3"}}, - }, - expected: []any{"query-1", "query-3", nil}, - }, - { - name: "root node is tenant-querier node", - treeAlgosByDepth: []QueuingAlgorithm{ - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &roundRobinState{}, - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": {}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-2", "query-component-1"}}, - {obj: "query-2", path: QueuePath{"tenant-1", "query-component-2"}}, - {obj: "query-3", path: QueuePath{"tenant-1", "query-component-1"}}, - }, - expected: []any{"query-2", "query-3", nil}, - }, - { - name: "dequeuing for one querier returns nil, but does return for a different querier", - treeAlgosByDepth: []QueuingAlgorithm{ - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &roundRobinState{}, - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-2", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, - }, - expected: []any{nil, "query-1"}, - }, - { - name: "no querier set in state", - treeAlgosByDepth: []QueuingAlgorithm{ - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{""}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, - }, - expected: []any{nil}, - }, - { - // This also dequeues if the tenant _is not_ in the tenant querier map; is this expected? (probably) - name: "dequeue from a tenant with a nil tenant-querier map", - treeAlgosByDepth: []QueuingAlgorithm{ - &tenantQuerierAssignments{tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{}}, - &roundRobinState{}, - &roundRobinState{}, - }, - state: &tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{"tenant-1": {"querier-1": {}}, "tenant-2": nil}, - tenantNodes: map[string][]*Node{}, - }, - currQuerier: []QuerierID{"querier-1", "querier-1", "querier-1"}, - enqueueObjs: []enqueueObj{ - {obj: "query-1", path: QueuePath{"tenant-1", "query-component-1"}}, - {obj: "query-2", path: QueuePath{"tenant-2", "query-component-1"}}, - {obj: "query-3", path: QueuePath{"tenant-3", "query-component-1"}}, - }, - expected: []any{"query-1", "query-2", "query-3"}, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - currentQuerier := QuerierID("placeholder") - tt.state.currentQuerier = ¤tQuerier - - // We need a reference to state in order to be able to - // update the state's currentQuerier. - for i, da := range tt.treeAlgosByDepth { - switch da.(type) { - case *tenantQuerierAssignments: - tqas := tt.treeAlgosByDepth[i].(*tenantQuerierAssignments) - tqas.tenantQuerierIDs = tt.state.tenantQuerierIDs - tqas.currentQuerier = tt.state.currentQuerier - } - } - - tree, err := NewTree(tt.treeAlgosByDepth...) - require.NoError(t, err) - - for _, o := range tt.enqueueObjs { - err = tree.EnqueueBackByPath(o.path, o.obj) - require.NoError(t, err) - } - // currQuerier at position i is used to dequeue the expected result at position i - require.Equal(t, len(tt.currQuerier), len(tt.expected)) - for i := 0; i < len(tt.expected); i++ { - currentQuerier = tt.currQuerier[i] - _, v := tree.Dequeue() - require.Equal(t, tt.expected[i], v) - } - }) - } - -} - -// Test_ChangeTenantQuerierAssignments illustrates that we can update a state in tenantQuerierAssignments, -// and the tree dequeue behavior will adjust accordingly. -func Test_ChangeTenantQuerierAssignments(t *testing.T) { - tqa := tenantQuerierAssignments{ - tenantQuerierIDs: map[TenantID]map[QuerierID]struct{}{ - "tenant-1": {"querier-1": {}}, - "tenant-2": {"querier-2": {}}, - }, - } +// TestDequeueBalancedTree checks dequeuing behavior from a balanced tree. +// +// Dequeuing from a balanced tree allows the test to have a simple looped structures +// while running checks to ensure that round-robin order is respected. +func TestDequeueBalancedTree(t *testing.T) { + firstDimensions := []string{"0", "1", "2"} + secondDimensions := []string{"a", "b", "c"} + itemsPerDimension := 5 + root := makeBalancedTreeQueue(t, firstDimensions, secondDimensions, itemsPerDimension) + require.NotNil(t, root) - state := &tenantQuerierAssignments{ - tenantQuerierIDs: tqa.tenantQuerierIDs, - currentQuerier: nil, - } + count := 0 + // tree queue will fairly dequeue from all levels of the tree + rotationsBeforeRepeat := len(firstDimensions) * len(secondDimensions) + // track dequeued paths to ensure round-robin dequeuing does not repeat before expected + dequeuedPathCache := make([]QueuePath, rotationsBeforeRepeat) - tree, err := NewTree(state, &roundRobinState{}, &roundRobinState{}) - require.NoError(t, err) + for !root.IsEmpty() { + _, v := root.Dequeue() + dequeuedPath := getChildPathFromQueueItem(v) - enqueueObjs := []enqueueObj{ - {"query-1", QueuePath{"tenant-1", "query-component-1"}}, - {"query-2", QueuePath{"tenant-2", "query-component-1"}}, - {"query-3", QueuePath{"tenant-2", "query-component-1"}}, - {"query-4", QueuePath{"tenant-2", "query-component-1"}}, - {"query-5", QueuePath{"tenant-3", "query-component-1"}}, - } + // require dequeued path has not repeated before the expected number of rotations + require.NotContains(t, dequeuedPathCache, dequeuedPath) - for _, eo := range enqueueObjs { - err = tree.EnqueueBackByPath(eo.path, eo.obj) - require.NoError(t, err) + dequeuedPathCache = append(dequeuedPathCache[1:], dequeuedPath) + count++ } - querier1 := QuerierID("querier-1") - querier2 := QuerierID("querier-2") - querier3 := QuerierID("querier-3") - - // set state to querier-2 should dequeue query-2 - state.currentQuerier = &querier2 - state.tenantOrderIndex = -1 - _, v := tree.Dequeue() - require.Equal(t, "query-2", v) - - // update state to querier-1 should dequeue query-1 - state.currentQuerier = &querier1 - state.tenantOrderIndex = -1 - _, v = tree.Dequeue() - require.Equal(t, "query-1", v) - - // update tqa map to add querier-3 as assigned to tenant-2, then set state to querier-3 should dequeue query-3 - tqa.tenantQuerierIDs["tenant-2"]["querier-3"] = struct{}{} - state.currentQuerier = &querier3 - state.tenantOrderIndex = -1 - _, v = tree.Dequeue() - require.Equal(t, "query-3", v) - - // during reshuffle, we only ever reassign tenant values, we don't assign an entirely new map value - // to tenantQuerierIDs. Reassign tenant-2 to an empty map value, and query-5 (tenant-3), which can be handled - // by any querier, should be dequeued, - tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} - _, v = tree.Dequeue() - require.Equal(t, "query-5", v) - - // then we should not be able to dequeue query-4 - tqa.tenantQuerierIDs["tenant-2"] = map[QuerierID]struct{}{} - _, v = tree.Dequeue() - require.Nil(t, v) + // count items enqueued to nodes at depth 1 + expectedFirstDimensionCount := len(firstDimensions) * itemsPerDimension + // count items enqueued to nodes at depth 2 + expectedSecondDimensionCount := len(firstDimensions) * len(secondDimensions) * itemsPerDimension + require.Equal(t, expectedFirstDimensionCount+expectedSecondDimensionCount, count) } -// Test_DequeueBalancedRoundRobinTree checks dequeuing behavior from a balanced round-robin tree. +// TestDequeueByPathBalancedTree checks dequeuing behavior by path from a balanced tree. // // Dequeuing from a balanced tree allows the test to have a simple looped structures // while running checks to ensure that round-robin order is respected. -func Test_DequeueBalancedRoundRobinTree(t *testing.T) { +func TestDequeueByPathBalancedTree(t *testing.T) { firstDimensions := []string{"0", "1", "2"} secondDimensions := []string{"a", "b", "c"} itemsPerDimension := 5 - tree := makeBalancedRoundRobinTree(t, firstDimensions, secondDimensions, itemsPerDimension) - require.NotNil(t, tree) + root := makeBalancedTreeQueue(t, firstDimensions, secondDimensions, itemsPerDimension) + require.NotNil(t, root) count := 0 - - // IntegratedTreeQueue will fairly dequeue from all levels of the tree - rotationsBeforeRepeat := len(firstDimensions) * len(secondDimensions) + // tree queue will fairly dequeue from all levels of the tree below the path provided; + // dequeuing by path skips the top level the tree and rotates only through the + // second-layer subtrees of the first layer node selected by the queue path + rotationsBeforeRepeat := len(secondDimensions) // track dequeued paths to ensure round-robin dequeuing does not repeat before expected dequeuedPathCache := make([]QueuePath, rotationsBeforeRepeat) - for !tree.IsEmpty() { - dequeuedPath, _ := tree.Dequeue() + for _, firstDimName := range firstDimensions { + firstDimPath := QueuePath{firstDimName} + for root.getNode(firstDimPath) != nil { + v := root.DequeueByPath(firstDimPath) + dequeuedPath := getChildPathFromQueueItem(v) - // require dequeued path has not repeated before the expected number of rotations - require.NotContains(t, dequeuedPathCache, dequeuedPath) + // require dequeued path has not repeated before the expected number of rotations + require.NotContains(t, dequeuedPathCache, dequeuedPath) - dequeuedPathCache = append(dequeuedPathCache[1:], dequeuedPath) - count++ + dequeuedPathCache = append(dequeuedPathCache[1:], dequeuedPath) + count++ + } } // count items enqueued to nodes at depth 1 @@ -699,181 +88,268 @@ func Test_DequeueBalancedRoundRobinTree(t *testing.T) { expectedSecondDimensionCount := len(firstDimensions) * len(secondDimensions) * itemsPerDimension require.Equal(t, expectedFirstDimensionCount+expectedSecondDimensionCount, count) - } -// Test_DequeueUnbalancedRoundRobinTree checks dequeuing behavior from an unbalanced tree. +// TestDequeuePathUnbalancedTree checks dequeuing behavior from an unbalanced tree. // // Assertions are done one by one to illustrate and check the behaviors of dequeuing from // an unbalanced tree, where the same node will be dequeued from twice if the node remains // nonempty while its sibling nodes have been exhausted and deleted from the tree. -func Test_DequeueUnbalancedRoundRobinTree(t *testing.T) { - tree := makeUnbalancedRoundRobinTree(t) +func TestDequeueUnbalancedTree(t *testing.T) { + root := makeUnbalancedTreeQueue(t) // dequeue from root until exhausted - _, v := tree.Dequeue() + _, v := root.Dequeue() require.Equal(t, "root:0:val0", v) - _, v = tree.Dequeue() + // root:0 and any subtrees are exhausted + path := QueuePath{"0"} + v = root.DequeueByPath(path) + require.Nil(t, v) + // root:0 was deleted + require.Nil(t, root.getNode(path)) + + _, v = root.Dequeue() require.Equal(t, "root:1:val0", v) - _, v = tree.Dequeue() + _, v = root.Dequeue() require.Equal(t, "root:2:0:val0", v) - _, v = tree.Dequeue() + _, v = root.Dequeue() require.Equal(t, "root:1:0:val0", v) - _, v = tree.Dequeue() + _, v = root.Dequeue() require.Equal(t, "root:2:1:val0", v) - _, v = tree.Dequeue() + _, v = root.Dequeue() require.Equal(t, "root:1:val1", v) - _, v = tree.Dequeue() + _, v = root.Dequeue() require.Equal(t, "root:2:0:val1", v) - _, v = tree.Dequeue() + _, v = root.Dequeue() require.Equal(t, "root:1:0:val1", v) - _, v = tree.Dequeue() + // root:1 and any subtrees are exhausted + path = QueuePath{"1"} + v = root.DequeueByPath(path) + require.Nil(t, v) + // root:1 was deleted + require.Nil(t, root.getNode(path)) + + _, v = root.Dequeue() require.Equal(t, "root:2:1:val1", v) - _, v = tree.Dequeue() + _, v = root.Dequeue() require.Equal(t, "root:2:1:val2", v) + // root:2 and any subtrees are exhausted + path = QueuePath{"2"} + v = root.DequeueByPath(path) + require.Nil(t, v) + // root:2 was deleted + require.Nil(t, root.getNode(path)) + // all items have been dequeued - require.Equal(t, 0, tree.rootNode.ItemCount()) - require.Equal(t, 1, tree.rootNode.nodeCount()) + require.Equal(t, 0, root.ItemCount()) + require.Equal(t, 1, root.NodeCount()) // require nothing in local or child queues - require.True(t, tree.IsEmpty()) + require.True(t, root.IsEmpty()) } -func Test_EnqueueDuringDequeueRespectsRoundRobin(t *testing.T) { - tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) - require.NoError(t, err) - require.NotNil(t, tree) +// TestDequeueByPathUnbalancedTree checks dequeuing behavior from an unbalanced tree by path. +// +// Assertions are done one by one to illustrate and check the behaviors of dequeuing from +// an unbalanced tree, where the same node will be dequeued from twice if the node remains +// nonempty while its sibling nodes have been exhausted and deleted from the tree. +func TestDequeueByPathUnbalancedTree(t *testing.T) { + root := makeUnbalancedTreeQueue(t) + + // dequeue from root:2 until exhausted + path := QueuePath{"2"} + v := root.DequeueByPath(path) + require.Equal(t, "root:2:0:val0", v) + + v = root.DequeueByPath(path) + require.Equal(t, "root:2:1:val0", v) - root := tree.rootNode + v = root.DequeueByPath(path) + require.Equal(t, "root:2:0:val1", v) + + v = root.DequeueByPath(path) + require.Equal(t, "root:2:1:val1", v) + + v = root.DequeueByPath(path) + require.Equal(t, "root:2:1:val2", v) + + // root:2 is exhausted; + v = root.DequeueByPath(path) + require.Nil(t, v) + // root:2 and its two children root:2:0 and root:2:1 were deleted + require.Nil(t, root.getNode(path)) + require.Equal(t, 2, len(root.childQueueMap)) + require.Equal(t, 2, len(root.childQueueOrder)) + require.Equal(t, 4, root.NodeCount()) + // 5 of 10 items were dequeued + require.Equal(t, 5, root.ItemCount()) + + // dequeue from root:1 until exhausted + path = QueuePath{"1"} + v = root.DequeueByPath(path) + require.Equal(t, "root:1:val0", v) + + v = root.DequeueByPath(path) + require.Equal(t, "root:1:0:val0", v) + + v = root.DequeueByPath(path) + require.Equal(t, "root:1:val1", v) + + v = root.DequeueByPath(path) + require.Equal(t, "root:1:0:val1", v) + + // root:1 is exhausted; + v = root.DequeueByPath(path) + require.Nil(t, v) + // root:1 and its child root:1:0 were deleted + require.Nil(t, root.getNode(path)) + require.Equal(t, 1, len(root.childQueueMap)) + require.Equal(t, 1, len(root.childQueueOrder)) + require.Equal(t, 2, root.NodeCount()) + // 9 of 10 items have been dequeued + require.Equal(t, 1, root.ItemCount()) + + // dequeue from root:0 until exhausted + path = QueuePath{"0"} + v = root.DequeueByPath(path) + require.Equal(t, "root:0:val0", v) + + // root:0 is exhausted; + v = root.DequeueByPath(path) + require.Nil(t, v) + // root:0 was deleted + require.Nil(t, root.getNode(path)) + require.Equal(t, 0, len(root.childQueueMap)) + require.Equal(t, 0, len(root.childQueueOrder)) + require.Equal(t, 1, root.NodeCount()) + // 10 of 10 items have been dequeued + require.Equal(t, 0, root.ItemCount()) + require.Equal(t, 1, root.NodeCount()) + + // require nothing in local or child queues + require.True(t, root.IsEmpty()) +} + +func TestEnqueueDuringDequeueRespectsRoundRobin(t *testing.T) { + root := NewTreeQueue("root") cache := map[string]struct{}{} // enqueue two items to path root:0 childPath := QueuePath{"0"} - item := makeItemForChildQueue(root, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - item = makeItemForChildQueue(root, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item := makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) // enqueue one item to path root:1 childPath = QueuePath{"1"} - item = makeItemForChildQueue(root, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) // enqueue two items to path root:2 childPath = QueuePath{"2"} - item = makeItemForChildQueue(root, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - item = makeItemForChildQueue(root, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) - require.Equal(t, []string{"0", "1", "2"}, root.queueOrder) + require.Equal(t, []string{"0", "1", "2"}, root.childQueueOrder) // dequeue first item - dequeuedPath, _ := tree.Dequeue() + _, v := root.Dequeue() + dequeuedPath := getChildPathFromQueueItem(v) require.Equal(t, QueuePath{"0"}, dequeuedPath) // dequeue second item; root:1 is now exhausted and deleted - dequeuedPath, _ = tree.Dequeue() + _, v = root.Dequeue() + dequeuedPath = getChildPathFromQueueItem(v) require.Equal(t, QueuePath{"1"}, dequeuedPath) require.Nil(t, root.getNode(QueuePath{"1"})) - require.Equal(t, []string{"0", "2"}, root.queueOrder) + require.Equal(t, []string{"0", "2"}, root.childQueueOrder) // dequeue third item - dequeuedPath, _ = tree.Dequeue() + _, v = root.Dequeue() + dequeuedPath = getChildPathFromQueueItem(v) require.Equal(t, QueuePath{"2"}, dequeuedPath) // root:1 was previously exhausted; root:0, then root:2 will be next in the rotation // here we insert something new into root:1 to test that it // does not jump the line in front of root:0 or root:2 - item = makeItemForChildQueue(root, QueuePath{"1"}, cache) - require.NoError(t, tree.EnqueueBackByPath(QueuePath{"1"}, item)) + item = makeQueueItemForChildPath(root, QueuePath{"1"}, cache) + require.NoError(t, root.EnqueueBackByPath(QueuePath{"1"}, item)) require.NotNil(t, root.getNode(QueuePath{"1"})) - require.Equal(t, []string{"0", "2", "1"}, root.queueOrder) + require.Equal(t, []string{"0", "2", "1"}, root.childQueueOrder) // dequeue fourth item; the newly-enqueued root:1 item // has not jumped the line in front of root:0 - dequeuedPath, _ = tree.Dequeue() + _, v = root.Dequeue() + dequeuedPath = getChildPathFromQueueItem(v) require.Equal(t, QueuePath{"0"}, dequeuedPath) // dequeue fifth item; the newly-enqueued root:1 item // has not jumped the line in front of root:2 - dequeuedPath, _ = tree.Dequeue() + _, v = root.Dequeue() + dequeuedPath = getChildPathFromQueueItem(v) require.Equal(t, QueuePath{"2"}, dequeuedPath) // dequeue sixth item; verifying the order 0->2->1 is being followed - dequeuedPath, _ = tree.Dequeue() + _, v = root.Dequeue() + dequeuedPath = getChildPathFromQueueItem(v) require.Equal(t, QueuePath{"1"}, dequeuedPath) // all items have been dequeued require.Equal(t, 0, root.ItemCount()) - require.Equal(t, 1, root.nodeCount()) + require.Equal(t, 1, root.NodeCount()) // require nothing in local or child queues - require.True(t, tree.IsEmpty()) + require.True(t, root.IsEmpty()) } -// Test_NodeCannotDeleteItself creates an empty node, dequeues from it, and ensures that the node still exists -// and has not deleted itself. -func Test_NodeCannotDeleteItself(t *testing.T) { - tests := []struct { - name string - nodeType QueuingAlgorithm - }{ - {"round robin", &roundRobinState{}}, - {"tenant querier assignment", &tenantQuerierAssignments{}}, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tree, err := NewTree(tt.nodeType) - require.NoError(t, err) - require.NotNil(t, tree) - - _, _ = tree.Dequeue() - - require.NotNil(t, tree.rootNode) - require.Zero(t, tree.rootNode.getLocalQueue().Len()) - require.Empty(t, tree.rootNode.queueMap) - }) - } +func TestNodeCannotDeleteItself(t *testing.T) { + root := NewTreeQueue("root") + require.False(t, root.deleteNode(QueuePath{})) + require.NotNil(t, root) } -func makeBalancedRoundRobinTree(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimension int) *IntegratedTreeQueue { - tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) - require.NoError(t, err) - require.Equal(t, 1, tree.rootNode.nodeCount()) - require.Equal(t, 0, tree.rootNode.ItemCount()) +func makeBalancedTreeQueue(t *testing.T, firstDimensions, secondDimensions []string, itemsPerDimensions int) *TreeQueue { + root := NewTreeQueue("root") + require.Equal(t, 1, root.NodeCount()) + require.Equal(t, 0, root.ItemCount()) cache := map[string]struct{}{} for _, firstDimName := range firstDimensions { - for k := 0; k < itemsPerDimension; k++ { + // insert first dimension local queue items + for k := 0; k < itemsPerDimensions; k++ { childPath := QueuePath{firstDimName} - item := makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item := makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) } for _, secondDimName := range secondDimensions { - for k := 0; k < itemsPerDimension; k++ { + // insert second dimension local queue items + for k := 0; k < itemsPerDimensions; k++ { childPath := QueuePath{firstDimName, secondDimName} - item := makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) + item := makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) } } } - return tree + + return root } -func makeUnbalancedRoundRobinTree(t *testing.T) *IntegratedTreeQueue { +func makeUnbalancedTreeQueue(t *testing.T) *TreeQueue { /* root ├── child0 @@ -900,86 +376,85 @@ func makeUnbalancedRoundRobinTree(t *testing.T) *IntegratedTreeQueue { │ └── localQueue └── localQueue */ - tree, err := NewTree(&roundRobinState{}, &roundRobinState{}, &roundRobinState{}) - require.NoError(t, err) - require.Equal(t, 1, tree.rootNode.nodeCount()) - require.Equal(t, 0, tree.rootNode.ItemCount()) + root := NewTreeQueue("root") + require.Equal(t, 1, root.NodeCount()) + require.Equal(t, 0, root.ItemCount()) cache := map[string]struct{}{} // enqueue one item to root:0 childPath := QueuePath{"0"} - item := makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 2, tree.rootNode.nodeCount()) - require.Equal(t, 1, tree.rootNode.ItemCount()) + item := makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 2, root.NodeCount()) + require.Equal(t, 1, root.ItemCount()) // enqueue two items to root:1 childPath = QueuePath{"1"} - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 3, tree.rootNode.nodeCount()) - require.Equal(t, 2, tree.rootNode.ItemCount()) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 3, root.NodeCount()) + require.Equal(t, 2, root.ItemCount()) - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 3, tree.rootNode.nodeCount()) - require.Equal(t, 3, tree.rootNode.ItemCount()) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 3, root.NodeCount()) + require.Equal(t, 3, root.ItemCount()) // enqueue two items to root:1:0 childPath = QueuePath{"1", "0"} - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 4, tree.rootNode.nodeCount()) - require.Equal(t, 4, tree.rootNode.ItemCount()) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 4, root.NodeCount()) + require.Equal(t, 4, root.ItemCount()) - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 4, tree.rootNode.nodeCount()) - require.Equal(t, 5, tree.rootNode.ItemCount()) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 4, root.NodeCount()) + require.Equal(t, 5, root.ItemCount()) // enqueue two items to root:2:0 childPath = QueuePath{"2", "0"} - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 6, tree.rootNode.nodeCount()) - require.Equal(t, 6, tree.rootNode.ItemCount()) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 6, root.NodeCount()) + require.Equal(t, 6, root.ItemCount()) - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 6, tree.rootNode.nodeCount()) - require.Equal(t, 7, tree.rootNode.ItemCount()) + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 6, root.NodeCount()) + require.Equal(t, 7, root.ItemCount()) // enqueue three items to root:2:1 childPath = QueuePath{"2", "1"} - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 7, tree.rootNode.nodeCount()) - require.Equal(t, 8, tree.rootNode.ItemCount()) - - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 7, tree.rootNode.nodeCount()) - require.Equal(t, 9, tree.rootNode.ItemCount()) - - item = makeItemForChildQueue(tree.rootNode, childPath, cache) - require.NoError(t, tree.EnqueueBackByPath(childPath, item)) - require.Equal(t, 7, tree.rootNode.nodeCount()) - require.Equal(t, 10, tree.rootNode.ItemCount()) - - return tree + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, root.NodeCount()) + require.Equal(t, 8, root.ItemCount()) + + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, root.NodeCount()) + require.Equal(t, 9, root.ItemCount()) + + item = makeQueueItemForChildPath(root, childPath, cache) + require.NoError(t, root.EnqueueBackByPath(childPath, item)) + require.Equal(t, 7, root.NodeCount()) + require.Equal(t, 10, root.ItemCount()) + + return root } -// makeItemForChildQueue constructs a queue item to match its enqueued path +// makeQueueItemForChildPath constructs a queue item to match its enqueued path // by joining the path components and appending an incrementing value for each path. // // e.g. for a tree named "root": // - childQueuePath{"1", "0"}'s first item will be "root:1:0:val0" // - childQueuePath{"1", "0"}'s second item will be "root:1:0:val1" -func makeItemForChildQueue( - parent *Node, childPath QueuePath, cache map[string]struct{}, +func makeQueueItemForChildPath( + treeQueue *TreeQueue, childPath QueuePath, cache map[string]struct{}, ) string { - path := append(QueuePath{parent.name}, childPath...) + path := append(QueuePath{treeQueue.name}, childPath...) i := 0 for { @@ -991,3 +466,11 @@ func makeItemForChildQueue( i++ } } + +// getChildPathFromQueueItem enables assertions on the queue path the item was dequeued from. +// Assertions require that the queue item value was generated via makeQueueItemForChildPath. +func getChildPathFromQueueItem(v any) QueuePath { + itemPath := strings.Split(v.(string), ":") + itemPathPrefix := itemPath[1 : len(itemPath)-1] // strip value from the end + return itemPathPrefix +} diff --git a/pkg/scheduler/queue/tree_queueing_algorithms.go b/pkg/scheduler/queue/tree_queueing_algorithms.go index 60a30e23b8d..3f66f13837a 100644 --- a/pkg/scheduler/queue/tree_queueing_algorithms.go +++ b/pkg/scheduler/queue/tree_queueing_algorithms.go @@ -3,7 +3,7 @@ package queue // QueuingAlgorithm represents the set of operations specific to different approaches to queuing/dequeuing. It is -// applied at the layer-level -- every Node at the same depth in a IntegratedTreeQueue shares the same QueuingAlgorithm, +// applied at the layer-level -- every Node at the same depth in a MultiQueuingAlgorithmTreeQueue shares the same QueuingAlgorithm, // including any state in structs that implement QueuingAlgorithm. type QueuingAlgorithm interface { // addChildNode creates a child Node of parent. QueuePaths passed to enqueue functions are allowed to contain @@ -26,7 +26,7 @@ type QueuingAlgorithm interface { } // roundRobinState is the simplest type of QueuingAlgorithm; nodes which use this QueuingAlgorithm and are at -// the same depth in a IntegratedTreeQueue do not share any state. When children are added to these nodes, they are placed at +// the same depth in a MultiQueuingAlgorithmTreeQueue do not share any state. When children are added to these nodes, they are placed at // the "end" of the order from the perspective of the node's current queuePosition (e.g., if queuePosition is 3, // a new child will be placed at index 2). Children are dequeued from using a simple round-robin ordering; // queuePosition is incremented on every dequeue. From ecc02ea6be4d42d2b36ed4c87f530d3b91b6f923 Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Tue, 25 Jun 2024 15:54:47 -0700 Subject: [PATCH 23/26] Update CHANGELOG and flag name --- CHANGELOG.md | 1 + pkg/frontend/v1/frontend.go | 1 + pkg/scheduler/queue/queue.go | 6 +-- pkg/scheduler/queue/queue_test.go | 50 +++++++++---------- .../queue/tenant_querier_assignment.go | 3 +- pkg/scheduler/queue/tenant_queues.go | 4 +- pkg/scheduler/queue/tenant_queues_test.go | 14 +++--- pkg/scheduler/scheduler.go | 6 +-- 8 files changed, 44 insertions(+), 41 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7baae76ea43..224f77400cb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,7 @@ # Changelog ## main / unreleased +* [ENHANCEMENT] Query-scheduler: Introduce `query-scheduler.prioritize-query-components`, which allows configuration of the request queue tree to prioritize dequeuing from a specific query component more highly than dequeueing from a specific tenant. #7873 ### Grafana Mimir diff --git a/pkg/frontend/v1/frontend.go b/pkg/frontend/v1/frontend.go index 9f7d6929eeb..82bfc98d1e1 100644 --- a/pkg/frontend/v1/frontend.go +++ b/pkg/frontend/v1/frontend.go @@ -128,6 +128,7 @@ func New(cfg Config, limits Limits, log log.Logger, registerer prometheus.Regist log, cfg.MaxOutstandingPerTenant, false, + false, cfg.QuerierForgetDelay, f.queueLength, f.discardedRequests, diff --git a/pkg/scheduler/queue/queue.go b/pkg/scheduler/queue/queue.go index 21f04bd182c..d04f63ae4bf 100644 --- a/pkg/scheduler/queue/queue.go +++ b/pkg/scheduler/queue/queue.go @@ -121,7 +121,7 @@ type RequestQueue struct { // settings maxOutstandingPerTenant int additionalQueueDimensionsEnabled bool - useIntegratedQueue bool + useMultiAlgoQueue bool forgetDelay time.Duration // metrics for reporting @@ -175,7 +175,7 @@ func NewRequestQueue( log log.Logger, maxOutstandingPerTenant int, additionalQueueDimensionsEnabled bool, - useIntegratedQueue bool, + useMultiAlgoQueue bool, forgetDelay time.Duration, queueLength *prometheus.GaugeVec, discardedRequests *prometheus.CounterVec, @@ -212,7 +212,7 @@ func NewRequestQueue( waitingQuerierConnsToDispatch: list.New(), QueryComponentUtilization: queryComponentCapacity, - queueBroker: newQueueBroker(maxOutstandingPerTenant, additionalQueueDimensionsEnabled, useIntegratedQueue, forgetDelay), + queueBroker: newQueueBroker(maxOutstandingPerTenant, additionalQueueDimensionsEnabled, useMultiAlgoQueue, forgetDelay), } q.Service = services.NewBasicService(q.starting, q.running, q.stop).WithName("request queue") diff --git a/pkg/scheduler/queue/queue_test.go b/pkg/scheduler/queue/queue_test.go index 290707b46c4..0309d43ddff 100644 --- a/pkg/scheduler/queue/queue_test.go +++ b/pkg/scheduler/queue/queue_test.go @@ -32,12 +32,12 @@ import ( // TODO (casie): Write tests for prioritizeQueryComponents is true func buildTreeTestsStruct() []struct { - name string - useIntegratedTreeQueue bool + name string + useMultiAlgoTreeQueue bool } { return []struct { - name string - useIntegratedTreeQueue bool + name string + useMultiAlgoTreeQueue bool }{ {"legacy tree queue", false}, {"integrated tree queue", true}, @@ -149,7 +149,7 @@ func TestMultiDimensionalQueueFairnessSlowConsumerEffects(t *testing.T) { log.NewNopLogger(), maxOutstandingRequestsPerTenant, additionalQueueDimensionsEnabled, - tt.useIntegratedTreeQueue, + tt.useMultiAlgoTreeQueue, forgetQuerierDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -254,21 +254,21 @@ func BenchmarkConcurrentQueueOperations(b *testing.B) { for _, numProducers := range []int{10, 25} { b.Run(fmt.Sprintf("%v concurrent producers", numProducers), func(b *testing.B) { - // Queriers run with parallelism of 16 when query sharding is enabled. - for _, numConsumers := range []int{16, 160, 1600} { - b.Run(fmt.Sprintf("%v concurrent consumers", numConsumers), func(b *testing.B) { - queue, err := NewRequestQueue( - log.NewNopLogger(), - maxOutstandingRequestsPerTenant, - true, - t.useIntegratedTreeQueue, - forgetQuerierDelay, - promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), - promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), - promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), - promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), - ) - require.NoError(b, err) + // Queriers run with parallelism of 16 when query sharding is enabled. + for _, numConsumers := range []int{16, 160, 1600} { + b.Run(fmt.Sprintf("%v concurrent consumers", numConsumers), func(b *testing.B) { + queue, err := NewRequestQueue( + log.NewNopLogger(), + maxOutstandingRequestsPerTenant, + true, + t.useMultiAlgoTreeQueue, + forgetQuerierDelay, + promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), + promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), + promauto.With(nil).NewHistogram(prometheus.HistogramOpts{}), + promauto.With(nil).NewSummaryVec(prometheus.SummaryOpts{}, []string{"query_component"}), + ) + require.NoError(b, err) startSignalChan := make(chan struct{}) queueActorsErrGroup, ctx := errgroup.WithContext(context.Background()) @@ -441,7 +441,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldGetRequestAfterReshardingBe queue, err := NewRequestQueue( log.NewNopLogger(), 1, true, - tt.useIntegratedTreeQueue, + tt.useMultiAlgoTreeQueue, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -517,7 +517,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ReshardNotifiedCorrectlyForMultip queue, err := NewRequestQueue( log.NewNopLogger(), 1, true, - tt.useIntegratedTreeQueue, + tt.useMultiAlgoTreeQueue, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -608,7 +608,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldReturnAfterContextCancelled log.NewNopLogger(), 1, true, - tt.useIntegratedTreeQueue, + tt.useMultiAlgoTreeQueue, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -668,7 +668,7 @@ func TestRequestQueue_GetNextRequestForQuerier_ShouldReturnImmediatelyIfQuerierI log.NewNopLogger(), 1, true, - tt.useIntegratedTreeQueue, + tt.useMultiAlgoTreeQueue, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), @@ -704,7 +704,7 @@ func TestRequestQueue_tryDispatchRequestToQuerier_ShouldReEnqueueAfterFailedSend log.NewNopLogger(), 1, true, - tt.useIntegratedTreeQueue, + tt.useMultiAlgoTreeQueue, forgetDelay, promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"user"}), promauto.With(nil).NewCounterVec(prometheus.CounterOpts{}, []string{"user"}), diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 7ee29266c5d..683906ac0a4 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -507,7 +507,8 @@ func (tqa *tenantQuerierAssignments) addChildNode(parent, child *Node) { } // updateQueuingAlgorithmState should be called before attempting to dequeue, and updates inputs required by this -// QueuingAlgorithm to dequeue the appropriate value for the given querier. +// QueuingAlgorithm to dequeue the appropriate value for the given querier. In some test cases, it need not be called +// before consecutive dequeues for the same querier, but in all operating cases, it should be called ahead of a dequeue. func (tqa *tenantQuerierAssignments) updateQueuingAlgorithmState(querierID QuerierID, tenantOrderIndex int) { tqa.currentQuerier = querierID tqa.tenantOrderIndex = tenantOrderIndex diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 86e36955085..4c59d06c40b 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -36,7 +36,7 @@ type queueBroker struct { func newQueueBroker( maxTenantQueueSize int, additionalQueueDimensionsEnabled bool, - useIntegratedTreeQueue bool, + useMultiAlgoTreeQueue bool, forgetDelay time.Duration, ) *queueBroker { currentQuerier := QuerierID("") @@ -54,7 +54,7 @@ func newQueueBroker( var tree Tree var err error - if useIntegratedTreeQueue { + if useMultiAlgoTreeQueue { tree, err = NewTree( tqas, // root; QueuingAlgorithm selects tenants &roundRobinState{}, // tenant queues; QueuingAlgorithm selects query component diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 7ba974ebdd6..884790dae6b 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -69,7 +69,7 @@ func TestQueues_NoShuffleSharding(t *testing.T) { treeTypes := buildTreeTestsStruct() for _, tt := range treeTypes { t.Run(tt.name, func(t *testing.T) { - qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, 0) + qb := newQueueBroker(0, true, tt.useMultiAlgoTreeQueue, 0) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -211,7 +211,7 @@ func TestQueuesRespectMaxTenantQueueSizeWithSubQueues(t *testing.T) { for _, tt := range treeTypes { t.Run(tt.name, func(t *testing.T) { maxTenantQueueSize := 100 - qb := newQueueBroker(maxTenantQueueSize, true, tt.useIntegratedTreeQueue, 0) + qb := newQueueBroker(maxTenantQueueSize, true, tt.useMultiAlgoTreeQueue, 0) additionalQueueDimensions := map[int][]string{ 0: nil, 1: {"ingester"}, @@ -290,7 +290,7 @@ func TestQueuesOnTerminatingQuerier(t *testing.T) { treeTypes := buildTreeTestsStruct() for _, tt := range treeTypes { t.Run(tt.name, func(t *testing.T) { - qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, 0) + qb := newQueueBroker(0, true, tt.useMultiAlgoTreeQueue, 0) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -364,7 +364,7 @@ func TestQueues_QuerierDistribution(t *testing.T) { treeTypes := buildTreeTestsStruct() for _, tt := range treeTypes { t.Run(tt.name, func(t *testing.T) { - qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, 0) + qb := newQueueBroker(0, true, tt.useMultiAlgoTreeQueue, 0) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -444,7 +444,7 @@ func TestQueuesConsistency(t *testing.T) { t.Run(tt.name, func(t *testing.T) { for testName, testData := range tests { t.Run(testName, func(t *testing.T) { - qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, testData.forgetDelay) + qb := newQueueBroker(0, true, tt.useMultiAlgoTreeQueue, testData.forgetDelay) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -500,7 +500,7 @@ func TestQueues_ForgetDelay(t *testing.T) { for _, tt := range treeTypes { t.Run(tt.name, func(t *testing.T) { now := time.Now() - qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, forgetDelay) + qb := newQueueBroker(0, true, tt.useMultiAlgoTreeQueue, forgetDelay) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) @@ -603,7 +603,7 @@ func TestQueues_ForgetDelay_ShouldCorrectlyHandleQuerierReconnectingBeforeForget for _, tt := range treeTypes { t.Run(tt.name, func(t *testing.T) { now := time.Now() - qb := newQueueBroker(0, true, tt.useIntegratedTreeQueue, forgetDelay) + qb := newQueueBroker(0, true, tt.useMultiAlgoTreeQueue, forgetDelay) assert.NotNil(t, qb) assert.NoError(t, isConsistent(qb)) diff --git a/pkg/scheduler/scheduler.go b/pkg/scheduler/scheduler.go index ec79d33abf8..1af366d29d2 100644 --- a/pkg/scheduler/scheduler.go +++ b/pkg/scheduler/scheduler.go @@ -95,7 +95,7 @@ type connectedFrontend struct { type Config struct { MaxOutstandingPerTenant int `yaml:"max_outstanding_requests_per_tenant"` AdditionalQueryQueueDimensionsEnabled bool `yaml:"additional_query_queue_dimensions_enabled" category:"experimental"` - UseIntegratedQueryQueue bool `yaml:"use_integrated_query_queue" category:"experimental"` + UseMultiAlgorithmQueryQueue bool `yaml:"use_multi_algorithm_query_queue" category:"experimental"` QuerierForgetDelay time.Duration `yaml:"querier_forget_delay" category:"experimental"` GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config" doc:"description=This configures the gRPC client used to report errors back to the query-frontend."` @@ -105,7 +105,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.IntVar(&cfg.MaxOutstandingPerTenant, "query-scheduler.max-outstanding-requests-per-tenant", 100, "Maximum number of outstanding requests per tenant per query-scheduler. In-flight requests above this limit will fail with HTTP response status code 429.") f.BoolVar(&cfg.AdditionalQueryQueueDimensionsEnabled, "query-scheduler.additional-query-queue-dimensions-enabled", false, "Enqueue query requests with additional queue dimensions to split tenant request queues into subqueues. This enables separate requests to proceed from a tenant's subqueues even when other subqueues are blocked on slow query requests. Must be set on both query-frontend and scheduler to take effect. (default false)") - f.BoolVar(&cfg.UseIntegratedQueryQueue, "query-scheduler.use-integrated-query-queue", false, "Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. (default false)") + f.BoolVar(&cfg.UseMultiAlgorithmQueryQueue, "query-scheduler.use-multi-algorithm-query-queue", false, "Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. (default false)") f.DurationVar(&cfg.QuerierForgetDelay, "query-scheduler.querier-forget-delay", 0, "If a querier disconnects without sending notification about graceful shutdown, the query-scheduler will keep the querier in the tenant's shard until the forget delay has passed. This feature is useful to reduce the blast radius when shuffle-sharding is enabled.") cfg.GRPCClientConfig.RegisterFlagsWithPrefix("query-scheduler.grpc-client-config", f) @@ -162,7 +162,7 @@ func NewScheduler(cfg Config, limits Limits, log log.Logger, registerer promethe s.log, cfg.MaxOutstandingPerTenant, cfg.AdditionalQueryQueueDimensionsEnabled, - cfg.UseIntegratedQueryQueue, + cfg.UseMultiAlgorithmQueryQueue, cfg.QuerierForgetDelay, s.queueLength, s.discardedRequests, From cd2163247af6855e066a9426623c336e2bbf8d2b Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Tue, 25 Jun 2024 16:32:05 -0700 Subject: [PATCH 24/26] update docs --- cmd/mimir/config-descriptor.json | 11 +++++++++++ cmd/mimir/help-all.txt.tmpl | 2 ++ .../mimir/configure/configuration-parameters/index.md | 6 ++++++ pkg/scheduler/queue/queue.go | 1 - pkg/scheduler/queue/tenant_queues_test.go | 2 +- 5 files changed, 20 insertions(+), 2 deletions(-) diff --git a/cmd/mimir/config-descriptor.json b/cmd/mimir/config-descriptor.json index 8da2843a345..6a5d553ecb4 100644 --- a/cmd/mimir/config-descriptor.json +++ b/cmd/mimir/config-descriptor.json @@ -15909,6 +15909,17 @@ "fieldType": "boolean", "fieldCategory": "experimental" }, + { + "kind": "field", + "name": "use_multi_algorithm_query_queue", + "required": false, + "desc": "Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. (default false)", + "fieldValue": null, + "fieldDefaultValue": false, + "fieldFlag": "query-scheduler.use-multi-algorithm-query-queue", + "fieldType": "boolean", + "fieldCategory": "experimental" + }, { "kind": "field", "name": "querier_forget_delay", diff --git a/cmd/mimir/help-all.txt.tmpl b/cmd/mimir/help-all.txt.tmpl index 7a6f26ade4b..36ac2ff31e9 100644 --- a/cmd/mimir/help-all.txt.tmpl +++ b/cmd/mimir/help-all.txt.tmpl @@ -2317,6 +2317,8 @@ Usage of ./cmd/mimir/mimir: Backend storage to use for the ring. Supported values are: consul, etcd, inmemory, memberlist, multi. (default "memberlist") -query-scheduler.service-discovery-mode string [experimental] Service discovery mode that query-frontends and queriers use to find query-scheduler instances. When query-scheduler ring-based service discovery is enabled, this option needs be set on query-schedulers, query-frontends and queriers. Supported values are: dns, ring. (default "dns") + -query-scheduler.use-multi-algorithm-query-queue + [experimental] Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. (default false) -ruler-storage.azure.account-key string Azure storage account key. If unset, Azure managed identities will be used for authentication instead. -ruler-storage.azure.account-name string diff --git a/docs/sources/mimir/configure/configuration-parameters/index.md b/docs/sources/mimir/configure/configuration-parameters/index.md index 5f114817834..c1d79e7ee28 100644 --- a/docs/sources/mimir/configure/configuration-parameters/index.md +++ b/docs/sources/mimir/configure/configuration-parameters/index.md @@ -1708,6 +1708,12 @@ The `query_scheduler` block configures the query-scheduler. # CLI flag: -query-scheduler.additional-query-queue-dimensions-enabled [additional_query_queue_dimensions_enabled: | default = false] +# (experimental) Use an experimental version of the query queue which has the +# same behavior as the existing queue, but integrates tenant selection into the +# tree model. (default false) +# CLI flag: -query-scheduler.use-multi-algorithm-query-queue +[use_multi_algorithm_query_queue: | default = false] + # (experimental) If a querier disconnects without sending notification about # graceful shutdown, the query-scheduler will keep the querier in the tenant's # shard until the forget delay has passed. This feature is useful to reduce the diff --git a/pkg/scheduler/queue/queue.go b/pkg/scheduler/queue/queue.go index d04f63ae4bf..13c74cf56f4 100644 --- a/pkg/scheduler/queue/queue.go +++ b/pkg/scheduler/queue/queue.go @@ -121,7 +121,6 @@ type RequestQueue struct { // settings maxOutstandingPerTenant int additionalQueueDimensionsEnabled bool - useMultiAlgoQueue bool forgetDelay time.Duration // metrics for reporting diff --git a/pkg/scheduler/queue/tenant_queues_test.go b/pkg/scheduler/queue/tenant_queues_test.go index 884790dae6b..443d5dc763b 100644 --- a/pkg/scheduler/queue/tenant_queues_test.go +++ b/pkg/scheduler/queue/tenant_queues_test.go @@ -173,7 +173,7 @@ func TestQueues_NoShuffleSharding(t *testing.T) { {buildExpectedObject(tenantTwo.tenantID, 7), tenantTwo}, } - lastTenantIndexQuerierTwo = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierTwo, "querier-2", expectedDequeueVals) + _ = assertExpectedValuesOnDequeue(t, qb, lastTenantIndexQuerierTwo, "querier-2", expectedDequeueVals) // Remove two: [four "" three] qb.removeTenantQueue("two") From c56cdfce14e37e1c420b75e9fc3da0a84625668f Mon Sep 17 00:00:00 2001 From: Casie Chen Date: Wed, 26 Jun 2024 12:04:22 -0700 Subject: [PATCH 25/26] Update docs --- cmd/mimir/config-descriptor.json | 2 +- cmd/mimir/help-all.txt.tmpl | 2 +- docs/sources/mimir/configure/configuration-parameters/index.md | 2 +- pkg/scheduler/scheduler.go | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cmd/mimir/config-descriptor.json b/cmd/mimir/config-descriptor.json index 6a5d553ecb4..44cb63e701d 100644 --- a/cmd/mimir/config-descriptor.json +++ b/cmd/mimir/config-descriptor.json @@ -15913,7 +15913,7 @@ "kind": "field", "name": "use_multi_algorithm_query_queue", "required": false, - "desc": "Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. (default false)", + "desc": "Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model.", "fieldValue": null, "fieldDefaultValue": false, "fieldFlag": "query-scheduler.use-multi-algorithm-query-queue", diff --git a/cmd/mimir/help-all.txt.tmpl b/cmd/mimir/help-all.txt.tmpl index 36ac2ff31e9..9357e2e29e2 100644 --- a/cmd/mimir/help-all.txt.tmpl +++ b/cmd/mimir/help-all.txt.tmpl @@ -2318,7 +2318,7 @@ Usage of ./cmd/mimir/mimir: -query-scheduler.service-discovery-mode string [experimental] Service discovery mode that query-frontends and queriers use to find query-scheduler instances. When query-scheduler ring-based service discovery is enabled, this option needs be set on query-schedulers, query-frontends and queriers. Supported values are: dns, ring. (default "dns") -query-scheduler.use-multi-algorithm-query-queue - [experimental] Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. (default false) + [experimental] Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. -ruler-storage.azure.account-key string Azure storage account key. If unset, Azure managed identities will be used for authentication instead. -ruler-storage.azure.account-name string diff --git a/docs/sources/mimir/configure/configuration-parameters/index.md b/docs/sources/mimir/configure/configuration-parameters/index.md index c1d79e7ee28..34e8a50cda2 100644 --- a/docs/sources/mimir/configure/configuration-parameters/index.md +++ b/docs/sources/mimir/configure/configuration-parameters/index.md @@ -1710,7 +1710,7 @@ The `query_scheduler` block configures the query-scheduler. # (experimental) Use an experimental version of the query queue which has the # same behavior as the existing queue, but integrates tenant selection into the -# tree model. (default false) +# tree model. # CLI flag: -query-scheduler.use-multi-algorithm-query-queue [use_multi_algorithm_query_queue: | default = false] diff --git a/pkg/scheduler/scheduler.go b/pkg/scheduler/scheduler.go index 1af366d29d2..70fce9050df 100644 --- a/pkg/scheduler/scheduler.go +++ b/pkg/scheduler/scheduler.go @@ -105,7 +105,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.IntVar(&cfg.MaxOutstandingPerTenant, "query-scheduler.max-outstanding-requests-per-tenant", 100, "Maximum number of outstanding requests per tenant per query-scheduler. In-flight requests above this limit will fail with HTTP response status code 429.") f.BoolVar(&cfg.AdditionalQueryQueueDimensionsEnabled, "query-scheduler.additional-query-queue-dimensions-enabled", false, "Enqueue query requests with additional queue dimensions to split tenant request queues into subqueues. This enables separate requests to proceed from a tenant's subqueues even when other subqueues are blocked on slow query requests. Must be set on both query-frontend and scheduler to take effect. (default false)") - f.BoolVar(&cfg.UseMultiAlgorithmQueryQueue, "query-scheduler.use-multi-algorithm-query-queue", false, "Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model. (default false)") + f.BoolVar(&cfg.UseMultiAlgorithmQueryQueue, "query-scheduler.use-multi-algorithm-query-queue", false, "Use an experimental version of the query queue which has the same behavior as the existing queue, but integrates tenant selection into the tree model.") f.DurationVar(&cfg.QuerierForgetDelay, "query-scheduler.querier-forget-delay", 0, "If a querier disconnects without sending notification about graceful shutdown, the query-scheduler will keep the querier in the tenant's shard until the forget delay has passed. This feature is useful to reduce the blast radius when shuffle-sharding is enabled.") cfg.GRPCClientConfig.RegisterFlagsWithPrefix("query-scheduler.grpc-client-config", f) From 3cd5486a69da3bb68f066b3fa27b01fb7c2f1f09 Mon Sep 17 00:00:00 2001 From: francoposa Date: Mon, 1 Jul 2024 12:00:20 -0700 Subject: [PATCH 26/26] debug setup; not enabled for any container --- .../mimir-microservices-mode/compose-up.sh | 10 +- .../config/mimir.yaml | 2 + .../mimir-microservices-mode/dev.dockerfile | 5 +- .../docker-compose.jsonnet | 4 +- .../docker-compose.yml | 62 +++++---- integration/e2emimir/client.go | 2 +- integration/e2emimir/service.go | 5 +- integration/e2emimir/services.go | 131 +++++++++++------- ...tegration_memberlist_single_binary_test.go | 2 +- integration/vault_test.go | 2 + .../queue/tenant_querier_assignment.go | 18 +++ pkg/scheduler/queue/tenant_queues.go | 2 +- vendor/github.com/grafana/e2e/cache/cache.go | 6 +- vendor/github.com/grafana/e2e/db/db.go | 11 +- vendor/github.com/grafana/e2e/db/kafka.go | 2 + vendor/github.com/grafana/e2e/service.go | 44 +++--- 16 files changed, 196 insertions(+), 112 deletions(-) diff --git a/development/mimir-microservices-mode/compose-up.sh b/development/mimir-microservices-mode/compose-up.sh index 229d82c13db..31ac40fac1a 100755 --- a/development/mimir-microservices-mode/compose-up.sh +++ b/development/mimir-microservices-mode/compose-up.sh @@ -25,10 +25,10 @@ cd "$SCRIPT_DIR" && make CGO_ENABLED=0 GOOS=linux go build -mod=vendor -tags=netgo,stringlabels -gcflags "all=-N -l" -o "${SCRIPT_DIR}"/mimir "${SCRIPT_DIR}"/../../cmd/mimir docker_compose -f "${SCRIPT_DIR}"/docker-compose.yml build --build-arg BUILD_IMAGE="${BUILD_IMAGE}" distributor-1 -if [ "$(yq '.services.query-tee' "${SCRIPT_DIR}"/docker-compose.yml)" != "null" ]; then - # If query-tee is enabled, build its binary and image as well. - CGO_ENABLED=0 GOOS=linux go build -mod=vendor -tags=netgo,stringlabels -gcflags "all=-N -l" -o "${SCRIPT_DIR}"/../../cmd/query-tee "${SCRIPT_DIR}"/../../cmd/query-tee - docker_compose -f "${SCRIPT_DIR}"/docker-compose.yml build --build-arg BUILD_IMAGE="${BUILD_IMAGE}" query-tee -fi +#if [ "$(yq '.services.query-tee' "${SCRIPT_DIR}"/docker-compose.yml)" != "null" ]; then +# # If query-tee is enabled, build its binary and image as well. +# CGO_ENABLED=0 GOOS=linux go build -mod=vendor -tags=netgo,stringlabels -gcflags "all=-N -l" -o "${SCRIPT_DIR}"/../../cmd/query-tee "${SCRIPT_DIR}"/../../cmd/query-tee +# docker_compose -f "${SCRIPT_DIR}"/docker-compose.yml build --build-arg BUILD_IMAGE="${BUILD_IMAGE}" query-tee +#fi docker_compose -f "${SCRIPT_DIR}"/docker-compose.yml up "$@" diff --git a/development/mimir-microservices-mode/config/mimir.yaml b/development/mimir-microservices-mode/config/mimir.yaml index b3ebb8b6da1..dcde389a377 100644 --- a/development/mimir-microservices-mode/config/mimir.yaml +++ b/development/mimir-microservices-mode/config/mimir.yaml @@ -138,6 +138,7 @@ frontend: cache_results: true additional_query_queue_dimensions_enabled: true shard_active_series_queries: true + max_outstanding_per_tenant: 10000 # Uncomment when using "dns" service discovery mode for query-scheduler. # scheduler_address: "query-scheduler:9011" @@ -163,6 +164,7 @@ query_scheduler: # Change to "dns" to switch to query-scheduler DNS-based service discovery. service_discovery_mode: "ring" additional_query_queue_dimensions_enabled: true + max_outstanding_requests_per_tenant: 10000 limits: # Limit max query time range to 31d diff --git a/development/mimir-microservices-mode/dev.dockerfile b/development/mimir-microservices-mode/dev.dockerfile index 1fce967c6d9..3210e9c971b 100644 --- a/development/mimir-microservices-mode/dev.dockerfile +++ b/development/mimir-microservices-mode/dev.dockerfile @@ -1,11 +1,12 @@ ARG BUILD_IMAGE # Use ./compose-up.sh to build this image. FROM $BUILD_IMAGE ENV CGO_ENABLED=0 -RUN go install github.com/go-delve/delve/cmd/dlv@v1.21.0 +RUN go install github.com/go-delve/delve/cmd/dlv@v1.22.1 FROM alpine:3.19.1 RUN mkdir /mimir WORKDIR /mimir COPY ./mimir ./ -COPY --from=0 /go/bin/dlv ./ +RUN ln -s ./mimir /usr/local/bin/mimir +COPY --from=0 /go/bin/dlv /usr/local/bin/dlv diff --git a/development/mimir-microservices-mode/docker-compose.jsonnet b/development/mimir-microservices-mode/docker-compose.jsonnet index 98b03d133a3..72f09c4509a 100644 --- a/development/mimir-microservices-mode/docker-compose.jsonnet +++ b/development/mimir-microservices-mode/docker-compose.jsonnet @@ -5,7 +5,7 @@ std.manifestYamlDoc({ // If true, Mimir services are run under Delve debugger, that can be attached to via remote-debugging session. // Note that Delve doesn't forward signals to the Mimir process, so Mimir components don't shutdown cleanly. - debug: false, + debug: true, // How long should Mimir docker containers sleep before Mimir is started. sleep_seconds: 3, @@ -23,7 +23,7 @@ std.manifestYamlDoc({ ring: 'memberlist', // If true, a load generator is started. - enable_load_generator: true, + enable_load_generator: false, // If true, start and enable scraping by these components. // Note that if more than one component is enabled, the dashboards shown in Grafana may contain duplicate series or aggregates may be doubled or tripled. diff --git a/development/mimir-microservices-mode/docker-compose.yml b/development/mimir-microservices-mode/docker-compose.yml index 8b4fe906391..291c859c49a 100644 --- a/development/mimir-microservices-mode/docker-compose.yml +++ b/development/mimir-microservices-mode/docker-compose.yml @@ -6,7 +6,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=alertmanager -server.http-listen-port=8031 -server.grpc-listen-port=9031 -activity-tracker.filepath=/activity/alertmanager-8031 -alertmanager.web.external-url=http://localhost:8031/alertmanager -memberlist.nodename=alertmanager-1 -memberlist.bind-port=10031 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18031 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=alertmanager -server.http-listen-port=8031 -server.grpc-listen-port=9031 -activity-tracker.filepath=/activity/alertmanager-8031 -alertmanager.web.external-url=http://localhost:8031/alertmanager -memberlist.nodename=alertmanager-1 -memberlist.bind-port=10031 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -22,6 +22,7 @@ "ports": - "8031:8031" - "10031:10031" + - "18031:18031" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -32,7 +33,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=alertmanager -server.http-listen-port=8032 -server.grpc-listen-port=9032 -activity-tracker.filepath=/activity/alertmanager-8032 -alertmanager.web.external-url=http://localhost:8032/alertmanager -memberlist.nodename=alertmanager-2 -memberlist.bind-port=10032 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18032 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=alertmanager -server.http-listen-port=8032 -server.grpc-listen-port=9032 -activity-tracker.filepath=/activity/alertmanager-8032 -alertmanager.web.external-url=http://localhost:8032/alertmanager -memberlist.nodename=alertmanager-2 -memberlist.bind-port=10032 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -48,6 +49,7 @@ "ports": - "8032:8032" - "10032:10032" + - "18032:18032" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -58,7 +60,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=alertmanager -server.http-listen-port=8033 -server.grpc-listen-port=9033 -activity-tracker.filepath=/activity/alertmanager-8033 -alertmanager.web.external-url=http://localhost:8033/alertmanager -memberlist.nodename=alertmanager-3 -memberlist.bind-port=10033 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18033 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=alertmanager -server.http-listen-port=8033 -server.grpc-listen-port=9033 -activity-tracker.filepath=/activity/alertmanager-8033 -alertmanager.web.external-url=http://localhost:8033/alertmanager -memberlist.nodename=alertmanager-3 -memberlist.bind-port=10033 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -74,6 +76,7 @@ "ports": - "8033:8033" - "10033:10033" + - "18033:18033" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -84,7 +87,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=compactor -server.http-listen-port=8006 -server.grpc-listen-port=9006 -activity-tracker.filepath=/activity/compactor-8006 -memberlist.nodename=compactor -memberlist.bind-port=10006 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18006 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=compactor -server.http-listen-port=8006 -server.grpc-listen-port=9006 -activity-tracker.filepath=/activity/compactor-8006 -memberlist.nodename=compactor -memberlist.bind-port=10006 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -100,6 +103,7 @@ "ports": - "8006:8006" - "10006:10006" + - "18006:18006" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -110,7 +114,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=distributor -server.http-listen-port=8000 -server.grpc-listen-port=9000 -activity-tracker.filepath=/activity/distributor-8000 -memberlist.nodename=distributor -memberlist.bind-port=10000 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18000 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=distributor -server.http-listen-port=8000 -server.grpc-listen-port=9000 -activity-tracker.filepath=/activity/distributor-8000 -memberlist.nodename=distributor -memberlist.bind-port=10000 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" "environment": @@ -125,6 +129,7 @@ "ports": - "8000:8000" - "10000:10000" + - "18000:18000" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -135,7 +140,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=distributor -server.http-listen-port=8001 -server.grpc-listen-port=9001 -activity-tracker.filepath=/activity/distributor-8001 -memberlist.nodename=distributor -memberlist.bind-port=10001 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18001 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=distributor -server.http-listen-port=8001 -server.grpc-listen-port=9001 -activity-tracker.filepath=/activity/distributor-8001 -memberlist.nodename=distributor -memberlist.bind-port=10001 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" "environment": @@ -150,6 +155,7 @@ "ports": - "8001:8001" - "10001:10001" + - "18001:18001" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -171,7 +177,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=ingester -server.http-listen-port=8002 -server.grpc-listen-port=9002 -activity-tracker.filepath=/activity/ingester-8002 -memberlist.nodename=ingester-1 -memberlist.bind-port=10002 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18002 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=ingester -server.http-listen-port=8002 -server.grpc-listen-port=9002 -activity-tracker.filepath=/activity/ingester-8002 -memberlist.nodename=ingester-1 -memberlist.bind-port=10002 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -187,6 +193,7 @@ "ports": - "8002:8002" - "10002:10002" + - "18002:18002" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -198,7 +205,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=ingester -server.http-listen-port=8003 -server.grpc-listen-port=9003 -activity-tracker.filepath=/activity/ingester-8003 -memberlist.nodename=ingester-2 -memberlist.bind-port=10003 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18003 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=ingester -server.http-listen-port=8003 -server.grpc-listen-port=9003 -activity-tracker.filepath=/activity/ingester-8003 -memberlist.nodename=ingester-2 -memberlist.bind-port=10003 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -214,6 +221,7 @@ "ports": - "8003:8003" - "10003:10003" + - "18003:18003" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -225,7 +233,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=ingester -server.http-listen-port=8004 -server.grpc-listen-port=9004 -activity-tracker.filepath=/activity/ingester-8004 -memberlist.nodename=ingester-3 -memberlist.bind-port=10004 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18004 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=ingester -server.http-listen-port=8004 -server.grpc-listen-port=9004 -activity-tracker.filepath=/activity/ingester-8004 -memberlist.nodename=ingester-3 -memberlist.bind-port=10004 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -241,6 +249,7 @@ "ports": - "8004:8004" - "10004:10004" + - "18004:18004" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -250,20 +259,6 @@ "ports": - "16686:16686" - "14268" - "load-generator": - "command": - - "--remote-url=http://distributor-2:8001/api/v1/push" - - "--remote-write-concurrency=5" - - "--remote-write-interval=10s" - - "--series-count=1000" - - "--tenants-count=1" - - "--query-enabled=true" - - "--query-interval=1s" - - "--query-url=http://querier:8005/prometheus" - - "--server-metrics-port=9900" - "image": "pracucci/cortex-load-generator:add-query-support-8633d4e" - "ports": - - "9900:9900" "memcached": "image": "memcached:1.6.19-alpine" "ports": @@ -321,7 +316,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=querier -server.http-listen-port=8005 -server.grpc-listen-port=9005 -activity-tracker.filepath=/activity/querier-8005 -querier.scheduler-address=query-scheduler:9011 -querier.frontend-address= -memberlist.nodename=querier -memberlist.bind-port=10005 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18005 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=querier -server.http-listen-port=8005 -server.grpc-listen-port=9005 -activity-tracker.filepath=/activity/querier-8005 -querier.scheduler-address=query-scheduler:9011 -querier.frontend-address= -memberlist.nodename=querier -memberlist.bind-port=10005 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -337,6 +332,7 @@ "ports": - "8005:8005" - "10005:10005" + - "18005:18005" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -347,7 +343,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=query-frontend -server.http-listen-port=8007 -server.grpc-listen-port=9007 -activity-tracker.filepath=/activity/query-frontend-8007 -query-frontend.max-total-query-length=8760h -query-frontend.scheduler-address=query-scheduler:9011 -memberlist.nodename=query-frontend -memberlist.bind-port=10007 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18007 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=query-frontend -server.http-listen-port=8007 -server.grpc-listen-port=9007 -activity-tracker.filepath=/activity/query-frontend-8007 -query-frontend.max-total-query-length=8760h -query-frontend.scheduler-address=query-scheduler:9011 -memberlist.nodename=query-frontend -memberlist.bind-port=10007 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -363,6 +359,7 @@ "ports": - "8007:8007" - "10007:10007" + - "18007:18007" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -373,7 +370,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=query-scheduler -server.http-listen-port=8011 -server.grpc-listen-port=9011 -activity-tracker.filepath=/activity/query-scheduler-8011 -query-frontend.max-total-query-length=8760h -memberlist.nodename=query-scheduler -memberlist.bind-port=10011 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18011 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=query-scheduler -server.http-listen-port=8011 -server.grpc-listen-port=9011 -activity-tracker.filepath=/activity/query-scheduler-8011 -query-frontend.max-total-query-length=8760h -memberlist.nodename=query-scheduler -memberlist.bind-port=10011 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -389,6 +386,7 @@ "ports": - "8011:8011" - "10011:10011" + - "18011:18011" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -399,7 +397,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=ruler -server.http-listen-port=8022 -server.grpc-listen-port=9022 -activity-tracker.filepath=/activity/ruler-8022 -memberlist.nodename=ruler-1 -memberlist.bind-port=10022 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18022 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=ruler -server.http-listen-port=8022 -server.grpc-listen-port=9022 -activity-tracker.filepath=/activity/ruler-8022 -memberlist.nodename=ruler-1 -memberlist.bind-port=10022 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -415,6 +413,7 @@ "ports": - "8022:8022" - "10022:10022" + - "18022:18022" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -425,7 +424,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=ruler -server.http-listen-port=8023 -server.grpc-listen-port=9023 -activity-tracker.filepath=/activity/ruler-8023 -memberlist.nodename=ruler-2 -memberlist.bind-port=10023 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18023 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=ruler -server.http-listen-port=8023 -server.grpc-listen-port=9023 -activity-tracker.filepath=/activity/ruler-8023 -memberlist.nodename=ruler-2 -memberlist.bind-port=10023 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -441,6 +440,7 @@ "ports": - "8023:8023" - "10023:10023" + - "18023:18023" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -451,7 +451,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=store-gateway -server.http-listen-port=8008 -server.grpc-listen-port=9008 -activity-tracker.filepath=/activity/store-gateway-8008 -memberlist.nodename=store-gateway-1 -memberlist.bind-port=10008 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18008 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=store-gateway -server.http-listen-port=8008 -server.grpc-listen-port=9008 -activity-tracker.filepath=/activity/store-gateway-8008 -memberlist.nodename=store-gateway-1 -memberlist.bind-port=10008 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -467,6 +467,7 @@ "ports": - "8008:8008" - "10008:10008" + - "18008:18008" "volumes": - "./config:/mimir/config" - "./activity:/activity" @@ -477,7 +478,7 @@ "command": - "sh" - "-c" - - "sleep 3 && exec ./mimir -config.file=./config/mimir.yaml -target=store-gateway -server.http-listen-port=8009 -server.grpc-listen-port=9009 -activity-tracker.filepath=/activity/store-gateway-8009 -memberlist.nodename=store-gateway-2 -memberlist.bind-port=10009 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" + - "sleep 3 && exec ./dlv exec ./mimir --listen=:18009 --headless=true --api-version=2 --accept-multiclient --continue -- -config.file=./config/mimir.yaml -target=store-gateway -server.http-listen-port=8009 -server.grpc-listen-port=9009 -activity-tracker.filepath=/activity/store-gateway-8009 -memberlist.nodename=store-gateway-2 -memberlist.bind-port=10009 -ingester.ring.store=memberlist -distributor.ring.store=memberlist -compactor.ring.store=memberlist -store-gateway.sharding-ring.store=memberlist -ruler.ring.store=memberlist -alertmanager.sharding-ring.store=memberlist -blocks-storage.bucket-store.index-cache.backend=memcached -blocks-storage.bucket-store.chunks-cache.backend=memcached -blocks-storage.bucket-store.metadata-cache.backend=memcached -query-frontend.results-cache.backend=memcached -ruler-storage.cache.backend=memcached -blocks-storage.bucket-store.index-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.chunks-cache.memcached.addresses=dns+memcached:11211 -blocks-storage.bucket-store.metadata-cache.memcached.addresses=dns+memcached:11211 -query-frontend.results-cache.memcached.addresses=dns+memcached:11211 -ruler-storage.cache.memcached.addresses=dns+memcached:11211" "depends_on": - "minio" - "distributor-1" @@ -493,6 +494,7 @@ "ports": - "8009:8009" - "10009:10009" + - "18009:18009" "volumes": - "./config:/mimir/config" - "./activity:/activity" diff --git a/integration/e2emimir/client.go b/integration/e2emimir/client.go index 0939fba89f5..ce0bbb4a3f5 100644 --- a/integration/e2emimir/client.go +++ b/integration/e2emimir/client.go @@ -1523,7 +1523,7 @@ func (c *Client) doRequest(method, url string, body io.Reader) (*http.Response, req.Header.Set("X-Scope-OrgID", c.orgID) client := *c.httpClient - client.Timeout = c.timeout + client.Timeout = 2 * time.Minute return client.Do(req) } diff --git a/integration/e2emimir/service.go b/integration/e2emimir/service.go index 77189fb07bc..c00be3d8d00 100644 --- a/integration/e2emimir/service.go +++ b/integration/e2emimir/service.go @@ -22,12 +22,13 @@ func NewMimirService( envVars map[string]string, httpPort int, grpcPort int, - otherPorts ...int, + otherPorts []int, + portMapContainerToLocal map[int]int, ) *MimirService { s := &MimirService{ // We don't expose the gRPC port cause we don't need to access it from the host // (exposing ports have a negative performance impact on starting/stopping containers). - HTTPService: e2e.NewHTTPService(name, image, command, readiness, httpPort, otherPorts...), + HTTPService: e2e.NewHTTPService(name, image, command, readiness, httpPort, otherPorts, portMapContainerToLocal), grpcPort: grpcPort, } s.SetEnvVars(envVars) diff --git a/integration/e2emimir/services.go b/integration/e2emimir/services.go index 138c98d870f..c83620bc810 100644 --- a/integration/e2emimir/services.go +++ b/integration/e2emimir/services.go @@ -15,12 +15,14 @@ import ( ) const ( - httpPort = 8080 - grpcPort = 9095 + httpPort = 8080 + grpcPort = 9095 + mimirBinaryName = "mimir" + delveBinaryName = "dlv" ) -// GetDefaultImage returns the Docker image to use to run Mimir. -func GetDefaultImage() string { +// GetMimirImageOrDefault returns the Docker image to use to run Mimir. +func GetMimirImageOrDefault() string { // Get the mimir image from the MIMIR_IMAGE env variable, // falling back to grafana/mimir:latest" if img := os.Getenv("MIMIR_IMAGE"); img != "" { @@ -68,20 +70,56 @@ func getExtraFlags() map[string]string { return extraArgs } -func newMimirServiceFromOptions(name string, defaultFlags, flags map[string]string, options ...Option) *MimirService { +func newMimirServiceFromOptions(name string, defaultFlags, overrideFlags map[string]string, options ...Option) *MimirService { + o := newOptions(options) + serviceFlags := o.MapFlags(e2e.MergeFlags(defaultFlags, overrideFlags, getExtraFlags())) + + return NewMimirService( + name, + "grafana/mimir:latest", + e2e.NewCommandWithoutEntrypoint(mimirBinaryName, e2e.BuildArgs(serviceFlags)...), + e2e.NewHTTPReadinessProbe(o.HTTPPort, "/ready", 200, 299), + o.Environment, + o.HTTPPort, + o.GRPCPort, + o.OtherPorts, + nil, + ) +} + +func newMimirDebugServiceFromOptions(name string, defaultFlags, flags map[string]string, options ...Option) *MimirService { o := newOptions(options) serviceFlags := o.MapFlags(e2e.MergeFlags(defaultFlags, flags, getExtraFlags())) - binaryName := getBinaryNameForBackwardsCompatibility() + serviceArgs := e2e.BuildArgs(serviceFlags) + + delveListenPort := o.HTTPPort + 10000 // follow convention in docker compose development environment + + // delve args must be ordered correctly due to the use of `--` + // to delineate the end of delve args and beginning of the binary's args; + // do not use any interface utilizing a map structure as order will not be preserved + delveArgs := []string{ + "--log", + "exec", + "mimir", + fmt.Sprintf("--listen=:%d", delveListenPort), + "--headless=true", + "--api-version=2", + "--accept-multiclient", + "--", + } + + cmd := e2e.NewCommandWithoutEntrypoint(delveBinaryName, append(delveArgs, serviceArgs...)...) return NewMimirService( name, o.Image, - e2e.NewCommandWithoutEntrypoint(binaryName, e2e.BuildArgs(serviceFlags)...), + cmd, e2e.NewHTTPReadinessProbe(o.HTTPPort, "/ready", 200, 299), o.Environment, o.HTTPPort, o.GRPCPort, - o.OtherPorts..., + []int{delveListenPort}, + map[int]int{delveListenPort: delveListenPort, o.HTTPPort: o.HTTPPort}, ) } @@ -95,10 +133,8 @@ func NewDistributor(name string, consulAddress string, flags map[string]string, "-ingester.ring.replication-factor": "1", "-distributor.remote-timeout": "2s", // Fail fast in integration tests. // Configure the ingesters ring backend - "-ingester.ring.store": "consul", - "-ingester.ring.consul.hostname": consulAddress, - "-ingester.partition-ring.store": "consul", - "-ingester.partition-ring.consul.hostname": consulAddress, + "-ingester.ring.store": "consul", + "-ingester.ring.consul.hostname": consulAddress, // Configure the distributor ring backend "-distributor.ring.store": "memberlist", }, @@ -115,10 +151,8 @@ func NewQuerier(name string, consulAddress string, flags map[string]string, opti "-log.level": "warn", "-ingester.ring.replication-factor": "1", // Ingesters ring backend. - "-ingester.ring.store": "consul", - "-ingester.ring.consul.hostname": consulAddress, - "-ingester.partition-ring.store": "consul", - "-ingester.partition-ring.consul.hostname": consulAddress, + "-ingester.ring.store": "consul", + "-ingester.ring.consul.hostname": consulAddress, // Query-frontend worker. "-querier.frontend-client.backoff-min-period": "100ms", "-querier.frontend-client.backoff-max-period": "100ms", @@ -160,10 +194,8 @@ func NewIngester(name string, consulAddress string, flags map[string]string, opt "-log.level": "warn", "-ingester.ring.num-tokens": "512", // Configure the ingesters ring backend - "-ingester.ring.store": "consul", - "-ingester.ring.consul.hostname": consulAddress, - "-ingester.partition-ring.store": "consul", - "-ingester.partition-ring.consul.hostname": consulAddress, + "-ingester.ring.store": "consul", + "-ingester.ring.consul.hostname": consulAddress, // Speed up the startup. "-ingester.ring.min-ready-duration": "0s", // Enable native histograms @@ -174,10 +206,6 @@ func NewIngester(name string, consulAddress string, flags map[string]string, opt ) } -func getBinaryNameForBackwardsCompatibility() string { - return "mimir" -} - func NewQueryFrontend(name string, flags map[string]string, options ...Option) *MimirService { return newMimirServiceFromOptions( name, @@ -186,8 +214,6 @@ func NewQueryFrontend(name string, flags map[string]string, options ...Option) * "-log.level": "warn", // Quickly detect query-scheduler when running it. "-query-frontend.scheduler-dns-lookup-period": "1s", - // Always exercise remote read limits in integration tests. - "-query-frontend.remote-read-limits-enabled": "true", }, flags, options..., @@ -225,20 +251,31 @@ func NewCompactor(name string, consulAddress string, flags map[string]string, op ) } -func NewSingleBinary(name string, flags map[string]string, options ...Option) *MimirService { +var singleBinaryDefaultFlags = map[string]string{ + // Do not pass any extra default flags (except few used to speed up the test) + // because the config could be driven by the config file. + "-target": "all", + "-log.level": "warn", + // Speed up the startup. + "-ingester.ring.min-ready-duration": "0s", + // Enable native histograms + "-ingester.native-histograms-ingestion-enabled": "true", +} + +func NewSingleBinary(name string, overrideFlags map[string]string, options ...Option) *MimirService { return newMimirServiceFromOptions( name, - map[string]string{ - // Do not pass any extra default flags (except few used to speed up the test) - // because the config could be driven by the config file. - "-target": "all", - "-log.level": "warn", - // Speed up the startup. - "-ingester.ring.min-ready-duration": "0s", - // Enable native histograms - "-ingester.native-histograms-ingestion-enabled": "true", - }, - flags, + singleBinaryDefaultFlags, + overrideFlags, + options..., + ) +} + +func NewDebugSingleBinary(name string, overrideFlags map[string]string, options ...Option) *MimirService { + return newMimirDebugServiceFromOptions( + name, + singleBinaryDefaultFlags, + overrideFlags, options..., ) } @@ -304,17 +341,17 @@ func NewAlertmanagerWithTLS(name string, flags map[string]string, options ...Opt "-target": "alertmanager", "-log.level": "warn", }, flags, getExtraFlags())) - binaryName := getBinaryNameForBackwardsCompatibility() return NewMimirService( name, - o.Image, - e2e.NewCommandWithoutEntrypoint(binaryName, e2e.BuildArgs(serviceFlags)...), + "grafana/mimir:latest", + e2e.NewCommandWithoutEntrypoint(mimirBinaryName, e2e.BuildArgs(serviceFlags)...), e2e.NewTCPReadinessProbe(o.HTTPPort), o.Environment, o.HTTPPort, o.GRPCPort, - o.OtherPorts..., + o.OtherPorts, + nil, ) } @@ -325,10 +362,8 @@ func NewRuler(name string, consulAddress string, flags map[string]string, option "-target": "ruler", "-log.level": "warn", // Configure the ingesters ring backend - "-ingester.ring.store": "consul", - "-ingester.ring.consul.hostname": consulAddress, - "-ingester.partition-ring.store": "consul", - "-ingester.partition-ring.consul.hostname": consulAddress, + "-ingester.ring.store": "consul", + "-ingester.ring.consul.hostname": consulAddress, }, flags, options..., @@ -363,7 +398,7 @@ type Option func(*Options) // newOptions creates an Options with default values and applies the options provided. func newOptions(options []Option) *Options { o := &Options{ - Image: GetDefaultImage(), + Image: GetMimirImageOrDefault(), MapFlags: NoopFlagMapper, HTTPPort: httpPort, GRPCPort: grpcPort, @@ -417,7 +452,7 @@ func WithConfigFile(configFile string) Option { } // WithNoopOption returns an option that doesn't change anything. -func WithNoopOption() Option { return func(*Options) {} } +func WithNoopOption() Option { return func(options *Options) {} } // FlagMapper is the type of function that maps flags, just to reduce some verbosity. type FlagMapper func(flags map[string]string) map[string]string diff --git a/integration/integration_memberlist_single_binary_test.go b/integration/integration_memberlist_single_binary_test.go index 56b8bec9ef0..a49ac21cd52 100644 --- a/integration/integration_memberlist_single_binary_test.go +++ b/integration/integration_memberlist_single_binary_test.go @@ -138,7 +138,7 @@ func newSingleBinary(name string, servername string, join string, testFlags map[ flags["-ingester.ring.observe-period"] = "0s" // No need to observe tokens because we're going to be the first instance. } - serv := e2emimir.NewSingleBinary( + serv := e2emimir.NewDebugSingleBinary( name, mergeFlags( DefaultSingleBinaryFlags(), diff --git a/integration/vault_test.go b/integration/vault_test.go index b7b92fd3c86..492497832c6 100644 --- a/integration/vault_test.go +++ b/integration/vault_test.go @@ -30,6 +30,8 @@ func TestVaultTokenRenewal(t *testing.T) { nil, e2e.NewHTTPReadinessProbe(httpPort, "/v1/sys/health", 200, 200), httpPort, + nil, + nil, ) vault.SetEnvVars(map[string]string{"VAULT_DEV_ROOT_TOKEN_ID": devToken}) require.NoError(t, s.StartAndWaitReady(vault)) diff --git a/pkg/scheduler/queue/tenant_querier_assignment.go b/pkg/scheduler/queue/tenant_querier_assignment.go index 683906ac0a4..5d525e53c0a 100644 --- a/pkg/scheduler/queue/tenant_querier_assignment.go +++ b/pkg/scheduler/queue/tenant_querier_assignment.go @@ -213,6 +213,24 @@ func (tqa *tenantQuerierAssignments) addQuerierConnection(querierID QuerierID) ( return tqa.recomputeTenantQueriers() } +func (tqa *tenantQuerierAssignments) removeTenantOld(tenantID TenantID) { + tenant := tqa.tenantsByID[tenantID] + if tenant == nil { + return + } + delete(tqa.tenantsByID, tenantID) + tqa.tenantIDOrder[tenant.orderIndex] = emptyTenantID + + // Shrink tenant list if possible by removing empty tenant IDs. + // We remove only from the end; removing from the middle would re-index all tenant IDs + // and skip tenants when starting iteration from a querier-provided lastTenantIndex. + // Empty tenant IDs stuck in the middle of the slice are handled + // by replacing them when a new tenant ID arrives in the queue. + for i := len(tqa.tenantIDOrder) - 1; i >= 0 && tqa.tenantIDOrder[i] == emptyTenantID; i-- { + tqa.tenantIDOrder = tqa.tenantIDOrder[:i] + } +} + // removeTenant only manages deletion of a *queueTenant from tenantsByID. All other // tenant deletion (e.g., from tenantIDOrder, or tenantNodes) is done during the dequeue operation, // as we cannot remove from those things arbitrarily; we must check whether other tenant diff --git a/pkg/scheduler/queue/tenant_queues.go b/pkg/scheduler/queue/tenant_queues.go index 4c59d06c40b..e6bed12f093 100644 --- a/pkg/scheduler/queue/tenant_queues.go +++ b/pkg/scheduler/queue/tenant_queues.go @@ -206,7 +206,7 @@ func (qb *queueBroker) dequeueRequestForQuerier( queueNodeAfterDequeue := tq.getNode(queuePath) if queueNodeAfterDequeue == nil { // queue node was deleted due to being empty after dequeue - qb.tenantQuerierAssignments.removeTenant(tenant.tenantID) + qb.tenantQuerierAssignments.removeTenantOld(tenant.tenantID) } } else if itq, ok := qb.tree.(*MultiQueuingAlgorithmTreeQueue); ok { queueNodeAfterDequeue := itq.GetNode(queuePath) diff --git a/vendor/github.com/grafana/e2e/cache/cache.go b/vendor/github.com/grafana/e2e/cache/cache.go index eda8a293e57..2fb49655d6d 100644 --- a/vendor/github.com/grafana/e2e/cache/cache.go +++ b/vendor/github.com/grafana/e2e/cache/cache.go @@ -16,7 +16,8 @@ func NewMemcached() *e2e.ConcreteService { images.Memcached, nil, e2e.NewTCPReadinessProbe(MemcachedPort), - MemcachedPort, + []int{MemcachedPort}, + nil, ) } @@ -35,6 +36,7 @@ func NewRedis() *e2e.ConcreteService { "--appendonly", "no", ), e2e.NewTCPReadinessProbe(RedisPort), - RedisPort, + []int{RedisPort}, + nil, ) } diff --git a/vendor/github.com/grafana/e2e/db/db.go b/vendor/github.com/grafana/e2e/db/db.go index e012c3c8eb0..3b19759bc90 100644 --- a/vendor/github.com/grafana/e2e/db/db.go +++ b/vendor/github.com/grafana/e2e/db/db.go @@ -45,6 +45,8 @@ func newMinio(port int, envVars map[string]string, bktNames ...string) *e2e.HTTP e2e.NewCommandWithoutEntrypoint("sh", "-c", strings.Join(commands, " && ")), e2e.NewHTTPReadinessProbe(port, "/minio/health/ready", 200, 200), port, + nil, + nil, ) envVars["MINIO_ACCESS_KEY"] = MinioAccessKey envVars["MINIO_SECRET_KEY"] = MinioSecretKey @@ -71,6 +73,8 @@ func NewKES(port int, serverName, serverKeyFile, serverCertFile, clientKeyFile, e2e.NewCommandWithoutEntrypoint("sh", "-c", command), readinessProbe, port, + nil, + nil, ), nil } @@ -82,6 +86,8 @@ func NewConsul() *e2e.HTTPService { e2e.NewCommand("agent", "-server", "-client=0.0.0.0", "-dev", "-log-level=err"), e2e.NewHTTPReadinessProbe(8500, "/v1/operator/autopilot/health", 200, 200, `"Healthy": true`), 8500, + nil, + nil, ) } @@ -92,7 +98,8 @@ func NewETCD() *e2e.HTTPService { e2e.NewCommand("/usr/local/bin/etcd", "--listen-client-urls=http://0.0.0.0:2379", "--advertise-client-urls=http://0.0.0.0:2379", "--listen-metrics-urls=http://0.0.0.0:9000", "--log-level=error"), e2e.NewHTTPReadinessProbe(9000, "/health", 200, 204), 2379, - 9000, // Metrics + []int{9000}, // Metrics + nil, ) } @@ -104,5 +111,7 @@ func NewDynamoDB() *e2e.HTTPService { // DynamoDB doesn't have a readiness probe, so we check if the / works even if returns 400 e2e.NewHTTPReadinessProbe(8000, "/", 400, 400), 8000, + nil, + nil, ) } diff --git a/vendor/github.com/grafana/e2e/db/kafka.go b/vendor/github.com/grafana/e2e/db/kafka.go index fe91aedc717..beb0163ea5a 100644 --- a/vendor/github.com/grafana/e2e/db/kafka.go +++ b/vendor/github.com/grafana/e2e/db/kafka.go @@ -25,6 +25,8 @@ func NewKafka() *KafkaService { nil, // No custom command. NewKafkaReadinessProbe(9092), 9092, + nil, + nil, ), } } diff --git a/vendor/github.com/grafana/e2e/service.go b/vendor/github.com/grafana/e2e/service.go index 047ecbcfa62..31143e060a5 100644 --- a/vendor/github.com/grafana/e2e/service.go +++ b/vendor/github.com/grafana/e2e/service.go @@ -33,9 +33,9 @@ var ( // at the time. type ConcreteService struct { name string - image string - networkPorts []int - env map[string]string + image string + ports []int + env map[string]string user string command *Command cmd *exec.Cmd @@ -43,7 +43,7 @@ type ConcreteService struct { privileged bool // Maps container ports to dynamically binded local ports. - networkPortsContainerToLocal map[int]int + portMapContainerToLocal map[int]int // Generic retry backoff. retryBackoff *backoff.Backoff @@ -58,14 +58,18 @@ func NewConcreteService( image string, command *Command, readiness ReadinessProbe, - networkPorts ...int, + ports []int, + portMapContainerToLocal map[int]int, ) *ConcreteService { + if portMapContainerToLocal == nil { + portMapContainerToLocal = map[int]int{} + } return &ConcreteService{ name: name, image: image, - networkPorts: networkPorts, + ports: ports, command: command, - networkPortsContainerToLocal: map[int]int{}, + portMapContainerToLocal: portMapContainerToLocal, readiness: readiness, retryBackoff: backoff.New(context.Background(), backoff.Config{ MinBackoff: 300 * time.Millisecond, @@ -123,7 +127,7 @@ func (s *ConcreteService) Start(networkName, sharedDir string) (err error) { } // Get the dynamic local ports mapped to the container. - for _, containerPort := range s.networkPorts { + for _, containerPort := range s.ports { var out []byte out, err = RunCommandAndGetOutput("docker", "port", s.containerName(), strconv.Itoa(containerPort)) @@ -140,10 +144,10 @@ func (s *ConcreteService) Start(networkName, sharedDir string) (err error) { return errors.Wrapf(err, "unable to get mapping for port %d (output: %s); service: %s", containerPort, string(out), s.name) } - s.networkPortsContainerToLocal[containerPort] = localPort + s.portMapContainerToLocal[containerPort] = localPort } - logger.Log("Ports for container:", s.containerName(), "Mapping:", s.networkPortsContainerToLocal) + logger.Log("Ports for container:", s.containerName(), "Mapping:", s.portMapContainerToLocal) return nil } @@ -195,7 +199,7 @@ func (s *ConcreteService) Endpoint(port int) string { } // Map the container port to the local port. - localPort, ok := s.networkPortsContainerToLocal[port] + localPort, ok := s.portMapContainerToLocal[port] if !ok { return "" } @@ -324,8 +328,13 @@ func (s *ConcreteService) buildDockerRunArgs(networkName, sharedDir string) []st } // Published ports - for _, port := range s.networkPorts { - args = append(args, "-p", strconv.Itoa(port)) + for _, containerPort := range s.ports { + if localPort, ok := s.portMapContainerToLocal[containerPort]; ok { + args = append(args, "-p", strconv.Itoa(localPort)+":"+strconv.Itoa(containerPort)) + + } else { + args = append(args, "-p", strconv.Itoa(containerPort)) + } } // Disable entrypoint if required @@ -560,10 +569,11 @@ func NewHTTPService( command *Command, readiness ReadinessProbe, httpPort int, - otherPorts ...int, + otherPorts []int, + networkPortsContainerToLocal map[int]int, ) *HTTPService { return &HTTPService{ - ConcreteService: NewConcreteService(name, image, command, readiness, append(otherPorts, httpPort)...), + ConcreteService: NewConcreteService(name, image, command, readiness, append(otherPorts, httpPort), networkPortsContainerToLocal), metricsTimeout: time.Second, httpPort: httpPort, } @@ -575,7 +585,7 @@ func (s *HTTPService) SetMetricsTimeout(timeout time.Duration) { func (s *HTTPService) Metrics() (_ string, err error) { // Map the container port to the local port - localPort := s.networkPortsContainerToLocal[s.httpPort] + localPort := s.portMapContainerToLocal[s.httpPort] // Fetch metrics. // Use an IPv4 address instead of "localhost" hostname because our port mapping assumes IPv4 @@ -745,4 +755,4 @@ func parseDockerIPv4Port(out string) (int, error) { // We've not been able to parse the output format. return 0, errors.New("unknown output format") -} +} \ No newline at end of file