-
Notifications
You must be signed in to change notification settings - Fork 828
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
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,33 @@ | ||
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 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
|
||
} | ||
|
||
// 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 | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
package plan_fragments | ||
|
||
import "github.com/thanos-io/promql-engine/logicalplan" | ||
|
||
type Fragmenter interface { | ||
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{} | ||
} |
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)) | ||
}) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -3,7 +3,9 @@ package worker | |
import ( | ||
"context" | ||
"flag" | ||
"net" | ||
"os" | ||
"strconv" | ||
"sync" | ||
"time" | ||
|
||
|
@@ -14,7 +16,9 @@ import ( | |
"github.com/weaveworks/common/httpgrpc" | ||
"google.golang.org/grpc" | ||
|
||
"github.com/cortexproject/cortex/pkg/ring" | ||
"github.com/cortexproject/cortex/pkg/util" | ||
"github.com/cortexproject/cortex/pkg/util/flagext" | ||
"github.com/cortexproject/cortex/pkg/util/grpcclient" | ||
"github.com/cortexproject/cortex/pkg/util/services" | ||
) | ||
|
@@ -33,6 +37,10 @@ type Config struct { | |
GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config"` | ||
|
||
TargetHeaders []string `yaml:"-"` // Propagated by config. | ||
|
||
InstanceInterfaceNames []string `yaml:"instance_interface_names"` | ||
ListenPort int `yaml:"-"` | ||
InstanceAddr string `yaml:"instance_addr" doc:"hidden"` | ||
} | ||
|
||
func (cfg *Config) RegisterFlags(f *flag.FlagSet) { | ||
|
@@ -46,6 +54,10 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { | |
f.StringVar(&cfg.QuerierID, "querier.id", "", "Querier ID, sent to frontend service to identify requests from the same querier. Defaults to hostname.") | ||
|
||
cfg.GRPCClientConfig.RegisterFlagsWithPrefix("querier.frontend-client", "", f) | ||
|
||
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.") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There is no ring for Querier |
||
} | ||
|
||
func (cfg *Config) Validate(log log.Logger) error { | ||
|
@@ -109,7 +121,14 @@ func NewQuerierWorker(cfg Config, handler RequestHandler, log log.Logger, reg pr | |
level.Info(log).Log("msg", "Starting querier worker connected to query-scheduler", "scheduler", cfg.SchedulerAddress) | ||
|
||
address = cfg.SchedulerAddress | ||
processor, servs = newSchedulerProcessor(cfg, handler, log, reg) | ||
|
||
ipAddr, err := ring.GetInstanceAddr(cfg.InstanceAddr, cfg.InstanceInterfaceNames, log) | ||
if err != nil { | ||
return nil, err | ||
} | ||
querierAddr := net.JoinHostPort(ipAddr, strconv.Itoa(cfg.ListenPort)) | ||
|
||
processor, servs = newSchedulerProcessor(cfg, handler, log, reg, querierAddr) | ||
|
||
case cfg.FrontendAddress != "": | ||
level.Info(log).Log("msg", "Starting querier worker connected to query-frontend", "frontend", cfg.FrontendAddress) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,70 @@ | ||
package fragment_table | ||
|
||
import ( | ||
"sync" | ||
|
||
"github.com/cortexproject/cortex/pkg/distributed_execution" | ||
) | ||
|
||
type FragmentTable struct { | ||
mappings map[distributed_execution.FragmentKey]string | ||
mu sync.RWMutex | ||
} | ||
|
||
func NewFragmentTable() *FragmentTable { | ||
return &FragmentTable{ | ||
mappings: make(map[distributed_execution.FragmentKey]string), | ||
} | ||
} | ||
|
||
func (f *FragmentTable) AddMapping(queryID uint64, fragmentID uint64, addr string) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we find a better name other than mapping? |
||
f.mu.Lock() | ||
defer f.mu.Unlock() | ||
|
||
key := distributed_execution.MakeFragmentKey(queryID, fragmentID) | ||
f.mappings[key] = addr | ||
} | ||
|
||
func (f *FragmentTable) GetAllChildAddresses(queryID uint64, fragmentIDs []uint64) ([]string, bool) { | ||
f.mu.RLock() | ||
defer f.mu.RUnlock() | ||
|
||
addresses := make([]string, 0, len(fragmentIDs)) | ||
|
||
for _, fragmentID := range fragmentIDs { | ||
key := distributed_execution.MakeFragmentKey(queryID, fragmentID) | ||
if addr, ok := f.mappings[key]; ok { | ||
addresses = append(addresses, addr) | ||
} else { | ||
return nil, false | ||
} | ||
} | ||
return addresses, true | ||
} | ||
|
||
func (f *FragmentTable) GetChildAddr(queryID uint64, fragmentID uint64) (string, bool) { | ||
f.mu.RLock() | ||
defer f.mu.RUnlock() | ||
|
||
key := distributed_execution.MakeFragmentKey(queryID, fragmentID) | ||
if addr, ok := f.mappings[key]; ok { | ||
return addr, true | ||
} | ||
return "", false | ||
} | ||
|
||
func (f *FragmentTable) ClearMappings(queryID uint64) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ditto |
||
f.mu.Lock() | ||
defer f.mu.Unlock() | ||
|
||
keysToDelete := make([]distributed_execution.FragmentKey, 0) | ||
for key := range f.mappings { | ||
if key.GetQueryID() == queryID { | ||
keysToDelete = append(keysToDelete, key) | ||
} | ||
} | ||
|
||
for _, key := range keysToDelete { | ||
delete(f.mappings, key) | ||
} | ||
} |
There was a problem hiding this comment.
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