Skip to content

Conversation

rubywtl
Copy link
Contributor

@rubywtl rubywtl commented Aug 13, 2025

What this PR does:
This PR introduces a Fragmenter interface that splits logical query plans into fragments when distributed execution is enabled. The Fragmenter appends metadata to each fragment for tracking, which the scheduler then uses to route fragments to appropriate queriers. The scheduler maintains a mapping between fragments and querier addresses to track fragment locations across the distributed system.

Which issue(s) this PR fixes:

Checklist

  • Tests updated
  • Documentation added
  • CHANGELOG.md updated - the order of entries should be [CHANGE], [FEATURE], [ENHANCEMENT], [BUGFIX]

@@ -414,7 +416,12 @@ func (t *Cortex) initQuerier() (serv services.Service, err error) {

t.Cfg.Worker.MaxConcurrentRequests = t.Cfg.Querier.MaxConcurrent
t.Cfg.Worker.TargetHeaders = t.Cfg.API.HTTPRequestHeadersToLog
return querier_worker.NewQuerierWorker(t.Cfg.Worker, httpgrpc_server.NewServer(internalQuerierRouter), util_log.Logger, prometheus.DefaultRegisterer)
ipAddr, err := ring.GetInstanceAddr(t.Cfg.Alertmanager.ShardingRing.InstanceAddr, t.Cfg.Alertmanager.ShardingRing.InstanceInterfaceNames, util_log.Logger)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why using alertmanager config here

Copy link
Contributor Author

@rubywtl rubywtl Aug 14, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The gPRC params I needed are under RingConfig struct, which is called ShardedRing here, but it doesn't exist under querier

[update] I will add new field (ring configs) for querier 👍

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Umm I don't think we want to add a Ring for querier. We just need the configurations for the addresses and interface, etc

@@ -0,0 +1,21 @@
package distributed_execution

type FragmentKey struct {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add comments for public exposed types

fragmentID uint64
}

func MakeFragmentKey(queryID uint64, fragmentID uint64) *FragmentKey {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is no need to return a pointer for this I think.


type Fragmenter interface {
Fragment(node logicalplan.Node) ([]Fragment, error)
getNewID() uint64
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this need to be part of the interface? It is very weird to have 1 method in the interface to be public and another one is private.
I would remove this from the interface

}

func (f *DummyFragmenter) getNewID() uint64 {
return 1 // for dummy plan_fragments testing
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If it is just for testing, you can just hardcode to 1 in the Fragment function

f.mappings[*key] = addr
}

func (f *FragmentTable) GetMappings(queryID uint64, fragmentIDs []uint64) ([]string, bool) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we find a more descriptive name? It took me a while to understand what mapping it is. If it is getting child querier addresses we find a better name

defer f.mu.Unlock()

keysToDelete := make([]distributed_execution.FragmentKey, 0)
for key := range f.mappings {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking at the methods you have, is it easier to change mappings from mappings map[distributed_execution.FragmentKey]string to map[uint64]map[uint64]string?

You can find the map by just a lookup

Copy link
Contributor Author

@rubywtl rubywtl Aug 14, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

True, but I made the FragmentKey struct so that it is easier to maintain (for example: if we ever want to change the types for the IDs or add more fields, we dont have to go through the codebase to fix it), and the code will be easier to understand (more literal). This fragment key type is also reused for remote nodes and child-root execution accesses to result cache in future PRs.


import "github.com/thanos-io/promql-engine/logicalplan"

type Fragmenter interface {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is better to move the Fragmenter to distributed_execution as fragmentation is specific to remote distribution.

The fragment table can be just moved to scheduler folder

@rubywtl rubywtl force-pushed the scheduler/logicalplan_fragment_coordination branch 3 times, most recently from 942674c to 89e8021 Compare August 20, 2025 16:44
@rubywtl rubywtl force-pushed the scheduler/logicalplan_fragment_coordination branch from 89e8021 to 67eff93 Compare August 21, 2025 21:39

cfg.InstanceInterfaceNames = []string{"eth0", "en0"}
f.Var((*flagext.StringSlice)(&cfg.InstanceInterfaceNames), "querier.instance-interface-names", "Name of network interface to read address from.")
f.StringVar(&cfg.InstanceAddr, "querier.instance-addr", "", "IP address to advertise in the ring.")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is no ring for Querier

}
}

func (f *FragmentTable) AddMapping(queryID uint64, fragmentID uint64, addr string) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we find a better name other than mapping?

return "", false
}

func (f *FragmentTable) ClearMappings(queryID uint64) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto


// queryKey <--> fragment-ids lookup table allows faster cancellation of the whole query
// compared to traversing through the pending requests to find matching fragments
queryToFragmentsLookUp map[queryKey][]uint64
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's find a better name for this. LookUp is not a correct name

}

type requestKey struct {
// additional layer to improve efficiency of deleting fragments of logical query plans
// while maintaining previous logics
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment is confusing. Which previous logic this struct maintains?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Previous logic is to cancel a query by its queryID and frontend address, but now there are multiple fragments under one queryID, and traversing through the pending request queue and checking the queryID is in-efficient, so I added an extra layer of mapping to keep track of the fragment IDs under the same queryKey.

return nil, err
}

fragmenter := plan_fragments.NewDummyFragmenter()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this method belongs to Scheduler, why are we creating a new dummy fragmenter everytime?
Should it be part of the scheduler itself

}

func (s *Scheduler) getPlanFromHTTPRequest(req *httpgrpc.HTTPRequest) ([]byte, error) {
if req.Body == nil {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see why this needs to be a method of scheduler itself

}

return nil
}(); err != nil {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we try to clean up the code a little bit? I see a lot of code duplicating here.

for _, childID := range req.fragment.ChildIDs {
addr, ok := s.fragmentTable.GetChildAddr(req.queryID, childID)
if !ok {
return
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to return some error here if missing child addr?

@@ -9,6 +9,7 @@ import "github.com/weaveworks/common/httpgrpc/httpgrpc.proto";

option (gogoproto.marshaler_all) = true;
option (gogoproto.unmarshaler_all) = true;
option (gogoproto.sizer_all) = true;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this required?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants