Skip to content

Commit

Permalink
Added new implementation that makes the distributor accept multiple H…
Browse files Browse the repository at this point in the history
…A pairs (cluster, replica) in the same requets/batch. This can be enabled with a new flag, accept_mixed_ha_samples, an will take effect only if accept_ha_samples is set to true.

Fixed test by reducing the number of ingesters to 2 and replication factor to 2. Added config reference.

Do not remove replica label if cluster label is not present.

Added more HA mixed replicas tests with no cluster and replica labels and with cluster label only.

Signed-off-by: eduardscaueru <edi_scaueru@yahoo.com>
  • Loading branch information
eduardscaueru committed Oct 21, 2024
1 parent e070ec6 commit db369f5
Show file tree
Hide file tree
Showing 5 changed files with 284 additions and 5 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@
* [FEATURE] Ruler: Minimize chances of missed rule group evaluations that can occur due to OOM kills, bad underlying nodes, or due to an unhealthy ruler that appears in the ring as healthy. This feature is enabled via `-ruler.enable-ha-evaluation` flag. #6129
* [FEATURE] Store Gateway: Add an in-memory chunk cache. #6245
* [FEATURE] Chunk Cache: Support multi level cache and add metrics. #6249
* [FEATURE] Distributor: Accept multiple HA Tracker pairs in the same request. #6256
* [ENHANCEMENT] Ingester: Add `blocks-storage.tsdb.wal-compression-type` to support zstd wal compression type. #6232
* [ENHANCEMENT] Query Frontend: Add info field to query response. #6207
* [ENHANCEMENT] Query Frontend: Add peakSample in query stats response. #6188
Expand Down
6 changes: 6 additions & 0 deletions docs/configuration/config-file-reference.md
Original file line number Diff line number Diff line change
Expand Up @@ -3163,6 +3163,12 @@ The `limits_config` configures default and per-tenant limits imposed by Cortex s
# CLI flag: -distributor.ha-tracker.enable-for-all-users
[accept_ha_samples: <boolean> | default = false]
# Flag to enable handling of samples with mixed external labels identifying
# replicas in an HA Prometheus setup. Supported only if
# -distributor.ha-tracker.enable-for-all-users is true.
# CLI flag: -distributor.ha-tracker.mixed-ha-samples
[accept_mixed_ha_samples: <boolean> | default = false]
# Prometheus label to look for in samples to identify a Prometheus HA cluster.
# CLI flag: -distributor.ha-tracker.cluster
[ha_cluster_label: <string> | default = "cluster"]
Expand Down
77 changes: 73 additions & 4 deletions pkg/distributor/distributor.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,11 @@ type InstanceLimits struct {
MaxInflightPushRequests int `yaml:"max_inflight_push_requests"`
}

type HAPair struct {
Cluster string
Replica string
}

// RegisterFlags adds the flags required to config this to the given FlagSet
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.PoolConfig.RegisterFlags(f)
Expand Down Expand Up @@ -649,10 +654,11 @@ func (d *Distributor) Push(ctx context.Context, req *cortexpb.WriteRequest) (*co
}

removeReplica := false
var validHAPairs map[HAPair]int
// Cache user limit with overrides so we spend less CPU doing locking. See issue #4904
limits := d.limits.GetOverridesForUser(userID)

