Skip to content

Commit 9c437b3

Browse files
committed
Create separate config for ruler partial data
Signed-off-by: Justin Jung <jungjust@amazon.com>
1 parent c191b77 commit 9c437b3

File tree

8 files changed

+44
-38
lines changed

8 files changed

+44
-38
lines changed

pkg/cortex/modules.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -258,7 +258,7 @@ func (t *Cortex) initQueryable() (serv services.Service, err error) {
258258
querierRegisterer := prometheus.WrapRegistererWith(prometheus.Labels{"engine": "querier"}, prometheus.DefaultRegisterer)
259259

260260
// Create a querier queryable and PromQL engine
261-
t.QuerierQueryable, t.ExemplarQueryable, t.QuerierEngine = querier.New(t.Cfg.Querier, t.Overrides, t.Distributor, t.StoreQueryables, querierRegisterer, util_log.Logger)
261+
t.QuerierQueryable, t.ExemplarQueryable, t.QuerierEngine = querier.New(t.Cfg.Querier, t.Overrides, t.Distributor, t.StoreQueryables, querierRegisterer, util_log.Logger, t.Overrides.QueryPartialData)
262262

263263
// Use distributor as default MetadataQuerier
264264
t.MetadataQuerier = t.Distributor
@@ -623,7 +623,7 @@ func (t *Cortex) initRuler() (serv services.Service, err error) {
623623
} else {
624624
rulerRegisterer := prometheus.WrapRegistererWith(prometheus.Labels{"engine": "ruler"}, prometheus.DefaultRegisterer)
625625
// TODO: Consider wrapping logger to differentiate from querier module logger
626-
queryable, _, engine := querier.New(t.Cfg.Querier, t.Overrides, t.Distributor, t.StoreQueryables, rulerRegisterer, util_log.Logger)
626+
queryable, _, engine := querier.New(t.Cfg.Querier, t.Overrides, t.Distributor, t.StoreQueryables, rulerRegisterer, util_log.Logger, t.Overrides.RulesPartialData)
627627

628628
managerFactory := ruler.DefaultTenantManagerFactory(t.Cfg.Ruler, t.Distributor, queryable, engine, t.Overrides, metrics, prometheus.DefaultRegisterer)
629629
manager, err = ruler.NewDefaultMultiTenantManager(t.Cfg.Ruler, t.Overrides, managerFactory, metrics, prometheus.DefaultRegisterer, util_log.Logger)

pkg/querier/distributor_queryable.go

Lines changed: 12 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@ import (
2222
"github.com/cortexproject/cortex/pkg/util"
2323
"github.com/cortexproject/cortex/pkg/util/chunkcompat"
2424
"github.com/cortexproject/cortex/pkg/util/spanlogger"
25-
"github.com/cortexproject/cortex/pkg/util/validation"
2625
)
2726

2827
// Distributor is the read interface to the distributor, made an interface here
@@ -39,14 +38,14 @@ type Distributor interface {
3938
MetricsMetadata(ctx context.Context) ([]scrape.MetricMetadata, error)
4039
}
4140

42-
func newDistributorQueryable(distributor Distributor, streamingMetdata bool, labelNamesWithMatchers bool, iteratorFn chunkIteratorFunc, queryIngestersWithin time.Duration, limits *validation.Overrides) QueryableWithFilter {
41+
func newDistributorQueryable(distributor Distributor, streamingMetdata bool, labelNamesWithMatchers bool, iteratorFn chunkIteratorFunc, queryIngestersWithin time.Duration, isPartialDataEnabled partialdata.IsCfgEnabledFunc) QueryableWithFilter {
4342
return distributorQueryable{
4443
distributor: distributor,
4544
streamingMetdata: streamingMetdata,
4645
labelNamesWithMatchers: labelNamesWithMatchers,
4746
iteratorFn: iteratorFn,
4847
queryIngestersWithin: queryIngestersWithin,
49-
limits: limits,
48+
isPartialDataEnabled: isPartialDataEnabled,
5049
}
5150
}
5251

@@ -56,7 +55,7 @@ type distributorQueryable struct {
5655
labelNamesWithMatchers bool
5756
iteratorFn chunkIteratorFunc
5857
queryIngestersWithin time.Duration
59-
limits *validation.Overrides
58+
isPartialDataEnabled partialdata.IsCfgEnabledFunc
6059
}
6160

6261
func (d distributorQueryable) Querier(mint, maxt int64) (storage.Querier, error) {
@@ -68,7 +67,7 @@ func (d distributorQueryable) Querier(mint, maxt int64) (storage.Querier, error)
6867
labelNamesMatchers: d.labelNamesWithMatchers,
6968
chunkIterFn: d.iteratorFn,
7069
queryIngestersWithin: d.queryIngestersWithin,
71-
limits: d.limits,
70+
isPartialDataEnabled: d.isPartialDataEnabled,
7271
}, nil
7372
}
7473

@@ -84,7 +83,7 @@ type distributorQuerier struct {
8483
labelNamesMatchers bool
8584
chunkIterFn chunkIteratorFunc
8685
queryIngestersWithin time.Duration
87-
limits *validation.Overrides
86+
isPartialDataEnabled partialdata.IsCfgEnabledFunc
8887
}
8988

9089
// Select implements storage.Querier interface.
@@ -117,7 +116,7 @@ func (q *distributorQuerier) Select(ctx context.Context, sortSeries bool, sp *st
117116
}
118117
}
119118

120-
partialDataEnabled := q.isPartialDataEnabled(ctx)
119+
partialDataEnabled := q.partialDataEnabled(ctx)
121120

122121
// In the recent versions of Prometheus, we pass in the hint but with Func set to "series".
123122
// See: https://github.com/prometheus/prometheus/pull/8050
@@ -191,7 +190,7 @@ func (q *distributorQuerier) LabelValues(ctx context.Context, name string, hints
191190
err error
192191
)
193192

194-
partialDataEnabled := q.isPartialDataEnabled(ctx)
193+
partialDataEnabled := q.partialDataEnabled(ctx)
195194

196195
if q.streamingMetadata {
197196
lvs, err = q.distributor.LabelValuesForLabelNameStream(ctx, model.Time(q.mint), model.Time(q.maxt), model.LabelName(name), hints, partialDataEnabled, matchers...)
@@ -203,8 +202,10 @@ func (q *distributorQuerier) LabelValues(ctx context.Context, name string, hints
203202
}
204203

205204
func (q *distributorQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) {
205+
partialDataEnabled := q.partialDataEnabled(ctx)
206+
206207
if len(matchers) > 0 && !q.labelNamesMatchers {
207-
return q.labelNamesWithMatchers(ctx, hints, q.isPartialDataEnabled(ctx), matchers...)
208+
return q.labelNamesWithMatchers(ctx, hints, partialDataEnabled, matchers...)
208209
}
209210

210211
log, ctx := spanlogger.New(ctx, "distributorQuerier.LabelNames")
@@ -215,8 +216,6 @@ func (q *distributorQuerier) LabelNames(ctx context.Context, hints *storage.Labe
215216
err error
216217
)
217218

218-
partialDataEnabled := q.isPartialDataEnabled(ctx)
219-
220219
if q.streamingMetadata {
221220
ln, err = q.distributor.LabelNamesStream(ctx, model.Time(q.mint), model.Time(q.maxt), hints, partialDataEnabled, matchers...)
222221
} else {
@@ -266,13 +265,13 @@ func (q *distributorQuerier) Close() error {
266265
return nil
267266
}
268267

269-
func (q *distributorQuerier) isPartialDataEnabled(ctx context.Context) bool {
268+
func (q *distributorQuerier) partialDataEnabled(ctx context.Context) bool {
270269
userID, err := tenant.TenantID(ctx)
271270
if err != nil {
272271
return false
273272
}
274273

275-
return q.limits != nil && q.limits.QueryPartialData(userID)
274+
return q.isPartialDataEnabled != nil && q.isPartialDataEnabled(userID)
276275
}
277276

278277
type distributorExemplarQueryable struct {

pkg/querier/distributor_queryable_test.go

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -179,11 +179,9 @@ func TestIngesterStreaming(t *testing.T) {
179179
d.On("QueryStream", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(queryResponse, partialdata.Error{})
180180
}
181181

182-
limits := validation.Limits{QueryPartialData: partialDataEnabled}
183-
overrides, err := validation.NewOverrides(limits, nil)
184-
require.NoError(t, err)
185-
186-
queryable := newDistributorQueryable(d, true, true, batch.NewChunkMergeIterator, 0, overrides)
182+
queryable := newDistributorQueryable(d, true, true, batch.NewChunkMergeIterator, 0, func(string) bool {
183+
return partialDataEnabled
184+
})
187185
querier, err := queryable.Querier(mint, maxt)
188186
require.NoError(t, err)
189187

pkg/querier/partialdata/partia_data.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,8 @@ import (
44
"errors"
55
)
66

7+
type IsCfgEnabledFunc func(userID string) bool
8+
79
const ErrorMsg string = "Query result may contain partial data."
810

911
type Error struct{}

pkg/querier/querier.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import (
2626

2727
"github.com/cortexproject/cortex/pkg/querier/batch"
2828
"github.com/cortexproject/cortex/pkg/querier/lazyquery"
29+
"github.com/cortexproject/cortex/pkg/querier/partialdata"
2930
querier_stats "github.com/cortexproject/cortex/pkg/querier/stats"
3031
"github.com/cortexproject/cortex/pkg/tenant"
3132
"github.com/cortexproject/cortex/pkg/util"
@@ -172,10 +173,10 @@ func getChunksIteratorFunction(_ Config) chunkIteratorFunc {
172173
}
173174

174175
// New builds a queryable and promql engine.
175-
func New(cfg Config, limits *validation.Overrides, distributor Distributor, stores []QueryableWithFilter, reg prometheus.Registerer, logger log.Logger) (storage.SampleAndChunkQueryable, storage.ExemplarQueryable, promql.QueryEngine) {
176+
func New(cfg Config, limits *validation.Overrides, distributor Distributor, stores []QueryableWithFilter, reg prometheus.Registerer, logger log.Logger, isPartialDataEnabled partialdata.IsCfgEnabledFunc) (storage.SampleAndChunkQueryable, storage.ExemplarQueryable, promql.QueryEngine) {
176177
iteratorFunc := getChunksIteratorFunction(cfg)
177178

178-
distributorQueryable := newDistributorQueryable(distributor, cfg.IngesterMetadataStreaming, cfg.IngesterLabelNamesWithMatchers, iteratorFunc, cfg.QueryIngestersWithin, limits)
179+
distributorQueryable := newDistributorQueryable(distributor, cfg.IngesterMetadataStreaming, cfg.IngesterLabelNamesWithMatchers, iteratorFunc, cfg.QueryIngestersWithin, isPartialDataEnabled)
179180

180181
ns := make([]QueryableWithFilter, len(stores))
181182
for ix, s := range stores {

pkg/querier/querier_test.go

Lines changed: 13 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -580,7 +580,7 @@ func TestQuerier(t *testing.T) {
580580
require.NoError(t, err)
581581

582582
queryables := []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}
583-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
583+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
584584
testRangeQuery(t, queryable, queryEngine, through, query, enc)
585585
})
586586
}
@@ -602,7 +602,7 @@ func TestQuerierMetric(t *testing.T) {
602602
queryables := []QueryableWithFilter{}
603603
r := prometheus.NewRegistry()
604604
reg := prometheus.WrapRegistererWith(prometheus.Labels{"engine": "querier"}, r)
605-
New(cfg, overrides, distributor, queryables, reg, log.NewNopLogger())
605+
New(cfg, overrides, distributor, queryables, reg, log.NewNopLogger(), nil)
606606
assert.NoError(t, promutil.GatherAndCompare(r, strings.NewReader(`
607607
# HELP cortex_max_concurrent_queries The maximum number of concurrent queries.
608608
# TYPE cortex_max_concurrent_queries gauge
@@ -684,7 +684,7 @@ func TestNoHistoricalQueryToIngester(t *testing.T) {
684684
require.NoError(t, err)
685685

686686
ctx := user.InjectOrgID(context.Background(), "0")
687-
queryable, _, _ := New(cfg, overrides, distributor, []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}, nil, log.NewNopLogger())
687+
queryable, _, _ := New(cfg, overrides, distributor, []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}, nil, log.NewNopLogger(), nil)
688688
query, err := queryEngine.NewRangeQuery(ctx, queryable, nil, "dummy", c.mint, c.maxt, 1*time.Minute)
689689
require.NoError(t, err)
690690

@@ -778,7 +778,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryIntoFuture(t *testing.T) {
778778

779779
ctx := user.InjectOrgID(context.Background(), "0")
780780
queryables := []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}
781-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
781+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
782782
query, err := queryEngine.NewRangeQuery(ctx, queryable, nil, "dummy", c.queryStartTime, c.queryEndTime, time.Minute)
783783
require.NoError(t, err)
784784

@@ -871,7 +871,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLength(t *testing.T) {
871871
distributor := &emptyDistributor{}
872872

873873
queryables := []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}
874-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
874+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
875875

876876
queryEngine := promql.NewEngine(opts)
877877
ctx := user.InjectOrgID(context.Background(), "test")
@@ -910,7 +910,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLength_Series(t *testing.T) {
910910
distributor := &emptyDistributor{}
911911

912912
queryables := []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}
913-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
913+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
914914

915915
ctx := user.InjectOrgID(context.Background(), "test")
916916
now := time.Now()
@@ -969,7 +969,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLength_Labels(t *testing.T) {
969969
distributor := &emptyDistributor{}
970970

971971
queryables := []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}
972-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
972+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
973973

974974
ctx := user.InjectOrgID(context.Background(), "test")
975975

@@ -1120,7 +1120,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLookback(t *testing.T) {
11201120
distributor := &MockDistributor{}
11211121
distributor.On("QueryStream", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&client.QueryStreamResponse{}, nil)
11221122

1123-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
1123+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
11241124
require.NoError(t, err)
11251125

11261126
query, err := queryEngine.NewRangeQuery(ctx, queryable, nil, testData.query, testData.queryStartTime, testData.queryEndTime, time.Minute)
@@ -1149,7 +1149,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLookback(t *testing.T) {
11491149
distributor.On("MetricsForLabelMatchers", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]model.Metric{}, nil)
11501150
distributor.On("MetricsForLabelMatchersStream", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]model.Metric{}, nil)
11511151

1152-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
1152+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
11531153
q, err := queryable.Querier(util.TimeToMillis(testData.queryStartTime), util.TimeToMillis(testData.queryEndTime))
11541154
require.NoError(t, err)
11551155

@@ -1190,7 +1190,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLookback(t *testing.T) {
11901190
distributor.On("LabelNames", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]string{}, nil)
11911191
distributor.On("LabelNamesStream", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]string{}, nil)
11921192

1193-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
1193+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
11941194
q, err := queryable.Querier(util.TimeToMillis(testData.queryStartTime), util.TimeToMillis(testData.queryEndTime))
11951195
require.NoError(t, err)
11961196

@@ -1218,7 +1218,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLookback(t *testing.T) {
12181218
distributor.On("MetricsForLabelMatchers", mock.Anything, mock.Anything, mock.Anything, mock.Anything, matchers).Return([]model.Metric{}, nil)
12191219
distributor.On("MetricsForLabelMatchersStream", mock.Anything, mock.Anything, mock.Anything, mock.Anything, matchers).Return([]model.Metric{}, nil)
12201220

1221-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
1221+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
12221222
q, err := queryable.Querier(util.TimeToMillis(testData.queryStartTime), util.TimeToMillis(testData.queryEndTime))
12231223
require.NoError(t, err)
12241224

@@ -1245,7 +1245,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLookback(t *testing.T) {
12451245
distributor.On("LabelValuesForLabelName", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]string{}, nil)
12461246
distributor.On("LabelValuesForLabelNameStream", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]string{}, nil)
12471247

1248-
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger())
1248+
queryable, _, _ := New(cfg, overrides, distributor, queryables, nil, log.NewNopLogger(), nil)
12491249
q, err := queryable.Querier(util.TimeToMillis(testData.queryStartTime), util.TimeToMillis(testData.queryEndTime))
12501250
require.NoError(t, err)
12511251

@@ -1582,7 +1582,7 @@ func TestShortTermQueryToLTS(t *testing.T) {
15821582
overrides, err := validation.NewOverrides(DefaultLimitsConfig(), nil)
15831583
require.NoError(t, err)
15841584

1585-
queryable, _, _ := New(cfg, overrides, distributor, []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}, nil, log.NewNopLogger())
1585+
queryable, _, _ := New(cfg, overrides, distributor, []QueryableWithFilter{UseAlwaysQueryable(NewMockStoreQueryable(chunkStore))}, nil, log.NewNopLogger(), nil)
15861586
ctx := user.InjectOrgID(context.Background(), "0")
15871587
query, err := engine.NewRangeQuery(ctx, queryable, nil, "dummy", c.mint, c.maxt, 1*time.Minute)
15881588
require.NoError(t, err)

pkg/ruler/ruler_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -223,7 +223,7 @@ func testQueryableFunc(querierTestConfig *querier.TestConfig, reg prometheus.Reg
223223
querierTestConfig.Cfg.ActiveQueryTrackerDir = ""
224224

225225
overrides, _ := validation.NewOverrides(querier.DefaultLimitsConfig(), nil)
226-
q, _, _ := querier.New(querierTestConfig.Cfg, overrides, querierTestConfig.Distributor, querierTestConfig.Stores, reg, logger)
226+
q, _, _ := querier.New(querierTestConfig.Cfg, overrides, querierTestConfig.Distributor, querierTestConfig.Stores, reg, logger, nil)
227227
return func(mint, maxt int64) (storage.Querier, error) {
228228
return q.Querier(mint, maxt)
229229
}

pkg/util/validation/limits.go

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -171,7 +171,8 @@ type Limits struct {
171171
MaxCacheFreshness model.Duration `yaml:"max_cache_freshness" json:"max_cache_freshness"`
172172
MaxQueriersPerTenant float64 `yaml:"max_queriers_per_tenant" json:"max_queriers_per_tenant"`
173173
QueryVerticalShardSize int `yaml:"query_vertical_shard_size" json:"query_vertical_shard_size" doc:"hidden"`
174-
QueryPartialData bool `yaml:"query_partial_data" json:"query_partial_data" doc:"nocli|description=Enable to allow partial query data from 1 zone to be returned with a warning message.|default=false"`
174+
QueryPartialData bool `yaml:"query_partial_data" json:"query_partial_data" doc:"nocli|description=Enable to allow queries to be evaluated with data from a single zone, if other zones are not available.|default=false"`
175+
RulesPartialData bool `yaml:"rules_partial_data" json:"rules_partial_data" doc:"nocli|description=Enable to allow rules to be evaluated with data from a single zone, if other zones are not available.|default=false"`
175176

176177
// Query Frontend / Scheduler enforced limits.
177178
MaxOutstandingPerTenant int `yaml:"max_outstanding_requests_per_tenant" json:"max_outstanding_requests_per_tenant"`
@@ -727,11 +728,16 @@ func (o *Overrides) QueryVerticalShardSize(userID string) int {
727728
return o.GetOverridesForUser(userID).QueryVerticalShardSize
728729
}
729730

730-
// QueryPartialData returns whether query result from a single zone is returned as a possible partial result.
731+
// QueryPartialData returns whether query may be evaluated with data from a single zone, if other zones are not available.
731732
func (o *Overrides) QueryPartialData(userID string) bool {
732733
return o.GetOverridesForUser(userID).QueryPartialData
733734
}
734735

736+
// RulesPartialData returns whether rule may be evaluated with data from a single zone, if other zones are not available.
737+
func (o *Overrides) RulesPartialData(userID string) bool {
738+
return o.GetOverridesForUser(userID).RulesPartialData
739+
}
740+
735741
// MaxQueryParallelism returns the limit to the number of split queries the
736742
// frontend will process in parallel.
737743
func (o *Overrides) MaxQueryParallelism(userID string) int {

0 commit comments

Comments
 (0)