Skip to content

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

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
116 changes: 116 additions & 0 deletions docs/configuration/config-file-reference.md
Original file line number Diff line number Diff line change
Expand Up @@ -2940,6 +2940,7 @@ The `consul_config` configures the consul client. The supported CLI flags `<pref
- `distributor.ha-tracker`
- `distributor.ring`
- `parquet-converter.ring`
- `querier.ring`
- `ruler.ring`
- `store-gateway.sharding-ring`

Expand Down Expand Up @@ -3277,6 +3278,7 @@ The `etcd_config` configures the etcd client. The supported CLI flags `<prefix>`
- `distributor.ha-tracker`
- `distributor.ring`
- `parquet-converter.ring`
- `querier.ring`
- `ruler.ring`
- `store-gateway.sharding-ring`

Expand Down Expand Up @@ -3480,6 +3482,120 @@ grpc_client_config:
# using default gRPC client connect timeout 20s.
# CLI flag: -querier.frontend-client.connect-timeout
[connect_timeout: <duration> | default = 5s]

ring:
# The key-value store used to share the hash ring across multiple instances.
kvstore:
# Backend storage to use for the ring. Supported values are: consul, etcd,
# inmemory, memberlist, multi.
# CLI flag: -querier.ring.store
[store: <string> | default = "consul"]

# The prefix for the keys in the store. Should end with a /.
# CLI flag: -querier.ring.prefix
[prefix: <string> | default = "querier/"]

dynamodb:
# Region to access dynamodb.
# CLI flag: -querier.ring.dynamodb.region
[region: <string> | default = ""]

# Table name to use on dynamodb.
# CLI flag: -querier.ring.dynamodb.table-name
[table_name: <string> | default = ""]

# Time to expire items on dynamodb.
# CLI flag: -querier.ring.dynamodb.ttl-time
[ttl: <duration> | default = 0s]

# Time to refresh local ring with information on dynamodb.
# CLI flag: -querier.ring.dynamodb.puller-sync-time
[puller_sync_time: <duration> | default = 1m]

# Maximum number of retries for DDB KV CAS.
# CLI flag: -querier.ring.dynamodb.max-cas-retries
[max_cas_retries: <int> | default = 10]

# Timeout of dynamoDbClient requests. Default is 2m.
# CLI flag: -querier.ring.dynamodb.timeout
[timeout: <duration> | default = 2m]

# The consul_config configures the consul client.
# The CLI flags prefix for this block config is: querier.ring
[consul: <consul_config>]

# The etcd_config configures the etcd client.
# The CLI flags prefix for this block config is: querier.ring
[etcd: <etcd_config>]

multi:
# Primary backend storage used by multi-client.
# CLI flag: -querier.ring.multi.primary
[primary: <string> | default = ""]

# Secondary backend storage used by multi-client.
# CLI flag: -querier.ring.multi.secondary
[secondary: <string> | default = ""]

# Mirror writes to secondary store.
# CLI flag: -querier.ring.multi.mirror-enabled
[mirror_enabled: <boolean> | default = false]

# Timeout for storing value to secondary store.
# CLI flag: -querier.ring.multi.mirror-timeout
[mirror_timeout: <duration> | default = 2s]

# Period at which to heartbeat to the ring. 0 = disabled.
# CLI flag: -querier.ring.heartbeat-period
[heartbeat_period: <duration> | default = 5s]

# The heartbeat timeout after which rulers are considered unhealthy within the
# ring. 0 = never (timeout disabled).
# CLI flag: -querier.ring.heartbeat-timeout
[heartbeat_timeout: <duration> | default = 1m]

# EXPERIMENTAL: The replication factor to use when loading rule groups for API
# HA.
# CLI flag: -querier.ring.replication-factor
[replication_factor: <int> | default = 1]

# EXPERIMENTAL: True to enable zone-awareness and load rule groups across
# different availability zones for API HA.
# CLI flag: -querier.ring.zone-awareness-enabled
[zone_awareness_enabled: <boolean> | default = false]

# EXPERIMENTAL: File path where tokens are stored. If empty, tokens are not
# stored at shutdown and restored at startup.
# CLI flag: -querier.ring.tokens-file-path
[tokens_file_path: <string> | default = ""]

# Set to true to enable ring detailed metrics. These metrics provide detailed
# information, such as token count and ownership per tenant. Disabling them
# can significantly decrease the number of metrics emitted.
# CLI flag: -querier.ring.detailed-metrics-enabled
[detailed_metrics_enabled: <boolean> | default = true]

# The sleep seconds when ruler is shutting down. Need to be close to or larger
# than KV Store information propagation delay
# CLI flag: -querier.ring.final-sleep
[final_sleep: <duration> | default = 0s]

# Keep instance in the ring on shut down.
# CLI flag: -querier.ring.keep-instance-in-the-ring-on-shutdown
[keep_instance_in_the_ring_on_shutdown: <boolean> | default = false]

# Name of network interface to read address from.
# CLI flag: -querier.ring.instance-interface-names
[instance_interface_names: <list of string> | default = [eth0 en0]]

# The availability zone where this instance is running. Required if
# zone-awareness is enabled.
# CLI flag: -querier.ring.instance-availability-zone
[instance_availability_zone: <string> | default = ""]

# Number of tokens for each ruler.
# CLI flag: -querier.ring.num-tokens
[num_tokens: <int> | default = 128]
```

