diff --git a/CHANGELOG.md b/CHANGELOG.md index 5233a07d3ef..371b9b13bbe 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -37,6 +37,7 @@ * [FEATURE] Ruler: Support for filtering rules in the API. #5417 * [FEATURE] Compactor: Add `-compactor.ring.tokens-file-path` to store generated tokens locally. #5432 * [FEATURE] Query Frontend: Add `-frontend.retry-on-too-many-outstanding-requests` to re-enqueue 429 requests if there are multiple query-schedulers available. #5496 +* [FEATURE] Compactor: Implemented partitioning compactor based on proposal #4843. #5465 * [FEATURE] Store Gateway: Add `-blocks-storage.bucket-store.max-inflight-requests` for store gateways to reject further requests upon reaching the limit. #5553 * [FEATURE] Store Gateway: Add `cortex_bucket_store_block_load_duration_seconds` histogram to track time to load blocks. #5580 * [FEATURE] AlertManager: Add `cortex_alertmanager_dispatcher_aggregation_groups` and `cortex_alertmanager_dispatcher_alert_processing_duration_seconds` metrics for dispatcher. #5592 diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 5103df9a380..a89fa6f3774 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -3057,6 +3057,14 @@ The `limits_config` configures default and per-tenant limits imposed by Cortex s # CLI flag: -compactor.tenant-shard-size [compactor_tenant_shard_size: | default = 0] +# Index size limit in bytes for each compaction partition. 0 means no limit +# CLI flag: -compactor.partition-index-size-limit-in-bytes +[compactor_partition_index_size_limit_in_bytes: | default = 0] + +# Time series count limit for each compaction partition. 0 means no limit +# CLI flag: -compactor.partition-series-count-limit +[compactor_partition_series_count_limit: | default = 0] + # S3 server-side encryption type. Required to enable server-side encryption # overrides for a specific tenant. If not set, the default S3 client settings # are used. diff --git a/pkg/compactor/background_chunks_series_set.go b/pkg/compactor/background_chunks_series_set.go new file mode 100644 index 00000000000..bca40f73d2e --- /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/block_visit_marker.go b/pkg/compactor/block_visit_marker.go index b391421fd35..585519c377a 100644 --- a/pkg/compactor/block_visit_marker.go +++ b/pkg/compactor/block_visit_marker.go @@ -1,151 +1,43 @@ package compactor import ( - "bytes" - "context" - "encoding/json" "fmt" - "io" "path" "strings" - "time" - "github.com/go-kit/log" - "github.com/go-kit/log/level" "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "github.com/thanos-io/objstore" - "github.com/thanos-io/thanos/pkg/block/metadata" - - "github.com/cortexproject/cortex/pkg/util/runutil" ) const ( - // BlockVisitMarkerFile is the known json filename for representing the most recent compactor visit. - BlockVisitMarkerFile = "visit-mark.json" + // BlockVisitMarkerFileSuffix is the known suffix of json filename for representing the most recent compactor visit. + BlockVisitMarkerFileSuffix = "visit-mark.json" + // BlockVisitMarkerFilePrefix is the known prefix of json filename for representing the most recent compactor visit. + BlockVisitMarkerFilePrefix = "partition-" // VisitMarkerVersion1 is the current supported version of visit-mark file. VisitMarkerVersion1 = 1 ) var ( - ErrorBlockVisitMarkerNotFound = errors.New("block visit marker not found") - ErrorUnmarshalBlockVisitMarker = errors.New("unmarshal block visit marker JSON") - ErrorNotBlockVisitMarker = errors.New("file is not block visit marker") + ErrorNotBlockVisitMarker = errors.New("file is not block visit marker") ) type BlockVisitMarker struct { - CompactorID string `json:"compactorID"` + CompactorID string `json:"compactorID"` + Status VisitStatus `json:"status"` + PartitionedGroupID uint32 `json:"partitionedGroupID"` + PartitionID int `json:"partitionID"` // VisitTime is a unix timestamp of when the block was visited (mark updated). VisitTime int64 `json:"visitTime"` // Version of the file. Version int `json:"version"` } -func (b *BlockVisitMarker) isVisited(blockVisitMarkerTimeout time.Duration) bool { - return time.Now().Before(time.Unix(b.VisitTime, 0).Add(blockVisitMarkerTimeout)) -} - -func (b *BlockVisitMarker) isVisitedByCompactor(blockVisitMarkerTimeout time.Duration, compactorID string) bool { - return b.CompactorID == compactorID && time.Now().Before(time.Unix(b.VisitTime, 0).Add(blockVisitMarkerTimeout)) -} - -func ReadBlockVisitMarker(ctx context.Context, bkt objstore.InstrumentedBucketReader, logger log.Logger, blockID string, blockVisitMarkerReadFailed prometheus.Counter) (*BlockVisitMarker, error) { - visitMarkerFile := path.Join(blockID, BlockVisitMarkerFile) - visitMarkerFileReader, err := bkt.ReaderWithExpectedErrs(bkt.IsObjNotFoundErr).Get(ctx, visitMarkerFile) - if err != nil { - if bkt.IsObjNotFoundErr(err) { - return nil, errors.Wrapf(ErrorBlockVisitMarkerNotFound, "block visit marker file: %s", visitMarkerFile) - } - blockVisitMarkerReadFailed.Inc() - return nil, errors.Wrapf(err, "get block visit marker file: %s", visitMarkerFile) - } - defer runutil.CloseWithLogOnErr(logger, visitMarkerFileReader, "close block visit marker reader") - b, err := io.ReadAll(visitMarkerFileReader) - if err != nil { - blockVisitMarkerReadFailed.Inc() - return nil, errors.Wrapf(err, "read block visit marker file: %s", visitMarkerFile) - } - blockVisitMarker := BlockVisitMarker{} - if err = json.Unmarshal(b, &blockVisitMarker); err != nil { - blockVisitMarkerReadFailed.Inc() - return nil, errors.Wrapf(ErrorUnmarshalBlockVisitMarker, "block visit marker file: %s, error: %v", visitMarkerFile, err.Error()) - } - if blockVisitMarker.Version != VisitMarkerVersion1 { - return nil, errors.Errorf("unexpected block visit mark file version %d, expected %d", blockVisitMarker.Version, VisitMarkerVersion1) - } - return &blockVisitMarker, nil -} - -func UpdateBlockVisitMarker(ctx context.Context, bkt objstore.Bucket, blockID string, reader io.Reader, blockVisitMarkerWriteFailed prometheus.Counter) error { - blockVisitMarkerFilePath := path.Join(blockID, BlockVisitMarkerFile) - if err := bkt.Upload(ctx, blockVisitMarkerFilePath, reader); err != nil { - blockVisitMarkerWriteFailed.Inc() - return err - } - return nil -} - -func markBlocksVisited( - ctx context.Context, - bkt objstore.Bucket, - logger log.Logger, - blocks []*metadata.Meta, - marker BlockVisitMarker, - blockVisitMarkerWriteFailed prometheus.Counter, -) { - visitMarkerFileContent, err := json.Marshal(marker) - if err != nil { - blockVisitMarkerWriteFailed.Inc() - return - } - reader := bytes.NewReader(visitMarkerFileContent) - for _, block := range blocks { - select { - // Exit early if possible. - case <-ctx.Done(): - return - default: - } - - blockID := block.ULID.String() - if err := UpdateBlockVisitMarker(ctx, bkt, blockID, reader, blockVisitMarkerWriteFailed); err != nil { - level.Error(logger).Log("msg", "unable to upsert visit marker file content for block", "blockID", blockID, "err", err) - } - reader.Reset(visitMarkerFileContent) - } -} - -func markBlocksVisitedHeartBeat(ctx context.Context, bkt objstore.Bucket, logger log.Logger, blocks []*metadata.Meta, compactorID string, blockVisitMarkerFileUpdateInterval time.Duration, blockVisitMarkerWriteFailed prometheus.Counter) { - var blockIds []string - for _, block := range blocks { - blockIds = append(blockIds, block.ULID.String()) - } - blocksInfo := strings.Join(blockIds, ",") - level.Info(logger).Log("msg", fmt.Sprintf("start heart beat for blocks: %s", blocksInfo)) - ticker := time.NewTicker(blockVisitMarkerFileUpdateInterval) - defer ticker.Stop() -heartBeat: - for { - level.Debug(logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) - blockVisitMarker := BlockVisitMarker{ - VisitTime: time.Now().Unix(), - CompactorID: compactorID, - Version: VisitMarkerVersion1, - } - markBlocksVisited(ctx, bkt, logger, blocks, blockVisitMarker, blockVisitMarkerWriteFailed) - - select { - case <-ctx.Done(): - break heartBeat - case <-ticker.C: - continue - } - } - level.Info(logger).Log("msg", fmt.Sprintf("stop heart beat for blocks: %s", blocksInfo)) +func GetBlockVisitMarkerFile(blockID string, partitionID int) string { + return path.Join(blockID, fmt.Sprintf("%s%d-%s", BlockVisitMarkerFilePrefix, partitionID, BlockVisitMarkerFileSuffix)) } func IsBlockVisitMarker(path string) bool { - return strings.HasSuffix(path, BlockVisitMarkerFile) + return strings.HasSuffix(path, BlockVisitMarkerFileSuffix) } func IsNotBlockVisitMarkerError(err error) bool { diff --git a/pkg/compactor/block_visit_marker_test.go b/pkg/compactor/block_visit_marker_test.go deleted file mode 100644 index 18b7c8e1b42..00000000000 --- a/pkg/compactor/block_visit_marker_test.go +++ /dev/null @@ -1,94 +0,0 @@ -package compactor - -import ( - "context" - "testing" - "time" - - "github.com/go-kit/log" - "github.com/oklog/ulid" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/tsdb" - "github.com/stretchr/testify/require" - "github.com/thanos-io/objstore" - "github.com/thanos-io/thanos/pkg/block/metadata" - - cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" -) - -func TestMarkBlocksVisited(t *testing.T) { - ulid0 := ulid.MustNew(0, nil) - ulid1 := ulid.MustNew(1, nil) - ulid2 := ulid.MustNew(2, nil) - now := time.Now().Unix() - nowBefore1h := time.Now().Add(-1 * time.Hour).Unix() - for _, tcase := range []struct { - name string - visitMarker BlockVisitMarker - blocks []*metadata.Meta - }{ - { - name: "write visit marker succeeded", - visitMarker: BlockVisitMarker{ - CompactorID: "foo", - VisitTime: now, - Version: VisitMarkerVersion1, - }, - blocks: []*metadata.Meta{ - { - BlockMeta: tsdb.BlockMeta{ - ULID: ulid0, - }, - }, - { - BlockMeta: tsdb.BlockMeta{ - ULID: ulid1, - }, - }, - { - BlockMeta: tsdb.BlockMeta{ - ULID: ulid2, - }, - }, - }, - }, - { - name: "write visit marker succeeded 2", - visitMarker: BlockVisitMarker{ - CompactorID: "bar", - VisitTime: nowBefore1h, - Version: VisitMarkerVersion1, - }, - blocks: []*metadata.Meta{ - { - BlockMeta: tsdb.BlockMeta{ - ULID: ulid0, - }, - }, - { - BlockMeta: tsdb.BlockMeta{ - ULID: ulid1, - }, - }, - { - BlockMeta: tsdb.BlockMeta{ - ULID: ulid2, - }, - }, - }, - }, - } { - t.Run(tcase.name, func(t *testing.T) { - ctx := context.Background() - dummyCounter := prometheus.NewCounter(prometheus.CounterOpts{}) - bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) - logger := log.NewNopLogger() - markBlocksVisited(ctx, bkt, logger, tcase.blocks, tcase.visitMarker, dummyCounter) - for _, meta := range tcase.blocks { - res, err := ReadBlockVisitMarker(ctx, objstore.WithNoopInstr(bkt), logger, meta.ULID.String(), dummyCounter) - require.NoError(t, err) - require.Equal(t, tcase.visitMarker, *res) - } - }) - } -} diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index 071c91d5593..4e9dbc19361 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -2,7 +2,10 @@ package compactor import ( "context" + "encoding/json" "fmt" + "io" + "path" "sync" "time" @@ -22,6 +25,7 @@ import ( "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/concurrency" util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/cortexproject/cortex/pkg/util/runutil" "github.com/cortexproject/cortex/pkg/util/services" ) @@ -62,9 +66,11 @@ type BlocksCleaner struct { tenantBlocksMarkedForNoCompaction *prometheus.GaugeVec tenantPartialBlocks *prometheus.GaugeVec tenantBucketIndexLastUpdate *prometheus.GaugeVec + compactorPartitionError *prometheus.CounterVec + partitionedGroupInfoReadFailed prometheus.Counter } -func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.Bucket, usersScanner *cortex_tsdb.UsersScanner, cfgProvider ConfigProvider, logger log.Logger, reg prometheus.Registerer) *BlocksCleaner { +func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.Bucket, usersScanner *cortex_tsdb.UsersScanner, cfgProvider ConfigProvider, logger log.Logger, reg prometheus.Registerer, partitionedGroupInfoReadFailed prometheus.Counter) *BlocksCleaner { c := &BlocksCleaner{ cfg: cfg, bucketClient: bucketClient, @@ -124,6 +130,12 @@ func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.Bucket, use Name: "cortex_bucket_index_last_successful_update_timestamp_seconds", Help: "Timestamp of the last successful update of a tenant's bucket index.", }, []string{"user"}), + compactorPartitionError: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: compactorPartitionErrorCountName, + Help: compactorPartitionErrorCountHelp, + ConstLabels: prometheus.Labels{"reason": "parent-block-mismatch"}, + }, []string{"user"}), + partitionedGroupInfoReadFailed: partitionedGroupInfoReadFailed, } c.Service = services.NewTimerService(cfg.CleanupInterval, c.starting, c.ticker, nil) @@ -293,6 +305,13 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID level.Info(userLogger).Log("msg", "deleted files under "+block.DebugMetas+" for tenant marked for deletion", "count", deleted) } + // Clean up partitioned group info files + if deleted, err := bucket.DeletePrefix(ctx, userBucket, PartitionedGroupDirectory, userLogger); err != nil { + return errors.Wrap(err, "failed to delete "+PartitionedGroupDirectory) + } else if deleted > 0 { + level.Info(userLogger).Log("msg", "deleted files under "+PartitionedGroupDirectory+" for tenant marked for deletion", "count", deleted) + } + // Tenant deletion mark file is inside Markers as well. if deleted, err := bucket.DeletePrefix(ctx, userBucket, bucketindex.MarkersPathname, userLogger); err != nil { return errors.Wrap(err, "failed to delete marker files") @@ -338,6 +357,7 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b idxs.SyncTime = time.Now().Unix() // Read the bucket index. + begin := time.Now() idx, err := bucketindex.ReadIndex(ctx, c.bucketClient, userID, c.cfgProvider, c.logger) defer func() { @@ -362,6 +382,7 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b idxs.Status = bucketindex.GenericError return err } + level.Info(userLogger).Log("msg", "finish reading index", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Mark blocks for future deletion based on the retention period for the user. // Note doing this before UpdateIndex, so it reads in the deletion marks. @@ -375,15 +396,18 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b } // Generate an updated in-memory version of the bucket index. + begin = time.Now() w := bucketindex.NewUpdater(c.bucketClient, userID, c.cfgProvider, c.logger) idx, partials, totalBlocksBlocksMarkedForNoCompaction, err := w.UpdateIndex(ctx, idx) if err != nil { idxs.Status = bucketindex.GenericError return err } + level.Info(userLogger).Log("msg", "finish updating index", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Delete blocks marked for deletion. We iterate over a copy of deletion marks because // we'll need to manipulate the index (removing blocks which get deleted). + begin = time.Now() blocksToDelete := make([]interface{}, 0, len(idx.BlockDeletionMarks)) var mux sync.Mutex for _, mark := range idx.BlockDeletionMarks.Clone() { @@ -392,8 +416,10 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b } blocksToDelete = append(blocksToDelete, mark.ID) } + level.Info(userLogger).Log("msg", "finish getting blocks to be deleted", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Concurrently deletes blocks marked for deletion, and removes blocks from index. + begin = time.Now() _ = concurrency.ForEach(ctx, blocksToDelete, defaultDeleteBlocksConcurrency, func(ctx context.Context, job interface{}) error { blockID := job.(ulid.ULID) @@ -412,26 +438,166 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b level.Info(userLogger).Log("msg", "deleted block marked for deletion", "block", blockID) return nil }) + level.Info(userLogger).Log("msg", "finish deleting blocks", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Partial blocks with a deletion mark can be cleaned up. This is a best effort, so we don't return // error if the cleanup of partial blocks fail. if len(partials) > 0 { + begin = time.Now() c.cleanUserPartialBlocks(ctx, partials, idx, userBucket, userLogger) + level.Info(userLogger).Log("msg", "finish cleaning partial blocks", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) } // Upload the updated index to the storage. + begin = time.Now() if err := bucketindex.WriteIndex(ctx, c.bucketClient, userID, c.cfgProvider, idx); err != nil { return err } + level.Info(userLogger).Log("msg", "finish writing new index", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) + + begin = time.Now() + c.cleanPartitionedGroupInfo(ctx, userBucket, userLogger, userID, idx) + level.Info(userLogger).Log("msg", "finish cleaning partitioned group info files", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) c.tenantBlocks.WithLabelValues(userID).Set(float64(len(idx.Blocks))) c.tenantBlocksMarkedForDelete.WithLabelValues(userID).Set(float64(len(idx.BlockDeletionMarks))) c.tenantBlocksMarkedForNoCompaction.WithLabelValues(userID).Set(float64(totalBlocksBlocksMarkedForNoCompaction)) c.tenantBucketIndexLastUpdate.WithLabelValues(userID).SetToCurrentTime() c.tenantPartialBlocks.WithLabelValues(userID).Set(float64(len(partials))) + + return nil +} + +func (c *BlocksCleaner) findResultBlocksForPartitionedGroup(ctx context.Context, userBucket objstore.InstrumentedBucket, userLogger log.Logger, index *bucketindex.Index, partitionedGroupInfo *PartitionedGroupInfo) map[int]ulid.ULID { + partitionedGroupID := partitionedGroupInfo.PartitionedGroupID + deletionMarkMap := index.BlockDeletionMarks.GetULIDSet() + var possibleResultBlocks []ulid.ULID + for _, b := range index.Blocks { + if b.MinTime >= partitionedGroupInfo.RangeStart && b.MaxTime <= partitionedGroupInfo.RangeEnd { + if _, ok := deletionMarkMap[b.ID]; !ok { + level.Info(userLogger).Log("msg", "found possible result block", "partitioned_group_id", partitionedGroupID, "block", b.ID.String()) + possibleResultBlocks = append(possibleResultBlocks, b.ID) + } + } + } + + resultBlocks := make(map[int]ulid.ULID) + for _, b := range possibleResultBlocks { + meta, err := block.DownloadMeta(ctx, userLogger, userBucket, b) + if err != nil { + level.Info(userLogger).Log("msg", "unable to get meta for block", "partitioned_group_id", partitionedGroupID, "block", b.String()) + continue + } + if meta.Compaction.Level == 1 { + level.Debug(userLogger).Log("msg", "skip level 1 block", "partitioned_group_id", partitionedGroupID, "block", b.String()) + continue + } + partitionInfo, err := GetPartitionInfo(meta) + if err != nil { + level.Warn(userLogger).Log("msg", "failed to get partition info for block", "partitioned_group_id", partitionedGroupID, "block", b.String(), "err", err) + continue + } + if partitionInfo == nil { + level.Warn(userLogger).Log("msg", "unable to get partition info for block", "partitioned_group_id", partitionedGroupID, "block", b.String()) + continue + } + if partitionInfo.PartitionedGroupID == partitionedGroupID { + level.Info(userLogger).Log("msg", "found result block", "partitioned_group_id", partitionedGroupID, "partition_id", partitionInfo.PartitionID, "block", b.String()) + resultBlocks[partitionInfo.PartitionID] = b + } + level.Info(userLogger).Log("msg", fmt.Sprintf("block does not belong to this partitioned group: %d", partitionedGroupID), "partitioned_group_id", partitionInfo.PartitionedGroupID, "partition_id", partitionInfo.PartitionID, "block", b.String()) + } + + return resultBlocks +} + +func (c *BlocksCleaner) validatePartitionedResultBlock(ctx context.Context, userBucket objstore.InstrumentedBucket, userLogger log.Logger, userID string, resultBlock ulid.ULID, partition Partition, partitionedGroupID uint32) error { + meta, err := readMeta(ctx, userBucket, userLogger, resultBlock) + if err != nil { + level.Warn(userLogger).Log("msg", "unable to read meta of result block", "partitioned_group_id", partitionedGroupID, "partition_id", partition.PartitionID, "block", resultBlock.String()) + return err + } + expectedSourceBlocks := partition.getBlocksSet() + if len(expectedSourceBlocks) != len(meta.Compaction.Parents) { + c.compactorPartitionError.WithLabelValues(userID).Inc() + level.Warn(userLogger).Log("msg", "result block has different number of parent blocks as partitioned group info", "partitioned_group_id", partitionedGroupID, "partition_id", partition.PartitionID, "block", resultBlock.String()) + return fmt.Errorf("result block %s has different number of parent blocks as partitioned group info with partitioned group id %d, partition id %d", resultBlock.String(), partitionedGroupID, partition.PartitionID) + } + for _, parentBlock := range meta.Compaction.Parents { + if _, ok := expectedSourceBlocks[parentBlock.ULID]; !ok { + c.compactorPartitionError.WithLabelValues(userID).Inc() + level.Warn(userLogger).Log("msg", "parent blocks of result block does not match partitioned group info", "partitioned_group_id", partitionedGroupID, "partition_id", partition.PartitionID, "block", resultBlock.String()) + return fmt.Errorf("parent blocks of result block %s does not match partitioned group info with partitioned group id %d, partition id %d", resultBlock.String(), partitionedGroupID, partition.PartitionID) + } + } return nil } +func (c *BlocksCleaner) cleanPartitionedGroupInfo(ctx context.Context, userBucket objstore.InstrumentedBucket, userLogger log.Logger, userID string, index *bucketindex.Index) { + var deletePartitionedGroupInfo []string + err := userBucket.Iter(ctx, PartitionedGroupDirectory, func(file string) error { + partitionedGroupInfo, err := ReadPartitionedGroupInfoFile(ctx, userBucket, userLogger, file, c.partitionedGroupInfoReadFailed) + if err != nil { + level.Warn(userLogger).Log("msg", "failed to read partitioned group info", "partitioned_group_info", file) + return nil + } + resultBlocks := c.findResultBlocksForPartitionedGroup(ctx, userBucket, userLogger, index, partitionedGroupInfo) + partitionedGroupID := partitionedGroupInfo.PartitionedGroupID + for _, partition := range partitionedGroupInfo.Partitions { + if _, ok := resultBlocks[partition.PartitionID]; !ok { + level.Info(userLogger).Log("msg", "unable to find result block for partition in partitioned group", "partitioned_group_id", partitionedGroupID, "partition_id", partition.PartitionID) + return nil + } + resultBlock := resultBlocks[partition.PartitionID] + err := c.validatePartitionedResultBlock(ctx, userBucket, userLogger, userID, resultBlock, partition, partitionedGroupID) + if err != nil { + level.Warn(userLogger).Log("msg", "validate result block failed", "partitioned_group_id", partitionedGroupID, "partition_id", partition.PartitionID, "block", resultBlock.String(), "err", err) + return nil + } + level.Info(userLogger).Log("msg", "result block has expected parent blocks", "partitioned_group_id", partitionedGroupID, "partition_id", partition.PartitionID, "block", resultBlock.String()) + } + + // since the partitioned group were all complete, we can make sure + // all source blocks would be deleted. + blocks := partitionedGroupInfo.getAllBlocks() + for _, blockID := range blocks { + metaExists, err := userBucket.Exists(ctx, path.Join(blockID.String(), metadata.MetaFilename)) + if err != nil { + level.Info(userLogger).Log("msg", "block already deleted", "partitioned_group_id", partitionedGroupID, "block", blockID.String()) + continue + } + if metaExists { + deletionMarkerExists, err := userBucket.Exists(ctx, path.Join(blockID.String(), metadata.DeletionMarkFilename)) + if err == nil && deletionMarkerExists { + level.Info(userLogger).Log("msg", "block already marked for deletion", "partitioned_group_id", partitionedGroupID, "block", blockID.String()) + continue + } + if err := block.MarkForDeletion(ctx, userLogger, userBucket, blockID, "delete block during partitioned group completion check", c.blocksMarkedForDeletion); err != nil { + level.Warn(userLogger).Log("msg", "unable to mark block for deletion", "partitioned_group_id", partitionedGroupID, "block", blockID.String()) + // if one block can not be marked for deletion, we should + // skip delete this partitioned group. next iteration + // would try it again. + return nil + } + level.Info(userLogger).Log("msg", "marked block for deletion during partitioned group info clean up", "partitioned_group_id", partitionedGroupID, "block", blockID.String()) + } + } + level.Info(userLogger).Log("msg", "partitioned group info can be cleaned up", "partitioned_group_id", partitionedGroupID) + deletePartitionedGroupInfo = append(deletePartitionedGroupInfo, file) + return nil + }) + if err != nil { + level.Warn(userLogger).Log("msg", "error return when going through partitioned group directory", "err", err) + } + for _, partitionedGroupInfoFile := range deletePartitionedGroupInfo { + if err := userBucket.Delete(ctx, partitionedGroupInfoFile); err != nil { + level.Warn(userLogger).Log("msg", "failed to delete partitioned group info", "partitioned_group_info", partitionedGroupInfoFile, "err", err) + } else { + level.Info(userLogger).Log("msg", "deleted partitioned group info", "partitioned_group_info", partitionedGroupInfoFile) + } + } +} + // cleanUserPartialBlocks delete partial blocks which are safe to be deleted. The provided partials map // and index are updated accordingly. func (c *BlocksCleaner) cleanUserPartialBlocks(ctx context.Context, partials map[ulid.ULID]error, idx *bucketindex.Index, userBucket objstore.InstrumentedBucket, userLogger log.Logger) { @@ -536,3 +702,20 @@ func listBlocksOutsideRetentionPeriod(idx *bucketindex.Index, threshold time.Tim return } + +func readMeta(ctx context.Context, userBucket objstore.InstrumentedBucket, userLogger log.Logger, blockID ulid.ULID) (*metadata.Meta, error) { + metaReader, err := userBucket.Get(ctx, path.Join(blockID.String(), block.MetaFilename)) + if err != nil { + return nil, err + } + defer runutil.CloseWithLogOnErr(userLogger, metaReader, "close meta reader") + b, err := io.ReadAll(metaReader) + if err != nil { + return nil, err + } + meta := metadata.Meta{} + if err = json.Unmarshal(b, &meta); err != nil { + return nil, err + } + return &meta, nil +} diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index 14e496d17f9..ddbc35682e2 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -3,6 +3,8 @@ package compactor import ( "context" "crypto/rand" + "encoding/json" + "errors" "fmt" "path" "strings" @@ -16,12 +18,14 @@ import ( prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" "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/cortexproject/cortex/pkg/storage/tsdb" "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" + "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/services" ) @@ -80,7 +84,7 @@ func TestBlockCleaner_KeyPermissionDenied(t *testing.T) { scanner := tsdb.NewUsersScanner(mbucket, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, mbucket, scanner, cfgProvider, logger, nil) + cleaner := NewBlocksCleaner(cfg, mbucket, scanner, cfgProvider, logger, nil, prometheus.NewCounter(prometheus.CounterOpts{})) // Clean User with no error cleaner.bucketClient = bkt @@ -138,7 +142,7 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions createDeletionMark(t, bucketClient, "user-1", block4, now.Add(-deletionDelay).Add(time.Hour)) // Partial block hasn't reached the deletion threshold yet. createDeletionMark(t, bucketClient, "user-1", block5, now.Add(-deletionDelay).Add(-time.Hour)) // Partial block reached the deletion threshold. require.NoError(t, bucketClient.Delete(ctx, path.Join("user-1", block6.String(), metadata.MetaFilename))) // Partial block without deletion mark. - createBlockVisitMarker(t, bucketClient, "user-1", block11) // Partial block only has visit marker. + createLegacyBlockVisitMarker(t, bucketClient, "user-1", block11) // Partial block only has visit marker. createDeletionMark(t, bucketClient, "user-2", block7, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. // Blocks for user-3, marked for deletion. @@ -177,7 +181,7 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, prometheus.NewCounter(prometheus.CounterOpts{})) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -203,7 +207,7 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions // Should not delete a partial block without deletion mark. {path: path.Join("user-1", block6.String(), "index"), expectedExists: true}, // Should delete a partial block with only visit marker. - {path: path.Join("user-1", block11.String(), BlockVisitMarkerFile), expectedExists: false}, + {path: path.Join("user-1", GetBlockVisitMarkerFile(block11.String(), 0)), expectedExists: false}, // Should completely delete blocks for user-3, marked for deletion {path: path.Join("user-3", block9.String(), metadata.MetaFilename), expectedExists: false}, {path: path.Join("user-3", block9.String(), "index"), expectedExists: false}, @@ -310,7 +314,7 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { createDeletionMark(t, bucketClient, userID, block4, now.Add(-deletionDelay).Add(-time.Hour)) // To emulate a failure deleting a block, we wrap the bucket client in a mocked one. - bucketClient = &cortex_testutil.MockBucketFailure{ + bucketClient = &mockBucketFailure{ Bucket: bucketClient, DeleteFailures: []string{path.Join(userID, block3.String(), metadata.MetaFilename)}, } @@ -325,7 +329,7 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil, prometheus.NewCounter(prometheus.CounterOpts{})) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -385,7 +389,7 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil, prometheus.NewCounter(prometheus.CounterOpts{})) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -439,7 +443,7 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, prometheus.NewCounter(prometheus.CounterOpts{})) require.NoError(t, cleaner.cleanUsers(ctx, true)) assert.NoError(t, prom_testutil.GatherAndCompare(reg, strings.NewReader(` @@ -570,7 +574,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, prometheus.NewCounter(prometheus.CounterOpts{})) assertBlockExists := func(user string, block ulid.ULID, expectExists bool) { exists, err := bucketClient.Exists(ctx, path.Join(user, block.String(), metadata.MetaFilename)) @@ -717,6 +721,19 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { } } +type mockBucketFailure struct { + objstore.Bucket + + DeleteFailures []string +} + +func (m *mockBucketFailure) Delete(ctx context.Context, name string) error { + if util.StringsContain(m.DeleteFailures, name) { + return errors.New("mocked delete failure") + } + return m.Bucket.Delete(ctx, name) +} + type mockConfigProvider struct { userRetentionPeriods map[string]time.Duration } @@ -745,3 +762,25 @@ func (m *mockConfigProvider) S3SSEKMSKeyID(userID string) string { func (m *mockConfigProvider) S3SSEKMSEncryptionContext(userID string) string { return "" } + +func createLegacyBlockVisitMarker(t *testing.T, bkt objstore.Bucket, userID string, blockID ulid.ULID) { + content := mockBlockVisitMarker() + markPath := path.Join(userID, GetBlockVisitMarkerFile(blockID.String(), 0)) + + require.NoError(t, bkt.Upload(context.Background(), markPath, strings.NewReader(content))) +} + +func mockBlockVisitMarker() string { + blockVisitMarker := BlockVisitMarker{ + CompactorID: "dummy", + VisitTime: time.Now().Unix(), + Version: 1, + } + + content, err := json.Marshal(blockVisitMarker) + if err != nil { + panic("failed to marshal mocked block visit marker") + } + + return string(content) +} diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 3f6757fbbca..b32d5149bf0 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -2,6 +2,7 @@ package compactor import ( "context" + crypto_rand "crypto/rand" "flag" "fmt" "hash/fnv" @@ -13,6 +14,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -39,8 +41,10 @@ const ( // ringKey is the key under which we store the compactors ring in the KVStore. ringKey = "compactor" - blocksMarkedForDeletionName = "cortex_compactor_blocks_marked_for_deletion_total" - blocksMarkedForDeletionHelp = "Total number of blocks marked for deletion in compactor." + blocksMarkedForDeletionName = "cortex_compactor_blocks_marked_for_deletion_total" + blocksMarkedForDeletionHelp = "Total number of blocks marked for deletion in compactor." + compactorPartitionErrorCountName = "cortex_compactor_partition_error" + compactorPartitionErrorCountHelp = "Count of errors happened during partitioning compaction." ) var ( @@ -51,7 +55,7 @@ var ( errInvalidShardingStrategy = errors.New("invalid sharding strategy") errInvalidTenantShardSize = errors.New("invalid tenant shard size, the value must be greater than 0") - DefaultBlocksGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, _ prometheus.Gauge, _ prometheus.Counter, _ prometheus.Counter, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string, _ *compact.GatherNoCompactionMarkFilter) compact.Grouper { + DefaultBlocksGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, _ prometheus.Gauge, _ prometheus.Counter, _ prometheus.Counter, _ prometheus.Counter, _ prometheus.Counter, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string, _ *compact.GatherNoCompactionMarkFilter) compact.Grouper { return compact.NewDefaultGrouper( logger, bkt, @@ -66,7 +70,7 @@ var ( cfg.BlocksFetchConcurrency) } - ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter) compact.Grouper { + ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, partitionedGroupInfoReadFailed prometheus.Counter, partitionedGroupInfoWriteFailed prometheus.Counter, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter) compact.Grouper { return NewShuffleShardingGrouper( ctx, logger, @@ -91,7 +95,10 @@ var ( cfg.BlockVisitMarkerTimeout, blockVisitMarkerReadFailed, blockVisitMarkerWriteFailed, - noCompactionMarkFilter.NoCompactMarkedBlocks) + partitionedGroupInfoReadFailed, + partitionedGroupInfoWriteFailed, + noCompactionMarkFilter.NoCompactMarkedBlocks, + ) } DefaultBlocksCompactorFactory = func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { @@ -119,6 +126,14 @@ var ( } return compactor, plannerFactory, nil } + + DefaultBlockDeletableCheckerFactory = func(_ context.Context, _ objstore.InstrumentedBucket, _ log.Logger, _ prometheus.Counter, _ prometheus.Counter) compact.BlockDeletableChecker { + return compact.DefaultBlockDeletableChecker{} + } + + PartitionCompactionBlockDeletableCheckerFactory = func(ctx context.Context, bkt objstore.InstrumentedBucket, logger log.Logger, blockVisitMarkerReadFailed prometheus.Counter, partitionedGroupInfoWriteFailed prometheus.Counter) compact.BlockDeletableChecker { + return NewPartitionCompactionBlockDeletableChecker(ctx, bkt, logger, blockVisitMarkerReadFailed, partitionedGroupInfoWriteFailed) + } ) // BlocksGrouperFactory builds and returns the grouper to use to compact a tenant's blocks. @@ -134,6 +149,8 @@ type BlocksGrouperFactory func( remainingPlannedCompactions prometheus.Gauge, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, + partitionedGroupInfoReadFailed prometheus.Counter, + partitionedGroupInfoWriteFailed prometheus.Counter, ring *ring.Ring, ringLifecycler *ring.Lifecycler, limit Limits, @@ -160,9 +177,19 @@ type PlannerFactory func( blockVisitMarkerWriteFailed prometheus.Counter, ) compact.Planner +type BlockDeletableCheckerFactory func( + ctx context.Context, + bkt objstore.InstrumentedBucket, + logger log.Logger, + blockVisitMarkerReadFailed prometheus.Counter, + partitionedGroupInfoReadFailed prometheus.Counter, +) compact.BlockDeletableChecker + // Limits defines limits used by the Compactor. type Limits interface { CompactorTenantShardSize(userID string) int + CompactorPartitionIndexSizeLimitInBytes(userID string) int64 + CompactorPartitionSeriesCountLimit(userID string) int64 } // Config holds the Compactor config. @@ -311,6 +338,8 @@ type Compactor struct { blocksPlannerFactory PlannerFactory + blockDeletableCheckerFactory BlockDeletableCheckerFactory + // Client used to run operations on the bucket storing blocks. bucketClient objstore.Bucket @@ -321,22 +350,24 @@ type Compactor struct { ringSubservicesWatcher *services.FailureWatcher // Metrics. - compactionRunsStarted prometheus.Counter - compactionRunsInterrupted prometheus.Counter - compactionRunsCompleted prometheus.Counter - compactionRunsFailed prometheus.Counter - compactionRunsLastSuccess prometheus.Gauge - compactionRunDiscoveredTenants prometheus.Gauge - compactionRunSkippedTenants prometheus.Gauge - compactionRunSucceededTenants prometheus.Gauge - compactionRunFailedTenants prometheus.Gauge - compactionRunInterval prometheus.Gauge - blocksMarkedForDeletion prometheus.Counter - blocksMarkedForNoCompaction prometheus.Counter - garbageCollectedBlocks prometheus.Counter - remainingPlannedCompactions prometheus.Gauge - blockVisitMarkerReadFailed prometheus.Counter - blockVisitMarkerWriteFailed prometheus.Counter + CompactionRunsStarted prometheus.Counter + CompactionRunsInterrupted prometheus.Counter + CompactionRunsCompleted prometheus.Counter + CompactionRunsFailed prometheus.Counter + CompactionRunsLastSuccess prometheus.Gauge + CompactionRunDiscoveredTenants prometheus.Gauge + CompactionRunSkippedTenants prometheus.Gauge + CompactionRunSucceededTenants prometheus.Gauge + CompactionRunFailedTenants prometheus.Gauge + CompactionRunInterval prometheus.Gauge + BlocksMarkedForDeletion prometheus.Counter + BlocksMarkedForNoCompaction prometheus.Counter + GarbageCollectedBlocks prometheus.Counter + RemainingPlannedCompactions prometheus.Gauge + BlockVisitMarkerReadFailed prometheus.Counter + BlockVisitMarkerWriteFailed prometheus.Counter + PartitionedGroupInfoReadFailed prometheus.Counter + PartitionedGroupInfoWriteFailed prometheus.Counter // TSDB syncer metrics syncerMetrics *syncerMetrics @@ -366,7 +397,14 @@ func NewCompactor(compactorCfg Config, storageCfg cortex_tsdb.BlocksStorageConfi } } - cortexCompactor, err := newCompactor(compactorCfg, storageCfg, logger, registerer, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, limits) + var blockDeletableCheckerFactory BlockDeletableCheckerFactory + if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle { + blockDeletableCheckerFactory = PartitionCompactionBlockDeletableCheckerFactory + } else { + blockDeletableCheckerFactory = DefaultBlockDeletableCheckerFactory + } + + cortexCompactor, err := newCompactor(compactorCfg, storageCfg, logger, registerer, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, blockDeletableCheckerFactory, limits) if err != nil { return nil, errors.Wrap(err, "failed to create Cortex blocks compactor") } @@ -382,6 +420,7 @@ func newCompactor( bucketClientFactory func(ctx context.Context) (objstore.Bucket, error), blocksGrouperFactory BlocksGrouperFactory, blocksCompactorFactory BlocksCompactorFactory, + blockDeletableCheckerFactory BlockDeletableCheckerFactory, limits *validation.Overrides, ) (*Compactor, error) { var remainingPlannedCompactions prometheus.Gauge @@ -392,79 +431,88 @@ func newCompactor( }) } c := &Compactor{ - compactorCfg: compactorCfg, - storageCfg: storageCfg, - parentLogger: logger, - logger: log.With(logger, "component", "compactor"), - registerer: registerer, - syncerMetrics: newSyncerMetrics(registerer), - bucketClientFactory: bucketClientFactory, - blocksGrouperFactory: blocksGrouperFactory, - blocksCompactorFactory: blocksCompactorFactory, - allowedTenants: util.NewAllowedTenants(compactorCfg.EnabledTenants, compactorCfg.DisabledTenants), - - compactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + compactorCfg: compactorCfg, + storageCfg: storageCfg, + parentLogger: logger, + logger: log.With(logger, "component", "compactor"), + registerer: registerer, + syncerMetrics: newSyncerMetrics(registerer), + bucketClientFactory: bucketClientFactory, + blocksGrouperFactory: blocksGrouperFactory, + blocksCompactorFactory: blocksCompactorFactory, + blockDeletableCheckerFactory: blockDeletableCheckerFactory, + allowedTenants: util.NewAllowedTenants(compactorCfg.EnabledTenants, compactorCfg.DisabledTenants), + + CompactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_runs_started_total", Help: "Total number of compaction runs started.", }), - compactionRunsInterrupted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + CompactionRunsInterrupted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_runs_interrupted_total", Help: "Total number of compaction runs interrupted.", }), - compactionRunsCompleted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + CompactionRunsCompleted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_runs_completed_total", Help: "Total number of compaction runs successfully completed.", }), - compactionRunsFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + CompactionRunsFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_runs_failed_total", Help: "Total number of compaction runs failed.", }), - compactionRunsLastSuccess: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunsLastSuccess: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_last_successful_run_timestamp_seconds", Help: "Unix timestamp of the last successful compaction run.", }), - compactionRunDiscoveredTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunDiscoveredTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_tenants_discovered", Help: "Number of tenants discovered during the current compaction run. Reset to 0 when compactor is idle.", }), - compactionRunSkippedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunSkippedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_tenants_skipped", Help: "Number of tenants skipped during the current compaction run. Reset to 0 when compactor is idle.", }), - compactionRunSucceededTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunSucceededTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_tenants_processing_succeeded", Help: "Number of tenants successfully processed during the current compaction run. Reset to 0 when compactor is idle.", }), - compactionRunFailedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunFailedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_tenants_processing_failed", Help: "Number of tenants failed processing during the current compaction run. Reset to 0 when compactor is idle.", }), - compactionRunInterval: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunInterval: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_compaction_interval_seconds", Help: "The configured interval on which compaction is run in seconds. Useful when compared to the last successful run metric to accurately detect multiple failed compaction runs.", }), - blocksMarkedForDeletion: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + BlocksMarkedForDeletion: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: blocksMarkedForDeletionName, Help: blocksMarkedForDeletionHelp, ConstLabels: prometheus.Labels{"reason": "compaction"}, }), - blocksMarkedForNoCompaction: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + BlocksMarkedForNoCompaction: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_blocks_marked_for_no_compaction_total", Help: "Total number of blocks marked for no compact during a compaction run.", }), - garbageCollectedBlocks: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + GarbageCollectedBlocks: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_garbage_collected_blocks_total", Help: "Total number of blocks marked for deletion by compactor.", }), - blockVisitMarkerReadFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + BlockVisitMarkerReadFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_block_visit_marker_read_failed", Help: "Number of block visit marker file failed to be read.", }), - blockVisitMarkerWriteFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + BlockVisitMarkerWriteFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_block_visit_marker_write_failed", Help: "Number of block visit marker file failed to be written.", }), - remainingPlannedCompactions: remainingPlannedCompactions, + PartitionedGroupInfoReadFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_partitioned_group_info_read_failed", + Help: "Number of partitioned group info file failed to be read.", + }), + PartitionedGroupInfoWriteFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_partitioned_group_info_write_failed", + Help: "Number of partitioned group info file failed to be written.", + }), + RemainingPlannedCompactions: remainingPlannedCompactions, limits: limits, } @@ -478,7 +526,7 @@ func newCompactor( c.Service = services.NewBasicService(c.starting, c.running, c.stopping) // The last successful compaction run metric is exposed as seconds since epoch, so we need to use seconds for this metric. - c.compactionRunInterval.Set(c.compactorCfg.CompactionInterval.Seconds()) + c.CompactionRunInterval.Set(c.compactorCfg.CompactionInterval.Seconds()) return c, nil } @@ -512,7 +560,7 @@ func (c *Compactor) starting(ctx context.Context) error { CleanupConcurrency: c.compactorCfg.CleanupConcurrency, BlockDeletionMarksMigrationEnabled: c.compactorCfg.BlockDeletionMarksMigrationEnabled, TenantCleanupDelay: c.compactorCfg.TenantCleanupDelay, - }, c.bucketClient, c.usersScanner, c.limits, c.parentLogger, c.registerer) + }, c.bucketClient, c.usersScanner, c.limits, c.parentLogger, c.registerer, c.PartitionedGroupInfoReadFailed) // Initialize the compactors ring if sharding is enabled. if c.compactorCfg.ShardingEnabled { @@ -594,12 +642,15 @@ func (c *Compactor) running(ctx context.Context) error { // Run an initial compaction before starting the interval. 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(util.DurationWithPositiveJitter(c.compactorCfg.CompactionInterval, 0.1)) c.compactUsers(ctx) case <-ctx.Done(): return nil @@ -610,43 +661,38 @@ 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() + 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 { - c.compactionRunsCompleted.Inc() - c.compactionRunsLastSuccess.SetToCurrentTime() - } - if interrupted { - c.compactionRunsInterrupted.Inc() - } - if failed { - c.compactionRunsFailed.Inc() + if succeeded && compactionErrorCount == 0 { + c.CompactionRunsCompleted.Inc() + c.CompactionRunsLastSuccess.SetToCurrentTime() + } else if interrupted { + c.CompactionRunsInterrupted.Inc() + } else { + c.CompactionRunsFailed.Inc() } // Reset progress metrics once done. - c.compactionRunDiscoveredTenants.Set(0) - c.compactionRunSkippedTenants.Set(0) - c.compactionRunSucceededTenants.Set(0) - c.compactionRunFailedTenants.Set(0) + c.CompactionRunDiscoveredTenants.Set(0) + c.CompactionRunSkippedTenants.Set(0) + c.CompactionRunSucceededTenants.Set(0) + c.CompactionRunFailedTenants.Set(0) }() 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 } level.Info(c.logger).Log("msg", "discovered users from bucket", "users", len(users)) - c.compactionRunDiscoveredTenants.Set(float64(len(users))) + c.CompactionRunDiscoveredTenants.Set(float64(len(users))) // When starting multiple compactor replicas nearly at the same time, running in a cluster with // a large number of tenants, we may end up in a situation where the 1st user is compacted by @@ -661,17 +707,17 @@ func (c *Compactor) compactUsers(ctx context.Context) { // Ensure the context has not been canceled (ie. compactor shutdown has been triggered). if ctx.Err() != nil { interrupted = true - level.Info(c.logger).Log("msg", "interrupting compaction of user blocks", "user", userID) + level.Info(c.logger).Log("msg", "interrupting compaction of user blocks", "err", err) return } // Ensure the user ID belongs to our shard. if owned, err := c.ownUserForCompaction(userID); err != nil { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Warn(c.logger).Log("msg", "unable to check if user is owned by this shard", "user", userID, "err", err) continue } else if !owned { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Debug(c.logger).Log("msg", "skipping user because it is not owned by this shard", "user", userID) continue } @@ -679,7 +725,7 @@ func (c *Compactor) compactUsers(ctx context.Context) { // Skipping compaction if the bucket index failed to sync due to CMK errors. if idxs, err := bucketindex.ReadSyncStatus(ctx, c.bucketClient, userID, util_log.WithUserID(userID, c.logger)); err == nil { if idxs.Status == bucketindex.CustomerManagedKeyError { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Info(c.logger).Log("msg", "skipping compactUser due CustomerManagedKeyError", "user", userID) continue } @@ -688,11 +734,11 @@ func (c *Compactor) compactUsers(ctx context.Context) { ownedUsers[userID] = struct{}{} if markedForDeletion, err := cortex_tsdb.TenantDeletionMarkExists(ctx, c.bucketClient, userID); err != nil { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Warn(c.logger).Log("msg", "unable to check if user is marked for deletion", "user", userID, "err", err) continue } else if markedForDeletion { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Debug(c.logger).Log("msg", "skipping user because it is marked for deletion", "user", userID) continue } @@ -700,20 +746,13 @@ func (c *Compactor) compactUsers(ctx context.Context) { level.Info(c.logger).Log("msg", "starting compaction of user blocks", "user", userID) if err = c.compactUserWithRetries(ctx, userID); err != nil { - // TODO: patch thanos error types to support errors.Is(err, context.Canceled) here - if ctx.Err() != nil && ctx.Err() == context.Canceled { - interrupted = true - level.Info(c.logger).Log("msg", "interrupting compaction of user blocks", "user", userID) - return - } - - c.compactionRunFailedTenants.Inc() - failed = true + c.CompactionRunFailedTenants.Inc() + compactionErrorCount++ level.Error(c.logger).Log("msg", "failed to compact user blocks", "user", userID, "err", err) continue } - c.compactionRunSucceededTenants.Inc() + c.CompactionRunSucceededTenants.Inc() level.Info(c.logger).Log("msg", "successfully compacted user blocks", "user", userID) } @@ -743,6 +782,8 @@ func (c *Compactor) compactUsers(ctx context.Context) { } } } + + succeeded = true } func (c *Compactor) compactUserWithRetries(ctx context.Context, userID string) error { @@ -773,10 +814,11 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { defer c.syncerMetrics.gatherThanosSyncerMetrics(reg) ulogger := util_log.WithUserID(userID, c.logger) + ulogger = util_log.WithExecutionID(ulid.MustNew(ulid.Now(), crypto_rand.Reader).String(), ulogger) // 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) + deduplicateBlocksFilter := &DisabledDeduplicateFilter{} // 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. @@ -818,21 +860,30 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { fetcher, deduplicateBlocksFilter, ignoreDeletionMarkFilter, - c.blocksMarkedForDeletion, - c.garbageCollectedBlocks, + c.BlocksMarkedForDeletion, + c.GarbageCollectedBlocks, ) if err != nil { return errors.Wrap(err, "failed to create syncer") } - currentCtx, cancel := context.WithCancel(ctx) - defer cancel() - compactor, err := compact.NewBucketCompactor( + blockDeletableChecker := c.blockDeletableCheckerFactory(ctx, bucket, ulogger, c.BlockVisitMarkerReadFailed, c.PartitionedGroupInfoReadFailed) + shardedCompactionLifecycleCallback := ShardedCompactionLifecycleCallback{ + ctx: ctx, + userBucket: bucket, + logger: ulogger, + metaSyncConcurrency: c.compactorCfg.MetaSyncConcurrency, + compactDir: c.compactDirForUser(userID), + partitionedGroupInfoReadFailed: c.PartitionedGroupInfoReadFailed, + } + compactor, err := compact.NewBucketCompactorWithCheckerAndCallback( ulogger, syncer, - c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed, c.ring, c.ringLifecycler, c.limits, userID, noCompactMarkerFilter), - c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed), + c.blocksGrouperFactory(ctx, c.compactorCfg, bucket, ulogger, reg, c.BlocksMarkedForDeletion, c.BlocksMarkedForNoCompaction, c.GarbageCollectedBlocks, c.RemainingPlannedCompactions, c.BlockVisitMarkerReadFailed, c.BlockVisitMarkerWriteFailed, c.PartitionedGroupInfoReadFailed, c.PartitionedGroupInfoWriteFailed, c.ring, c.ringLifecycler, c.limits, userID, noCompactMarkerFilter), + c.blocksPlannerFactory(ctx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, c.BlockVisitMarkerReadFailed, c.BlockVisitMarkerWriteFailed), c.blocksCompactor, + blockDeletableChecker, + shardedCompactionLifecycleCallback, c.compactDirForUser(userID), bucket, c.compactorCfg.CompactionConcurrency, @@ -982,3 +1033,15 @@ func (c *Compactor) listTenantsWithMetaSyncDirectories() map[string]struct{} { return result } + +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_test.go b/pkg/compactor/compactor_test.go index 36ae05bda9f..6a79845a6da 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -177,14 +177,15 @@ func TestCompactor_SkipCompactionWhenCmkError(t *testing.T) { bucketClient.MockUpload(userID+"/bucket-index-sync-status.json", nil) bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) bucketClient.MockExists(path.Join(userID, cortex_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter(userID+"/"+PartitionedGroupDirectory, nil, nil) cfg := prepareConfig() - c, _, _, logs, _ := prepare(t, cfg, bucketClient, nil) + c, _, _, logs, _ := prepare(t, cfg, bucketClient, nil, nil) require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -198,17 +199,17 @@ func TestCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{}, nil) cfg := prepareConfig() - c, _, _, logs, registry := prepare(t, cfg, bucketClient, nil) + c, _, _, logs, registry := prepare(t, cfg, bucketClient, nil, nil) require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + 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.Equal(t, prom_testutil.ToFloat64(c.CompactionRunInterval), cfg.CompactionInterval.Seconds()) assert.Equal(t, []string{ `level=info component=cleaner msg="started blocks cleanup and maintenance"`, @@ -348,12 +349,12 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", nil, errors.New("failed to iterate the bucket")) - c, _, _, logs, registry := prepare(t, prepareConfig(), bucketClient, nil) + c, _, _, logs, registry := prepare(t, prepareConfig(), bucketClient, nil, nil) require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) // Wait until all retry attempts have completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsFailed) + return prom_testutil.ToFloat64(c.CompactionRunsFailed) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -496,6 +497,7 @@ func TestCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASingleTenant( t.Parallel() userID := "test-user" + partitionedGroupID := getPartitionedGroupID(userID) bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{userID}, nil) bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D/meta.json", userID + "/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) @@ -504,25 +506,28 @@ func TestCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASingleTenant( 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/visit-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.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/visit-mark.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/visit-mark.json", "", nil) - bucketClient.MockUpload(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/visit-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 := prepare(t, prepareConfig(), bucketClient, nil) + c, _, tsdbPlannerMock, _, registry := prepare(t, prepareConfig(), 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, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsFailed) + return prom_testutil.ToFloat64(c.CompactionRunsFailed) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -547,6 +552,7 @@ func TestCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASingleTenant( } func TestCompactor_ShouldCompactAndRemoveUserFolder(t *testing.T) { + partitionedGroupID1 := getPartitionedGroupID("user-1") bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockExists(path.Join("user-1", cortex_tsdb.TenantDeletionMarkPath), false, nil) @@ -555,19 +561,22 @@ func TestCompactor_ShouldCompactAndRemoveUserFolder(t *testing.T) { 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/visit-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/visit-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, _, _ := prepare(t, prepareConfig(), bucketClient, nil) + c, _, tsdbPlanner, _, _ := prepare(t, prepareConfig(), bucketClient, nil, nil) // Make sure the user folder is created and is being used // This will be called during compaction @@ -580,7 +589,7 @@ func TestCompactor_ShouldCompactAndRemoveUserFolder(t *testing.T) { // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) _, err := os.Stat(c.compactDirForUser("user-1")) @@ -590,6 +599,9 @@ func TestCompactor_ShouldCompactAndRemoveUserFolder(t *testing.T) { func TestCompactor_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) @@ -602,20 +614,20 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { 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/visit-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/visit-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/visit-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/visit-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) @@ -626,8 +638,14 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { 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 := prepare(t, prepareConfig(), bucketClient, nil) + c, _, tsdbPlanner, logs, registry := prepare(t, prepareConfig(), 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 @@ -639,7 +657,7 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -652,24 +670,48 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { assert.ElementsMatch(t, []string{ `level=info component=cleaner msg="started blocks cleanup and maintenance"`, `level=info component=cleaner org_id=user-1 msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-1 msg="completed blocks cleanup and maintenance"`, `level=info component=cleaner org_id=user-2 msg="started blocks cleanup and maintenance"`, + `level=info component=cleaner org_id=user-1 msg="finish reading index"`, + `level=info component=cleaner org_id=user-1 msg="finish iterating markers" iteration_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish getting deleted blocks" old_blocks_count=0 deleted_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish getting new deletion markers" discovered_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish iterating blocks" iteration_count=2`, + `level=info component=cleaner org_id=user-1 msg="finish adding blocks" old_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish updating block entries" discovered_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish updating index"`, + `level=info component=cleaner org_id=user-1 msg="finish getting blocks to be deleted"`, + `level=info component=cleaner org_id=user-1 msg="finish deleting blocks"`, + `level=info component=cleaner org_id=user-2 msg="finish reading index"`, + `level=info component=cleaner org_id=user-2 msg="finish iterating markers" iteration_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish getting deleted blocks" old_blocks_count=0 deleted_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish getting new deletion markers" discovered_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish iterating blocks" iteration_count=2`, + `level=info component=cleaner org_id=user-2 msg="finish adding blocks" old_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish updating block entries" discovered_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish updating index"`, + `level=info component=cleaner org_id=user-2 msg="finish getting blocks to be deleted"`, + `level=info component=cleaner org_id=user-2 msg="finish deleting blocks"`, + `level=info component=cleaner org_id=user-1 msg="finish writing new index"`, + `level=info component=cleaner org_id=user-1 msg="finish cleaning partitioned group info files"`, + `level=info component=cleaner org_id=user-2 msg="finish writing new index"`, + `level=info component=cleaner org_id=user-2 msg="finish cleaning partitioned group info files"`, `level=info component=cleaner org_id=user-2 msg="completed blocks cleanup and maintenance"`, + `level=info component=cleaner org_id=user-1 msg="completed blocks cleanup and maintenance"`, `level=info component=cleaner msg="successfully completed blocks cleanup and maintenance"`, `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`, + `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. @@ -765,14 +807,15 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { 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 := prepare(t, cfg, bucketClient, nil) + c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient, nil, nil) require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -783,9 +826,21 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { assert.ElementsMatch(t, []string{ `level=info component=cleaner msg="started blocks cleanup and maintenance"`, `level=info component=cleaner org_id=user-1 msg="started blocks cleanup and maintenance"`, + `level=info component=cleaner org_id=user-1 msg="finish reading index"`, + `level=info component=cleaner org_id=user-1 msg="finish iterating markers" iteration_count=2`, + `level=info component=cleaner org_id=user-1 msg="finish getting deleted blocks" old_blocks_count=0 deleted_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish getting new deletion markers" discovered_blocks_count=2 deletion_markers_count=2`, + `level=info component=cleaner org_id=user-1 msg="finish iterating blocks" iteration_count=2`, + `level=info component=cleaner org_id=user-1 msg="finish adding blocks" old_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish updating block entries" discovered_blocks_count=2 new_blocks_count=2`, + `level=info component=cleaner org_id=user-1 msg="finish updating index"`, + `level=info component=cleaner org_id=user-1 msg="finish getting blocks to be deleted"`, `level=debug component=cleaner org_id=user-1 msg="deleted file" file=01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json bucket=mock`, `level=debug component=cleaner org_id=user-1 msg="deleted file" file=01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json bucket=mock`, `level=info component=cleaner org_id=user-1 msg="deleted block marked for deletion" block=01DTW0ZCPDDNV4BV83Q2SV4QAZ`, + `level=info component=cleaner org_id=user-1 msg="finish deleting blocks"`, + `level=info component=cleaner org_id=user-1 msg="finish writing new index"`, + `level=info component=cleaner org_id=user-1 msg="finish cleaning partitioned group info files"`, `level=info component=cleaner org_id=user-1 msg="completed blocks cleanup and maintenance"`, `level=info component=cleaner msg="successfully completed blocks cleanup and maintenance"`, `level=info component=compactor msg="discovering users from bucket"`, @@ -853,6 +908,8 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { func TestCompactor_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) @@ -865,25 +922,25 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForSkipCompact(t *testing.T) { 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/visit-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.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/visit-mark.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/visit-mark.json", "", nil) - bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/visit-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/visit-mark.json", "", nil) - bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/visit-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/visit-mark.json", "", nil) - bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/visit-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) @@ -895,15 +952,21 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForSkipCompact(t *testing.T) { 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 := prepare(t, prepareConfig(), bucketClient, nil) + c, _, tsdbPlanner, _, registry := prepare(t, prepareConfig(), 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, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -929,6 +992,7 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) 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) @@ -940,16 +1004,19 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) 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/visit-mark.json", "", nil) - bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/visit-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.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 := prepare(t, cfg, bucketClient, nil) + c, _, tsdbPlanner, logs, registry := prepare(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 @@ -961,7 +1028,7 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -1048,7 +1115,7 @@ func TestCompactor_ShouldSkipOutOrOrderBlocks(t *testing.T) { cfg := prepareConfig() cfg.SkipBlocksWithOutOfOrderChunksEnabled = true - c, tsdbCompac, tsdbPlanner, _, registry := prepare(t, cfg, bucketClient, nil) + c, tsdbCompac, tsdbPlanner, _, registry := prepare(t, cfg, bucketClient, nil, nil) tsdbCompac.On("CompactWithBlockPopulator", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(b1, nil) @@ -1091,6 +1158,8 @@ func TestCompactor_ShouldSkipOutOrOrderBlocks(t *testing.T) { func TestCompactor_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) @@ -1103,24 +1172,24 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni 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/visit-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.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/visit-mark.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/visit-mark.json", "", nil) - bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/visit-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/visit-mark.json", "", nil) - bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/visit-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/visit-mark.json", "", nil) - bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/visit-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) @@ -1129,6 +1198,12 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni 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()) }) @@ -1139,7 +1214,7 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni cfg.ShardingRing.InstanceAddr = "1.2.3.4" cfg.ShardingRing.KVStore.Mock = ringStore - c, _, tsdbPlanner, logs, _ := prepare(t, cfg, bucketClient, nil) + c, _, tsdbPlanner, logs, _ := prepare(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 @@ -1151,7 +1226,7 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni // Wait until a run has completed. cortex_testutil.Poll(t, 5*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -1164,24 +1239,48 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=cleaner msg="started blocks cleanup and maintenance"`, `level=info component=cleaner org_id=user-1 msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-1 msg="completed blocks cleanup and maintenance"`, `level=info component=cleaner org_id=user-2 msg="started blocks cleanup and maintenance"`, + `level=info component=cleaner org_id=user-1 msg="finish reading index"`, + `level=info component=cleaner org_id=user-2 msg="finish reading index"`, + `level=info component=cleaner org_id=user-2 msg="finish iterating markers" iteration_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish getting deleted blocks" old_blocks_count=0 deleted_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish getting new deletion markers" discovered_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish iterating blocks" iteration_count=2`, + `level=info component=cleaner org_id=user-2 msg="finish adding blocks" old_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish updating block entries" discovered_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-2 msg="finish updating index"`, + `level=info component=cleaner org_id=user-2 msg="finish getting blocks to be deleted"`, + `level=info component=cleaner org_id=user-2 msg="finish deleting blocks"`, + `level=info component=cleaner org_id=user-1 msg="finish iterating markers" iteration_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish getting deleted blocks" old_blocks_count=0 deleted_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish getting new deletion markers" discovered_blocks_count=0 deletion_markers_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish iterating blocks" iteration_count=2`, + `level=info component=cleaner org_id=user-1 msg="finish adding blocks" old_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish updating block entries" discovered_blocks_count=0 new_blocks_count=0`, + `level=info component=cleaner org_id=user-1 msg="finish updating index"`, + `level=info component=cleaner org_id=user-1 msg="finish getting blocks to be deleted"`, + `level=info component=cleaner org_id=user-1 msg="finish deleting blocks"`, + `level=info component=cleaner org_id=user-2 msg="finish writing new index"`, + `level=info component=cleaner org_id=user-2 msg="finish cleaning partitioned group info files"`, `level=info component=cleaner org_id=user-2 msg="completed blocks cleanup and maintenance"`, + `level=info component=cleaner org_id=user-1 msg="finish writing new index"`, + `level=info component=cleaner org_id=user-1 msg="finish cleaning partitioned group info files"`, + `level=info component=cleaner org_id=user-1 msg="completed blocks cleanup and maintenance"`, `level=info component=cleaner msg="successfully completed blocks cleanup and maintenance"`, `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`, + `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"))) } @@ -1200,18 +1299,20 @@ func TestCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndM bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", userIDs, nil) for _, userID := range userIDs { + partitionedGroupID := getPartitionedGroupID(userID) bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D"}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) bucketClient.MockExists(path.Join(userID, cortex_tsdb.TenantDeletionMarkPath), 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/visit-mark.json", "", nil) bucketClient.MockGet(userID+"/bucket-index-sync-status.json", "", nil) - bucketClient.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/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) } // Create a shared KV Store @@ -1231,7 +1332,7 @@ func TestCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndM cfg.ShardingRing.WaitStabilityMaxDuration = 10 * time.Second cfg.ShardingRing.KVStore.Mock = kvstore - c, _, tsdbPlanner, l, _ := prepare(t, cfg, bucketClient, nil) + c, _, tsdbPlanner, l, _ := prepare(t, cfg, bucketClient, nil, nil) defer services.StopAndAwaitTerminated(context.Background(), c) //nolint:errcheck compactors = append(compactors, c) @@ -1252,7 +1353,7 @@ 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) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) } @@ -1311,25 +1412,30 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit blockFiles := []string{} for blockID, blockTimes := range blocks { - blockVisitMarker := BlockVisitMarker{ - CompactorID: "test-compactor", - VisitTime: time.Now().Unix(), - Version: VisitMarkerVersion1, + groupHash := HashGroup(userID, blockTimes["startTime"], blockTimes["endTime"]) + partitionVisitMarker := PartitionVisitMarker{ + CompactorID: "test-compactor", + VisitTime: time.Now().Unix(), + PartitionedGroupID: groupHash, + PartitionID: 0, + Status: Pending, + Version: PartitionVisitMarkerVersion1, } - visitMarkerFileContent, _ := json.Marshal(blockVisitMarker) + visitMarkerFileContent, _ := json.Marshal(partitionVisitMarker) 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+"/visit-mark.json", string(visitMarkerFileContent), nil) - bucketClient.MockGetRequireUpload(userID+"/"+blockID+"/visit-mark.json", string(visitMarkerFileContent), nil) - bucketClient.MockUpload(userID+"/"+blockID+"/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 - groupHash := hashGroup(userID, blockTimes["startTime"], blockTimes["endTime"]) 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) @@ -1338,6 +1444,7 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit 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 @@ -1363,7 +1470,7 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit flagext.DefaultValues(limits) limits.CompactorTenantShardSize = 3 - c, _, tsdbPlanner, l, _ := prepare(t, cfg, bucketClient, limits) + c, _, tsdbPlanner, l, _ := prepare(t, cfg, bucketClient, limits, nil) defer services.StopAndAwaitTerminated(context.Background(), c) //nolint:errcheck compactors = append(compactors, c) @@ -1390,7 +1497,7 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit // Wait until a run has been completed on each compactor for _, c := range compactors { cortex_testutil.Poll(t, 60*time.Second, 2.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) } @@ -1531,14 +1638,6 @@ func createNoCompactionMark(t *testing.T, bkt objstore.Bucket, userID string, bl require.NoError(t, bkt.Upload(context.Background(), markPath, strings.NewReader(content))) } -func createBlockVisitMarker(t *testing.T, bkt objstore.Bucket, userID string, blockID ulid.ULID) { - content := mockBlockVisitMarker() - blockPath := path.Join(userID, blockID.String()) - markPath := path.Join(blockPath, BlockVisitMarkerFile) - - require.NoError(t, bkt.Upload(context.Background(), markPath, strings.NewReader(content))) -} - func findCompactorByUserID(compactors []*Compactor, logs []*concurrency.SyncBuffer, userID string) (*Compactor, *concurrency.SyncBuffer, error) { var compactor *Compactor var log *concurrency.SyncBuffer @@ -1586,10 +1685,15 @@ func removeIgnoredLogs(input []string) []string { } out := make([]string, 0, len(input)) - durationRe := regexp.MustCompile(`\s?duration=\S+`) + executionIDRe := regexp.MustCompile(`\s?execution_id=\S+`) + durationRe := regexp.MustCompile(`\s?duration(_ms)?=\S+`) for i := 0; i < len(input); i++ { log := input[i] + + // Remove any execution_id from logs. + log = executionIDRe.ReplaceAllString(log, "") + if strings.Contains(log, "block.MetaFetcher") || strings.Contains(log, "block.BaseFetcher") { continue } @@ -1627,7 +1731,7 @@ func prepareConfig() Config { return compactorCfg } -func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, limits *validation.Limits) (*Compactor, *tsdbCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { +func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, limits *validation.Limits, tsdbGrouper *tsdbGrouperMock) (*Compactor, *tsdbCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { storageCfg := cortex_tsdb.BlocksStorageConfig{} flagext.DefaultValues(&storageCfg) @@ -1664,13 +1768,26 @@ func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, li } var blocksGrouperFactory BlocksGrouperFactory + if tsdbGrouper != nil { + blocksGrouperFactory = func(_ context.Context, _ Config, _ objstore.InstrumentedBucket, _ log.Logger, _ prometheus.Registerer, _, _, _ prometheus.Counter, _ prometheus.Gauge, _ prometheus.Counter, _ prometheus.Counter, _ prometheus.Counter, _ prometheus.Counter, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string, _ *compact.GatherNoCompactionMarkFilter) compact.Grouper { + return tsdbGrouper + } + } else { + if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle { + blocksGrouperFactory = ShuffleShardingGrouperFactory + } else { + blocksGrouperFactory = DefaultBlocksGrouperFactory + } + } + + var blockDeletableCheckerFactory BlockDeletableCheckerFactory if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle { - blocksGrouperFactory = ShuffleShardingGrouperFactory + blockDeletableCheckerFactory = PartitionCompactionBlockDeletableCheckerFactory } else { - blocksGrouperFactory = DefaultBlocksGrouperFactory + blockDeletableCheckerFactory = DefaultBlockDeletableCheckerFactory } - c, err := newCompactor(compactorCfg, storageCfg, logger, registry, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, overrides) + c, err := newCompactor(compactorCfg, storageCfg, logger, registry, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, blockDeletableCheckerFactory, overrides) require.NoError(t, err) return c, tsdbCompactor, tsdbPlanner, logs, registry @@ -1723,17 +1840,73 @@ func (m *tsdbPlannerMock) getNoCompactBlocks() []string { return result } -func mockBlockMetaJSON(id string) string { - meta := tsdb.BlockMeta{ +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 mockBlockMeta(id string) tsdb.BlockMeta { + return tsdb.BlockMeta{ Version: 1, ULID: ulid.MustParse(id), - MinTime: 1574776800000, - MaxTime: 1574784000000, + MinTime: BlockMinTime, + MaxTime: BlockMaxTime, Compaction: tsdb.BlockMetaCompaction{ Level: 1, Sources: []ulid.ULID{ulid.MustParse(id)}, }, } +} + +func mockBlockMetaJSON(id string) string { + meta := mockBlockMeta(id) content, err := json.Marshal(meta) if err != nil { @@ -1801,21 +1974,6 @@ func mockBlockMetaJSONWithTime(id string, orgID string, minTime int64, maxTime i return string(content) } -func mockBlockVisitMarker() string { - blockVisitMarker := BlockVisitMarker{ - CompactorID: "dummy", - VisitTime: time.Now().Unix(), - Version: 1, - } - - content, err := json.Marshal(blockVisitMarker) - if err != nil { - panic("failed to marshal mocked block visit marker") - } - - return string(content) -} - func TestCompactor_DeleteLocalSyncFiles(t *testing.T) { numUsers := 10 @@ -1851,7 +2009,7 @@ func TestCompactor_DeleteLocalSyncFiles(t *testing.T) { cfg.ShardingRing.KVStore.Mock = kvstore // Each compactor will get its own temp dir for storing local files. - c, _, tsdbPlanner, _, _ := prepare(t, cfg, inmem, nil) + c, _, tsdbPlanner, _, _ := prepare(t, cfg, inmem, nil, nil) t.Cleanup(func() { require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) }) @@ -1874,7 +2032,7 @@ func TestCompactor_DeleteLocalSyncFiles(t *testing.T) { // Wait until a run has been completed on first compactor. This happens as soon as compactor starts. cortex_testutil.Poll(t, 10*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c1.compactionRunsCompleted) + return prom_testutil.ToFloat64(c1.CompactionRunsCompleted) }) require.NoError(t, os.Mkdir(c1.metaSyncDirForUser("new-user"), 0600)) @@ -1885,7 +2043,7 @@ func TestCompactor_DeleteLocalSyncFiles(t *testing.T) { // Now start second compactor, and wait until it runs compaction. require.NoError(t, services.StartAndAwaitRunning(context.Background(), c2)) cortex_testutil.Poll(t, 10*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c2.compactionRunsCompleted) + return prom_testutil.ToFloat64(c2.CompactionRunsCompleted) }) // Let's check how many users second compactor has. @@ -1920,7 +2078,7 @@ func TestCompactor_ShouldFailCompactionOnTimeout(t *testing.T) { // 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, _ := prepare(t, cfg, bucketClient, nil) + c, _, _, logs, _ := prepare(t, cfg, bucketClient, nil, nil) // Try to start the compactor with a bad consul kv-store. The err := services.StartAndAwaitRunning(context.Background(), c) @@ -1934,63 +2092,54 @@ func TestCompactor_ShouldFailCompactionOnTimeout(t *testing.T) { }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) } -func TestCompactor_ShouldNotTreatInterruptionsAsErrors(t *testing.T) { - bucketClient := objstore.NewInMemBucket() - id := ulid.MustNew(ulid.Now(), rand.Reader) - require.NoError(t, bucketClient.Upload(context.Background(), "user-1/"+id.String()+"/meta.json", strings.NewReader(mockBlockMetaJSON(id.String())))) +func TestCompactor_ShouldNotHangIfPlannerReturnNothing(t *testing.T) { + t.Parallel() - 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"}) + ss := bucketindex.Status{Status: bucketindex.CustomerManagedKeyError, Version: bucketindex.SyncStatusFileVersion} + content, err := json.Marshal(ss) + require.NoError(t, err) - c, tsdbCompactor, tsdbPlanner, logs, registry := prepare(t, prepareConfig(), bucketClient, nil) + partitionedGroupID := getPartitionedGroupID("user-1") + bucketClient := &bucket.ClientMock{} + 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.MockExists(path.Join("user-1", cortex_tsdb.TenantDeletionMarkPath), 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) - ctx, cancel := context.WithCancel(context.Background()) - tsdbCompactor.On("CompactWithBlockPopulator", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(ulid.ULID{}, context.Canceled).Run(func(args mock.Arguments) { - cancel() - }) - 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(ctx, c)) + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) - cortex_testutil.Poll(t, 1*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsInterrupted) - }) + cfg := prepareConfig() + cfg.ShardingEnabled = true + cfg.ShardingRing.InstanceID = "compactor-1" + cfg.ShardingRing.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.KVStore.Mock = ringStore - require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + tsdbGrouper := tsdbGrouperMock{} + mockGroups := []*compact.Group{mockBlockGroup("user-1", []string{"01DTVP434PA9VFXSW2JKB3392D", "01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, bucketClient)} + tsdbGrouper.On("Groups", mock.Anything).Return(mockGroups, nil) - assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # 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 + c, _, tsdbPlanner, _, _ := prepare(t, cfg, bucketClient, nil, &tsdbGrouper) + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{}, nil) - # TYPE cortex_compactor_runs_interrupted_total counter - # HELP cortex_compactor_runs_interrupted_total Total number of compaction runs interrupted. - cortex_compactor_runs_interrupted_total 1 + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) - # 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 - `), - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_interrupted_total", - "cortex_compactor_runs_failed_total", - )) + // Wait until a run has completed. + cortex_testutil.Poll(t, 5*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) + }) - lines := strings.Split(logs.String(), "\n") - require.Contains(t, lines, `level=info component=compactor msg="interrupting compaction of user blocks" user=user-1`) - require.NotContains(t, logs.String(), `level=error`) + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) } diff --git a/pkg/compactor/meta_extenstions.go b/pkg/compactor/meta_extenstions.go new file mode 100644 index 00000000000..427bc6fa357 --- /dev/null +++ b/pkg/compactor/meta_extenstions.go @@ -0,0 +1,62 @@ +package compactor + +import ( + "fmt" + + "github.com/thanos-io/thanos/pkg/block/metadata" +) + +type CortexMetaExtensions struct { + PartitionInfo *PartitionInfo `json:"partition_info,omitempty"` +} + +type PartitionInfo struct { + PartitionedGroupID uint32 `json:"partitioned_group_id"` + PartitionCount int `json:"partition_count"` + PartitionID int `json:"partition_id"` +} + +var ( + DefaultPartitionInfo = PartitionInfo{ + PartitionedGroupID: 0, + PartitionID: 0, + PartitionCount: 1, + } +) + +func ConvertToCortexMetaExtensions(extensions any) (*CortexMetaExtensions, error) { + defaultPartitionInfo := DefaultPartitionInfo + cortexExtensions, err := metadata.ConvertExtensions(extensions, &CortexMetaExtensions{ + PartitionInfo: &defaultPartitionInfo, + }) + if err != nil { + return nil, err + } + if cortexExtensions == nil { + return nil, nil + } + converted, ok := cortexExtensions.(*CortexMetaExtensions) + if !ok { + return nil, fmt.Errorf("unable to convert extensions to CortexMetaExtensions") + } + return converted, nil +} + +func ConvertToPartitionInfo(extensions any) (*PartitionInfo, error) { + cortexExtensions, err := ConvertToCortexMetaExtensions(extensions) + if err != nil { + return nil, err + } + if cortexExtensions == nil { + return nil, nil + } + return cortexExtensions.PartitionInfo, nil +} + +func GetCortexMetaExtensionsFromMeta(meta metadata.Meta) (*CortexMetaExtensions, error) { + return ConvertToCortexMetaExtensions(meta.Thanos.Extensions) +} + +func GetPartitionInfo(meta metadata.Meta) (*PartitionInfo, error) { + return ConvertToPartitionInfo(meta.Thanos.Extensions) +} diff --git a/pkg/compactor/meta_extenstions_test.go b/pkg/compactor/meta_extenstions_test.go new file mode 100644 index 00000000000..f067c79fb83 --- /dev/null +++ b/pkg/compactor/meta_extenstions_test.go @@ -0,0 +1,182 @@ +package compactor + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/thanos-io/thanos/pkg/block/metadata" +) + +func TestGetPartitionedInfo(t *testing.T) { + for _, tcase := range []struct { + name string + meta metadata.Meta + expected *PartitionInfo + }{ + { + name: "partition info with all information provided", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: &CortexMetaExtensions{ + PartitionInfo: &PartitionInfo{ + PartitionedGroupID: 123, + PartitionID: 8, + PartitionCount: 32, + }, + }, + }, + }, + expected: &PartitionInfo{ + PartitionedGroupID: 123, + PartitionID: 8, + PartitionCount: 32, + }, + }, + { + name: "partition info with only PartitionedGroupID provided", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: &CortexMetaExtensions{ + PartitionInfo: &PartitionInfo{ + PartitionedGroupID: 123, + }, + }, + }, + }, + expected: &PartitionInfo{ + PartitionedGroupID: 123, + PartitionID: 0, + PartitionCount: 0, + }, + }, + { + name: "partition info with only PartitionID provided", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: &CortexMetaExtensions{ + PartitionInfo: &PartitionInfo{ + PartitionID: 5, + }, + }, + }, + }, + expected: &PartitionInfo{ + PartitionedGroupID: 0, + PartitionID: 5, + PartitionCount: 0, + }, + }, + { + name: "partition info with only PartitionCount provided", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: &CortexMetaExtensions{ + PartitionInfo: &PartitionInfo{ + PartitionCount: 4, + }, + }, + }, + }, + expected: &PartitionInfo{ + PartitionedGroupID: 0, + PartitionID: 0, + PartitionCount: 4, + }, + }, + { + name: "meta with empty partition info provided", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: &CortexMetaExtensions{ + PartitionInfo: &PartitionInfo{}, + }, + }, + }, + expected: &PartitionInfo{ + PartitionedGroupID: 0, + PartitionID: 0, + PartitionCount: 0, + }, + }, + { + name: "meta with nil partition info provided", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: &CortexMetaExtensions{ + PartitionInfo: nil, + }, + }, + }, + expected: &PartitionInfo{ + PartitionedGroupID: 0, + PartitionID: 0, + PartitionCount: 1, + }, + }, + { + name: "meta with non CortexMetaExtensions provided", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: struct { + dummy string + }{ + dummy: "test_dummy", + }, + }, + }, + expected: &PartitionInfo{ + PartitionedGroupID: 0, + PartitionID: 0, + PartitionCount: 1, + }, + }, + { + name: "meta with invalid CortexMetaExtensions provided", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: struct { + PartitionInfo struct { + PartitionedGroupID uint32 `json:"partitionedGroupId"` + PartitionCount int `json:"partitionCount"` + PartitionID int `json:"partitionId"` + } `json:"partition_info,omitempty"` + }{ + PartitionInfo: struct { + PartitionedGroupID uint32 `json:"partitionedGroupId"` + PartitionCount int `json:"partitionCount"` + PartitionID int `json:"partitionId"` + }{ + PartitionedGroupID: 123, + PartitionID: 8, + PartitionCount: 32, + }, + }, + }, + }, + expected: &PartitionInfo{ + PartitionedGroupID: 0, + PartitionID: 0, + PartitionCount: 1, + }, + }, + { + name: "meta does not have any extensions", + meta: metadata.Meta{ + Thanos: metadata.Thanos{ + Extensions: nil, + }, + }, + expected: nil, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + result, err := GetPartitionInfo(tcase.meta) + assert.NoError(t, err) + if tcase.expected == nil { + assert.Nil(t, result) + } else { + assert.Equal(t, *tcase.expected, *result) + } + }) + } +} diff --git a/pkg/compactor/partition_compaction_complete_checker.go b/pkg/compactor/partition_compaction_complete_checker.go new file mode 100644 index 00000000000..2ea7df71d39 --- /dev/null +++ b/pkg/compactor/partition_compaction_complete_checker.go @@ -0,0 +1,75 @@ +package compactor + +import ( + "context" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/compact" +) + +type PartitionCompactionBlockDeletableChecker struct { + ctx context.Context + bkt objstore.InstrumentedBucket + logger log.Logger + visitMarkerReadFailed prometheus.Counter + partitionedGroupInfoReadFailed prometheus.Counter +} + +func NewPartitionCompactionBlockDeletableChecker( + ctx context.Context, + bkt objstore.InstrumentedBucket, + logger log.Logger, + visitMarkerReadFailed prometheus.Counter, + partitionedGroupInfoReadFailed prometheus.Counter, +) *PartitionCompactionBlockDeletableChecker { + return &PartitionCompactionBlockDeletableChecker{ + ctx: ctx, + bkt: bkt, + logger: logger, + visitMarkerReadFailed: visitMarkerReadFailed, + partitionedGroupInfoReadFailed: partitionedGroupInfoReadFailed, + } +} + +func (p *PartitionCompactionBlockDeletableChecker) CanDelete(group *compact.Group, blockID ulid.ULID) bool { + partitionInfo, err := ConvertToPartitionInfo(group.Extensions()) + if err != nil { + return false + } + if partitionInfo == nil { + return true + } + partitionedGroupID := partitionInfo.PartitionedGroupID + currentPartitionID := partitionInfo.PartitionID + partitionedGroupInfo, err := ReadPartitionedGroupInfo(p.ctx, p.bkt, p.logger, partitionedGroupID, p.partitionedGroupInfoReadFailed) + if err != nil { + level.Warn(p.logger).Log("msg", "unable to read partitioned group info", "partitioned_group_id", partitionedGroupID, "block_id", blockID, "err", err) + return false + } + return p.isPartitionedBlockComplete(partitionedGroupInfo, currentPartitionID, blockID) +} + +func (p *PartitionCompactionBlockDeletableChecker) isPartitionedBlockComplete(partitionedGroupInfo *PartitionedGroupInfo, currentPartitionID int, blockID ulid.ULID) bool { + partitionedGroupID := partitionedGroupInfo.PartitionedGroupID + for _, partitionID := range partitionedGroupInfo.getPartitionIDsByBlock(blockID) { + // Skip current partition ID since current one is completed + if partitionID != currentPartitionID { + partitionVisitMarker, err := ReadPartitionVisitMarker(p.ctx, p.bkt, p.logger, partitionedGroupID, partitionID, p.visitMarkerReadFailed) + if err != nil { + level.Warn(p.logger).Log("msg", "unable to read all visit markers for partition", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "block_id", blockID, "err", err) + return false + } + if !partitionVisitMarker.isCompleted() { + level.Warn(p.logger).Log("msg", "block has incomplete partition", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "block_id", blockID) + return false + } + } + } + level.Info(p.logger).Log("msg", "block has all partitions completed", "partitioned_group_id", partitionedGroupID, "block_id", blockID) + return true +} diff --git a/pkg/compactor/partition_compaction_complete_checker_test.go b/pkg/compactor/partition_compaction_complete_checker_test.go new file mode 100644 index 00000000000..da359618b4f --- /dev/null +++ b/pkg/compactor/partition_compaction_complete_checker_test.go @@ -0,0 +1,335 @@ +package compactor + +import ( + "context" + "encoding/json" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/compact" + + "github.com/cortexproject/cortex/pkg/storage/bucket" +) + +func TestPartitionCompactionCompleteChecker(t *testing.T) { + ulid0 := ulid.MustNew(0, nil) + ulid1 := ulid.MustNew(1, nil) + ulid2 := ulid.MustNew(2, nil) + + rangeStart := (1 * time.Hour).Milliseconds() + rangeEnd := (2 * time.Hour).Milliseconds() + partitionedGroupID := uint32(12345) + compactorID := "compactor1" + timeBefore1h := time.Now().Add(-1 * time.Hour).Unix() + timeNow := time.Now().Unix() + + for _, tcase := range []struct { + name string + partitionedGroupInfo PartitionedGroupInfo + blocks map[ulid.ULID]struct { + expectComplete bool + } + visitMarkers []PartitionVisitMarker + }{ + { + name: "all partitions are complete 1", + partitionedGroupInfo: PartitionedGroupInfo{ + PartitionedGroupID: partitionedGroupID, + PartitionCount: 2, + Partitions: []Partition{ + { + PartitionID: 0, + Blocks: []ulid.ULID{ + ulid0, + ulid1, + }, + }, + { + PartitionID: 1, + Blocks: []ulid.ULID{ + ulid0, + ulid2, + }, + }, + }, + RangeStart: rangeStart, + RangeEnd: rangeEnd, + Version: PartitionedGroupInfoVersion1, + }, + blocks: map[ulid.ULID]struct { + expectComplete bool + }{ + ulid0: { + expectComplete: true, + }, + ulid1: { + expectComplete: true, + }, + ulid2: { + expectComplete: true, + }, + }, + visitMarkers: []PartitionVisitMarker{ + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 0, + CompactorID: compactorID, + VisitTime: timeBefore1h, + Version: PartitionVisitMarkerVersion1, + }, + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 1, + CompactorID: compactorID, + VisitTime: timeBefore1h, + Version: PartitionVisitMarkerVersion1, + }, + }, + }, + { + name: "all partitions are complete 2", + partitionedGroupInfo: PartitionedGroupInfo{ + PartitionedGroupID: partitionedGroupID, + PartitionCount: 3, + Partitions: []Partition{ + { + PartitionID: 0, + Blocks: []ulid.ULID{ + ulid0, + }, + }, + { + PartitionID: 1, + Blocks: []ulid.ULID{ + ulid1, + }, + }, + { + PartitionID: 2, + Blocks: []ulid.ULID{ + ulid2, + }, + }, + }, + RangeStart: rangeStart, + RangeEnd: rangeEnd, + Version: PartitionedGroupInfoVersion1, + }, + blocks: map[ulid.ULID]struct { + expectComplete bool + }{ + ulid0: { + expectComplete: true, + }, + ulid1: { + expectComplete: true, + }, + ulid2: { + expectComplete: true, + }, + }, + visitMarkers: []PartitionVisitMarker{ + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 0, + CompactorID: compactorID, + VisitTime: timeNow, + Version: PartitionVisitMarkerVersion1, + }, + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 1, + CompactorID: compactorID, + VisitTime: timeNow, + Version: PartitionVisitMarkerVersion1, + }, + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 2, + CompactorID: compactorID, + VisitTime: timeNow, + Version: PartitionVisitMarkerVersion1, + }, + }, + }, + { + name: "not all partitions are complete 1", + partitionedGroupInfo: PartitionedGroupInfo{ + PartitionedGroupID: partitionedGroupID, + PartitionCount: 3, + Partitions: []Partition{ + { + PartitionID: 0, + Blocks: []ulid.ULID{ + ulid0, + }, + }, + { + PartitionID: 1, + Blocks: []ulid.ULID{ + ulid0, + ulid1, + }, + }, + { + PartitionID: 2, + Blocks: []ulid.ULID{ + ulid2, + }, + }, + }, + RangeStart: rangeStart, + RangeEnd: rangeEnd, + Version: PartitionedGroupInfoVersion1, + }, + blocks: map[ulid.ULID]struct { + expectComplete bool + }{ + ulid0: { + expectComplete: false, + }, + ulid1: { + expectComplete: true, + }, + ulid2: { + expectComplete: true, + }, + }, + visitMarkers: []PartitionVisitMarker{ + { + Status: Pending, + PartitionedGroupID: partitionedGroupID, + PartitionID: 0, + CompactorID: compactorID, + VisitTime: timeBefore1h, + Version: PartitionVisitMarkerVersion1, + }, + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 1, + CompactorID: compactorID, + VisitTime: timeBefore1h, + Version: PartitionVisitMarkerVersion1, + }, + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 2, + CompactorID: compactorID, + VisitTime: timeBefore1h, + Version: PartitionVisitMarkerVersion1, + }, + }, + }, + { + name: "not all partitions are complete 2", + partitionedGroupInfo: PartitionedGroupInfo{ + PartitionedGroupID: partitionedGroupID, + PartitionCount: 3, + Partitions: []Partition{ + { + PartitionID: 0, + Blocks: []ulid.ULID{ + ulid0, + }, + }, + { + PartitionID: 1, + Blocks: []ulid.ULID{ + ulid0, + ulid1, + }, + }, + { + PartitionID: 2, + Blocks: []ulid.ULID{ + ulid2, + }, + }, + }, + RangeStart: rangeStart, + RangeEnd: rangeEnd, + Version: PartitionedGroupInfoVersion1, + }, + blocks: map[ulid.ULID]struct { + expectComplete bool + }{ + ulid0: { + expectComplete: false, + }, + ulid1: { + expectComplete: false, + }, + ulid2: { + expectComplete: true, + }, + }, + visitMarkers: []PartitionVisitMarker{ + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 0, + CompactorID: compactorID, + VisitTime: timeNow, + Version: PartitionVisitMarkerVersion1, + }, + { + Status: Pending, + PartitionedGroupID: partitionedGroupID, + PartitionID: 1, + CompactorID: compactorID, + VisitTime: timeNow, + Version: PartitionVisitMarkerVersion1, + }, + { + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: 2, + CompactorID: compactorID, + VisitTime: timeNow, + Version: PartitionVisitMarkerVersion1, + }, + }, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + bkt := &bucket.ClientMock{} + partitionedGroupInfoFileContent, _ := json.Marshal(tcase.partitionedGroupInfo) + bkt.MockGet(GetPartitionedGroupFile(partitionedGroupID), string(partitionedGroupInfoFileContent), nil) + checker := NewPartitionCompactionBlockDeletableChecker( + context.Background(), + objstore.WithNoopInstr(bkt), + log.NewNopLogger(), + prometheus.NewCounter(prometheus.CounterOpts{}), + prometheus.NewCounter(prometheus.CounterOpts{}), + ) + group := compact.Group{} + // set partitionID to -1, so it will go through all partitionIDs when checking + group.SetExtensions(&CortexMetaExtensions{ + PartitionInfo: &PartitionInfo{ + PartitionedGroupID: tcase.partitionedGroupInfo.PartitionedGroupID, + PartitionCount: tcase.partitionedGroupInfo.PartitionCount, + PartitionID: -1, + }, + }) + for _, visitMarker := range tcase.visitMarkers { + visitMarkerFileContent, _ := json.Marshal(visitMarker) + bkt.MockGet(GetPartitionVisitMarkerFile(visitMarker.PartitionedGroupID, visitMarker.PartitionID), string(visitMarkerFileContent), nil) + } + for blockID, blockTCase := range tcase.blocks { + require.Equal(t, blockTCase.expectComplete, checker.CanDelete(&group, blockID)) + } + }) + } +} diff --git a/pkg/compactor/partition_visit_marker.go b/pkg/compactor/partition_visit_marker.go new file mode 100644 index 00000000000..7b364d84e7b --- /dev/null +++ b/pkg/compactor/partition_visit_marker.go @@ -0,0 +1,214 @@ +package compactor + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "io" + "path" + "strings" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/objstore" + + "github.com/cortexproject/cortex/pkg/util/runutil" +) + +const ( + // PartitionVisitMarkerDirectory is the name of directory where all visit markers are saved. + PartitionVisitMarkerDirectory = "visit-marks" + // PartitionVisitMarkerFileSuffix is the known suffix of json filename for representing the most recent compactor visit. + PartitionVisitMarkerFileSuffix = "visit-mark.json" + // PartitionVisitMarkerFilePrefix is the known prefix of json filename for representing the most recent compactor visit. + PartitionVisitMarkerFilePrefix = "partition-" + // PartitionVisitMarkerVersion1 is the current supported version of visit-mark file. + PartitionVisitMarkerVersion1 = 1 +) + +var ( + ErrorPartitionVisitMarkerNotFound = errors.New("partition visit marker not found") + ErrorUnmarshalPartitionVisitMarker = errors.New("unmarshal partition visit marker JSON") + ErrorNotPartitionVisitMarker = errors.New("file is not partition visit marker") +) + +type VisitStatus string + +const ( + Pending VisitStatus = "pending" + Completed VisitStatus = "completed" +) + +type PartitionVisitMarker struct { + CompactorID string `json:"compactorID"` + Status VisitStatus `json:"status"` + PartitionedGroupID uint32 `json:"partitionedGroupID"` + PartitionID int `json:"partitionID"` + // VisitTime is a unix timestamp of when the partition was visited (mark updated). + VisitTime int64 `json:"visitTime"` + // Version of the file. + Version int `json:"version"` +} + +func (b *PartitionVisitMarker) isVisited(visitMarkerTimeout time.Duration, partitionID int) bool { + return b.isCompleted() || partitionID == b.PartitionID && time.Now().Before(time.Unix(b.VisitTime, 0).Add(visitMarkerTimeout)) +} + +func (b *PartitionVisitMarker) isVisitedByCompactor(visitMarkerTimeout time.Duration, partitionID int, compactorID string) bool { + return b.CompactorID == compactorID && b.isVisited(visitMarkerTimeout, partitionID) +} + +func (b *PartitionVisitMarker) isCompleted() bool { + return b.Status == Completed +} + +func GetPartitionVisitMarkerFile(partitionedGroupID uint32, partitionID int) string { + return path.Join(PartitionedGroupDirectory, PartitionVisitMarkerDirectory, fmt.Sprintf("%d", partitionedGroupID), fmt.Sprintf("%s%d-%s", PartitionVisitMarkerFilePrefix, partitionID, PartitionVisitMarkerFileSuffix)) +} + +func ReadPartitionVisitMarker(ctx context.Context, bkt objstore.InstrumentedBucketReader, logger log.Logger, partitionedGroupID uint32, partitionID int, visitMarkerReadFailed prometheus.Counter) (*PartitionVisitMarker, error) { + visitMarkerFile := GetPartitionVisitMarkerFile(partitionedGroupID, partitionID) + visitMarkerFileReader, err := bkt.ReaderWithExpectedErrs(bkt.IsObjNotFoundErr).Get(ctx, visitMarkerFile) + if err != nil { + if bkt.IsObjNotFoundErr(err) { + return nil, errors.Wrapf(ErrorPartitionVisitMarkerNotFound, "partition visit marker file: %s", visitMarkerFile) + } + visitMarkerReadFailed.Inc() + return nil, errors.Wrapf(err, "get partition visit marker file: %s", visitMarkerFile) + } + defer runutil.CloseWithLogOnErr(logger, visitMarkerFileReader, "close partition visit marker reader") + b, err := io.ReadAll(visitMarkerFileReader) + if err != nil { + visitMarkerReadFailed.Inc() + return nil, errors.Wrapf(err, "read partition visit marker file: %s", visitMarkerFile) + } + partitionVisitMarker := PartitionVisitMarker{} + if err = json.Unmarshal(b, &partitionVisitMarker); err != nil { + visitMarkerReadFailed.Inc() + return nil, errors.Wrapf(ErrorUnmarshalPartitionVisitMarker, "partition visit marker file: %s, error: %v", visitMarkerFile, err.Error()) + } + if partitionVisitMarker.Version != PartitionVisitMarkerVersion1 { + return nil, errors.Errorf("unexpected partition visit mark file version %d, expected %d", partitionVisitMarker.Version, PartitionVisitMarkerVersion1) + } + return &partitionVisitMarker, nil +} + +func UpdatePartitionVisitMarker(ctx context.Context, bkt objstore.Bucket, partitionedGroupID uint32, partitionID int, reader io.Reader, blockVisitMarkerWriteFailed prometheus.Counter) error { + blockVisitMarkerFilePath := GetPartitionVisitMarkerFile(partitionedGroupID, partitionID) + if err := bkt.Upload(ctx, blockVisitMarkerFilePath, reader); err != nil { + blockVisitMarkerWriteFailed.Inc() + return err + } + return nil +} + +func markPartitionVisited( + ctx context.Context, + bkt objstore.Bucket, + logger log.Logger, + marker PartitionVisitMarker, + visitMarkerWriteFailed prometheus.Counter, +) { + visitMarkerFileContent, err := json.Marshal(marker) + if err != nil { + visitMarkerWriteFailed.Inc() + return + } + + reader := bytes.NewReader(visitMarkerFileContent) + if err := UpdatePartitionVisitMarker(ctx, bkt, marker.PartitionedGroupID, marker.PartitionID, reader, visitMarkerWriteFailed); err != nil { + level.Error(logger).Log("msg", "unable to upsert visit marker file content for partition", "partitioned_group_id", marker.PartitionedGroupID, "partition_id", marker.PartitionID, "err", err) + } + level.Debug(logger).Log("msg", "marked partition visited", "partitioned_group_id", marker.PartitionedGroupID, "partition_id", marker.PartitionID) +} + +func markPartitionVisitedHeartBeat( + ctx context.Context, + bkt objstore.Bucket, + logger log.Logger, + partitionedGroupID uint32, + partitionID int, + compactorID string, + visitMarkerFileUpdateInterval time.Duration, + visitMarkerWriteFailed prometheus.Counter, + errChan chan error, +) { + level.Info(logger).Log("msg", "start partition visit marker heart beat", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID) + ticker := time.NewTicker(visitMarkerFileUpdateInterval) + defer ticker.Stop() + isComplete := false +heartBeat: + for { + blockVisitMarker := PartitionVisitMarker{ + VisitTime: time.Now().Unix(), + CompactorID: compactorID, + Status: Pending, + PartitionedGroupID: partitionedGroupID, + PartitionID: partitionID, + Version: PartitionVisitMarkerVersion1, + } + markPartitionVisited(ctx, bkt, logger, blockVisitMarker, visitMarkerWriteFailed) + + select { + case <-ctx.Done(): + level.Warn(logger).Log("msg", "partition visit marker heart beat got cancelled", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID) + break heartBeat + case <-ticker.C: + continue + case err := <-errChan: + isComplete = err == nil + if err != nil { + level.Warn(logger).Log("msg", "stop partition visit marker heart beat due to error", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "err", err) + } + break heartBeat + } + } + if isComplete { + level.Info(logger).Log("msg", "update partition visit marker to completed status", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID) + markPartitionVisitMarkerCompleted(context.Background(), bkt, logger, partitionedGroupID, partitionID, compactorID, visitMarkerWriteFailed) + } + level.Info(logger).Log("msg", "stop partition visit marker heart beat", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID) +} + +func markPartitionVisitMarkerCompleted( + ctx context.Context, + bkt objstore.Bucket, + logger log.Logger, + partitionedGroupID uint32, + partitionID int, + compactorID string, + blockVisitMarkerWriteFailed prometheus.Counter, +) { + visitMarker := PartitionVisitMarker{ + VisitTime: time.Now().Unix(), + CompactorID: compactorID, + Status: Completed, + PartitionedGroupID: partitionedGroupID, + PartitionID: partitionID, + Version: PartitionVisitMarkerVersion1, + } + visitMarkerFileContent, err := json.Marshal(visitMarker) + if err != nil { + blockVisitMarkerWriteFailed.Inc() + return + } + reader := bytes.NewReader(visitMarkerFileContent) + if err := UpdatePartitionVisitMarker(ctx, bkt, visitMarker.PartitionedGroupID, visitMarker.PartitionID, reader, blockVisitMarkerWriteFailed); err != nil { + level.Error(logger).Log("msg", "unable to upsert completed visit marker file content for partition", "partitioned_group_id", visitMarker.PartitionedGroupID, "partition_id", visitMarker.PartitionID, "err", err) + } else { + level.Info(logger).Log("msg", "partition is completed", "partitioned_group_id", visitMarker.PartitionedGroupID, "partition_id", visitMarker.PartitionID) + } +} + +func IsPartitionVisitMarker(path string) bool { + return strings.HasSuffix(path, PartitionVisitMarkerFileSuffix) +} + +func IsNotPartitionVisitMarkerError(err error) bool { + return errors.Is(err, ErrorNotPartitionVisitMarker) +} diff --git a/pkg/compactor/partition_visit_marker_test.go b/pkg/compactor/partition_visit_marker_test.go new file mode 100644 index 00000000000..e2453348bf9 --- /dev/null +++ b/pkg/compactor/partition_visit_marker_test.go @@ -0,0 +1,113 @@ +package compactor + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/stretchr/testify/require" + + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/objstore" + + cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" +) + +func TestMarkPartitionVisited(t *testing.T) { + now := time.Now().Unix() + nowBefore1h := time.Now().Add(-1 * time.Hour).Unix() + partitionedGroupID1 := uint32(1234) + partitionedGroupID2 := uint32(5678) + for _, tcase := range []struct { + name string + visitMarker PartitionVisitMarker + partitionedGroupID uint32 + }{ + { + name: "write visit marker succeeded", + visitMarker: PartitionVisitMarker{ + CompactorID: "foo", + PartitionedGroupID: partitionedGroupID1, + PartitionID: 0, + VisitTime: now, + Version: PartitionVisitMarkerVersion1, + }, + partitionedGroupID: partitionedGroupID1, + }, + { + name: "write visit marker succeeded 2", + visitMarker: PartitionVisitMarker{ + CompactorID: "bar", + PartitionedGroupID: partitionedGroupID2, + PartitionID: 3, + VisitTime: nowBefore1h, + Version: PartitionVisitMarkerVersion1, + }, + partitionedGroupID: partitionedGroupID2, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + ctx := context.Background() + dummyCounter := prometheus.NewCounter(prometheus.CounterOpts{}) + bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + logger := log.NewNopLogger() + markPartitionVisited(ctx, bkt, logger, tcase.visitMarker, dummyCounter) + res, err := ReadPartitionVisitMarker(ctx, objstore.WithNoopInstr(bkt), logger, tcase.partitionedGroupID, tcase.visitMarker.PartitionID, dummyCounter) + require.NoError(t, err) + require.Equal(t, tcase.visitMarker, *res) + }) + } +} + +func TestMarkPartitionVisitedHeartBeat(t *testing.T) { + partitionedGroupID := uint32(12345) + partitionID := 0 + compactorID := "test-compactor" + for _, tcase := range []struct { + name string + isCancelled bool + compactionErr error + expectedStatus VisitStatus + }{ + { + name: "heart beat got cancelled", + isCancelled: true, + compactionErr: nil, + expectedStatus: Pending, + }, + { + name: "heart beat complete without error", + isCancelled: false, + compactionErr: nil, + expectedStatus: Completed, + }, + { + name: "heart beat stopped due to compaction error", + isCancelled: false, + compactionErr: fmt.Errorf("some compaction failure"), + expectedStatus: Pending, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + dummyCounter := prometheus.NewCounter(prometheus.CounterOpts{}) + bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + logger := log.NewNopLogger() + errChan := make(chan error, 1) + go markPartitionVisitedHeartBeat(ctx, objstore.WithNoopInstr(bkt), logger, partitionedGroupID, partitionID, compactorID, time.Second, dummyCounter, errChan) + time.Sleep(2 * time.Second) + if tcase.isCancelled { + cancel() + } else { + errChan <- tcase.compactionErr + defer cancel() + } + time.Sleep(2 * time.Second) + res, err := ReadPartitionVisitMarker(context.Background(), objstore.WithNoopInstr(bkt), logger, partitionedGroupID, partitionID, dummyCounter) + require.NoError(t, err) + require.Equal(t, tcase.expectedStatus, res.Status) + }) + } +} diff --git a/pkg/compactor/partitioned_group_info.go b/pkg/compactor/partitioned_group_info.go new file mode 100644 index 00000000000..807aecc2d8a --- /dev/null +++ b/pkg/compactor/partitioned_group_info.go @@ -0,0 +1,147 @@ +package compactor + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "io" + "path" + "strings" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/objstore" + + "github.com/cortexproject/cortex/pkg/util/runutil" +) + +const ( + PartitionedGroupDirectory = "partitioned-groups" + PartitionedGroupInfoVersion1 = 1 +) + +var ( + ErrorPartitionedGroupInfoNotFound = errors.New("partitioned group info not found") + ErrorUnmarshalPartitionedGroupInfo = errors.New("unmarshal partitioned group info JSON") +) + +type Partition struct { + PartitionID int `json:"partitionID"` + Blocks []ulid.ULID `json:"blocks"` +} + +func (p *Partition) getBlocksSet() map[ulid.ULID]struct{} { + res := make(map[ulid.ULID]struct{}) + for _, blockID := range p.Blocks { + res[blockID] = struct{}{} + } + return res +} + +type PartitionedGroupInfo struct { + PartitionedGroupID uint32 `json:"partitionedGroupID"` + PartitionCount int `json:"partitionCount"` + Partitions []Partition `json:"partitions"` + RangeStart int64 `json:"rangeStart"` + RangeEnd int64 `json:"rangeEnd"` + CreationTime int64 `json:"creation_time"` + // Version of the file. + Version int `json:"version"` +} + +func (p *PartitionedGroupInfo) getPartitionIDsByBlock(blockID ulid.ULID) []int { + var partitionIDs []int +partitionLoop: + for _, partition := range p.Partitions { + for _, block := range partition.Blocks { + if block == blockID { + partitionIDs = append(partitionIDs, partition.PartitionID) + continue partitionLoop + } + } + } + return partitionIDs +} + +func (p *PartitionedGroupInfo) getAllBlocks() []ulid.ULID { + uniqueBlocks := make(map[ulid.ULID]struct{}) + for _, partition := range p.Partitions { + for _, block := range partition.Blocks { + uniqueBlocks[block] = struct{}{} + } + } + blocks := make([]ulid.ULID, len(uniqueBlocks)) + i := 0 + for block := range uniqueBlocks { + blocks[i] = block + i++ + } + return blocks +} + +func (p PartitionedGroupInfo) String() string { + var partitions []string + for _, partition := range p.Partitions { + partitions = append(partitions, fmt.Sprintf("(PartitionID: %d, Blocks: %s)", partition.PartitionID, partition.Blocks)) + } + return fmt.Sprintf("{PartitionedGroupID: %d, PartitionCount: %d, Partitions: %s}", p.PartitionedGroupID, p.PartitionCount, strings.Join(partitions, ", ")) +} + +func GetPartitionedGroupFile(partitionedGroupID uint32) string { + return path.Join(PartitionedGroupDirectory, fmt.Sprintf("%d.json", partitionedGroupID)) +} + +func ReadPartitionedGroupInfo(ctx context.Context, bkt objstore.InstrumentedBucketReader, logger log.Logger, partitionedGroupID uint32, partitionedGroupInfoReadFailed prometheus.Counter) (*PartitionedGroupInfo, error) { + return ReadPartitionedGroupInfoFile(ctx, bkt, logger, GetPartitionedGroupFile(partitionedGroupID), partitionedGroupInfoReadFailed) +} + +func ReadPartitionedGroupInfoFile(ctx context.Context, bkt objstore.InstrumentedBucketReader, logger log.Logger, partitionedGroupFile string, partitionedGroupInfoReadFailed prometheus.Counter) (*PartitionedGroupInfo, error) { + partitionedGroupReader, err := bkt.ReaderWithExpectedErrs(bkt.IsObjNotFoundErr).Get(ctx, partitionedGroupFile) + if err != nil { + if bkt.IsObjNotFoundErr(err) { + return nil, errors.Wrapf(ErrorPartitionedGroupInfoNotFound, "partitioned group file: %s", partitionedGroupReader) + } + partitionedGroupInfoReadFailed.Inc() + return nil, errors.Wrapf(err, "get partitioned group file: %s", partitionedGroupReader) + } + defer runutil.CloseWithLogOnErr(logger, partitionedGroupReader, "close partitioned group reader") + p, err := io.ReadAll(partitionedGroupReader) + if err != nil { + partitionedGroupInfoReadFailed.Inc() + return nil, errors.Wrapf(err, "read partitioned group file: %s", partitionedGroupFile) + } + partitionedGroupInfo := PartitionedGroupInfo{} + if err = json.Unmarshal(p, &partitionedGroupInfo); err != nil { + partitionedGroupInfoReadFailed.Inc() + return nil, errors.Wrapf(ErrorUnmarshalPartitionedGroupInfo, "partitioned group file: %s, error: %v", partitionedGroupFile, err.Error()) + } + if partitionedGroupInfo.Version != VisitMarkerVersion1 { + partitionedGroupInfoReadFailed.Inc() + return nil, errors.Errorf("unexpected partitioned group file version %d, expected %d", partitionedGroupInfo.Version, VisitMarkerVersion1) + } + return &partitionedGroupInfo, nil +} + +func UpdatePartitionedGroupInfo(ctx context.Context, bkt objstore.InstrumentedBucket, logger log.Logger, partitionedGroupInfo PartitionedGroupInfo, partitionedGroupInfoReadFailed prometheus.Counter, partitionedGroupInfoWriteFailed prometheus.Counter) (*PartitionedGroupInfo, error) { + existingPartitionedGroup, _ := ReadPartitionedGroupInfo(ctx, bkt, logger, partitionedGroupInfo.PartitionedGroupID, partitionedGroupInfoReadFailed) + if existingPartitionedGroup != nil { + level.Warn(logger).Log("msg", "partitioned group info already exists", "partitioned_group_id", partitionedGroupInfo.PartitionedGroupID) + return existingPartitionedGroup, nil + } + partitionedGroupFile := GetPartitionedGroupFile(partitionedGroupInfo.PartitionedGroupID) + partitionedGroupInfoContent, err := json.Marshal(partitionedGroupInfo) + if err != nil { + partitionedGroupInfoWriteFailed.Inc() + return nil, err + } + reader := bytes.NewReader(partitionedGroupInfoContent) + if err := bkt.Upload(ctx, partitionedGroupFile, reader); err != nil { + return nil, err + } + return &partitionedGroupInfo, nil +} diff --git a/pkg/compactor/partitioned_group_info_test.go b/pkg/compactor/partitioned_group_info_test.go new file mode 100644 index 00000000000..d3bcd94ab5b --- /dev/null +++ b/pkg/compactor/partitioned_group_info_test.go @@ -0,0 +1,160 @@ +package compactor + +import ( + "context" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" +) + +func TestPartitionedGroupInfo(t *testing.T) { + ulid0 := ulid.MustNew(0, nil) + ulid1 := ulid.MustNew(1, nil) + ulid2 := ulid.MustNew(2, nil) + rangeStart := (1 * time.Hour).Milliseconds() + rangeEnd := (2 * time.Hour).Milliseconds() + partitionedGroupID := uint32(12345) + for _, tcase := range []struct { + name string + partitionedGroupInfo PartitionedGroupInfo + }{ + { + name: "write partitioned group info 1", + partitionedGroupInfo: PartitionedGroupInfo{ + PartitionedGroupID: partitionedGroupID, + PartitionCount: 2, + Partitions: []Partition{ + { + PartitionID: 0, + Blocks: []ulid.ULID{ + ulid0, + ulid1, + }, + }, + { + PartitionID: 1, + Blocks: []ulid.ULID{ + ulid0, + ulid2, + }, + }, + }, + RangeStart: rangeStart, + RangeEnd: rangeEnd, + Version: VisitMarkerVersion1, + }, + }, + { + name: "write partitioned group info 2", + partitionedGroupInfo: PartitionedGroupInfo{ + PartitionedGroupID: partitionedGroupID, + PartitionCount: 3, + Partitions: []Partition{ + { + PartitionID: 0, + Blocks: []ulid.ULID{ + ulid0, + }, + }, + { + PartitionID: 1, + Blocks: []ulid.ULID{ + ulid1, + }, + }, + { + PartitionID: 2, + Blocks: []ulid.ULID{ + ulid2, + }, + }, + }, + RangeStart: rangeStart, + RangeEnd: rangeEnd, + CreationTime: time.Now().Unix(), + Version: VisitMarkerVersion1, + }, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + ctx := context.Background() + dummyReadCounter := prometheus.NewCounter(prometheus.CounterOpts{}) + dummyWriteCounter := prometheus.NewCounter(prometheus.CounterOpts{}) + testBkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + bkt := objstore.WithNoopInstr(testBkt) + logger := log.NewNopLogger() + writeRes, err := UpdatePartitionedGroupInfo(ctx, bkt, logger, tcase.partitionedGroupInfo, dummyReadCounter, dummyWriteCounter) + require.NoError(t, err) + require.Equal(t, tcase.partitionedGroupInfo, *writeRes) + readRes, err := ReadPartitionedGroupInfo(ctx, bkt, logger, tcase.partitionedGroupInfo.PartitionedGroupID, dummyReadCounter) + require.NoError(t, err) + require.Equal(t, tcase.partitionedGroupInfo, *readRes) + }) + } +} + +func TestGetPartitionIDsByBlock(t *testing.T) { + ulid0 := ulid.MustNew(0, nil) + ulid1 := ulid.MustNew(1, nil) + ulid2 := ulid.MustNew(2, nil) + ulid3 := ulid.MustNew(3, nil) + partitionedGroupInfo := PartitionedGroupInfo{ + PartitionedGroupID: uint32(12345), + PartitionCount: 3, + Partitions: []Partition{ + { + PartitionID: 0, + Blocks: []ulid.ULID{ + ulid0, + ulid1, + }, + }, + { + PartitionID: 1, + Blocks: []ulid.ULID{ + ulid0, + ulid2, + }, + }, + { + PartitionID: 2, + Blocks: []ulid.ULID{ + ulid0, + ulid1, + ulid2, + ulid3, + }, + }, + }, + RangeStart: (1 * time.Hour).Milliseconds(), + RangeEnd: (2 * time.Hour).Milliseconds(), + Version: VisitMarkerVersion1, + } + + res0 := partitionedGroupInfo.getPartitionIDsByBlock(ulid0) + require.Equal(t, 3, len(res0)) + require.Contains(t, res0, 0) + require.Contains(t, res0, 1) + require.Contains(t, res0, 2) + + res1 := partitionedGroupInfo.getPartitionIDsByBlock(ulid1) + require.Equal(t, 2, len(res1)) + require.Contains(t, res1, 0) + require.Contains(t, res1, 2) + + res2 := partitionedGroupInfo.getPartitionIDsByBlock(ulid2) + require.Equal(t, 2, len(res2)) + require.Contains(t, res2, 1) + require.Contains(t, res2, 2) + + res3 := partitionedGroupInfo.getPartitionIDsByBlock(ulid3) + require.Equal(t, 1, len(res3)) + require.Contains(t, res3, 2) +} diff --git a/pkg/compactor/sharded_block_populator.go b/pkg/compactor/sharded_block_populator.go new file mode 100644 index 00000000000..3854c94224f --- /dev/null +++ b/pkg/compactor/sharded_block_populator.go @@ -0,0 +1,221 @@ +package compactor + +import ( + "context" + "io" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "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" + "github.com/prometheus/prometheus/tsdb/index" +) + +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) (err error) { + if len(blocks) == 0 { + return errors.New("cannot populate block from no readers") + } + + var ( + sets []storage.ChunkSeriesSet + setsMtx sync.Mutex + symbols index.StringIter + closers []io.Closer + overlapping bool + ) + 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) + + k, v := index.AllPostingsKey() + all, err := indexr.Postings(ctx, k, v) + if err != nil { + return err + } + all = indexr.SortedPostings(all) + g.Go(func() error { + shardStart := time.Now() + shardedPosting, 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)) + setsMtx.Unlock() + return nil + }) + syms := indexr.Symbols() + if i == 0 { + symbols = syms + continue + } + symbols = tsdb.NewMergedStringIter(symbols, syms) + } + if err := g.Wait(); err != nil { + return err + } + + for symbols.Next() { + if err := indexw.AddSymbol(symbols.At()); err != nil { + return errors.Wrap(err, "add symbol") + } + } + if symbols.Err() != nil { + return errors.Wrap(symbols.Err(), "next 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 00000000000..d35a9f7381d --- /dev/null +++ b/pkg/compactor/sharded_compaction_lifecycle_callback.go @@ -0,0 +1,238 @@ +package compactor + +import ( + "context" + "path" + "path/filepath" + "sort" + "strings" + "sync" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/tsdb" + "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" + "github.com/thanos-io/thanos/pkg/runutil" + "golang.org/x/sync/errgroup" + + "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" +) + +type ShardedCompactionLifecycleCallback struct { + ctx context.Context + userBucket objstore.InstrumentedBucket + logger log.Logger + metaSyncConcurrency int + compactDir string + partitionedGroupInfoReadFailed prometheus.Counter +} + +func (c ShardedCompactionLifecycleCallback) PreCompactionCallback(_ context.Context, logger log.Logger, g *compact.Group, meta []*metadata.Meta) error { + // 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(ctx context.Context, logger log.Logger, cg *compact.Group, blockID ulid.ULID) error { + partitionInfo, err := ConvertToPartitionInfo(cg.Extensions()) + if err != nil { + return err + } + if partitionInfo == nil { + return nil + } + partitionedGroupID := partitionInfo.PartitionedGroupID + partitionedGroupInfo, err := ReadPartitionedGroupInfo(ctx, c.userBucket, logger, partitionedGroupID, c.partitionedGroupInfoReadFailed) + if err != nil { + return err + } + + // Only check potential sample missing from result block here. Warning log + // will be emitted. But no action would be taken. + c.isSampleMissing(partitionedGroupInfo) + + // Only try to delete PartitionedGroupFile if there is only one partition. + // For partition count greater than one, cleaner should handle the deletion. + if partitionedGroupInfo.PartitionCount == 1 { + partitionedGroupFile := GetPartitionedGroupFile(partitionedGroupID) + if err := c.userBucket.Delete(ctx, partitionedGroupFile); err != nil { + level.Warn(logger).Log("msg", "failed to delete partitioned group info", "partitioned_group_id", partitionedGroupID, "partitioned_group_info", partitionedGroupFile, "err", err) + } else { + level.Info(logger).Log("msg", "deleted partitioned group info", "partitioned_group_id", partitionedGroupID, "partitioned_group_info", partitionedGroupFile) + } + } + return nil +} + +func (c ShardedCompactionLifecycleCallback) GetBlockPopulator(_ context.Context, logger log.Logger, cg *compact.Group) (tsdb.BlockPopulator, error) { + partitionInfo, err := ConvertToPartitionInfo(cg.Extensions()) + if err != nil { + return nil, err + } + if partitionInfo == nil { + return tsdb.DefaultBlockPopulator{}, nil + } + if partitionInfo.PartitionCount <= 0 { + partitionInfo = &PartitionInfo{ + PartitionCount: 1, + PartitionID: partitionInfo.PartitionID, + PartitionedGroupID: partitionInfo.PartitionedGroupID, + } + cg.SetExtensions(&CortexMetaExtensions{ + PartitionInfo: partitionInfo, + }) + } + populateBlockFunc := ShardedBlockPopulator{ + partitionCount: partitionInfo.PartitionCount, + partitionID: partitionInfo.PartitionID, + logger: logger, + } + return populateBlockFunc, nil +} + +func (c ShardedCompactionLifecycleCallback) isSampleMissing(partitionedGroupInfo *PartitionedGroupInfo) bool { + allBlocks, err := c.getBlocksInTimeRange(partitionedGroupInfo.RangeStart, partitionedGroupInfo.RangeEnd) + if err != nil { + level.Warn(c.logger).Log("msg", "unable to get blocks in time range", "partitioned_group_id", partitionedGroupInfo.PartitionedGroupID, "range_start", partitionedGroupInfo.RangeStart, "range_end", partitionedGroupInfo.RangeEnd) + return true + } + resultBlocks := c.getBlocksByPartitionedGroupID(partitionedGroupInfo, allBlocks) + if len(resultBlocks) != partitionedGroupInfo.PartitionCount { + level.Warn(c.logger).Log("msg", "number of result blocks does not match partition count", "partitioned_group_id", partitionedGroupInfo.PartitionedGroupID, "partition_count", partitionedGroupInfo.PartitionCount, "result_blocks", getMetaInfo(resultBlocks)) + return true + } + resultSamples := countSamplesFromMeta(resultBlocks) + sourceBlocksInPartitionGroup := partitionedGroupInfo.getAllBlocks() + sourceBlocks := make(map[ulid.ULID]*metadata.Meta) + for _, b := range sourceBlocksInPartitionGroup { + if _, ok := allBlocks[b]; !ok { + level.Warn(c.logger).Log("msg", "unable to find source block", "partitioned_group_id", partitionedGroupInfo.PartitionedGroupID, "source_block_id", b.String()) + return true + } + sourceBlocks[b] = allBlocks[b] + } + sourceSamples := countSamplesFromMeta(sourceBlocks) + if sourceSamples > resultSamples { + sourceBlocksInfo := getMetaInfo(sourceBlocks) + resultBlocksInfo := getMetaInfo(resultBlocks) + level.Warn(c.logger).Log("msg", "samples are missing from result blocks", "partitioned_group_id", partitionedGroupInfo.PartitionedGroupID, "result_samples", resultSamples, "source_samples", sourceSamples, "result_blocks", resultBlocksInfo, "source_blocks", sourceBlocksInfo) + return true + } + level.Info(c.logger).Log("msg", "samples check complete successfully", "partitioned_group_id", partitionedGroupInfo.PartitionedGroupID, "result_samples", resultSamples, "source_samples", sourceSamples) + return false +} + +func (c ShardedCompactionLifecycleCallback) getBlocksByPartitionedGroupID(partitionedGroupInfo *PartitionedGroupInfo, blocksMeta map[ulid.ULID]*metadata.Meta) map[ulid.ULID]*metadata.Meta { + blocks := make(map[ulid.ULID]*metadata.Meta) + for b, meta := range blocksMeta { + partitionInfo, err := GetPartitionInfo(*meta) + if partitionInfo == nil || err != nil { + level.Debug(c.logger).Log("msg", "unable to get partition info for block", "block", b.String(), "meta", *meta) + continue + } + if partitionInfo.PartitionedGroupID == partitionedGroupInfo.PartitionedGroupID { + blocks[b] = meta + } + } + return blocks +} + +func (c ShardedCompactionLifecycleCallback) getBlocksInTimeRange(rangeStart int64, rangeEnd int64) (map[ulid.ULID]*metadata.Meta, error) { + discovered := make(map[ulid.ULID]struct{}) + idx, err := bucketindex.ReadUserIndex(c.ctx, c.userBucket, c.logger) + if err != nil { + return nil, err + } + var blockIDs []ulid.ULID + for _, b := range idx.Blocks { + discovered[b.ID] = struct{}{} + if b.MinTime >= rangeStart && b.MaxTime <= rangeEnd { + blockIDs = append(blockIDs, b.ID) + } + } + err = c.userBucket.Iter(c.ctx, "", func(name string) error { + if id, ok := block.IsBlockDir(name); ok { + if _, ok := discovered[id]; !ok { + blockIDs = append(blockIDs, id) + } + } + return nil + }) + if err != nil { + return nil, err + } + blocks, err := c.getMetaByBlockIDs(blockIDs) + if err != nil { + return nil, err + } + result := make(map[ulid.ULID]*metadata.Meta) + for id, b := range blocks { + if b.MinTime >= rangeStart && b.MaxTime <= rangeEnd { + level.Debug(c.logger).Log("msg", "found block in time range", "range_start", rangeStart, "range_end", rangeEnd, "block", id.String(), "block_range_start", b.MinTime, "block_range_end", b.MaxTime) + result[id] = b + } + } + return result, nil +} + +func (c ShardedCompactionLifecycleCallback) getMetaByBlockIDs(blocks []ulid.ULID) (map[ulid.ULID]*metadata.Meta, error) { + var mtx sync.Mutex + blocksMeta := make(map[ulid.ULID]*metadata.Meta, len(blocks)) + g, _ := errgroup.WithContext(c.ctx) + g.SetLimit(c.metaSyncConcurrency) + for _, b := range blocks { + select { + case <-c.ctx.Done(): + return nil, c.ctx.Err() + default: + } + blockID := b + g.Go(func() error { + metaReader, err := c.userBucket.ReaderWithExpectedErrs(c.userBucket.IsObjNotFoundErr).Get(c.ctx, path.Join(blockID.String(), metadata.MetaFilename)) + if err != nil { + return err + } + meta, err := metadata.Read(metaReader) + if err != nil { + return err + } + mtx.Lock() + level.Debug(c.logger).Log("msg", "read metadata for block", "block", blockID, "meta", *meta) + blocksMeta[blockID] = meta + mtx.Unlock() + return nil + }) + } + if err := g.Wait(); err != nil { + return nil, err + } + return blocksMeta, nil +} + +func countSamplesFromMeta(blocks map[ulid.ULID]*metadata.Meta) uint64 { + samples := uint64(0) + for _, b := range blocks { + samples += b.Stats.NumSamples + } + return samples +} + +func getMetaInfo(blocks map[ulid.ULID]*metadata.Meta) string { + var ids []string + for b := range blocks { + ids = append(ids, b.String()) + } + sort.Strings(ids) + return strings.Join(ids, ",") +} 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 00000000000..09157c895b8 --- /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 00000000000..7d03d137ede --- /dev/null +++ b/pkg/compactor/sharded_posting.go @@ -0,0 +1,25 @@ +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, error) { + bufChks := make([]chunks.Meta, 0) + series := make([]storage.SeriesRef, 0) + var builder labels.ScratchBuilder + for postings.Next() { + err := labelsFn(postings.At(), &builder, &bufChks) + if err != nil { + return nil, err + } + if builder.Labels().Hash()%partitionCount == partitionID { + posting := postings.At() + series = append(series, posting) + } + } + return index.NewListPostings(series), nil +} diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index 2d4dc748cb3..9840f004e91 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -4,6 +4,8 @@ import ( "context" "fmt" "hash/fnv" + "math" + "math/rand" "sort" "strings" "time" @@ -16,6 +18,7 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" "github.com/thanos-io/objstore" + thanosblock "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/compact" @@ -50,11 +53,12 @@ type ShuffleShardingGrouper struct { ringLifecyclerAddr string ringLifecyclerID string - blockVisitMarkerTimeout time.Duration - blockVisitMarkerReadFailed prometheus.Counter - blockVisitMarkerWriteFailed prometheus.Counter - - noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark + noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark + visitMarkerTimeout time.Duration + visitMarkerReadFailed prometheus.Counter + visitMarkerWriteFailed prometheus.Counter + partitionedGroupInfoReadFailed prometheus.Counter + partitionedGroupInfoWriteFailed prometheus.Counter } func NewShuffleShardingGrouper( @@ -78,9 +82,11 @@ func NewShuffleShardingGrouper( blockFilesConcurrency int, blocksFetchConcurrency int, compactionConcurrency int, - blockVisitMarkerTimeout time.Duration, - blockVisitMarkerReadFailed prometheus.Counter, - blockVisitMarkerWriteFailed prometheus.Counter, + visitMarkerTimeout time.Duration, + visitMarkerReadFailed prometheus.Counter, + visitMarkerWriteFailed prometheus.Counter, + partitionedGroupInfoReadFailed prometheus.Counter, + partitionedGroupInfoWriteFailed prometheus.Counter, noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark, ) *ShuffleShardingGrouper { if logger == nil { @@ -120,19 +126,21 @@ func NewShuffleShardingGrouper( Name: "thanos_compact_group_vertical_compactions_total", Help: "Total number of group compaction attempts that resulted in a new block based on overlapping blocks.", }, []string{"group"}), - compactorCfg: compactorCfg, - ring: ring, - ringLifecyclerAddr: ringLifecyclerAddr, - ringLifecyclerID: ringLifecyclerID, - limits: limits, - userID: userID, - blockFilesConcurrency: blockFilesConcurrency, - blocksFetchConcurrency: blocksFetchConcurrency, - compactionConcurrency: compactionConcurrency, - blockVisitMarkerTimeout: blockVisitMarkerTimeout, - blockVisitMarkerReadFailed: blockVisitMarkerReadFailed, - blockVisitMarkerWriteFailed: blockVisitMarkerWriteFailed, - noCompBlocksFunc: noCompBlocksFunc, + compactorCfg: compactorCfg, + ring: ring, + ringLifecyclerAddr: ringLifecyclerAddr, + ringLifecyclerID: ringLifecyclerID, + limits: limits, + userID: userID, + blockFilesConcurrency: blockFilesConcurrency, + blocksFetchConcurrency: blocksFetchConcurrency, + compactionConcurrency: compactionConcurrency, + visitMarkerTimeout: visitMarkerTimeout, + visitMarkerReadFailed: visitMarkerReadFailed, + visitMarkerWriteFailed: visitMarkerWriteFailed, + partitionedGroupInfoReadFailed: partitionedGroupInfoReadFailed, + partitionedGroupInfoWriteFailed: partitionedGroupInfoWriteFailed, + noCompBlocksFunc: noCompBlocksFunc, } } @@ -165,9 +173,6 @@ func (g *ShuffleShardingGrouper) Groups(blocks map[ulid.ULID]*metadata.Meta) (re level.Debug(g.logger).Log("msg", "compactor is not on the current sub-ring skipping user", "user", g.userID) return outGroups, nil } - // Metrics for the remaining planned compactions - var remainingCompactions = 0. - defer func() { g.remainingPlannedCompactions.Set(remainingCompactions) }() var groups []blocksGroup for _, mainBlocks := range mainGroups { @@ -181,29 +186,29 @@ func (g *ShuffleShardingGrouper) Groups(blocks map[ulid.ULID]*metadata.Meta) (re sort.SliceStable(groups, func(i, j int) bool { iGroup := groups[i] jGroup := groups[j] - iMinTime := iGroup.minTime() - iMaxTime := iGroup.maxTime() - jMinTime := jGroup.minTime() - jMaxTime := jGroup.maxTime() - iLength := iMaxTime - iMinTime - jLength := jMaxTime - jMinTime + iRangeStart := iGroup.rangeStart + iRangeEnd := iGroup.rangeEnd + jRangeStart := jGroup.rangeStart + jRangeEnd := jGroup.rangeEnd + iLength := iRangeEnd - iRangeStart + jLength := jRangeEnd - jRangeStart if iLength != jLength { return iLength < jLength } - if iMinTime != jMinTime { - return iMinTime < jMinTime + if iRangeStart != jRangeStart { + return iRangeStart < jRangeStart } - iGroupHash := hashGroup(g.userID, iGroup.rangeStart, iGroup.rangeEnd) + iGroupHash := HashGroup(g.userID, iRangeStart, iRangeEnd) iGroupKey := createGroupKey(iGroupHash, iGroup) - jGroupHash := hashGroup(g.userID, jGroup.rangeStart, jGroup.rangeEnd) + jGroupHash := HashGroup(g.userID, jRangeStart, jRangeEnd) jGroupKey := createGroupKey(jGroupHash, jGroup) // Guarantee stable sort for tests. return iGroupKey < jGroupKey }) -mainLoop: + var blockGroups []*blocksGroup for _, group := range groups { var blockIds []string for _, block := range group.blocks { @@ -217,45 +222,88 @@ mainLoop: continue } - groupHash := hashGroup(g.userID, group.rangeStart, group.rangeEnd) + groupHash := HashGroup(g.userID, group.rangeStart, group.rangeEnd) - if isVisited, err := g.isGroupVisited(group.blocks, g.ringLifecyclerID); err != nil { - level.Warn(g.logger).Log("msg", "unable to check if blocks in group are visited", "group hash", groupHash, "err", err, "group", group.String()) - continue - } else if isVisited { - level.Info(g.logger).Log("msg", "skipping group because at least one block in group is visited", "group_hash", groupHash) + partitionedGroupInfo, err := g.generatePartitionBlockGroup(group, groupHash) + if err != nil { + level.Warn(g.logger).Log("msg", "unable to update partitioned group info", "partitioned_group_id", groupHash, "err", err) continue } + level.Debug(g.logger).Log("msg", "generated partitioned groups", "groups", partitionedGroupInfo) + + partitionedGroupID := partitionedGroupInfo.PartitionedGroupID + partitionCount := partitionedGroupInfo.PartitionCount + partitionAdded := 0 + // Randomly pick partitions from partitioned group to avoid all compactors + // trying to get same partition at same time. + r := rand.New(rand.NewSource(time.Now().UnixMicro() + int64(hashString(g.ringLifecyclerID)))) + for _, i := range r.Perm(len(partitionedGroupInfo.Partitions)) { + partition := partitionedGroupInfo.Partitions[i] + partitionID := partition.PartitionID + partitionedGroup, err := createBlocksGroup(blocks, partition.Blocks, partitionedGroupInfo.RangeStart, partitionedGroupInfo.RangeEnd) + if err != nil { + level.Error(g.logger).Log("msg", "unable to create partitioned group", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "err", err) + continue + } + partitionedGroup.groupHash = groupHash + partitionedGroup.partitionedGroupInfo = partitionedGroupInfo + partitionedGroup.partition = partition + blockGroups = append(blockGroups, partitionedGroup) + partitionAdded++ - remainingCompactions++ - groupKey := createGroupKey(groupHash, group) - - level.Info(g.logger).Log("msg", "found compactable group for user", "group_hash", groupHash, "group", group.String()) - blockVisitMarker := BlockVisitMarker{ - VisitTime: time.Now().Unix(), - CompactorID: g.ringLifecyclerID, - Version: VisitMarkerVersion1, + level.Debug(g.logger).Log("msg", "found available partition", "group_hash", groupHash, "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "partition_count", partitionCount) } - markBlocksVisited(g.ctx, g.bkt, g.logger, group.blocks, blockVisitMarker, g.blockVisitMarkerWriteFailed) + level.Info(g.logger).Log("msg", fmt.Sprintf("found available partitions: %d", partitionAdded)) + } + level.Info(g.logger).Log("msg", fmt.Sprintf("total possible group for compaction: %d", len(blockGroups))) + + // Metrics for the remaining planned compactions + var remainingCompactions = float64(len(blockGroups)) + defer func() { g.remainingPlannedCompactions.Set(remainingCompactions) }() - // All the blocks within the same group have the same downsample - // resolution and external labels. - resolution := group.blocks[0].Thanos.Downsample.Resolution - externalLabels := labels.FromMap(group.blocks[0].Thanos.Labels) + for _, partitionedGroup := range blockGroups { + groupHash := partitionedGroup.groupHash + partitionedGroupID := partitionedGroup.partitionedGroupInfo.PartitionedGroupID + partitionCount := partitionedGroup.partitionedGroupInfo.PartitionCount + partitionID := partitionedGroup.partition.PartitionID + if isVisited, err := g.isGroupVisited(partitionedGroupID, partitionID, g.ringLifecyclerID); err != nil { + level.Warn(g.logger).Log("msg", "unable to check if partition is visited", "group hash", groupHash, "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "err", err, "group", partitionedGroup.String()) + continue + } else if isVisited { + level.Info(g.logger).Log("msg", "skipping group because partition is visited", "group_hash", groupHash, "partitioned_group_id", partitionedGroupID, "partition_id", partitionID) + remainingCompactions-- + continue + } + partitionedGroupKey := createGroupKeyWithPartitionID(groupHash, partitionID, *partitionedGroup) + + level.Info(g.logger).Log("msg", "found compactable group for user", "group_hash", groupHash, "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "partition_count", partitionCount, "group", partitionedGroup.String()) + begin := time.Now() + partitionVisitMarker := PartitionVisitMarker{ + VisitTime: time.Now().Unix(), + CompactorID: g.ringLifecyclerID, + Status: Pending, + PartitionedGroupID: partitionedGroupID, + PartitionID: partitionID, + Version: VisitMarkerVersion1, + } + markPartitionVisited(g.ctx, g.bkt, g.logger, partitionVisitMarker, g.visitMarkerWriteFailed) + level.Info(g.logger).Log("msg", "marked partition visited in group", "group_hash", groupHash, "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "partition_count", partitionCount, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds(), "group", partitionedGroup.String()) + resolution := partitionedGroup.blocks[0].Thanos.Downsample.Resolution + externalLabels := labels.FromMap(partitionedGroup.blocks[0].Thanos.Labels) thanosGroup, err := compact.NewGroup( - log.With(g.logger, "groupKey", groupKey, "rangeStart", group.rangeStartTime().String(), "rangeEnd", group.rangeEndTime().String(), "externalLabels", externalLabels, "downsampleResolution", resolution), + log.With(g.logger, "groupKey", partitionedGroupKey, "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "partition_count", partitionCount, "rangeStart", partitionedGroup.rangeStartTime().String(), "rangeEnd", partitionedGroup.rangeEndTime().String(), "externalLabels", externalLabels, "downsampleResolution", resolution), g.bkt, - groupKey, + partitionedGroupKey, externalLabels, resolution, g.acceptMalformedIndex, true, // Enable vertical compaction. - g.compactions.WithLabelValues(groupKey), - g.compactionRunsStarted.WithLabelValues(groupKey), - g.compactionRunsCompleted.WithLabelValues(groupKey), - g.compactionFailures.WithLabelValues(groupKey), - g.verticalCompactions.WithLabelValues(groupKey), + g.compactions.WithLabelValues(partitionedGroupKey), + g.compactionRunsStarted.WithLabelValues(partitionedGroupKey), + g.compactionRunsCompleted.WithLabelValues(partitionedGroupKey), + g.compactionFailures.WithLabelValues(partitionedGroupKey), + g.verticalCompactions.WithLabelValues(partitionedGroupKey), g.garbageCollectedBlocks, g.blocksMarkedForDeletion, g.blocksMarkedForNoCompact, @@ -264,18 +312,28 @@ mainLoop: g.blocksFetchConcurrency, ) if err != nil { - return nil, errors.Wrap(err, "create compaction group") + level.Error(g.logger).Log("msg", "failed to create partitioned group", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "partition_count", partitionCount, "blocks", partitionedGroup.partition.Blocks) } - for _, m := range group.blocks { + for _, m := range partitionedGroup.blocks { if err := thanosGroup.AppendMeta(m); err != nil { - return nil, errors.Wrap(err, "add block to compaction group") + level.Error(g.logger).Log("msg", "failed to add block to partitioned group", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "partition_count", partitionCount, "block", m.ULID) } } + thanosGroup.SetExtensions(&CortexMetaExtensions{ + PartitionInfo: &PartitionInfo{ + PartitionedGroupID: partitionedGroupID, + PartitionCount: partitionCount, + PartitionID: partitionID, + }, + }) outGroups = append(outGroups, thanosGroup) - if len(outGroups) == g.compactionConcurrency { - break mainLoop + level.Debug(g.logger).Log("msg", "added partition to compaction groups", "group_hash", groupHash, "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "partition_count", partitionCount) + // Grouper holds additional groups for compaction. In case, it lost + // competition for the first group it claimed. + if len(outGroups) >= g.compactionConcurrency+int(math.Min(float64(g.compactionConcurrency*3), 5)) { + break } } @@ -284,22 +342,176 @@ mainLoop: return outGroups, nil } -func (g *ShuffleShardingGrouper) isGroupVisited(blocks []*metadata.Meta, compactorID string) (bool, error) { - for _, block := range blocks { - blockID := block.ULID.String() - blockVisitMarker, err := ReadBlockVisitMarker(g.ctx, g.bkt, g.logger, blockID, g.blockVisitMarkerReadFailed) - if err != nil { - if errors.Is(err, ErrorBlockVisitMarkerNotFound) { - level.Debug(g.logger).Log("msg", "no visit marker file for block", "blockID", blockID) - continue +func (g *ShuffleShardingGrouper) generatePartitionBlockGroup(group blocksGroup, groupHash uint32) (*PartitionedGroupInfo, error) { + partitionedGroupInfo, err := g.partitionBlockGroup(group, groupHash) + if err != nil { + return nil, err + } + updatedPartitionedGroupInfo, err := UpdatePartitionedGroupInfo(g.ctx, g.bkt, g.logger, *partitionedGroupInfo, g.partitionedGroupInfoReadFailed, g.partitionedGroupInfoWriteFailed) + if err != nil { + return nil, err + } + return updatedPartitionedGroupInfo, nil +} + +func (g *ShuffleShardingGrouper) partitionBlockGroup(group blocksGroup, groupHash uint32) (*PartitionedGroupInfo, error) { + partitionCount := g.calculatePartitionCount(group) + blocksByMinTime := g.groupBlocksByMinTime(group) + partitionedGroups, err := g.partitionBlocksGroup(partitionCount, blocksByMinTime, group.rangeStart, group.rangeEnd) + if err != nil { + return nil, err + } + + var partitions []Partition + for partitionID, partitionedGroup := range partitionedGroups { + var blockIDs []ulid.ULID + for _, m := range partitionedGroup.blocks { + blockIDs = append(blockIDs, m.ULID) + } + partitions = append(partitions, Partition{ + PartitionID: partitionID, + Blocks: blockIDs, + }) + } + partitionedGroupInfo := PartitionedGroupInfo{ + PartitionedGroupID: groupHash, + PartitionCount: partitionCount, + Partitions: partitions, + RangeStart: group.rangeStart, + RangeEnd: group.rangeEnd, + CreationTime: time.Now().Unix(), + Version: PartitionedGroupInfoVersion1, + } + return &partitionedGroupInfo, nil +} + +func (g *ShuffleShardingGrouper) calculatePartitionCount(group blocksGroup) int { + indexSizeLimit := g.limits.CompactorPartitionIndexSizeLimitInBytes(g.userID) + seriesCountLimit := g.limits.CompactorPartitionSeriesCountLimit(g.userID) + totalIndexSizeInBytes := int64(0) + totalSeriesCount := int64(0) + for _, block := range group.blocks { + blockFiles := block.Thanos.Files + totalSeriesCount += int64(block.Stats.NumSeries) + var indexFile *metadata.File + for _, file := range blockFiles { + if file.RelPath == thanosblock.IndexFilename { + indexFile = &file + } + } + if indexFile == nil { + level.Debug(g.logger).Log("msg", "unable to find index file in metadata", "block", block.ULID) + break + } + indexSize := indexFile.SizeBytes + totalIndexSizeInBytes += indexSize + } + partitionNumberBasedOnIndex := 1 + if indexSizeLimit > 0 && totalIndexSizeInBytes > indexSizeLimit { + partitionNumberBasedOnIndex = g.findNearestPartitionNumber(float64(totalIndexSizeInBytes), float64(indexSizeLimit)) + } + partitionNumberBasedOnSeries := 1 + if seriesCountLimit > 0 && totalSeriesCount > seriesCountLimit { + partitionNumberBasedOnSeries = g.findNearestPartitionNumber(float64(totalSeriesCount), float64(seriesCountLimit)) + } + partitionNumber := partitionNumberBasedOnIndex + if partitionNumberBasedOnSeries > partitionNumberBasedOnIndex { + partitionNumber = partitionNumberBasedOnSeries + } + level.Debug(g.logger).Log("msg", "calculated partition number for group", "group", group.String(), "partition_number", partitionNumber, "total_index_size", totalIndexSizeInBytes, "index_size_limit", indexSizeLimit, "total_series_count", totalSeriesCount, "series_count_limit", seriesCountLimit) + return partitionNumber +} + +func (g *ShuffleShardingGrouper) findNearestPartitionNumber(size float64, limit float64) int { + return int(math.Pow(2, math.Ceil(math.Log2(size/limit)))) +} + +func (g *ShuffleShardingGrouper) groupBlocksByMinTime(group blocksGroup) map[int64][]*metadata.Meta { + blocksByMinTime := make(map[int64][]*metadata.Meta) + for _, block := range group.blocks { + blockRange := block.MaxTime - block.MinTime + minTime := block.MinTime + for _, tr := range g.compactorCfg.BlockRanges.ToMilliseconds() { + if blockRange <= tr { + minTime = tr * (block.MinTime / tr) + break + } + } + blocksByMinTime[minTime] = append(blocksByMinTime[minTime], block) + } + return blocksByMinTime +} + +func (g *ShuffleShardingGrouper) partitionBlocksGroup(partitionCount int, blocksByMinTime map[int64][]*metadata.Meta, rangeStart int64, rangeEnd int64) (map[int]blocksGroup, error) { + partitionedGroups := make(map[int]blocksGroup) + addToPartitionedGroups := func(blocks []*metadata.Meta, partitionID int) { + if _, ok := partitionedGroups[partitionID]; !ok { + partitionedGroups[partitionID] = blocksGroup{ + rangeStart: rangeStart, + rangeEnd: rangeEnd, + blocks: []*metadata.Meta{}, + } + } + partitionedGroup := partitionedGroups[partitionID] + partitionedGroup.blocks = append(partitionedGroup.blocks, blocks...) + partitionedGroups[partitionID] = partitionedGroup + } + + for _, blocksInSameTimeInterval := range blocksByMinTime { + numOfBlocks := len(blocksInSameTimeInterval) + numBlocksCheck := math.Log2(float64(numOfBlocks)) + if math.Ceil(numBlocksCheck) == math.Floor(numBlocksCheck) { + // Case that number of blocks in this time interval is 2^n, should + // use modulo calculation to find blocks for each partition ID. + for _, block := range blocksInSameTimeInterval { + partitionInfo, err := GetPartitionInfo(*block) + if err != nil { + return nil, err + } + if partitionInfo == nil || partitionInfo.PartitionCount < 1 { + // For legacy blocks with level > 1, treat PartitionID is always 0. + // So it can be included in every partition. + defaultPartitionInfo := DefaultPartitionInfo + partitionInfo = &defaultPartitionInfo + } + if partitionInfo.PartitionCount < partitionCount { + for partitionID := partitionInfo.PartitionID; partitionID < partitionCount; partitionID += partitionInfo.PartitionCount { + addToPartitionedGroups([]*metadata.Meta{block}, partitionID) + } + } else if partitionInfo.PartitionCount == partitionCount { + addToPartitionedGroups([]*metadata.Meta{block}, partitionInfo.PartitionID) + } else { + addToPartitionedGroups([]*metadata.Meta{block}, partitionInfo.PartitionID%partitionCount) + } + } + } else { + // Case that number of blocks in this time interval is not 2^n, should + // include all blocks in all partitions. + for partitionID := 0; partitionID < partitionCount; partitionID++ { + addToPartitionedGroups(blocksInSameTimeInterval, partitionID) } - level.Error(g.logger).Log("msg", "unable to read block visit marker file", "blockID", blockID, "err", err) - return true, err } - if compactorID != blockVisitMarker.CompactorID && blockVisitMarker.isVisited(g.blockVisitMarkerTimeout) { - level.Debug(g.logger).Log("msg", fmt.Sprintf("visited block: %s", blockID)) - return true, nil + } + return partitionedGroups, nil +} + +func (g *ShuffleShardingGrouper) isGroupVisited(partitionedGroupID uint32, partitionID int, compactorID string) (bool, error) { + partitionVisitMarker, err := ReadPartitionVisitMarker(g.ctx, g.bkt, g.logger, partitionedGroupID, partitionID, g.visitMarkerReadFailed) + if err != nil { + if errors.Is(err, ErrorPartitionVisitMarkerNotFound) { + level.Warn(g.logger).Log("msg", "no visit marker file for partition", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID) + return false, nil } + level.Error(g.logger).Log("msg", "unable to read partition visit marker file", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "err", err) + return true, err + } + if partitionVisitMarker.isCompleted() { + level.Info(g.logger).Log("msg", "partition visit marker with partition ID is completed", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID) + return true, nil + } + if compactorID != partitionVisitMarker.CompactorID && partitionVisitMarker.isVisited(g.visitMarkerTimeout, partitionID) { + level.Info(g.logger).Log("msg", "visited partition with partition ID", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID) + return true, nil } return false, nil } @@ -316,27 +528,53 @@ func (g *ShuffleShardingGrouper) checkSubringForCompactor() (bool, error) { return rs.Includes(g.ringLifecyclerAddr), nil } -// Get the hash of a group based on the UserID, and the starting and ending time of the group's range. -func hashGroup(userID string, rangeStart int64, rangeEnd int64) uint32 { +// HashGroup Get the hash of a group based on the UserID, and the starting and ending time of the group's range. +func HashGroup(userID string, rangeStart int64, rangeEnd int64) uint32 { groupString := fmt.Sprintf("%v%v%v", userID, rangeStart, rangeEnd) - groupHasher := fnv.New32a() + + return hashString(groupString) +} + +func hashString(s string) uint32 { + hasher := fnv.New32a() // Hasher never returns err. - _, _ = groupHasher.Write([]byte(groupString)) - groupHash := groupHasher.Sum32() + _, _ = hasher.Write([]byte(s)) + result := hasher.Sum32() - return groupHash + return result } func createGroupKey(groupHash uint32, group blocksGroup) string { return fmt.Sprintf("%v%s", groupHash, group.blocks[0].Thanos.GroupKey()) } +func createGroupKeyWithPartitionID(groupHash uint32, partitionID int, group blocksGroup) string { + return fmt.Sprintf("%v%d%s", groupHash, partitionID, group.blocks[0].Thanos.GroupKey()) +} + +func createBlocksGroup(blocks map[ulid.ULID]*metadata.Meta, blockIDs []ulid.ULID, rangeStart int64, rangeEnd int64) (*blocksGroup, error) { + var group blocksGroup + group.rangeStart = rangeStart + group.rangeEnd = rangeEnd + for _, blockID := range blockIDs { + m, ok := blocks[blockID] + if !ok { + return nil, fmt.Errorf("block not found: %s", blockID) + } + group.blocks = append(group.blocks, m) + } + return &group, nil +} + // blocksGroup struct and functions copied and adjusted from https://github.com/cortexproject/cortex/pull/2616 type blocksGroup struct { - rangeStart int64 // Included. - rangeEnd int64 // Excluded. - blocks []*metadata.Meta - key string + rangeStart int64 // Included. + rangeEnd int64 // Excluded. + blocks []*metadata.Meta + key string + groupHash uint32 + partitionedGroupInfo *PartitionedGroupInfo + partition Partition } // overlaps returns whether the group range overlaps with the input group. @@ -377,12 +615,6 @@ func (g blocksGroup) rangeLength() int64 { return g.rangeEnd - g.rangeStart } -// minTime returns the MinTime across all blocks in the group. -func (g blocksGroup) minTime() int64 { - // Blocks are expected to be sorted by MinTime. - return g.blocks[0].MinTime -} - // maxTime returns the MaxTime across all blocks in the group. func (g blocksGroup) maxTime() int64 { max := g.blocks[0].MaxTime @@ -428,7 +660,25 @@ func groupBlocksByCompactableRanges(blocks []*metadata.Meta, ranges []int64) []b } } - groups = append(groups, group) + // Ensure this group has at least one block having PartitionedGroupID is 0 or at least + // one block having different PartitionedGroupID as others. Because we don't want to + // compact a group of blocks all having same non-zero PartitionedGroupID. + firstBlockPartitionInfo, err := GetPartitionInfo(*group.blocks[0]) + if err != nil || firstBlockPartitionInfo == nil { + defaultPartitionInfo := DefaultPartitionInfo + firstBlockPartitionInfo = &defaultPartitionInfo + } + for _, block := range group.blocks { + blockPartitionInfo, err := GetPartitionInfo(*block) + if err != nil || blockPartitionInfo == nil { + defaultPartitionInfo := DefaultPartitionInfo + blockPartitionInfo = &defaultPartitionInfo + } + if blockPartitionInfo.PartitionedGroupID <= 0 || blockPartitionInfo.PartitionedGroupID != firstBlockPartitionInfo.PartitionedGroupID { + groups = append(groups, group) + continue nextGroup + } + } } } @@ -445,12 +695,6 @@ func groupBlocksByCompactableRanges(blocks []*metadata.Meta, ranges []int64) []b continue } - // If the group covers the full range, it's fine. - if group.maxTime()-group.minTime() == group.rangeLength() { - idx++ - continue - } - // If the group's maxTime is after 1 block range, we can compact assuming that // all the required blocks have already been uploaded. if int64(ulid.Now()) > group.maxTime()+group.rangeLength() { @@ -491,6 +735,11 @@ func groupBlocksByRange(blocks []*metadata.Meta, tr int64) []blocksGroup { continue } + if skipHighLevelBlock(m, tr) { + i++ + continue + } + // Add all blocks to the current group that are within [t0, t0+tr]. for ; i < len(blocks); i++ { // If the block does not start within this group, then we should break the iteration @@ -505,6 +754,10 @@ func groupBlocksByRange(blocks []*metadata.Meta, tr int64) []blocksGroup { continue } + if skipHighLevelBlock(blocks[i], tr) { + continue + } + group.blocks = append(group.blocks, blocks[i]) } @@ -516,6 +769,14 @@ func groupBlocksByRange(blocks []*metadata.Meta, tr int64) []blocksGroup { return ret } +func skipHighLevelBlock(block *metadata.Meta, tr int64) bool { + // Skip blocks that have rounded range equal to tr, and level > 1 + // Because tr is divisible by the previous tr, block range falls in + // (tr/2, tr] should be rounded to tr. + blockRange := block.MaxTime - block.MinTime + return blockRange <= tr && blockRange > tr/2 && block.Compaction.Level > 1 +} + func getRangeStart(m *metadata.Meta, tr int64) int64 { // Compute start of aligned time range of size tr closest to the current block's start. // This code has been copied from TSDB. diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index e237a43a921..fe462e9c6f0 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -4,7 +4,6 @@ import ( "bytes" "context" "encoding/json" - "path" "testing" "time" @@ -17,10 +16,12 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/thanos-io/objstore" + thanosblock "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/cortexproject/cortex/pkg/ring" "github.com/cortexproject/cortex/pkg/storage/bucket" + cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" "github.com/cortexproject/cortex/pkg/util/validation" ) @@ -119,13 +120,13 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { otherCompactorID := "other-compactor" tests := map[string]struct { - concurrency int - ranges []time.Duration - blocks map[ulid.ULID]*metadata.Meta - visitedBlocks []struct { - id ulid.ULID - compactorID string - isExpired bool + concurrency int + ranges []time.Duration + blocks map[ulid.ULID]*metadata.Meta + visitedPartitions []struct { + partitionedGroupID uint32 + compactorID string + isExpired bool } expected [][]ulid.ULID metrics string @@ -216,82 +217,63 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { cortex_compactor_remaining_planned_compactions 0 `, }, - "test group with all blocks visited": { + "test group with partition visited": { concurrency: 1, ranges: []time.Duration{2 * time.Hour, 4 * time.Hour}, - blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid], block1hto2hExt2Ulid: blocks[block1hto2hExt2Ulid], block0hto1hExt2Ulid: blocks[block0hto1hExt2Ulid]}, + blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid]}, expected: [][]ulid.ULID{ - {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, + {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, }, - visitedBlocks: []struct { - id ulid.ULID - compactorID string - isExpired bool + visitedPartitions: []struct { + partitionedGroupID uint32 + compactorID string + isExpired bool }{ - {id: block1hto2hExt2Ulid, compactorID: otherCompactorID, isExpired: false}, - {id: block0hto1hExt2Ulid, compactorID: otherCompactorID, isExpired: false}, + {partitionedGroupID: HashGroup("", 0*time.Hour.Milliseconds(), 2*time.Hour.Milliseconds()), compactorID: otherCompactorID, isExpired: false}, }, metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. # TYPE cortex_compactor_remaining_planned_compactions gauge cortex_compactor_remaining_planned_compactions 1 `, }, - "test group with one block visited": { + "test group with partition visit marker file expired": { concurrency: 1, ranges: []time.Duration{2 * time.Hour, 4 * time.Hour}, - blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid], block1hto2hExt2Ulid: blocks[block1hto2hExt2Ulid], block0hto1hExt2Ulid: blocks[block0hto1hExt2Ulid]}, + blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, + {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, }, - visitedBlocks: []struct { - id ulid.ULID - compactorID string - isExpired bool - }{ - {id: block1hto2hExt2Ulid, compactorID: otherCompactorID, isExpired: false}, - }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. - # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 -`, - }, - "test group block visit marker file expired": { - concurrency: 1, - ranges: []time.Duration{2 * time.Hour, 4 * time.Hour}, - blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid], block1hto2hExt2Ulid: blocks[block1hto2hExt2Ulid], block0hto1hExt2Ulid: blocks[block0hto1hExt2Ulid]}, - expected: [][]ulid.ULID{ - {block1hto2hExt2Ulid, block0hto1hExt2Ulid}, - }, - visitedBlocks: []struct { - id ulid.ULID - compactorID string - isExpired bool + visitedPartitions: []struct { + partitionedGroupID uint32 + compactorID string + isExpired bool }{ - {id: block1hto2hExt2Ulid, compactorID: otherCompactorID, isExpired: true}, - {id: block0hto1hExt2Ulid, compactorID: otherCompactorID, isExpired: true}, + {partitionedGroupID: HashGroup("", 0*time.Hour.Milliseconds(), 2*time.Hour.Milliseconds()), compactorID: otherCompactorID, isExpired: true}, }, metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 + cortex_compactor_remaining_planned_compactions 2 `, }, - "test group with one block visited by current compactor": { + "test group with partition visited by current compactor": { concurrency: 1, ranges: []time.Duration{2 * time.Hour, 4 * time.Hour}, - blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid], block1hto2hExt2Ulid: blocks[block1hto2hExt2Ulid], block0hto1hExt2Ulid: blocks[block0hto1hExt2Ulid]}, + blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid]}, expected: [][]ulid.ULID{ - {block1hto2hExt2Ulid, block0hto1hExt2Ulid}, + {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, + {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, }, - visitedBlocks: []struct { - id ulid.ULID - compactorID string - isExpired bool + visitedPartitions: []struct { + partitionedGroupID uint32 + compactorID string + isExpired bool }{ - {id: block1hto2hExt2Ulid, compactorID: testCompactorID, isExpired: false}, + {partitionedGroupID: HashGroup("", 0*time.Hour.Milliseconds(), 2*time.Hour.Milliseconds()), compactorID: testCompactorID, isExpired: false}, }, metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 + cortex_compactor_remaining_planned_compactions 2 `, }, "test basic grouping with concurrency 2": { @@ -301,10 +283,11 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { expected: [][]ulid.ULID{ {block1hto2hExt2Ulid, block0hto1hExt2Ulid}, {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, + {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, }, metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 2 + cortex_compactor_remaining_planned_compactions 3 `, }, "test should skip block with no compact marker": { @@ -359,19 +342,29 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { Name: "cortex_compactor_block_visit_marker_write_failed", Help: "Number of block visit marker file failed to be written.", }) + partitionedGroupInfoReadFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_partitioned_group_info_read_failed", + Help: "Number of partitioned group info file failed to be read.", + }) + partitionedGroupInfoWriteFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_partitioned_group_info_write_failed", + Help: "Number of partitioned group info file failed to be written.", + }) bkt := &bucket.ClientMock{} - blockVisitMarkerTimeout := 5 * time.Minute - for _, visitedBlock := range testData.visitedBlocks { - visitMarkerFile := path.Join(visitedBlock.id.String(), BlockVisitMarkerFile) + visitMarkerTimeout := 5 * time.Minute + for _, visitedPartition := range testData.visitedPartitions { + visitMarkerFile := GetPartitionVisitMarkerFile(visitedPartition.partitionedGroupID, 0) expireTime := time.Now() - if visitedBlock.isExpired { - expireTime = expireTime.Add(-1 * blockVisitMarkerTimeout) + if visitedPartition.isExpired { + expireTime = expireTime.Add(-1 * visitMarkerTimeout) } - blockVisitMarker := BlockVisitMarker{ - CompactorID: visitedBlock.compactorID, - VisitTime: expireTime.Unix(), - Version: VisitMarkerVersion1, + blockVisitMarker := PartitionVisitMarker{ + CompactorID: visitedPartition.compactorID, + PartitionedGroupID: visitedPartition.partitionedGroupID, + PartitionID: 0, + VisitTime: expireTime.Unix(), + Version: VisitMarkerVersion1, } visitMarkerFileContent, _ := json.Marshal(blockVisitMarker) bkt.MockGet(visitMarkerFile, string(visitMarkerFileContent), nil) @@ -406,9 +399,11 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { 10, 3, testData.concurrency, - blockVisitMarkerTimeout, + visitMarkerTimeout, blockVisitMarkerReadFailed, blockVisitMarkerWriteFailed, + partitionedGroupInfoReadFailed, + partitionedGroupInfoWriteFailed, noCompactFilter, ) actual, err := g.Groups(testData.blocks) @@ -426,6 +421,52 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { } func TestGroupBlocksByCompactableRanges(t *testing.T) { + block1Ulid := ulid.MustNew(1, nil) + block2Ulid := ulid.MustNew(2, nil) + block3Ulid := ulid.MustNew(3, nil) + block4Ulid := ulid.MustNew(4, nil) + block5Ulid := ulid.MustNew(5, nil) + block6Ulid := ulid.MustNew(6, nil) + block7Ulid := ulid.MustNew(7, nil) + block8Ulid := ulid.MustNew(8, nil) + block9Ulid := ulid.MustNew(9, nil) + + zeroGroupIDPartitionInfo := &PartitionInfo{ + PartitionedGroupID: 0, + PartitionCount: 1, + PartitionID: 0, + } + + partition3ID0 := &PartitionInfo{ + PartitionedGroupID: uint32(12345), + PartitionCount: 3, + PartitionID: 0, + } + + partition3ID1 := &PartitionInfo{ + PartitionedGroupID: uint32(12345), + PartitionCount: 3, + PartitionID: 1, + } + + partition3ID2 := &PartitionInfo{ + PartitionedGroupID: uint32(12345), + PartitionCount: 3, + PartitionID: 2, + } + + partition2ID0 := &PartitionInfo{ + PartitionedGroupID: uint32(54321), + PartitionCount: 2, + PartitionID: 0, + } + + partition2ID1 := &PartitionInfo{ + PartitionedGroupID: uint32(54321), + PartitionCount: 2, + PartitionID: 1, + } + tests := map[string]struct { ranges []int64 blocks []*metadata.Meta @@ -439,167 +480,244 @@ func TestGroupBlocksByCompactableRanges(t *testing.T) { "only 1 block in input": { ranges: []int64{20}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 10, MaxTime: 20}}, }, expected: nil, }, "only 1 block for each range (single range)": { ranges: []int64{20}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 40, MaxTime: 60}}, }, expected: nil, }, "only 1 block for each range (multiple ranges)": { ranges: []int64{10, 20}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 40, MaxTime: 60}}, }, expected: nil, }, "input blocks can be compacted on the 1st range only": { ranges: []int64{20, 40}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 25, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 50}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 50, MaxTime: 60}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 60, MaxTime: 70}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 25, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 30, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 40, MaxTime: 50}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6Ulid, MinTime: 50, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{ULID: block7Ulid, MinTime: 60, MaxTime: 70}}, }, expected: []blocksGroup{ {rangeStart: 20, rangeEnd: 40, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 25, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 25, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 30, MaxTime: 40}}, }}, {rangeStart: 40, rangeEnd: 60, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 50}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 50, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 40, MaxTime: 50}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6Ulid, MinTime: 50, MaxTime: 60}}, }}, }, }, "input blocks can be compacted on the 2nd range only": { ranges: []int64{10, 20}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 60, MaxTime: 70}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 70, MaxTime: 80}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 30, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 40, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 60, MaxTime: 70}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6Ulid, MinTime: 70, MaxTime: 80}}, }, expected: []blocksGroup{ {rangeStart: 20, rangeEnd: 40, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 30, MaxTime: 40}}, }}, {rangeStart: 60, rangeEnd: 80, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 60, MaxTime: 70}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 70, MaxTime: 80}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 60, MaxTime: 70}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6Ulid, MinTime: 70, MaxTime: 80}}, }}, }, }, "input blocks can be compacted on a mix of 1st and 2nd ranges, guaranteeing no overlaps and giving preference to smaller ranges": { ranges: []int64{10, 20}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 7, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 60, MaxTime: 70}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 70, MaxTime: 80}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 75, MaxTime: 80}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 7, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 30, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6Ulid, MinTime: 40, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{ULID: block7Ulid, MinTime: 60, MaxTime: 70}}, + {BlockMeta: tsdb.BlockMeta{ULID: block8Ulid, MinTime: 70, MaxTime: 80}}, + {BlockMeta: tsdb.BlockMeta{ULID: block9Ulid, MinTime: 75, MaxTime: 80}}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 10, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 7, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 7, MaxTime: 10}}, }}, {rangeStart: 70, rangeEnd: 80, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 70, MaxTime: 80}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 75, MaxTime: 80}}, + {BlockMeta: tsdb.BlockMeta{ULID: block8Ulid, MinTime: 70, MaxTime: 80}}, + {BlockMeta: tsdb.BlockMeta{ULID: block9Ulid, MinTime: 75, MaxTime: 80}}, }}, {rangeStart: 20, rangeEnd: 40, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 30, MaxTime: 40}}, }}, }, }, "input blocks have already been compacted with the largest range": { ranges: []int64{10, 20, 40}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 70}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 40, MaxTime: 70}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 80, MaxTime: 120}}, }, expected: nil, }, "input blocks match the largest range but can be compacted because overlapping": { ranges: []int64{10, 20, 40}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 70}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 40, MaxTime: 70}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 80, MaxTime: 120}}, }, expected: []blocksGroup{ {rangeStart: 80, rangeEnd: 120, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 80, MaxTime: 120}}, }}, }, }, "a block with time range crossing two 1st level ranges should be NOT considered for 1st level compaction": { ranges: []int64{20, 40}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 30}}, // This block spans across two 1st level ranges. - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 10, MaxTime: 30}}, // This block spans across two 1st level ranges. + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 30, MaxTime: 40}}, }, expected: []blocksGroup{ {rangeStart: 20, rangeEnd: 40, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 30, MaxTime: 40}}, }}, }, }, "a block with time range crossing two 1st level ranges should BE considered for 2nd level compaction": { ranges: []int64{20, 40}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 30}}, // This block spans across two 1st level ranges. - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 10, MaxTime: 30}}, // This block spans across two 1st level ranges. + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 20, MaxTime: 40}}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 40, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 10, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 20, MaxTime: 40}}, }}, }, }, "a block with time range larger then the largest compaction range should NOT be considered for compaction": { ranges: []int64{10, 20, 40}, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 30, MaxTime: 150}}, // This block is larger then the largest compaction range. - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 70}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 30, MaxTime: 150}}, // This block is larger then the largest compaction range. + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 40, MaxTime: 70}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 80, MaxTime: 120}}, }, expected: []blocksGroup{ {rangeStart: 80, rangeEnd: 120, blocks: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 80, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 80, MaxTime: 120}}, + }}, + }, + }, + "a group with all blocks having same partitioned group id should be ignored": { + ranges: []int64{10, 20, 40}, + blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID0}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID1}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 10, MaxTime: 20}}, + }, + expected: []blocksGroup{ + {rangeStart: 10, rangeEnd: 20, blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 10, MaxTime: 20}}, + }}, + }, + }, + "a group with all blocks having partitioned group id is 0 should not be ignored": { + ranges: []int64{10, 20, 40}, + blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 0, MaxTime: 10}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 0, MaxTime: 10}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 10, MaxTime: 20}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 10, MaxTime: 20}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + }, + expected: []blocksGroup{ + {rangeStart: 0, rangeEnd: 10, blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 0, MaxTime: 10}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 0, MaxTime: 10}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + }}, + {rangeStart: 10, rangeEnd: 20, blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 10, MaxTime: 20}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 10, MaxTime: 20}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: zeroGroupIDPartitionInfo}}}, + }}, + }, + }, + "a group with blocks from two different partitioned groups": { + ranges: []int64{10, 20, 40}, + blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID0}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID1}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition2ID0}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition2ID1}}}, + }, + expected: []blocksGroup{ + {rangeStart: 0, rangeEnd: 20, blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID0}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID1}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition2ID0}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition2ID1}}}, + }}, + }, + }, + "a group with some blocks not having partition info": { + ranges: []int64{10, 20, 40}, + blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition2ID0}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition2ID1}}}, + }, + expected: []blocksGroup{ + {rangeStart: 0, rangeEnd: 20, blocks: []*metadata.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3Ulid, MinTime: 0, MaxTime: 10, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition3ID2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition2ID0}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6Ulid, MinTime: 10, MaxTime: 20, Compaction: tsdb.BlockMetaCompaction{Level: 2}}, Thanos: metadata.Thanos{Extensions: &CortexMetaExtensions{PartitionInfo: partition2ID1}}}, }}, }, }, @@ -743,6 +861,885 @@ func TestBlocksGroup_overlaps(t *testing.T) { } } +func TestGroupPartitioning(t *testing.T) { + t0block1Ulid := ulid.MustNew(1, nil) + t0block2Ulid := ulid.MustNew(2, nil) + t0block3Ulid := ulid.MustNew(3, nil) + t1block1Ulid := ulid.MustNew(4, nil) + t1block2Ulid := ulid.MustNew(5, nil) + t2block1Ulid := ulid.MustNew(6, nil) + t2block2Ulid := ulid.MustNew(7, nil) + t2block3Ulid := ulid.MustNew(8, nil) + t2block4Ulid := ulid.MustNew(9, nil) + t3block1Ulid := ulid.MustNew(10, nil) + t3block2Ulid := ulid.MustNew(11, nil) + t3block3Ulid := ulid.MustNew(12, nil) + t3block4Ulid := ulid.MustNew(13, nil) + t3block5Ulid := ulid.MustNew(14, nil) + t3block6Ulid := ulid.MustNew(15, nil) + t3block7Ulid := ulid.MustNew(16, nil) + t3block8Ulid := ulid.MustNew(17, nil) + t4block1Ulid := ulid.MustNew(18, nil) + t5block1Ulid := ulid.MustNew(19, nil) + + blocks := + map[ulid.ULID]*metadata.Meta{ + t0block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t0block1Ulid, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 3 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t0block2Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t0block2Ulid, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 3 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t0block3Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t0block3Ulid, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 3 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t1block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t1block1Ulid, MinTime: 3 * time.Hour.Milliseconds(), MaxTime: 5 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t1block2Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t1block2Ulid, MinTime: 3 * time.Hour.Milliseconds(), MaxTime: 5 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t2block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t2block1Ulid, MinTime: 5 * time.Hour.Milliseconds(), MaxTime: 7 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t2block2Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t2block2Ulid, MinTime: 5 * time.Hour.Milliseconds(), MaxTime: 7 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t2block3Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t2block3Ulid, MinTime: 5 * time.Hour.Milliseconds(), MaxTime: 7 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t2block4Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t2block4Ulid, MinTime: 5 * time.Hour.Milliseconds(), MaxTime: 7 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block1Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block2Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block2Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block3Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block3Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block4Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block4Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block5Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block5Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block6Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block6Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block7Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block7Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block8Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block8Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t4block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t4block1Ulid, MinTime: 9 * time.Hour.Milliseconds(), MaxTime: 15 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t5block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t5block1Ulid, MinTime: 15 * time.Hour.Milliseconds(), MaxTime: 21 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + } + + testCompactorID := "test-compactor" + + tests := map[string]struct { + ranges []time.Duration + rangeStart int64 + rangeEnd int64 + indexSize int64 + indexLimit int64 + seriesCount int64 + seriesLimit int64 + blocks map[*metadata.Meta]*PartitionInfo + expected struct { + partitionCount int + partitions map[int][]ulid.ULID + } + }{ + "test blocks generated by partition": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + indexSize: int64(14), + indexLimit: int64(64), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t1block1Ulid]: {PartitionID: 0, PartitionCount: 2}, blocks[t1block2Ulid]: {PartitionID: 1, PartitionCount: 2}, + blocks[t2block1Ulid]: {PartitionID: 0, PartitionCount: 4}, blocks[t2block2Ulid]: {PartitionID: 1, PartitionCount: 4}, + blocks[t2block3Ulid]: {PartitionID: 2, PartitionCount: 4}, blocks[t2block4Ulid]: {PartitionID: 3, PartitionCount: 4}, + blocks[t3block1Ulid]: {PartitionID: 0, PartitionCount: 8}, blocks[t3block2Ulid]: {PartitionID: 1, PartitionCount: 8}, + blocks[t3block3Ulid]: {PartitionID: 2, PartitionCount: 8}, blocks[t3block4Ulid]: {PartitionID: 3, PartitionCount: 8}, + blocks[t3block5Ulid]: {PartitionID: 4, PartitionCount: 8}, blocks[t3block6Ulid]: {PartitionID: 5, PartitionCount: 8}, + blocks[t3block7Ulid]: {PartitionID: 6, PartitionCount: 8}, blocks[t3block8Ulid]: {PartitionID: 7, PartitionCount: 8}}, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 4, + partitions: map[int][]ulid.ULID{ + 0: {t1block1Ulid, t2block1Ulid, t3block1Ulid, t3block5Ulid}, + 1: {t1block2Ulid, t2block2Ulid, t3block2Ulid, t3block6Ulid}, + 2: {t1block1Ulid, t2block3Ulid, t3block3Ulid, t3block7Ulid}, + 3: {t1block2Ulid, t2block4Ulid, t3block4Ulid, t3block8Ulid}, + }, + }, + }, + "test all level 1 blocks": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + indexSize: int64(30), + indexLimit: int64(64), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t0block1Ulid]: nil, blocks[t0block2Ulid]: nil, blocks[t0block3Ulid]: nil, + }, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 2, + partitions: map[int][]ulid.ULID{ + 0: {t0block1Ulid, t0block2Ulid, t0block3Ulid}, + 1: {t0block1Ulid, t0block2Ulid, t0block3Ulid}, + }, + }, + }, + "test high level blocks generated without partitioning": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + indexSize: int64(50), + indexLimit: int64(64), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t4block1Ulid]: nil, blocks[t5block1Ulid]: nil, + }, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 2, + partitions: map[int][]ulid.ULID{ + 0: {t4block1Ulid, t5block1Ulid}, + 1: {t4block1Ulid, t5block1Ulid}, + }, + }, + }, + "test blocks generated by partition with series limit": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + seriesCount: int64(14), + seriesLimit: int64(64), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t1block1Ulid]: {PartitionID: 0, PartitionCount: 2}, blocks[t1block2Ulid]: {PartitionID: 1, PartitionCount: 2}, + blocks[t2block1Ulid]: {PartitionID: 0, PartitionCount: 4}, blocks[t2block2Ulid]: {PartitionID: 1, PartitionCount: 4}, + blocks[t2block3Ulid]: {PartitionID: 2, PartitionCount: 4}, blocks[t2block4Ulid]: {PartitionID: 3, PartitionCount: 4}, + blocks[t3block1Ulid]: {PartitionID: 0, PartitionCount: 8}, blocks[t3block2Ulid]: {PartitionID: 1, PartitionCount: 8}, + blocks[t3block3Ulid]: {PartitionID: 2, PartitionCount: 8}, blocks[t3block4Ulid]: {PartitionID: 3, PartitionCount: 8}, + blocks[t3block5Ulid]: {PartitionID: 4, PartitionCount: 8}, blocks[t3block6Ulid]: {PartitionID: 5, PartitionCount: 8}, + blocks[t3block7Ulid]: {PartitionID: 6, PartitionCount: 8}, blocks[t3block8Ulid]: {PartitionID: 7, PartitionCount: 8}}, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 4, + partitions: map[int][]ulid.ULID{ + 0: {t1block1Ulid, t2block1Ulid, t3block1Ulid, t3block5Ulid}, + 1: {t1block2Ulid, t2block2Ulid, t3block2Ulid, t3block6Ulid}, + 2: {t1block1Ulid, t2block3Ulid, t3block3Ulid, t3block7Ulid}, + 3: {t1block2Ulid, t2block4Ulid, t3block4Ulid, t3block8Ulid}, + }, + }, + }, + "test blocks generated by partition with both index and series limit set": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + indexSize: int64(1), + indexLimit: int64(64), + seriesCount: int64(14), + seriesLimit: int64(64), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t1block1Ulid]: {PartitionID: 0, PartitionCount: 2}, blocks[t1block2Ulid]: {PartitionID: 1, PartitionCount: 2}, + blocks[t2block1Ulid]: {PartitionID: 0, PartitionCount: 4}, blocks[t2block2Ulid]: {PartitionID: 1, PartitionCount: 4}, + blocks[t2block3Ulid]: {PartitionID: 2, PartitionCount: 4}, blocks[t2block4Ulid]: {PartitionID: 3, PartitionCount: 4}, + blocks[t3block1Ulid]: {PartitionID: 0, PartitionCount: 8}, blocks[t3block2Ulid]: {PartitionID: 1, PartitionCount: 8}, + blocks[t3block3Ulid]: {PartitionID: 2, PartitionCount: 8}, blocks[t3block4Ulid]: {PartitionID: 3, PartitionCount: 8}, + blocks[t3block5Ulid]: {PartitionID: 4, PartitionCount: 8}, blocks[t3block6Ulid]: {PartitionID: 5, PartitionCount: 8}, + blocks[t3block7Ulid]: {PartitionID: 6, PartitionCount: 8}, blocks[t3block8Ulid]: {PartitionID: 7, PartitionCount: 8}}, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 4, + partitions: map[int][]ulid.ULID{ + 0: {t1block1Ulid, t2block1Ulid, t3block1Ulid, t3block5Ulid}, + 1: {t1block2Ulid, t2block2Ulid, t3block2Ulid, t3block6Ulid}, + 2: {t1block1Ulid, t2block3Ulid, t3block3Ulid, t3block7Ulid}, + 3: {t1block2Ulid, t2block4Ulid, t3block4Ulid, t3block8Ulid}, + }, + }, + }, + "test blocks generated by partition with partition count equals to 1": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t1block1Ulid]: {PartitionID: 0, PartitionCount: 2}, blocks[t1block2Ulid]: {PartitionID: 1, PartitionCount: 2}, + blocks[t2block1Ulid]: {PartitionID: 0, PartitionCount: 4}, blocks[t2block2Ulid]: {PartitionID: 1, PartitionCount: 4}, + blocks[t2block3Ulid]: {PartitionID: 2, PartitionCount: 4}, blocks[t2block4Ulid]: {PartitionID: 3, PartitionCount: 4}, + blocks[t3block1Ulid]: {PartitionID: 0, PartitionCount: 8}, blocks[t3block2Ulid]: {PartitionID: 1, PartitionCount: 8}, + blocks[t3block3Ulid]: {PartitionID: 2, PartitionCount: 8}, blocks[t3block4Ulid]: {PartitionID: 3, PartitionCount: 8}, + blocks[t3block5Ulid]: {PartitionID: 4, PartitionCount: 8}, blocks[t3block6Ulid]: {PartitionID: 5, PartitionCount: 8}, + blocks[t3block7Ulid]: {PartitionID: 6, PartitionCount: 8}, blocks[t3block8Ulid]: {PartitionID: 7, PartitionCount: 8}}, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 1, + partitions: map[int][]ulid.ULID{ + 0: {t1block1Ulid, t1block2Ulid, t2block1Ulid, t2block2Ulid, t2block3Ulid, t2block4Ulid, t3block1Ulid, + t3block2Ulid, t3block3Ulid, t3block4Ulid, t3block5Ulid, t3block6Ulid, t3block7Ulid, t3block8Ulid}, + }, + }, + }, + "test blocks generated by partition in random order": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + indexSize: int64(14), + indexLimit: int64(64), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t1block2Ulid]: {PartitionID: 1, PartitionCount: 2}, blocks[t1block1Ulid]: {PartitionID: 0, PartitionCount: 2}, + blocks[t2block4Ulid]: {PartitionID: 3, PartitionCount: 4}, blocks[t2block1Ulid]: {PartitionID: 0, PartitionCount: 4}, + blocks[t2block3Ulid]: {PartitionID: 2, PartitionCount: 4}, blocks[t2block2Ulid]: {PartitionID: 1, PartitionCount: 4}, + blocks[t3block1Ulid]: {PartitionID: 0, PartitionCount: 8}, blocks[t3block6Ulid]: {PartitionID: 5, PartitionCount: 8}, + blocks[t3block3Ulid]: {PartitionID: 2, PartitionCount: 8}, blocks[t3block5Ulid]: {PartitionID: 4, PartitionCount: 8}, + blocks[t3block2Ulid]: {PartitionID: 1, PartitionCount: 8}, blocks[t3block7Ulid]: {PartitionID: 6, PartitionCount: 8}, + blocks[t3block4Ulid]: {PartitionID: 3, PartitionCount: 8}, blocks[t3block8Ulid]: {PartitionID: 7, PartitionCount: 8}}, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 4, + partitions: map[int][]ulid.ULID{ + 0: {t1block1Ulid, t2block1Ulid, t3block1Ulid, t3block5Ulid}, + 1: {t1block2Ulid, t2block2Ulid, t3block2Ulid, t3block6Ulid}, + 2: {t1block1Ulid, t2block3Ulid, t3block3Ulid, t3block7Ulid}, + 3: {t1block2Ulid, t2block4Ulid, t3block4Ulid, t3block8Ulid}, + }, + }, + }, + "test part of blocks generated by incompatible partitioning compactor and the rest generated by compatible compactor": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + indexSize: int64(14), + indexLimit: int64(64), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t1block1Ulid]: nil, blocks[t1block2Ulid]: nil, + blocks[t2block1Ulid]: {PartitionID: 0, PartitionCount: 4}, blocks[t2block2Ulid]: {PartitionID: 1, PartitionCount: 4}, + blocks[t2block3Ulid]: nil, blocks[t2block4Ulid]: {PartitionID: 3, PartitionCount: 4}, + blocks[t3block1Ulid]: {PartitionID: 0, PartitionCount: 8}, blocks[t3block2Ulid]: nil, + blocks[t3block3Ulid]: {PartitionID: 2, PartitionCount: 8}, blocks[t3block4Ulid]: nil, + blocks[t3block5Ulid]: {PartitionID: 4, PartitionCount: 8}, blocks[t3block6Ulid]: nil, + blocks[t3block7Ulid]: {PartitionID: 6, PartitionCount: 8}, blocks[t3block8Ulid]: nil}, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 4, + partitions: map[int][]ulid.ULID{ + 0: {t1block1Ulid, t1block2Ulid, t2block1Ulid, t2block3Ulid, t3block1Ulid, t3block2Ulid, t3block4Ulid, t3block5Ulid, t3block6Ulid, t3block8Ulid}, + 1: {t1block1Ulid, t1block2Ulid, t2block2Ulid, t2block3Ulid, t3block2Ulid, t3block4Ulid, t3block6Ulid, t3block8Ulid}, + 2: {t1block1Ulid, t1block2Ulid, t2block3Ulid, t3block2Ulid, t3block3Ulid, t3block4Ulid, t3block6Ulid, t3block7Ulid, t3block8Ulid}, + 3: {t1block1Ulid, t1block2Ulid, t2block3Ulid, t2block4Ulid, t3block2Ulid, t3block4Ulid, t3block6Ulid, t3block8Ulid}, + }, + }, + }, + "test part of blocks have partially invalid partition info": { + ranges: []time.Duration{2 * time.Hour, 6 * time.Hour}, + rangeStart: 1 * time.Hour.Milliseconds(), + rangeEnd: 9 * time.Hour.Milliseconds(), + indexSize: int64(14), + indexLimit: int64(64), + blocks: map[*metadata.Meta]*PartitionInfo{ + blocks[t1block1Ulid]: {}, blocks[t1block2Ulid]: nil, + blocks[t2block1Ulid]: {PartitionID: 0, PartitionCount: 4}, blocks[t2block2Ulid]: {PartitionID: 1, PartitionCount: 4}, + blocks[t2block3Ulid]: nil, blocks[t2block4Ulid]: {PartitionID: 3, PartitionCount: 4}, + blocks[t3block1Ulid]: {PartitionID: 0, PartitionCount: 8}, blocks[t3block2Ulid]: nil, + blocks[t3block3Ulid]: {PartitionID: 2, PartitionCount: 8}, blocks[t3block4Ulid]: {}, + blocks[t3block5Ulid]: {PartitionID: 4, PartitionCount: 8}, blocks[t3block6Ulid]: {PartitionID: 0, PartitionCount: 0}, + blocks[t3block7Ulid]: {PartitionID: 6, PartitionCount: 8}, blocks[t3block8Ulid]: nil}, + expected: struct { + partitionCount int + partitions map[int][]ulid.ULID + }{ + partitionCount: 4, + partitions: map[int][]ulid.ULID{ + 0: {t1block1Ulid, t1block2Ulid, t2block1Ulid, t2block3Ulid, t3block1Ulid, t3block2Ulid, t3block4Ulid, t3block5Ulid, t3block6Ulid, t3block8Ulid}, + 1: {t1block1Ulid, t1block2Ulid, t2block2Ulid, t2block3Ulid, t3block2Ulid, t3block4Ulid, t3block6Ulid, t3block8Ulid}, + 2: {t1block1Ulid, t1block2Ulid, t2block3Ulid, t3block2Ulid, t3block3Ulid, t3block4Ulid, t3block6Ulid, t3block7Ulid, t3block8Ulid}, + 3: {t1block1Ulid, t1block2Ulid, t2block3Ulid, t2block4Ulid, t3block2Ulid, t3block4Ulid, t3block6Ulid, t3block8Ulid}, + }, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + compactorCfg := &Config{ + BlockRanges: testData.ranges, + } + + limits := &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: testData.indexLimit, + CompactorPartitionSeriesCountLimit: testData.seriesLimit, + } + + overrides, err := validation.NewOverrides(*limits, nil) + require.NoError(t, err) + + ring := &RingMock{} + + bkt := &bucket.ClientMock{} + + noCompactFilter := func() map[ulid.ULID]*metadata.NoCompactMark { + return make(map[ulid.ULID]*metadata.NoCompactMark) + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + g := NewShuffleShardingGrouper( + ctx, + nil, + objstore.WithNoopInstr(bkt), + false, // Do not accept malformed indexes + true, // Enable vertical compaction + nil, + nil, + nil, + nil, + nil, + metadata.NoneFunc, + *compactorCfg, + ring, + "test-addr", + testCompactorID, + overrides, + "", + 10, + 3, + 1, + 5*time.Minute, + nil, + nil, + nil, + nil, + noCompactFilter, + ) + var testBlocks []*metadata.Meta + for block, partitionInfo := range testData.blocks { + block.Thanos.Files = []metadata.File{ + {RelPath: thanosblock.IndexFilename, SizeBytes: testData.indexSize}, + } + block.Stats.NumSeries = uint64(testData.seriesCount) + testBlocks = append(testBlocks, block) + if partitionInfo == nil { + block.Thanos.Extensions = nil + } else { + block.Thanos.Extensions = &CortexMetaExtensions{PartitionInfo: partitionInfo} + } + } + testGroup := blocksGroup{ + rangeStart: testData.rangeStart, + rangeEnd: testData.rangeEnd, + blocks: testBlocks, + } + actual, err := g.partitionBlockGroup(testGroup, uint32(0)) + require.NoError(t, err) + require.Equal(t, testData.expected.partitionCount, actual.PartitionCount) + require.Len(t, actual.Partitions, len(testData.expected.partitions)) + for _, actualPartition := range actual.Partitions { + actualPartitionID := actualPartition.PartitionID + require.ElementsMatch(t, testData.expected.partitions[actualPartitionID], actualPartition.Blocks) + } + }) + } +} + +func TestPartitionStrategyChange_shouldUseOriginalPartitionedGroup(t *testing.T) { + t1block1Ulid := ulid.MustNew(4, nil) + t1block2Ulid := ulid.MustNew(5, nil) + t2block1Ulid := ulid.MustNew(6, nil) + t2block2Ulid := ulid.MustNew(7, nil) + t2block3Ulid := ulid.MustNew(8, nil) + t2block4Ulid := ulid.MustNew(9, nil) + t3block1Ulid := ulid.MustNew(10, nil) + t3block2Ulid := ulid.MustNew(11, nil) + t3block3Ulid := ulid.MustNew(12, nil) + t3block4Ulid := ulid.MustNew(13, nil) + t3block5Ulid := ulid.MustNew(14, nil) + t3block6Ulid := ulid.MustNew(15, nil) + t3block7Ulid := ulid.MustNew(16, nil) + t3block8Ulid := ulid.MustNew(17, nil) + + blocks := + map[ulid.ULID]*metadata.Meta{ + t1block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t1block1Ulid, MinTime: 3 * time.Hour.Milliseconds(), MaxTime: 5 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t1block2Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t1block2Ulid, MinTime: 3 * time.Hour.Milliseconds(), MaxTime: 5 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t2block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t2block1Ulid, MinTime: 5 * time.Hour.Milliseconds(), MaxTime: 7 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t2block2Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t2block2Ulid, MinTime: 5 * time.Hour.Milliseconds(), MaxTime: 7 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t2block3Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t2block3Ulid, MinTime: 5 * time.Hour.Milliseconds(), MaxTime: 7 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t2block4Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t2block4Ulid, MinTime: 5 * time.Hour.Milliseconds(), MaxTime: 7 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block1Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block2Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block2Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block3Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block3Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block4Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block4Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block5Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block5Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block6Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block6Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block7Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block7Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + t3block8Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t3block8Ulid, MinTime: 7 * time.Hour.Milliseconds(), MaxTime: 9 * time.Hour.Milliseconds()}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}}, + }, + } + + partitionedGroupID := uint32(12345) + indexSize := int64(10) + seriesCount := int64(10) + testRanges := []time.Duration{2 * time.Hour, 6 * time.Hour} + testRangeStart := 1 * time.Hour.Milliseconds() + testRangeEnd := 9 * time.Hour.Milliseconds() + testBlocks := map[*metadata.Meta]*PartitionInfo{ + blocks[t1block1Ulid]: {PartitionID: 0, PartitionCount: 2}, blocks[t1block2Ulid]: {PartitionID: 1, PartitionCount: 2}, + blocks[t2block1Ulid]: {PartitionID: 0, PartitionCount: 4}, blocks[t2block2Ulid]: {PartitionID: 1, PartitionCount: 4}, + blocks[t2block3Ulid]: {PartitionID: 2, PartitionCount: 4}, blocks[t2block4Ulid]: {PartitionID: 3, PartitionCount: 4}, + blocks[t3block1Ulid]: {PartitionID: 0, PartitionCount: 8}, blocks[t3block2Ulid]: {PartitionID: 1, PartitionCount: 8}, + blocks[t3block3Ulid]: {PartitionID: 2, PartitionCount: 8}, blocks[t3block4Ulid]: {PartitionID: 3, PartitionCount: 8}, + blocks[t3block5Ulid]: {PartitionID: 4, PartitionCount: 8}, blocks[t3block6Ulid]: {PartitionID: 5, PartitionCount: 8}, + blocks[t3block7Ulid]: {PartitionID: 6, PartitionCount: 8}, blocks[t3block8Ulid]: {PartitionID: 7, PartitionCount: 8}, + } + bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + var updatedTestBlocks []*metadata.Meta + for block, partitionInfo := range testBlocks { + block.Thanos.Files = []metadata.File{ + {RelPath: thanosblock.IndexFilename, SizeBytes: indexSize}, + } + block.Stats.NumSeries = uint64(seriesCount) + updatedTestBlocks = append(updatedTestBlocks, block) + block.Thanos.Extensions = CortexMetaExtensions{PartitionInfo: partitionInfo} + } + testGroup := blocksGroup{ + rangeStart: testRangeStart, + rangeEnd: testRangeEnd, + blocks: updatedTestBlocks, + } + createGrouper := func(ctx context.Context, bkt objstore.Bucket, compactorCfg *Config, overrides *validation.Overrides) *ShuffleShardingGrouper { + r := &RingMock{} + + noCompactFilter := func() map[ulid.ULID]*metadata.NoCompactMark { + return make(map[ulid.ULID]*metadata.NoCompactMark) + } + + return NewShuffleShardingGrouper( + ctx, + nil, + objstore.WithNoopInstr(bkt), + false, // Do not accept malformed indexes + true, // Enable vertical compaction + nil, + nil, + nil, + nil, + nil, + metadata.NoneFunc, + *compactorCfg, + r, + "test-addr", + "test-compactor", + overrides, + "", + 10, + 3, + 1, + 5*time.Minute, + nil, + nil, + nil, + nil, + noCompactFilter, + ) + } + + expectedPartitions := map[int][]ulid.ULID{ + 0: {t1block1Ulid, t2block1Ulid, t3block1Ulid, t3block5Ulid}, + 1: {t1block2Ulid, t2block2Ulid, t3block2Ulid, t3block6Ulid}, + 2: {t1block1Ulid, t2block3Ulid, t3block3Ulid, t3block7Ulid}, + 3: {t1block2Ulid, t2block4Ulid, t3block4Ulid, t3block8Ulid}, + } + + // test base case + compactorCfg1 := &Config{ + BlockRanges: testRanges, + } + limits1 := &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: int64(40), + } + + overrides1, err := validation.NewOverrides(*limits1, nil) + require.NoError(t, err) + ctx, cancel := context.WithCancel(context.Background()) + grouper1 := createGrouper(ctx, bkt, compactorCfg1, overrides1) + partitionedGroup1, err := grouper1.generatePartitionBlockGroup(testGroup, partitionedGroupID) + cancel() + require.NoError(t, err) + require.Equal(t, 4, partitionedGroup1.PartitionCount) + require.Len(t, partitionedGroup1.Partitions, 4) + partitionMap := make(map[int][]ulid.ULID) + for _, partition := range partitionedGroup1.Partitions { + partitionID := partition.PartitionID + require.ElementsMatch(t, expectedPartitions[partitionID], partition.Blocks) + partitionMap[partitionID] = partition.Blocks + } + + // test limit increased + compactorCfg2 := &Config{ + BlockRanges: testRanges, + } + limits2 := &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: int64(80), + } + + overrides2, err := validation.NewOverrides(*limits2, nil) + require.NoError(t, err) + ctx, cancel = context.WithCancel(context.Background()) + grouper2 := createGrouper(ctx, bkt, compactorCfg2, overrides2) + partitionedGroup2, err := grouper2.generatePartitionBlockGroup(testGroup, partitionedGroupID) + cancel() + require.NoError(t, err) + require.Equal(t, partitionedGroup1.PartitionCount, partitionedGroup2.PartitionCount) + require.Len(t, partitionedGroup2.Partitions, len(partitionedGroup1.Partitions)) + for _, partition := range partitionedGroup2.Partitions { + partitionID := partition.PartitionID + require.ElementsMatch(t, partitionMap[partitionID], partition.Blocks) + } + + // test limit decreased + compactorCfg3 := &Config{ + BlockRanges: testRanges, + } + limits3 := &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: int64(20), + } + + overrides3, err := validation.NewOverrides(*limits3, nil) + require.NoError(t, err) + ctx, cancel = context.WithCancel(context.Background()) + grouper3 := createGrouper(ctx, bkt, compactorCfg3, overrides3) + partitionedGroup3, err := grouper3.generatePartitionBlockGroup(testGroup, partitionedGroupID) + cancel() + require.NoError(t, err) + require.Equal(t, partitionedGroup1.PartitionCount, partitionedGroup3.PartitionCount) + require.Len(t, partitionedGroup3.Partitions, len(partitionedGroup1.Partitions)) + for _, partition := range partitionedGroup3.Partitions { + partitionID := partition.PartitionID + require.ElementsMatch(t, partitionMap[partitionID], partition.Blocks) + } +} + +func TestGroupCalculatePartitionCount(t *testing.T) { + t0block1Ulid := ulid.MustNew(1, nil) + t0block2Ulid := ulid.MustNew(2, nil) + t0block3Ulid := ulid.MustNew(3, nil) + t0block4Ulid := ulid.MustNew(4, nil) + + blocks := + map[ulid.ULID]*metadata.Meta{ + t0block1Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t0block1Ulid, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 3 * time.Hour.Milliseconds(), Stats: tsdb.BlockStats{ + NumSeries: uint64(6), + }}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}, Files: []metadata.File{ + {RelPath: thanosblock.IndexFilename, SizeBytes: int64(14)}, + }}, + }, + t0block2Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t0block2Ulid, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 3 * time.Hour.Milliseconds(), Stats: tsdb.BlockStats{ + NumSeries: uint64(6), + }}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}, Files: []metadata.File{ + {RelPath: thanosblock.IndexFilename, SizeBytes: int64(14)}, + }}, + }, + t0block3Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t0block3Ulid, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 3 * time.Hour.Milliseconds(), Stats: tsdb.BlockStats{ + NumSeries: uint64(6), + }}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}, Files: []metadata.File{ + {RelPath: thanosblock.IndexFilename, SizeBytes: int64(14)}, + }}, + }, + t0block4Ulid: { + BlockMeta: tsdb.BlockMeta{ULID: t0block4Ulid, MinTime: 1 * time.Hour.Milliseconds(), MaxTime: 3 * time.Hour.Milliseconds(), Stats: tsdb.BlockStats{ + NumSeries: uint64(6), + }}, + Thanos: metadata.Thanos{Labels: map[string]string{"external": "1"}, Files: []metadata.File{ + {RelPath: thanosblock.IndexFilename, SizeBytes: int64(14)}, + }}, + }, + } + + testCompactorID := "test-compactor" + userID := "test_workspace" + + tests := map[string]struct { + indexLimit int64 + seriesLimit int64 + userLimit *validation.Limits + blocks []*metadata.Meta + expectedPartitionCount int + }{ + "test global index limit": { + indexLimit: int64(20), + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test global series limit": { + seriesLimit: int64(10), + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test both global index limit and global series limit are defined and global index limit gives more partitions": { + indexLimit: int64(20), + seriesLimit: int64(20), + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test both global index limit and global series limit are defined and global index series gives more partitions": { + indexLimit: int64(40), + seriesLimit: int64(10), + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test user index limit": { + userLimit: &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: int64(20), + }, + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test user series limit": { + userLimit: &validation.Limits{ + CompactorPartitionSeriesCountLimit: int64(10), + }, + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test both user index limit and user series limit are defined and user index limit gives more partitions": { + userLimit: &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: int64(20), + CompactorPartitionSeriesCountLimit: int64(20), + }, + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test both user index limit and user series limit are defined and user index series gives more partitions": { + userLimit: &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: int64(40), + CompactorPartitionSeriesCountLimit: int64(10), + }, + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test both global index limit and user index limit are defined and user index limit is used": { + indexLimit: int64(1), + userLimit: &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: int64(20), + }, + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + "test both global series limit and user series limit are defined and user series limit is used": { + seriesLimit: int64(1), + userLimit: &validation.Limits{ + CompactorPartitionSeriesCountLimit: int64(10), + }, + blocks: []*metadata.Meta{ + blocks[t0block1Ulid], + blocks[t0block2Ulid], + blocks[t0block3Ulid], + blocks[t0block4Ulid]}, + expectedPartitionCount: 4, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + ranges := []time.Duration{2 * time.Hour} + rangeStart := 1 * time.Hour.Milliseconds() + rangeEnd := 3 * time.Hour.Milliseconds() + compactorCfg := &Config{ + BlockRanges: ranges, + } + + limits := &validation.Limits{ + CompactorPartitionIndexSizeLimitInBytes: testData.indexLimit, + CompactorPartitionSeriesCountLimit: testData.seriesLimit, + } + tenantLimit := &mockTenantLimits{ + limits: map[string]*validation.Limits{ + userID: testData.userLimit, + }, + } + overrides, err := validation.NewOverrides(*limits, tenantLimit) + require.NoError(t, err) + + ring := &RingMock{} + + bkt := &bucket.ClientMock{} + + noCompactFilter := func() map[ulid.ULID]*metadata.NoCompactMark { + return make(map[ulid.ULID]*metadata.NoCompactMark) + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + g := NewShuffleShardingGrouper( + ctx, + nil, + objstore.WithNoopInstr(bkt), + false, // Do not accept malformed indexes + true, // Enable vertical compaction + nil, + nil, + nil, + nil, + nil, + metadata.NoneFunc, + *compactorCfg, + ring, + "test-addr", + testCompactorID, + overrides, + userID, + 10, + 3, + 1, + 5*time.Minute, + nil, + nil, + nil, + nil, + noCompactFilter, + ) + testGroup := blocksGroup{ + rangeStart: rangeStart, + rangeEnd: rangeEnd, + blocks: testData.blocks, + } + actual := g.calculatePartitionCount(testGroup) + require.Equal(t, testData.expectedPartitionCount, actual) + }) + } +} + type RingMock struct { mock.Mock } @@ -804,3 +1801,15 @@ func (r *RingMock) HasInstance(instanceID string) bool { } func (r *RingMock) CleanupShuffleShardCache(identifier string) {} + +type mockTenantLimits struct { + limits map[string]*validation.Limits +} + +func (l *mockTenantLimits) ByUserID(userID string) *validation.Limits { + return l.limits[userID] +} + +func (l *mockTenantLimits) AllByUserID() map[string]*validation.Limits { + return l.limits +} diff --git a/pkg/compactor/shuffle_sharding_planner.go b/pkg/compactor/shuffle_sharding_planner.go index 5da27b0bec5..a8a2dd147e4 100644 --- a/pkg/compactor/shuffle_sharding_planner.go +++ b/pkg/compactor/shuffle_sharding_planner.go @@ -14,16 +14,16 @@ import ( ) type ShuffleShardingPlanner struct { - ctx context.Context - bkt objstore.InstrumentedBucket - logger log.Logger - ranges []int64 - noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark - ringLifecyclerID string - blockVisitMarkerTimeout time.Duration - blockVisitMarkerFileUpdateInterval time.Duration - blockVisitMarkerReadFailed prometheus.Counter - blockVisitMarkerWriteFailed prometheus.Counter + ctx context.Context + bkt objstore.InstrumentedBucket + logger log.Logger + ranges []int64 + noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark + ringLifecyclerID string + visitMarkerTimeout time.Duration + visitMarkerFileUpdateInterval time.Duration + visitMarkerReadFailed prometheus.Counter + visitMarkerWriteFailed prometheus.Counter } func NewShuffleShardingPlanner( @@ -33,26 +33,52 @@ func NewShuffleShardingPlanner( ranges []int64, noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark, ringLifecyclerID string, - blockVisitMarkerTimeout time.Duration, - blockVisitMarkerFileUpdateInterval time.Duration, - blockVisitMarkerReadFailed prometheus.Counter, - blockVisitMarkerWriteFailed prometheus.Counter, + visitMarkerTimeout time.Duration, + visitMarkerFileUpdateInterval time.Duration, + visitMarkerReadFailed prometheus.Counter, + visitMarkerWriteFailed prometheus.Counter, ) *ShuffleShardingPlanner { return &ShuffleShardingPlanner{ - ctx: ctx, - bkt: bkt, - logger: logger, - ranges: ranges, - noCompBlocksFunc: noCompBlocksFunc, - ringLifecyclerID: ringLifecyclerID, - blockVisitMarkerTimeout: blockVisitMarkerTimeout, - blockVisitMarkerFileUpdateInterval: blockVisitMarkerFileUpdateInterval, - blockVisitMarkerReadFailed: blockVisitMarkerReadFailed, - blockVisitMarkerWriteFailed: blockVisitMarkerWriteFailed, + ctx: ctx, + bkt: bkt, + logger: logger, + ranges: ranges, + noCompBlocksFunc: noCompBlocksFunc, + ringLifecyclerID: ringLifecyclerID, + visitMarkerTimeout: visitMarkerTimeout, + visitMarkerFileUpdateInterval: visitMarkerFileUpdateInterval, + visitMarkerReadFailed: visitMarkerReadFailed, + visitMarkerWriteFailed: visitMarkerWriteFailed, } } -func (p *ShuffleShardingPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta, _ chan error, _ any) ([]*metadata.Meta, error) { +func (p *ShuffleShardingPlanner) Plan(ctx context.Context, metasByMinTime []*metadata.Meta, errChan chan error, extensions any) ([]*metadata.Meta, error) { + partitionInfo, err := ConvertToPartitionInfo(extensions) + if err != nil { + return nil, err + } + if partitionInfo == nil { + return nil, fmt.Errorf("partitionInfo cannot be nil") + } + return p.PlanWithPartition(ctx, metasByMinTime, partitionInfo.PartitionedGroupID, partitionInfo.PartitionID, errChan) +} + +func (p *ShuffleShardingPlanner) PlanWithPartition(_ context.Context, metasByMinTime []*metadata.Meta, partitionedGroupID uint32, partitionID int, errChan chan error) ([]*metadata.Meta, error) { + partitionVisitMarker, err := ReadPartitionVisitMarker(p.ctx, p.bkt, p.logger, partitionedGroupID, partitionID, p.visitMarkerReadFailed) + if err != nil { + // shuffle_sharding_grouper should put visit marker file for blocks ready for + // compaction. So error should be returned if visit marker file does not exist. + return nil, fmt.Errorf("unable to get visit marker file for partition with partition ID %d, partitioned group ID %d: %s", partitionID, partitionedGroupID, err.Error()) + } + if partitionVisitMarker.isCompleted() { + return nil, fmt.Errorf("partition with partition ID %d is in completed status", partitionID) + } + if !partitionVisitMarker.isVisitedByCompactor(p.visitMarkerTimeout, partitionID, p.ringLifecyclerID) { + level.Warn(p.logger).Log("msg", "partition is not visited by current compactor", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "compactor_id", p.ringLifecyclerID, + "marker_partitioned_group_id", partitionVisitMarker.PartitionedGroupID, "marker_partition_id", partitionVisitMarker.PartitionID, "marker_compactor_id", partitionVisitMarker.CompactorID, "marker_visit_time", partitionVisitMarker.VisitTime) + return nil, nil + } + // Ensure all blocks fits within the largest range. This is a double check // to ensure there's no bug in the previous blocks grouping, given this Plan() // is just a pass-through. @@ -73,25 +99,15 @@ func (p *ShuffleShardingPlanner) Plan(_ context.Context, metasByMinTime []*metad return nil, fmt.Errorf("block %s with time range %d:%d is outside the largest expected range %d:%d", blockID, b.MinTime, b.MaxTime, rangeStart, rangeEnd) } - blockVisitMarker, err := ReadBlockVisitMarker(p.ctx, p.bkt, p.logger, blockID, p.blockVisitMarkerReadFailed) - if err != nil { - // shuffle_sharding_grouper should put visit marker file for blocks ready for - // compaction. So error should be returned if visit marker file does not exist. - return nil, fmt.Errorf("unable to get visit marker file for block %s: %s", blockID, err.Error()) - } - if !blockVisitMarker.isVisitedByCompactor(p.blockVisitMarkerTimeout, p.ringLifecyclerID) { - level.Warn(p.logger).Log("msg", "block is not visited by current compactor", "block_id", blockID, "compactor_id", p.ringLifecyclerID) - return nil, nil - } - resultMetas = append(resultMetas, b) } if len(resultMetas) < 2 { + level.Info(p.logger).Log("msg", "result meta size is less than 2", "partitioned_group_id", partitionedGroupID, "partition_id", partitionID, "size", len(resultMetas)) return nil, nil } - go markBlocksVisitedHeartBeat(p.ctx, p.bkt, p.logger, resultMetas, p.ringLifecyclerID, p.blockVisitMarkerFileUpdateInterval, p.blockVisitMarkerWriteFailed) + go markPartitionVisitedHeartBeat(p.ctx, p.bkt, p.logger, partitionedGroupID, partitionID, p.ringLifecyclerID, p.visitMarkerFileUpdateInterval, p.visitMarkerWriteFailed, errChan) return resultMetas, nil } diff --git a/pkg/compactor/shuffle_sharding_planner_test.go b/pkg/compactor/shuffle_sharding_planner_test.go index 83de3ad164f..ad7b7be3391 100644 --- a/pkg/compactor/shuffle_sharding_planner_test.go +++ b/pkg/compactor/shuffle_sharding_planner_test.go @@ -4,7 +4,6 @@ import ( "context" "encoding/json" "fmt" - "path" "testing" "time" @@ -24,8 +23,7 @@ import ( ) func TestShuffleShardingPlanner_Plan(t *testing.T) { - type VisitedBlock struct { - id ulid.ULID + type VisitedPartition struct { isExpired bool compactorID string } @@ -38,12 +36,12 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { block3ulid := ulid.MustNew(3, nil) tests := map[string]struct { - ranges []int64 - noCompactBlocks map[ulid.ULID]*metadata.NoCompactMark - blocks []*metadata.Meta - expected []*metadata.Meta - expectedErr error - visitedBlocks []VisitedBlock + ranges []int64 + noCompactBlocks map[ulid.ULID]*metadata.NoCompactMark + blocks []*metadata.Meta + expected []*metadata.Meta + expectedErr error + visitedPartition VisitedPartition }{ "test basic plan": { ranges: []int64{2 * time.Hour.Milliseconds()}, @@ -63,17 +61,9 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - visitedBlocks: []VisitedBlock{ - { - id: block1ulid, - isExpired: false, - compactorID: currentCompactor, - }, - { - id: block2ulid, - isExpired: false, - compactorID: currentCompactor, - }, + visitedPartition: VisitedPartition{ + isExpired: false, + compactorID: currentCompactor, }, expected: []*metadata.Meta{ { @@ -110,17 +100,9 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - visitedBlocks: []VisitedBlock{ - { - id: block1ulid, - isExpired: false, - compactorID: currentCompactor, - }, - { - id: block2ulid, - isExpired: false, - compactorID: currentCompactor, - }, + visitedPartition: VisitedPartition{ + isExpired: false, + compactorID: currentCompactor, }, expectedErr: fmt.Errorf("block %s with time range %d:%d is outside the largest expected range %d:%d", block2ulid.String(), 0*time.Hour.Milliseconds(), 2*time.Hour.Milliseconds(), 2*time.Hour.Milliseconds(), 4*time.Hour.Milliseconds()), }, @@ -142,17 +124,9 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - visitedBlocks: []VisitedBlock{ - { - id: block1ulid, - isExpired: false, - compactorID: currentCompactor, - }, - { - id: block2ulid, - isExpired: false, - compactorID: currentCompactor, - }, + visitedPartition: VisitedPartition{ + isExpired: false, + compactorID: currentCompactor, }, expectedErr: fmt.Errorf("block %s with time range %d:%d is outside the largest expected range %d:%d", block1ulid.String(), 0*time.Hour.Milliseconds(), 4*time.Hour.Milliseconds(), 0*time.Hour.Milliseconds(), 2*time.Hour.Milliseconds()), }, @@ -174,17 +148,9 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - visitedBlocks: []VisitedBlock{ - { - id: block1ulid, - isExpired: false, - compactorID: currentCompactor, - }, - { - id: block2ulid, - isExpired: false, - compactorID: currentCompactor, - }, + visitedPartition: VisitedPartition{ + isExpired: false, + compactorID: currentCompactor, }, expectedErr: fmt.Errorf("block %s with time range %d:%d is outside the largest expected range %d:%d", block2ulid.String(), 0*time.Hour.Milliseconds(), 4*time.Hour.Milliseconds(), 0*time.Hour.Milliseconds(), 2*time.Hour.Milliseconds()), }, @@ -214,22 +180,9 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - visitedBlocks: []VisitedBlock{ - { - id: block1ulid, - isExpired: false, - compactorID: currentCompactor, - }, - { - id: block2ulid, - isExpired: false, - compactorID: currentCompactor, - }, - { - id: block3ulid, - isExpired: false, - compactorID: currentCompactor, - }, + visitedPartition: VisitedPartition{ + isExpired: false, + compactorID: currentCompactor, }, expected: []*metadata.Meta{ { @@ -267,17 +220,9 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - visitedBlocks: []VisitedBlock{ - { - id: block1ulid, - isExpired: false, - compactorID: currentCompactor, - }, - { - id: block2ulid, - isExpired: false, - compactorID: currentCompactor, - }, + visitedPartition: VisitedPartition{ + isExpired: false, + compactorID: currentCompactor, }, expected: []*metadata.Meta{}, }, @@ -299,12 +244,9 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - visitedBlocks: []VisitedBlock{ - { - id: block1ulid, - isExpired: false, - compactorID: otherCompactor, - }, + visitedPartition: VisitedPartition{ + isExpired: false, + compactorID: otherCompactor, }, expected: []*metadata.Meta{}, }, @@ -326,35 +268,34 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - visitedBlocks: []VisitedBlock{ - { - id: block1ulid, - isExpired: true, - compactorID: currentCompactor, - }, + visitedPartition: VisitedPartition{ + isExpired: true, + compactorID: currentCompactor, }, expected: []*metadata.Meta{}, }, } blockVisitMarkerTimeout := 5 * time.Minute + partitionedGroupID := uint32(1) + partitionID := 0 for testName, testData := range tests { t.Run(testName, func(t *testing.T) { bkt := &bucket.ClientMock{} - for _, visitedBlock := range testData.visitedBlocks { - visitMarkerFile := path.Join(visitedBlock.id.String(), BlockVisitMarkerFile) - expireTime := time.Now() - if visitedBlock.isExpired { - expireTime = expireTime.Add(-1 * blockVisitMarkerTimeout) - } - blockVisitMarker := BlockVisitMarker{ - CompactorID: visitedBlock.compactorID, - VisitTime: expireTime.Unix(), - Version: VisitMarkerVersion1, - } - visitMarkerFileContent, _ := json.Marshal(blockVisitMarker) - bkt.MockGet(visitMarkerFile, string(visitMarkerFileContent), nil) + visitMarkerFile := GetPartitionVisitMarkerFile(partitionedGroupID, partitionID) + expireTime := time.Now() + if testData.visitedPartition.isExpired { + expireTime = expireTime.Add(-1 * blockVisitMarkerTimeout) + } + blockVisitMarker := PartitionVisitMarker{ + CompactorID: testData.visitedPartition.compactorID, + PartitionedGroupID: partitionedGroupID, + PartitionID: partitionID, + VisitTime: expireTime.Unix(), + Version: PartitionVisitMarkerVersion1, } + visitMarkerFileContent, _ := json.Marshal(blockVisitMarker) + bkt.MockGet(visitMarkerFile, string(visitMarkerFileContent), nil) bkt.MockUpload(mock.Anything, nil) registerer := prometheus.NewPedanticRegistry() @@ -383,7 +324,13 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { blockVisitMarkerReadFailed, blockVisitMarkerWriteFailed, ) - actual, err := p.Plan(context.Background(), testData.blocks, nil, nil) + actual, err := p.Plan(context.Background(), testData.blocks, nil, &CortexMetaExtensions{ + PartitionInfo: &PartitionInfo{ + PartitionCount: 1, + PartitionID: partitionID, + PartitionedGroupID: partitionedGroupID, + }, + }) if testData.expectedErr != nil { assert.Equal(t, err, testData.expectedErr) diff --git a/pkg/compactor/testutils/block.go b/pkg/compactor/testutils/block.go new file mode 100644 index 00000000000..3a21acdd9ac --- /dev/null +++ b/pkg/compactor/testutils/block.go @@ -0,0 +1,149 @@ +package testutils + +import ( + "context" + "flag" + "fmt" + "hash/fnv" + "path/filepath" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "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/model" + "github.com/thanos-io/thanos/pkg/testutil/e2eutil" + + "github.com/cortexproject/cortex/pkg/compactor" + "github.com/cortexproject/cortex/pkg/storage/bucket" + "github.com/cortexproject/cortex/pkg/storage/tsdb" +) + +func GenerateBlock(ctx context.Context, logger log.Logger, cfg BlockGenConfig, storageCfg tsdb.BlocksStorageConfig) error { + seriesGroups := generateSeriesGroups(cfg) + partitionedGroupID := generatePartitionedGroupID(cfg.WorkspaceID, cfg.BlockStartTime.Time.UnixMilli(), cfg.BlockEndTime.Time.UnixMilli()) + for seriesGroupIdx, seriesGroup := range seriesGroups { + blockID, err := e2eutil.CreateBlock( + ctx, + cfg.OutputDirectory, + seriesGroup, + cfg.SamplesPerSeries, + cfg.BlockStartTime.Time.UnixMilli(), + cfg.BlockEndTime.Time.UnixMilli(), + labels.Labels{ + {Name: "__org_id__", Value: cfg.WorkspaceID}, + }, + 0, + metadata.NoneFunc) + if err != nil { + level.Error(logger).Log("msg", "unable to create block", "series_group_idx", seriesGroupIdx, "err", err) + return errors.Wrapf(err, "unable to create block") + } + if err = updateMeta(logger, blockID, seriesGroupIdx, partitionedGroupID, cfg); err != nil { + level.Error(logger).Log("msg", "unable to update meta for block", "series_group_idx", seriesGroupIdx, "err", err) + return errors.Wrapf(err, "unable to update meta for block") + } + level.Info(logger).Log("msg", "block created", "series_group_idx", seriesGroupIdx, "block_id", blockID.String()) + err = uploadBlock(ctx, logger, cfg, storageCfg, blockID) + if err != nil { + level.Error(logger).Log("msg", "failed to upload block", "series_group_idx", seriesGroupIdx, "block_id", blockID.String()) + return errors.Wrapf(err, "failed to upload block") + } + level.Info(logger).Log("msg", "uploaded block", "series_group_idx", seriesGroupIdx, "block_id", blockID.String()) + } + return nil +} + +func generateSeriesGroups(cfg BlockGenConfig) map[int][]labels.Labels { + seriesGroups := make(map[int][]labels.Labels) + for i := 0; i < cfg.MetricsPerWorkspace; i++ { + for j := 0; j < cfg.ActiveSeriesPerMetric; j++ { + metricLabels := labels.Labels{ + {Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}, + {Name: fmt.Sprintf("extra_label_%d", j), Value: fmt.Sprintf("extra_label_value_%d", j)}, + } + if cfg.CompactionLevel > 1 { + partitionID := int(metricLabels.Hash() % uint64(cfg.PartitionCount)) + seriesGroups[partitionID] = append(seriesGroups[partitionID], metricLabels) + } else { + shardID := (cfg.ActiveSeriesPerMetric*i + j) % cfg.L1ShardSize + seriesGroups[shardID] = append(seriesGroups[shardID], metricLabels) + } + } + } + return seriesGroups +} + +func updateMeta(logger log.Logger, blockID ulid.ULID, seriesGroupIdx int, partitionedGroupID uint32, cfg BlockGenConfig) error { + blockDir := filepath.Join(cfg.OutputDirectory, blockID.String()) + newMeta, err := metadata.ReadFromDir(blockDir) + if err != nil { + return err + } + newMeta.Compaction.Level = cfg.CompactionLevel + if cfg.CompactionLevel > 1 { + newMeta.Thanos.Extensions = compactor.CortexMetaExtensions{ + PartitionInfo: &compactor.PartitionInfo{ + PartitionedGroupID: partitionedGroupID, + PartitionID: seriesGroupIdx, + PartitionCount: cfg.PartitionCount, + }, + } + } else { + newMeta.Thanos.Labels["__ingester_id__"] = fmt.Sprintf("ingester-%d", seriesGroupIdx) + } + if err := newMeta.WriteToDir(logger, blockDir); err != nil { + return err + } + return nil +} + +func generatePartitionedGroupID(workspaceID string, rangeStart int64, rangeEnd int64) uint32 { + groupString := fmt.Sprintf("%v%v%v", workspaceID, rangeStart, rangeEnd) + groupHasher := fnv.New32a() + _, _ = groupHasher.Write([]byte(groupString)) + groupHash := groupHasher.Sum32() + return groupHash +} + +func uploadBlock(ctx context.Context, logger log.Logger, cfg BlockGenConfig, storageCfg tsdb.BlocksStorageConfig, blockID ulid.ULID) error { + blockDir := filepath.Join(cfg.OutputDirectory, blockID.String()) + bkt, err := bucket.NewClient(ctx, storageCfg.Bucket, "generate-block", logger, prometheus.NewPedanticRegistry()) + if err != nil { + return err + } + bkt = bucket.NewPrefixedBucketClient(bkt, cfg.WorkspaceID) + return block.Upload(ctx, logger, bkt, blockDir, metadata.NoneFunc, objstore.WithUploadConcurrency(4)) +} + +type BlockGenConfig struct { + OutputDirectory string `yaml:"output_directory"` + WorkspaceID string `yaml:"workspace_id"` + MetricsPerWorkspace int `yaml:"metrics_per_workspace"` + ActiveSeriesPerMetric int `yaml:"active_series_per_metric"` + SamplesPerSeries int `yaml:"samples_per_series"` + BlockStartTime model.TimeOrDurationValue `yaml:"block_start_time"` + BlockEndTime model.TimeOrDurationValue `yaml:"block_end_time"` + CompactionLevel int `yaml:"compaction_level"` + PartitionCount int `yaml:"partition_count"` + L1ShardSize int `yaml:"shard_size"` +} + +func (c *BlockGenConfig) RegisterFlags(f *flag.FlagSet) { + f.StringVar(&c.OutputDirectory, "output-directory", "", "Path to output directory of generated blocks on local disk") + f.StringVar(&c.WorkspaceID, "workspace-id", "", "Workspace ID") + f.IntVar(&c.MetricsPerWorkspace, "metrics-per-workspace", 0, "Number of metrics per workspace") + f.IntVar(&c.ActiveSeriesPerMetric, "active-series-per-metric", 0, "Number of series per metric") + f.IntVar(&c.SamplesPerSeries, "samples-per-series", 0, "Number of samples per series") + f.Var(&c.BlockStartTime, "block-start-time", "Start time of generated block. It can be in either RFC3339 timestamp format or relative time compared to now") + f.Var(&c.BlockEndTime, "block-end-time", "End time of generated block. It can be in either RFC3339 timestamp format or relative time compared to now") + f.IntVar(&c.CompactionLevel, "compaction-level", 1, "Compaction level of generated block") + f.IntVar(&c.PartitionCount, "partition-count", 1, "Number of partitions generated block would be split into. This only works when CompactionLevel is greater than 1") + f.IntVar(&c.L1ShardSize, "shard-size", 1, "Number of shards. This only works when CompactionLevel set to 1") +} diff --git a/pkg/storage/tsdb/bucketindex/index.go b/pkg/storage/tsdb/bucketindex/index.go index 987d528ed06..36bdc05c2b1 100644 --- a/pkg/storage/tsdb/bucketindex/index.go +++ b/pkg/storage/tsdb/bucketindex/index.go @@ -236,6 +236,14 @@ func (s BlockDeletionMarks) GetULIDs() []ulid.ULID { return ids } +func (s BlockDeletionMarks) GetULIDSet() map[ulid.ULID]struct{} { + res := make(map[ulid.ULID]struct{}) + for _, m := range s { + res[m.ID] = struct{}{} + } + return res +} + func (s BlockDeletionMarks) Clone() BlockDeletionMarks { clone := make(BlockDeletionMarks, len(s)) for i, m := range s { diff --git a/pkg/storage/tsdb/bucketindex/storage.go b/pkg/storage/tsdb/bucketindex/storage.go index a884cfd57dc..35af5fb57c5 100644 --- a/pkg/storage/tsdb/bucketindex/storage.go +++ b/pkg/storage/tsdb/bucketindex/storage.go @@ -69,7 +69,11 @@ func (s *Status) GetNonQueryableUntil() time.Time { // ReadIndex reads, parses and returns a bucket index from the bucket. func ReadIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvider bucket.TenantConfigProvider, logger log.Logger) (*Index, error) { userBkt := bucket.NewUserBucketClient(userID, bkt, cfgProvider) + return ReadUserIndex(ctx, userBkt, logger) +} +// ReadUserIndex reads, parses and returns user bucket index from the bucket. +func ReadUserIndex(ctx context.Context, userBkt objstore.InstrumentedBucket, logger log.Logger) (*Index, error) { // Get the bucket index. reader, err := userBkt.WithExpectedErrs(tsdb.IsOneOfTheExpectedErrors(userBkt.IsAccessDeniedErr, userBkt.IsObjNotFoundErr)).Get(ctx, IndexCompressedFilename) if err != nil { diff --git a/pkg/storage/tsdb/bucketindex/updater.go b/pkg/storage/tsdb/bucketindex/updater.go index cee3e6e3bf4..b6de8c06106 100644 --- a/pkg/storage/tsdb/bucketindex/updater.go +++ b/pkg/storage/tsdb/bucketindex/updater.go @@ -79,17 +79,22 @@ func (w *Updater) updateBlocks(ctx context.Context, old []*Block, deletedBlocks partials = map[ulid.ULID]error{} // Find all blocks in the storage. + begin := time.Now() + count := 0 err := w.bkt.Iter(ctx, "", func(name string) error { if id, ok := block.IsBlockDir(name); ok { discovered[id] = struct{}{} } + count++ return nil }) if err != nil { return nil, nil, errors.Wrap(err, "list blocks") } + level.Info(w.logger).Log("msg", "finish iterating blocks", "iteration_count", count, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Since blocks are immutable, all blocks already existing in the index can just be copied. + begin = time.Now() for _, b := range old { if _, ok := discovered[b.ID]; ok { delete(discovered, b.ID) @@ -101,10 +106,12 @@ func (w *Updater) updateBlocks(ctx context.Context, old []*Block, deletedBlocks blocks = append(blocks, b) } } + level.Info(w.logger).Log("msg", "finish adding blocks", "old_blocks_count", len(old), "new_blocks_count", len(blocks), "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Remaining blocks are new ones and we have to fetch the meta.json for each of them, in order // to find out if their upload has been completed (meta.json is uploaded last) and get the block // information to store in the bucket index. + begin = time.Now() for id := range discovered { b, err := w.updateBlockIndexEntry(ctx, id) if err == nil { @@ -129,6 +136,7 @@ func (w *Updater) updateBlocks(ctx context.Context, old []*Block, deletedBlocks } return nil, nil, err } + level.Info(w.logger).Log("msg", "finish updating block entries", "discovered_blocks_count", len(discovered), "new_blocks_count", len(blocks), "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) return blocks, partials, nil } @@ -187,6 +195,8 @@ func (w *Updater) updateBlockMarks(ctx context.Context, old []*BlockDeletionMark totalBlocksBlocksMarkedForNoCompaction := int64(0) // Find all markers in the storage. + begin := time.Now() + count := 0 err := w.bkt.Iter(ctx, MarkersPathname+"/", func(name string) error { if blockID, ok := IsBlockDeletionMarkFilename(path.Base(name)); ok { discovered[blockID] = struct{}{} @@ -196,13 +206,16 @@ func (w *Updater) updateBlockMarks(ctx context.Context, old []*BlockDeletionMark totalBlocksBlocksMarkedForNoCompaction++ } + count++ return nil }) if err != nil { return nil, nil, totalBlocksBlocksMarkedForNoCompaction, errors.Wrap(err, "list block deletion marks") } + level.Info(w.logger).Log("msg", "finish iterating markers", "iteration_count", count, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Since deletion marks are immutable, all markers already existing in the index can just be copied. + begin = time.Now() for _, m := range old { if _, ok := discovered[m.ID]; ok { out = append(out, m) @@ -211,8 +224,10 @@ func (w *Updater) updateBlockMarks(ctx context.Context, old []*BlockDeletionMark deletedBlocks[m.ID] = struct{}{} } } + level.Info(w.logger).Log("msg", "finish getting deleted blocks", "old_blocks_count", len(old), "deleted_blocks_count", len(deletedBlocks), "deletion_markers_count", len(out), "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Remaining markers are new ones and we have to fetch them. + begin = time.Now() for id := range discovered { m, err := w.updateBlockDeletionMarkIndexEntry(ctx, id) if errors.Is(err, ErrBlockDeletionMarkNotFound) { @@ -230,6 +245,7 @@ func (w *Updater) updateBlockMarks(ctx context.Context, old []*BlockDeletionMark out = append(out, m) } + level.Info(w.logger).Log("msg", "finish getting new deletion markers", "discovered_blocks_count", len(discovered), "deletion_markers_count", len(out), "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) return out, deletedBlocks, totalBlocksBlocksMarkedForNoCompaction, nil } diff --git a/pkg/util/log/wrappers.go b/pkg/util/log/wrappers.go index 50567f61442..f30103bec3a 100644 --- a/pkg/util/log/wrappers.go +++ b/pkg/util/log/wrappers.go @@ -17,6 +17,12 @@ func WithUserID(userID string, l kitlog.Logger) kitlog.Logger { return kitlog.With(l, "org_id", userID) } +// WithExecutionID returns a Logger that has information about the execution id in +// its details. +func WithExecutionID(executionID string, l kitlog.Logger) kitlog.Logger { + return kitlog.With(l, "execution_id", executionID) +} + // WithTraceID returns a Logger that has information about the traceID in // its details. func WithTraceID(traceID string, l kitlog.Logger) kitlog.Logger { diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index d60b2ca8e65..ca05c8cc100 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -114,8 +114,10 @@ type Limits struct { MaxDownloadedBytesPerRequest int `yaml:"max_downloaded_bytes_per_request" json:"max_downloaded_bytes_per_request"` // Compactor. - CompactorBlocksRetentionPeriod model.Duration `yaml:"compactor_blocks_retention_period" json:"compactor_blocks_retention_period"` - CompactorTenantShardSize int `yaml:"compactor_tenant_shard_size" json:"compactor_tenant_shard_size"` + CompactorBlocksRetentionPeriod model.Duration `yaml:"compactor_blocks_retention_period" json:"compactor_blocks_retention_period"` + CompactorTenantShardSize int `yaml:"compactor_tenant_shard_size" json:"compactor_tenant_shard_size"` + CompactorPartitionIndexSizeLimitInBytes int64 `yaml:"compactor_partition_index_size_limit_in_bytes" json:"compactor_partition_index_size_limit_in_bytes"` + CompactorPartitionSeriesCountLimit int64 `yaml:"compactor_partition_series_count_limit" json:"compactor_partition_series_count_limit"` // This config doesn't have a CLI flag registered here because they're registered in // their own original config struct. @@ -196,6 +198,8 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.Var(&l.CompactorBlocksRetentionPeriod, "compactor.blocks-retention-period", "Delete blocks containing samples older than the specified retention period. 0 to disable.") f.IntVar(&l.CompactorTenantShardSize, "compactor.tenant-shard-size", 0, "The default tenant's shard size when the shuffle-sharding strategy is used by the compactor. When this setting is specified in the per-tenant overrides, a value of 0 disables shuffle sharding for the tenant.") + f.Int64Var(&l.CompactorPartitionIndexSizeLimitInBytes, "compactor.partition-index-size-limit-in-bytes", 0, "Index size limit in bytes for each compaction partition. 0 means no limit") + f.Int64Var(&l.CompactorPartitionSeriesCountLimit, "compactor.partition-series-count-limit", 0, "Time series count limit for each compaction partition. 0 means no limit") // Store-gateway. f.Float64Var(&l.StoreGatewayTenantShardSize, "store-gateway.tenant-shard-size", 0, "The default tenant's shard size when the shuffle-sharding strategy is used. Must be set when the store-gateway sharding is enabled with the shuffle-sharding strategy. When this setting is specified in the per-tenant overrides, a value of 0 disables shuffle sharding for the tenant. If the value is < 1 the shard size will be a percentage of the total store-gateways.") @@ -542,6 +546,16 @@ func (o *Overrides) CompactorTenantShardSize(userID string) int { return o.GetOverridesForUser(userID).CompactorTenantShardSize } +// CompactorPartitionIndexSizeLimitInBytes returns shard size (number of rulers) used by this tenant when using shuffle-sharding strategy. +func (o *Overrides) CompactorPartitionIndexSizeLimitInBytes(userID string) int64 { + return o.GetOverridesForUser(userID).CompactorPartitionIndexSizeLimitInBytes +} + +// CompactorPartitionSeriesCountLimit returns shard size (number of rulers) used by this tenant when using shuffle-sharding strategy. +func (o *Overrides) CompactorPartitionSeriesCountLimit(userID string) int64 { + return o.GetOverridesForUser(userID).CompactorPartitionSeriesCountLimit +} + // MetricRelabelConfigs returns the metric relabel configs for a given user. func (o *Overrides) MetricRelabelConfigs(userID string) []*relabel.Config { return o.GetOverridesForUser(userID).MetricRelabelConfigs