diff --git a/pkg/compactor/background_chunks_series_set.go b/pkg/compactor/background_chunks_series_set.go new file mode 100644 index 0000000000..bca40f73d2 --- /dev/null +++ b/pkg/compactor/background_chunks_series_set.go @@ -0,0 +1,60 @@ +package compactor + +import ( + "context" + + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" +) + +type backgrounChunkSeriesSet struct { + nextSet chan storage.ChunkSeries + actual storage.ChunkSeries + cs storage.ChunkSeriesSet +} + +func (b *backgrounChunkSeriesSet) Next() bool { + s, ok := <-b.nextSet + b.actual = s + return ok +} + +func (b *backgrounChunkSeriesSet) At() storage.ChunkSeries { + return b.actual +} + +func (b *backgrounChunkSeriesSet) Err() error { + return b.cs.Err() +} + +func (b *backgrounChunkSeriesSet) Warnings() annotations.Annotations { + return b.cs.Warnings() +} + +func (b *backgrounChunkSeriesSet) run(ctx context.Context) { + for { + if !b.cs.Next() { + close(b.nextSet) + return + } + + select { + case b.nextSet <- b.cs.At(): + case <-ctx.Done(): + return + } + } +} + +func NewBackgroundChunkSeriesSet(ctx context.Context, cs storage.ChunkSeriesSet) storage.ChunkSeriesSet { + r := &backgrounChunkSeriesSet{ + cs: cs, + nextSet: make(chan storage.ChunkSeries, 1000), + } + + go func() { + r.run(ctx) + }() + + return r +} diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index 71dfc775df..d8970de61a 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -152,23 +152,23 @@ func NewBlocksCleaner( tenantBlocks: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ Name: "cortex_bucket_blocks_count", Help: "Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks.", - }, []string{"user"}), + }, commonLabels), tenantBlocksMarkedForDelete: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ Name: "cortex_bucket_blocks_marked_for_deletion_count", Help: "Total number of blocks marked for deletion in the bucket.", - }, []string{"user"}), + }, commonLabels), tenantBlocksMarkedForNoCompaction: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ Name: "cortex_bucket_blocks_marked_for_no_compaction_count", Help: "Total number of blocks marked for no compaction in the bucket.", - }, []string{"user"}), + }, commonLabels), tenantPartialBlocks: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ Name: "cortex_bucket_blocks_partials_count", Help: "Total number of partial blocks.", - }, []string{"user"}), + }, commonLabels), tenantBucketIndexLastUpdate: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ Name: "cortex_bucket_index_last_successful_update_timestamp_seconds", Help: "Timestamp of the last successful update of a tenant's bucket index.", - }, []string{"user"}), + }, commonLabels), tenantBlocksCleanedTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: "cortex_bucket_blocks_cleaned_total", Help: "Total number of blocks deleted for a tenant.", diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 01f534f629..16ae0b9c9c 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -160,6 +160,30 @@ var ( } return compactor, plannerFactory, nil } + + DefaultBlockDeletableCheckerFactory = func(_ context.Context, _ objstore.InstrumentedBucket, _ log.Logger) compact.BlockDeletableChecker { + return compact.DefaultBlockDeletableChecker{} + } + + PartitionCompactionBlockDeletableCheckerFactory = func(ctx context.Context, bkt objstore.InstrumentedBucket, logger log.Logger) compact.BlockDeletableChecker { + return NewPartitionCompactionBlockDeletableChecker() + } + + DefaultCompactionLifecycleCallbackFactory = func(_ context.Context, _ objstore.InstrumentedBucket, _ log.Logger, _ int, _ string, _ string, _ *compactorMetrics) compact.CompactionLifecycleCallback { + return compact.DefaultCompactionLifecycleCallback{} + } + + ShardedCompactionLifecycleCallbackFactory = func(ctx context.Context, userBucket objstore.InstrumentedBucket, logger log.Logger, metaSyncConcurrency int, compactDir string, userID string, compactorMetrics *compactorMetrics) compact.CompactionLifecycleCallback { + return NewShardedCompactionLifecycleCallback( + ctx, + userBucket, + logger, + metaSyncConcurrency, + compactDir, + userID, + compactorMetrics, + ) + } ) // BlocksGrouperFactory builds and returns the grouper to use to compact a tenant's blocks. @@ -202,6 +226,22 @@ type PlannerFactory func( compactorMetrics *compactorMetrics, ) compact.Planner +type CompactionLifecycleCallbackFactory func( + ctx context.Context, + userBucket objstore.InstrumentedBucket, + logger log.Logger, + metaSyncConcurrency int, + compactDir string, + userID string, + compactorMetrics *compactorMetrics, +) compact.CompactionLifecycleCallback + +type BlockDeletableCheckerFactory func( + ctx context.Context, + bkt objstore.InstrumentedBucket, + logger log.Logger, +) compact.BlockDeletableChecker + // Limits defines limits used by the Compactor. type Limits interface { CompactorTenantShardSize(userID string) int @@ -380,6 +420,10 @@ type Compactor struct { blocksPlannerFactory PlannerFactory + blockDeletableCheckerFactory BlockDeletableCheckerFactory + + compactionLifecycleCallbackFactory CompactionLifecycleCallbackFactory + // Client used to run operations on the bucket storing blocks. bucketClient objstore.InstrumentedBucket @@ -436,11 +480,25 @@ func NewCompactor(compactorCfg Config, storageCfg cortex_tsdb.BlocksStorageConfi } } + var blockDeletableCheckerFactory BlockDeletableCheckerFactory + if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle && compactorCfg.CompactionStrategy == util.CompactionStrategyPartitioning { + blockDeletableCheckerFactory = PartitionCompactionBlockDeletableCheckerFactory + } else { + blockDeletableCheckerFactory = DefaultBlockDeletableCheckerFactory + } + + var compactionLifecycleCallbackFactory CompactionLifecycleCallbackFactory + if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle && compactorCfg.CompactionStrategy == util.CompactionStrategyPartitioning { + compactionLifecycleCallbackFactory = ShardedCompactionLifecycleCallbackFactory + } else { + compactionLifecycleCallbackFactory = DefaultCompactionLifecycleCallbackFactory + } + if ingestionReplicationFactor <= 0 { ingestionReplicationFactor = 1 } - cortexCompactor, err := newCompactor(compactorCfg, storageCfg, logger, registerer, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, limits, ingestionReplicationFactor) + cortexCompactor, err := newCompactor(compactorCfg, storageCfg, logger, registerer, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, blockDeletableCheckerFactory, compactionLifecycleCallbackFactory, limits, ingestionReplicationFactor) if err != nil { return nil, errors.Wrap(err, "failed to create Cortex blocks compactor") } @@ -456,6 +514,8 @@ func newCompactor( bucketClientFactory func(ctx context.Context) (objstore.InstrumentedBucket, error), blocksGrouperFactory BlocksGrouperFactory, blocksCompactorFactory BlocksCompactorFactory, + blockDeletableCheckerFactory BlockDeletableCheckerFactory, + compactionLifecycleCallbackFactory CompactionLifecycleCallbackFactory, limits *validation.Overrides, ingestionReplicationFactor int, ) (*Compactor, error) { @@ -466,15 +526,17 @@ func newCompactor( compactorMetrics = newDefaultCompactorMetrics(registerer) } c := &Compactor{ - compactorCfg: compactorCfg, - storageCfg: storageCfg, - parentLogger: logger, - logger: log.With(logger, "component", "compactor"), - registerer: registerer, - bucketClientFactory: bucketClientFactory, - blocksGrouperFactory: blocksGrouperFactory, - blocksCompactorFactory: blocksCompactorFactory, - allowedTenants: util.NewAllowedTenants(compactorCfg.EnabledTenants, compactorCfg.DisabledTenants), + compactorCfg: compactorCfg, + storageCfg: storageCfg, + parentLogger: logger, + logger: log.With(logger, "component", "compactor"), + registerer: registerer, + bucketClientFactory: bucketClientFactory, + blocksGrouperFactory: blocksGrouperFactory, + blocksCompactorFactory: blocksCompactorFactory, + blockDeletableCheckerFactory: blockDeletableCheckerFactory, + compactionLifecycleCallbackFactory: compactionLifecycleCallbackFactory, + allowedTenants: util.NewAllowedTenants(compactorCfg.EnabledTenants, compactorCfg.DisabledTenants), CompactorStartDurationSeconds: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_start_duration_seconds", @@ -662,12 +724,6 @@ func (c *Compactor) starting(ctx context.Context) error { }, c.bucketClient, c.usersScanner, c.compactorCfg.CompactionVisitMarkerTimeout, c.limits, c.parentLogger, cleanerRingLifecyclerID, c.registerer, c.compactorCfg.CleanerVisitMarkerTimeout, c.compactorCfg.CleanerVisitMarkerFileUpdateInterval, c.compactorMetrics.syncerBlocksMarkedForDeletion, c.compactorMetrics.remainingPlannedCompactions) - // Ensure an initial cleanup occurred before starting the compactor. - if err := services.StartAndAwaitRunning(ctx, c.blocksCleaner); err != nil { - c.ringSubservices.StopAsync() - return errors.Wrap(err, "failed to start the blocks cleaner") - } - if c.compactorCfg.CachingBucketEnabled { matchers := cortex_tsdb.NewMatchers() // Do not cache tenant deletion marker and block deletion marker for compactor @@ -698,15 +754,26 @@ func (c *Compactor) stopping(_ error) error { } func (c *Compactor) running(ctx context.Context) error { + // Ensure an initial cleanup occurred as first thing when running compactor. + if err := services.StartAndAwaitRunning(ctx, c.blocksCleaner); err != nil { + c.ringSubservices.StopAsync() + return errors.Wrap(err, "failed to start the blocks cleaner") + } + // Run an initial compaction before starting the interval. + // Insert jitter right before compaction starts to avoid multiple starting compactor to be in sync + time.Sleep(time.Duration(rand.Int63n(int64(float64(c.compactorCfg.CompactionInterval) * 0.1)))) c.compactUsers(ctx) - ticker := time.NewTicker(util.DurationWithJitter(c.compactorCfg.CompactionInterval, 0.05)) + ticker := time.NewTicker(c.compactorCfg.CompactionInterval) defer ticker.Stop() for { select { case <-ticker.C: + // Insert jitter right before compaction starts, so that there will always + // have jitter even compaction time is longer than CompactionInterval + time.Sleep(time.Duration(rand.Int63n(int64(float64(c.compactorCfg.CompactionInterval) * 0.1)))) c.compactUsers(ctx) case <-ctx.Done(): return nil @@ -717,23 +784,19 @@ func (c *Compactor) running(ctx context.Context) error { } func (c *Compactor) compactUsers(ctx context.Context) { - failed := false + succeeded := false interrupted := false + compactionErrorCount := 0 c.CompactionRunsStarted.Inc() defer func() { - // interruptions and successful runs are considered - // mutually exclusive but we consider a run failed if any - // tenant runs failed even if later runs are interrupted - if !interrupted && !failed { + if succeeded && compactionErrorCount == 0 { c.CompactionRunsCompleted.Inc() c.CompactionRunsLastSuccess.SetToCurrentTime() - } - if interrupted { + } else if interrupted { c.CompactionRunsInterrupted.Inc() - } - if failed { + } else { c.CompactionRunsFailed.Inc() } @@ -747,7 +810,6 @@ func (c *Compactor) compactUsers(ctx context.Context) { level.Info(c.logger).Log("msg", "discovering users from bucket") users, err := c.discoverUsersWithRetries(ctx) if err != nil { - failed = true level.Error(c.logger).Log("msg", "failed to discover users from bucket", "err", err) return } @@ -816,7 +878,7 @@ func (c *Compactor) compactUsers(ctx context.Context) { } c.CompactionRunFailedTenants.Inc() - failed = true + compactionErrorCount++ level.Error(c.logger).Log("msg", "failed to compact user blocks", "user", userID, "err", err) continue } @@ -851,6 +913,7 @@ func (c *Compactor) compactUsers(ctx context.Context) { } } } + succeeded = true } func (c *Compactor) compactUserWithRetries(ctx context.Context, userID string) error { @@ -885,6 +948,11 @@ func (c *Compactor) compactUserWithRetries(ctx context.Context, userID string) e retries.Wait() } + err := errors.Unwrap(errors.Cause(lastErr)) + if errors.Is(err, plannerCompletedPartitionError) || errors.Is(err, plannerVisitedPartitionError) { + return nil + } + return lastErr } @@ -898,7 +966,12 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { // Filters out duplicate blocks that can be formed from two or more overlapping // blocks that fully submatches the source blocks of the older blocks. - deduplicateBlocksFilter := block.NewDeduplicateFilter(c.compactorCfg.BlockSyncConcurrency) + var deduplicateBlocksFilter CortexMetadataFilter + if c.compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle && c.compactorCfg.CompactionStrategy == util.CompactionStrategyPartitioning { + deduplicateBlocksFilter = &DisabledDeduplicateFilter{} + } else { + deduplicateBlocksFilter = block.NewDeduplicateFilter(c.compactorCfg.BlockSyncConcurrency) + } // While fetching blocks, we filter out blocks that were marked for deletion by using IgnoreDeletionMarkFilter. // No delay is used -- all blocks with deletion marker are ignored, and not considered for compaction. @@ -966,12 +1039,14 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { currentCtx, cancel := context.WithCancel(ctx) defer cancel() - compactor, err := compact.NewBucketCompactor( + compactor, err := compact.NewBucketCompactorWithCheckerAndCallback( ulogger, syncer, c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, c.BlocksMarkedForNoCompaction, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed, syncerMetrics, c.compactorMetrics, c.ring, c.ringLifecycler, c.limits, userID, noCompactMarkerFilter, c.ingestionReplicationFactor), c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, userID, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed, c.compactorMetrics), c.blocksCompactor, + c.blockDeletableCheckerFactory(currentCtx, bucket, ulogger), + c.compactionLifecycleCallbackFactory(currentCtx, bucket, ulogger, c.compactorCfg.MetaSyncConcurrency, c.compactDirForUser(userID), userID, c.compactorMetrics), c.compactDirForUser(userID), bucket, c.compactorCfg.CompactionConcurrency, @@ -982,6 +1057,7 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { } if err := compactor.Compact(ctx); err != nil { + level.Warn(ulogger).Log("msg", "compaction failed with error", "err", err) return errors.Wrap(err, "compaction") } @@ -1148,3 +1224,20 @@ func (c *Compactor) isPermissionDeniedErr(err error) bool { } return s.Code() == codes.PermissionDenied } + +type CortexMetadataFilter interface { + block.DeduplicateFilter + block.MetadataFilter +} + +type DisabledDeduplicateFilter struct { +} + +func (f *DisabledDeduplicateFilter) Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, synced block.GaugeVec, modified block.GaugeVec) error { + // don't do any deduplicate filtering + return nil +} + +func (f *DisabledDeduplicateFilter) DuplicateIDs() []ulid.ULID { + return nil +} diff --git a/pkg/compactor/compactor_metrics.go b/pkg/compactor/compactor_metrics.go index 23e7bca6c0..03d4686ee1 100644 --- a/pkg/compactor/compactor_metrics.go +++ b/pkg/compactor/compactor_metrics.go @@ -40,6 +40,7 @@ type compactorMetrics struct { compactionErrorsCount *prometheus.CounterVec partitionCount *prometheus.GaugeVec compactionsNotPlanned *prometheus.CounterVec + compactionDuration *prometheus.GaugeVec } const ( @@ -179,6 +180,10 @@ func newCompactorMetricsWithLabels(reg prometheus.Registerer, commonLabels []str Name: "cortex_compactor_group_compactions_not_planned_total", Help: "Total number of group compaction not planned due to error.", }, compactionLabels) + m.compactionDuration = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_compact_group_compaction_duration_seconds", + Help: "Duration of completed compactions in seconds", + }, compactionLabels) return &m } @@ -231,6 +236,7 @@ func (m *compactorMetrics) initMetricWithCompactionLabelValues(labelValue ...str m.verticalCompactions.WithLabelValues(labelValue...) m.partitionCount.WithLabelValues(labelValue...) m.compactionsNotPlanned.WithLabelValues(labelValue...) + m.compactionDuration.WithLabelValues(labelValue...) } func (m *compactorMetrics) deleteMetricsForDeletedTenant(userID string) { @@ -243,4 +249,5 @@ func (m *compactorMetrics) deleteMetricsForDeletedTenant(userID string) { m.verticalCompactions.DeleteLabelValues(userID) m.partitionCount.DeleteLabelValues(userID) m.compactionsNotPlanned.DeleteLabelValues(userID) + m.compactionDuration.DeleteLabelValues(userID) } diff --git a/pkg/compactor/compactor_metrics_test.go b/pkg/compactor/compactor_metrics_test.go index 947fd7f396..b0e212962c 100644 --- a/pkg/compactor/compactor_metrics_test.go +++ b/pkg/compactor/compactor_metrics_test.go @@ -9,7 +9,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestSyncerMetrics(t *testing.T) { +func TestCompactorMetrics(t *testing.T) { reg := prometheus.NewPedanticRegistry() cm := newCompactorMetricsWithLabels(reg, commonLabels, commonLabels) @@ -140,6 +140,11 @@ func TestSyncerMetrics(t *testing.T) { cortex_compactor_group_compactions_not_planned_total{user="aaa"} 544390 cortex_compactor_group_compactions_not_planned_total{user="bbb"} 555500 cortex_compactor_group_compactions_not_planned_total{user="ccc"} 566610 + # HELP cortex_compact_group_compaction_duration_seconds Duration of completed compactions in seconds + # TYPE cortex_compact_group_compaction_duration_seconds gauge + cortex_compact_group_compaction_duration_seconds{user="aaa"} 577720 + cortex_compact_group_compaction_duration_seconds{user="bbb"} 588830 + cortex_compact_group_compaction_duration_seconds{user="ccc"} 599940 `)) require.NoError(t, err) @@ -199,4 +204,7 @@ func generateTestData(cm *compactorMetrics, base float64) { cm.compactionsNotPlanned.WithLabelValues("aaa").Add(49 * base) cm.compactionsNotPlanned.WithLabelValues("bbb").Add(50 * base) cm.compactionsNotPlanned.WithLabelValues("ccc").Add(51 * base) + cm.compactionDuration.WithLabelValues("aaa").Add(52 * base) + cm.compactionDuration.WithLabelValues("bbb").Add(53 * base) + cm.compactionDuration.WithLabelValues("ccc").Add(54 * base) } diff --git a/pkg/compactor/compactor_paritioning_test.go b/pkg/compactor/compactor_paritioning_test.go new file mode 100644 index 0000000000..cccf5761cf --- /dev/null +++ b/pkg/compactor/compactor_paritioning_test.go @@ -0,0 +1,1807 @@ +package compactor + +import ( + "context" + "crypto/rand" + "encoding/json" + "flag" + "fmt" + "io" + "os" + "path" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/block" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/compact" + "gopkg.in/yaml.v2" + + "github.com/cortexproject/cortex/pkg/ring" + "github.com/cortexproject/cortex/pkg/ring/kv/consul" + "github.com/cortexproject/cortex/pkg/storage/bucket" + cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" + "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" + cortex_storage_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" + "github.com/cortexproject/cortex/pkg/util" + "github.com/cortexproject/cortex/pkg/util/concurrency" + "github.com/cortexproject/cortex/pkg/util/flagext" + "github.com/cortexproject/cortex/pkg/util/services" + cortex_testutil "github.com/cortexproject/cortex/pkg/util/test" + "github.com/cortexproject/cortex/pkg/util/validation" +) + +func TestPartitionConfig_ShouldSupportYamlConfig(t *testing.T) { + yamlCfg := ` +block_ranges: [2h, 48h] +consistency_delay: 1h +block_sync_concurrency: 123 +data_dir: /tmp +compaction_interval: 15m +compaction_retries: 123 +compaction_strategy: partitioning +` + + cfg := Config{} + flagext.DefaultValues(&cfg) + assert.NoError(t, yaml.Unmarshal([]byte(yamlCfg), &cfg)) + assert.Equal(t, cortex_tsdb.DurationList{2 * time.Hour, 48 * time.Hour}, cfg.BlockRanges) + assert.Equal(t, time.Hour, cfg.ConsistencyDelay) + assert.Equal(t, 123, cfg.BlockSyncConcurrency) + assert.Equal(t, "/tmp", cfg.DataDir) + assert.Equal(t, 15*time.Minute, cfg.CompactionInterval) + assert.Equal(t, 123, cfg.CompactionRetries) + assert.Equal(t, util.CompactionStrategyPartitioning, cfg.CompactionStrategy) +} + +func TestPartitionConfig_ShouldSupportCliFlags(t *testing.T) { + fs := flag.NewFlagSet("", flag.PanicOnError) + cfg := Config{} + cfg.RegisterFlags(fs) + require.NoError(t, fs.Parse([]string{ + "-compactor.block-ranges=2h,48h", + "-compactor.consistency-delay=1h", + "-compactor.block-sync-concurrency=123", + "-compactor.data-dir=/tmp", + "-compactor.compaction-interval=15m", + "-compactor.compaction-retries=123", + "-compactor.compaction-strategy=partitioning", + })) + + assert.Equal(t, cortex_tsdb.DurationList{2 * time.Hour, 48 * time.Hour}, cfg.BlockRanges) + assert.Equal(t, time.Hour, cfg.ConsistencyDelay) + assert.Equal(t, 123, cfg.BlockSyncConcurrency) + assert.Equal(t, "/tmp", cfg.DataDir) + assert.Equal(t, 15*time.Minute, cfg.CompactionInterval) + assert.Equal(t, 123, cfg.CompactionRetries) + assert.Equal(t, util.CompactionStrategyPartitioning, cfg.CompactionStrategy) +} + +func TestPartitionConfig_Validate(t *testing.T) { + tests := map[string]struct { + setup func(cfg *Config) + initLimits func(*validation.Limits) + expected string + }{ + "should pass with the default config": { + setup: func(cfg *Config) {}, + initLimits: func(_ *validation.Limits) {}, + expected: "", + }, + "should pass with only 1 block range period": { + setup: func(cfg *Config) { + cfg.BlockRanges = cortex_tsdb.DurationList{time.Hour} + }, + initLimits: func(_ *validation.Limits) {}, + expected: "", + }, + "should fail with non divisible block range periods": { + setup: func(cfg *Config) { + cfg.BlockRanges = cortex_tsdb.DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour, 30 * time.Hour} + }, + + initLimits: func(_ *validation.Limits) {}, + expected: errors.Errorf(errInvalidBlockRanges, 30*time.Hour, 24*time.Hour).Error(), + }, + "should fail with duration values of zero": { + setup: func(cfg *Config) { + cfg.BlockRanges = cortex_tsdb.DurationList{2 * time.Hour, 0, 24 * time.Hour, 30 * time.Hour} + }, + initLimits: func(_ *validation.Limits) {}, + expected: errors.Errorf("compactor block range period cannot be zero").Error(), + }, + "should pass with valid shuffle sharding config": { + setup: func(cfg *Config) { + cfg.ShardingStrategy = util.ShardingStrategyShuffle + cfg.ShardingEnabled = true + }, + initLimits: func(limits *validation.Limits) { + limits.CompactorTenantShardSize = 1 + }, + expected: "", + }, + "should fail with bad compactor tenant shard size": { + setup: func(cfg *Config) { + cfg.ShardingStrategy = util.ShardingStrategyShuffle + cfg.ShardingEnabled = true + }, + initLimits: func(_ *validation.Limits) {}, + expected: errInvalidTenantShardSize.Error(), + }, + "should pass with valid compaction strategy config": { + setup: func(cfg *Config) { + cfg.ShardingEnabled = true + cfg.CompactionStrategy = util.CompactionStrategyPartitioning + }, + initLimits: func(_ *validation.Limits) {}, + expected: "", + }, + "should fail with bad compaction strategy": { + setup: func(cfg *Config) { + cfg.CompactionStrategy = "dummy" + }, + initLimits: func(_ *validation.Limits) {}, + expected: errInvalidCompactionStrategy.Error(), + }, + "should fail with partitioning compaction strategy but sharding disabled": { + setup: func(cfg *Config) { + cfg.ShardingEnabled = false + cfg.CompactionStrategy = util.CompactionStrategyPartitioning + }, + initLimits: func(_ *validation.Limits) {}, + expected: errInvalidCompactionStrategyPartitioning.Error(), + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + cfg := &Config{} + limits := validation.Limits{} + flagext.DefaultValues(cfg, &limits) + testData.setup(cfg) + testData.initLimits(&limits) + + if actualErr := cfg.Validate(limits); testData.expected != "" { + assert.EqualError(t, actualErr, testData.expected) + } else { + assert.NoError(t, actualErr) + } + }) + } +} + +func TestPartitionCompactor_SkipCompactionWhenCmkError(t *testing.T) { + t.Parallel() + userID := "user-1" + + ss := bucketindex.Status{Status: bucketindex.CustomerManagedKeyError, Version: bucketindex.SyncStatusFileVersion} + content, err := json.Marshal(ss) + require.NoError(t, err) + + // No user blocks stored in the bucket. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{userID}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockIter(userID+"/", []string{}, nil) + bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockGet(userID+"/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockGet(userID+"/bucket-index-sync-status.json", string(content), nil) + bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/bucket-index-sync-status.json", nil) + bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath(userID), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath(userID), false, nil) + bucketClient.MockIter(userID+"/"+PartitionedGroupDirectory, nil, nil) + + cfg := prepareConfigForPartitioning() + c, _, _, logs, _ := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + assert.Contains(t, strings.Split(strings.TrimSpace(logs.String()), "\n"), `level=info component=compactor msg="skipping compactUser due CustomerManagedKeyError" user=user-1`) +} + +func TestPartitionCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { + t.Parallel() + + // No user blocks stored in the bucket. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + cfg := prepareConfigForPartitioning() + c, _, _, logs, registry := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + assert.Equal(t, prom_testutil.ToFloat64(c.CompactionRunInterval), cfg.CompactionInterval.Seconds()) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="compactor started"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=0`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total{user_status="active"} 1 + cortex_compactor_block_cleanup_started_total{user_status="deleted"} 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_completed_total{user_status="deleted"} 1 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_garbage_collected_blocks_total", + "cortex_compactor_block_cleanup_failures_total", + "cortex_compactor_blocks_cleaned_total", + "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_blocks_marked_for_no_compaction_total", + "cortex_compactor_block_cleanup_started_total", + "cortex_compactor_block_cleanup_completed_total", + "cortex_compactor_block_cleanup_failed_total", + )) +} + +func TestPartitionCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket(t *testing.T) { + t.Parallel() + + // Fail to iterate over the bucket while discovering users. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("__markers__", nil, errors.New("failed to iterate the bucket")) + bucketClient.MockIter("", nil, errors.New("failed to iterate the bucket")) + + c, _, _, logs, registry := prepareForPartitioning(t, prepareConfigForPartitioning(), bucketClient, nil, nil) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until all retry attempts have completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsFailed) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Ensure the bucket iteration has been retried the configured number of times. + bucketClient.AssertNumberOfCalls(t, "Iter", 1+3) + + assert.ElementsMatch(t, []string{ + `level=error component=cleaner msg="failed to scan users on startup" err="failed to discover users from bucket: failed to iterate the bucket"`, + `level=info component=compactor msg="compactor started"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=error component=compactor msg="failed to discover users from bucket" err="failed to iterate the bucket"`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 0 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total 1 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_failed_total{user_status="deleted"} 1 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total 0 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_garbage_collected_blocks_total", + "cortex_compactor_block_cleanup_failures_total", + "cortex_compactor_blocks_cleaned_total", + "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_blocks_marked_for_no_compaction_total", + "cortex_compactor_block_cleanup_started_total", + "cortex_compactor_block_cleanup_completed_total", + "cortex_compactor_block_cleanup_failed_total", + )) +} + +func TestPartitionCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASingleTenant(t *testing.T) { + t.Parallel() + + userID := "test-user" + partitionedGroupID := getPartitionedGroupID(userID) + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{userID}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D/meta.json", userID + "/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) + bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockGet(userID+"/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath(userID), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath(userID), false, nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", nil) + bucketClient.MockGet(userID+"/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json", mockBlockMetaJSON("01FN6CDF3PNEWWRY5MPGJPE3EX"), nil) + bucketClient.MockGet(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/partition-0-visit-mark.json", nil) + bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + bucketClient.MockUpload(userID+"/bucket-index-sync-status.json", nil) + bucketClient.MockGet(userID+"/partitioned-groups/"+partitionedGroupID+".json", "", nil) + bucketClient.MockUpload(userID+"/partitioned-groups/"+partitionedGroupID+".json", nil) + bucketClient.MockIter(userID+"/"+PartitionedGroupDirectory, nil, nil) + + c, _, tsdbPlannerMock, _, registry := prepareForPartitioning(t, prepareConfigForPartitioning(), bucketClient, nil, nil) + tsdbPlannerMock.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, errors.New("Failed to plan")) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until all retry attempts have completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsFailed) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 0 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total 1 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + )) +} + +func TestPartitionCompactor_ShouldCompactAndRemoveUserFolder(t *testing.T) { + partitionedGroupID1 := getPartitionedGroupID("user-1") + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json", mockBlockMetaJSON("01FN6CDF3PNEWWRY5MPGJPE3EX"), nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockGet("user-1/partitioned-groups/"+partitionedGroupID1+".json", "", nil) + bucketClient.MockUpload("user-1/partitioned-groups/"+partitionedGroupID1+".json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + + c, _, tsdbPlanner, _, _ := prepareForPartitioning(t, prepareConfigForPartitioning(), bucketClient, nil, nil) + + // Make sure the user folder is created and is being used + // This will be called during compaction + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Run(func(args mock.Arguments) { + _, err := os.Stat(c.compactDirForUser("user-1")) + require.NoError(t, err) + }).Return([]*metadata.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + _, err := os.Stat(c.compactDirForUser("user-1")) + require.True(t, os.IsNotExist(err)) +} + +func TestPartitionCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { + t.Parallel() + + partitionedGroupID1 := getPartitionedGroupID("user-1") + partitionedGroupID2 := getPartitionedGroupID("user-2") + + // Mock the bucket to contain two users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-2"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-2"), false, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) + bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", "user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockGet("user-2/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json", mockBlockMetaJSON("01FN6CDF3PNEWWRY5MPGJPE3EX"), nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json", mockBlockMetaJSON("01FN3V83ABR9992RF8WRJZ76ZQ"), nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet("user-2/bucket-index-sync-status.json", "", nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-2/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockUpload("user-2/bucket-index-sync-status.json", nil) + bucketClient.MockGet("user-1/partitioned-groups/"+partitionedGroupID1+".json", "", nil) + bucketClient.MockUpload("user-1/partitioned-groups/"+partitionedGroupID1+".json", nil) + bucketClient.MockGet("user-2/partitioned-groups/"+partitionedGroupID2+".json", "", nil) + bucketClient.MockUpload("user-2/partitioned-groups/"+partitionedGroupID2+".json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + bucketClient.MockIter("user-2/"+PartitionedGroupDirectory, nil, nil) + + c, _, tsdbPlanner, logs, registry := prepareForPartitioning(t, prepareConfigForPartitioning(), bucketClient, nil, nil) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Ensure a plan has been executed for the blocks of each user. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 2) + + assert.Len(t, tsdbPlanner.getNoCompactBlocks(), 0) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="compactor started"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=2`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-2`, + `level=info component=compactor org_id=user-2 msg="start sync of metas"`, + `level=info component=compactor org_id=user-2 msg="start of GC"`, + `level=info component=compactor org_id=user-2 msg="start of compactions"`, + `level=info component=compactor org_id=user-2 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-2`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor org_id=user-1 msg="start sync of metas"`, + `level=info component=compactor org_id=user-1 msg="start of GC"`, + `level=info component=compactor org_id=user-1 msg="start of compactions"`, + `level=info component=compactor org_id=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + // Instead of testing for shipper metrics, we only check our metrics here. + // Real shipper metrics are too variable to embed into a test. + testedMetrics := []string{ + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "cortex_compactor_blocks_marked_for_no_compaction_total", + } + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-1"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-2"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-2"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total{user_status="active"} 1 + cortex_compactor_block_cleanup_started_total{user_status="deleted"} 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_completed_total{user_status="deleted"} 1 + + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total 0 + `), testedMetrics...)) +} + +func TestPartitionCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { + t.Parallel() + + cfg := prepareConfigForPartitioning() + cfg.DeletionDelay = 10 * time.Minute // Delete block after 10 minutes + + // Mock the bucket to contain two users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + + // Block that has just been marked for deletion. It will not be deleted just yet, and it also will not be compacted. + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", mockDeletionMarkJSON("01DTVP434PA9VFXSW2JKB3392D", time.Now()), nil) + bucketClient.MockGet("user-1/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", mockDeletionMarkJSON("01DTVP434PA9VFXSW2JKB3392D", time.Now()), nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + + // This block will be deleted by cleaner. + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", mockDeletionMarkJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ", time.Now().Add(-cfg.DeletionDelay)), nil) + bucketClient.MockGet("user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", mockDeletionMarkJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ", time.Now().Add(-cfg.DeletionDelay)), nil) + + bucketClient.MockIter("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", []string{ + "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", + "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", + }, nil) + + bucketClient.MockIter("user-1/markers/", []string{ + "user-1/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", + "user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", + }, nil) + + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + + bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", nil) + bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", nil) + bucketClient.MockDelete("user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", nil) + bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + + c, _, tsdbPlanner, logs, registry := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Since both blocks are marked for deletion, none of them are going to be compacted. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 0) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="compactor started"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=1`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor org_id=user-1 msg="start sync of metas"`, + `level=info component=compactor org_id=user-1 msg="start of GC"`, + `level=info component=compactor org_id=user-1 msg="start of compactions"`, + `level=info component=compactor org_id=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + // Instead of testing for shipper metrics, we only check our metrics here. + // Real shipper metrics are too variable to embed into a test. + testedMetrics := []string{ + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "cortex_compactor_blocks_marked_for_no_compaction_total", + } + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 1 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-1"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total{user_status="active"} 1 + cortex_compactor_block_cleanup_started_total{user_status="deleted"} 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_completed_total{user_status="deleted"} 1 + + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total 0 + `), testedMetrics...)) +} + +func TestPartitionCompactor_ShouldNotCompactBlocksMarkedForSkipCompact(t *testing.T) { + t.Parallel() + + partitionedGroupID1 := getPartitionedGroupID("user-1") + partitionedGroupID2 := getPartitionedGroupID("user-2") + // Mock the bucket to contain two users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-2"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-2"), false, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) + bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", "user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockGet("user-2/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", mockNoCompactBlockJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json", mockBlockMetaJSON("01FN6CDF3PNEWWRY5MPGJPE3EX"), nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/no-compact-mark.json", mockNoCompactBlockJSON("01FN6CDF3PNEWWRY5MPGJPE3EX"), nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/partition-0-visit-mark.json", nil) + + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/partition-0-visit-mark.json", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json", mockBlockMetaJSON("01FN3V83ABR9992RF8WRJZ76ZQ"), nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/partition-0-visit-mark.json", nil) + + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet("user-2/bucket-index-sync-status.json", "", nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-2/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockUpload("user-2/bucket-index-sync-status.json", nil) + bucketClient.MockGet("user-1/partitioned-groups/"+partitionedGroupID1+".json", "", nil) + bucketClient.MockUpload("user-1/partitioned-groups/"+partitionedGroupID1+".json", nil) + bucketClient.MockGet("user-2/partitioned-groups/"+partitionedGroupID2+".json", "", nil) + bucketClient.MockUpload("user-2/partitioned-groups/"+partitionedGroupID2+".json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + bucketClient.MockIter("user-2/"+PartitionedGroupDirectory, nil, nil) + + c, _, tsdbPlanner, _, registry := prepareForPartitioning(t, prepareConfigForPartitioning(), bucketClient, nil, nil) + + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Planner still called for user with all blocks makred for skip compaction. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 2) + + assert.ElementsMatch(t, []string{"01DTVP434PA9VFXSW2JKB3392D", "01FN6CDF3PNEWWRY5MPGJPE3EX"}, tsdbPlanner.getNoCompactBlocks()) + + testedMetrics := []string{"cortex_compactor_blocks_marked_for_no_compaction_total"} + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total 0 + `), testedMetrics...)) +} + +func TestPartitionCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) { + t.Parallel() + + cfg := prepareConfigForPartitioning() + cfg.DeletionDelay = 10 * time.Minute // Delete block after 10 minutes + cfg.TenantCleanupDelay = 10 * time.Minute // To make sure it's not 0. + + partitionedGroupID1 := getPartitionedGroupID("user-1") + // Mock the bucket to contain two users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("__markers__", []string{"__markers__/user-1/"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) + bucketClient.MockGet(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), `{"deletion_time": 1}`, nil) + bucketClient.MockUpload(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), nil) + + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + + bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTVP434PA9VFXSW2JKB3392D/index"}, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/index", "some index content", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", nil) + bucketClient.MockExists("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", false, nil) + + bucketClient.MockDelete("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", nil) + bucketClient.MockDelete("user-1/01DTVP434PA9VFXSW2JKB3392D/index", nil) + bucketClient.MockDelete("user-1/bucket-index.json.gz", nil) + bucketClient.MockDelete("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockGet("user-1/partitioned-groups/"+partitionedGroupID1+".json", "", nil) + bucketClient.MockUpload("user-1/partitioned-groups/"+partitionedGroupID1+".json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + + c, _, tsdbPlanner, logs, registry := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // No user is compacted, single user we have is marked for deletion. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 0) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="compactor started"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=1`, + `level=debug component=compactor msg="skipping user because it is marked for deletion" user=user-1`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + // Instead of testing for shipper metrics, we only check our metrics here. + // Real shipper metrics are too variable to embed into a test. + testedMetrics := []string{ + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", "cortex_bucket_index_last_successful_update_timestamp_seconds", + "cortex_compactor_blocks_marked_for_no_compaction_total", + } + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 1 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total{user_status="active"} 1 + cortex_compactor_block_cleanup_started_total{user_status="deleted"} 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_completed_total{user_status="deleted"} 1 + + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total 0 + `), testedMetrics...)) +} + +func TestPartitionCompactor_ShouldSkipOutOrOrderBlocks(t *testing.T) { + bucketClient, tmpDir := cortex_storage_testutil.PrepareFilesystemBucket(t) + bucketClient = bucketindex.BucketWithGlobalMarkers(bucketClient) + + b1 := createTSDBBlock(t, bucketClient, "user-1", 10, 20, map[string]string{"__name__": "Teste"}) + b2 := createTSDBBlock(t, bucketClient, "user-1", 20, 30, map[string]string{"__name__": "Teste"}) + + // Read bad index file. + indexFile, err := os.Open("testdata/out_of_order_chunks/index") + require.NoError(t, err) + indexFileStat, err := indexFile.Stat() + require.NoError(t, err) + + dir := path.Join(tmpDir, "user-1", b1.String()) + outputFile, err := os.OpenFile(path.Join(dir, "index"), os.O_RDWR|os.O_TRUNC, 0755) + require.NoError(t, err) + + n, err := io.Copy(outputFile, indexFile) + require.NoError(t, err) + require.Equal(t, indexFileStat.Size(), n) + + cfg := prepareConfigForPartitioning() + cfg.SkipBlocksWithOutOfOrderChunksEnabled = true + c, tsdbCompac, tsdbPlanner, _, registry := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + + tsdbCompac.On("CompactWithBlockPopulator", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(b1, nil) + + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + ULID: b1, + MinTime: 10, + MaxTime: 20, + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + ULID: b2, + MinTime: 20, + MaxTime: 30, + }, + }, + }, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + defer services.StopAndAwaitTerminated(context.Background(), c) //nolint:errcheck + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, true, func() interface{} { + if _, err := os.Stat(path.Join(dir, "no-compact-mark.json")); err == nil { + return true + } + return false + }) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total 1 + `), "cortex_compactor_blocks_marked_for_no_compaction_total")) +} + +func TestPartitionCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunning(t *testing.T) { + t.Parallel() + + partitionedGroupID1 := getPartitionedGroupID("user-1") + partitionedGroupID2 := getPartitionedGroupID("user-2") + // Mock the bucket to contain two users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-2"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-2"), false, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) + bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", "user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockGet("user-2/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json", mockBlockMetaJSON("01FN6CDF3PNEWWRY5MPGJPE3EX"), nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/partition-0-visit-mark.json", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/partition-0-visit-mark.json", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json", mockBlockMetaJSON("01FN3V83ABR9992RF8WRJZ76ZQ"), nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/partition-0-visit-mark.json", "", nil) + bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/partition-0-visit-mark.json", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet("user-2/bucket-index-sync-status.json", "", nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-2/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockUpload("user-2/bucket-index-sync-status.json", nil) + bucketClient.MockGet("user-1/partitioned-groups/"+partitionedGroupID1+".json", "", nil) + bucketClient.MockUpload("user-1/partitioned-groups/"+partitionedGroupID1+".json", nil) + bucketClient.MockGet("user-2/partitioned-groups/"+partitionedGroupID2+".json", "", nil) + bucketClient.MockUpload("user-2/partitioned-groups/"+partitionedGroupID2+".json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + bucketClient.MockIter("user-2/"+PartitionedGroupDirectory, nil, nil) + + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + cfg := prepareConfigForPartitioning() + cfg.ShardingEnabled = true + cfg.ShardingRing.InstanceID = "compactor-1" + cfg.ShardingRing.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.KVStore.Mock = ringStore + + c, _, tsdbPlanner, logs, _ := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Ensure a plan has been executed for the blocks of each user. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 2) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="auto joined with new tokens" ring=compactor state=ACTIVE`, + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor started"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=2`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor org_id=user-1 msg="start sync of metas"`, + `level=info component=compactor org_id=user-1 msg="start of GC"`, + `level=info component=compactor org_id=user-1 msg="start of compactions"`, + `level=info component=compactor org_id=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-2`, + `level=info component=compactor org_id=user-2 msg="start sync of metas"`, + `level=info component=compactor org_id=user-2 msg="start of GC"`, + `level=info component=compactor org_id=user-2 msg="start of compactions"`, + `level=info component=compactor org_id=user-2 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-2`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) +} + +func TestPartitionCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndMultipleInstancesRunning(t *testing.T) { + + numUsers := 100 + + // Setup user IDs + userIDs := make([]string, 0, numUsers) + for i := 1; i <= numUsers; i++ { + userIDs = append(userIDs, fmt.Sprintf("user-%d", i)) + } + + // Mock the bucket to contain all users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", userIDs, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + for _, userID := range userIDs { + partitionedGroupID := getPartitionedGroupID(userID) + bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D"}, nil) + bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockGet(userID+"/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath(userID), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath(userID), false, nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet(userID+"/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + bucketClient.MockUpload(userID+"/bucket-index-sync-status.json", nil) + bucketClient.MockGet(userID+"/partitioned-groups/"+partitionedGroupID+".json", "", nil) + bucketClient.MockUpload(userID+"/partitioned-groups/"+partitionedGroupID+".json", nil) + bucketClient.MockIter(userID+"/"+PartitionedGroupDirectory, nil, nil) + } + + // Create a shared KV Store + kvstore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + // Create two compactors + var compactors []*Compactor + var logs []*concurrency.SyncBuffer + + for i := 1; i <= 2; i++ { + cfg := prepareConfigForPartitioning() + cfg.ShardingEnabled = true + cfg.ShardingRing.InstanceID = fmt.Sprintf("compactor-%d", i) + cfg.ShardingRing.InstanceAddr = fmt.Sprintf("127.0.0.%d", i) + cfg.ShardingRing.WaitStabilityMinDuration = 3 * time.Second + cfg.ShardingRing.WaitStabilityMaxDuration = 10 * time.Second + cfg.ShardingRing.KVStore.Mock = kvstore + + c, _, tsdbPlanner, l, _ := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + defer services.StopAndAwaitTerminated(context.Background(), c) //nolint:errcheck + + compactors = append(compactors, c) + logs = append(logs, l) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + } + + // Start all compactors + for _, c := range compactors { + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + } + + // Wait until a run has been completed on each compactor + for _, c := range compactors { + cortex_testutil.Poll(t, 60*time.Second, true, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) >= 1 + }) + } + + // Ensure that each user has been compacted by the correct instance + for _, userID := range userIDs { + _, l, err := findCompactorByUserID(compactors, logs, userID) + require.NoError(t, err) + assert.Contains(t, l.String(), fmt.Sprintf(`level=info component=compactor msg="successfully compacted user blocks" user=%s`, userID)) + } +} + +func TestPartitionCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWithShuffleShardingAndMultipleInstancesRunning(t *testing.T) { + t.Parallel() + + numUsers := 3 + + // Setup user IDs + userIDs := make([]string, 0, numUsers) + for i := 1; i <= numUsers; i++ { + userIDs = append(userIDs, fmt.Sprintf("user-%d", i)) + } + + startTime := int64(1574776800000) + // Define blocks mapping block IDs to start and end times + blocks := map[string]map[string]int64{ + "01DTVP434PA9VFXSW2JKB3392D": { + "startTime": startTime, + "endTime": startTime + time.Hour.Milliseconds()*2, + }, + "01DTVP434PA9VFXSW2JKB3392E": { + "startTime": startTime, + "endTime": startTime + time.Hour.Milliseconds()*2, + }, + "01DTVP434PA9VFXSW2JKB3392F": { + "startTime": startTime + time.Hour.Milliseconds()*2, + "endTime": startTime + time.Hour.Milliseconds()*4, + }, + "01DTVP434PA9VFXSW2JKB3392G": { + "startTime": startTime + time.Hour.Milliseconds()*2, + "endTime": startTime + time.Hour.Milliseconds()*4, + }, + // Add another new block as the final block so that the previous groups will be planned for compaction + "01DTVP434PA9VFXSW2JKB3392H": { + "startTime": startTime + time.Hour.Milliseconds()*4, + "endTime": startTime + time.Hour.Milliseconds()*6, + }, + } + + // Mock the bucket to contain all users, each one with five blocks, 2 sets of overlapping blocks and 1 separate block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", userIDs, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + + // Keys with a value greater than 1 will be groups that should be compacted + groupHashes := make(map[uint32]int) + for _, userID := range userIDs { + blockFiles := []string{} + + for blockID, blockTimes := range blocks { + groupHash := hashGroup(userID, blockTimes["startTime"], blockTimes["endTime"]) + visitMarker := partitionVisitMarker{ + CompactorID: "test-compactor", + VisitTime: time.Now().Unix(), + PartitionedGroupID: groupHash, + PartitionID: 0, + Status: Pending, + Version: PartitionVisitMarkerVersion1, + } + visitMarkerFileContent, _ := json.Marshal(visitMarker) + bucketClient.MockGet(userID+"/bucket-index-sync-status.json", "", nil) + bucketClient.MockGet(userID+"/"+blockID+"/meta.json", mockBlockMetaJSONWithTime(blockID, userID, blockTimes["startTime"], blockTimes["endTime"]), nil) + bucketClient.MockGet(userID+"/"+blockID+"/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/"+blockID+"/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/"+blockID+"/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet(userID+"/partitioned-groups/visit-marks/"+fmt.Sprint(groupHash)+"/partition-0-visit-mark.json", string(visitMarkerFileContent), nil) + bucketClient.MockGetRequireUpload(userID+"/partitioned-groups/visit-marks/"+fmt.Sprint(groupHash)+"/partition-0-visit-mark.json", string(visitMarkerFileContent), nil) + bucketClient.MockUpload(userID+"/partitioned-groups/visit-marks/"+fmt.Sprint(groupHash)+"/partition-0-visit-mark.json", nil) + // Iter with recursive so expected to get objects rather than directories. + blockFiles = append(blockFiles, path.Join(userID, blockID, block.MetaFilename)) + + // Get all of the unique group hashes so that they can be used to ensure all groups were compacted + groupHashes[groupHash]++ + bucketClient.MockGet(userID+"/partitioned-groups/"+fmt.Sprint(groupHash)+".json", "", nil) + bucketClient.MockUpload(userID+"/partitioned-groups/"+fmt.Sprint(groupHash)+".json", nil) + } + + bucketClient.MockIter(userID+"/", blockFiles, nil) + bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockGet(userID+"/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath(userID), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath(userID), false, nil) + bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + bucketClient.MockUpload(userID+"/bucket-index-sync-status.json", nil) + bucketClient.MockIter(userID+"/"+PartitionedGroupDirectory, nil, nil) + } + + // Create a shared KV Store + kvstore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + // Create four compactors + var compactors []*Compactor + var logs []*concurrency.SyncBuffer + + for i := 1; i <= 4; i++ { + cfg := prepareConfigForPartitioning() + cfg.ShardingEnabled = true + cfg.CompactionInterval = 15 * time.Second + cfg.ShardingStrategy = util.ShardingStrategyShuffle + cfg.ShardingRing.InstanceID = fmt.Sprintf("compactor-%d", i) + cfg.ShardingRing.InstanceAddr = fmt.Sprintf("127.0.0.%d", i) + cfg.ShardingRing.WaitStabilityMinDuration = 3 * time.Second + cfg.ShardingRing.WaitStabilityMaxDuration = 10 * time.Second + cfg.ShardingRing.KVStore.Mock = kvstore + + limits := &validation.Limits{} + flagext.DefaultValues(limits) + limits.CompactorTenantShardSize = 3 + + c, _, tsdbPlanner, l, _ := prepareForPartitioning(t, cfg, bucketClient, limits, nil) + defer services.StopAndAwaitTerminated(context.Background(), c) //nolint:errcheck + + compactors = append(compactors, c) + logs = append(logs, l) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + } + + // Start all compactors + for _, c := range compactors { + require.NoError(t, c.StartAsync(context.Background())) + } + // Wait for all the compactors to get into the Running state without errors. + // Cannot use StartAndAwaitRunning as this would cause the compactions to start before + // all the compactors are initialized + for _, c := range compactors { + require.NoError(t, c.AwaitRunning(context.Background())) + } + + // Wait until a run has been completed on each compactor + for _, c := range compactors { + cortex_testutil.Poll(t, 60*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + } + + // Ensure that each group was only compacted by exactly one compactor + for groupHash, blockCount := range groupHashes { + + l, found, err := checkLogsForPartitionCompaction(compactors, logs, groupHash) + require.NoError(t, err) + + // If the blockCount < 2 then the group shouldn't have been compacted, therefore not found in the logs + if blockCount < 2 { + assert.False(t, found) + } else { + assert.True(t, found) + assert.Contains(t, l.String(), fmt.Sprintf(`group_hash=%d msg="found compactable group for user"`, groupHash)) + } + } +} + +// checkLogsForPartitionCompaction checks the logs to see if a compaction has happened on the groupHash, +// if there has been a compaction it will return the logs of the compactor that handled the group +// and will return true. Otherwise this function will return a nil value for the logs and false +// as the group was not compacted +func checkLogsForPartitionCompaction(compactors []*Compactor, logs []*concurrency.SyncBuffer, groupHash uint32) (*concurrency.SyncBuffer, bool, error) { + var log *concurrency.SyncBuffer + + for _, l := range logs { + owned := strings.Contains(l.String(), fmt.Sprintf(`group_hash=%d msg="found compactable group for user"`, groupHash)) + if owned { + log = l + } + } + + // Return false if we've not been able to find it + if log == nil { + return nil, false, nil + } + + return log, true, nil +} + +func prepareConfigForPartitioning() Config { + compactorCfg := prepareConfig() + + compactorCfg.CompactionStrategy = util.CompactionStrategyPartitioning + + return compactorCfg +} + +func prepareForPartitioning(t *testing.T, compactorCfg Config, bucketClient objstore.InstrumentedBucket, limits *validation.Limits, tsdbGrouper *tsdbGrouperMock) (*Compactor, *tsdbCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { + storageCfg := cortex_tsdb.BlocksStorageConfig{} + flagext.DefaultValues(&storageCfg) + storageCfg.BucketStore.BlockDiscoveryStrategy = string(cortex_tsdb.RecursiveDiscovery) + + // Create a temporary directory for compactor data. + compactorCfg.DataDir = t.TempDir() + + tsdbCompactor := &tsdbCompactorMock{} + tsdbPlanner := &tsdbPlannerMock{ + noCompactMarkFilters: []*compact.GatherNoCompactionMarkFilter{}, + } + logs := &concurrency.SyncBuffer{} + logger := log.NewLogfmtLogger(logs) + registry := prometheus.NewRegistry() + + if limits == nil { + limits = &validation.Limits{} + flagext.DefaultValues(limits) + } + + overrides, err := validation.NewOverrides(*limits, nil) + require.NoError(t, err) + + bucketClientFactory := func(ctx context.Context) (objstore.InstrumentedBucket, error) { + return bucketClient, nil + } + + blocksCompactorFactory := func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { + return tsdbCompactor, + func(ctx context.Context, bkt objstore.InstrumentedBucket, _ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ string, _ prometheus.Counter, _ prometheus.Counter, _ *compactorMetrics) compact.Planner { + tsdbPlanner.noCompactMarkFilters = append(tsdbPlanner.noCompactMarkFilters, noCompactMarkFilter) + return tsdbPlanner + }, + nil + } + + var blocksGrouperFactory BlocksGrouperFactory + if tsdbGrouper != nil { + blocksGrouperFactory = func(_ context.Context, _ Config, _ objstore.InstrumentedBucket, _ log.Logger, _ prometheus.Counter, _ prometheus.Counter, _ prometheus.Counter, _ *compact.SyncerMetrics, _ *compactorMetrics, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string, _ *compact.GatherNoCompactionMarkFilter, _ int) compact.Grouper { + return tsdbGrouper + } + } else { + if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle { + blocksGrouperFactory = ShuffleShardingGrouperFactory + } else { + blocksGrouperFactory = DefaultBlocksGrouperFactory + } + } + + var blockDeletableCheckerFactory BlockDeletableCheckerFactory + if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle { + blockDeletableCheckerFactory = PartitionCompactionBlockDeletableCheckerFactory + } else { + blockDeletableCheckerFactory = DefaultBlockDeletableCheckerFactory + } + + var compactionLifecycleCallbackFactory CompactionLifecycleCallbackFactory + if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle { + compactionLifecycleCallbackFactory = ShardedCompactionLifecycleCallbackFactory + } else { + compactionLifecycleCallbackFactory = DefaultCompactionLifecycleCallbackFactory + } + + c, err := newCompactor(compactorCfg, storageCfg, logger, registry, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, blockDeletableCheckerFactory, compactionLifecycleCallbackFactory, overrides, 1) + require.NoError(t, err) + + return c, tsdbCompactor, tsdbPlanner, logs, registry +} + +type tsdbGrouperMock struct { + mock.Mock +} + +func (m *tsdbGrouperMock) Groups(blocks map[ulid.ULID]*metadata.Meta) (res []*compact.Group, err error) { + args := m.Called(blocks) + return args.Get(0).([]*compact.Group), args.Error(1) +} + +var ( + BlockMinTime = int64(1574776800000) + BlockMaxTime = int64(1574784000000) +) + +func getPartitionedGroupID(userID string) string { + return fmt.Sprint(hashGroup(userID, BlockMinTime, BlockMaxTime)) +} + +func mockBlockGroup(userID string, ids []string, bkt *bucket.ClientMock) *compact.Group { + dummyCounter := prometheus.NewCounter(prometheus.CounterOpts{}) + group, _ := compact.NewGroup( + log.NewNopLogger(), + bkt, + getPartitionedGroupID(userID), + nil, + 0, + true, + true, + dummyCounter, + dummyCounter, + dummyCounter, + dummyCounter, + dummyCounter, + dummyCounter, + dummyCounter, + dummyCounter, + metadata.NoneFunc, + 1, + 1, + ) + for _, id := range ids { + meta := mockBlockMeta(id) + err := group.AppendMeta(&metadata.Meta{ + BlockMeta: meta, + }) + if err != nil { + continue + } + } + return group +} + +func TestPartitionCompactor_DeleteLocalSyncFiles(t *testing.T) { + numUsers := 10 + + // Setup user IDs + userIDs := make([]string, 0, numUsers) + for i := 1; i <= numUsers; i++ { + userIDs = append(userIDs, fmt.Sprintf("user-%d", i)) + } + + inmem := objstore.WithNoopInstr(objstore.NewInMemBucket()) + for _, userID := range userIDs { + id, err := ulid.New(ulid.Now(), rand.Reader) + require.NoError(t, err) + require.NoError(t, inmem.Upload(context.Background(), userID+"/"+id.String()+"/meta.json", strings.NewReader(mockBlockMetaJSON(id.String())))) + } + + // Create a shared KV Store + kvstore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + // Create two compactors + var compactors []*Compactor + + for i := 1; i <= 2; i++ { + cfg := prepareConfigForPartitioning() + + cfg.ShardingEnabled = true + cfg.ShardingRing.InstanceID = fmt.Sprintf("compactor-%d", i) + cfg.ShardingRing.InstanceAddr = fmt.Sprintf("127.0.0.%d", i) + cfg.ShardingRing.WaitStabilityMinDuration = 3 * time.Second + cfg.ShardingRing.WaitStabilityMaxDuration = 10 * time.Second + cfg.ShardingRing.KVStore.Mock = kvstore + + // Each compactor will get its own temp dir for storing local files. + c, _, tsdbPlanner, _, _ := prepareForPartitioning(t, cfg, inmem, nil, nil) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + }) + + compactors = append(compactors, c) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + } + + require.Equal(t, 2, len(compactors)) + c1 := compactors[0] + c2 := compactors[1] + + // Start first compactor + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c1)) + + // Wait until a run has been completed on first compactor. This happens as soon as compactor starts. + cortex_testutil.Poll(t, 20*time.Second, true, func() interface{} { + return prom_testutil.ToFloat64(c1.CompactionRunsCompleted) >= 1 + }) + + require.NoError(t, os.Mkdir(c1.metaSyncDirForUser("new-user"), 0600)) + + // Verify that first compactor has synced all the users, plus there is one extra we have just created. + require.Equal(t, numUsers+1, len(c1.listTenantsWithMetaSyncDirectories())) + + // Now start second compactor, and wait until it runs compaction. + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c2)) + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c2.CompactionRunsCompleted) + }) + + // Let's check how many users second compactor has. + c2Users := len(c2.listTenantsWithMetaSyncDirectories()) + require.NotZero(t, c2Users) + + // Force new compaction cycle on first compactor. It will run the cleanup of un-owned users at the end of compaction cycle. + c1.compactUsers(context.Background()) + c1Users := len(c1.listTenantsWithMetaSyncDirectories()) + + // Now compactor 1 should have cleaned old sync files. + require.NotEqual(t, numUsers, c1Users) + require.Equal(t, numUsers, c1Users+c2Users) +} + +func TestPartitionCompactor_ShouldFailCompactionOnTimeout(t *testing.T) { + t.Parallel() + + // Mock the bucket + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{}, nil) + bucketClient.MockIter("__markers__", []string{}, nil) + + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + cfg := prepareConfigForPartitioning() + cfg.ShardingEnabled = true + cfg.ShardingRing.InstanceID = "compactor-1" + cfg.ShardingRing.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.KVStore.Mock = ringStore + + // Set ObservePeriod to longer than the timeout period to mock a timeout while waiting on ring to become ACTIVE + cfg.ShardingRing.ObservePeriod = time.Second * 10 + + c, _, _, logs, _ := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + + // Try to start the compactor with a bad consul kv-store. The + err := services.StartAndAwaitRunning(context.Background(), c) + + // Assert that the compactor timed out + assert.Equal(t, context.DeadlineExceeded, err) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="compactor started"`, + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="auto joined with new tokens" ring=compactor state=JOINING`, + `level=error component=compactor msg="compactor failed to become ACTIVE in the ring" err="context deadline exceeded"`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) +} + +func TestPartitionCompactor_ShouldNotHangIfPlannerReturnNothing(t *testing.T) { + t.Parallel() + + ss := bucketindex.Status{Status: bucketindex.CustomerManagedKeyError, Version: bucketindex.SyncStatusFileVersion} + content, err := json.Marshal(ss) + require.NoError(t, err) + + partitionedGroupID := getPartitionedGroupID("user-1") + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/partition-0-visit-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", string(content), nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + bucketClient.MockGet("user-1/partitioned-groups/visit-marks/"+string(partitionedGroupID)+"/partition-0-visit-mark.json", "", nil) + + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + cfg := prepareConfigForPartitioning() + cfg.ShardingEnabled = true + cfg.ShardingRing.InstanceID = "compactor-1" + cfg.ShardingRing.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.KVStore.Mock = ringStore + + tsdbGrouper := tsdbGrouperMock{} + mockGroups := []*compact.Group{mockBlockGroup("user-1", []string{"01DTVP434PA9VFXSW2JKB3392D", "01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, bucketClient)} + tsdbGrouper.On("Groups", mock.Anything).Return(mockGroups, nil) + + c, _, tsdbPlanner, _, _ := prepareForPartitioning(t, cfg, bucketClient, nil, &tsdbGrouper) + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) +} + +func TestPartitionCompactor_ShouldNotFailCompactionIfAccessDeniedErrDuringMetaSync(t *testing.T) { + t.Parallel() + + ss := bucketindex.Status{Status: bucketindex.Ok, Version: bucketindex.SyncStatusFileVersion} + content, err := json.Marshal(ss) + require.NoError(t, err) + + partitionedGroupID := getPartitionedGroupID("user-1") + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), bucket.ErrKeyPermissionDenied) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", bucket.ErrKeyPermissionDenied) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", bucket.ErrKeyPermissionDenied) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), bucket.ErrKeyPermissionDenied) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", bucket.ErrKeyPermissionDenied) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", bucket.ErrKeyPermissionDenied) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", string(content), nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + bucketClient.MockGet("user-1/partitioned-groups/visit-marks/"+string(partitionedGroupID)+"/partition-0-visit-mark.json", "", nil) + + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + cfg := prepareConfigForPartitioning() + cfg.ShardingEnabled = true + cfg.ShardingRing.InstanceID = "compactor-1" + cfg.ShardingRing.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.KVStore.Mock = ringStore + + c, _, tsdbPlanner, _, _ := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) +} + +func TestPartitionCompactor_ShouldNotFailCompactionIfAccessDeniedErrReturnedFromBucket(t *testing.T) { + t.Parallel() + + ss := bucketindex.Status{Status: bucketindex.Ok, Version: bucketindex.SyncStatusFileVersion} + content, err := json.Marshal(ss) + require.NoError(t, err) + + partitionedGroupID := getPartitionedGroupID("user-1") + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", string(content), nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + bucketClient.MockIter("user-1/"+PartitionedGroupDirectory, nil, nil) + bucketClient.MockGet("user-1/partitioned-groups/visit-marks/"+string(partitionedGroupID)+"/partition-0-visit-mark.json", "", nil) + + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + cfg := prepareConfigForPartitioning() + cfg.ShardingEnabled = true + cfg.ShardingRing.InstanceID = "compactor-1" + cfg.ShardingRing.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.KVStore.Mock = ringStore + + c, _, tsdbPlanner, _, _ := prepareForPartitioning(t, cfg, bucketClient, nil, nil) + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, bucket.ErrKeyPermissionDenied) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a run has completed. + cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) +} diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index fd67e6b650..27cdc311de 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1135,8 +1135,8 @@ func TestCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndM // Wait until a run has been completed on each compactor for _, c := range compactors { - cortex_testutil.Poll(t, 10*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + cortex_testutil.Poll(t, 120*time.Second, true, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) >= 1 }) } @@ -1521,6 +1521,9 @@ func prepareConfig() Config { compactorCfg.retryMinBackoff = 0 compactorCfg.retryMaxBackoff = 0 + //Avoid jitter in startup + compactorCfg.CompactionInterval = 1 * time.Second + // The migration is tested in a dedicated test. compactorCfg.BlockDeletionMarksMigrationEnabled = false @@ -1578,7 +1581,7 @@ func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Instrument blocksGrouperFactory = DefaultBlocksGrouperFactory } - c, err := newCompactor(compactorCfg, storageCfg, logger, registry, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, overrides, 1) + c, err := newCompactor(compactorCfg, storageCfg, logger, registry, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, DefaultBlockDeletableCheckerFactory, DefaultCompactionLifecycleCallbackFactory, overrides, 1) require.NoError(t, err) return c, tsdbCompactor, tsdbPlanner, logs, registry @@ -1748,7 +1751,6 @@ func TestCompactor_DeleteLocalSyncFiles(t *testing.T) { for i := 1; i <= 2; i++ { cfg := prepareConfig() - cfg.CompactionInterval = 10 * time.Minute // We will only call compaction manually. cfg.ShardingEnabled = true cfg.ShardingRing.InstanceID = fmt.Sprintf("compactor-%d", i) diff --git a/pkg/compactor/partition_compaction_complete_checker.go b/pkg/compactor/partition_compaction_complete_checker.go new file mode 100644 index 0000000000..3bb6b37b05 --- /dev/null +++ b/pkg/compactor/partition_compaction_complete_checker.go @@ -0,0 +1,16 @@ +package compactor + +import ( + "github.com/oklog/ulid" + "github.com/thanos-io/thanos/pkg/compact" +) + +type PartitionCompactionBlockDeletableChecker struct{} + +func NewPartitionCompactionBlockDeletableChecker() *PartitionCompactionBlockDeletableChecker { + return &PartitionCompactionBlockDeletableChecker{} +} + +func (p *PartitionCompactionBlockDeletableChecker) CanDelete(_ *compact.Group, _ ulid.ULID) bool { + return false +} diff --git a/pkg/compactor/sharded_block_populator.go b/pkg/compactor/sharded_block_populator.go new file mode 100644 index 0000000000..721498344b --- /dev/null +++ b/pkg/compactor/sharded_block_populator.go @@ -0,0 +1,217 @@ +package compactor + +import ( + "context" + "io" + "maps" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "golang.org/x/exp/slices" + "golang.org/x/sync/errgroup" + + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" +) + +type ShardedBlockPopulator struct { + partitionCount int + partitionID int + logger log.Logger +} + +// PopulateBlock fills the index and chunk writers with new data gathered as the union +// of the provided blocks. It returns meta information for the new block. +// It expects sorted blocks input by mint. +// The main logic is copied from tsdb.DefaultPopulateBlockFunc +func (c ShardedBlockPopulator) PopulateBlock(ctx context.Context, metrics *tsdb.CompactorMetrics, _ log.Logger, chunkPool chunkenc.Pool, mergeFunc storage.VerticalChunkSeriesMergeFunc, blocks []tsdb.BlockReader, meta *tsdb.BlockMeta, indexw tsdb.IndexWriter, chunkw tsdb.ChunkWriter, postingsFunc tsdb.IndexReaderPostingsFunc) (err error) { + if len(blocks) == 0 { + return errors.New("cannot populate block from no readers") + } + + var ( + sets []storage.ChunkSeriesSet + setsMtx sync.Mutex + symbols map[string]struct{} + closers []io.Closer + overlapping bool + ) + symbols = make(map[string]struct{}) + defer func() { + errs := tsdb_errors.NewMulti(err) + if cerr := tsdb_errors.CloseAll(closers); cerr != nil { + errs.Add(errors.Wrap(cerr, "close")) + } + err = errs.Err() + metrics.PopulatingBlocks.Set(0) + }() + metrics.PopulatingBlocks.Set(1) + + globalMaxt := blocks[0].Meta().MaxTime + g, _ := errgroup.WithContext(ctx) + g.SetLimit(8) + for i, b := range blocks { + select { + case <-ctx.Done(): + return ctx.Err() + default: + } + + if !overlapping { + if i > 0 && b.Meta().MinTime < globalMaxt { + metrics.OverlappingBlocks.Inc() + overlapping = true + level.Info(c.logger).Log("msg", "Found overlapping blocks during compaction", "ulid", meta.ULID) + } + if b.Meta().MaxTime > globalMaxt { + globalMaxt = b.Meta().MaxTime + } + } + + indexr, err := b.Index() + if err != nil { + return errors.Wrapf(err, "open index reader for block %+v", b.Meta()) + } + closers = append(closers, indexr) + + chunkr, err := b.Chunks() + if err != nil { + return errors.Wrapf(err, "open chunk reader for block %+v", b.Meta()) + } + closers = append(closers, chunkr) + + tombsr, err := b.Tombstones() + if err != nil { + return errors.Wrapf(err, "open tombstone reader for block %+v", b.Meta()) + } + closers = append(closers, tombsr) + + all := postingsFunc(ctx, indexr) + g.Go(func() error { + shardStart := time.Now() + shardedPosting, syms, err := NewShardedPosting(all, uint64(c.partitionCount), uint64(c.partitionID), indexr.Series) + if err != nil { + return err + } + level.Debug(c.logger).Log("msg", "finished sharding", "duration", time.Since(shardStart)) + // Blocks meta is half open: [min, max), so subtract 1 to ensure we don't hold samples with exact meta.MaxTime timestamp. + setsMtx.Lock() + sets = append(sets, tsdb.NewBlockChunkSeriesSet(meta.ULID, indexr, chunkr, tombsr, shardedPosting, meta.MinTime, meta.MaxTime-1, false)) + maps.Copy(symbols, syms) + setsMtx.Unlock() + return nil + }) + } + if err := g.Wait(); err != nil { + return err + } + + symbolsList := make([]string, len(symbols)) + symbolIdx := 0 + for symbol := range symbols { + symbolsList[symbolIdx] = symbol + symbolIdx++ + } + slices.Sort(symbolsList) + for _, symbol := range symbolsList { + if err := indexw.AddSymbol(symbol); err != nil { + return errors.Wrap(err, "add symbol") + } + } + + var ( + ref = storage.SeriesRef(0) + ch = make(chan func() error, 1000) + ) + + set := sets[0] + if len(sets) > 1 { + iCtx, cancel := context.WithCancel(ctx) + // Merge series using specified chunk series merger. + // The default one is the compacting series merger. + set = NewBackgroundChunkSeriesSet(iCtx, storage.NewMergeChunkSeriesSet(sets, mergeFunc)) + defer cancel() + } + + go func() { + // Iterate over all sorted chunk series. + for set.Next() { + select { + case <-ctx.Done(): + ch <- func() error { return ctx.Err() } + default: + } + s := set.At() + curChksIter := s.Iterator(nil) + + var chks []chunks.Meta + var wg sync.WaitGroup + r := ref + wg.Add(1) + go func() { + for curChksIter.Next() { + // We are not iterating in streaming way over chunk as + // it's more efficient to do bulk write for index and + // chunk file purposes. + chks = append(chks, curChksIter.At()) + } + wg.Done() + }() + + ch <- func() error { + wg.Wait() + if curChksIter.Err() != nil { + return errors.Wrap(curChksIter.Err(), "chunk iter") + } + + // Skip the series with all deleted chunks. + if len(chks) == 0 { + return nil + } + + if err := chunkw.WriteChunks(chks...); err != nil { + return errors.Wrap(err, "write chunks") + } + if err := indexw.AddSeries(r, s.Labels(), chks...); err != nil { + return errors.Wrap(err, "add series") + } + + meta.Stats.NumChunks += uint64(len(chks)) + meta.Stats.NumSeries++ + for _, chk := range chks { + meta.Stats.NumSamples += uint64(chk.Chunk.NumSamples()) + } + + for _, chk := range chks { + if err := chunkPool.Put(chk.Chunk); err != nil { + return errors.Wrap(err, "put chunk") + } + } + + return nil + } + + ref++ + } + close(ch) + }() + + for callback := range ch { + err := callback() + if err != nil { + return err + } + } + + if set.Err() != nil { + return errors.Wrap(set.Err(), "iterate compaction set") + } + + return nil +} diff --git a/pkg/compactor/sharded_compaction_lifecycle_callback.go b/pkg/compactor/sharded_compaction_lifecycle_callback.go new file mode 100644 index 0000000000..ab6fc93845 --- /dev/null +++ b/pkg/compactor/sharded_compaction_lifecycle_callback.go @@ -0,0 +1,108 @@ +package compactor + +import ( + "context" + "path/filepath" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/tsdb" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/compact" + "github.com/thanos-io/thanos/pkg/runutil" + + cortextsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" +) + +type ShardedCompactionLifecycleCallback struct { + ctx context.Context + userBucket objstore.InstrumentedBucket + logger log.Logger + metaSyncConcurrency int + compactDir string + userID string + compactorMetrics *compactorMetrics + + startTime time.Time +} + +func NewShardedCompactionLifecycleCallback( + ctx context.Context, + userBucket objstore.InstrumentedBucket, + logger log.Logger, + metaSyncConcurrency int, + compactDir string, + userID string, + compactorMetrics *compactorMetrics, +) *ShardedCompactionLifecycleCallback { + return &ShardedCompactionLifecycleCallback{ + ctx: ctx, + userBucket: userBucket, + logger: logger, + metaSyncConcurrency: metaSyncConcurrency, + compactDir: compactDir, + userID: userID, + compactorMetrics: compactorMetrics, + } +} + +func (c *ShardedCompactionLifecycleCallback) PreCompactionCallback(_ context.Context, logger log.Logger, g *compact.Group, meta []*metadata.Meta) error { + c.startTime = time.Now() + + metaExt, err := cortextsdb.ConvertToCortexMetaExtensions(g.Extensions()) + if err != nil { + level.Warn(logger).Log("msg", "unable to get cortex meta extensions", "err", err) + } else if metaExt != nil { + c.compactorMetrics.compactionPlanned.WithLabelValues(c.userID, metaExt.TimeRangeStr()).Inc() + } + + // Delete local files other than current group + var ignoreDirs []string + for _, m := range meta { + ignoreDirs = append(ignoreDirs, filepath.Join(g.Key(), m.ULID.String())) + } + if err := runutil.DeleteAll(c.compactDir, ignoreDirs...); err != nil { + level.Warn(logger).Log("msg", "failed deleting non-current compaction group files, disk space usage might have leaked.", "err", err, "dir", c.compactDir) + } + return nil +} + +func (c *ShardedCompactionLifecycleCallback) PostCompactionCallback(_ context.Context, logger log.Logger, cg *compact.Group, _ ulid.ULID) error { + metaExt, err := cortextsdb.ConvertToCortexMetaExtensions(cg.Extensions()) + if err != nil { + level.Warn(logger).Log("msg", "unable to get cortex meta extensions", "err", err) + } else if metaExt != nil { + c.compactorMetrics.compactionDuration.WithLabelValues(c.userID, metaExt.TimeRangeStr()).Set(time.Since(c.startTime).Seconds()) + } + return nil +} + +func (c *ShardedCompactionLifecycleCallback) GetBlockPopulator(_ context.Context, logger log.Logger, cg *compact.Group) (tsdb.BlockPopulator, error) { + partitionInfo, err := cortextsdb.ConvertToPartitionInfo(cg.Extensions()) + if err != nil { + return nil, err + } + if partitionInfo == nil { + return tsdb.DefaultBlockPopulator{}, nil + } + if partitionInfo.PartitionCount <= 0 { + partitionInfo = &cortextsdb.PartitionInfo{ + PartitionCount: 1, + PartitionID: partitionInfo.PartitionID, + PartitionedGroupID: partitionInfo.PartitionedGroupID, + PartitionedGroupCreationTime: partitionInfo.PartitionedGroupCreationTime, + } + cg.SetExtensions(&cortextsdb.CortexMetaExtensions{ + PartitionInfo: partitionInfo, + }) + } + populateBlockFunc := ShardedBlockPopulator{ + partitionCount: partitionInfo.PartitionCount, + partitionID: partitionInfo.PartitionID, + logger: logger, + } + return populateBlockFunc, nil +} diff --git a/pkg/compactor/sharded_compaction_lifecycle_callback_test.go b/pkg/compactor/sharded_compaction_lifecycle_callback_test.go new file mode 100644 index 0000000000..09157c895b --- /dev/null +++ b/pkg/compactor/sharded_compaction_lifecycle_callback_test.go @@ -0,0 +1,96 @@ +package compactor + +import ( + "context" + "os" + "path/filepath" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/require" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/compact" +) + +func TestPreCompactionCallback(t *testing.T) { + compactDir, err := os.MkdirTemp(os.TempDir(), "compact") + require.NoError(t, err) + + t.Cleanup(func() { + require.NoError(t, os.RemoveAll(compactDir)) + }) + + lifecycleCallback := ShardedCompactionLifecycleCallback{ + compactDir: compactDir, + } + + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + block3 := ulid.MustNew(3, nil) + meta := []*metadata.Meta{ + { + BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 2 * time.Hour.Milliseconds()}, + }, + { + BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 2 * time.Hour.Milliseconds()}, + }, + { + BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 2 * time.Hour.Milliseconds(), MaxTime: 3 * time.Hour.Milliseconds()}, + }, + } + testGroupKey := "test_group_key" + testGroup, _ := compact.NewGroup( + log.NewNopLogger(), + nil, + testGroupKey, + nil, + 0, + true, + true, + nil, + nil, + nil, + nil, + nil, + nil, + nil, + nil, + metadata.NoneFunc, + 1, + 1, + ) + for _, m := range meta { + err := testGroup.AppendMeta(m) + require.NoError(t, err) + } + + dummyGroupID1 := "dummy_dir_1" + dummyGroupID2 := "dummy_dir_2" + err = os.MkdirAll(filepath.Join(compactDir, testGroupKey), 0750) + require.NoError(t, err) + err = os.MkdirAll(filepath.Join(compactDir, testGroupKey, block1.String()), 0750) + require.NoError(t, err) + err = os.MkdirAll(filepath.Join(compactDir, dummyGroupID1), 0750) + require.NoError(t, err) + err = os.MkdirAll(filepath.Join(compactDir, dummyGroupID2), 0750) + require.NoError(t, err) + + err = lifecycleCallback.PreCompactionCallback(context.Background(), log.NewNopLogger(), testGroup, meta) + require.NoError(t, err) + + info, err := os.Stat(filepath.Join(compactDir, testGroupKey)) + require.NoError(t, err) + require.True(t, info.IsDir()) + info, err = os.Stat(filepath.Join(compactDir, testGroupKey, block1.String())) + require.NoError(t, err) + require.True(t, info.IsDir()) + _, err = os.Stat(filepath.Join(compactDir, dummyGroupID1)) + require.Error(t, err) + require.True(t, os.IsNotExist(err)) + _, err = os.Stat(filepath.Join(compactDir, dummyGroupID2)) + require.Error(t, err) + require.True(t, os.IsNotExist(err)) +} diff --git a/pkg/compactor/sharded_posting.go b/pkg/compactor/sharded_posting.go new file mode 100644 index 0000000000..aca18a804d --- /dev/null +++ b/pkg/compactor/sharded_posting.go @@ -0,0 +1,30 @@ +package compactor + +import ( + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" +) + +func NewShardedPosting(postings index.Postings, partitionCount uint64, partitionID uint64, labelsFn func(ref storage.SeriesRef, builder *labels.ScratchBuilder, chks *[]chunks.Meta) error) (index.Postings, map[string]struct{}, error) { + bufChks := make([]chunks.Meta, 0) + series := make([]storage.SeriesRef, 0) + symbols := make(map[string]struct{}) + var builder labels.ScratchBuilder + for postings.Next() { + err := labelsFn(postings.At(), &builder, &bufChks) + if err != nil { + return nil, nil, err + } + if builder.Labels().Hash()%partitionCount == partitionID { + posting := postings.At() + series = append(series, posting) + for _, label := range builder.Labels() { + symbols[label.Name] = struct{}{} + symbols[label.Value] = struct{}{} + } + } + } + return index.NewListPostings(series), symbols, nil +} diff --git a/pkg/compactor/sharded_posting_test.go b/pkg/compactor/sharded_posting_test.go new file mode 100644 index 0000000000..5077d740ed --- /dev/null +++ b/pkg/compactor/sharded_posting_test.go @@ -0,0 +1,109 @@ +package compactor + +import ( + "context" + "io" + "math/rand" + "os" + "path/filepath" + "strconv" + "testing" + "time" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/stretchr/testify/require" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/testutil/e2eutil" +) + +const ( + MetricLabelName = "__name__" + MetricName = "test_metric" + TestLabelName = "test_label" + ConstLabelName = "const_label" + ConstLabelValue = "const_value" +) + +func TestShardPostingAndSymbolBasedOnPartitionID(t *testing.T) { + partitionCount := 8 + + tmpdir, err := os.MkdirTemp("", "sharded_posting_test") + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, os.RemoveAll(tmpdir)) + }) + + r := rand.New(rand.NewSource(0)) + var series []labels.Labels + expectedSymbols := make(map[string]bool) + metricName := labels.Label{Name: MetricLabelName, Value: MetricName} + expectedSymbols[MetricLabelName] = false + expectedSymbols[MetricName] = false + expectedSymbols[ConstLabelName] = false + expectedSymbols[ConstLabelValue] = false + expectedSeriesCount := 10 + for i := 0; i < expectedSeriesCount; i++ { + labelValue := strconv.Itoa(r.Int()) + series = append(series, labels.Labels{ + metricName, + {Name: ConstLabelName, Value: ConstLabelValue}, + {Name: TestLabelName, Value: labelValue}, + }) + expectedSymbols[TestLabelName] = false + expectedSymbols[labelValue] = false + } + blockID, err := e2eutil.CreateBlock(context.Background(), tmpdir, series, 10, time.Now().Add(-10*time.Minute).UnixMilli(), time.Now().UnixMilli(), nil, 0, metadata.NoneFunc) + require.NoError(t, err) + + var closers []io.Closer + defer func() { + for _, c := range closers { + c.Close() + } + }() + seriesCount := 0 + for partitionID := 0; partitionID < partitionCount; partitionID++ { + ir, err := index.NewFileReader(filepath.Join(tmpdir, blockID.String(), "index")) + closers = append(closers, ir) + require.NoError(t, err) + k, v := index.AllPostingsKey() + postings, err := ir.Postings(context.Background(), k, v) + require.NoError(t, err) + postings = ir.SortedPostings(postings) + shardedPostings, syms, err := NewShardedPosting(postings, uint64(partitionCount), uint64(partitionID), ir.Series) + require.NoError(t, err) + bufChks := make([]chunks.Meta, 0) + expectedShardedSymbols := make(map[string]struct{}) + for shardedPostings.Next() { + var builder labels.ScratchBuilder + err = ir.Series(shardedPostings.At(), &builder, &bufChks) + require.NoError(t, err) + require.Equal(t, uint64(partitionID), builder.Labels().Hash()%uint64(partitionCount)) + seriesCount++ + for _, label := range builder.Labels() { + expectedShardedSymbols[label.Name] = struct{}{} + expectedShardedSymbols[label.Value] = struct{}{} + } + } + err = ir.Close() + if err == nil { + closers = closers[0 : len(closers)-1] + } + symbolsCount := 0 + for s := range syms { + symbolsCount++ + _, ok := expectedSymbols[s] + require.True(t, ok) + expectedSymbols[s] = true + _, ok = expectedShardedSymbols[s] + require.True(t, ok) + } + require.Equal(t, len(expectedShardedSymbols), symbolsCount) + } + require.Equal(t, expectedSeriesCount, seriesCount) + for _, visited := range expectedSymbols { + require.True(t, visited) + } +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/copy.go b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/copy.go new file mode 100644 index 0000000000..6464cd02ba --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/copy.go @@ -0,0 +1,55 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package e2eutil + +import ( + "io" + "os" + "path/filepath" + "testing" + + "github.com/efficientgo/core/testutil" + "github.com/pkg/errors" + "github.com/thanos-io/thanos/pkg/runutil" +) + +func Copy(t testing.TB, src, dst string) { + testutil.Ok(t, copyRecursive(src, dst)) +} + +func copyRecursive(src, dst string) error { + return filepath.Walk(src, func(path string, info os.FileInfo, err error) error { + if err != nil { + return err + } + + relPath, err := filepath.Rel(src, path) + if err != nil { + return err + } + + if info.IsDir() { + return os.MkdirAll(filepath.Join(dst, relPath), os.ModePerm) + } + + if !info.Mode().IsRegular() { + return errors.Errorf("%s is not a regular file", path) + } + + source, err := os.Open(filepath.Clean(path)) + if err != nil { + return err + } + defer runutil.CloseWithErrCapture(&err, source, "close file") + + destination, err := os.Create(filepath.Join(dst, relPath)) + if err != nil { + return err + } + defer runutil.CloseWithErrCapture(&err, destination, "close file") + + _, err = io.Copy(destination, source) + return err + }) +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/port.go b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/port.go new file mode 100644 index 0000000000..986f1c7d7f --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/port.go @@ -0,0 +1,20 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package e2eutil + +import "net" + +// FreePort returns port that is free now. +func FreePort() (int, error) { + addr, err := net.ResolveTCPAddr("tcp", ":0") + if err != nil { + return 0, err + } + + l, err := net.ListenTCP("tcp", addr) + if err != nil { + return 0, err + } + return l.Addr().(*net.TCPAddr).Port, l.Close() +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/prometheus.go b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/prometheus.go new file mode 100644 index 0000000000..5d784a9cc0 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/prometheus.go @@ -0,0 +1,818 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package e2eutil + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "io" + "math" + "math/rand" + "net/http" + "os" + "os/exec" + "path" + "path/filepath" + "runtime" + "strings" + "sync" + "syscall" + "testing" + "time" + + "github.com/efficientgo/core/testutil" + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/timestamp" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/index" + "go.uber.org/atomic" + "golang.org/x/sync/errgroup" + + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/runutil" +) + +const ( + defaultPrometheusVersion = "v0.54.1" + defaultAlertmanagerVersion = "v0.20.0" + defaultMinioVersion = "RELEASE.2022-07-30T05-21-40Z" + + // Space delimited list of versions. + promPathsEnvVar = "THANOS_TEST_PROMETHEUS_PATHS" + alertmanagerBinEnvVar = "THANOS_TEST_ALERTMANAGER_PATH" + minioBinEnvVar = "THANOS_TEST_MINIO_PATH" + + // A placeholder for actual Prometheus instance address in the scrape config. + PromAddrPlaceHolder = "PROMETHEUS_ADDRESS" +) + +var ( + histogramSample = histogram.Histogram{ + Schema: 0, + Count: 20, + Sum: -3.1415, + ZeroCount: 12, + ZeroThreshold: 0.001, + NegativeSpans: []histogram.Span{ + {Offset: 0, Length: 4}, + {Offset: 1, Length: 1}, + }, + NegativeBuckets: []int64{1, 2, -2, 1, -1}, + } + + floatHistogramSample = histogram.FloatHistogram{ + ZeroThreshold: 0.01, + ZeroCount: 5.5, + Count: 15, + Sum: 11.5, + PositiveSpans: []histogram.Span{ + {Offset: -2, Length: 2}, + {Offset: 1, Length: 3}, + }, + PositiveBuckets: []float64{0.5, 0, 1.5, 2, 3.5}, + NegativeSpans: []histogram.Span{ + {Offset: 3, Length: 2}, + {Offset: 3, Length: 2}, + }, + NegativeBuckets: []float64{1.5, 0.5, 2.5, 3}, + } +) + +func PrometheusBinary() string { + return "prometheus-" + defaultPrometheusVersion +} + +func AlertmanagerBinary() string { + b := os.Getenv(alertmanagerBinEnvVar) + if b == "" { + return fmt.Sprintf("alertmanager-%s", defaultAlertmanagerVersion) + } + return b +} + +func MinioBinary() string { + b := os.Getenv(minioBinEnvVar) + if b == "" { + return fmt.Sprintf("minio-%s", defaultMinioVersion) + } + return b +} + +// Prometheus represents a test instance for integration testing. +// It can be populated with data before being started. +type Prometheus struct { + dir string + db *tsdb.DB + prefix string + binPath string + + running bool + cmd *exec.Cmd + disabledCompaction bool + addr string + + config string + + stdout, stderr bytes.Buffer +} + +func NewTSDB() (*tsdb.DB, error) { + dir, err := os.MkdirTemp("", "prometheus-test") + if err != nil { + return nil, err + } + opts := tsdb.DefaultOptions() + opts.RetentionDuration = math.MaxInt64 + return tsdb.Open(dir, nil, nil, opts, nil) +} + +func ForeachPrometheus(t *testing.T, testFn func(t testing.TB, p *Prometheus)) { + paths := os.Getenv(promPathsEnvVar) + if paths == "" { + paths = PrometheusBinary() + } + + for _, path := range strings.Split(paths, " ") { + if ok := t.Run(path, func(t *testing.T) { + p, err := newPrometheus(path, "") + testutil.Ok(t, err) + + testFn(t, p) + testutil.Ok(t, p.Stop()) + }); !ok { + return + } + } +} + +// NewPrometheus creates a new test Prometheus instance that will listen on local address. +// Use ForeachPrometheus if you want to test against set of Prometheus versions. +// TODO(bwplotka): Improve it with https://github.com/thanos-io/thanos/issues/758. +func NewPrometheus() (*Prometheus, error) { + return newPrometheus("", "") +} + +// NewPrometheusOnPath creates a new test Prometheus instance that will listen on local address and given prefix path. +func NewPrometheusOnPath(prefix string) (*Prometheus, error) { + return newPrometheus("", prefix) +} + +func newPrometheus(binPath, prefix string) (*Prometheus, error) { + if binPath == "" { + binPath = PrometheusBinary() + } + + db, err := NewTSDB() + if err != nil { + return nil, err + } + + f, err := os.Create(filepath.Join(db.Dir(), "prometheus.yml")) + if err != nil { + return nil, err + } + defer f.Close() + + // Some well-known external labels so that we can test label resorting + if _, err = io.WriteString(f, "global:\n external_labels:\n region: eu-west"); err != nil { + return nil, err + } + + return &Prometheus{ + dir: db.Dir(), + db: db, + prefix: prefix, + binPath: binPath, + addr: "", + }, nil +} + +// Start running the Prometheus instance and return. +func (p *Prometheus) Start(ctx context.Context, l log.Logger) error { + if p.running { + return errors.New("Already started") + } + + if err := p.db.Close(); err != nil { + return err + } + if err := p.start(); err != nil { + return err + } + if err := p.waitPrometheusUp(ctx, l, p.prefix); err != nil { + return err + } + return nil +} + +func (p *Prometheus) start() error { + port, err := FreePort() + if err != nil { + return err + } + + var extra []string + if p.disabledCompaction { + extra = append(extra, + "--storage.tsdb.min-block-duration=2h", + "--storage.tsdb.max-block-duration=2h", + ) + } + p.addr = fmt.Sprintf("localhost:%d", port) + // Write the final config to the config file. + // The address placeholder will be replaced with the actual address. + if err := p.writeConfig(strings.ReplaceAll(p.config, PromAddrPlaceHolder, p.addr)); err != nil { + return err + } + args := append([]string{ + "--storage.tsdb.retention=2d", // Pass retention cause prometheus since 2.8.0 don't show default value for that flags in web/api: https://github.com/prometheus/prometheus/pull/5433. + "--storage.tsdb.path=" + p.db.Dir(), + "--web.listen-address=" + p.addr, + "--web.route-prefix=" + p.prefix, + "--web.enable-admin-api", + "--config.file=" + filepath.Join(p.db.Dir(), "prometheus.yml"), + }, extra...) + + p.cmd = exec.Command(p.binPath, args...) + p.cmd.SysProcAttr = SysProcAttr() + + p.stderr.Reset() + p.stdout.Reset() + + p.cmd.Stdout = &p.stdout + p.cmd.Stderr = &p.stderr + + if err := p.cmd.Start(); err != nil { + return fmt.Errorf("starting Prometheus failed: %w", err) + } + + p.running = true + return nil +} + +func (p *Prometheus) waitPrometheusUp(ctx context.Context, logger log.Logger, prefix string) error { + if !p.running { + return errors.New("method Start was not invoked.") + } + return runutil.RetryWithLog(logger, time.Second, ctx.Done(), func() error { + r, err := http.Get(fmt.Sprintf("http://%s%s/-/ready", p.addr, prefix)) + if err != nil { + return err + } + defer runutil.ExhaustCloseWithLogOnErr(logger, r.Body, "failed to exhaust and close body") + + if r.StatusCode != 200 { + return errors.Errorf("Got non 200 response: %v", r.StatusCode) + } + return nil + }) +} + +func (p *Prometheus) Restart(ctx context.Context, l log.Logger) error { + if err := p.cmd.Process.Signal(syscall.SIGTERM); err != nil { + return errors.Wrap(err, "failed to kill Prometheus. Kill it manually") + } + _ = p.cmd.Wait() + if err := p.start(); err != nil { + return err + } + return p.waitPrometheusUp(ctx, l, p.prefix) +} + +// Dir returns TSDB dir. +func (p *Prometheus) Dir() string { + return p.dir +} + +// Addr returns correct address after Start method. +func (p *Prometheus) Addr() string { + return p.addr + p.prefix +} + +func (p *Prometheus) DisableCompaction() { + p.disabledCompaction = true +} + +// SetConfig updates the contents of the config. +func (p *Prometheus) SetConfig(s string) { + p.config = s +} + +// writeConfig writes the Prometheus config to the config file. +func (p *Prometheus) writeConfig(config string) (err error) { + f, err := os.Create(filepath.Join(p.dir, "prometheus.yml")) + if err != nil { + return err + } + defer runutil.CloseWithErrCapture(&err, f, "prometheus config") + _, err = f.Write([]byte(config)) + return err +} + +// Stop terminates Prometheus and clean up its data directory. +func (p *Prometheus) Stop() (rerr error) { + if !p.running { + return nil + } + + if p.cmd.Process != nil { + if err := p.cmd.Process.Signal(syscall.SIGTERM); err != nil { + return errors.Wrapf(err, "failed to Prometheus. Kill it manually and clean %s dir", p.db.Dir()) + } + + err := p.cmd.Wait() + if err != nil { + var exitErr *exec.ExitError + if errors.As(err, &exitErr) { + if exitErr.ExitCode() != -1 { + fmt.Fprintln(os.Stderr, "Prometheus exited with", exitErr.ExitCode()) + fmt.Fprintln(os.Stderr, "stdout:\n", p.stdout.String(), "\nstderr:\n", p.stderr.String()) + } else { + err = nil + } + } + } + + if err != nil { + return fmt.Errorf("waiting for Prometheus to exit: %w", err) + } + } + + return p.cleanup() +} + +func (p *Prometheus) cleanup() error { + p.running = false + return os.RemoveAll(p.db.Dir()) +} + +// Appender returns a new appender to populate the Prometheus instance with data. +// All appenders must be closed before Start is called and no new ones must be opened +// afterwards. +func (p *Prometheus) Appender() storage.Appender { + if p.running { + panic("Appender must not be called after start") + } + return p.db.Appender(context.Background()) +} + +// CreateEmptyBlock produces empty block like it was the case before fix: https://github.com/prometheus/tsdb/pull/374. +// (Prometheus pre v2.7.0). +func CreateEmptyBlock(dir string, mint, maxt int64, extLset labels.Labels, resolution int64) (ulid.ULID, error) { + entropy := rand.New(rand.NewSource(time.Now().UnixNano())) + uid := ulid.MustNew(ulid.Now(), entropy) + + if err := os.Mkdir(path.Join(dir, uid.String()), os.ModePerm); err != nil { + return ulid.ULID{}, errors.Wrap(err, "close index") + } + + if err := os.Mkdir(path.Join(dir, uid.String(), "chunks"), os.ModePerm); err != nil { + return ulid.ULID{}, errors.Wrap(err, "close index") + } + + w, err := index.NewWriter(context.Background(), path.Join(dir, uid.String(), "index")) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "new index") + } + + if err := w.Close(); err != nil { + return ulid.ULID{}, errors.Wrap(err, "close index") + } + + m := tsdb.BlockMeta{ + Version: 1, + ULID: uid, + MinTime: mint, + MaxTime: maxt, + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{uid}, + }, + } + b, err := json.Marshal(&m) + if err != nil { + return ulid.ULID{}, err + } + + if err := os.WriteFile(path.Join(dir, uid.String(), "meta.json"), b, os.ModePerm); err != nil { + return ulid.ULID{}, errors.Wrap(err, "saving meta.json") + } + + if _, err = metadata.InjectThanos(log.NewNopLogger(), filepath.Join(dir, uid.String()), metadata.Thanos{ + Labels: extLset.Map(), + Downsample: metadata.ThanosDownsample{Resolution: resolution}, + Source: metadata.TestSource, + }, nil); err != nil { + return ulid.ULID{}, errors.Wrap(err, "finalize block") + } + + return uid, nil +} + +// CreateBlock writes a block with the given series and numSamples samples each. +// Samples will be in the time range [mint, maxt). +func CreateBlock( + ctx context.Context, + dir string, + series []labels.Labels, + numSamples int, + mint, maxt int64, + extLset labels.Labels, + resolution int64, + hashFunc metadata.HashFunc, +) (id ulid.ULID, err error) { + return createBlock(ctx, dir, series, numSamples, mint, maxt, extLset, resolution, false, hashFunc, chunkenc.ValFloat) +} + +// CreateBlockWithTombstone is same as CreateBlock but leaves tombstones which mimics the Prometheus local block. +func CreateBlockWithTombstone( + ctx context.Context, + dir string, + series []labels.Labels, + numSamples int, + mint, maxt int64, + extLset labels.Labels, + resolution int64, + hashFunc metadata.HashFunc, +) (id ulid.ULID, err error) { + return createBlock(ctx, dir, series, numSamples, mint, maxt, extLset, resolution, true, hashFunc, chunkenc.ValFloat) +} + +// CreateBlockWithBlockDelay writes a block with the given series and numSamples samples each. +// Samples will be in the time range [mint, maxt) +// Block ID will be created with a delay of time duration blockDelay. +func CreateBlockWithBlockDelay( + ctx context.Context, + dir string, + series []labels.Labels, + numSamples int, + mint, maxt int64, + blockDelay time.Duration, + extLset labels.Labels, + resolution int64, + hashFunc metadata.HashFunc, +) (ulid.ULID, error) { + return createBlockWithDelay(ctx, dir, series, numSamples, mint, maxt, blockDelay, extLset, resolution, hashFunc, chunkenc.ValFloat) +} + +// CreateHistogramBlockWithDelay writes a block with the given native histogram series and numSamples samples each. +// Samples will be in the time range [mint, maxt). +func CreateHistogramBlockWithDelay( + ctx context.Context, + dir string, + series []labels.Labels, + numSamples int, + mint, maxt int64, + blockDelay time.Duration, + extLset labels.Labels, + resolution int64, + hashFunc metadata.HashFunc, +) (id ulid.ULID, err error) { + return createBlockWithDelay(ctx, dir, series, numSamples, mint, maxt, blockDelay, extLset, resolution, hashFunc, chunkenc.ValHistogram) +} + +// CreateFloatHistogramBlockWithDelay writes a block with the given float native histogram series and numSamples samples each. +// Samples will be in the time range [mint, maxt). +func CreateFloatHistogramBlockWithDelay( + ctx context.Context, + dir string, + series []labels.Labels, + numSamples int, + mint, maxt int64, + blockDelay time.Duration, + extLset labels.Labels, + resolution int64, + hashFunc metadata.HashFunc, +) (id ulid.ULID, err error) { + return createBlockWithDelay(ctx, dir, series, numSamples, mint, maxt, blockDelay, extLset, resolution, hashFunc, chunkenc.ValFloatHistogram) +} + +func createBlockWithDelay(ctx context.Context, dir string, series []labels.Labels, numSamples int, mint int64, maxt int64, blockDelay time.Duration, extLset labels.Labels, resolution int64, hashFunc metadata.HashFunc, samplesType chunkenc.ValueType) (ulid.ULID, error) { + blockID, err := createBlock(ctx, dir, series, numSamples, mint, maxt, extLset, resolution, false, hashFunc, samplesType) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "block creation") + } + + id, err := ulid.New(uint64(timestamp.FromTime(timestamp.Time(int64(blockID.Time())).Add(-blockDelay))), bytes.NewReader(blockID.Entropy())) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "create block id") + } + + bdir := path.Join(dir, blockID.String()) + m, err := metadata.ReadFromDir(bdir) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "open meta file") + } + + logger := log.NewNopLogger() + m.ULID = id + m.Compaction.Sources = []ulid.ULID{id} + if err := m.WriteToDir(logger, path.Join(dir, blockID.String())); err != nil { + return ulid.ULID{}, errors.Wrap(err, "write meta.json file") + } + + return id, os.Rename(path.Join(dir, blockID.String()), path.Join(dir, id.String())) +} + +func createBlock( + ctx context.Context, + dir string, + series []labels.Labels, + numSamples int, + mint, maxt int64, + extLset labels.Labels, + resolution int64, + tombstones bool, + hashFunc metadata.HashFunc, + sampleType chunkenc.ValueType, +) (id ulid.ULID, err error) { + headOpts := tsdb.DefaultHeadOptions() + headOpts.ChunkDirRoot = filepath.Join(dir, "chunks") + headOpts.ChunkRange = 10000000000 + headOpts.EnableNativeHistograms = *atomic.NewBool(true) + h, err := tsdb.NewHead(nil, nil, nil, nil, headOpts, nil) + if err != nil { + return id, errors.Wrap(err, "create head block") + } + defer func() { + runutil.CloseWithErrCapture(&err, h, "TSDB Head") + if e := os.RemoveAll(headOpts.ChunkDirRoot); e != nil { + err = errors.Wrap(e, "delete chunks dir") + } + }() + + var g errgroup.Group + var timeStepSize = (maxt - mint) / int64(numSamples+1) + var batchSize = len(series) / runtime.GOMAXPROCS(0) + r := rand.New(rand.NewSource(int64(numSamples))) + var randMutex sync.Mutex + + for len(series) > 0 { + l := batchSize + if len(series) < 1000 { + l = len(series) + } + batch := series[:l] + series = series[l:] + + g.Go(func() error { + t := mint + + for i := 0; i < numSamples; i++ { + app := h.Appender(ctx) + + for _, lset := range batch { + var err error + if sampleType == chunkenc.ValFloat { + randMutex.Lock() + _, err = app.Append(0, lset, t, r.Float64()) + randMutex.Unlock() + } else if sampleType == chunkenc.ValHistogram { + _, err = app.AppendHistogram(0, lset, t, &histogramSample, nil) + } else if sampleType == chunkenc.ValFloatHistogram { + _, err = app.AppendHistogram(0, lset, t, nil, &floatHistogramSample) + } + if err != nil { + if rerr := app.Rollback(); rerr != nil { + err = errors.Wrapf(err, "rollback failed: %v", rerr) + } + + return errors.Wrap(err, "add sample") + } + } + if err := app.Commit(); err != nil { + return errors.Wrap(err, "commit") + } + t += timeStepSize + } + return nil + }) + } + if err := g.Wait(); err != nil { + return id, err + } + c, err := tsdb.NewLeveledCompactor(ctx, nil, log.NewNopLogger(), []int64{maxt - mint}, nil, nil) + if err != nil { + return id, errors.Wrap(err, "create compactor") + } + + ids, err := c.Write(dir, h, mint, maxt, nil) + if err != nil { + return id, errors.Wrap(err, "write block") + } + if len(ids) == 0 { + return id, errors.Errorf("nothing to write, asked for %d samples", numSamples) + } + id = ids[0] + + blockDir := filepath.Join(dir, id.String()) + logger := log.NewNopLogger() + seriesSize, err := gatherMaxSeriesSize(ctx, filepath.Join(blockDir, "index")) + if err != nil { + return id, errors.Wrap(err, "gather max series size") + } + + files := []metadata.File{} + if hashFunc != metadata.NoneFunc { + paths := []string{} + if err := filepath.Walk(blockDir, func(path string, info os.FileInfo, err error) error { + if info.IsDir() { + return nil + } + paths = append(paths, path) + return nil + }); err != nil { + return id, errors.Wrapf(err, "walking %s", dir) + } + + for _, p := range paths { + pHash, err := metadata.CalculateHash(p, metadata.SHA256Func, log.NewNopLogger()) + if err != nil { + return id, errors.Wrapf(err, "calculating hash of %s", blockDir+p) + } + files = append(files, metadata.File{ + RelPath: strings.TrimPrefix(p, blockDir+"/"), + Hash: &pHash, + }) + } + } + + if _, err = metadata.InjectThanos(logger, blockDir, metadata.Thanos{ + Labels: extLset.Map(), + Downsample: metadata.ThanosDownsample{Resolution: resolution}, + Source: metadata.TestSource, + Files: files, + IndexStats: metadata.IndexStats{SeriesMaxSize: seriesSize}, + }, nil); err != nil { + return id, errors.Wrap(err, "finalize block") + } + + if !tombstones { + if err = os.Remove(filepath.Join(dir, id.String(), "tombstones")); err != nil { + return id, errors.Wrap(err, "remove tombstones") + } + } + + return id, nil +} + +func gatherMaxSeriesSize(ctx context.Context, fn string) (int64, error) { + r, err := index.NewFileReader(fn) + if err != nil { + return 0, errors.Wrap(err, "open index file") + } + defer runutil.CloseWithErrCapture(&err, r, "gather index issue file reader") + + key, value := index.AllPostingsKey() + p, err := r.Postings(ctx, key, value) + if err != nil { + return 0, errors.Wrap(err, "get all postings") + } + + // As of version two all series entries are 16 byte padded. All references + // we get have to account for that to get the correct offset. + offsetMultiplier := 1 + version := r.Version() + if version >= 2 { + offsetMultiplier = 16 + } + + // Per series. + var ( + prevId storage.SeriesRef + maxSeriesSize int64 + ) + for p.Next() { + id := p.At() + if prevId != 0 { + // Approximate size. + seriesSize := int64(id-prevId) * int64(offsetMultiplier) + if seriesSize > maxSeriesSize { + maxSeriesSize = seriesSize + } + } + prevId = id + } + if p.Err() != nil { + return 0, errors.Wrap(err, "walk postings") + } + + return maxSeriesSize, nil +} + +// CreateBlockWithChurn writes a block with the given series. Start time of each series +// will be randomized in the given time window to create churn. Only float chunk is supported right now. +func CreateBlockWithChurn( + ctx context.Context, + rnd *rand.Rand, + dir string, + series []labels.Labels, + numSamples int, + mint, maxt int64, + extLset labels.Labels, + resolution int64, + scrapeInterval int64, + seriesSize int64, +) (id ulid.ULID, err error) { + headOpts := tsdb.DefaultHeadOptions() + headOpts.ChunkDirRoot = filepath.Join(dir, "chunks") + headOpts.ChunkRange = 10000000000 + h, err := tsdb.NewHead(nil, nil, nil, nil, headOpts, nil) + if err != nil { + return id, errors.Wrap(err, "create head block") + } + defer func() { + runutil.CloseWithErrCapture(&err, h, "TSDB Head") + if e := os.RemoveAll(headOpts.ChunkDirRoot); e != nil { + err = errors.Wrap(e, "delete chunks dir") + } + }() + + app := h.Appender(ctx) + for i := 0; i < len(series); i++ { + + var ref storage.SeriesRef + start := RandRange(rnd, mint, maxt) + for j := 0; j < numSamples; j++ { + if ref == 0 { + ref, err = app.Append(0, series[i], start, float64(i+j)) + } else { + ref, err = app.Append(ref, series[i], start, float64(i+j)) + } + if err != nil { + if rerr := app.Rollback(); rerr != nil { + err = errors.Wrapf(err, "rollback failed: %v", rerr) + } + return id, errors.Wrap(err, "add sample") + } + start += scrapeInterval + if start > maxt { + break + } + } + } + if err := app.Commit(); err != nil { + return id, errors.Wrap(err, "commit") + } + + c, err := tsdb.NewLeveledCompactor(ctx, nil, log.NewNopLogger(), []int64{maxt - mint}, nil, nil) + if err != nil { + return id, errors.Wrap(err, "create compactor") + } + + ids, err := c.Write(dir, h, mint, maxt, nil) + if err != nil { + return id, errors.Wrap(err, "write block") + } + + if len(ids) == 0 { + return id, errors.Errorf("nothing to write, asked for %d samples", numSamples) + } + id = ids[0] + + blockDir := filepath.Join(dir, id.String()) + logger := log.NewNopLogger() + + if _, err = metadata.InjectThanos(logger, blockDir, metadata.Thanos{ + Labels: extLset.Map(), + Downsample: metadata.ThanosDownsample{Resolution: resolution}, + Source: metadata.TestSource, + IndexStats: metadata.IndexStats{SeriesMaxSize: seriesSize}, + }, nil); err != nil { + return id, errors.Wrap(err, "finalize block") + } + + return id, nil +} + +// AddDelay rewrites a given block with delay. +func AddDelay(blockID ulid.ULID, dir string, blockDelay time.Duration) (ulid.ULID, error) { + id, err := ulid.New(uint64(timestamp.FromTime(timestamp.Time(int64(blockID.Time())).Add(-blockDelay))), bytes.NewReader(blockID.Entropy())) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "create block id") + } + + bdir := path.Join(dir, blockID.String()) + m, err := metadata.ReadFromDir(bdir) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "open meta file") + } + + logger := log.NewNopLogger() + m.ULID = id + m.Compaction.Sources = []ulid.ULID{id} + if err := m.WriteToDir(logger, path.Join(dir, blockID.String())); err != nil { + return ulid.ULID{}, errors.Wrap(err, "write meta.json file") + } + + return id, os.Rename(path.Join(dir, blockID.String()), path.Join(dir, id.String())) +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/rand.go b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/rand.go new file mode 100644 index 0000000000..5cac2d6f07 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/rand.go @@ -0,0 +1,11 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package e2eutil + +import "math/rand" + +// RandRange returns a random int64 from [min, max]. +func RandRange(rnd *rand.Rand, min, max int64) int64 { + return rnd.Int63n(max-min) + min +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/sysprocattr.go b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/sysprocattr.go new file mode 100644 index 0000000000..53aaa7039f --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/sysprocattr.go @@ -0,0 +1,13 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +//go:build !linux +// +build !linux + +package e2eutil + +import "syscall" + +func SysProcAttr() *syscall.SysProcAttr { + return &syscall.SysProcAttr{} +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/sysprocattr_linux.go b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/sysprocattr_linux.go new file mode 100644 index 0000000000..dd77ed32a1 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/testutil/e2eutil/sysprocattr_linux.go @@ -0,0 +1,13 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package e2eutil + +import "syscall" + +func SysProcAttr() *syscall.SysProcAttr { + return &syscall.SysProcAttr{ + // For linux only, kill this if the go test process dies before the cleanup. + Pdeathsig: syscall.SIGKILL, + } +} diff --git a/vendor/modules.txt b/vendor/modules.txt index 0ebdd81496..f9ba9b961a 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1038,6 +1038,7 @@ github.com/thanos-io/thanos/pkg/store/storepb/prompb github.com/thanos-io/thanos/pkg/strutil github.com/thanos-io/thanos/pkg/targets/targetspb github.com/thanos-io/thanos/pkg/tenancy +github.com/thanos-io/thanos/pkg/testutil/e2eutil github.com/thanos-io/thanos/pkg/tls github.com/thanos-io/thanos/pkg/tracing github.com/thanos-io/thanos/pkg/tracing/interceptors