if limits.AcceptHASamples && len(req.Timeseries) > 0 {
if limits.AcceptHASamples && len(req.Timeseries) > 0 && !limits.AcceptMixedHASamples {
cluster, replica := findHALabels(limits.HAReplicaLabel, limits.HAClusterLabel, req.Timeseries[0].Labels)
removeReplica, err = d.checkSample(ctx, userID, cluster, replica, limits)
if err != nil {
Expand All @@ -676,10 +682,14 @@ func (d *Distributor) Push(ctx context.Context, req *cortexpb.WriteRequest) (*co
if !removeReplica { // False, Nil
d.nonHASamples.WithLabelValues(userID).Add(float64(numFloatSamples + numHistogramSamples))
}
} else if limits.AcceptHASamples && len(req.Timeseries) > 0 && limits.AcceptMixedHASamples {
haPairs := findAllHAPairs(limits.HAReplicaLabel, limits.HAClusterLabel, req.Timeseries)
validHAPairs = d.getValidHAPairs(haPairs, ctx, userID, limits)
removeReplica = true
}

// A WriteRequest can only contain series or metadata but not both. This might change in the future.
seriesKeys, validatedTimeseries, validatedFloatSamples, validatedHistogramSamples, validatedExemplars, firstPartialErr, err := d.prepareSeriesKeys(ctx, req, userID, limits, removeReplica)
seriesKeys, validatedTimeseries, validatedFloatSamples, validatedHistogramSamples, validatedExemplars, firstPartialErr, err := d.prepareSeriesKeys(ctx, req, userID, limits, removeReplica, validHAPairs)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -833,7 +843,7 @@ func (d *Distributor) prepareMetadataKeys(req *cortexpb.WriteRequest, limits *va
return metadataKeys, validatedMetadata, firstPartialErr
}

func (d *Distributor) prepareSeriesKeys(ctx context.Context, req *cortexpb.WriteRequest, userID string, limits *validation.Limits, removeReplica bool) ([]uint32, []cortexpb.PreallocTimeseries, int, int, int, error, error) {
func (d *Distributor) prepareSeriesKeys(ctx context.Context, req *cortexpb.WriteRequest, userID string, limits *validation.Limits, removeReplica bool, validHAPairs map[HAPair]int) ([]uint32, []cortexpb.PreallocTimeseries, int, int, int, error, error) {
pSpan, _ := opentracing.StartSpanFromContext(ctx, "prepareSeriesKeys")
defer pSpan.Finish()

Expand All @@ -859,6 +869,19 @@ func (d *Distributor) prepareSeriesKeys(ctx context.Context, req *cortexpb.Write
// check each sample and discard if outside limits.
skipLabelNameValidation := d.cfg.SkipLabelNameValidation || req.GetSkipLabelNameValidation()
for _, ts := range req.Timeseries {
removeReplicaTs := removeReplica
if len(validHAPairs) > 0 {
validCode, cluster := isValidHAPair(ts.Labels, validHAPairs, limits.HAClusterLabel, limits.HAReplicaLabel)
if validCode == 1 {
d.dedupedSamples.WithLabelValues(userID, cluster).Add(float64(len(ts.Samples)))
continue
} else if validCode == 2 {
removeReplicaTs = false
d.nonHASamples.WithLabelValues(userID).Add(float64(len(ts.Samples)))
} else if validCode == 0 {
removeReplicaTs = true
}
}
// Use timestamp of latest sample in the series. If samples for series are not ordered, metric for user may be wrong.
if len(ts.Samples) > 0 {
latestSampleTimestampMs = max(latestSampleTimestampMs, ts.Samples[len(ts.Samples)-1].TimestampMs)
Expand Down Expand Up @@ -889,7 +912,7 @@ func (d *Distributor) prepareSeriesKeys(ctx context.Context, req *cortexpb.Write
// If we found both the cluster and replica labels, we only want to include the cluster label when
// storing series in Cortex. If we kept the replica label we would end up with another series for the same
// series we're trying to dedupe when HA tracking moves over to a different replica.
if removeReplica {
if removeReplicaTs {
removeLabel(limits.HAReplicaLabel, &ts.Labels)
}

Expand Down Expand Up @@ -1467,6 +1490,52 @@ func findHALabels(replicaLabel, clusterLabel string, labels []cortexpb.LabelAdap
return cluster, replica
}

func findAllHAPairs(replicaLabel, clusterLabel string, timeseries []cortexpb.PreallocTimeseries) map[HAPair]int {
var haPairs = make(map[HAPair]int)

for _, ts := range timeseries {
cluster, replica := findHALabels(replicaLabel, clusterLabel, ts.Labels)
if cluster != "" && replica != "" {
_, ok := haPairs[HAPair{Cluster: cluster, Replica: replica}]
if ok {
haPairs[HAPair{Cluster: cluster, Replica: replica}] += 1
} else {
haPairs[HAPair{Cluster: cluster, Replica: replica}] = 0
}
}
}

return haPairs
}

func (d *Distributor) getValidHAPairs(haPairs map[HAPair]int, ctx context.Context, userID string, limits *validation.Limits) map[HAPair]int {
var validHAPairs = make(map[HAPair]int)

for haPair, v := range haPairs {
removeReplica, err := d.checkSample(ctx, userID, haPair.Cluster, haPair.Replica, limits)
if err != nil || !removeReplica {
continue
}

validHAPairs[haPair] = v
}

return validHAPairs
}

func isValidHAPair(labels []cortexpb.LabelAdapter, validHAPairs map[HAPair]int, clusterLabel, replicaLabel string) (int, string) {
cluster, replica := findHALabels(replicaLabel, clusterLabel, labels)

if cluster != "" && replica != "" {
_, ok := validHAPairs[HAPair{Cluster: cluster, Replica: replica}]
if ok {
return 0, cluster
}
return 1, cluster
}
return 2, cluster
}

func getLimitFromLabelHints(hints *storage.LabelHints) int {
if hints != nil {
return hints.Limit
Expand Down
198 changes: 197 additions & 1 deletion pkg/distributor/distributor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1031,6 +1031,102 @@ func TestDistributor_PushHAInstances(t *testing.T) {
}
}

func TestDistributor_PushMixedHAInstances(t *testing.T) {
t.Parallel()
ctx := user.InjectOrgID(context.Background(), "user")

for i, tc := range []struct {
enableTracker bool
acceptMixedHASamples bool
samples int
expectedResponse *cortexpb.WriteResponse
expectedCode int32
}{
{
enableTracker: true,
acceptMixedHASamples: true,
samples: 5,
expectedResponse: emptyResponse,
expectedCode: 202,
},
} {
for _, shardByAllLabels := range []bool{true} {
tc := tc
shardByAllLabels := shardByAllLabels
for _, enableHistogram := range []bool{false} {
enableHistogram := enableHistogram
t.Run(fmt.Sprintf("[%d](shardByAllLabels=%v, histogram=%v)", i, shardByAllLabels, enableHistogram), func(t *testing.T) {
t.Parallel()
var limits validation.Limits
flagext.DefaultValues(&limits)
limits.AcceptHASamples = true
limits.AcceptMixedHASamples = tc.acceptMixedHASamples
limits.MaxLabelValueLength = 25

ds, ingesters, _, _ := prepare(t, prepConfig{
numIngesters: 2,
happyIngesters: 2,
numDistributors: 1,
replicationFactor: 2,
shardByAllLabels: shardByAllLabels,
limits: &limits,
enableTracker: tc.enableTracker,
})

d := ds[0]

request := makeWriteRequestHAMixedSamples(tc.samples, enableHistogram)
response, _ := d.Push(ctx, request)
assert.Equal(t, tc.expectedResponse, response)

for i := range ingesters {
timeseries := ingesters[i].series()
assert.Equal(t, 5, len(timeseries))
clusters := make(map[string]int)
replicas := make(map[string]int)
for _, v := range timeseries {
replicaLabel := ""
clusterLabel := ""
for _, label := range v.Labels {
if label.Name == "__replica__" {
replicaLabel = label.Value
_, ok := replicas[label.Value]
if !ok {
replicas[label.Value] = 1
} else {
assert.Fail(t, fmt.Sprintf("Two timeseries with same replica label, %s, were found, but only one should be present", label.Value))
}
}
if label.Name == "cluster" {
clusterLabel = label.Value
_, ok := clusters[label.Value]
if !ok {
clusters[label.Value] = 1
} else {
assert.Fail(t, fmt.Sprintf("Two timeseries with same cluster label, %s, were found, but only one should be present", label.Value))
}
}
}
if clusterLabel == "" && replicaLabel != "" {
assert.Equal(t, "replicaNoCluster", replicaLabel)
}
assert.Equal(t, tc.samples, len(v.Samples))
}
assert.Equal(t, 3, len(clusters))
for _, nr := range clusters {
assert.Equal(t, true, nr == 1)
}
assert.Equal(t, 1, len(replicas))
for _, nr := range clusters {
assert.Equal(t, true, nr == 1)
}
}
})
}
}
}
}

func TestDistributor_PushQuery(t *testing.T) {
t.Parallel()
const shuffleShardSize = 5
Expand Down Expand Up @@ -2831,7 +2927,7 @@ func prepare(tb testing.TB, cfg prepConfig) ([]*Distributor, []*mockIngester, []
EnableHATracker: true,
KVStore: kv.Config{Mock: mock},
UpdateTimeout: 100 * time.Millisecond,
FailoverTimeout: time.Second,
FailoverTimeout: time.Hour,
}
cfg.limits.HAMaxClusters = 100
}
Expand Down Expand Up @@ -2950,6 +3046,106 @@ func makeWriteRequestHA(samples int, replica, cluster string, histogram bool) *c
return request
}

func makeWriteRequestHAMixedSamples(samples int, histogram bool) *cortexpb.WriteRequest {
request := &cortexpb.WriteRequest{}

for _, haPair := range []struct {
cluster string
replica string
}{
{
cluster: "cluster0",
replica: "replica0",
},
{
cluster: "cluster0",
replica: "replica1",
},
{
cluster: "cluster1",
replica: "replica0",
},
{
cluster: "cluster1",
replica: "replica1",
},
{
cluster: "",
replica: "replicaNoCluster",
},
{
cluster: "clusterNoReplica",
replica: "",
},
{
cluster: "",
replica: "",
},
} {
cluster := haPair.cluster
replica := haPair.replica
var ts cortexpb.PreallocTimeseries
if cluster == "" && replica == "" {
ts = cortexpb.PreallocTimeseries{
TimeSeries: &cortexpb.TimeSeries{
Labels: []cortexpb.LabelAdapter{
{Name: "__name__", Value: "foo"},
{Name: "bar", Value: "baz"},
},
},
}
} else if cluster == "" && replica != "" {
ts = cortexpb.PreallocTimeseries{
TimeSeries: &cortexpb.TimeSeries{
Labels: []cortexpb.LabelAdapter{
{Name: "__name__", Value: "foo"},
{Name: "__replica__", Value: replica},
{Name: "bar", Value: "baz"},
},
},
}
} else if cluster != "" && replica == "" {
ts = cortexpb.PreallocTimeseries{
TimeSeries: &cortexpb.TimeSeries{
Labels: []cortexpb.LabelAdapter{
{Name: "__name__", Value: "foo"},
{Name: "bar", Value: "baz"},
{Name: "cluster", Value: cluster},
},
},
}
} else {
ts = cortexpb.PreallocTimeseries{
TimeSeries: &cortexpb.TimeSeries{
Labels: []cortexpb.LabelAdapter{
{Name: "__name__", Value: "foo"},
{Name: "__replica__", Value: replica},
{Name: "bar", Value: "baz"},
{Name: "cluster", Value: cluster},
},
},
}
}
if histogram {
ts.Histograms = []cortexpb.Histogram{
cortexpb.HistogramToHistogramProto(int64(samples), tsdbutil.GenerateTestHistogram(samples)),
}
} else {
var s = make([]cortexpb.Sample, 0)
for i := 0; i < samples; i++ {
sample := cortexpb.Sample{
Value: float64(i),
TimestampMs: int64(i),
}
s = append(s, sample)
}
ts.Samples = s
}
request.Timeseries = append(request.Timeseries, ts)
}
return request
}

func makeWriteRequestExemplar(seriesLabels []string, timestamp int64, exemplarLabels []string) *cortexpb.WriteRequest {
return &cortexpb.WriteRequest{
Timeseries: []cortexpb.PreallocTimeseries{
Expand Down
7 changes: 7 additions & 0 deletions pkg/util/validation/limits.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,7 @@ type Limits struct {
IngestionRateStrategy string `yaml:"ingestion_rate_strategy" json:"ingestion_rate_strategy"`
IngestionBurstSize int `yaml:"ingestion_burst_size" json:"ingestion_burst_size"`
AcceptHASamples bool `yaml:"accept_ha_samples" json:"accept_ha_samples"`
AcceptMixedHASamples bool `yaml:"accept_mixed_ha_samples" json:"accept_mixed_ha_samples"`
HAClusterLabel string `yaml:"ha_cluster_label" json:"ha_cluster_label"`
HAReplicaLabel string `yaml:"ha_replica_label" json:"ha_replica_label"`
HAMaxClusters int `yaml:"ha_max_clusters" json:"ha_max_clusters"`
Expand Down Expand Up @@ -220,6 +221,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) {
f.StringVar(&l.IngestionRateStrategy, "distributor.ingestion-rate-limit-strategy", "local", "Whether the ingestion rate limit should be applied individually to each distributor instance (local), or evenly shared across the cluster (global).")
f.IntVar(&l.IngestionBurstSize, "distributor.ingestion-burst-size", 50000, "Per-user allowed ingestion burst size (in number of samples).")
f.BoolVar(&l.AcceptHASamples, "distributor.ha-tracker.enable-for-all-users", false, "Flag to enable, for all users, handling of samples with external labels identifying replicas in an HA Prometheus setup.")
f.BoolVar(&l.AcceptMixedHASamples, "distributor.ha-tracker.mixed-ha-samples", false, "Flag to enable handling of samples with mixed external labels identifying replicas in an HA Prometheus setup. Supported only if -distributor.ha-tracker.enable-for-all-users is true.")
f.StringVar(&l.HAClusterLabel, "distributor.ha-tracker.cluster", "cluster", "Prometheus label to look for in samples to identify a Prometheus HA cluster.")
f.StringVar(&l.HAReplicaLabel, "distributor.ha-tracker.replica", "__replica__", "Prometheus label to look for in samples to identify a Prometheus HA replica.")
f.IntVar(&l.HAMaxClusters, "distributor.ha-tracker.max-clusters", 0, "Maximum number of clusters that HA tracker will keep track of for single user. 0 to disable the limit.")
Expand Down Expand Up @@ -547,6 +549,11 @@ func (o *Overrides) AcceptHASamples(userID string) bool {
return o.GetOverridesForUser(userID).AcceptHASamples
}

// AcceptMixedHASamples returns whether the distributor should track and accept mixed samples from HA replicas for this user.
func (o *Overrides) AcceptMixedHASamples(userID string) bool {
return o.GetOverridesForUser(userID).AcceptMixedHASamples
}

// HAClusterLabel returns the cluster label to look for when deciding whether to accept a sample from a Prometheus HA replica.
func (o *Overrides) HAClusterLabel(userID string) string {
return o.GetOverridesForUser(userID).HAClusterLabel
Expand Down

0 comments on commit db369f5

Please sign in to comment.