From a7719d9f1eb88aeae1f0f9eb455259c192e946a0 Mon Sep 17 00:00:00 2001 From: Matthias Loibl Date: Tue, 25 Feb 2020 17:27:37 +0100 Subject: [PATCH 01/14] Create ReplicaLabelsFilter to allow for offline deduplication Signed-off-by: Matthias Loibl --- cmd/thanos/compact.go | 8 ++++++++ pkg/block/fetcher.go | 18 ++++++++++++++++++ pkg/block/fetcher_test.go | 23 +++++++++++++++++++++++ 3 files changed, 49 insertions(+) diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index 03f1524330..749f82decd 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -133,6 +133,9 @@ func registerCompact(m map[string]setupFunc, app *kingpin.Application) { "or compactor is ignoring the deletion because it's compacting the block at the same time."). Default("48h")) + dedupReplicaLabels := cmd.Flag("offline-deduplication.replica-labels", "Label to treat as a replica indicator of blocks that can be deduplicated. This will merge multiple replica blocks into one. This process is irrevertable. Experminteal"). + Hidden().String() + selectorRelabelConf := regSelectorRelabelFlags(cmd) m[component.Compact.String()] = func(g *run.Group, logger log.Logger, reg *prometheus.Registry, tracer opentracing.Tracer, _ <-chan struct{}, _ bool) error { @@ -157,6 +160,7 @@ func registerCompact(m map[string]setupFunc, app *kingpin.Application) { *maxCompactionLevel, *blockSyncConcurrency, *compactionConcurrency, + *dedupReplicaLabels, selectorRelabelConf, *waitInterval, ) @@ -183,6 +187,7 @@ func runCompact( maxCompactionLevel int, blockSyncConcurrency int, concurrency int, + dedupReplicaLabels string, selectorRelabelConf *extflag.PathOrContent, waitInterval time.Duration, ) error { @@ -278,11 +283,14 @@ func runCompact( ignoreDeletionMarkFilter := block.NewIgnoreDeletionMarkFilter(logger, bkt, time.Duration(deleteDelay.Seconds()/2)*time.Second) duplicateBlocksFilter := block.NewDeduplicateFilter() prometheusRegisterer := extprom.WrapRegistererWithPrefix("thanos_", reg) + replicaLabelFilter := block.ReplicaLabelsFilter{ReplicaLabels: strings.Split(dedupReplicaLabels, ",")} + metaFetcher, err := block.NewMetaFetcher(logger, 32, bkt, "", prometheusRegisterer, block.NewLabelShardedMetaFilter(relabelConfig).Filter, block.NewConsistencyDelayMetaFilter(logger, consistencyDelay, prometheusRegisterer).Filter, ignoreDeletionMarkFilter.Filter, duplicateBlocksFilter.Filter, + replicaLabelFilter.Filter, ) if err != nil { return errors.Wrap(err, "create meta fetcher") diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index 2ab188bd0f..bbff4f79c6 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -57,6 +57,7 @@ const ( // Blocks that are marked for deletion can be loaded as well. This is done to make sure that we load blocks that are meant to be deleted, // but don't have a replacement block yet. markedForDeletionMeta = "marked-for-deletion" + replicaExclude = "replica-exclude" ) func newSyncMetrics(reg prometheus.Registerer) *syncMetrics { @@ -540,6 +541,23 @@ func contains(s1 []ulid.ULID, s2 []ulid.ULID) bool { return true } +type ReplicaLabelsFilter struct { + ReplicaLabels []string +} + +func (f *ReplicaLabelsFilter) Filter(metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, view bool) { + for u, meta := range metas { + labels := meta.Thanos.Labels + for _, replicaLabel := range f.ReplicaLabels { + if _, exists := labels[replicaLabel]; exists { + delete(labels, replicaLabel) + synced.WithLabelValues(replicaExclude).Inc() + } + } + metas[u].Thanos.Labels = labels + } +} + // ConsistencyDelayMetaFilter is a MetaFetcher filter that filters out blocks that are created before a specified consistency delay. // Not go-routine safe. type ConsistencyDelayMetaFilter struct { diff --git a/pkg/block/fetcher_test.go b/pkg/block/fetcher_test.go index 3c181886dd..0961048c6f 100644 --- a/pkg/block/fetcher_test.go +++ b/pkg/block/fetcher_test.go @@ -857,6 +857,29 @@ func TestDeduplicateFilter_Filter(t *testing.T) { } } +func TestReplicaLabel_Filter(t *testing.T) { + rf := ReplicaLabelsFilter{ReplicaLabels: []string{"replica", "rule_replica"}} + + input := map[ulid.ULID]*metadata.Meta{ + ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(2): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "message": "something"}}}, + ULID(3): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "rule_replica": "rule1", "message": "something"}}}, + ULID(4): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "rule_replica": "rule1"}}}, + } + expected := map[ulid.ULID]*metadata.Meta{ + ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(2): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(3): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(4): {Thanos: metadata.Thanos{Labels: map[string]string{}}}, + } + + synced := prometheus.NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) + rf.Filter(input, synced, false) + + testutil.Equals(t, 5.0, promtest.ToFloat64(synced.WithLabelValues(replicaExclude))) + testutil.Equals(t, expected, input) +} + func compareSliceWithMapKeys(tb testing.TB, m map[ulid.ULID]*metadata.Meta, s []ulid.ULID) { _, file, line, _ := runtime.Caller(1) matching := true From d789ac3e0557192d9ee754cb768a8eef3543a110 Mon Sep 17 00:00:00 2001 From: Matthias Loibl Date: Thu, 5 Mar 2020 14:15:19 +0100 Subject: [PATCH 02/14] Start adding a e2e test for offline-deduplication with Thanos compact Signed-off-by: Matthias Loibl --- test/e2e/compact_test.go | 69 ++++++++++++++++++++++++++++++++++ test/e2e/e2ethanos/services.go | 29 ++++++++++++++ 2 files changed, 98 insertions(+) create mode 100644 test/e2e/compact_test.go diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go new file mode 100644 index 0000000000..b0f6c3a06b --- /dev/null +++ b/test/e2e/compact_test.go @@ -0,0 +1,69 @@ +package e2e + +import ( + "context" + "os" + "path" + "path/filepath" + "testing" + "time" + + "github.com/cortexproject/cortex/integration/e2e" + e2edb "github.com/cortexproject/cortex/integration/e2e/db" + "github.com/go-kit/kit/log" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/pkg/timestamp" + "github.com/thanos-io/thanos/pkg/objstore" + "github.com/thanos-io/thanos/pkg/objstore/client" + "github.com/thanos-io/thanos/pkg/objstore/s3" + "github.com/thanos-io/thanos/pkg/testutil" + "github.com/thanos-io/thanos/pkg/testutil/e2eutil" + "github.com/thanos-io/thanos/test/e2e/e2ethanos" +) + +func TestCompact(t *testing.T) { + t.Parallel() + l := log.NewLogfmtLogger(os.Stdout) + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) + defer cancel() + + s, err := e2e.NewScenario("e2e_test_compact") + testutil.Ok(t, err) + defer s.Close() + + const bucket = "thanos" + + m := e2edb.NewMinio(80, bucket) + testutil.Ok(t, s.StartAndWaitReady(m)) + + s3Config := s3.Config{ + Bucket: bucket, + AccessKey: e2edb.MinioAccessKey, + SecretKey: e2edb.MinioSecretKey, + Endpoint: m.NetworkHTTPEndpoint(), + Insecure: true, + } + + series := []labels.Labels{labels.FromStrings("a", "1", "b", "2")} + extLset := labels.FromStrings("ext1", "value1", "replica", "1") + + bkt, err := s3.NewBucketWithConfig(l, s3Config, "test-feed") + testutil.Ok(t, err) + + dir := filepath.Join(s.SharedDir(), "tmp") + + now := time.Now() + id1, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, series, 10, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset, 0) + testutil.Ok(t, err) + + testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id1.String()), id1.String())) + + compact, err := e2ethanos.NewCompact(s.SharedDir(), "compact", client.BucketConfig{ + Type: client.S3, + Config: s3Config, + }) + testutil.Ok(t, err) + testutil.Ok(t, s.StartAndWaitReady(compact)) + +} diff --git a/test/e2e/e2ethanos/services.go b/test/e2e/e2ethanos/services.go index afd20b6edc..1f13925a07 100644 --- a/test/e2e/e2ethanos/services.go +++ b/test/e2e/e2ethanos/services.go @@ -313,3 +313,32 @@ func NewStoreGW(sharedDir string, name string, bucketConfig client.BucketConfig, 9091, ), nil } + +func NewCompact(sharedDir string, name string, bucketConfig client.BucketConfig) (*Service, error) { + dir := filepath.Join(sharedDir, "data", "compact", name) + container := filepath.Join(e2e.ContainerSharedDir, "data", "compact", name) + + if err := os.MkdirAll(dir, 0777); err != nil { + return nil, errors.Wrap(err, "create compact dir") + } + + bktConfigBytes, err := yaml.Marshal(bucketConfig) + if err != nil { + return nil, errors.Wrapf(err, "generate compact config file: %v", bucketConfig) + } + + fmt.Println(string(bktConfigBytes)) + + return NewService( + fmt.Sprintf("compact-%s", name), + DefaultImage(), + e2e.NewCommand("compact", append(e2e.BuildArgs(map[string]string{ + "--data-dir": container, + "--objstore.config": string(bktConfigBytes), + "--http-address": ":80", + }))...), + e2e.NewReadinessProbe(80, "/-/ready", 200), + 80, + 9091, + ), nil +} From 1360bb2d5f6acda9cceb284d7e18d5e4d170db93 Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Tue, 10 Mar 2020 16:24:55 +0100 Subject: [PATCH 03/14] Address issues that have discovered after review Signed-off-by: Kemal Akkoyun --- cmd/thanos/compact.go | 10 ++--- pkg/block/fetcher.go | 75 ++++++++++++++++++++------------- pkg/block/fetcher_test.go | 77 ++++++++++++++++++++++------------ test/e2e/compact_test.go | 5 ++- test/e2e/e2ethanos/services.go | 2 - 5 files changed, 106 insertions(+), 63 deletions(-) diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index 749f82decd..423321fb78 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -133,8 +133,9 @@ func registerCompact(m map[string]setupFunc, app *kingpin.Application) { "or compactor is ignoring the deletion because it's compacting the block at the same time."). Default("48h")) - dedupReplicaLabels := cmd.Flag("offline-deduplication.replica-labels", "Label to treat as a replica indicator of blocks that can be deduplicated. This will merge multiple replica blocks into one. This process is irrevertable. Experminteal"). - Hidden().String() + dedupReplicaLabels := cmd.Flag("deduplication.replica-label", "Label to treat as a replica indicator of blocks that can be deduplicated (repeated flag). This will merge multiple replica blocks into one. This process is irreversible."+ + "Experimental. When it is set true, this will given labels from blocks so that vertical compaction could merge blocks."). + Hidden().Strings() selectorRelabelConf := regSelectorRelabelFlags(cmd) @@ -187,7 +188,7 @@ func runCompact( maxCompactionLevel int, blockSyncConcurrency int, concurrency int, - dedupReplicaLabels string, + dedupReplicaLabels []string, selectorRelabelConf *extflag.PathOrContent, waitInterval time.Duration, ) error { @@ -283,14 +284,13 @@ func runCompact( ignoreDeletionMarkFilter := block.NewIgnoreDeletionMarkFilter(logger, bkt, time.Duration(deleteDelay.Seconds()/2)*time.Second) duplicateBlocksFilter := block.NewDeduplicateFilter() prometheusRegisterer := extprom.WrapRegistererWithPrefix("thanos_", reg) - replicaLabelFilter := block.ReplicaLabelsFilter{ReplicaLabels: strings.Split(dedupReplicaLabels, ",")} metaFetcher, err := block.NewMetaFetcher(logger, 32, bkt, "", prometheusRegisterer, block.NewLabelShardedMetaFilter(relabelConfig).Filter, block.NewConsistencyDelayMetaFilter(logger, consistencyDelay, prometheusRegisterer).Filter, ignoreDeletionMarkFilter.Filter, duplicateBlocksFilter.Filter, - replicaLabelFilter.Filter, + block.NewReplicaLabelRemover(dedupReplicaLabels).Modify, ) if err != nil { return errors.Wrap(err, "create meta fetcher") diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index bbff4f79c6..ac28a0ada3 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -32,12 +32,19 @@ import ( "github.com/thanos-io/thanos/pkg/runutil" ) +type txLabeledGauge interface { + WithLabelValues(lvs ...string) prometheus.Gauge + ResetTx() + Submit() +} + type syncMetrics struct { syncs prometheus.Counter syncFailures prometheus.Counter syncDuration prometheus.Histogram - synced *extprom.TxGaugeVec + synced txLabeledGauge + modified txLabeledGauge } const ( @@ -49,15 +56,15 @@ const ( failedMeta = "failed" // Filter's label values. - labelExcludedMeta = "label-excluded" - timeExcludedMeta = "time-excluded" - tooFreshMeta = "too-fresh" - duplicateMeta = "duplicate" + labelExcludedMeta = "label-excluded" + timeExcludedMeta = "time-excluded" + tooFreshMeta = "too-fresh" + duplicateMeta = "duplicate" + replicaRemovedMeta = "replica-label-removed" // Blocks that are marked for deletion can be loaded as well. This is done to make sure that we load blocks that are meant to be deleted, // but don't have a replacement block yet. markedForDeletionMeta = "marked-for-deletion" - replicaExclude = "replica-exclude" ) func newSyncMetrics(reg prometheus.Registerer) *syncMetrics { @@ -95,6 +102,14 @@ func newSyncMetrics(reg prometheus.Registerer) *syncMetrics { []string{duplicateMeta}, []string{markedForDeletionMeta}, ) + m.modified = extprom.NewTxGaugeVec(reg, prometheus.GaugeOpts{ + Subsystem: syncMetricSubSys, + Name: "modified", + Help: "Number of block metadata that modified", + }, + []string{"modified"}, + []string{replicaRemovedMeta}, + ) return &m } @@ -102,11 +117,7 @@ type MetadataFetcher interface { Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata.Meta, partial map[ulid.ULID]error, err error) } -type GaugeLabeled interface { - WithLabelValues(lvs ...string) prometheus.Gauge -} - -type MetaFetcherFilter func(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, incompleteView bool) error +type MetaFetcherFilter func(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, incompleteView bool) error // MetaFetcher is a struct that synchronizes filtered metadata of all block in the object storage with the local state. // Not go-routine safe. @@ -352,7 +363,7 @@ func (s *MetaFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata. for _, f := range s.filters { // NOTE: filter can update synced metric accordingly to the reason of the exclude. - if err := f(ctx, metas, s.metrics.synced, incompleteView); err != nil { + if err := f(ctx, metas, s.metrics, incompleteView); err != nil { return nil, nil, errors.Wrap(err, "filter metas") } } @@ -382,12 +393,12 @@ func NewTimePartitionMetaFilter(MinTime, MaxTime model.TimeOrDurationValue) *Tim } // Filter filters out blocks that are outside of specified time range. -func (f *TimePartitionMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, _ bool) error { +func (f *TimePartitionMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, _ bool) error { for id, m := range metas { if m.MaxTime >= f.minTime.PrometheusTimestamp() && m.MinTime <= f.maxTime.PrometheusTimestamp() { continue } - synced.WithLabelValues(timeExcludedMeta).Inc() + metrics.synced.WithLabelValues(timeExcludedMeta).Inc() delete(metas, id) } return nil @@ -410,7 +421,7 @@ func NewLabelShardedMetaFilter(relabelConfig []*relabel.Config) *LabelShardedMet const blockIDLabel = "__block_id" // Filter filters out blocks that have no labels after relabelling of each block external (Thanos) labels. -func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, _ bool) error { +func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, _ bool) error { var lbls labels.Labels for id, m := range metas { lbls = lbls[:0] @@ -420,7 +431,7 @@ func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]* } if processedLabels := relabel.Process(lbls, f.relabelConfig...); len(processedLabels) == 0 { - synced.WithLabelValues(labelExcludedMeta).Inc() + metrics.synced.WithLabelValues(labelExcludedMeta).Inc() delete(metas, id) } } @@ -440,7 +451,7 @@ func NewDeduplicateFilter() *DeduplicateFilter { // Filter filters out duplicate blocks that can be formed // from two or more overlapping blocks that fully submatches the source blocks of the older blocks. -func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, _ bool) error { +func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, _ bool) error { var wg sync.WaitGroup metasByResolution := make(map[int64][]*metadata.Meta) @@ -457,7 +468,7 @@ func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metad BlockMeta: tsdb.BlockMeta{ ULID: ulid.MustNew(uint64(0), nil), }, - }), metasByResolution[res], metas, res, synced) + }), metasByResolution[res], metas, res, metrics.synced) }(res) } @@ -466,7 +477,7 @@ func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metad return nil } -func (f *DeduplicateFilter) filterForResolution(root *Node, metaSlice []*metadata.Meta, metas map[ulid.ULID]*metadata.Meta, res int64, synced GaugeLabeled) { +func (f *DeduplicateFilter) filterForResolution(root *Node, metaSlice []*metadata.Meta, metas map[ulid.ULID]*metadata.Meta, res int64, synced txLabeledGauge) { sort.Slice(metaSlice, func(i, j int) bool { ilen := len(metaSlice[i].Compaction.Sources) jlen := len(metaSlice[j].Compaction.Sources) @@ -541,21 +552,29 @@ func contains(s1 []ulid.ULID, s2 []ulid.ULID) bool { return true } -type ReplicaLabelsFilter struct { - ReplicaLabels []string +// ReplicaLabelRemover is a MetaFetcher modifier modifies external labels of existing blocks, it removes given replica labels from the metadata of blocks that have it. +type ReplicaLabelRemover struct { + replicaLabels []string } -func (f *ReplicaLabelsFilter) Filter(metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, view bool) { +// NewReplicaLabelRemover creates a ReplicaLabelRemover. +func NewReplicaLabelRemover(replicaLabels []string) *ReplicaLabelRemover { + return &ReplicaLabelRemover{replicaLabels: replicaLabels} +} + +// Modify modifies external labels of existing blocks, it removes given replica labels from the metadata of blocks that have it. +func (r *ReplicaLabelRemover) Modify(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, view bool) error { for u, meta := range metas { labels := meta.Thanos.Labels - for _, replicaLabel := range f.ReplicaLabels { + for _, replicaLabel := range r.replicaLabels { if _, exists := labels[replicaLabel]; exists { delete(labels, replicaLabel) - synced.WithLabelValues(replicaExclude).Inc() + metrics.modified.WithLabelValues(replicaRemovedMeta).Inc() } } metas[u].Thanos.Labels = labels } + return nil } // ConsistencyDelayMetaFilter is a MetaFetcher filter that filters out blocks that are created before a specified consistency delay. @@ -584,7 +603,7 @@ func NewConsistencyDelayMetaFilter(logger log.Logger, consistencyDelay time.Dura } // Filter filters out blocks that filters blocks that have are created before a specified consistency delay. -func (f *ConsistencyDelayMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, _ bool) error { +func (f *ConsistencyDelayMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, _ bool) error { for id, meta := range metas { // TODO(khyatisoneji): Remove the checks about Thanos Source // by implementing delete delay to fetch metas. @@ -595,7 +614,7 @@ func (f *ConsistencyDelayMetaFilter) Filter(_ context.Context, metas map[ulid.UL meta.Thanos.Source != metadata.CompactorRepairSource { level.Debug(f.logger).Log("msg", "block is too fresh for now", "block", id) - synced.WithLabelValues(tooFreshMeta).Inc() + metrics.synced.WithLabelValues(tooFreshMeta).Inc() delete(metas, id) } } @@ -630,7 +649,7 @@ func (f *IgnoreDeletionMarkFilter) DeletionMarkBlocks() map[ulid.ULID]*metadata. // Filter filters out blocks that are marked for deletion after a given delay. // It also returns the blocks that can be deleted since they were uploaded delay duration before current time. -func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, _ bool) error { +func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, m *syncMetrics, _ bool) error { f.deletionMarkMap = make(map[ulid.ULID]*metadata.DeletionMark) for id := range metas { @@ -647,7 +666,7 @@ func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.UL } f.deletionMarkMap[id] = deletionMark if time.Since(time.Unix(deletionMark.DeletionTime, 0)).Seconds() > f.delay.Seconds() { - synced.WithLabelValues(markedForDeletionMeta).Inc() + m.synced.WithLabelValues(markedForDeletionMeta).Inc() delete(metas, id) } } diff --git a/pkg/block/fetcher_test.go b/pkg/block/fetcher_test.go index 0961048c6f..3281b1c2a5 100644 --- a/pkg/block/fetcher_test.go +++ b/pkg/block/fetcher_test.go @@ -35,6 +35,23 @@ import ( "gopkg.in/yaml.v2" ) +type testTxLabeledGauge struct { + g *prometheus.GaugeVec +} + +func (t *testTxLabeledGauge) WithLabelValues(lvs ...string) prometheus.Gauge { + return t.g.WithLabelValues(lvs...) +} +func (t *testTxLabeledGauge) ResetTx() {} +func (t *testTxLabeledGauge) Submit() {} + +func newTestSyncMetrics() *syncMetrics { + return &syncMetrics{ + synced: &testTxLabeledGauge{promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"})}, + modified: &testTxLabeledGauge{promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"modified"})}, + } +} + func ULID(i int) ulid.ULID { return ulid.MustNew(uint64(i), nil) } func ULIDs(is ...int) []ulid.ULID { @@ -57,9 +74,9 @@ func TestMetaFetcher_Fetch(t *testing.T) { var ulidToDelete ulid.ULID r := prometheus.NewRegistry() - f, err := NewMetaFetcher(log.NewNopLogger(), 20, bkt, dir, r, func(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeLabeled, _ bool) error { + f, err := NewMetaFetcher(log.NewNopLogger(), 20, bkt, dir, r, func(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, incompleteView bool) error { if _, ok := metas[ulidToDelete]; ok { - synced.WithLabelValues("filtered").Inc() + metrics.synced.WithLabelValues("filtered").Inc() delete(metas, ulidToDelete) } return nil @@ -337,9 +354,10 @@ func TestLabelShardedMetaFilter_Filter_Basic(t *testing.T) { ULID(6): input[ULID(6)], } - synced := promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) - testutil.Ok(t, f.Filter(ctx, input, synced, false)) - testutil.Equals(t, 3.0, promtest.ToFloat64(synced.WithLabelValues(labelExcludedMeta))) + m := newTestSyncMetrics() + testutil.Ok(t, f.Filter(ctx, input, m, false)) + + testutil.Equals(t, 3.0, promtest.ToFloat64(m.synced.WithLabelValues(labelExcludedMeta))) testutil.Equals(t, expected, input) } @@ -434,10 +452,11 @@ func TestLabelShardedMetaFilter_Filter_Hashmod(t *testing.T) { } deleted := len(input) - len(expected) - synced := promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) - testutil.Ok(t, f.Filter(ctx, input, synced, false)) + m := newTestSyncMetrics() + testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Equals(t, expected, input) - testutil.Equals(t, float64(deleted), promtest.ToFloat64(synced.WithLabelValues(labelExcludedMeta))) + testutil.Equals(t, float64(deleted), promtest.ToFloat64(m.synced.WithLabelValues(labelExcludedMeta))) }) @@ -497,9 +516,10 @@ func TestTimePartitionMetaFilter_Filter(t *testing.T) { ULID(4): input[ULID(4)], } - synced := promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) - testutil.Ok(t, f.Filter(ctx, input, synced, false)) - testutil.Equals(t, 2.0, promtest.ToFloat64(synced.WithLabelValues(timeExcludedMeta))) + m := newTestSyncMetrics() + testutil.Ok(t, f.Filter(ctx, input, m, false)) + + testutil.Equals(t, 2.0, promtest.ToFloat64(m.synced.WithLabelValues(timeExcludedMeta))) testutil.Equals(t, expected, input) } @@ -830,7 +850,7 @@ func TestDeduplicateFilter_Filter(t *testing.T) { } { f := NewDeduplicateFilter() if ok := t.Run(tcase.name, func(t *testing.T) { - synced := promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) + m := newTestSyncMetrics() metas := make(map[ulid.ULID]*metadata.Meta) inputLen := len(tcase.input) for id, metaInfo := range tcase.input { @@ -848,9 +868,9 @@ func TestDeduplicateFilter_Filter(t *testing.T) { }, } } - testutil.Ok(t, f.Filter(ctx, metas, synced, false)) + testutil.Ok(t, f.Filter(ctx, metas, m, false)) compareSliceWithMapKeys(t, metas, tcase.expected) - testutil.Equals(t, float64(inputLen-len(tcase.expected)), promtest.ToFloat64(synced.WithLabelValues(duplicateMeta))) + testutil.Equals(t, float64(inputLen-len(tcase.expected)), promtest.ToFloat64(m.synced.WithLabelValues(duplicateMeta))) }); !ok { return } @@ -858,7 +878,10 @@ func TestDeduplicateFilter_Filter(t *testing.T) { } func TestReplicaLabel_Filter(t *testing.T) { - rf := ReplicaLabelsFilter{ReplicaLabels: []string{"replica", "rule_replica"}} + ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) + defer cancel() + + rm := NewReplicaLabelRemover([]string{"replica", "rule_replica"}) input := map[ulid.ULID]*metadata.Meta{ ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, @@ -873,10 +896,10 @@ func TestReplicaLabel_Filter(t *testing.T) { ULID(4): {Thanos: metadata.Thanos{Labels: map[string]string{}}}, } - synced := prometheus.NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) - rf.Filter(input, synced, false) + m := newTestSyncMetrics() + testutil.Ok(t, rm.Modify(ctx, input, m, false)) - testutil.Equals(t, 5.0, promtest.ToFloat64(synced.WithLabelValues(replicaExclude))) + testutil.Equals(t, 5.0, promtest.ToFloat64(m.modified.WithLabelValues(replicaRemovedMeta))) testutil.Equals(t, expected, input) } @@ -968,7 +991,7 @@ func TestConsistencyDelayMetaFilter_Filter_0(t *testing.T) { } t.Run("consistency 0 (turned off)", func(t *testing.T) { - synced := promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) + m := newTestSyncMetrics() expected := map[ulid.ULID]*metadata.Meta{} // Copy all. for _, id := range u.created { @@ -979,13 +1002,13 @@ func TestConsistencyDelayMetaFilter_Filter_0(t *testing.T) { f := NewConsistencyDelayMetaFilter(nil, 0*time.Second, reg) testutil.Equals(t, map[string]float64{"consistency_delay_seconds": 0.0}, extprom.CurrentGaugeValuesFor(t, reg, "consistency_delay_seconds")) - testutil.Ok(t, f.Filter(ctx, input, synced, false)) - testutil.Equals(t, 0.0, promtest.ToFloat64(synced.WithLabelValues(tooFreshMeta))) + testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Equals(t, 0.0, promtest.ToFloat64(m.synced.WithLabelValues(tooFreshMeta))) testutil.Equals(t, expected, input) }) t.Run("consistency 30m.", func(t *testing.T) { - synced := promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) + m := newTestSyncMetrics() expected := map[ulid.ULID]*metadata.Meta{} // Only certain sources and those with 30m or more age go through. for i, id := range u.created { @@ -1004,8 +1027,8 @@ func TestConsistencyDelayMetaFilter_Filter_0(t *testing.T) { f := NewConsistencyDelayMetaFilter(nil, 30*time.Minute, reg) testutil.Equals(t, map[string]float64{"consistency_delay_seconds": (30 * time.Minute).Seconds()}, extprom.CurrentGaugeValuesFor(t, reg, "consistency_delay_seconds")) - testutil.Ok(t, f.Filter(ctx, input, synced, false)) - testutil.Equals(t, float64(len(u.created)-len(expected)), promtest.ToFloat64(synced.WithLabelValues(tooFreshMeta))) + testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Equals(t, float64(len(u.created)-len(expected)), promtest.ToFloat64(m.synced.WithLabelValues(tooFreshMeta))) testutil.Equals(t, expected, input) }) } @@ -1056,9 +1079,9 @@ func TestIgnoreDeletionMarkFilter_Filter(t *testing.T) { ULID(4): {}, } - synced := promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"}) - testutil.Ok(t, f.Filter(ctx, input, synced, false)) - testutil.Equals(t, 1.0, promtest.ToFloat64(synced.WithLabelValues(markedForDeletionMeta))) + m := newTestSyncMetrics() + testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Equals(t, 1.0, promtest.ToFloat64(m.synced.WithLabelValues(markedForDeletionMeta))) testutil.Equals(t, expected, input) }) } diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index b0f6c3a06b..4f5d9ffc34 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -1,3 +1,6 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + package e2e import ( @@ -63,7 +66,7 @@ func TestCompact(t *testing.T) { Type: client.S3, Config: s3Config, }) + testutil.Ok(t, err) testutil.Ok(t, s.StartAndWaitReady(compact)) - } diff --git a/test/e2e/e2ethanos/services.go b/test/e2e/e2ethanos/services.go index 1f13925a07..66658dee3a 100644 --- a/test/e2e/e2ethanos/services.go +++ b/test/e2e/e2ethanos/services.go @@ -327,8 +327,6 @@ func NewCompact(sharedDir string, name string, bucketConfig client.BucketConfig) return nil, errors.Wrapf(err, "generate compact config file: %v", bucketConfig) } - fmt.Println(string(bktConfigBytes)) - return NewService( fmt.Sprintf("compact-%s", name), DefaultImage(), From 12aa6d338862a24e979e9a4d8f0ee68321bb82f6 Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Tue, 10 Mar 2020 22:04:10 +0100 Subject: [PATCH 04/14] Fix e2e test service issue Signed-off-by: Kemal Akkoyun --- test/e2e/compact_test.go | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index 4f5d9ffc34..457a518a30 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -40,18 +40,16 @@ func TestCompact(t *testing.T) { m := e2edb.NewMinio(80, bucket) testutil.Ok(t, s.StartAndWaitReady(m)) - s3Config := s3.Config{ + series := []labels.Labels{labels.FromStrings("a", "1", "b", "2")} + extLset := labels.FromStrings("ext1", "value1", "replica", "1") + + bkt, err := s3.NewBucketWithConfig(l, s3.Config{ Bucket: bucket, AccessKey: e2edb.MinioAccessKey, SecretKey: e2edb.MinioSecretKey, - Endpoint: m.NetworkHTTPEndpoint(), + Endpoint: m.HTTPEndpoint(), Insecure: true, - } - - series := []labels.Labels{labels.FromStrings("a", "1", "b", "2")} - extLset := labels.FromStrings("ext1", "value1", "replica", "1") - - bkt, err := s3.NewBucketWithConfig(l, s3Config, "test-feed") + }, "test-feed") testutil.Ok(t, err) dir := filepath.Join(s.SharedDir(), "tmp") @@ -63,8 +61,14 @@ func TestCompact(t *testing.T) { testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id1.String()), id1.String())) compact, err := e2ethanos.NewCompact(s.SharedDir(), "compact", client.BucketConfig{ - Type: client.S3, - Config: s3Config, + Type: client.S3, + Config: s3.Config{ + Bucket: bucket, + AccessKey: e2edb.MinioAccessKey, + SecretKey: e2edb.MinioSecretKey, + Endpoint: m.NetworkHTTPEndpoint(), + Insecure: true, + }, }) testutil.Ok(t, err) From ed27772e4eb33af48409b1d4eac3b0f62e2bed5f Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Tue, 10 Mar 2020 22:04:48 +0100 Subject: [PATCH 05/14] Improve fetcher unit tests Signed-off-by: Kemal Akkoyun --- pkg/block/fetcher_test.go | 62 +++++++++++++++++++++++++++------------ 1 file changed, 43 insertions(+), 19 deletions(-) diff --git a/pkg/block/fetcher_test.go b/pkg/block/fetcher_test.go index 3281b1c2a5..60f94c8fba 100644 --- a/pkg/block/fetcher_test.go +++ b/pkg/block/fetcher_test.go @@ -877,30 +877,54 @@ func TestDeduplicateFilter_Filter(t *testing.T) { } } -func TestReplicaLabel_Filter(t *testing.T) { +func TestReplicaLabelRemover_Modify(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) defer cancel() - rm := NewReplicaLabelRemover([]string{"replica", "rule_replica"}) - input := map[ulid.ULID]*metadata.Meta{ - ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, - ULID(2): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "message": "something"}}}, - ULID(3): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "rule_replica": "rule1", "message": "something"}}}, - ULID(4): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "rule_replica": "rule1"}}}, - } - expected := map[ulid.ULID]*metadata.Meta{ - ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, - ULID(2): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, - ULID(3): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, - ULID(4): {Thanos: metadata.Thanos{Labels: map[string]string{}}}, - } - - m := newTestSyncMetrics() - testutil.Ok(t, rm.Modify(ctx, input, m, false)) + for _, tcase := range []struct { + name string + input map[ulid.ULID]*metadata.Meta + expected map[ulid.ULID]*metadata.Meta + modified float64 + }{ + { + name: "without replica labels", + input: map[ulid.ULID]*metadata.Meta{ + ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(2): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(3): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something1"}}}, + }, + expected: map[ulid.ULID]*metadata.Meta{ + ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(2): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(3): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something1"}}}, + }, + modified: 0, + }, + { + name: "with replica labels", + input: map[ulid.ULID]*metadata.Meta{ + ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(2): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "message": "something"}}}, + ULID(3): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "rule_replica": "rule1", "message": "something"}}}, + ULID(4): {Thanos: metadata.Thanos{Labels: map[string]string{"replica": "cluster1", "rule_replica": "rule1"}}}, + }, + expected: map[ulid.ULID]*metadata.Meta{ + ULID(1): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(2): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(3): {Thanos: metadata.Thanos{Labels: map[string]string{"message": "something"}}}, + ULID(4): {Thanos: metadata.Thanos{Labels: map[string]string{}}}, + }, + modified: 5.0, + }, + } { + m := newTestSyncMetrics() + testutil.Ok(t, rm.Modify(ctx, tcase.input, m, false)) - testutil.Equals(t, 5.0, promtest.ToFloat64(m.modified.WithLabelValues(replicaRemovedMeta))) - testutil.Equals(t, expected, input) + testutil.Equals(t, tcase.modified, promtest.ToFloat64(m.modified.WithLabelValues(replicaRemovedMeta))) + testutil.Equals(t, tcase.expected, tcase.input) + } } func compareSliceWithMapKeys(tb testing.TB, m map[ulid.ULID]*metadata.Meta, s []ulid.ULID) { From 51fbdd3db12a78f5f94cd93e0c259148d8b9fd7b Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Wed, 11 Mar 2020 14:57:20 +0100 Subject: [PATCH 06/14] Add simple compactor e2e tests with replica remover Signed-off-by: Kemal Akkoyun --- cmd/thanos/compact.go | 10 ++++- pkg/block/fetcher.go | 8 +++- pkg/block/fetcher_test.go | 2 +- test/e2e/compact_test.go | 80 ++++++++++++++++++++++++++++++---- test/e2e/e2ethanos/services.go | 7 ++- 5 files changed, 91 insertions(+), 16 deletions(-) diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index 423321fb78..97ed17b930 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -290,13 +290,19 @@ func runCompact( block.NewConsistencyDelayMetaFilter(logger, consistencyDelay, prometheusRegisterer).Filter, ignoreDeletionMarkFilter.Filter, duplicateBlocksFilter.Filter, - block.NewReplicaLabelRemover(dedupReplicaLabels).Modify, + block.NewReplicaLabelRemover(logger, dedupReplicaLabels).Modify, ) if err != nil { return errors.Wrap(err, "create meta fetcher") } - sy, err := compact.NewSyncer(logger, reg, bkt, metaFetcher, duplicateBlocksFilter, ignoreDeletionMarkFilter, blocksMarkedForDeletion, blockSyncConcurrency, acceptMalformedIndex, false) + enableVerticalCompaction := false + if len(dedupReplicaLabels) > 0 { + enableVerticalCompaction = true + level.Info(logger).Log("msg", "deduplication.replica-label specified, vertical compaction is enabled", "dedup-replica-labels", strings.Join(dedupReplicaLabels, ",")) + } + + sy, err := compact.NewSyncer(logger, reg, bkt, metaFetcher, duplicateBlocksFilter, ignoreDeletionMarkFilter, blocksMarkedForDeletion, blockSyncConcurrency, acceptMalformedIndex, enableVerticalCompaction) if err != nil { return errors.Wrap(err, "create syncer") } diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index ac28a0ada3..2144c3466b 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -370,6 +370,7 @@ func (s *MetaFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata. s.metrics.synced.WithLabelValues(loadedMeta).Set(float64(len(metas))) s.metrics.synced.Submit() + s.metrics.modified.Submit() if incompleteView { return metas, partial, errors.Wrap(metaErrs, "incomplete view") @@ -554,12 +555,14 @@ func contains(s1 []ulid.ULID, s2 []ulid.ULID) bool { // ReplicaLabelRemover is a MetaFetcher modifier modifies external labels of existing blocks, it removes given replica labels from the metadata of blocks that have it. type ReplicaLabelRemover struct { + logger log.Logger + replicaLabels []string } // NewReplicaLabelRemover creates a ReplicaLabelRemover. -func NewReplicaLabelRemover(replicaLabels []string) *ReplicaLabelRemover { - return &ReplicaLabelRemover{replicaLabels: replicaLabels} +func NewReplicaLabelRemover(logger log.Logger, replicaLabels []string) *ReplicaLabelRemover { + return &ReplicaLabelRemover{logger: logger, replicaLabels: replicaLabels} } // Modify modifies external labels of existing blocks, it removes given replica labels from the metadata of blocks that have it. @@ -568,6 +571,7 @@ func (r *ReplicaLabelRemover) Modify(_ context.Context, metas map[ulid.ULID]*met labels := meta.Thanos.Labels for _, replicaLabel := range r.replicaLabels { if _, exists := labels[replicaLabel]; exists { + level.Debug(r.logger).Log("msg", "replica label removed", "label", replicaLabel) delete(labels, replicaLabel) metrics.modified.WithLabelValues(replicaRemovedMeta).Inc() } diff --git a/pkg/block/fetcher_test.go b/pkg/block/fetcher_test.go index 60f94c8fba..4b44b508c4 100644 --- a/pkg/block/fetcher_test.go +++ b/pkg/block/fetcher_test.go @@ -880,7 +880,7 @@ func TestDeduplicateFilter_Filter(t *testing.T) { func TestReplicaLabelRemover_Modify(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) defer cancel() - rm := NewReplicaLabelRemover([]string{"replica", "rule_replica"}) + rm := NewReplicaLabelRemover(log.NewNopLogger(), []string{"replica", "rule_replica"}) for _, tcase := range []struct { name string diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index 457a518a30..b4c12697e9 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -1,7 +1,7 @@ // Copyright (c) The Thanos Authors. // Licensed under the Apache License 2.0. -package e2e +package e2e_test import ( "context" @@ -14,11 +14,13 @@ import ( "github.com/cortexproject/cortex/integration/e2e" e2edb "github.com/cortexproject/cortex/integration/e2e/db" "github.com/go-kit/kit/log" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/timestamp" "github.com/thanos-io/thanos/pkg/objstore" "github.com/thanos-io/thanos/pkg/objstore/client" "github.com/thanos-io/thanos/pkg/objstore/s3" + "github.com/thanos-io/thanos/pkg/promclient" "github.com/thanos-io/thanos/pkg/testutil" "github.com/thanos-io/thanos/pkg/testutil/e2eutil" "github.com/thanos-io/thanos/test/e2e/e2ethanos" @@ -40,27 +42,58 @@ func TestCompact(t *testing.T) { m := e2edb.NewMinio(80, bucket) testutil.Ok(t, s.StartAndWaitReady(m)) - series := []labels.Labels{labels.FromStrings("a", "1", "b", "2")} - extLset := labels.FromStrings("ext1", "value1", "replica", "1") - bkt, err := s3.NewBucketWithConfig(l, s3.Config{ Bucket: bucket, AccessKey: e2edb.MinioAccessKey, SecretKey: e2edb.MinioSecretKey, - Endpoint: m.HTTPEndpoint(), + Endpoint: m.HTTPEndpoint(), // We need separate client config, when connecting to minio from outside. Insecure: true, }, "test-feed") testutil.Ok(t, err) dir := filepath.Join(s.SharedDir(), "tmp") + testutil.Ok(t, os.MkdirAll(filepath.Join(s.SharedDir(), dir), os.ModePerm)) + + series := []labels.Labels{labels.FromStrings("a", "1", "b", "2")} + extLset1 := labels.FromStrings("ext1", "value1", "replica", "1") + extLset2 := labels.FromStrings("ext1", "value1", "replica", "2") + extLset3 := labels.FromStrings("ext1", "value1", "rule_replica", "1") now := time.Now() - id1, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, series, 10, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset, 0) + id1, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, series, 10, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset1, 0) testutil.Ok(t, err) - testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id1.String()), id1.String())) - compact, err := e2ethanos.NewCompact(s.SharedDir(), "compact", client.BucketConfig{ + id2, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, series, 10, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset2, 0) + testutil.Ok(t, err) + testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id2.String()), id2.String())) + + id3, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, series, 10, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset3, 0) + testutil.Ok(t, err) + testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id3.String()), id3.String())) + + cmpt, err := e2ethanos.NewCompactor(s.SharedDir(), "1", client.BucketConfig{ + Type: client.S3, + Config: s3.Config{ + Bucket: bucket, + AccessKey: e2edb.MinioAccessKey, + SecretKey: e2edb.MinioSecretKey, + Endpoint: m.NetworkHTTPEndpoint(), + Insecure: true, + }, + }, + "--deduplication.replica-label=replica", + "--deduplication.replica-label=rule_replica", + ) + + testutil.Ok(t, err) + testutil.Ok(t, s.StartAndWaitReady(cmpt)) + + testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(3), "thanos_blocks_meta_synced")) + testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) + testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(3), "thanos_blocks_meta_modified")) + + str, err := e2ethanos.NewStoreGW(s.SharedDir(), "1", client.BucketConfig{ Type: client.S3, Config: s3.Config{ Bucket: bucket, @@ -70,7 +103,36 @@ func TestCompact(t *testing.T) { Insecure: true, }, }) + testutil.Ok(t, err) + testutil.Ok(t, s.StartAndWaitReady(str)) + + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(1), "thanos_blocks_meta_synced")) + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) + q, err := e2ethanos.NewQuerier(s.SharedDir(), "1", []string{str.GRPCNetworkEndpoint()}, nil) testutil.Ok(t, err) - testutil.Ok(t, s.StartAndWaitReady(compact)) + testutil.Ok(t, s.StartAndWaitReady(q)) + + ctx, cancel = context.WithTimeout(context.Background(), 3*time.Minute) + defer cancel() + + t.Run("query works", func(t *testing.T) { + queryAndAssert(t, ctx, q.HTTPEndpoint(), "{a=\"1\"}", + promclient.QueryOptions{ + Deduplicate: false, // This should be false, so that we can be sure deduplication was offline. + }, + []model.Metric{ + { + "a": "1", + "b": "2", + "ext1": "value1", + }, + }, + ) + + // Make sure only necessary amount of blocks fetched from store, to observe affects of offline deduplication. + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(3), "thanos_bucket_store_series_data_touched")) + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(3), "thanos_bucket_store_series_data_fetched")) + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(1), "thanos_bucket_store_series_blocks_queried")) + }) } diff --git a/test/e2e/e2ethanos/services.go b/test/e2e/e2ethanos/services.go index 66658dee3a..84f25269f6 100644 --- a/test/e2e/e2ethanos/services.go +++ b/test/e2e/e2ethanos/services.go @@ -314,7 +314,7 @@ func NewStoreGW(sharedDir string, name string, bucketConfig client.BucketConfig, ), nil } -func NewCompact(sharedDir string, name string, bucketConfig client.BucketConfig) (*Service, error) { +func NewCompactor(sharedDir string, name string, bucketConfig client.BucketConfig, extArgs ...string) (*Service, error) { dir := filepath.Join(sharedDir, "data", "compact", name) container := filepath.Join(e2e.ContainerSharedDir, "data", "compact", name) @@ -331,10 +331,13 @@ func NewCompact(sharedDir string, name string, bucketConfig client.BucketConfig) fmt.Sprintf("compact-%s", name), DefaultImage(), e2e.NewCommand("compact", append(e2e.BuildArgs(map[string]string{ + "--debug.name": fmt.Sprintf("compact-%s", name), + "--log.level": logLevel, "--data-dir": container, "--objstore.config": string(bktConfigBytes), "--http-address": ":80", - }))...), + "--wait": "", + }), extArgs...)...), e2e.NewReadinessProbe(80, "/-/ready", 200), 80, 9091, From 25d2abe58998bb4d5dd4cf37c01a224728266324 Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Wed, 11 Mar 2020 18:48:51 +0100 Subject: [PATCH 07/14] Remove unnecessary interface Signed-off-by: Kemal Akkoyun --- pkg/block/fetcher.go | 12 +++--------- pkg/block/fetcher_test.go | 15 ++------------- 2 files changed, 5 insertions(+), 22 deletions(-) diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index 2144c3466b..c3ef935148 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -32,19 +32,13 @@ import ( "github.com/thanos-io/thanos/pkg/runutil" ) -type txLabeledGauge interface { - WithLabelValues(lvs ...string) prometheus.Gauge - ResetTx() - Submit() -} - type syncMetrics struct { syncs prometheus.Counter syncFailures prometheus.Counter syncDuration prometheus.Histogram - synced txLabeledGauge - modified txLabeledGauge + synced *extprom.TxGaugeVec + modified *extprom.TxGaugeVec } const ( @@ -478,7 +472,7 @@ func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metad return nil } -func (f *DeduplicateFilter) filterForResolution(root *Node, metaSlice []*metadata.Meta, metas map[ulid.ULID]*metadata.Meta, res int64, synced txLabeledGauge) { +func (f *DeduplicateFilter) filterForResolution(root *Node, metaSlice []*metadata.Meta, metas map[ulid.ULID]*metadata.Meta, res int64, synced *extprom.TxGaugeVec) { sort.Slice(metaSlice, func(i, j int) bool { ilen := len(metaSlice[i].Compaction.Sources) jlen := len(metaSlice[j].Compaction.Sources) diff --git a/pkg/block/fetcher_test.go b/pkg/block/fetcher_test.go index 4b44b508c4..12cdd3a45b 100644 --- a/pkg/block/fetcher_test.go +++ b/pkg/block/fetcher_test.go @@ -22,7 +22,6 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" promtest "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/prometheus/pkg/relabel" "github.com/prometheus/prometheus/tsdb" @@ -35,20 +34,10 @@ import ( "gopkg.in/yaml.v2" ) -type testTxLabeledGauge struct { - g *prometheus.GaugeVec -} - -func (t *testTxLabeledGauge) WithLabelValues(lvs ...string) prometheus.Gauge { - return t.g.WithLabelValues(lvs...) -} -func (t *testTxLabeledGauge) ResetTx() {} -func (t *testTxLabeledGauge) Submit() {} - func newTestSyncMetrics() *syncMetrics { return &syncMetrics{ - synced: &testTxLabeledGauge{promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"state"})}, - modified: &testTxLabeledGauge{promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{"modified"})}, + synced: extprom.NewTxGaugeVec(nil, prometheus.GaugeOpts{}, []string{"state"}), + modified: extprom.NewTxGaugeVec(nil, prometheus.GaugeOpts{}, []string{"modified"}), } } From 86b9949417ceaef060f0a6ca2484c3041831aeee Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Wed, 11 Mar 2020 18:49:40 +0100 Subject: [PATCH 08/14] Address review issues Signed-off-by: Kemal Akkoyun --- test/e2e/compact_test.go | 308 +++++++++++++++++++++++++++------------ 1 file changed, 212 insertions(+), 96 deletions(-) diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index b4c12697e9..e3aef8e1c4 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -8,15 +8,18 @@ import ( "os" "path" "path/filepath" + "strconv" "testing" "time" "github.com/cortexproject/cortex/integration/e2e" e2edb "github.com/cortexproject/cortex/integration/e2e/db" "github.com/go-kit/kit/log" + "github.com/oklog/ulid" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/timestamp" + "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/objstore" "github.com/thanos-io/thanos/pkg/objstore/client" "github.com/thanos-io/thanos/pkg/objstore/s3" @@ -30,109 +33,222 @@ func TestCompact(t *testing.T) { t.Parallel() l := log.NewLogfmtLogger(os.Stdout) - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) - defer cancel() - - s, err := e2e.NewScenario("e2e_test_compact") - testutil.Ok(t, err) - defer s.Close() - - const bucket = "thanos" - - m := e2edb.NewMinio(80, bucket) - testutil.Ok(t, s.StartAndWaitReady(m)) - - bkt, err := s3.NewBucketWithConfig(l, s3.Config{ - Bucket: bucket, - AccessKey: e2edb.MinioAccessKey, - SecretKey: e2edb.MinioSecretKey, - Endpoint: m.HTTPEndpoint(), // We need separate client config, when connecting to minio from outside. - Insecure: true, - }, "test-feed") - testutil.Ok(t, err) - - dir := filepath.Join(s.SharedDir(), "tmp") - testutil.Ok(t, os.MkdirAll(filepath.Join(s.SharedDir(), dir), os.ModePerm)) - - series := []labels.Labels{labels.FromStrings("a", "1", "b", "2")} - extLset1 := labels.FromStrings("ext1", "value1", "replica", "1") - extLset2 := labels.FromStrings("ext1", "value1", "replica", "2") - extLset3 := labels.FromStrings("ext1", "value1", "rule_replica", "1") - - now := time.Now() - id1, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, series, 10, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset1, 0) - testutil.Ok(t, err) - testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id1.String()), id1.String())) - - id2, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, series, 10, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset2, 0) - testutil.Ok(t, err) - testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id2.String()), id2.String())) - - id3, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, series, 10, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset3, 0) - testutil.Ok(t, err) - testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id3.String()), id3.String())) - - cmpt, err := e2ethanos.NewCompactor(s.SharedDir(), "1", client.BucketConfig{ - Type: client.S3, - Config: s3.Config{ - Bucket: bucket, - AccessKey: e2edb.MinioAccessKey, - SecretKey: e2edb.MinioSecretKey, - Endpoint: m.NetworkHTTPEndpoint(), - Insecure: true, - }, - }, - "--deduplication.replica-label=replica", - "--deduplication.replica-label=rule_replica", - ) - - testutil.Ok(t, err) - testutil.Ok(t, s.StartAndWaitReady(cmpt)) - - testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(3), "thanos_blocks_meta_synced")) - testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) - testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(3), "thanos_blocks_meta_modified")) - - str, err := e2ethanos.NewStoreGW(s.SharedDir(), "1", client.BucketConfig{ - Type: client.S3, - Config: s3.Config{ - Bucket: bucket, - AccessKey: e2edb.MinioAccessKey, - SecretKey: e2edb.MinioSecretKey, - Endpoint: m.NetworkHTTPEndpoint(), - Insecure: true, - }, - }) - testutil.Ok(t, err) - testutil.Ok(t, s.StartAndWaitReady(str)) - - testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(1), "thanos_blocks_meta_synced")) - testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) - - q, err := e2ethanos.NewQuerier(s.SharedDir(), "1", []string{str.GRPCNetworkEndpoint()}, nil) - testutil.Ok(t, err) - testutil.Ok(t, s.StartAndWaitReady(q)) - - ctx, cancel = context.WithTimeout(context.Background(), 3*time.Minute) - defer cancel() + // blockDesc describes a recipe to generate blocks from the given series and external labels. + type blockDesc struct { + series []labels.Labels + extLsets []labels.Labels + numberOfSamplesPerSeries int + } + + for i, tcase := range []struct { + name string + blocks []blockDesc + replicaLabels []string + query string + + expected []model.Metric + numberOfModifiedBlocks float64 + numberOfBlocks uint64 + numberOfSamples uint64 + numberOfSeries uint64 + numberOfChunks uint64 + }{ + { + name: "overlapping blocks with matching replica labels", + blocks: []blockDesc{ + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLsets: []labels.Labels{ + labels.FromStrings("ext1", "value1", "replica", "1"), + labels.FromStrings("ext1", "value1", "replica", "2"), + labels.FromStrings("ext1", "value1", "rule_replica", "1"), + }, + numberOfSamplesPerSeries: 10, + }, + }, + replicaLabels: []string{"replica", "rule_replica"}, + query: "{a=\"1\"}", - t.Run("query works", func(t *testing.T) { - queryAndAssert(t, ctx, q.HTTPEndpoint(), "{a=\"1\"}", - promclient.QueryOptions{ - Deduplicate: false, // This should be false, so that we can be sure deduplication was offline. + expected: []model.Metric{ + { + "a": "1", + "b": "2", + "ext1": "value1", + }, + }, + numberOfModifiedBlocks: 3, + numberOfBlocks: 1, + numberOfSamples: 10, + numberOfSeries: 1, + numberOfChunks: 1, + }, + { + name: "overlapping blocks with non-matching replica labels", + blocks: []blockDesc{ + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLsets: []labels.Labels{ + labels.FromStrings("ext1", "value1"), + labels.FromStrings("ext1", "value2"), + }, + numberOfSamplesPerSeries: 2, + }, }, - []model.Metric{ + replicaLabels: []string{"replica"}, + query: "{a=\"1\"}", + + expected: []model.Metric{ { "a": "1", "b": "2", "ext1": "value1", }, + { + "a": "1", + "b": "2", + "ext1": "value2", + }, }, - ) + numberOfModifiedBlocks: 0, + numberOfBlocks: 2, + numberOfSamples: 4, + numberOfSeries: 2, + numberOfChunks: 2, + }, + } { + i := i + tcase := tcase + t.Run(tcase.name, func(t *testing.T) { + t.Parallel() + + s, err := e2e.NewScenario("e2e_test_compact_" + strconv.Itoa(i)) + testutil.Ok(t, err) + defer s.Close() + + dir := filepath.Join(s.SharedDir(), "tmp") + testutil.Ok(t, os.MkdirAll(filepath.Join(s.SharedDir(), dir), os.ModePerm)) + + bucket := "thanos_" + strconv.Itoa(i) + + m := e2edb.NewMinio(8080+i, bucket) + testutil.Ok(t, s.StartAndWaitReady(m)) + + bkt, err := s3.NewBucketWithConfig(l, s3.Config{ + Bucket: bucket, + AccessKey: e2edb.MinioAccessKey, + SecretKey: e2edb.MinioSecretKey, + Endpoint: m.HTTPEndpoint(), // We need separate client config, when connecting to minio from outside. + Insecure: true, + }, "test-feed") + testutil.Ok(t, err) + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) + defer cancel() + + now := time.Now() + + var rawBlockIds []ulid.ULID + for _, b := range tcase.blocks { + for _, extLset := range b.extLsets { + id, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, b.series, b.numberOfSamplesPerSeries, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset, 0) + testutil.Ok(t, err) + testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id.String()), id.String())) + rawBlockIds = append(rawBlockIds, id) + } + } + + dedupFlags := make([]string, 0, len(tcase.replicaLabels)) + for _, l := range tcase.replicaLabels { + dedupFlags = append(dedupFlags, "--deduplication.replica-label="+l) + } + + cmpt, err := e2ethanos.NewCompactor(s.SharedDir(), strconv.Itoa(i), client.BucketConfig{ + Type: client.S3, + Config: s3.Config{ + Bucket: bucket, + AccessKey: e2edb.MinioAccessKey, + SecretKey: e2edb.MinioSecretKey, + Endpoint: m.NetworkHTTPEndpoint(), + Insecure: true, + }, + }, + dedupFlags..., + ) + + testutil.Ok(t, err) + testutil.Ok(t, s.StartAndWaitReady(cmpt)) + + testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(float64(len(rawBlockIds))), "thanos_blocks_meta_synced")) + testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) + testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(tcase.numberOfModifiedBlocks), "thanos_blocks_meta_modified")) + + str, err := e2ethanos.NewStoreGW(s.SharedDir(), "compact_"+strconv.Itoa(i), client.BucketConfig{ + Type: client.S3, + Config: s3.Config{ + Bucket: bucket, + AccessKey: e2edb.MinioAccessKey, + SecretKey: e2edb.MinioSecretKey, + Endpoint: m.NetworkHTTPEndpoint(), + Insecure: true, + }, + }) + testutil.Ok(t, err) + testutil.Ok(t, s.StartAndWaitReady(str)) + + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(float64(tcase.numberOfBlocks)), "thanos_blocks_meta_synced")) + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) - // Make sure only necessary amount of blocks fetched from store, to observe affects of offline deduplication. - testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(3), "thanos_bucket_store_series_data_touched")) - testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(3), "thanos_bucket_store_series_data_fetched")) - testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(1), "thanos_bucket_store_series_blocks_queried")) - }) + q, err := e2ethanos.NewQuerier(s.SharedDir(), "compact_"+strconv.Itoa(i), []string{str.GRPCNetworkEndpoint()}, nil) + testutil.Ok(t, err) + testutil.Ok(t, s.StartAndWaitReady(q)) + + ctx, cancel = context.WithTimeout(context.Background(), 3*time.Minute) + defer cancel() + + queryAndAssert(t, ctx, q.HTTPEndpoint(), + tcase.query, + promclient.QueryOptions{ + Deduplicate: false, // This should be false, so that we can be sure deduplication was offline. + }, + tcase.expected, + ) + + var numberOfBlocks uint64 + var numberOfSamples uint64 + var numberOfSeries uint64 + var numberOfChunks uint64 + var sources []ulid.ULID + + testutil.Ok(t, bkt.Iter(ctx, "", func(n string) error { + id, ok := block.IsBlockDir(n) + if !ok { + return nil + } + + numberOfBlocks += 1 + + meta, err := block.DownloadMeta(ctx, l, bkt, id) + if err != nil { + return err + } + + numberOfSamples += meta.Stats.NumSamples + numberOfSeries += meta.Stats.NumSeries + numberOfChunks += meta.Stats.NumChunks + sources = append(sources, meta.Compaction.Sources...) + + return nil + })) + + // Make sure only necessary amount of blocks fetched from store, to observe affects of offline deduplication. + testutil.Equals(t, tcase.numberOfBlocks, numberOfBlocks) + if len(rawBlockIds) < int(tcase.numberOfBlocks) { // check sources only if compacted. + testutil.Equals(t, rawBlockIds, sources) + } + + testutil.Equals(t, tcase.numberOfSamples, numberOfSamples) + testutil.Equals(t, tcase.numberOfSeries, numberOfSeries) + testutil.Equals(t, tcase.numberOfChunks, numberOfChunks) + }) + } } From 74763e4e96aa31fca8bf7dc285c45b3c45093a24 Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Fri, 13 Mar 2020 19:06:37 +0100 Subject: [PATCH 09/14] Add more test cases Signed-off-by: Kemal Akkoyun --- CHANGELOG.md | 1 + pkg/block/fetcher.go | 1 + test/e2e/compact_test.go | 372 ++++++++++++++++++++++++++------- test/e2e/e2ethanos/services.go | 24 ++- 4 files changed, 318 insertions(+), 80 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 20b1ac5c5d..abcf91055b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,6 +20,7 @@ We use *breaking* word for marking changes that are not backward compatible (rel ### Added - [#2265](https://github.com/thanos-io/thanos/pull/2265) Compactor: Add `--wait-interval` to specify compaction wait interval between consecutive compact runs when `--wait` enabled. +- [#2250](https://github.com/thanos-io/thanos/pull/2250) Compactor: Enable vertical compaction for offline deduplication (Experimental). Uses `--deduplication.replica-label` flag to specify the replica label to deduplicate on (Hidden). ### Changed diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index c3ef935148..4f88b0dba0 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -261,6 +261,7 @@ func (s *MetaFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata. ) s.metrics.synced.ResetTx() + s.metrics.modified.ResetTx() for i := 0; i < s.concurrency; i++ { wg.Add(1) diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index e3aef8e1c4..720328cacc 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -35,39 +35,65 @@ func TestCompact(t *testing.T) { // blockDesc describes a recipe to generate blocks from the given series and external labels. type blockDesc struct { - series []labels.Labels - extLsets []labels.Labels - numberOfSamplesPerSeries int + series []labels.Labels + extLset labels.Labels + mint int64 + maxt int64 + samplesPerSeries int } + type retention struct { + resRaw string + res5m string + res1h string + } + + delay := 30 * time.Minute + now := time.Now() + for i, tcase := range []struct { - name string - blocks []blockDesc - replicaLabels []string - query string - - expected []model.Metric - numberOfModifiedBlocks float64 - numberOfBlocks uint64 - numberOfSamples uint64 - numberOfSeries uint64 - numberOfChunks uint64 + name string + blocks []blockDesc + replicaLabels []string + downsamplingEnabled bool + retention *retention + query string + + expected []model.Metric + expectOfModBlocks float64 + expectOfBlocks uint64 + expectOfSamples uint64 + expectOfSeries uint64 + expectOfChunks uint64 }{ { - name: "overlapping blocks with matching replica labels", + name: "(full) vertically overlapping blocks with replica labels", blocks: []blockDesc{ { - series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLsets: []labels.Labels{ - labels.FromStrings("ext1", "value1", "replica", "1"), - labels.FromStrings("ext1", "value1", "replica", "2"), - labels.FromStrings("ext1", "value1", "rule_replica", "1"), - }, - numberOfSamplesPerSeries: 10, + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "rule_replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, }, }, - replicaLabels: []string{"replica", "rule_replica"}, - query: "{a=\"1\"}", + replicaLabels: []string{"replica", "rule_replica"}, + downsamplingEnabled: true, + query: "{a=\"1\"}", expected: []model.Metric{ { @@ -76,26 +102,226 @@ func TestCompact(t *testing.T) { "ext1": "value1", }, }, - numberOfModifiedBlocks: 3, - numberOfBlocks: 1, - numberOfSamples: 10, - numberOfSeries: 1, - numberOfChunks: 1, + expectOfModBlocks: 3, + expectOfBlocks: 1, + expectOfSamples: 120, + expectOfSeries: 1, + expectOfChunks: 2, }, { - name: "overlapping blocks with non-matching replica labels", + name: "(full) vertically overlapping blocks with replica labels, downsampling disabled", blocks: []blockDesc{ { - series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLsets: []labels.Labels{ - labels.FromStrings("ext1", "value1"), - labels.FromStrings("ext1", "value2"), - }, - numberOfSamplesPerSeries: 2, + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "rule_replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, }, }, - replicaLabels: []string{"replica"}, - query: "{a=\"1\"}", + replicaLabels: []string{"replica", "rule_replica"}, + downsamplingEnabled: false, + query: "{a=\"1\"}", + + expected: []model.Metric{ + { + "a": "1", + "b": "2", + "ext1": "value1", + }, + }, + expectOfModBlocks: 3, + expectOfBlocks: 1, + expectOfSamples: 120, + expectOfSeries: 1, + expectOfChunks: 2, + }, + { + name: "(partial) vertically overlapping blocks with replica labels", + blocks: []blockDesc{ + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(1 * time.Hour)), + samplesPerSeries: 60, + }, + }, + replicaLabels: []string{"replica"}, + downsamplingEnabled: true, + query: "{a=\"1\"}", + + expected: []model.Metric{ + { + "a": "1", + "b": "2", + "ext1": "value1", + }, + }, + expectOfModBlocks: 2, + expectOfBlocks: 1, + expectOfSamples: 179, // TODO(kakkoyun): ? + expectOfSeries: 1, + expectOfChunks: 2, + }, + { + name: "(contains) vertically overlapping blocks with replica labels", + blocks: []blockDesc{ + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + mint: timestamp.FromTime(now.Add(30 * time.Minute)), + maxt: timestamp.FromTime(now.Add(1 * time.Hour)), + samplesPerSeries: 90, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + }, + replicaLabels: []string{"replica"}, + downsamplingEnabled: true, + query: "{a=\"1\"}", + + expected: []model.Metric{ + { + "a": "1", + "b": "2", + "ext1": "value1", + }, + }, + expectOfModBlocks: 2, + expectOfBlocks: 1, + expectOfSamples: 210, // TODO(kakkoyun): ? + expectOfSeries: 1, + expectOfChunks: 2, + }, + { + name: "(shifted) vertically overlapping blocks with replica labels", + blocks: []blockDesc{ + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + mint: timestamp.FromTime(now.Add(30 * time.Minute)), + maxt: timestamp.FromTime(now.Add(150 * time.Minute)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + mint: timestamp.FromTime(now.Add(-30 * time.Minute)), + maxt: timestamp.FromTime(now.Add(90 * time.Minute)), + samplesPerSeries: 120, + }, + }, + replicaLabels: []string{"replica"}, + downsamplingEnabled: true, + query: "{a=\"1\"}", + + expected: []model.Metric{ + { + "a": "1", + "b": "2", + "ext1": "value1", + }, + }, + expectOfModBlocks: 2, + expectOfBlocks: 1, + expectOfSamples: 240, // TODO(kakkoyun): ? + expectOfSeries: 1, + expectOfChunks: 2, + }, + { + name: "(full) vertically overlapping blocks with replica labels, retention specified", + blocks: []blockDesc{ + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "rule_replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + }, + replicaLabels: []string{"replica", "rule_replica"}, + downsamplingEnabled: true, + retention: &retention{ + resRaw: "0d", + res5m: "5m", + res1h: "5m", + }, + query: "{a=\"1\"}", + + expected: []model.Metric{ + { + "a": "1", + "b": "2", + "ext1": "value1", + }, + }, + expectOfModBlocks: 3, + expectOfBlocks: 1, + expectOfSamples: 120, + expectOfSeries: 1, + expectOfChunks: 2, + }, + { + name: "(full) vertically overlapping blocks without replica labels", + blocks: []blockDesc{ + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 2, + }, { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 2, + }, + }, + replicaLabels: []string{"replica"}, + downsamplingEnabled: true, + query: "{a=\"1\"}", expected: []model.Metric{ { @@ -109,11 +335,11 @@ func TestCompact(t *testing.T) { "ext1": "value2", }, }, - numberOfModifiedBlocks: 0, - numberOfBlocks: 2, - numberOfSamples: 4, - numberOfSeries: 2, - numberOfChunks: 2, + expectOfModBlocks: 0, + expectOfBlocks: 2, + expectOfSamples: 4, + expectOfSeries: 2, + expectOfChunks: 2, }, } { i := i @@ -125,7 +351,7 @@ func TestCompact(t *testing.T) { testutil.Ok(t, err) defer s.Close() - dir := filepath.Join(s.SharedDir(), "tmp") + dir := filepath.Join(s.SharedDir(), "tmp_"+strconv.Itoa(i)) testutil.Ok(t, os.MkdirAll(filepath.Join(s.SharedDir(), dir), os.ModePerm)) bucket := "thanos_" + strconv.Itoa(i) @@ -145,16 +371,12 @@ func TestCompact(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) defer cancel() - now := time.Now() - var rawBlockIds []ulid.ULID for _, b := range tcase.blocks { - for _, extLset := range b.extLsets { - id, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, b.series, b.numberOfSamplesPerSeries, timestamp.FromTime(now), timestamp.FromTime(now.Add(2*time.Hour)), 30*time.Minute, extLset, 0) - testutil.Ok(t, err) - testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id.String()), id.String())) - rawBlockIds = append(rawBlockIds, id) - } + id, err := e2eutil.CreateBlockWithBlockDelay(ctx, dir, b.series, b.samplesPerSeries, b.mint, b.maxt, delay, b.extLset, 0) + testutil.Ok(t, err) + testutil.Ok(t, objstore.UploadDir(ctx, l, bkt, path.Join(dir, id.String()), id.String())) + rawBlockIds = append(rawBlockIds, id) } dedupFlags := make([]string, 0, len(tcase.replicaLabels)) @@ -162,6 +384,13 @@ func TestCompact(t *testing.T) { dedupFlags = append(dedupFlags, "--deduplication.replica-label="+l) } + retenFlags := make([]string, 0, 3) + if tcase.retention != nil { + retenFlags = append(retenFlags, "--retention.resolution-raw="+tcase.retention.resRaw) + retenFlags = append(retenFlags, "--retention.resolution-5m="+tcase.retention.res5m) + retenFlags = append(retenFlags, "--retention.resolution-1h="+tcase.retention.res1h) + } + cmpt, err := e2ethanos.NewCompactor(s.SharedDir(), strconv.Itoa(i), client.BucketConfig{ Type: client.S3, Config: s3.Config{ @@ -172,15 +401,16 @@ func TestCompact(t *testing.T) { Insecure: true, }, }, - dedupFlags..., + nil, // relabel configs. + tcase.downsamplingEnabled, + append(dedupFlags, retenFlags...)..., ) testutil.Ok(t, err) testutil.Ok(t, s.StartAndWaitReady(cmpt)) - testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(float64(len(rawBlockIds))), "thanos_blocks_meta_synced")) testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) - testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(tcase.numberOfModifiedBlocks), "thanos_blocks_meta_modified")) + testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(tcase.expectOfModBlocks), "thanos_blocks_meta_modified")) str, err := e2ethanos.NewStoreGW(s.SharedDir(), "compact_"+strconv.Itoa(i), client.BucketConfig{ Type: client.S3, @@ -194,8 +424,7 @@ func TestCompact(t *testing.T) { }) testutil.Ok(t, err) testutil.Ok(t, s.StartAndWaitReady(str)) - - testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(float64(tcase.numberOfBlocks)), "thanos_blocks_meta_synced")) + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(float64(tcase.expectOfBlocks)), "thanos_blocks_meta_synced")) testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) q, err := e2ethanos.NewQuerier(s.SharedDir(), "compact_"+strconv.Itoa(i), []string{str.GRPCNetworkEndpoint()}, nil) @@ -213,42 +442,39 @@ func TestCompact(t *testing.T) { tcase.expected, ) - var numberOfBlocks uint64 - var numberOfSamples uint64 - var numberOfSeries uint64 - var numberOfChunks uint64 + var actualOfBlocks uint64 + var actualOfChunks uint64 + var actualOfSeries uint64 + var actualOfSamples uint64 var sources []ulid.ULID - testutil.Ok(t, bkt.Iter(ctx, "", func(n string) error { id, ok := block.IsBlockDir(n) if !ok { return nil } - numberOfBlocks += 1 + actualOfBlocks += 1 meta, err := block.DownloadMeta(ctx, l, bkt, id) if err != nil { return err } - numberOfSamples += meta.Stats.NumSamples - numberOfSeries += meta.Stats.NumSeries - numberOfChunks += meta.Stats.NumChunks + actualOfChunks += meta.Stats.NumChunks + actualOfSeries += meta.Stats.NumSeries + actualOfSamples += meta.Stats.NumSamples sources = append(sources, meta.Compaction.Sources...) - return nil })) // Make sure only necessary amount of blocks fetched from store, to observe affects of offline deduplication. - testutil.Equals(t, tcase.numberOfBlocks, numberOfBlocks) - if len(rawBlockIds) < int(tcase.numberOfBlocks) { // check sources only if compacted. + testutil.Equals(t, tcase.expectOfBlocks, actualOfBlocks) + if len(rawBlockIds) < int(tcase.expectOfBlocks) { // check sources only if compacted. testutil.Equals(t, rawBlockIds, sources) } - - testutil.Equals(t, tcase.numberOfSamples, numberOfSamples) - testutil.Equals(t, tcase.numberOfSeries, numberOfSeries) - testutil.Equals(t, tcase.numberOfChunks, numberOfChunks) + testutil.Equals(t, tcase.expectOfChunks, actualOfChunks) + testutil.Equals(t, tcase.expectOfSeries, actualOfSeries) + testutil.Equals(t, tcase.expectOfSamples, actualOfSamples) }) } } diff --git a/test/e2e/e2ethanos/services.go b/test/e2e/e2ethanos/services.go index 84f25269f6..201b2f959b 100644 --- a/test/e2e/e2ethanos/services.go +++ b/test/e2e/e2ethanos/services.go @@ -314,7 +314,7 @@ func NewStoreGW(sharedDir string, name string, bucketConfig client.BucketConfig, ), nil } -func NewCompactor(sharedDir string, name string, bucketConfig client.BucketConfig, extArgs ...string) (*Service, error) { +func NewCompactor(sharedDir string, name string, bucketConfig client.BucketConfig, relabelConfig []relabel.Config, downsamplingEnabled bool, extArgs ...string) (*Service, error) { dir := filepath.Join(sharedDir, "data", "compact", name) container := filepath.Join(e2e.ContainerSharedDir, "data", "compact", name) @@ -327,16 +327,26 @@ func NewCompactor(sharedDir string, name string, bucketConfig client.BucketConfi return nil, errors.Wrapf(err, "generate compact config file: %v", bucketConfig) } + relabelConfigBytes, err := yaml.Marshal(relabelConfig) + if err != nil { + return nil, errors.Wrapf(err, "generate compact relabel file: %v", relabelConfig) + } + + if !downsamplingEnabled { + extArgs = append(extArgs, "--downsampling.disable") + } + return NewService( fmt.Sprintf("compact-%s", name), DefaultImage(), e2e.NewCommand("compact", append(e2e.BuildArgs(map[string]string{ - "--debug.name": fmt.Sprintf("compact-%s", name), - "--log.level": logLevel, - "--data-dir": container, - "--objstore.config": string(bktConfigBytes), - "--http-address": ":80", - "--wait": "", + "--debug.name": fmt.Sprintf("compact-%s", name), + "--log.level": logLevel, + "--data-dir": container, + "--objstore.config": string(bktConfigBytes), + "--http-address": ":80", + "--selector.relabel-config": string(relabelConfigBytes), + "--wait": "", }), extArgs...)...), e2e.NewReadinessProbe(80, "/-/ready", 200), 80, From ea102f51b1974dd7cada6dadfc1f0b56fb953fcf Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Wed, 18 Mar 2020 17:25:09 +0100 Subject: [PATCH 10/14] Improve and stabilize e2e tests Signed-off-by: Kemal Akkoyun --- CHANGELOG.md | 5 +- Makefile | 6 ++- pkg/block/fetcher.go | 34 ++++++++++-- test/e2e/compact_test.go | 96 ++++++++++++++++++++-------------- test/e2e/e2ethanos/services.go | 74 +++++++++++++++++++------- test/e2e/query_test.go | 2 - test/e2e/receive_test.go | 6 --- test/e2e/rule_test.go | 6 +-- test/e2e/store_gateway_test.go | 2 - 9 files changed, 150 insertions(+), 81 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index abcf91055b..0db48e3a20 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -24,9 +24,8 @@ We use *breaking* word for marking changes that are not backward compatible (rel ### Changed -- [#2136](https://github.com/thanos-io/thanos/pull/2136) store, compact, bucket: schedule block deletion by adding deletion-mark.json. This adds a consistent way for multiple readers and writers to access object storage. -Since there are no consistency guarantees provided by some Object Storage providers, this PR adds a consistent lock-free way of dealing with Object Storage irrespective of the choice of object storage. In order to achieve this co-ordination, blocks are not deleted directly. Instead, blocks are marked for deletion by uploading `deletion-mark.json` file for the block that was chosen to be deleted. This file contains unix time of when the block was marked for deletion. - +- [#2136](https://github.com/thanos-io/thanos/pull/2136) *breaking* store, compact, bucket: schedule block deletion by adding deletion-mark.json. This adds a consistent way for multiple readers and writers to access object storage. +Since there are no consistency guarantees provided by some Object Storage providers, this PR adds a consistent lock-free way of dealing with Object Storage irrespective of the choice of object storage. In order to achieve this co-ordination, blocks are not deleted directly. Instead, blocks are marked for deletion by uploading `deletion-mark.json` file for the block that was chosen to be deleted. This file contains unix time of when the block was marked for deletion. If you want to keep existing behavior, you should add `--delete-delay=0s` as a flag. - [#2090](https://github.com/thanos-io/thanos/issues/2090) *breaking* Downsample command: the `downsample` command has moved as the `thanos bucket` sub-command, and cannot be called via `thanos downsample` any more. ## [v0.11.0](https://github.com/thanos-io/thanos/releases/tag/v0.11.0) - 2020.03.02 diff --git a/Makefile b/Makefile index ca5193f454..b274cf55da 100644 --- a/Makefile +++ b/Makefile @@ -245,8 +245,12 @@ test-local: .PHONY: test-e2e test-e2e: ## Runs all Thanos e2e docker-based e2e tests from test/e2e. Required access to docker daemon. test-e2e: docker + @echo ">> cleaning docker environment." + @docker system prune -f --volumes + @echo ">> cleaning e2e test garbage." + @rm -rf ./test/e2e/e2e_integration_test* @echo ">> running /test/e2e tests." - @go test -v ./test/e2e/... + @go test -failfast -parallel 1 -timeout 5m -v ./test/e2e/... .PHONY: install-deps install-deps: ## Installs dependencies for integration tests. It installs supported versions of Prometheus and alertmanager to test against in integration tests. diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index 4f88b0dba0..4b95702ca4 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -41,6 +41,34 @@ type syncMetrics struct { modified *extprom.TxGaugeVec } +func (s *syncMetrics) submit() { + if s == nil { + return + } + + if s.synced != nil { + s.synced.Submit() + } + + if s.modified != nil { + s.modified.Submit() + } +} + +func (s *syncMetrics) resetTx() { + if s == nil { + return + } + + if s.synced != nil { + s.synced.ResetTx() + } + + if s.modified != nil { + s.modified.ResetTx() + } +} + const ( syncMetricSubSys = "blocks_meta" @@ -260,8 +288,7 @@ func (s *MetaFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata. metaErrs tsdberrors.MultiError ) - s.metrics.synced.ResetTx() - s.metrics.modified.ResetTx() + s.metrics.resetTx() for i := 0; i < s.concurrency; i++ { wg.Add(1) @@ -364,8 +391,7 @@ func (s *MetaFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata. } s.metrics.synced.WithLabelValues(loadedMeta).Set(float64(len(metas))) - s.metrics.synced.Submit() - s.metrics.modified.Submit() + s.metrics.submit() if incompleteView { return metas, partial, errors.Wrap(metaErrs, "incomplete view") diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index 720328cacc..f45688d728 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -30,7 +30,6 @@ import ( ) func TestCompact(t *testing.T) { - t.Parallel() l := log.NewLogfmtLogger(os.Stdout) // blockDesc describes a recipe to generate blocks from the given series and external labels. @@ -109,25 +108,25 @@ func TestCompact(t *testing.T) { expectOfChunks: 2, }, { - name: "(full) vertically overlapping blocks with replica labels, downsampling disabled", + name: "(full) vertically overlapping blocks with replica labels downsampling disabled", blocks: []blockDesc{ { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + extLset: labels.FromStrings("ext1", "value1", "ext2", "value2", "replica", "1"), mint: timestamp.FromTime(now), maxt: timestamp.FromTime(now.Add(2 * time.Hour)), samplesPerSeries: 120, }, { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + extLset: labels.FromStrings("ext2", "value2", "ext1", "value1", "replica", "2"), mint: timestamp.FromTime(now), maxt: timestamp.FromTime(now.Add(2 * time.Hour)), samplesPerSeries: 120, }, { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLset: labels.FromStrings("ext1", "value1", "rule_replica", "1"), + extLset: labels.FromStrings("ext1", "value1", "rule_replica", "1", "ext2", "value2"), mint: timestamp.FromTime(now), maxt: timestamp.FromTime(now.Add(2 * time.Hour)), samplesPerSeries: 120, @@ -142,6 +141,7 @@ func TestCompact(t *testing.T) { "a": "1", "b": "2", "ext1": "value1", + "ext2": "value2", }, }, expectOfModBlocks: 3, @@ -151,25 +151,46 @@ func TestCompact(t *testing.T) { expectOfChunks: 2, }, { - name: "(partial) vertically overlapping blocks with replica labels", + name: "(full) vertically overlapping blocks with replica labels, downsampling disabled and extra blocks", blocks: []blockDesc{ { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + extLset: labels.FromStrings("ext1", "value1", "ext2", "value2", "replica", "1"), mint: timestamp.FromTime(now), maxt: timestamp.FromTime(now.Add(2 * time.Hour)), samplesPerSeries: 120, }, { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + extLset: labels.FromStrings("ext2", "value2", "ext1", "value1", "replica", "2"), mint: timestamp.FromTime(now), - maxt: timestamp.FromTime(now.Add(1 * time.Hour)), - samplesPerSeries: 60, + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "rule_replica", "1", "ext2", "value2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("c", "1", "d", "2")}, + extLset: labels.FromStrings("ext1", "value1", "ext2", "value2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, + }, + { + series: []labels.Labels{labels.FromStrings("c", "1", "d", "2")}, + extLset: labels.FromStrings("ext3", "value3"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 120, }, }, - replicaLabels: []string{"replica"}, - downsamplingEnabled: true, + replicaLabels: []string{"replica", "rule_replica"}, + downsamplingEnabled: false, query: "{a=\"1\"}", expected: []model.Metric{ @@ -177,30 +198,31 @@ func TestCompact(t *testing.T) { "a": "1", "b": "2", "ext1": "value1", + "ext2": "value2", }, }, - expectOfModBlocks: 2, - expectOfBlocks: 1, - expectOfSamples: 179, // TODO(kakkoyun): ? - expectOfSeries: 1, - expectOfChunks: 2, + expectOfModBlocks: 3, + expectOfBlocks: 2, + expectOfSamples: 360, + expectOfSeries: 3, + expectOfChunks: 6, }, { - name: "(contains) vertically overlapping blocks with replica labels", + name: "(partial) vertically overlapping blocks with replica labels", blocks: []blockDesc{ { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLset: labels.FromStrings("ext1", "value1", "replica", "1"), - mint: timestamp.FromTime(now.Add(30 * time.Minute)), - maxt: timestamp.FromTime(now.Add(1 * time.Hour)), - samplesPerSeries: 90, + extLset: labels.FromStrings("ext1", "value1", "ext2", "value2", "replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 119, }, { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, - extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + extLset: labels.FromStrings("ext2", "value2", "ext1", "value1", "replica", "2"), mint: timestamp.FromTime(now), - maxt: timestamp.FromTime(now.Add(2 * time.Hour)), - samplesPerSeries: 120, + maxt: timestamp.FromTime(now.Add(1 * time.Hour)), + samplesPerSeries: 59, }, }, replicaLabels: []string{"replica"}, @@ -212,11 +234,12 @@ func TestCompact(t *testing.T) { "a": "1", "b": "2", "ext1": "value1", + "ext2": "value2", }, }, expectOfModBlocks: 2, expectOfBlocks: 1, - expectOfSamples: 210, // TODO(kakkoyun): ? + expectOfSamples: 119, expectOfSeries: 1, expectOfChunks: 2, }, @@ -228,14 +251,14 @@ func TestCompact(t *testing.T) { extLset: labels.FromStrings("ext1", "value1", "replica", "1"), mint: timestamp.FromTime(now.Add(30 * time.Minute)), maxt: timestamp.FromTime(now.Add(150 * time.Minute)), - samplesPerSeries: 120, + samplesPerSeries: 119, }, { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, extLset: labels.FromStrings("ext1", "value1", "replica", "2"), - mint: timestamp.FromTime(now.Add(-30 * time.Minute)), - maxt: timestamp.FromTime(now.Add(90 * time.Minute)), - samplesPerSeries: 120, + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(120 * time.Minute)), + samplesPerSeries: 119, }, }, replicaLabels: []string{"replica"}, @@ -251,12 +274,12 @@ func TestCompact(t *testing.T) { }, expectOfModBlocks: 2, expectOfBlocks: 1, - expectOfSamples: 240, // TODO(kakkoyun): ? + expectOfSamples: 149, expectOfSeries: 1, expectOfChunks: 2, }, { - name: "(full) vertically overlapping blocks with replica labels, retention specified", + name: "(full) vertically overlapping blocks with replica labels retention specified", blocks: []blockDesc{ { series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, @@ -345,11 +368,9 @@ func TestCompact(t *testing.T) { i := i tcase := tcase t.Run(tcase.name, func(t *testing.T) { - t.Parallel() - s, err := e2e.NewScenario("e2e_test_compact_" + strconv.Itoa(i)) testutil.Ok(t, err) - defer s.Close() + defer s.Close() // TODO(kakkoyun): Change with t.CleanUp after go 1.14 update. dir := filepath.Join(s.SharedDir(), "tmp_"+strconv.Itoa(i)) testutil.Ok(t, os.MkdirAll(filepath.Join(s.SharedDir(), dir), os.ModePerm)) @@ -368,8 +389,8 @@ func TestCompact(t *testing.T) { }, "test-feed") testutil.Ok(t, err) - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) - defer cancel() + ctx, cancel := context.WithTimeout(context.Background(), 90*time.Second) + defer cancel() // TODO(kakkoyun): Change with t.CleanUp after go 1.14 update. var rawBlockIds []ulid.ULID for _, b := range tcase.blocks { @@ -405,7 +426,6 @@ func TestCompact(t *testing.T) { tcase.downsamplingEnabled, append(dedupFlags, retenFlags...)..., ) - testutil.Ok(t, err) testutil.Ok(t, s.StartAndWaitReady(cmpt)) testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(float64(len(rawBlockIds))), "thanos_blocks_meta_synced")) diff --git a/test/e2e/e2ethanos/services.go b/test/e2e/e2ethanos/services.go index 201b2f959b..6bd9a9fed1 100644 --- a/test/e2e/e2ethanos/services.go +++ b/test/e2e/e2ethanos/services.go @@ -10,8 +10,10 @@ import ( "os" "path/filepath" "strconv" + "time" "github.com/cortexproject/cortex/integration/e2e" + "github.com/cortexproject/cortex/pkg/util" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/discovery/targetgroup" @@ -25,6 +27,11 @@ import ( const logLevel = "info" +var defaultBackoffConfig = util.BackoffConfig{ + MinBackoff: 300 * time.Millisecond, + MaxBackoff: 30 * time.Second, +} + // TODO(bwplotka): Run against multiple? func DefaultPrometheusImage() string { return "quay.io/prometheus/prometheus:v2.16.0" @@ -69,6 +76,7 @@ func NewPrometheus(sharedDir string, name string, config, promImage string) (*e2 9090, ) prom.SetUser("root") + prom.SetBackoff(defaultBackoffConfig) return prom, container, nil } @@ -78,6 +86,7 @@ func NewPrometheusWithSidecar(sharedDir string, netName string, name string, con if err != nil { return nil, nil, err } + prom.SetBackoff(defaultBackoffConfig) sidecar := NewService( fmt.Sprintf("sidecar-%s", name), @@ -95,6 +104,8 @@ func NewPrometheusWithSidecar(sharedDir string, netName string, name string, con 80, 9091, ) + sidecar.SetBackoff(defaultBackoffConfig) + return prom, sidecar, nil } @@ -109,6 +120,7 @@ func NewQuerier(sharedDir string, name string, storeAddresses []string, fileSDSt "--query.replica-label": replicaLabel, "--store.sd-dns-interval": "5s", "--log.level": logLevel, + "--query.max-concurrent": "1", "--store.sd-interval": "5s", }) for _, addr := range storeAddresses { @@ -139,14 +151,17 @@ func NewQuerier(sharedDir string, name string, storeAddresses []string, fileSDSt args = append(args, "--store.sd-files="+filepath.Join(container, "filesd.yaml")) } - return NewService( + querier := NewService( fmt.Sprintf("querier-%v", name), DefaultImage(), e2e.NewCommand("query", args...), e2e.NewReadinessProbe(80, "/-/ready", 200), 80, 9091, - ), nil + ) + querier.SetBackoff(defaultBackoffConfig) + + return querier, nil } func RemoteWriteEndpoint(addr string) string { return fmt.Sprintf("http://%s/api/v1/receive", addr) } @@ -171,7 +186,7 @@ func NewReceiver(sharedDir string, networkName string, name string, replicationF return nil, errors.Wrap(err, "creating receive config") } - return NewService( + receiver := NewService( fmt.Sprintf("receive-%v", name), DefaultImage(), // TODO(bwplotka): BuildArgs should be interface. @@ -193,7 +208,10 @@ func NewReceiver(sharedDir string, networkName string, name string, replicationF 80, 9091, 81, - ), nil + ) + receiver.SetBackoff(defaultBackoffConfig) + + return receiver, nil } func NewRuler(sharedDir string, name string, ruleSubDir string, amCfg []alert.AlertmanagerConfig, queryCfg []query.Config) (*Service, error) { @@ -215,7 +233,7 @@ func NewRuler(sharedDir string, name string, ruleSubDir string, amCfg []alert.Al return nil, errors.Wrapf(err, "generate query file: %v", queryCfg) } - return NewService( + ruler := NewService( fmt.Sprintf("rule-%v", name), DefaultImage(), e2e.NewCommand("rule", e2e.BuildArgs(map[string]string{ @@ -226,7 +244,7 @@ func NewRuler(sharedDir string, name string, ruleSubDir string, amCfg []alert.Al "--label": fmt.Sprintf(`replica="%s"`, name), "--data-dir": container, "--rule-file": filepath.Join(e2e.ContainerSharedDir, ruleSubDir, "*.yaml"), - "--eval-interval": "1s", + "--eval-interval": "3s", "--alertmanagers.config": string(amCfgBytes), "--alertmanagers.sd-dns-interval": "1s", "--log.level": logLevel, @@ -237,7 +255,10 @@ func NewRuler(sharedDir string, name string, ruleSubDir string, amCfg []alert.Al e2e.NewReadinessProbe(80, "/-/ready", 200), 80, 9091, - ), nil + ) + ruler.SetBackoff(defaultBackoffConfig) + + return ruler, nil } func NewAlertmanager(sharedDir string, name string) (*e2e.HTTPService, error) { @@ -263,15 +284,19 @@ receivers: fmt.Sprintf("alertmanager-%v", name), DefaultAlertmanagerImage(), e2e.NewCommandWithoutEntrypoint("/bin/alertmanager", e2e.BuildArgs(map[string]string{ - "--config.file": filepath.Join(container, "config.yaml"), - "--web.listen-address": "0.0.0.0:80", - "--log.level": logLevel, - "--storage.path": container, + "--config.file": filepath.Join(container, "config.yaml"), + "--web.listen-address": "0.0.0.0:80", + "--log.level": logLevel, + "--storage.path": container, + "--web.get-concurrency": "1", + "--web.timeout": "2m", })...), e2e.NewReadinessProbe(80, "/-/ready", 200), 80, ) s.SetUser("root") + s.SetBackoff(defaultBackoffConfig) + return s, nil } @@ -292,7 +317,7 @@ func NewStoreGW(sharedDir string, name string, bucketConfig client.BucketConfig, return nil, errors.Wrapf(err, "generate store relabel file: %v", relabelConfig) } - return NewService( + store := NewService( fmt.Sprintf("store-gw-%v", name), DefaultImage(), e2e.NewCommand("store", append(e2e.BuildArgs(map[string]string{ @@ -304,17 +329,22 @@ func NewStoreGW(sharedDir string, name string, bucketConfig client.BucketConfig, "--data-dir": container, "--objstore.config": string(bktConfigBytes), // Accelerated sync time for quicker test (3m by default). - "--sync-block-duration": "1s", - "--selector.relabel-config": string(relabelConfigBytes), - "--consistency-delay": "30m", + "--sync-block-duration": "3s", + "--block-sync-concurrency": "1", + "--store.grpc.series-max-concurrency": "1", + "--selector.relabel-config": string(relabelConfigBytes), + "--consistency-delay": "30m", }), "--experimental.enable-index-header")...), e2e.NewReadinessProbe(80, "/-/ready", 200), 80, 9091, - ), nil + ) + store.SetBackoff(defaultBackoffConfig) + + return store, nil } -func NewCompactor(sharedDir string, name string, bucketConfig client.BucketConfig, relabelConfig []relabel.Config, downsamplingEnabled bool, extArgs ...string) (*Service, error) { +func NewCompactor(sharedDir string, name string, bucketConfig client.BucketConfig, relabelConfig []relabel.Config, downsamplingEnabled bool, extArgs ...string) (*e2e.HTTPService, error) { dir := filepath.Join(sharedDir, "data", "compact", name) container := filepath.Join(e2e.ContainerSharedDir, "data", "compact", name) @@ -336,7 +366,7 @@ func NewCompactor(sharedDir string, name string, bucketConfig client.BucketConfi extArgs = append(extArgs, "--downsampling.disable") } - return NewService( + compactor := e2e.NewHTTPService( fmt.Sprintf("compact-%s", name), DefaultImage(), e2e.NewCommand("compact", append(e2e.BuildArgs(map[string]string{ @@ -345,11 +375,15 @@ func NewCompactor(sharedDir string, name string, bucketConfig client.BucketConfi "--data-dir": container, "--objstore.config": string(bktConfigBytes), "--http-address": ":80", + "--delete-delay": "0s", + "--block-sync-concurrency": "1", "--selector.relabel-config": string(relabelConfigBytes), "--wait": "", }), extArgs...)...), e2e.NewReadinessProbe(80, "/-/ready", 200), 80, - 9091, - ), nil + ) + compactor.SetBackoff(defaultBackoffConfig) + + return compactor, nil } diff --git a/test/e2e/query_test.go b/test/e2e/query_test.go index 914d1c6a92..5e11841bc6 100644 --- a/test/e2e/query_test.go +++ b/test/e2e/query_test.go @@ -63,8 +63,6 @@ func sortResults(res model.Vector) { } func TestQuery(t *testing.T) { - t.Parallel() - s, err := e2e.NewScenario("e2e_test_query") testutil.Ok(t, err) defer s.Close() diff --git a/test/e2e/receive_test.go b/test/e2e/receive_test.go index fda1f59591..b772e5eef4 100644 --- a/test/e2e/receive_test.go +++ b/test/e2e/receive_test.go @@ -17,11 +17,7 @@ import ( ) func TestReceive(t *testing.T) { - t.Parallel() - t.Run("hashring", func(t *testing.T) { - t.Parallel() - s, err := e2e.NewScenario("e2e_test_receive_hashring") testutil.Ok(t, err) defer s.Close() @@ -98,7 +94,6 @@ func TestReceive(t *testing.T) { }) t.Run("replication", func(t *testing.T) { - t.Parallel() s, err := e2e.NewScenario("e2e_test_receive_replication") testutil.Ok(t, err) @@ -170,7 +165,6 @@ func TestReceive(t *testing.T) { }) t.Run("replication_with_outage", func(t *testing.T) { - t.Parallel() s, err := e2e.NewScenario("e2e_test_receive_replication_with_outage") testutil.Ok(t, err) diff --git a/test/e2e/rule_test.go b/test/e2e/rule_test.go index 5e41c5f941..62f773efa7 100644 --- a/test/e2e/rule_test.go +++ b/test/e2e/rule_test.go @@ -179,8 +179,6 @@ func (m *mockAlertmanager) ServeHTTP(resp http.ResponseWriter, req *http.Request func TestRule_AlertmanagerHTTPClient(t *testing.T) { t.Skip("TODO: Allow HTTP ports from binaries running on host to be accessible.") - t.Parallel() - s, err := e2e.NewScenario("e2e_test_rule_am_http_client") testutil.Ok(t, err) defer s.Close() @@ -265,8 +263,6 @@ func TestRule_AlertmanagerHTTPClient(t *testing.T) { } func TestRule(t *testing.T) { - t.Parallel() - s, err := e2e.NewScenario("e2e_test_rule") testutil.Ok(t, err) defer s.Close() @@ -435,7 +431,7 @@ func TestRule(t *testing.T) { testutil.Ok(t, r.WaitSumMetrics(e2e.Equals(1), "thanos_ruler_alertmanagers_dns_provider_results")) }) - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) + ctx, cancel := context.WithTimeout(context.Background(), 3*time.Minute) defer cancel() queryAndAssert(t, ctx, q.HTTPEndpoint(), "ALERTS", promclient.QueryOptions{ diff --git a/test/e2e/store_gateway_test.go b/test/e2e/store_gateway_test.go index d663c71e71..0abe8fdbbe 100644 --- a/test/e2e/store_gateway_test.go +++ b/test/e2e/store_gateway_test.go @@ -31,8 +31,6 @@ import ( // TODO(bwplotka): Extend this test to have multiple stores and memcached. // TODO(bwplotka): Extend this test for downsampling. func TestStoreGateway(t *testing.T) { - t.Parallel() - s, err := e2e.NewScenario("e2e_test_store_gateway") testutil.Ok(t, err) defer s.Close() From a2dca302d1a81fddda119647c320ccc8d4a3d84e Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Thu, 19 Mar 2020 18:16:05 +0100 Subject: [PATCH 11/14] Address review issues Signed-off-by: Kemal Akkoyun --- .github/workflows/e2e.yaml | 2 +- CHANGELOG.md | 2 +- Makefile | 12 ++- cmd/thanos/compact.go | 6 +- pkg/block/fetcher.go | 81 ++++++--------- pkg/block/fetcher_test.go | 22 ++--- test/e2e/compact_test.go | 174 ++++++++++++++++++++++----------- test/e2e/e2ethanos/services.go | 2 +- test/e2e/query_test.go | 2 + test/e2e/receive_test.go | 6 ++ test/e2e/rule_test.go | 2 + test/e2e/store_gateway_test.go | 2 + 12 files changed, 190 insertions(+), 123 deletions(-) diff --git a/.github/workflows/e2e.yaml b/.github/workflows/e2e.yaml index 1cc20d0dc2..d3af411328 100644 --- a/.github/workflows/e2e.yaml +++ b/.github/workflows/e2e.yaml @@ -25,4 +25,4 @@ jobs: key: ${{ runner.os }}-go-${{ hashFiles('**/go.sum') }} - name: Run e2e docker-based tests. - run: make test-e2e + run: make test-e2e-ci diff --git a/CHANGELOG.md b/CHANGELOG.md index 0db48e3a20..2753e6c3fb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,7 +20,7 @@ We use *breaking* word for marking changes that are not backward compatible (rel ### Added - [#2265](https://github.com/thanos-io/thanos/pull/2265) Compactor: Add `--wait-interval` to specify compaction wait interval between consecutive compact runs when `--wait` enabled. -- [#2250](https://github.com/thanos-io/thanos/pull/2250) Compactor: Enable vertical compaction for offline deduplication (Experimental). Uses `--deduplication.replica-label` flag to specify the replica label to deduplicate on (Hidden). +- [#2250](https://github.com/thanos-io/thanos/pull/2250) Compactor: Enable vertical compaction for offline deduplication (Experimental). Uses `--deduplication.replica-label` flag to specify the replica label to deduplicate on (Hidden). Please note that this uses a NAIVE algorithm for merging (no smart replica deduplication, just chaining samples together). This works well for deduplication of blocks with **precisely the same samples** like produced by Receiver replication. We plan to add a smarter algorithm in the following weeks. ### Changed diff --git a/Makefile b/Makefile index b274cf55da..91ccdbf1d9 100644 --- a/Makefile +++ b/Makefile @@ -244,7 +244,17 @@ test-local: .PHONY: test-e2e test-e2e: ## Runs all Thanos e2e docker-based e2e tests from test/e2e. Required access to docker daemon. -test-e2e: docker +test-e2e: docker-multi-stage + @echo ">> cleaning docker environment." + @docker system prune -f --volumes + @echo ">> cleaning e2e test garbage." + @rm -rf ./test/e2e/e2e_integration_test* + @echo ">> running /test/e2e tests." + @go test -failfast -timeout 5m -v ./test/e2e/... + +.PHONY: test-e2e-ci +test-e2e-ci: ## Runs all Thanos e2e docker-based e2e tests from test/e2e, using limited resources. Required access to docker daemon. +test-e2e-ci: docker @echo ">> cleaning docker environment." @docker system prune -f --volumes @echo ">> cleaning e2e test garbage." diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index 97ed17b930..6a066f3b0b 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -134,7 +134,9 @@ func registerCompact(m map[string]setupFunc, app *kingpin.Application) { Default("48h")) dedupReplicaLabels := cmd.Flag("deduplication.replica-label", "Label to treat as a replica indicator of blocks that can be deduplicated (repeated flag). This will merge multiple replica blocks into one. This process is irreversible."+ - "Experimental. When it is set true, this will given labels from blocks so that vertical compaction could merge blocks."). + "Experimental. When it is set true, this will given labels from blocks so that vertical compaction could merge blocks."+ + "Please note that this uses a NAIVE algorithm for merging (no smart replica deduplication, just chaining samples together)."+ + "This works well for deduplication of blocks with **precisely the same samples** like produced by Receiver replication."). Hidden().Strings() selectorRelabelConf := regSelectorRelabelFlags(cmd) @@ -299,7 +301,7 @@ func runCompact( enableVerticalCompaction := false if len(dedupReplicaLabels) > 0 { enableVerticalCompaction = true - level.Info(logger).Log("msg", "deduplication.replica-label specified, vertical compaction is enabled", "dedup-replica-labels", strings.Join(dedupReplicaLabels, ",")) + level.Info(logger).Log("msg", "deduplication.replica-label specified, vertical compaction is enabled", "dedupReplicaLabels", strings.Join(dedupReplicaLabels, ",")) } sy, err := compact.NewSyncer(logger, reg, bkt, metaFetcher, duplicateBlocksFilter, ignoreDeletionMarkFilter, blocksMarkedForDeletion, blockSyncConcurrency, acceptMalformedIndex, enableVerticalCompaction) diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index 4b95702ca4..3b531164c7 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -32,7 +32,7 @@ import ( "github.com/thanos-io/thanos/pkg/runutil" ) -type syncMetrics struct { +type fetcherMetrics struct { syncs prometheus.Counter syncFailures prometheus.Counter syncDuration prometheus.Histogram @@ -41,75 +41,58 @@ type syncMetrics struct { modified *extprom.TxGaugeVec } -func (s *syncMetrics) submit() { - if s == nil { - return - } - - if s.synced != nil { - s.synced.Submit() - } - - if s.modified != nil { - s.modified.Submit() - } +func (s *fetcherMetrics) submit() { + s.synced.Submit() + s.modified.Submit() } -func (s *syncMetrics) resetTx() { - if s == nil { - return - } - - if s.synced != nil { - s.synced.ResetTx() - } - - if s.modified != nil { - s.modified.ResetTx() - } +func (s *fetcherMetrics) resetTx() { + s.synced.ResetTx() + s.modified.ResetTx() } const ( - syncMetricSubSys = "blocks_meta" + fetcherSubSys = "blocks_meta" corruptedMeta = "corrupted-meta-json" noMeta = "no-meta-json" loadedMeta = "loaded" failedMeta = "failed" - // Filter's label values. - labelExcludedMeta = "label-excluded" - timeExcludedMeta = "time-excluded" - tooFreshMeta = "too-fresh" - duplicateMeta = "duplicate" - replicaRemovedMeta = "replica-label-removed" - + // Fetcher's synced label values. + labelExcludedMeta = "label-excluded" + timeExcludedMeta = "time-excluded" + tooFreshMeta = "too-fresh" + duplicateMeta = "duplicate" // Blocks that are marked for deletion can be loaded as well. This is done to make sure that we load blocks that are meant to be deleted, // but don't have a replacement block yet. markedForDeletionMeta = "marked-for-deletion" + + // Fetcher's synced label values. + replicaRemovedMeta = "replica-label-removed" ) -func newSyncMetrics(reg prometheus.Registerer) *syncMetrics { - var m syncMetrics +func newFetcherMetrics(reg prometheus.Registerer) *fetcherMetrics { + var m fetcherMetrics m.syncs = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Subsystem: syncMetricSubSys, + Subsystem: fetcherSubSys, Name: "syncs_total", Help: "Total blocks metadata synchronization attempts", }) m.syncFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Subsystem: syncMetricSubSys, + Subsystem: fetcherSubSys, Name: "sync_failures_total", Help: "Total blocks metadata synchronization failures", }) m.syncDuration = promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ - Subsystem: syncMetricSubSys, + Subsystem: fetcherSubSys, Name: "sync_duration_seconds", Help: "Duration of the blocks metadata synchronization in seconds", Buckets: []float64{0.01, 1, 10, 100, 1000}, }) m.synced = extprom.NewTxGaugeVec(reg, prometheus.GaugeOpts{ - Subsystem: syncMetricSubSys, + Subsystem: fetcherSubSys, Name: "synced", Help: "Number of block metadata synced", }, @@ -125,7 +108,7 @@ func newSyncMetrics(reg prometheus.Registerer) *syncMetrics { []string{markedForDeletionMeta}, ) m.modified = extprom.NewTxGaugeVec(reg, prometheus.GaugeOpts{ - Subsystem: syncMetricSubSys, + Subsystem: fetcherSubSys, Name: "modified", Help: "Number of block metadata that modified", }, @@ -139,7 +122,7 @@ type MetadataFetcher interface { Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata.Meta, partial map[ulid.ULID]error, err error) } -type MetaFetcherFilter func(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, incompleteView bool) error +type MetaFetcherFilter func(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, incompleteView bool) error // MetaFetcher is a struct that synchronizes filtered metadata of all block in the object storage with the local state. // Not go-routine safe. @@ -150,7 +133,7 @@ type MetaFetcher struct { // Optional local directory to cache meta.json files. cacheDir string - metrics *syncMetrics + metrics *fetcherMetrics filters []MetaFetcherFilter @@ -176,7 +159,7 @@ func NewMetaFetcher(logger log.Logger, concurrency int, bkt objstore.BucketReade concurrency: concurrency, bkt: bkt, cacheDir: cacheDir, - metrics: newSyncMetrics(r), + metrics: newFetcherMetrics(r), filters: filters, cached: map[ulid.ULID]*metadata.Meta{}, }, nil @@ -415,7 +398,7 @@ func NewTimePartitionMetaFilter(MinTime, MaxTime model.TimeOrDurationValue) *Tim } // Filter filters out blocks that are outside of specified time range. -func (f *TimePartitionMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, _ bool) error { +func (f *TimePartitionMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, _ bool) error { for id, m := range metas { if m.MaxTime >= f.minTime.PrometheusTimestamp() && m.MinTime <= f.maxTime.PrometheusTimestamp() { continue @@ -443,7 +426,7 @@ func NewLabelShardedMetaFilter(relabelConfig []*relabel.Config) *LabelShardedMet const blockIDLabel = "__block_id" // Filter filters out blocks that have no labels after relabelling of each block external (Thanos) labels. -func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, _ bool) error { +func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, _ bool) error { var lbls labels.Labels for id, m := range metas { lbls = lbls[:0] @@ -473,7 +456,7 @@ func NewDeduplicateFilter() *DeduplicateFilter { // Filter filters out duplicate blocks that can be formed // from two or more overlapping blocks that fully submatches the source blocks of the older blocks. -func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, _ bool) error { +func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, _ bool) error { var wg sync.WaitGroup metasByResolution := make(map[int64][]*metadata.Meta) @@ -587,7 +570,7 @@ func NewReplicaLabelRemover(logger log.Logger, replicaLabels []string) *ReplicaL } // Modify modifies external labels of existing blocks, it removes given replica labels from the metadata of blocks that have it. -func (r *ReplicaLabelRemover) Modify(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, view bool) error { +func (r *ReplicaLabelRemover) Modify(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, view bool) error { for u, meta := range metas { labels := meta.Thanos.Labels for _, replicaLabel := range r.replicaLabels { @@ -628,7 +611,7 @@ func NewConsistencyDelayMetaFilter(logger log.Logger, consistencyDelay time.Dura } // Filter filters out blocks that filters blocks that have are created before a specified consistency delay. -func (f *ConsistencyDelayMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, _ bool) error { +func (f *ConsistencyDelayMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, _ bool) error { for id, meta := range metas { // TODO(khyatisoneji): Remove the checks about Thanos Source // by implementing delete delay to fetch metas. @@ -674,7 +657,7 @@ func (f *IgnoreDeletionMarkFilter) DeletionMarkBlocks() map[ulid.ULID]*metadata. // Filter filters out blocks that are marked for deletion after a given delay. // It also returns the blocks that can be deleted since they were uploaded delay duration before current time. -func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, m *syncMetrics, _ bool) error { +func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, m *fetcherMetrics, _ bool) error { f.deletionMarkMap = make(map[ulid.ULID]*metadata.DeletionMark) for id := range metas { diff --git a/pkg/block/fetcher_test.go b/pkg/block/fetcher_test.go index 12cdd3a45b..b0060f6636 100644 --- a/pkg/block/fetcher_test.go +++ b/pkg/block/fetcher_test.go @@ -34,8 +34,8 @@ import ( "gopkg.in/yaml.v2" ) -func newTestSyncMetrics() *syncMetrics { - return &syncMetrics{ +func newTestFetcherMetrics() *fetcherMetrics { + return &fetcherMetrics{ synced: extprom.NewTxGaugeVec(nil, prometheus.GaugeOpts{}, []string{"state"}), modified: extprom.NewTxGaugeVec(nil, prometheus.GaugeOpts{}, []string{"modified"}), } @@ -63,7 +63,7 @@ func TestMetaFetcher_Fetch(t *testing.T) { var ulidToDelete ulid.ULID r := prometheus.NewRegistry() - f, err := NewMetaFetcher(log.NewNopLogger(), 20, bkt, dir, r, func(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *syncMetrics, incompleteView bool) error { + f, err := NewMetaFetcher(log.NewNopLogger(), 20, bkt, dir, r, func(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, incompleteView bool) error { if _, ok := metas[ulidToDelete]; ok { metrics.synced.WithLabelValues("filtered").Inc() delete(metas, ulidToDelete) @@ -343,7 +343,7 @@ func TestLabelShardedMetaFilter_Filter_Basic(t *testing.T) { ULID(6): input[ULID(6)], } - m := newTestSyncMetrics() + m := newTestFetcherMetrics() testutil.Ok(t, f.Filter(ctx, input, m, false)) testutil.Equals(t, 3.0, promtest.ToFloat64(m.synced.WithLabelValues(labelExcludedMeta))) @@ -441,7 +441,7 @@ func TestLabelShardedMetaFilter_Filter_Hashmod(t *testing.T) { } deleted := len(input) - len(expected) - m := newTestSyncMetrics() + m := newTestFetcherMetrics() testutil.Ok(t, f.Filter(ctx, input, m, false)) testutil.Equals(t, expected, input) @@ -505,7 +505,7 @@ func TestTimePartitionMetaFilter_Filter(t *testing.T) { ULID(4): input[ULID(4)], } - m := newTestSyncMetrics() + m := newTestFetcherMetrics() testutil.Ok(t, f.Filter(ctx, input, m, false)) testutil.Equals(t, 2.0, promtest.ToFloat64(m.synced.WithLabelValues(timeExcludedMeta))) @@ -839,7 +839,7 @@ func TestDeduplicateFilter_Filter(t *testing.T) { } { f := NewDeduplicateFilter() if ok := t.Run(tcase.name, func(t *testing.T) { - m := newTestSyncMetrics() + m := newTestFetcherMetrics() metas := make(map[ulid.ULID]*metadata.Meta) inputLen := len(tcase.input) for id, metaInfo := range tcase.input { @@ -908,7 +908,7 @@ func TestReplicaLabelRemover_Modify(t *testing.T) { modified: 5.0, }, } { - m := newTestSyncMetrics() + m := newTestFetcherMetrics() testutil.Ok(t, rm.Modify(ctx, tcase.input, m, false)) testutil.Equals(t, tcase.modified, promtest.ToFloat64(m.modified.WithLabelValues(replicaRemovedMeta))) @@ -1004,7 +1004,7 @@ func TestConsistencyDelayMetaFilter_Filter_0(t *testing.T) { } t.Run("consistency 0 (turned off)", func(t *testing.T) { - m := newTestSyncMetrics() + m := newTestFetcherMetrics() expected := map[ulid.ULID]*metadata.Meta{} // Copy all. for _, id := range u.created { @@ -1021,7 +1021,7 @@ func TestConsistencyDelayMetaFilter_Filter_0(t *testing.T) { }) t.Run("consistency 30m.", func(t *testing.T) { - m := newTestSyncMetrics() + m := newTestFetcherMetrics() expected := map[ulid.ULID]*metadata.Meta{} // Only certain sources and those with 30m or more age go through. for i, id := range u.created { @@ -1092,7 +1092,7 @@ func TestIgnoreDeletionMarkFilter_Filter(t *testing.T) { ULID(4): {}, } - m := newTestSyncMetrics() + m := newTestFetcherMetrics() testutil.Ok(t, f.Filter(ctx, input, m, false)) testutil.Equals(t, 1.0, promtest.ToFloat64(m.synced.WithLabelValues(markedForDeletionMeta))) testutil.Equals(t, expected, input) diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index f45688d728..47b76be5ef 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -19,6 +19,7 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/timestamp" + "github.com/prometheus/prometheus/tsdb" "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/objstore" "github.com/thanos-io/thanos/pkg/objstore/client" @@ -30,6 +31,8 @@ import ( ) func TestCompact(t *testing.T) { + t.Parallel() + l := log.NewLogfmtLogger(os.Stdout) // blockDesc describes a recipe to generate blocks from the given series and external labels. @@ -58,12 +61,10 @@ func TestCompact(t *testing.T) { retention *retention query string - expected []model.Metric - expectOfModBlocks float64 - expectOfBlocks uint64 - expectOfSamples uint64 - expectOfSeries uint64 - expectOfChunks uint64 + expected []model.Metric + expectNumModBlocks float64 + expectNumBlocks uint64 + expectedStats tsdb.BlockStats }{ { name: "(full) vertically overlapping blocks with replica labels", @@ -101,11 +102,57 @@ func TestCompact(t *testing.T) { "ext1": "value1", }, }, - expectOfModBlocks: 3, - expectOfBlocks: 1, - expectOfSamples: 120, - expectOfSeries: 1, - expectOfChunks: 2, + expectNumModBlocks: 3, + expectNumBlocks: 1, + expectedStats: tsdb.BlockStats{ + NumChunks: 2, + NumSeries: 1, + NumSamples: 120, + }, + }, + { + name: "(full) vertically overlapping blocks without replica labels", + blocks: []blockDesc{ + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "1"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 12, + }, + { + series: []labels.Labels{labels.FromStrings("a", "1", "b", "2")}, + extLset: labels.FromStrings("ext1", "value1", "replica", "2"), + mint: timestamp.FromTime(now), + maxt: timestamp.FromTime(now.Add(2 * time.Hour)), + samplesPerSeries: 12, + }, + }, + replicaLabels: []string{}, + downsamplingEnabled: true, + query: "{a=\"1\"}", + + expected: []model.Metric{ + { + "a": "1", + "b": "2", + "ext1": "value1", + "replica": "1", + }, + { + "a": "1", + "b": "2", + "ext1": "value1", + "replica": "2", + }, + }, + expectNumModBlocks: 0, + expectNumBlocks: 2, + expectedStats: tsdb.BlockStats{ + NumChunks: 2, + NumSeries: 2, + NumSamples: 24, + }, }, { name: "(full) vertically overlapping blocks with replica labels downsampling disabled", @@ -144,11 +191,13 @@ func TestCompact(t *testing.T) { "ext2": "value2", }, }, - expectOfModBlocks: 3, - expectOfBlocks: 1, - expectOfSamples: 120, - expectOfSeries: 1, - expectOfChunks: 2, + expectNumModBlocks: 3, + expectNumBlocks: 1, + expectedStats: tsdb.BlockStats{ + NumChunks: 2, + NumSeries: 1, + NumSamples: 120, + }, }, { name: "(full) vertically overlapping blocks with replica labels, downsampling disabled and extra blocks", @@ -201,11 +250,13 @@ func TestCompact(t *testing.T) { "ext2": "value2", }, }, - expectOfModBlocks: 3, - expectOfBlocks: 2, - expectOfSamples: 360, - expectOfSeries: 3, - expectOfChunks: 6, + expectNumModBlocks: 3, + expectNumBlocks: 2, + expectedStats: tsdb.BlockStats{ + NumChunks: 6, + NumSeries: 3, + NumSamples: 360, + }, }, { name: "(partial) vertically overlapping blocks with replica labels", @@ -237,11 +288,13 @@ func TestCompact(t *testing.T) { "ext2": "value2", }, }, - expectOfModBlocks: 2, - expectOfBlocks: 1, - expectOfSamples: 119, - expectOfSeries: 1, - expectOfChunks: 2, + expectNumModBlocks: 2, + expectNumBlocks: 1, + expectedStats: tsdb.BlockStats{ + NumChunks: 2, + NumSeries: 1, + NumSamples: 119, + }, }, { name: "(shifted) vertically overlapping blocks with replica labels", @@ -272,11 +325,13 @@ func TestCompact(t *testing.T) { "ext1": "value1", }, }, - expectOfModBlocks: 2, - expectOfBlocks: 1, - expectOfSamples: 149, - expectOfSeries: 1, - expectOfChunks: 2, + expectNumModBlocks: 2, + expectNumBlocks: 1, + expectedStats: tsdb.BlockStats{ + NumChunks: 2, + NumSeries: 1, + NumSamples: 149, + }, }, { name: "(full) vertically overlapping blocks with replica labels retention specified", @@ -319,11 +374,13 @@ func TestCompact(t *testing.T) { "ext1": "value1", }, }, - expectOfModBlocks: 3, - expectOfBlocks: 1, - expectOfSamples: 120, - expectOfSeries: 1, - expectOfChunks: 2, + expectNumModBlocks: 3, + expectNumBlocks: 1, + expectedStats: tsdb.BlockStats{ + NumChunks: 2, + NumSeries: 1, + NumSamples: 120, + }, }, { name: "(full) vertically overlapping blocks without replica labels", @@ -358,11 +415,13 @@ func TestCompact(t *testing.T) { "ext1": "value2", }, }, - expectOfModBlocks: 0, - expectOfBlocks: 2, - expectOfSamples: 4, - expectOfSeries: 2, - expectOfChunks: 2, + expectNumModBlocks: 0, + expectNumBlocks: 2, + expectedStats: tsdb.BlockStats{ + NumChunks: 2, + NumSeries: 2, + NumSamples: 4, + }, }, } { i := i @@ -377,6 +436,7 @@ func TestCompact(t *testing.T) { bucket := "thanos_" + strconv.Itoa(i) + // TODO(kakkoyun): Move to shared minio to improve test speed. m := e2edb.NewMinio(8080+i, bucket) testutil.Ok(t, s.StartAndWaitReady(m)) @@ -430,7 +490,7 @@ func TestCompact(t *testing.T) { testutil.Ok(t, s.StartAndWaitReady(cmpt)) testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(float64(len(rawBlockIds))), "thanos_blocks_meta_synced")) testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) - testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(tcase.expectOfModBlocks), "thanos_blocks_meta_modified")) + testutil.Ok(t, cmpt.WaitSumMetrics(e2e.Equals(tcase.expectNumModBlocks), "thanos_blocks_meta_modified")) str, err := e2ethanos.NewStoreGW(s.SharedDir(), "compact_"+strconv.Itoa(i), client.BucketConfig{ Type: client.S3, @@ -444,7 +504,7 @@ func TestCompact(t *testing.T) { }) testutil.Ok(t, err) testutil.Ok(t, s.StartAndWaitReady(str)) - testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(float64(tcase.expectOfBlocks)), "thanos_blocks_meta_synced")) + testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(float64(tcase.expectNumBlocks)), "thanos_blocks_meta_synced")) testutil.Ok(t, str.WaitSumMetrics(e2e.Equals(0), "thanos_blocks_meta_sync_failures_total")) q, err := e2ethanos.NewQuerier(s.SharedDir(), "compact_"+strconv.Itoa(i), []string{str.GRPCNetworkEndpoint()}, nil) @@ -462,39 +522,39 @@ func TestCompact(t *testing.T) { tcase.expected, ) - var actualOfBlocks uint64 - var actualOfChunks uint64 - var actualOfSeries uint64 - var actualOfSamples uint64 - var sources []ulid.ULID + var ( + actualNumBlocks uint64 + actual tsdb.BlockStats + sources []ulid.ULID + ) testutil.Ok(t, bkt.Iter(ctx, "", func(n string) error { id, ok := block.IsBlockDir(n) if !ok { return nil } - actualOfBlocks += 1 + actualNumBlocks += 1 meta, err := block.DownloadMeta(ctx, l, bkt, id) if err != nil { return err } - actualOfChunks += meta.Stats.NumChunks - actualOfSeries += meta.Stats.NumSeries - actualOfSamples += meta.Stats.NumSamples + actual.NumChunks += meta.Stats.NumChunks + actual.NumSeries += meta.Stats.NumSeries + actual.NumSamples += meta.Stats.NumSamples sources = append(sources, meta.Compaction.Sources...) return nil })) // Make sure only necessary amount of blocks fetched from store, to observe affects of offline deduplication. - testutil.Equals(t, tcase.expectOfBlocks, actualOfBlocks) - if len(rawBlockIds) < int(tcase.expectOfBlocks) { // check sources only if compacted. + testutil.Equals(t, tcase.expectNumBlocks, actualNumBlocks) + if len(rawBlockIds) < int(tcase.expectNumBlocks) { // check sources only if compacted. testutil.Equals(t, rawBlockIds, sources) } - testutil.Equals(t, tcase.expectOfChunks, actualOfChunks) - testutil.Equals(t, tcase.expectOfSeries, actualOfSeries) - testutil.Equals(t, tcase.expectOfSamples, actualOfSamples) + testutil.Equals(t, tcase.expectedStats.NumChunks, actual.NumChunks) + testutil.Equals(t, tcase.expectedStats.NumSeries, actual.NumSeries) + testutil.Equals(t, tcase.expectedStats.NumSamples, actual.NumSamples) }) } } diff --git a/test/e2e/e2ethanos/services.go b/test/e2e/e2ethanos/services.go index 6bd9a9fed1..49887505ba 100644 --- a/test/e2e/e2ethanos/services.go +++ b/test/e2e/e2ethanos/services.go @@ -29,7 +29,7 @@ const logLevel = "info" var defaultBackoffConfig = util.BackoffConfig{ MinBackoff: 300 * time.Millisecond, - MaxBackoff: 30 * time.Second, + MaxBackoff: 15 * time.Second, } // TODO(bwplotka): Run against multiple? diff --git a/test/e2e/query_test.go b/test/e2e/query_test.go index 5e11841bc6..914d1c6a92 100644 --- a/test/e2e/query_test.go +++ b/test/e2e/query_test.go @@ -63,6 +63,8 @@ func sortResults(res model.Vector) { } func TestQuery(t *testing.T) { + t.Parallel() + s, err := e2e.NewScenario("e2e_test_query") testutil.Ok(t, err) defer s.Close() diff --git a/test/e2e/receive_test.go b/test/e2e/receive_test.go index b772e5eef4..fda1f59591 100644 --- a/test/e2e/receive_test.go +++ b/test/e2e/receive_test.go @@ -17,7 +17,11 @@ import ( ) func TestReceive(t *testing.T) { + t.Parallel() + t.Run("hashring", func(t *testing.T) { + t.Parallel() + s, err := e2e.NewScenario("e2e_test_receive_hashring") testutil.Ok(t, err) defer s.Close() @@ -94,6 +98,7 @@ func TestReceive(t *testing.T) { }) t.Run("replication", func(t *testing.T) { + t.Parallel() s, err := e2e.NewScenario("e2e_test_receive_replication") testutil.Ok(t, err) @@ -165,6 +170,7 @@ func TestReceive(t *testing.T) { }) t.Run("replication_with_outage", func(t *testing.T) { + t.Parallel() s, err := e2e.NewScenario("e2e_test_receive_replication_with_outage") testutil.Ok(t, err) diff --git a/test/e2e/rule_test.go b/test/e2e/rule_test.go index 62f773efa7..06d38a7def 100644 --- a/test/e2e/rule_test.go +++ b/test/e2e/rule_test.go @@ -263,6 +263,8 @@ func TestRule_AlertmanagerHTTPClient(t *testing.T) { } func TestRule(t *testing.T) { + t.Parallel() + s, err := e2e.NewScenario("e2e_test_rule") testutil.Ok(t, err) defer s.Close() diff --git a/test/e2e/store_gateway_test.go b/test/e2e/store_gateway_test.go index 0abe8fdbbe..d663c71e71 100644 --- a/test/e2e/store_gateway_test.go +++ b/test/e2e/store_gateway_test.go @@ -31,6 +31,8 @@ import ( // TODO(bwplotka): Extend this test to have multiple stores and memcached. // TODO(bwplotka): Extend this test for downsampling. func TestStoreGateway(t *testing.T) { + t.Parallel() + s, err := e2e.NewScenario("e2e_test_store_gateway") testutil.Ok(t, err) defer s.Close() From 9b3352a3576092172764925438bc71fd63f674dc Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Mon, 23 Mar 2020 08:16:33 +0100 Subject: [PATCH 12/14] Increase ruler sd refresh interval Signed-off-by: Kemal Akkoyun --- test/e2e/rule_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/e2e/rule_test.go b/test/e2e/rule_test.go index 06d38a7def..a6d7d845ad 100644 --- a/test/e2e/rule_test.go +++ b/test/e2e/rule_test.go @@ -310,7 +310,7 @@ func TestRule(t *testing.T) { { // FileSD which will be used to register discover dynamically q. Files: []string{filepath.Join(e2e.ContainerSharedDir, queryTargetsSubDir, "*.yaml")}, - RefreshInterval: model.Duration(time.Hour), + RefreshInterval: model.Duration(time.Second), }, }, Scheme: "http", From dc291cf5afb967ec1ae0afe4e3aa2a7ee4684646 Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Mon, 23 Mar 2020 15:14:22 +0100 Subject: [PATCH 13/14] Address review issues Signed-off-by: Kemal Akkoyun --- Makefile | 2 +- pkg/block/fetcher.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Makefile b/Makefile index 91ccdbf1d9..1b07bf27b0 100644 --- a/Makefile +++ b/Makefile @@ -244,7 +244,7 @@ test-local: .PHONY: test-e2e test-e2e: ## Runs all Thanos e2e docker-based e2e tests from test/e2e. Required access to docker daemon. -test-e2e: docker-multi-stage +test-e2e: docker @echo ">> cleaning docker environment." @docker system prune -f --volumes @echo ">> cleaning e2e test garbage." diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index 3b531164c7..a66b12cd79 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -59,7 +59,7 @@ const ( loadedMeta = "loaded" failedMeta = "failed" - // Fetcher's synced label values. + // Synced label values. labelExcludedMeta = "label-excluded" timeExcludedMeta = "time-excluded" tooFreshMeta = "too-fresh" @@ -68,7 +68,7 @@ const ( // but don't have a replacement block yet. markedForDeletionMeta = "marked-for-deletion" - // Fetcher's synced label values. + // Modified label values. replicaRemovedMeta = "replica-label-removed" ) @@ -110,7 +110,7 @@ func newFetcherMetrics(reg prometheus.Registerer) *fetcherMetrics { m.modified = extprom.NewTxGaugeVec(reg, prometheus.GaugeOpts{ Subsystem: fetcherSubSys, Name: "modified", - Help: "Number of block metadata that modified", + Help: "Number of blocks that their metadata modified", }, []string{"modified"}, []string{replicaRemovedMeta}, From f5cbaac0d8a5a5c20b24cc6f9342f0e9e53e9f35 Mon Sep 17 00:00:00 2001 From: Kemal Akkoyun Date: Mon, 23 Mar 2020 18:39:41 +0100 Subject: [PATCH 14/14] Separate filters and modifiers Signed-off-by: Kemal Akkoyun --- Makefile | 1 + cmd/thanos/bucket.go | 8 +++--- cmd/thanos/compact.go | 13 +++++---- cmd/thanos/downsample.go | 2 +- cmd/thanos/main_test.go | 4 +-- cmd/thanos/store.go | 14 ++++----- pkg/block/fetcher.go | 51 +++++++++++++++++++++------------ pkg/block/fetcher_test.go | 36 ++++++++++++++--------- pkg/compact/clean_test.go | 2 +- pkg/compact/compact_e2e_test.go | 14 ++++----- pkg/compact/retention_test.go | 2 +- pkg/replicate/replicator.go | 2 +- pkg/replicate/scheme_test.go | 2 +- pkg/store/bucket_e2e_test.go | 8 +++--- pkg/store/bucket_test.go | 8 +++--- test/e2e/compact_test.go | 1 - 16 files changed, 96 insertions(+), 72 deletions(-) diff --git a/Makefile b/Makefile index 1b07bf27b0..47c81e6335 100644 --- a/Makefile +++ b/Makefile @@ -260,6 +260,7 @@ test-e2e-ci: docker @echo ">> cleaning e2e test garbage." @rm -rf ./test/e2e/e2e_integration_test* @echo ">> running /test/e2e tests." + @go clean -testcache @go test -failfast -parallel 1 -timeout 5m -v ./test/e2e/... .PHONY: install-deps diff --git a/cmd/thanos/bucket.go b/cmd/thanos/bucket.go index 9ca8d3cb0d..75cc07593b 100644 --- a/cmd/thanos/bucket.go +++ b/cmd/thanos/bucket.go @@ -140,7 +140,7 @@ func registerBucketVerify(m map[string]setupFunc, root *kingpin.CmdClause, name issues = append(issues, issueFn) } - fetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, "", extprom.WrapRegistererWithPrefix(extpromPrefix, reg)) + fetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, "", extprom.WrapRegistererWithPrefix(extpromPrefix, reg), nil) if err != nil { return err } @@ -189,7 +189,7 @@ func registerBucketLs(m map[string]setupFunc, root *kingpin.CmdClause, name stri return err } - fetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, "", extprom.WrapRegistererWithPrefix(extpromPrefix, reg)) + fetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, "", extprom.WrapRegistererWithPrefix(extpromPrefix, reg), nil) if err != nil { return err } @@ -289,7 +289,7 @@ func registerBucketInspect(m map[string]setupFunc, root *kingpin.CmdClause, name return err } - fetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, "", extprom.WrapRegistererWithPrefix(extpromPrefix, reg)) + fetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, "", extprom.WrapRegistererWithPrefix(extpromPrefix, reg), nil) if err != nil { return err } @@ -458,7 +458,7 @@ func refresh(ctx context.Context, logger log.Logger, bucketUI *ui.Bucket, durati return errors.Wrap(err, "bucket client") } - fetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, "", extprom.WrapRegistererWithPrefix(extpromPrefix, reg)) + fetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, "", extprom.WrapRegistererWithPrefix(extpromPrefix, reg), nil) if err != nil { return err } diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index 6a066f3b0b..fbc5c12b6c 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -287,12 +287,13 @@ func runCompact( duplicateBlocksFilter := block.NewDeduplicateFilter() prometheusRegisterer := extprom.WrapRegistererWithPrefix("thanos_", reg) - metaFetcher, err := block.NewMetaFetcher(logger, 32, bkt, "", prometheusRegisterer, - block.NewLabelShardedMetaFilter(relabelConfig).Filter, - block.NewConsistencyDelayMetaFilter(logger, consistencyDelay, prometheusRegisterer).Filter, - ignoreDeletionMarkFilter.Filter, - duplicateBlocksFilter.Filter, - block.NewReplicaLabelRemover(logger, dedupReplicaLabels).Modify, + metaFetcher, err := block.NewMetaFetcher(logger, 32, bkt, "", prometheusRegisterer, []block.MetadataFilter{ + block.NewLabelShardedMetaFilter(relabelConfig), + block.NewConsistencyDelayMetaFilter(logger, consistencyDelay, prometheusRegisterer), + ignoreDeletionMarkFilter, + duplicateBlocksFilter, + }, + block.NewReplicaLabelRemover(logger, dedupReplicaLabels), ) if err != nil { return errors.Wrap(err, "create meta fetcher") diff --git a/cmd/thanos/downsample.go b/cmd/thanos/downsample.go index 293ef59adf..beb45b95a9 100644 --- a/cmd/thanos/downsample.go +++ b/cmd/thanos/downsample.go @@ -72,7 +72,7 @@ func RunDownsample( return err } - metaFetcher, err := block.NewMetaFetcher(logger, 32, bkt, "", extprom.WrapRegistererWithPrefix("thanos_", reg)) + metaFetcher, err := block.NewMetaFetcher(logger, 32, bkt, "", extprom.WrapRegistererWithPrefix("thanos_", reg), nil) if err != nil { return errors.Wrap(err, "create meta fetcher") } diff --git a/cmd/thanos/main_test.go b/cmd/thanos/main_test.go index d44844d768..58e340db3d 100644 --- a/cmd/thanos/main_test.go +++ b/cmd/thanos/main_test.go @@ -76,7 +76,7 @@ func TestCleanupIndexCacheFolder(t *testing.T) { Name: metricIndexGenerateName, Help: metricIndexGenerateHelp, }) - metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil) + metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, nil) testutil.Ok(t, err) testutil.Ok(t, genMissingIndexCacheFiles(ctx, logger, reg, bkt, metaFetcher, dir)) @@ -116,7 +116,7 @@ func TestCleanupDownsampleCacheFolder(t *testing.T) { metrics := newDownsampleMetrics(prometheus.NewRegistry()) testutil.Equals(t, 0.0, promtest.ToFloat64(metrics.downsamples.WithLabelValues(compact.GroupKey(meta.Thanos)))) - metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil) + metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, nil) testutil.Ok(t, err) testutil.Ok(t, downsampleBucket(ctx, logger, metrics, bkt, metaFetcher, dir)) diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index 7a4f88a038..8031cd84e8 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -236,13 +236,13 @@ func runStore( ignoreDeletionMarkFilter := block.NewIgnoreDeletionMarkFilter(logger, bkt, ignoreDeletionMarksDelay) prometheusRegisterer := extprom.WrapRegistererWithPrefix("thanos_", reg) - metaFetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, dataDir, prometheusRegisterer, - block.NewTimePartitionMetaFilter(filterConf.MinTime, filterConf.MaxTime).Filter, - block.NewLabelShardedMetaFilter(relabelConfig).Filter, - block.NewConsistencyDelayMetaFilter(logger, consistencyDelay, prometheusRegisterer).Filter, - ignoreDeletionMarkFilter.Filter, - block.NewDeduplicateFilter().Filter, - ) + metaFetcher, err := block.NewMetaFetcher(logger, fetcherConcurrency, bkt, dataDir, prometheusRegisterer, []block.MetadataFilter{ + block.NewTimePartitionMetaFilter(filterConf.MinTime, filterConf.MaxTime), + block.NewLabelShardedMetaFilter(relabelConfig), + block.NewConsistencyDelayMetaFilter(logger, consistencyDelay, prometheusRegisterer), + ignoreDeletionMarkFilter, + block.NewDeduplicateFilter(), + }) if err != nil { return errors.Wrap(err, "meta fetcher") } diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index a66b12cd79..a2a5be2723 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -122,7 +122,13 @@ type MetadataFetcher interface { Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata.Meta, partial map[ulid.ULID]error, err error) } -type MetaFetcherFilter func(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, incompleteView bool) error +type MetadataFilter interface { + Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, synced *extprom.TxGaugeVec, incompleteView bool) error +} + +type MetadataModifier interface { + Modify(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, modified *extprom.TxGaugeVec, incompleteView bool) error +} // MetaFetcher is a struct that synchronizes filtered metadata of all block in the object storage with the local state. // Not go-routine safe. @@ -135,13 +141,14 @@ type MetaFetcher struct { cacheDir string metrics *fetcherMetrics - filters []MetaFetcherFilter + filters []MetadataFilter + modifiers []MetadataModifier cached map[ulid.ULID]*metadata.Meta } // NewMetaFetcher constructs MetaFetcher. -func NewMetaFetcher(logger log.Logger, concurrency int, bkt objstore.BucketReader, dir string, r prometheus.Registerer, filters ...MetaFetcherFilter) (*MetaFetcher, error) { +func NewMetaFetcher(logger log.Logger, concurrency int, bkt objstore.BucketReader, dir string, r prometheus.Registerer, filters []MetadataFilter, modifiers ...MetadataModifier) (*MetaFetcher, error) { if logger == nil { logger = log.NewNopLogger() } @@ -161,6 +168,7 @@ func NewMetaFetcher(logger log.Logger, concurrency int, bkt objstore.BucketReade cacheDir: cacheDir, metrics: newFetcherMetrics(r), filters: filters, + modifiers: modifiers, cached: map[ulid.ULID]*metadata.Meta{}, }, nil } @@ -368,11 +376,18 @@ func (s *MetaFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata. for _, f := range s.filters { // NOTE: filter can update synced metric accordingly to the reason of the exclude. - if err := f(ctx, metas, s.metrics, incompleteView); err != nil { + if err := f.Filter(ctx, metas, s.metrics.synced, incompleteView); err != nil { return nil, nil, errors.Wrap(err, "filter metas") } } + for _, m := range s.modifiers { + // NOTE: modifier can update modified metric accordingly to the reason of the modification. + if err := m.Modify(ctx, metas, s.metrics.modified, incompleteView); err != nil { + return nil, nil, errors.Wrap(err, "modify metas") + } + } + s.metrics.synced.WithLabelValues(loadedMeta).Set(float64(len(metas))) s.metrics.submit() @@ -384,7 +399,7 @@ func (s *MetaFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*metadata. return metas, partial, nil } -var _ MetaFetcherFilter = (&TimePartitionMetaFilter{}).Filter +var _ MetadataFilter = &TimePartitionMetaFilter{} // TimePartitionMetaFilter is a MetaFetcher filter that filters out blocks that are outside of specified time range. // Not go-routine safe. @@ -398,18 +413,18 @@ func NewTimePartitionMetaFilter(MinTime, MaxTime model.TimeOrDurationValue) *Tim } // Filter filters out blocks that are outside of specified time range. -func (f *TimePartitionMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, _ bool) error { +func (f *TimePartitionMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced *extprom.TxGaugeVec, _ bool) error { for id, m := range metas { if m.MaxTime >= f.minTime.PrometheusTimestamp() && m.MinTime <= f.maxTime.PrometheusTimestamp() { continue } - metrics.synced.WithLabelValues(timeExcludedMeta).Inc() + synced.WithLabelValues(timeExcludedMeta).Inc() delete(metas, id) } return nil } -var _ MetaFetcherFilter = (&LabelShardedMetaFilter{}).Filter +var _ MetadataFilter = &LabelShardedMetaFilter{} // LabelShardedMetaFilter represents struct that allows sharding. // Not go-routine safe. @@ -426,7 +441,7 @@ func NewLabelShardedMetaFilter(relabelConfig []*relabel.Config) *LabelShardedMet const blockIDLabel = "__block_id" // Filter filters out blocks that have no labels after relabelling of each block external (Thanos) labels. -func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, _ bool) error { +func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced *extprom.TxGaugeVec, _ bool) error { var lbls labels.Labels for id, m := range metas { lbls = lbls[:0] @@ -436,7 +451,7 @@ func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]* } if processedLabels := relabel.Process(lbls, f.relabelConfig...); len(processedLabels) == 0 { - metrics.synced.WithLabelValues(labelExcludedMeta).Inc() + synced.WithLabelValues(labelExcludedMeta).Inc() delete(metas, id) } } @@ -456,7 +471,7 @@ func NewDeduplicateFilter() *DeduplicateFilter { // Filter filters out duplicate blocks that can be formed // from two or more overlapping blocks that fully submatches the source blocks of the older blocks. -func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, _ bool) error { +func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced *extprom.TxGaugeVec, _ bool) error { var wg sync.WaitGroup metasByResolution := make(map[int64][]*metadata.Meta) @@ -473,7 +488,7 @@ func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metad BlockMeta: tsdb.BlockMeta{ ULID: ulid.MustNew(uint64(0), nil), }, - }), metasByResolution[res], metas, res, metrics.synced) + }), metasByResolution[res], metas, res, synced) }(res) } @@ -570,14 +585,14 @@ func NewReplicaLabelRemover(logger log.Logger, replicaLabels []string) *ReplicaL } // Modify modifies external labels of existing blocks, it removes given replica labels from the metadata of blocks that have it. -func (r *ReplicaLabelRemover) Modify(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, view bool) error { +func (r *ReplicaLabelRemover) Modify(_ context.Context, metas map[ulid.ULID]*metadata.Meta, modified *extprom.TxGaugeVec, view bool) error { for u, meta := range metas { labels := meta.Thanos.Labels for _, replicaLabel := range r.replicaLabels { if _, exists := labels[replicaLabel]; exists { level.Debug(r.logger).Log("msg", "replica label removed", "label", replicaLabel) delete(labels, replicaLabel) - metrics.modified.WithLabelValues(replicaRemovedMeta).Inc() + modified.WithLabelValues(replicaRemovedMeta).Inc() } } metas[u].Thanos.Labels = labels @@ -611,7 +626,7 @@ func NewConsistencyDelayMetaFilter(logger log.Logger, consistencyDelay time.Dura } // Filter filters out blocks that filters blocks that have are created before a specified consistency delay. -func (f *ConsistencyDelayMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, _ bool) error { +func (f *ConsistencyDelayMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced *extprom.TxGaugeVec, _ bool) error { for id, meta := range metas { // TODO(khyatisoneji): Remove the checks about Thanos Source // by implementing delete delay to fetch metas. @@ -622,7 +637,7 @@ func (f *ConsistencyDelayMetaFilter) Filter(_ context.Context, metas map[ulid.UL meta.Thanos.Source != metadata.CompactorRepairSource { level.Debug(f.logger).Log("msg", "block is too fresh for now", "block", id) - metrics.synced.WithLabelValues(tooFreshMeta).Inc() + synced.WithLabelValues(tooFreshMeta).Inc() delete(metas, id) } } @@ -657,7 +672,7 @@ func (f *IgnoreDeletionMarkFilter) DeletionMarkBlocks() map[ulid.ULID]*metadata. // Filter filters out blocks that are marked for deletion after a given delay. // It also returns the blocks that can be deleted since they were uploaded delay duration before current time. -func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, m *fetcherMetrics, _ bool) error { +func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, synced *extprom.TxGaugeVec, _ bool) error { f.deletionMarkMap = make(map[ulid.ULID]*metadata.DeletionMark) for id := range metas { @@ -674,7 +689,7 @@ func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.UL } f.deletionMarkMap[id] = deletionMark if time.Since(time.Unix(deletionMark.DeletionTime, 0)).Seconds() > f.delay.Seconds() { - m.synced.WithLabelValues(markedForDeletionMeta).Inc() + synced.WithLabelValues(markedForDeletionMeta).Inc() delete(metas, id) } } diff --git a/pkg/block/fetcher_test.go b/pkg/block/fetcher_test.go index b0060f6636..99920b8e5f 100644 --- a/pkg/block/fetcher_test.go +++ b/pkg/block/fetcher_test.go @@ -41,6 +41,18 @@ func newTestFetcherMetrics() *fetcherMetrics { } } +type ulidFilter struct { + ulidToDelete *ulid.ULID +} + +func (f *ulidFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced *extprom.TxGaugeVec, incompleteView bool) error { + if _, ok := metas[*f.ulidToDelete]; ok { + synced.WithLabelValues("filtered").Inc() + delete(metas, *f.ulidToDelete) + } + return nil +} + func ULID(i int) ulid.ULID { return ulid.MustNew(uint64(i), nil) } func ULIDs(is ...int) []ulid.ULID { @@ -63,12 +75,8 @@ func TestMetaFetcher_Fetch(t *testing.T) { var ulidToDelete ulid.ULID r := prometheus.NewRegistry() - f, err := NewMetaFetcher(log.NewNopLogger(), 20, bkt, dir, r, func(_ context.Context, metas map[ulid.ULID]*metadata.Meta, metrics *fetcherMetrics, incompleteView bool) error { - if _, ok := metas[ulidToDelete]; ok { - metrics.synced.WithLabelValues("filtered").Inc() - delete(metas, ulidToDelete) - } - return nil + f, err := NewMetaFetcher(log.NewNopLogger(), 20, bkt, dir, r, []MetadataFilter{ + &ulidFilter{ulidToDelete: &ulidToDelete}, }) testutil.Ok(t, err) @@ -344,7 +352,7 @@ func TestLabelShardedMetaFilter_Filter_Basic(t *testing.T) { } m := newTestFetcherMetrics() - testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Ok(t, f.Filter(ctx, input, m.synced, false)) testutil.Equals(t, 3.0, promtest.ToFloat64(m.synced.WithLabelValues(labelExcludedMeta))) testutil.Equals(t, expected, input) @@ -442,7 +450,7 @@ func TestLabelShardedMetaFilter_Filter_Hashmod(t *testing.T) { deleted := len(input) - len(expected) m := newTestFetcherMetrics() - testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Ok(t, f.Filter(ctx, input, m.synced, false)) testutil.Equals(t, expected, input) testutil.Equals(t, float64(deleted), promtest.ToFloat64(m.synced.WithLabelValues(labelExcludedMeta))) @@ -506,7 +514,7 @@ func TestTimePartitionMetaFilter_Filter(t *testing.T) { } m := newTestFetcherMetrics() - testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Ok(t, f.Filter(ctx, input, m.synced, false)) testutil.Equals(t, 2.0, promtest.ToFloat64(m.synced.WithLabelValues(timeExcludedMeta))) testutil.Equals(t, expected, input) @@ -857,7 +865,7 @@ func TestDeduplicateFilter_Filter(t *testing.T) { }, } } - testutil.Ok(t, f.Filter(ctx, metas, m, false)) + testutil.Ok(t, f.Filter(ctx, metas, m.synced, false)) compareSliceWithMapKeys(t, metas, tcase.expected) testutil.Equals(t, float64(inputLen-len(tcase.expected)), promtest.ToFloat64(m.synced.WithLabelValues(duplicateMeta))) }); !ok { @@ -909,7 +917,7 @@ func TestReplicaLabelRemover_Modify(t *testing.T) { }, } { m := newTestFetcherMetrics() - testutil.Ok(t, rm.Modify(ctx, tcase.input, m, false)) + testutil.Ok(t, rm.Modify(ctx, tcase.input, m.modified, false)) testutil.Equals(t, tcase.modified, promtest.ToFloat64(m.modified.WithLabelValues(replicaRemovedMeta))) testutil.Equals(t, tcase.expected, tcase.input) @@ -1015,7 +1023,7 @@ func TestConsistencyDelayMetaFilter_Filter_0(t *testing.T) { f := NewConsistencyDelayMetaFilter(nil, 0*time.Second, reg) testutil.Equals(t, map[string]float64{"consistency_delay_seconds": 0.0}, extprom.CurrentGaugeValuesFor(t, reg, "consistency_delay_seconds")) - testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Ok(t, f.Filter(ctx, input, m.synced, false)) testutil.Equals(t, 0.0, promtest.ToFloat64(m.synced.WithLabelValues(tooFreshMeta))) testutil.Equals(t, expected, input) }) @@ -1040,7 +1048,7 @@ func TestConsistencyDelayMetaFilter_Filter_0(t *testing.T) { f := NewConsistencyDelayMetaFilter(nil, 30*time.Minute, reg) testutil.Equals(t, map[string]float64{"consistency_delay_seconds": (30 * time.Minute).Seconds()}, extprom.CurrentGaugeValuesFor(t, reg, "consistency_delay_seconds")) - testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Ok(t, f.Filter(ctx, input, m.synced, false)) testutil.Equals(t, float64(len(u.created)-len(expected)), promtest.ToFloat64(m.synced.WithLabelValues(tooFreshMeta))) testutil.Equals(t, expected, input) }) @@ -1093,7 +1101,7 @@ func TestIgnoreDeletionMarkFilter_Filter(t *testing.T) { } m := newTestFetcherMetrics() - testutil.Ok(t, f.Filter(ctx, input, m, false)) + testutil.Ok(t, f.Filter(ctx, input, m.synced, false)) testutil.Equals(t, 1.0, promtest.ToFloat64(m.synced.WithLabelValues(markedForDeletionMeta))) testutil.Equals(t, expected, input) }) diff --git a/pkg/compact/clean_test.go b/pkg/compact/clean_test.go index 74cfbaaf5a..4507b3f5a5 100644 --- a/pkg/compact/clean_test.go +++ b/pkg/compact/clean_test.go @@ -29,7 +29,7 @@ func TestBestEffortCleanAbortedPartialUploads(t *testing.T) { bkt := inmem.NewBucket() logger := log.NewNopLogger() - metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil) + metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, nil) testutil.Ok(t, err) // 1. No meta, old block, should be removed. diff --git a/pkg/compact/compact_e2e_test.go b/pkg/compact/compact_e2e_test.go index 5520a9a848..52e4147623 100644 --- a/pkg/compact/compact_e2e_test.go +++ b/pkg/compact/compact_e2e_test.go @@ -90,9 +90,9 @@ func TestSyncer_GarbageCollect_e2e(t *testing.T) { } duplicateBlocksFilter := block.NewDeduplicateFilter() - metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, - duplicateBlocksFilter.Filter, - ) + metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, []block.MetadataFilter{ + duplicateBlocksFilter, + }) testutil.Ok(t, err) blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) @@ -178,10 +178,10 @@ func TestGroup_Compact_e2e(t *testing.T) { ignoreDeletionMarkFilter := block.NewIgnoreDeletionMarkFilter(logger, bkt, 48*time.Hour) duplicateBlocksFilter := block.NewDeduplicateFilter() - metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, - ignoreDeletionMarkFilter.Filter, - duplicateBlocksFilter.Filter, - ) + metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, []block.MetadataFilter{ + ignoreDeletionMarkFilter, + duplicateBlocksFilter, + }) testutil.Ok(t, err) blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) diff --git a/pkg/compact/retention_test.go b/pkg/compact/retention_test.go index 5c9813c117..980b883839 100644 --- a/pkg/compact/retention_test.go +++ b/pkg/compact/retention_test.go @@ -245,7 +245,7 @@ func TestApplyRetentionPolicyByResolution(t *testing.T) { uploadMockBlock(t, bkt, b.id, b.minTime, b.maxTime, int64(b.resolution)) } - metaFetcher, err := block.NewMetaFetcher(logger, 32, bkt, "", nil) + metaFetcher, err := block.NewMetaFetcher(logger, 32, bkt, "", nil, nil) testutil.Ok(t, err) blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) diff --git a/pkg/replicate/replicator.go b/pkg/replicate/replicator.go index 1e674fa6a4..d419681f85 100644 --- a/pkg/replicate/replicator.go +++ b/pkg/replicate/replicator.go @@ -150,7 +150,7 @@ func RunReplicate( Help: "The Duration of replication runs split by success and error.", }, []string{"result"}) - fetcher, err := thanosblock.NewMetaFetcher(logger, 32, fromBkt, "", reg) + fetcher, err := thanosblock.NewMetaFetcher(logger, 32, fromBkt, "", reg, nil) if err != nil { return errors.Wrapf(err, "create meta fetcher with bucket %v", fromBkt) } diff --git a/pkg/replicate/scheme_test.go b/pkg/replicate/scheme_test.go index 9aaf9cff05..6b441a572b 100644 --- a/pkg/replicate/scheme_test.go +++ b/pkg/replicate/scheme_test.go @@ -313,7 +313,7 @@ func TestReplicationSchemeAll(t *testing.T) { } filter := NewBlockFilter(logger, selector, compact.ResolutionLevelRaw, 1).Filter - fetcher, err := block.NewMetaFetcher(logger, 32, originBucket, "", nil) + fetcher, err := block.NewMetaFetcher(logger, 32, originBucket, "", nil, nil) testutil.Ok(t, err) r := newReplicationScheme( diff --git a/pkg/store/bucket_e2e_test.go b/pkg/store/bucket_e2e_test.go index 6bdc6a5950..821cd70d27 100644 --- a/pkg/store/bucket_e2e_test.go +++ b/pkg/store/bucket_e2e_test.go @@ -147,10 +147,10 @@ func prepareStoreWithTestBlocks(t testing.TB, dir string, bkt objstore.Bucket, m maxTime: maxTime, } - metaFetcher, err := block.NewMetaFetcher(s.logger, 20, bkt, dir, nil, - block.NewTimePartitionMetaFilter(filterConf.MinTime, filterConf.MaxTime).Filter, - block.NewLabelShardedMetaFilter(relabelConfig).Filter, - ) + metaFetcher, err := block.NewMetaFetcher(s.logger, 20, bkt, dir, nil, []block.MetadataFilter{ + block.NewTimePartitionMetaFilter(filterConf.MinTime, filterConf.MaxTime), + block.NewLabelShardedMetaFilter(relabelConfig), + }) testutil.Ok(t, err) store, err := NewBucketStore( diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index a95372ec8f..75cf446643 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -710,10 +710,10 @@ func testSharding(t *testing.T, reuseDisk string, bkt objstore.Bucket, all ...ul testutil.Ok(t, yaml.Unmarshal([]byte(sc.relabel), &relabelConf)) rec := &recorder{Bucket: bkt} - metaFetcher, err := block.NewMetaFetcher(logger, 20, bkt, dir, nil, - block.NewTimePartitionMetaFilter(allowAllFilterConf.MinTime, allowAllFilterConf.MaxTime).Filter, - block.NewLabelShardedMetaFilter(relabelConf).Filter, - ) + metaFetcher, err := block.NewMetaFetcher(logger, 20, bkt, dir, nil, []block.MetadataFilter{ + block.NewTimePartitionMetaFilter(allowAllFilterConf.MinTime, allowAllFilterConf.MaxTime), + block.NewLabelShardedMetaFilter(relabelConf), + }) testutil.Ok(t, err) bucketStore, err := NewBucketStore( diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index 47b76be5ef..a906117905 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -128,7 +128,6 @@ func TestCompact(t *testing.T) { samplesPerSeries: 12, }, }, - replicaLabels: []string{}, downsamplingEnabled: true, query: "{a=\"1\"}",