### `ingester_config`
Expand Down
6 changes: 4 additions & 2 deletions pkg/cortex/modules.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,6 @@ import (
"context"
"flag"
"fmt"

"log/slog"
"net/http"
"runtime"
Expand Down Expand Up @@ -414,6 +413,9 @@ 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

t.Cfg.Worker.QuerierRing.ListenPort = t.Cfg.Server.GRPCListenPort

return querier_worker.NewQuerierWorker(t.Cfg.Worker, httpgrpc_server.NewServer(internalQuerierRouter), util_log.Logger, prometheus.DefaultRegisterer)
}

Expand Down Expand Up @@ -813,7 +815,7 @@ func (t *Cortex) initQueryScheduler() (services.Service, error) {
tenant.WithDefaultResolver(tenantfederation.NewRegexValidator())
}

s, err := scheduler.NewScheduler(t.Cfg.QueryScheduler, t.Overrides, util_log.Logger, prometheus.DefaultRegisterer)
s, err := scheduler.NewScheduler(t.Cfg.QueryScheduler, t.Overrides, util_log.Logger, prometheus.DefaultRegisterer, t.Cfg.Querier.DistributedExecEnabled)
if err != nil {
return nil, errors.Wrap(err, "query-scheduler init")
}
Expand Down
33 changes: 33 additions & 0 deletions pkg/distributed_execution/fragment_key.go
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
}

// 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
}
49 changes: 49 additions & 0 deletions pkg/distributed_execution/plan_fragments/fragmenter.go
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{}
}
46 changes: 46 additions & 0 deletions pkg/distributed_execution/plan_fragments/fragmenter_test.go
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))
})
}
}
3 changes: 2 additions & 1 deletion pkg/querier/querier.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,8 @@ type Config struct {
ParquetQueryableShardCacheSize int `yaml:"parquet_queryable_shard_cache_size"`
ParquetQueryableDefaultBlockStore string `yaml:"parquet_queryable_default_block_store"`
ParquetQueryableFallbackDisabled bool `yaml:"parquet_queryable_fallback_disabled"`
DistributedExecEnabled bool `yaml:"distributed_exec_enabled" doc:"hidden"`

DistributedExecEnabled bool `yaml:"distributed_exec_enabled" doc:"hidden"`
}

var (
Expand Down
6 changes: 4 additions & 2 deletions pkg/querier/worker/scheduler_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import (
"github.com/cortexproject/cortex/pkg/util/services"
)

func newSchedulerProcessor(cfg Config, handler RequestHandler, log log.Logger, reg prometheus.Registerer) (*schedulerProcessor, []services.Service) {
func newSchedulerProcessor(cfg Config, handler RequestHandler, log log.Logger, reg prometheus.Registerer, querierAddress string) (*schedulerProcessor, []services.Service) {
p := &schedulerProcessor{
log: log,
handler: handler,
Expand All @@ -47,6 +47,7 @@ func newSchedulerProcessor(cfg Config, handler RequestHandler, log log.Logger, r
Help: "Time spend doing requests to frontend.",
Buckets: prometheus.ExponentialBuckets(0.001, 4, 6),
}, []string{"operation", "status_code"}),
querierAddress: querierAddress,
}

frontendClientsGauge := promauto.With(reg).NewGauge(prometheus.GaugeOpts{
Expand All @@ -71,6 +72,7 @@ type schedulerProcessor struct {
grpcConfig grpcclient.Config
maxMessageSize int
querierID string
querierAddress string

frontendPool *client.Pool
frontendClientRequestDuration *prometheus.HistogramVec
Expand All @@ -97,7 +99,7 @@ func (sp *schedulerProcessor) processQueriesOnSingleStream(ctx context.Context,
for backoff.Ongoing() {
c, err := schedulerClient.QuerierLoop(ctx)
if err == nil {
err = c.Send(&schedulerpb.QuerierToScheduler{QuerierID: sp.querierID})
err = c.Send(&schedulerpb.QuerierToScheduler{QuerierID: sp.querierID, QuerierAddress: sp.querierAddress})
}

if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/querier/worker/scheduler_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,7 +144,7 @@ func Test_ToShowNotPanic_RelatedIssue6599(t *testing.T) {
go stat.AddFetchedChunkBytes(10)
}).Return(&httpgrpc.HTTPResponse{}, nil)

sp, _ := newSchedulerProcessor(cfg, requestHandler, log.NewNopLogger(), nil)
sp, _ := newSchedulerProcessor(cfg, requestHandler, log.NewNopLogger(), nil, "")
schedulerClient := &mockSchedulerForQuerierClient{}
schedulerClient.On("QuerierLoop", mock.Anything, mock.Anything).Return(querierLoopClient, nil)

Expand Down
Loading
Loading