-
Notifications
You must be signed in to change notification settings - Fork 833
scheduler: fragment queue and querier pick-up coordination #6968
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
yeya24
merged 1 commit into
cortexproject:master
from
rubywtl:scheduler/logicalplan_fragment_coordination
Sep 5, 2025
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,21 +1,32 @@ | ||
package distributed_execution | ||
|
||
// FragmentKey uniquely identifies a fragment of a distributed logical query plan. | ||
// It combines a queryID (to identify the overall query) and a fragmentID | ||
// (to identify the specific fragment within that query). | ||
type FragmentKey struct { | ||
queryID uint64 | ||
// queryID identifies the distributed query this fragment belongs to | ||
queryID uint64 | ||
// fragmentID identifies this specific fragment within the query | ||
fragmentID uint64 | ||
yeya24 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
func MakeFragmentKey(queryID uint64, fragmentID uint64) *FragmentKey { | ||
return &FragmentKey{ | ||
// MakeFragmentKey creates a new FragmentKey with the given queryID and fragmentID. | ||
// It's used to track and identify fragments during distributed query execution. | ||
func MakeFragmentKey(queryID uint64, fragmentID uint64) FragmentKey { | ||
return FragmentKey{ | ||
queryID: queryID, | ||
fragmentID: fragmentID, | ||
} | ||
} | ||
|
||
// GetQueryID returns the queryID for the current key | ||
// This ID is shared across all fragments of the same distributed query. | ||
func (f FragmentKey) GetQueryID() uint64 { | ||
return f.queryID | ||
} | ||
|
||
// GetFragmentID returns the ID for this specific fragment | ||
// within its parent query. | ||
func (f FragmentKey) GetFragmentID() uint64 { | ||
return f.fragmentID | ||
} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,52 @@ | ||
package plan_fragments | ||
|
||
import "github.com/thanos-io/promql-engine/logicalplan" | ||
|
||
// Fragmenter interface | ||
type Fragmenter interface { | ||
// Fragment function fragments the logical query plan and will always return the fragment in the order of child-to-root | ||
// in other words, the order of the fragment in the array will be the order they are being scheduled | ||
Fragment(node logicalplan.Node) ([]Fragment, error) | ||
} | ||
|
||
type DummyFragmenter struct { | ||
} | ||
|
||
func (f *DummyFragmenter) Fragment(node logicalplan.Node) ([]Fragment, error) { | ||
// simple logic without distributed optimizer | ||
return []Fragment{ | ||
{ | ||
Node: node, | ||
FragmentID: uint64(1), | ||
ChildIDs: []uint64{}, | ||
IsRoot: true, | ||
}, | ||
}, nil | ||
} | ||
|
||
type Fragment struct { | ||
Node logicalplan.Node | ||
FragmentID uint64 | ||
ChildIDs []uint64 | ||
IsRoot bool | ||
} | ||
|
||
func (s *Fragment) IsEmpty() bool { | ||
if s.Node != nil { | ||
return false | ||
} | ||
if s.FragmentID != 0 { | ||
return false | ||
} | ||
if s.IsRoot { | ||
return false | ||
} | ||
if len(s.ChildIDs) != 0 { | ||
return false | ||
} | ||
return true | ||
} | ||
|
||
func NewDummyFragmenter() Fragmenter { | ||
return &DummyFragmenter{} | ||
} |
46 changes: 46 additions & 0 deletions
46
pkg/distributed_execution/plan_fragments/fragmenter_test.go
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
package plan_fragments | ||
|
||
import ( | ||
"testing" | ||
"time" | ||
|
||
"github.com/stretchr/testify/require" | ||
|
||
"github.com/cortexproject/cortex/pkg/util/logical_plan" | ||
) | ||
|
||
func TestFragmenter(t *testing.T) { | ||
type testCase struct { | ||
name string | ||
query string | ||
start time.Time | ||
end time.Time | ||
expectedFragments int | ||
} | ||
|
||
now := time.Now() | ||
|
||
// more tests will be added when distributed optimizer and fragmenter are implemented | ||
tests := []testCase{ | ||
{ | ||
name: "simple logical query plan - no fragmentation", | ||
query: "up", | ||
start: now, | ||
end: now, | ||
expectedFragments: 1, | ||
}, | ||
} | ||
|
||
for _, tc := range tests { | ||
t.Run(tc.name, func(t *testing.T) { | ||
lp, err := logical_plan.CreateTestLogicalPlan(tc.query, tc.start, tc.end, 0) | ||
require.NoError(t, err) | ||
|
||
fragmenter := NewDummyFragmenter() | ||
res, err := fragmenter.Fragment((*lp).Root()) | ||
|
||
require.NoError(t, err) | ||
require.Equal(t, tc.expectedFragments, len(res)) | ||
}) | ||
} | ||
} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,79 @@ | ||
package fragment_table | ||
|
||
import ( | ||
"sync" | ||
"time" | ||
|
||
"github.com/cortexproject/cortex/pkg/distributed_execution" | ||
) | ||
|
||
type fragmentEntry struct { | ||
addr string | ||
createdAt time.Time | ||
} | ||
|
||
// FragmentTable maintains a mapping between query fragments and their assigned querier addresses. | ||
// Entries automatically expire after a configured duration to prevent stale mappings. | ||
type FragmentTable struct { | ||
mappings map[distributed_execution.FragmentKey]*fragmentEntry | ||
mu sync.RWMutex | ||
expiration time.Duration | ||
} | ||
|
||
// NewFragmentTable creates a new FragmentTable with the specified expiration duration. | ||
// It starts a background goroutine that periodically removes expired entries. | ||
// The cleanup interval is set to half of the expiration duration. | ||
func NewFragmentTable(expiration time.Duration) *FragmentTable { | ||
ft := &FragmentTable{ | ||
mappings: make(map[distributed_execution.FragmentKey]*fragmentEntry), | ||
expiration: expiration, | ||
} | ||
go ft.periodicCleanup() | ||
return ft | ||
} | ||
|
||
// AddAddressByID associates a querier address with a specific fragment of a query. | ||
// The association will automatically expire after the configured duration. | ||
func (f *FragmentTable) AddAddressByID(queryID uint64, fragmentID uint64, addr string) { | ||
f.mu.Lock() | ||
defer f.mu.Unlock() | ||
key := distributed_execution.MakeFragmentKey(queryID, fragmentID) | ||
f.mappings[key] = &fragmentEntry{ | ||
addr: addr, | ||
createdAt: time.Now(), | ||
} | ||
} | ||
|
||
// GetAddrByID retrieves the querier address associated with a specific fragment. | ||
func (f *FragmentTable) GetAddrByID(queryID uint64, fragmentID uint64) (string, bool) { | ||
f.mu.RLock() | ||
defer f.mu.RUnlock() | ||
key := distributed_execution.MakeFragmentKey(queryID, fragmentID) | ||
if entry, ok := f.mappings[key]; ok { | ||
return entry.addr, true | ||
} | ||
return "", false | ||
} | ||
|
||
func (f *FragmentTable) cleanupExpired() { | ||
f.mu.Lock() | ||
defer f.mu.Unlock() | ||
now := time.Now() | ||
keysToDelete := make([]distributed_execution.FragmentKey, 0) | ||
for key, entry := range f.mappings { | ||
if now.Sub(entry.createdAt) > f.expiration { | ||
keysToDelete = append(keysToDelete, key) | ||
} | ||
} | ||
for _, key := range keysToDelete { | ||
delete(f.mappings, key) | ||
} | ||
} | ||
|
||
func (f *FragmentTable) periodicCleanup() { | ||
ticker := time.NewTicker(f.expiration / 2) | ||
defer ticker.Stop() | ||
for range ticker.C { | ||
f.cleanupExpired() | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.