From dc65d9514e43d797b5881cac812ebca1b2afc240 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 27 Jul 2022 15:28:50 -0700 Subject: [PATCH 01/17] Introduced lock file to shuffle sharding grouper Signed-off-by: Alex Le --- pkg/compactor/compactor.go | 8 +- pkg/compactor/compactor_test.go | 31 +++ pkg/compactor/shuffle_sharding_grouper.go | 235 +++++++++++++----- .../shuffle_sharding_grouper_test.go | 96 ++++++- pkg/storage/bucket/client_mock.go | 23 ++ 5 files changed, 317 insertions(+), 76 deletions(-) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index f6a208591fe..f0e4538e956 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -70,6 +70,7 @@ var ( ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.Bucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string) compact.Grouper { return NewShuffleShardingGrouper( + ctx, logger, bkt, false, // Do not accept malformed indexes @@ -86,7 +87,8 @@ var ( limits, userID, cfg.BlockFilesConcurrency, - cfg.BlocksFetchConcurrency) + cfg.BlocksFetchConcurrency, + 1) } DefaultBlocksCompactorFactory = func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { @@ -760,10 +762,12 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { return errors.Wrap(err, "failed to create syncer") } + currentCtx, cancel := context.WithCancel(ctx) + defer cancel() compactor, err := compact.NewBucketCompactor( ulogger, syncer, - c.blocksGrouperFactory(ctx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.ring, c.ringLifecycler, c.limits, userID), + c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.ring, c.ringLifecycler, c.limits, userID), c.blocksPlannerFactory(ulogger, c.compactorCfg, noCompactMarkerFilter), c.blocksCompactor, path.Join(c.compactorCfg.DataDir, "compact"), diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index ecf333d86d0..80ac1a93027 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -465,9 +465,13 @@ 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/block.lock", "", nil) + bucketClient.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/block.lock", 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/block.lock", "", nil) + bucketClient.MockUpload(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/block.lock", nil) bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) @@ -516,15 +520,19 @@ 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/block.lock", "", 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/block.lock", "", 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/block.lock", "", 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/block.lock", "", nil) bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) bucketClient.MockIter("user-1/markers/", nil, nil) @@ -767,16 +775,24 @@ 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/block.lock", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.lock", 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/block.lock", "", nil) + bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.lock", 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/block.lock", "", nil) + bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.lock", 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/block.lock", "", nil) + bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.lock", nil) bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) @@ -828,6 +844,8 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTVP434PA9VFXSW2JKB3392D/index"}, nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/index", "some index content", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/block.lock", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.lock", nil) bucketClient.MockExists("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", false, nil) bucketClient.MockDelete("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", nil) @@ -988,15 +1006,23 @@ 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/block.lock", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.lock", 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/block.lock", "", nil) + bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.lock", 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/block.lock", "", nil) + bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.lock", 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/block.lock", "", nil) + bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.lock", nil) bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) @@ -1078,6 +1104,8 @@ func TestCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndM 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/block.lock", "", nil) + bucketClient.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/block.lock", nil) bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) } @@ -1182,6 +1210,9 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit 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.MockGetTimes(userID+"/"+blockID+"/block.lock", "", nil, 1) + bucketClient.MockGet(userID+"/"+blockID+"/block.lock", time.Now().Format(DefaultTimeFormat), nil) + bucketClient.MockUpload(userID+"/"+blockID+"/block.lock", nil) blockDirectory = append(blockDirectory, userID+"/"+blockID) // Get all of the unique group hashes so that they can be used to ensure all groups were compacted diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index 146fd627211..8c25029b940 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -1,8 +1,12 @@ package compactor import ( + "bytes" + "context" "fmt" "hash/fnv" + "io/ioutil" + "path" "sort" "strings" "time" @@ -21,7 +25,12 @@ import ( "github.com/cortexproject/cortex/pkg/ring" ) +const BlockLockFile = "block.lock" +const DefaultTimeFormat = time.RFC3339 +const HeartBeatTimeout = 5 * time.Minute + type ShuffleShardingGrouper struct { + ctx context.Context logger log.Logger bkt objstore.Bucket acceptMalformedIndex bool @@ -45,9 +54,13 @@ type ShuffleShardingGrouper struct { ring ring.ReadRing ringLifecyclerAddr string + + groupCallLimit int + groupCallCount int } func NewShuffleShardingGrouper( + ctx context.Context, logger log.Logger, bkt objstore.Bucket, acceptMalformedIndex bool, @@ -65,12 +78,14 @@ func NewShuffleShardingGrouper( userID string, blockFilesConcurrency int, blocksFetchConcurrency int, + groupCallLimit int, ) *ShuffleShardingGrouper { if logger == nil { logger = log.NewNopLogger() } return &ShuffleShardingGrouper{ + ctx: ctx, logger: logger, bkt: bkt, acceptMalformedIndex: acceptMalformedIndex, @@ -109,11 +124,19 @@ func NewShuffleShardingGrouper( userID: userID, blockFilesConcurrency: blockFilesConcurrency, blocksFetchConcurrency: blocksFetchConcurrency, + groupCallLimit: groupCallLimit, + groupCallCount: 0, } } // Groups function modified from https://github.com/cortexproject/cortex/pull/2616 func (g *ShuffleShardingGrouper) Groups(blocks map[ulid.ULID]*metadata.Meta) (res []*compact.Group, err error) { + g.groupCallCount++ + if g.groupCallCount > g.groupCallLimit { + level.Debug(g.logger).Log("msg", "exceed group call limit, return empty group", "groupCallLimit", g.groupCallLimit) + return []*compact.Group{}, nil + } + // First of all we have to group blocks using the Thanos default // grouping (based on downsample resolution + external labels). mainGroups := map[string][]*metadata.Meta{} @@ -142,64 +165,10 @@ func (g *ShuffleShardingGrouper) Groups(blocks map[ulid.ULID]*metadata.Meta) (re var remainingCompactions = 0. defer func() { g.remainingPlannedCompactions.Set(remainingCompactions) }() + var groups []blocksGroup for _, mainBlocks := range mainGroups { for _, group := range groupBlocksByCompactableRanges(mainBlocks, g.compactorCfg.BlockRanges.ToMilliseconds()) { - // Nothing to do if we don't have at least 2 blocks. - if len(group.blocks) < 2 { - continue - } - - groupHash := hashGroup(g.userID, group.rangeStart, group.rangeEnd) - - if owned, err := g.ownGroup(groupHash); err != nil { - level.Warn(g.logger).Log("msg", "unable to check if user is owned by this shard", "group hash", groupHash, "err", err, "group", group.String()) - continue - } else if !owned { - level.Info(g.logger).Log("msg", "skipping group because it is not owned by this shard", "group_hash", groupHash) - continue - } - - remainingCompactions++ - groupKey := fmt.Sprintf("%v%s", groupHash, group.blocks[0].Thanos.GroupKey()) - - level.Info(g.logger).Log("msg", "found compactable group for user", "group_hash", groupHash, "group", group.String()) - - // 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) - - thanosGroup, err := compact.NewGroup( - log.With(g.logger, "groupKey", groupKey, "rangeStart", group.rangeStartTime().String(), "rangeEnd", group.rangeEndTime().String(), "externalLabels", externalLabels, "downsampleResolution", resolution), - g.bkt, - groupKey, - externalLabels, - resolution, - false, // No malformed index. - 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.garbageCollectedBlocks, - g.blocksMarkedForDeletion, - g.blocksMarkedForNoCompact, - g.hashFunc, - g.blockFilesConcurrency, - g.blocksFetchConcurrency, - ) - if err != nil { - return nil, errors.Wrap(err, "create compaction group") - } - - for _, m := range group.blocks { - if err := thanosGroup.AppendMeta(m); err != nil { - return nil, errors.Wrap(err, "add block to compaction group") - } - } - - outGroups = append(outGroups, thanosGroup) + groups = append(groups, group) } } @@ -207,21 +176,102 @@ func (g *ShuffleShardingGrouper) Groups(blocks map[ulid.ULID]*metadata.Meta) (re // is that we wanna favor smaller ranges first (ie. to deduplicate samples sooner // than later) and older ones are more likely to be "complete" (no missing block still // to be uploaded). - sort.SliceStable(outGroups, func(i, j int) bool { - iLength := outGroups[i].MaxTime() - outGroups[i].MinTime() - jLength := outGroups[j].MaxTime() - outGroups[j].MinTime() + 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 if iLength != jLength { return iLength < jLength } - if outGroups[i].MinTime() != outGroups[j].MinTime() { - return outGroups[i].MinTime() < outGroups[j].MinTime() + if iMinTime != jMinTime { + return iMinTime < jMinTime } + iGroupHash := hashGroup(g.userID, iGroup.rangeStart, iGroup.rangeEnd) + iGroupKey := createGroupKey(iGroupHash, iGroup) + jGroupHash := hashGroup(g.userID, jGroup.rangeStart, jGroup.rangeEnd) + jGroupKey := createGroupKey(jGroupHash, jGroup) // Guarantee stable sort for tests. - return outGroups[i].Key() < outGroups[j].Key() + return iGroupKey < jGroupKey }) +mainLoop: + for _, group := range groups { + var blockIds []string + for _, block := range group.blocks { + blockIds = append(blockIds, block.ULID.String()) + } + blocksInfo := strings.Join(blockIds, ",") + level.Info(g.logger).Log("msg", "check group", "blocks", blocksInfo) + + // Nothing to do if we don't have at least 2 blocks. + if len(group.blocks) < 2 { + continue + } + + groupHash := hashGroup(g.userID, group.rangeStart, group.rangeEnd) + + if isLocked, err := g.isGroupLocked(group.blocks); err != nil { + level.Warn(g.logger).Log("msg", "unable to check if blocks in group are locked", "group hash", groupHash, "err", err, "group", group.String()) + continue + } else if isLocked { + level.Info(g.logger).Log("msg", "skipping group because at least one block in group are locked", "group_hash", groupHash) + continue + } + + remainingCompactions++ + groupKey := createGroupKey(groupHash, group) + + level.Info(g.logger).Log("msg", "found compactable group for user", "group_hash", groupHash, "group", group.String()) + go g.groupLockHeartBeat(group.blocks) + + // 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) + + thanosGroup, err := compact.NewGroup( + log.With(g.logger, "groupKey", groupKey, "rangeStart", group.rangeStartTime().String(), "rangeEnd", group.rangeEndTime().String(), "externalLabels", externalLabels, "downsampleResolution", resolution), + g.bkt, + groupKey, + externalLabels, + resolution, + false, // No malformed index. + 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.garbageCollectedBlocks, + g.blocksMarkedForDeletion, + g.blocksMarkedForNoCompact, + g.hashFunc, + g.blockFilesConcurrency, + g.blocksFetchConcurrency, + ) + if err != nil { + return nil, errors.Wrap(err, "create compaction group") + } + + for _, m := range group.blocks { + if err := thanosGroup.AppendMeta(m); err != nil { + return nil, errors.Wrap(err, "add block to compaction group") + } + } + + outGroups = append(outGroups, thanosGroup) + break mainLoop + } + + level.Info(g.logger).Log("msg", fmt.Sprintf("total groups for compaction: %d", len(outGroups))) + return outGroups, nil } @@ -241,6 +291,63 @@ func (g *ShuffleShardingGrouper) ownGroup(groupHash uint32) (bool, error) { return rs.Instances[0].Addr == g.ringLifecyclerAddr, nil } +func (g *ShuffleShardingGrouper) isGroupLocked(blocks []*metadata.Meta) (bool, error) { + for _, block := range blocks { + blockId := block.ULID.String() + lockFileReader, err := g.bkt.Get(g.ctx, path.Join(blockId, BlockLockFile)) + if err != nil { + if g.bkt.IsObjNotFoundErr(err) { + level.Debug(g.logger).Log("msg", fmt.Sprintf("no lock file for block: %s", blockId)) + continue + } + return true, err + } + b, err := ioutil.ReadAll(lockFileReader) + if err != nil { + level.Error(g.logger).Log("msg", fmt.Sprintf("unable to reach lock file for block: %s", blockId), "err", err) + return true, err + } + heartBeatTime, err := time.Parse(DefaultTimeFormat, string(b)) + if err != nil { + level.Error(g.logger).Log("msg", fmt.Sprintf("unable to parse timestamp in lock file for block: %s", blockId), "err", err) + return true, err + } + if time.Now().Before(heartBeatTime.Add(HeartBeatTimeout)) { + level.Debug(g.logger).Log("msg", fmt.Sprintf("locked block: %s", blockId)) + return true, nil + } + } + return false, nil +} + +func (g *ShuffleShardingGrouper) groupLockHeartBeat(blocks []*metadata.Meta) { + var blockIds []string + for _, block := range blocks { + blockIds = append(blockIds, block.ULID.String()) + } + blocksInfo := strings.Join(blockIds, ",") + level.Info(g.logger).Log("msg", fmt.Sprintf("start heart beat for blocks: %s", blocksInfo)) +heartBeat: + for { + select { + case <-g.ctx.Done(): + break heartBeat + default: + level.Debug(g.logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) + for _, block := range blocks { + blockId := block.ULID.String() + blockLockFilePath := path.Join(blockId, BlockLockFile) + err := g.bkt.Upload(g.ctx, blockLockFilePath, bytes.NewReader([]byte(time.Now().Format(DefaultTimeFormat)))) + if err != nil { + level.Error(g.logger).Log("msg", "unable to update heart beat for block", "block_id", blockId, "err", err) + } + } + time.Sleep(HeartBeatTimeout / 5) // it allows up to 5 failures on heart heat for single block + } + } + level.Info(g.logger).Log("msg", fmt.Sprintf("stop heart beat for blocks: %s", blocksInfo)) +} + // Check whether this compactor exists on the subring based on user ID func (g *ShuffleShardingGrouper) checkSubringForCompactor() (bool, error) { subRing := g.ring.ShuffleShard(g.userID, g.limits.CompactorTenantShardSize(g.userID)) @@ -264,6 +371,10 @@ func hashGroup(userID string, rangeStart int64, rangeEnd int64) uint32 { return groupHash } +func createGroupKey(groupHash uint32, group blocksGroup) string { + return fmt.Sprintf("%v%s", groupHash, group.blocks[0].Thanos.GroupKey()) +} + // blocksGroup struct and functions copied and adjusted from https://github.com/cortexproject/cortex/pull/2616 type blocksGroup struct { rangeStart int64 // Included. diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index d1ae8d2a8e9..6745bb8e554 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -2,6 +2,9 @@ package compactor import ( "bytes" + "context" + "github.com/cortexproject/cortex/pkg/storage/bucket" + "path" "testing" "time" @@ -112,8 +115,12 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { } tests := map[string]struct { - ranges []time.Duration - blocks map[ulid.ULID]*metadata.Meta + ranges []time.Duration + blocks map[ulid.ULID]*metadata.Meta + lockedBlocks []struct { + id ulid.ULID + isExpired bool + } expected [][]ulid.ULID metrics string }{ @@ -122,12 +129,10 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { 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}, - {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 3 + cortex_compactor_remaining_planned_compactions 1 `, }, "test no compaction": { @@ -144,12 +149,10 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid], block4hto6hExt2Ulid: blocks[block4hto6hExt2Ulid], block6hto8hExt2Ulid: blocks[block6hto8hExt2Ulid]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, - {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, - {block4hto6hExt2Ulid, block6hto8hExt2Ulid}, }, 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 3 + cortex_compactor_remaining_planned_compactions 1 `, }, "test oldest min time first": { @@ -157,11 +160,10 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { blocks: map[ulid.ULID]*metadata.Meta{block1hto2hExt1Ulid: blocks[block1hto2hExt1Ulid], block3hto4hExt1Ulid: blocks[block3hto4hExt1Ulid], block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block2hto3hExt1Ulid: blocks[block2hto3hExt1Ulid], block1hto2hExt1UlidCopy: blocks[block1hto2hExt1UlidCopy]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid, block1hto2hExt1UlidCopy}, - {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 1 `, }, "test overlapping blocks": { @@ -193,6 +195,59 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { 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 0 +`, + }, + "test group with all blocks locked": { + 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{ + {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, + }, + lockedBlocks: []struct { + id ulid.ULID + isExpired bool + }{ + {id: block1hto2hExt2Ulid, isExpired: false}, + {id: block0hto1hExt2Ulid, 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 locked": { + 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{ + {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, + }, + lockedBlocks: []struct { + id ulid.ULID + isExpired bool + }{ + {id: block1hto2hExt2Ulid, 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 lock file expired": { + 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}, + }, + lockedBlocks: []struct { + id ulid.ULID + isExpired bool + }{ + {id: block1hto2hExt2Ulid, isExpired: true}, + {id: block0hto1hExt2Ulid, 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 `, }, } @@ -226,8 +281,24 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { Help: "Total number of plans that remain to be compacted.", }) - g := NewShuffleShardingGrouper(nil, + bkt := &bucket.ClientMock{} + for _, lockedBlock := range testData.lockedBlocks { + lockFile := path.Join(lockedBlock.id.String(), BlockLockFile) + expireTime := time.Now() + if lockedBlock.isExpired { + expireTime = expireTime.Add(-1 * HeartBeatTimeout) + } + bkt.MockGet(lockFile, expireTime.Format(DefaultTimeFormat), nil) + } + bkt.MockUpload(mock.Anything, nil) + bkt.MockGet(mock.Anything, "", nil) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + g := NewShuffleShardingGrouper( + ctx, nil, + bkt, false, // Do not accept malformed indexes true, // Enable vertical compaction registerer, @@ -242,7 +313,8 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { overrides, "", 10, - 3) + 3, + 1) actual, err := g.Groups(testData.blocks) require.NoError(t, err) require.Len(t, actual, len(testData.expected)) diff --git a/pkg/storage/bucket/client_mock.go b/pkg/storage/bucket/client_mock.go index 5f6784a68c3..82b82f4afb4 100644 --- a/pkg/storage/bucket/client_mock.go +++ b/pkg/storage/bucket/client_mock.go @@ -102,6 +102,29 @@ func (m *ClientMock) MockGet(name, content string, err error) { } } +// MockGetTimes is a convenient method to mock Get() and Exists() to run x time +func (m *ClientMock) MockGetTimes(name, content string, err error, times int) { + if content != "" { + m.On("Exists", mock.Anything, name).Return(true, err).Times(times) + m.On("Attributes", mock.Anything, name).Return(objstore.ObjectAttributes{ + Size: int64(len(content)), + LastModified: time.Now(), + }, nil).Times(times) + + // Since we return an ReadCloser and it can be consumed only once, + // each time the mocked Get() is called we do create a new one, so + // that getting the same mocked object twice works as expected. + mockedGet := m.On("Get", mock.Anything, name).Times(times) + mockedGet.Run(func(args mock.Arguments) { + mockedGet.Return(ioutil.NopCloser(bytes.NewReader([]byte(content))), err) + }) + } else { + m.On("Exists", mock.Anything, name).Return(false, err).Times(times) + m.On("Get", mock.Anything, name).Return(nil, errObjectDoesNotExist).Times(times) + m.On("Attributes", mock.Anything, name).Return(nil, errObjectDoesNotExist).Times(times) + } +} + func (m *ClientMock) MockDelete(name string, err error) { m.On("Delete", mock.Anything, name).Return(err) } From 41e1f0842523a4b50617f2ed6986113477022126 Mon Sep 17 00:00:00 2001 From: Mengmeng Yang Date: Thu, 21 Jul 2022 22:17:28 -0700 Subject: [PATCH 02/17] let redis cache logs log with context (#4785) * let redis cache logs log with context Signed-off-by: Mengmeng Yang * fix import Signed-off-by: Mengmeng Yang Signed-off-by: Alex Le --- pkg/chunk/cache/redis_cache.go | 4 ++-- pkg/chunk/cache/snappy.go | 4 +++- pkg/querier/queryrange/results_cache.go | 14 +++++++------- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/pkg/chunk/cache/redis_cache.go b/pkg/chunk/cache/redis_cache.go index 2614cf803d8..2d20f759b4e 100644 --- a/pkg/chunk/cache/redis_cache.go +++ b/pkg/chunk/cache/redis_cache.go @@ -69,7 +69,7 @@ func (c *RedisCache) Fetch(ctx context.Context, keys []string) (found []string, items, err = c.redis.MGet(ctx, keys) if err != nil { log.Error(err) - level.Error(c.logger).Log("msg", "failed to get from redis", "name", c.name, "err", err) + level.Error(util_log.WithContext(ctx, c.logger)).Log("msg", "failed to get from redis", "name", c.name, "err", err) return err } @@ -97,7 +97,7 @@ func (c *RedisCache) Fetch(ctx context.Context, keys []string) (found []string, func (c *RedisCache) Store(ctx context.Context, keys []string, bufs [][]byte) { err := c.redis.MSet(ctx, keys, bufs) if err != nil { - level.Error(c.logger).Log("msg", "failed to put to redis", "name", c.name, "err", err) + level.Error(util_log.WithContext(ctx, c.logger)).Log("msg", "failed to put to redis", "name", c.name, "err", err) } } diff --git a/pkg/chunk/cache/snappy.go b/pkg/chunk/cache/snappy.go index b19b6dc4a0b..eb0bd908c13 100644 --- a/pkg/chunk/cache/snappy.go +++ b/pkg/chunk/cache/snappy.go @@ -6,6 +6,8 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/golang/snappy" + + util_log "github.com/cortexproject/cortex/pkg/util/log" ) type snappyCache struct { @@ -36,7 +38,7 @@ func (s *snappyCache) Fetch(ctx context.Context, keys []string) ([]string, [][]b for _, buf := range bufs { d, err := snappy.Decode(nil, buf) if err != nil { - level.Error(s.logger).Log("msg", "failed to decode cache entry", "err", err) + level.Error(util_log.WithContext(ctx, s.logger)).Log("msg", "failed to decode cache entry", "err", err) return nil, nil, keys } ds = append(ds, d) diff --git a/pkg/querier/queryrange/results_cache.go b/pkg/querier/queryrange/results_cache.go index 3096300bf7d..b34dd386ad1 100644 --- a/pkg/querier/queryrange/results_cache.go +++ b/pkg/querier/queryrange/results_cache.go @@ -272,12 +272,12 @@ func (s resultsCache) shouldCacheResponse(ctx context.Context, req Request, r Re headerValues := getHeaderValuesWithName(r, cacheControlHeader) for _, v := range headerValues { if v == noStoreValue { - level.Debug(s.logger).Log("msg", fmt.Sprintf("%s header in response is equal to %s, not caching the response", cacheControlHeader, noStoreValue)) + level.Debug(util_log.WithContext(ctx, s.logger)).Log("msg", fmt.Sprintf("%s header in response is equal to %s, not caching the response", cacheControlHeader, noStoreValue)) return false } } - if !s.isAtModifierCachable(req, maxCacheTime) { + if !s.isAtModifierCachable(ctx, req, maxCacheTime) { return false } @@ -289,13 +289,13 @@ func (s resultsCache) shouldCacheResponse(ctx context.Context, req Request, r Re genNumberFromCtx := cache.ExtractCacheGenNumber(ctx) if len(genNumbersFromResp) == 0 && genNumberFromCtx != "" { - level.Debug(s.logger).Log("msg", fmt.Sprintf("we found results cache gen number %s set in store but none in headers", genNumberFromCtx)) + level.Debug(util_log.WithContext(ctx, s.logger)).Log("msg", fmt.Sprintf("we found results cache gen number %s set in store but none in headers", genNumberFromCtx)) return false } for _, gen := range genNumbersFromResp { if gen != genNumberFromCtx { - level.Debug(s.logger).Log("msg", fmt.Sprintf("inconsistency in results cache gen numbers %s (GEN-FROM-RESPONSE) != %s (GEN-FROM-STORE), not caching the response", gen, genNumberFromCtx)) + level.Debug(util_log.WithContext(ctx, s.logger)).Log("msg", fmt.Sprintf("inconsistency in results cache gen numbers %s (GEN-FROM-RESPONSE) != %s (GEN-FROM-STORE), not caching the response", gen, genNumberFromCtx)) return false } } @@ -307,7 +307,7 @@ var errAtModifierAfterEnd = errors.New("at modifier after end") // isAtModifierCachable returns true if the @ modifier result // is safe to cache. -func (s resultsCache) isAtModifierCachable(r Request, maxCacheTime int64) bool { +func (s resultsCache) isAtModifierCachable(ctx context.Context, r Request, maxCacheTime int64) bool { // There are 2 cases when @ modifier is not safe to cache: // 1. When @ modifier points to time beyond the maxCacheTime. // 2. If the @ modifier time is > the query range end while being @@ -321,7 +321,7 @@ func (s resultsCache) isAtModifierCachable(r Request, maxCacheTime int64) bool { expr, err := parser.ParseExpr(query) if err != nil { // We are being pessimistic in such cases. - level.Warn(s.logger).Log("msg", "failed to parse query, considering @ modifier as not cachable", "query", query, "err", err) + level.Warn(util_log.WithContext(ctx, s.logger)).Log("msg", "failed to parse query, considering @ modifier as not cachable", "query", query, "err", err) return false } @@ -632,7 +632,7 @@ func (s resultsCache) put(ctx context.Context, key string, extents []Extent) { Extents: extents, }) if err != nil { - level.Error(s.logger).Log("msg", "error marshalling cached value", "err", err) + level.Error(util_log.WithContext(ctx, s.logger)).Log("msg", "error marshalling cached value", "err", err) return } From 20e6502b7a402629037ebaaee0ffc5563f878948 Mon Sep 17 00:00:00 2001 From: Daniel Blando Date: Tue, 26 Jul 2022 14:18:07 -0700 Subject: [PATCH 03/17] DoBatch preference to 4xx if error (#4783) * DoBatch preference to 4xx if error Signed-off-by: Daniel Blando * Fix comment Signed-off-by: Daniel Blando Signed-off-by: Alex Le --- CHANGELOG.md | 2 ++ pkg/distributor/distributor_test.go | 2 +- pkg/ring/batch.go | 21 ++++++++++++++++----- 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 98a292a883d..e46dd9069ac 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,9 +3,11 @@ ## master / unreleased * [FEATURE] Compactor: Added `-compactor.block-files-concurrency` allowing to configure number of go routines for download/upload block files during compaction. #4784 * [ENHANCEMENT] Querier/Ruler: Retry store-gateway in case of unexpected failure, instead of failing the query. #4532 +* [ENHANCEMENT] Ring: DoBatch prioritize 4xx errors when failing. #4783 * [FEATURE] Compactor: Added -compactor.blocks-fetch-concurrency` allowing to configure number of go routines for blocks during compaction. #4787 ## 1.13.0 2022-07-14 + * [CHANGE] Changed default for `-ingester.min-ready-duration` from 1 minute to 15 seconds. #4539 * [CHANGE] query-frontend: Do not print anything in the logs of `query-frontend` if a in-progress query has been canceled (context canceled) to avoid spam. #4562 * [CHANGE] Compactor block deletion mark migration, needed when upgrading from v1.7, is now disabled by default. #4597 diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index 4fc49fa6359..907e14ecf03 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -554,7 +554,7 @@ func TestPush_QuorumError(t *testing.T) { _, err := d.Push(ctx, request) status, ok := status.FromError(err) require.True(t, ok) - require.True(t, status.Code() == 429 || status.Code() == 500) + require.Equal(t, codes.Code(429), status.Code()) } // Simulating 1 error -> Should return 2xx diff --git a/pkg/ring/batch.go b/pkg/ring/batch.go index 26d4cb203a4..db3aa4800fe 100644 --- a/pkg/ring/batch.go +++ b/pkg/ring/batch.go @@ -30,19 +30,29 @@ type itemTracker struct { failed4xx atomic.Int32 failed5xx atomic.Int32 remaining atomic.Int32 - err atomic.Error + err4xx atomic.Error + err5xx atomic.Error } func (i *itemTracker) recordError(err error) int32 { - i.err.Store(err) if status, ok := status.FromError(err); ok && status.Code()/100 == 4 { + i.err4xx.Store(err) return i.failed4xx.Inc() } + i.err5xx.Store(err) return i.failed5xx.Inc() } +func (i *itemTracker) getError() error { + if i.failed5xx.Load() > i.failed4xx.Load() { + return i.err5xx.Load() + } + + return i.err4xx.Load() +} + // DoBatch request against a set of keys in the ring, handling replication and // failures. For example if we want to write N items where they may all // hit different instances, and we want them all replicated R ways with @@ -142,12 +152,13 @@ func (b *batchTracker) record(sampleTrackers []*itemTracker, err error) { errCount := sampleTrackers[i].recordError(err) // We should return an error if we reach the maxFailure (quorum) on a given error family OR // we dont have any remaining ingesters to try - // Ex: 2xx, 4xx, 5xx -> return 5xx + // Ex: 2xx, 4xx, 5xx -> return 4xx + // Ex: 2xx, 5xx, 4xx -> return 4xx // Ex: 4xx, 4xx, _ -> return 4xx // Ex: 5xx, _, 5xx -> return 5xx if errCount > int32(sampleTrackers[i].maxFailures) || sampleTrackers[i].remaining.Dec() == 0 { if b.rpcsFailed.Inc() == 1 { - b.err <- err + b.err <- sampleTrackers[i].getError() } } } else { @@ -165,7 +176,7 @@ func (b *batchTracker) record(sampleTrackers []*itemTracker, err error) { // Ex: 4xx, 5xx, 2xx if sampleTrackers[i].remaining.Dec() == 0 { if b.rpcsFailed.Inc() == 1 { - b.err <- sampleTrackers[i].err.Load() + b.err <- sampleTrackers[i].getError() } } } From 735e10d6aa0a929b1eac53d246323fe5a4b9adf3 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 27 Jul 2022 18:18:02 -0700 Subject: [PATCH 04/17] Updated CHANGELOG and ordered imports Signed-off-by: Alex Le --- CHANGELOG.md | 3 ++- pkg/compactor/shuffle_sharding_grouper_test.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e46dd9069ac..21851a0003c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,9 +2,10 @@ ## master / unreleased * [FEATURE] Compactor: Added `-compactor.block-files-concurrency` allowing to configure number of go routines for download/upload block files during compaction. #4784 +* [FEATURE] Compactor: Added -compactor.blocks-fetch-concurrency` allowing to configure number of go routines for blocks during compaction. #4787 * [ENHANCEMENT] Querier/Ruler: Retry store-gateway in case of unexpected failure, instead of failing the query. #4532 * [ENHANCEMENT] Ring: DoBatch prioritize 4xx errors when failing. #4783 -* [FEATURE] Compactor: Added -compactor.blocks-fetch-concurrency` allowing to configure number of go routines for blocks during compaction. #4787 +* [ENHANCEMENT] Compactor: Introduced lock file for blocks so blocks are under compaction will not be picked up by another compactor. #4805 ## 1.13.0 2022-07-14 diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index 6745bb8e554..559fcf424ab 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -3,7 +3,6 @@ package compactor import ( "bytes" "context" - "github.com/cortexproject/cortex/pkg/storage/bucket" "path" "testing" "time" @@ -20,6 +19,7 @@ import ( "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/cortexproject/cortex/pkg/ring" + "github.com/cortexproject/cortex/pkg/storage/bucket" "github.com/cortexproject/cortex/pkg/util/validation" ) From a74e8de9791ec59977b86da31266074a785eff5a Mon Sep 17 00:00:00 2001 From: Alex Le Date: Fri, 29 Jul 2022 14:44:35 -0700 Subject: [PATCH 05/17] Fixed lint and removed groupCallLimit Signed-off-by: Alex Le --- pkg/compactor/compactor.go | 3 +- pkg/compactor/shuffle_sharding_grouper.go | 50 ++++--------------- .../shuffle_sharding_grouper_test.go | 3 +- 3 files changed, 12 insertions(+), 44 deletions(-) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index f0e4538e956..a04e7b7dd03 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -87,8 +87,7 @@ var ( limits, userID, cfg.BlockFilesConcurrency, - cfg.BlocksFetchConcurrency, - 1) + cfg.BlocksFetchConcurrency) } DefaultBlocksCompactorFactory = func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index 8c25029b940..bfd4ba3ef8b 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -54,9 +54,6 @@ type ShuffleShardingGrouper struct { ring ring.ReadRing ringLifecyclerAddr string - - groupCallLimit int - groupCallCount int } func NewShuffleShardingGrouper( @@ -78,7 +75,6 @@ func NewShuffleShardingGrouper( userID string, blockFilesConcurrency int, blocksFetchConcurrency int, - groupCallLimit int, ) *ShuffleShardingGrouper { if logger == nil { logger = log.NewNopLogger() @@ -124,19 +120,11 @@ func NewShuffleShardingGrouper( userID: userID, blockFilesConcurrency: blockFilesConcurrency, blocksFetchConcurrency: blocksFetchConcurrency, - groupCallLimit: groupCallLimit, - groupCallCount: 0, } } // Groups function modified from https://github.com/cortexproject/cortex/pull/2616 func (g *ShuffleShardingGrouper) Groups(blocks map[ulid.ULID]*metadata.Meta) (res []*compact.Group, err error) { - g.groupCallCount++ - if g.groupCallCount > g.groupCallLimit { - level.Debug(g.logger).Log("msg", "exceed group call limit, return empty group", "groupCallLimit", g.groupCallLimit) - return []*compact.Group{}, nil - } - // First of all we have to group blocks using the Thanos default // grouping (based on downsample resolution + external labels). mainGroups := map[string][]*metadata.Meta{} @@ -167,9 +155,7 @@ func (g *ShuffleShardingGrouper) Groups(blocks map[ulid.ULID]*metadata.Meta) (re var groups []blocksGroup for _, mainBlocks := range mainGroups { - for _, group := range groupBlocksByCompactableRanges(mainBlocks, g.compactorCfg.BlockRanges.ToMilliseconds()) { - groups = append(groups, group) - } + groups = append(groups, groupBlocksByCompactableRanges(mainBlocks, g.compactorCfg.BlockRanges.ToMilliseconds())...) } // Ensure groups are sorted by smallest range, oldest min time first. The rationale @@ -275,45 +261,29 @@ mainLoop: return outGroups, nil } -// Check whether this compactor instance owns the group. -func (g *ShuffleShardingGrouper) ownGroup(groupHash uint32) (bool, error) { - subRing := g.ring.ShuffleShard(g.userID, g.limits.CompactorTenantShardSize(g.userID)) - - rs, err := subRing.Get(groupHash, RingOp, nil, nil, nil) - if err != nil { - return false, err - } - - if len(rs.Instances) != 1 { - return false, fmt.Errorf("unexpected number of compactors in the shard (expected 1, got %d)", len(rs.Instances)) - } - - return rs.Instances[0].Addr == g.ringLifecyclerAddr, nil -} - func (g *ShuffleShardingGrouper) isGroupLocked(blocks []*metadata.Meta) (bool, error) { for _, block := range blocks { - blockId := block.ULID.String() - lockFileReader, err := g.bkt.Get(g.ctx, path.Join(blockId, BlockLockFile)) + blockID := block.ULID.String() + lockFileReader, err := g.bkt.Get(g.ctx, path.Join(blockID, BlockLockFile)) if err != nil { if g.bkt.IsObjNotFoundErr(err) { - level.Debug(g.logger).Log("msg", fmt.Sprintf("no lock file for block: %s", blockId)) + level.Debug(g.logger).Log("msg", fmt.Sprintf("no lock file for block: %s", blockID)) continue } return true, err } b, err := ioutil.ReadAll(lockFileReader) if err != nil { - level.Error(g.logger).Log("msg", fmt.Sprintf("unable to reach lock file for block: %s", blockId), "err", err) + level.Error(g.logger).Log("msg", fmt.Sprintf("unable to reach lock file for block: %s", blockID), "err", err) return true, err } heartBeatTime, err := time.Parse(DefaultTimeFormat, string(b)) if err != nil { - level.Error(g.logger).Log("msg", fmt.Sprintf("unable to parse timestamp in lock file for block: %s", blockId), "err", err) + level.Error(g.logger).Log("msg", fmt.Sprintf("unable to parse timestamp in lock file for block: %s", blockID), "err", err) return true, err } if time.Now().Before(heartBeatTime.Add(HeartBeatTimeout)) { - level.Debug(g.logger).Log("msg", fmt.Sprintf("locked block: %s", blockId)) + level.Debug(g.logger).Log("msg", fmt.Sprintf("locked block: %s", blockID)) return true, nil } } @@ -335,11 +305,11 @@ heartBeat: default: level.Debug(g.logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) for _, block := range blocks { - blockId := block.ULID.String() - blockLockFilePath := path.Join(blockId, BlockLockFile) + blockID := block.ULID.String() + blockLockFilePath := path.Join(blockID, BlockLockFile) err := g.bkt.Upload(g.ctx, blockLockFilePath, bytes.NewReader([]byte(time.Now().Format(DefaultTimeFormat)))) if err != nil { - level.Error(g.logger).Log("msg", "unable to update heart beat for block", "block_id", blockId, "err", err) + level.Error(g.logger).Log("msg", "unable to update heart beat for block", "block_id", blockID, "err", err) } } time.Sleep(HeartBeatTimeout / 5) // it allows up to 5 failures on heart heat for single block diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index 559fcf424ab..b3680957732 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -313,8 +313,7 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { overrides, "", 10, - 3, - 1) + 3) actual, err := g.Groups(testData.blocks) require.NoError(t, err) require.Len(t, actual, len(testData.expected)) From 91e381a17ccb46c250ff00415b9c793cc4d9161f Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 2 Aug 2022 10:16:35 -0700 Subject: [PATCH 06/17] Changed lock file to json format and make sure planner would not pick up group that is locked by other compactor Signed-off-by: Alex Le --- pkg/compactor/compactor.go | 12 +- pkg/compactor/compactor_test.go | 9 +- pkg/compactor/shuffle_sharding_grouper.go | 47 ++++++-- .../shuffle_sharding_grouper_test.go | 9 +- pkg/compactor/shuffle_sharding_planner.go | 44 +++++++- .../shuffle_sharding_planner_test.go | 106 +++++++++++++++++- 6 files changed, 205 insertions(+), 22 deletions(-) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index a04e7b7dd03..31b92bb18d8 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -84,6 +84,7 @@ var ( cfg, ring, ringLifecycle.Addr, + ringLifecycle.ID, limits, userID, cfg.BlockFilesConcurrency, @@ -96,7 +97,7 @@ var ( return nil, nil, err } - plannerFactory := func(logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler) compact.Planner { return compact.NewPlanner(logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter) } @@ -109,9 +110,9 @@ var ( return nil, nil, err } - plannerFactory := func(logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler) compact.Planner { - return NewShuffleShardingPlanner(logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks) + return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID) } return compactor, plannerFactory, nil } @@ -143,9 +144,12 @@ type BlocksCompactorFactory func( ) (compact.Compactor, PlannerFactory, error) type PlannerFactory func( + ctx context.Context, + bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, + ringLifecycle *ring.Lifecycler, ) compact.Planner // Limits defines limits used by the Compactor. @@ -767,7 +771,7 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { ulogger, syncer, c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.ring, c.ringLifecycler, c.limits, userID), - c.blocksPlannerFactory(ulogger, c.compactorCfg, noCompactMarkerFilter), + c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler), c.blocksCompactor, path.Join(c.compactorCfg.DataDir, "compact"), bucket, diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 80ac1a93027..c8d72041dba 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1207,11 +1207,16 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit blockDirectory := []string{} for blockID, blockTimes := range blocks { + blockLocker := BlockLocker{ + CompactorID: "test-compactor", + LockTime: time.Now(), + } + lockFileContent, _ := json.Marshal(blockLocker) 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.MockGetTimes(userID+"/"+blockID+"/block.lock", "", nil, 1) - bucketClient.MockGet(userID+"/"+blockID+"/block.lock", time.Now().Format(DefaultTimeFormat), nil) + bucketClient.MockGet(userID+"/"+blockID+"/block.lock", string(lockFileContent), nil) bucketClient.MockUpload(userID+"/"+blockID+"/block.lock", nil) blockDirectory = append(blockDirectory, userID+"/"+blockID) @@ -1534,7 +1539,7 @@ func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, li blocksCompactorFactory := func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { return tsdbCompactor, - func(_ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter) compact.Planner { + func(ctx context.Context, bkt objstore.Bucket, _ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler) compact.Planner { tsdbPlanner.noCompactMarkFilters = append(tsdbPlanner.noCompactMarkFilters, noCompactMarkFilter) return tsdbPlanner }, diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index bfd4ba3ef8b..7478f379a8d 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -3,6 +3,7 @@ package compactor import ( "bytes" "context" + "encoding/json" "fmt" "hash/fnv" "io/ioutil" @@ -54,6 +55,7 @@ type ShuffleShardingGrouper struct { ring ring.ReadRing ringLifecyclerAddr string + ringLifecyclerID string } func NewShuffleShardingGrouper( @@ -71,6 +73,7 @@ func NewShuffleShardingGrouper( compactorCfg Config, ring ring.ReadRing, ringLifecyclerAddr string, + ringLifecyclerID string, limits Limits, userID string, blockFilesConcurrency int, @@ -116,6 +119,7 @@ func NewShuffleShardingGrouper( compactorCfg: compactorCfg, ring: ring, ringLifecyclerAddr: ringLifecyclerAddr, + ringLifecyclerID: ringLifecyclerID, limits: limits, userID: userID, blockFilesConcurrency: blockFilesConcurrency, @@ -215,6 +219,7 @@ mainLoop: groupKey := createGroupKey(groupHash, group) level.Info(g.logger).Log("msg", "found compactable group for user", "group_hash", groupHash, "group", group.String()) + g.lockBlocks(group.blocks) go g.groupLockHeartBeat(group.blocks) // All the blocks within the same group have the same downsample @@ -274,15 +279,16 @@ func (g *ShuffleShardingGrouper) isGroupLocked(blocks []*metadata.Meta) (bool, e } b, err := ioutil.ReadAll(lockFileReader) if err != nil { - level.Error(g.logger).Log("msg", fmt.Sprintf("unable to reach lock file for block: %s", blockID), "err", err) + level.Error(g.logger).Log("msg", fmt.Sprintf("unable to read lock file for block: %s", blockID), "err", err) return true, err } - heartBeatTime, err := time.Parse(DefaultTimeFormat, string(b)) + blockLocker := BlockLocker{} + err = json.Unmarshal(b, &blockLocker) if err != nil { - level.Error(g.logger).Log("msg", fmt.Sprintf("unable to parse timestamp in lock file for block: %s", blockID), "err", err) + level.Error(g.logger).Log("msg", fmt.Sprintf("unable to parse lock file for block: %s", blockID), "err", err) return true, err } - if time.Now().Before(heartBeatTime.Add(HeartBeatTimeout)) { + if time.Now().Before(blockLocker.LockTime.Add(HeartBeatTimeout)) { level.Debug(g.logger).Log("msg", fmt.Sprintf("locked block: %s", blockID)) return true, nil } @@ -304,20 +310,32 @@ heartBeat: break heartBeat default: level.Debug(g.logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) - for _, block := range blocks { - blockID := block.ULID.String() - blockLockFilePath := path.Join(blockID, BlockLockFile) - err := g.bkt.Upload(g.ctx, blockLockFilePath, bytes.NewReader([]byte(time.Now().Format(DefaultTimeFormat)))) - if err != nil { - level.Error(g.logger).Log("msg", "unable to update heart beat for block", "block_id", blockID, "err", err) - } - } + g.lockBlocks(blocks) time.Sleep(HeartBeatTimeout / 5) // it allows up to 5 failures on heart heat for single block } } level.Info(g.logger).Log("msg", fmt.Sprintf("stop heart beat for blocks: %s", blocksInfo)) } +func (g *ShuffleShardingGrouper) lockBlocks(blocks []*metadata.Meta) { + for _, block := range blocks { + blockID := block.ULID.String() + blockLockFilePath := path.Join(blockID, BlockLockFile) + blockLocker := BlockLocker{ + CompactorID: g.ringLifecyclerID, + LockTime: time.Now(), + } + lockFileContent, err := json.Marshal(blockLocker) + if err != nil { + level.Error(g.logger).Log("msg", "unable to create lock file content for block", "block_id", blockID, "err", err) + } + err = g.bkt.Upload(g.ctx, blockLockFilePath, bytes.NewReader(lockFileContent)) + if err != nil { + level.Error(g.logger).Log("msg", "unable to update heart beat for block", "block_id", blockID, "err", err) + } + } +} + // Check whether this compactor exists on the subring based on user ID func (g *ShuffleShardingGrouper) checkSubringForCompactor() (bool, error) { subRing := g.ring.ShuffleShard(g.userID, g.limits.CompactorTenantShardSize(g.userID)) @@ -545,3 +563,8 @@ func sortMetasByMinTime(metas []*metadata.Meta) { return metas[i].BlockMeta.MinTime < metas[j].BlockMeta.MinTime }) } + +type BlockLocker struct { + CompactorID string `json:"compactorID"` + LockTime time.Time `json:"lockTime"` +} diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index b3680957732..74c45c01b5b 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -3,6 +3,7 @@ package compactor import ( "bytes" "context" + "encoding/json" "path" "testing" "time" @@ -288,7 +289,12 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { if lockedBlock.isExpired { expireTime = expireTime.Add(-1 * HeartBeatTimeout) } - bkt.MockGet(lockFile, expireTime.Format(DefaultTimeFormat), nil) + blockLocker := BlockLocker{ + CompactorID: "test-compactor", + LockTime: expireTime, + } + lockFileContent, _ := json.Marshal(blockLocker) + bkt.MockGet(lockFile, string(lockFileContent), nil) } bkt.MockUpload(mock.Anything, nil) bkt.MockGet(mock.Anything, "", nil) @@ -310,6 +316,7 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { *compactorCfg, ring, "test-addr", + "test-compactor", overrides, "", 10, diff --git a/pkg/compactor/shuffle_sharding_planner.go b/pkg/compactor/shuffle_sharding_planner.go index 4c38ff59823..b328676ac65 100644 --- a/pkg/compactor/shuffle_sharding_planner.go +++ b/pkg/compactor/shuffle_sharding_planner.go @@ -2,24 +2,42 @@ package compactor import ( "context" + "encoding/json" "fmt" + "io/ioutil" + "path" + "time" "github.com/go-kit/log" "github.com/oklog/ulid" "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/objstore" ) type ShuffleShardingPlanner struct { + ctx context.Context + bkt objstore.Bucket logger log.Logger ranges []int64 noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark + ringLifecyclerID string } -func NewShuffleShardingPlanner(logger log.Logger, ranges []int64, noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark) *ShuffleShardingPlanner { +func NewShuffleShardingPlanner( + ctx context.Context, + bkt objstore.Bucket, + logger log.Logger, + ranges []int64, + noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark, + ringLifecyclerID string, +) *ShuffleShardingPlanner { return &ShuffleShardingPlanner{ + ctx: ctx, + bkt: bkt, logger: logger, ranges: ranges, noCompBlocksFunc: noCompBlocksFunc, + ringLifecyclerID: ringLifecyclerID, } } @@ -35,12 +53,34 @@ func (p *ShuffleShardingPlanner) Plan(_ context.Context, metasByMinTime []*metad resultMetas := make([]*metadata.Meta, 0, len(metasByMinTime)) for _, b := range metasByMinTime { + blockID := b.ULID.String() if _, excluded := noCompactMarked[b.ULID]; excluded { continue } if b.MinTime < rangeStart || b.MaxTime > rangeEnd { - return nil, fmt.Errorf("block %s with time range %d:%d is outside the largest expected range %d:%d", b.ULID.String(), b.MinTime, b.MaxTime, rangeStart, rangeEnd) + 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) + } + + lockFileReader, err := p.bkt.Get(p.ctx, path.Join(blockID, BlockLockFile)) + if err != nil { + if p.bkt.IsObjNotFoundErr(err) { + resultMetas = append(resultMetas, b) + continue + } + return nil, fmt.Errorf("unable to get lock file for block %s: %s", blockID, err.Error()) + } + bytes, err := ioutil.ReadAll(lockFileReader) + if err != nil { + return nil, fmt.Errorf("unable to read lock file for block %s: %s", blockID, err.Error()) + } + blockLocker := BlockLocker{} + err = json.Unmarshal(bytes, &blockLocker) + if err != nil { + return nil, fmt.Errorf("unable to parse lock file for block %s: %s", blockID, err.Error()) + } + if time.Now().Before(blockLocker.LockTime.Add(HeartBeatTimeout)) && blockLocker.CompactorID != p.ringLifecyclerID { + return nil, fmt.Errorf("block %s is locked for compactor %s. but current compactor is %s", blockID, blockLocker.CompactorID, p.ringLifecyclerID) } resultMetas = append(resultMetas, b) diff --git a/pkg/compactor/shuffle_sharding_planner_test.go b/pkg/compactor/shuffle_sharding_planner_test.go index 8dbf1c9de44..904dfcb9f8b 100644 --- a/pkg/compactor/shuffle_sharding_planner_test.go +++ b/pkg/compactor/shuffle_sharding_planner_test.go @@ -2,18 +2,32 @@ package compactor import ( "context" + "encoding/json" "fmt" + "path" "testing" "time" "github.com/oklog/ulid" "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/thanos-io/thanos/pkg/block/metadata" + + "github.com/cortexproject/cortex/pkg/storage/bucket" ) func TestShuffleShardingPlanner_Plan(t *testing.T) { + type LockedBlock struct { + id ulid.ULID + isExpired bool + compactorID string + } + + currentCompactor := "test-compactor" + otherCompactor := "other-compactor" + block1ulid := ulid.MustNew(1, nil) block2ulid := ulid.MustNew(2, nil) block3ulid := ulid.MustNew(3, nil) @@ -24,6 +38,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { blocks []*metadata.Meta expected []*metadata.Meta expectedErr error + lockedBlocks []LockedBlock }{ "test basic plan": { ranges: []int64{2 * time.Hour.Milliseconds()}, @@ -184,15 +199,104 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, expected: []*metadata.Meta{}, }, + "test should not compact if lock file is not expired and locked by other compactor": { + ranges: []int64{2 * time.Hour.Milliseconds()}, + blocks: []*metadata.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + ULID: block1ulid, + MinTime: 1 * time.Hour.Milliseconds(), + MaxTime: 2 * time.Hour.Milliseconds(), + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + ULID: block2ulid, + MinTime: 1 * time.Hour.Milliseconds(), + MaxTime: 2 * time.Hour.Milliseconds(), + }, + }, + }, + lockedBlocks: []LockedBlock{ + { + id: block1ulid, + isExpired: false, + compactorID: otherCompactor, + }, + }, + expectedErr: fmt.Errorf("block %s is locked for compactor %s. but current compactor is %s", block1ulid.String(), otherCompactor, currentCompactor), + }, + "test should compact if lock file is expired and was locked by other compactor": { + ranges: []int64{2 * time.Hour.Milliseconds()}, + blocks: []*metadata.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + ULID: block1ulid, + MinTime: 1 * time.Hour.Milliseconds(), + MaxTime: 2 * time.Hour.Milliseconds(), + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + ULID: block2ulid, + MinTime: 1 * time.Hour.Milliseconds(), + MaxTime: 2 * time.Hour.Milliseconds(), + }, + }, + }, + expected: []*metadata.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + ULID: block1ulid, + MinTime: 1 * time.Hour.Milliseconds(), + MaxTime: 2 * time.Hour.Milliseconds(), + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + ULID: block2ulid, + MinTime: 1 * time.Hour.Milliseconds(), + MaxTime: 2 * time.Hour.Milliseconds(), + }, + }, + }, + lockedBlocks: []LockedBlock{ + { + id: block1ulid, + isExpired: true, + compactorID: otherCompactor, + }, + }, + }, } for testName, testData := range tests { t.Run(testName, func(t *testing.T) { - p := NewShuffleShardingPlanner(nil, + bkt := &bucket.ClientMock{} + for _, lockedBlock := range testData.lockedBlocks { + lockFile := path.Join(lockedBlock.id.String(), BlockLockFile) + expireTime := time.Now() + if lockedBlock.isExpired { + expireTime = expireTime.Add(-1 * HeartBeatTimeout) + } + blockLocker := BlockLocker{ + CompactorID: lockedBlock.compactorID, + LockTime: expireTime, + } + lockFileContent, _ := json.Marshal(blockLocker) + bkt.MockGet(lockFile, string(lockFileContent), nil) + } + bkt.MockGet(mock.Anything, "", nil) + + p := NewShuffleShardingPlanner( + context.Background(), + bkt, + nil, testData.ranges, func() map[ulid.ULID]*metadata.NoCompactMark { return testData.noCompactBlocks }, + currentCompactor, ) actual, err := p.Plan(context.Background(), testData.blocks) From f45290c83635c2b5124a68f1b4bbfb0e8b81ae95 Mon Sep 17 00:00:00 2001 From: Alan Protasio Date: Wed, 27 Jul 2022 20:53:10 -0700 Subject: [PATCH 07/17] Fix updateCachedShippedBlocks - new thanos (#4806) Signed-off-by: Alan Protasio Signed-off-by: Alex Le --- pkg/ingester/ingester_v2.go | 2 +- pkg/ingester/ingester_v2_test.go | 39 ++++++++++++++++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/pkg/ingester/ingester_v2.go b/pkg/ingester/ingester_v2.go index 653126a649d..b35411b72d7 100644 --- a/pkg/ingester/ingester_v2.go +++ b/pkg/ingester/ingester_v2.go @@ -310,7 +310,7 @@ func (u *userTSDB) blocksToDelete(blocks []*tsdb.Block) map[ulid.ULID]struct{} { // updateCachedShipperBlocks reads the shipper meta file and updates the cached shipped blocks. func (u *userTSDB) updateCachedShippedBlocks() error { shipperMeta, err := shipper.ReadMetaFile(u.db.Dir()) - if os.IsNotExist(err) { + if os.IsNotExist(err) || os.IsNotExist(errors.Cause(err)) { // If the meta file doesn't exist it means the shipper hasn't run yet. shipperMeta = &shipper.Meta{} } else if err != nil { diff --git a/pkg/ingester/ingester_v2_test.go b/pkg/ingester/ingester_v2_test.go index 7e48abf9f65..e83d5a10996 100644 --- a/pkg/ingester/ingester_v2_test.go +++ b/pkg/ingester/ingester_v2_test.go @@ -2475,6 +2475,45 @@ func TestIngester_seriesCountIsCorrectAfterClosingTSDBForDeletedTenant(t *testin require.Equal(t, int64(0), i.TSDBState.seriesCount.Load()) } +func TestIngester_sholdUpdateCacheShippedBlocks(t *testing.T) { + ctx := context.Background() + cfg := defaultIngesterTestConfig(t) + cfg.LifecyclerConfig.JoinAfter = 0 + cfg.BlocksStorageConfig.TSDB.ShipConcurrency = 2 + + // Create ingester + i, err := prepareIngesterWithBlocksStorage(t, cfg, nil) + require.NoError(t, err) + + require.NoError(t, services.StartAndAwaitRunning(ctx, i)) + defer services.StopAndAwaitTerminated(ctx, i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + mockUserShipper(t, i) + + // Mock the shipper meta (no blocks). + db := i.getTSDB(userID) + err = db.updateCachedShippedBlocks() + require.NoError(t, err) + + require.Equal(t, len(db.getCachedShippedBlocks()), 0) + shippedBlock, _ := ulid.Parse("01D78XZ44G0000000000000000") + + require.NoError(t, shipper.WriteMetaFile(log.NewNopLogger(), db.db.Dir(), &shipper.Meta{ + Version: shipper.MetaVersion1, + Uploaded: []ulid.ULID{shippedBlock}, + })) + + err = db.updateCachedShippedBlocks() + require.NoError(t, err) + + require.Equal(t, len(db.getCachedShippedBlocks()), 1) +} + func TestIngester_closeAndDeleteUserTSDBIfIdle_shouldNotCloseTSDBIfShippingIsInProgress(t *testing.T) { ctx := context.Background() cfg := defaultIngesterTestConfig(t) From 9c3efbc4973f3168569369351f3833430ffad867 Mon Sep 17 00:00:00 2001 From: Daniel Blando Date: Thu, 28 Jul 2022 09:13:08 -0700 Subject: [PATCH 08/17] Join memberlist on starting with no retry (#4804) Signed-off-by: Daniel Blando --- CHANGELOG.md | 1 + pkg/ring/kv/memberlist/memberlist_client.go | 25 ++++++++++++-- .../kv/memberlist/memberlist_client_test.go | 33 +++++++++++++++++++ 3 files changed, 56 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 21851a0003c..3e6b851aa74 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ * [ENHANCEMENT] Querier/Ruler: Retry store-gateway in case of unexpected failure, instead of failing the query. #4532 * [ENHANCEMENT] Ring: DoBatch prioritize 4xx errors when failing. #4783 * [ENHANCEMENT] Compactor: Introduced lock file for blocks so blocks are under compaction will not be picked up by another compactor. #4805 +* [BUGFIX] Memberlist: Add join with no retrying when starting service. #4804 ## 1.13.0 2022-07-14 diff --git a/pkg/ring/kv/memberlist/memberlist_client.go b/pkg/ring/kv/memberlist/memberlist_client.go index 66ffa731fe0..db38617abef 100644 --- a/pkg/ring/kv/memberlist/memberlist_client.go +++ b/pkg/ring/kv/memberlist/memberlist_client.go @@ -411,7 +411,7 @@ func (m *KV) buildMemberlistConfig() (*memberlist.Config, error) { return mlCfg, nil } -func (m *KV) starting(_ context.Context) error { +func (m *KV) starting(ctx context.Context) error { mlCfg, err := m.buildMemberlistConfig() if err != nil { return err @@ -438,6 +438,15 @@ func (m *KV) starting(_ context.Context) error { } m.initWG.Done() + if len(m.cfg.JoinMembers) > 0 { + // Lookup SRV records for given addresses to discover members. + members := m.discoverMembers(ctx, m.cfg.JoinMembers) + + err := m.joinMembersOnStarting(members) + if err != nil { + level.Warn(m.logger).Log("msg", "failed to join memberlist cluster on startup", "err", err) + } + } return nil } @@ -450,7 +459,7 @@ func (m *KV) running(ctx context.Context) error { // Lookup SRV records for given addresses to discover members. members := m.discoverMembers(ctx, m.cfg.JoinMembers) - err := m.joinMembersOnStartup(ctx, members) + err := m.joinMembersOnRunning(ctx, members) if err != nil { level.Error(m.logger).Log("msg", "failed to join memberlist cluster", "err", err) @@ -517,7 +526,7 @@ func (m *KV) JoinMembers(members []string) (int, error) { return m.memberlist.Join(members) } -func (m *KV) joinMembersOnStartup(ctx context.Context, members []string) error { +func (m *KV) joinMembersOnRunning(ctx context.Context, members []string) error { reached, err := m.memberlist.Join(members) if err == nil { level.Info(m.logger).Log("msg", "joined memberlist cluster", "reached_nodes", reached) @@ -556,6 +565,16 @@ func (m *KV) joinMembersOnStartup(ctx context.Context, members []string) error { return lastErr } +func (m *KV) joinMembersOnStarting(members []string) error { + reached, err := m.memberlist.Join(members) + if err == nil { + level.Info(m.logger).Log("msg", "joined memberlist cluster", "reached_nodes", reached) + return nil + } + + return err +} + // Provides a dns-based member disovery to join a memberlist cluster w/o knowning members' addresses upfront. func (m *KV) discoverMembers(ctx context.Context, members []string) []string { if len(members) == 0 { diff --git a/pkg/ring/kv/memberlist/memberlist_client_test.go b/pkg/ring/kv/memberlist/memberlist_client_test.go index 758ee49dc3d..0a66b52c7b0 100644 --- a/pkg/ring/kv/memberlist/memberlist_client_test.go +++ b/pkg/ring/kv/memberlist/memberlist_client_test.go @@ -789,6 +789,39 @@ func TestMemberlistFailsToJoin(t *testing.T) { require.Equal(t, mkv.FailureCase(), errFailedToJoinCluster) } +func TestMemberlistJoinOnStarting(t *testing.T) { + ports, err := getFreePorts(2) + require.NoError(t, err) + + var cfg1 KVConfig + flagext.DefaultValues(&cfg1) + cfg1.TCPTransport = TCPTransportConfig{ + BindAddrs: []string{"localhost"}, + BindPort: ports[0], + } + + cfg1.RandomizeNodeName = true + cfg1.Codecs = []codec.Codec{dataCodec{}} + cfg1.AbortIfJoinFails = false + + cfg2 := cfg1 + cfg2.TCPTransport.BindPort = ports[1] + cfg2.JoinMembers = []string{fmt.Sprintf("localhost:%d", ports[0])} + cfg2.RejoinInterval = 1 * time.Second + + mkv1 := NewKV(cfg1, log.NewNopLogger(), &dnsProviderMock{}, prometheus.NewPedanticRegistry()) + require.NoError(t, mkv1.starting(context.Background())) + + mkv2 := NewKV(cfg2, log.NewNopLogger(), &dnsProviderMock{}, prometheus.NewPedanticRegistry()) + require.NoError(t, mkv2.starting(context.Background())) + + membersFunc := func() interface{} { + return mkv2.memberlist.NumMembers() + } + + poll(t, 5*time.Second, 2, membersFunc) +} + func getFreePorts(count int) ([]int, error) { var ports []int for i := 0; i < count; i++ { From 275881a9ee69b751ee5e02558a00ec59345508ab Mon Sep 17 00:00:00 2001 From: Xiaochao Dong Date: Fri, 29 Jul 2022 09:25:38 +0800 Subject: [PATCH 09/17] Fix alertmanager log message (#4801) Signed-off-by: Xiaochao Dong (@damnever) Signed-off-by: Alex Le --- pkg/alertmanager/multitenant.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/alertmanager/multitenant.go b/pkg/alertmanager/multitenant.go index a02aa4a5c1a..5d6dbe7880c 100644 --- a/pkg/alertmanager/multitenant.go +++ b/pkg/alertmanager/multitenant.go @@ -477,7 +477,7 @@ func (am *MultitenantAlertmanager) starting(ctx context.Context) (err error) { am.subservicesWatcher.WatchManager(am.subservices) // We wait until the instance is in the JOINING state, once it does we know that tokens are assigned to this instance and we'll be ready to perform an initial sync of configs. - level.Info(am.logger).Log("waiting until alertmanager is JOINING in the ring") + level.Info(am.logger).Log("msg", "waiting until alertmanager is JOINING in the ring") if err = ring.WaitInstanceState(ctx, am.ring, am.ringLifecycler.GetInstanceID(), ring.JOINING); err != nil { return err } From d6d93498aa1fb64ed92cc2d97189d42ed039adc3 Mon Sep 17 00:00:00 2001 From: Alvin Lin Date: Sun, 31 Jul 2022 22:48:37 -0700 Subject: [PATCH 10/17] Grafana Cloud uses Mimir now, so remove Grafana Cloud as hosted service in documents (#4809) * Grafana Cloud uses Mimir, for of Cortex, now Signed-off-by: Alvin Lin * Improve doc Signed-off-by: Alvin Lin Signed-off-by: Alex Le --- README.md | 9 +++------ docs/_index.md | 9 ++------- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/README.md b/README.md index 3e5e664419c..dd6e6436a52 100644 --- a/README.md +++ b/README.md @@ -162,12 +162,9 @@ in the [Weave Cloud documentation](https://www.weave.works/docs/cloud/latest/ove [Instrumenting Your App: Best Practices](https://www.weave.works/docs/cloud/latest/tasks/monitor/best-instrumenting/) -### Grafana Cloud - -The Cortex project was started by Tom Wilkie (Grafana Labs' VP Product) and Julius Volz (Prometheus' co-founder) in June 2016. Employing 6 out of 8 maintainers for Cortex enables [Grafana Labs](https://grafana.com/) to offer Cortex-as-a-service with exceptional performance and reliability. As the creators of [Grafana](https://grafana.com/oss/grafana/), [Loki](https://grafana.com/oss/loki/), and [Tempo](https://grafana.com/oss/tempo/), Grafana Labs can offer you the most wholistic Observability-as-a-Service stack out there. - -For further information see Grafana Cloud [documentation](https://grafana.com/docs/grafana-cloud/), [tutorials](https://grafana.com/tutorials/), [webinars](https://grafana.com/videos/), and [KubeCon talks](https://grafana.com/categories/cortex/). Get started today and [sign up here](https://grafana.com/products/cloud/). - ### Amazon Managed Service for Prometheus (AMP) [Amazon Managed Service for Prometheus (AMP)](https://aws.amazon.com/prometheus/) is a Prometheus-compatible monitoring service that makes it easy to monitor containerized applications at scale. It is a highly available, secure, and managed monitoring for your containers. Get started [here](https://console.aws.amazon.com/prometheus/home). To learn more about the AMP, reference our [documentation](https://docs.aws.amazon.com/prometheus/latest/userguide/what-is-Amazon-Managed-Service-Prometheus.html) and [Getting Started with AMP blog](https://aws.amazon.com/blogs/mt/getting-started-amazon-managed-service-for-prometheus/). + +## History of Cortex +The Cortex project was started by Tom Wilkie (Grafana Labs' VP Product) and Julius Volz (Prometheus' co-founder) in June 2016. \ No newline at end of file diff --git a/docs/_index.md b/docs/_index.md index a11bead3d90..1ff85c2846e 100644 --- a/docs/_index.md +++ b/docs/_index.md @@ -154,13 +154,8 @@ in the [Weave Cloud documentation](https://www.weave.works/docs/cloud/latest/ove [Instrumenting Your App: Best Practices](https://www.weave.works/docs/cloud/latest/tasks/monitor/best-instrumenting/) -### Grafana Cloud - -The Cortex project was started by Tom Wilkie (Grafana Labs' VP Product) and Julius Volz (Prometheus' co-founder) in June 2016. Employing 6 out of 8 maintainers for Cortex enables [Grafana Labs](https://grafana.com/) to offer Cortex-as-a-service with exceptional performance and reliability. As the creators of [Grafana](https://grafana.com/oss/grafana/), [Loki](https://grafana.com/oss/loki/), and [Tempo](https://grafana.com/oss/tempo/), Grafana Labs can offer you the most wholistic Observability-as-a-Service stack out there. - -For further information see Grafana Cloud [documentation](https://grafana.com/docs/grafana-cloud/), [tutorials](https://grafana.com/tutorials/), [webinars](https://grafana.com/videos/), and [KubeCon talks](https://grafana.com/categories/cortex/). Get started today and [sign up here](https://grafana.com/products/cloud/). - ### Amazon Managed Service for Prometheus (AMP) [Amazon Managed Service for Prometheus (AMP)](https://aws.amazon.com/prometheus/) is a Prometheus-compatible monitoring service that makes it easy to monitor containerized applications at scale. It is a highly available, secure, and managed monitoring for your containers. Get started [here](https://console.aws.amazon.com/prometheus/home). To learn more about the AMP, reference our [documentation](https://docs.aws.amazon.com/prometheus/latest/userguide/what-is-Amazon-Managed-Service-Prometheus.html) and [Getting Started with AMP blog](https://aws.amazon.com/blogs/mt/getting-started-amazon-managed-service-for-prometheus/). - +## History of Cortex +The Cortex project was started by Tom Wilkie (Grafana Labs' VP Product) and Julius Volz (Prometheus' co-founder) in June 2016. \ No newline at end of file From 7cf4862fe4c4edb1aa2974d38e22b4cea5ff502a Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 3 Aug 2022 11:58:34 -0700 Subject: [PATCH 11/17] Created block_locker to handle all block lock file operations. Added block lock metrics. Signed-off-by: Alex Le --- pkg/compactor/block_locker.go | 77 +++++++++++++++++++ pkg/compactor/compactor.go | 40 ++++++++-- pkg/compactor/compactor_test.go | 2 +- pkg/compactor/shuffle_sharding_grouper.go | 77 +++++++------------ .../shuffle_sharding_grouper_test.go | 18 ++++- pkg/compactor/shuffle_sharding_planner.go | 50 ++++++------ .../shuffle_sharding_planner_test.go | 12 ++- 7 files changed, 189 insertions(+), 87 deletions(-) create mode 100644 pkg/compactor/block_locker.go diff --git a/pkg/compactor/block_locker.go b/pkg/compactor/block_locker.go new file mode 100644 index 00000000000..455948374d1 --- /dev/null +++ b/pkg/compactor/block_locker.go @@ -0,0 +1,77 @@ +package compactor + +import ( + "bytes" + "context" + "encoding/json" + "io/ioutil" + "path" + "time" + + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/thanos/pkg/objstore" +) + +const BlockLockFile = "block.lock" + +var ( + ErrorBlockLockNotFound = errors.New("block lock not found") + ErrorUnmarshalBlockLock = errors.New("unmarshal block lock JSON") +) + +type BlockLocker struct { + CompactorID string `json:"compactorID"` + LockTime time.Time `json:"lockTime"` +} + +func (b *BlockLocker) isLocked(blockLockTimeout time.Duration) bool { + return time.Now().Before(b.LockTime.Add(blockLockTimeout)) +} + +func (b *BlockLocker) isLockedForCompactor(blockLockTimeout time.Duration, compactorID string) bool { + return time.Now().Before(b.LockTime.Add(blockLockTimeout)) && b.CompactorID != compactorID +} + +func ReadBlockLocker(ctx context.Context, bkt objstore.Bucket, blockID string, blockLockReadFailed prometheus.Counter) (*BlockLocker, error) { + lockFile := path.Join(blockID, BlockLockFile) + lockFileReader, err := bkt.Get(ctx, lockFile) + if err != nil { + if bkt.IsObjNotFoundErr(err) { + return nil, errors.Wrapf(ErrorBlockLockNotFound, "block lcok file: %s", lockFile) + } + blockLockReadFailed.Inc() + return nil, errors.Wrapf(err, "get block lock file: %s", lockFile) + } + b, err := ioutil.ReadAll(lockFileReader) + if err != nil { + blockLockReadFailed.Inc() + return nil, errors.Wrapf(err, "read block lock file: %s", lockFile) + } + blockLocker := BlockLocker{} + err = json.Unmarshal(b, &blockLocker) + if err != nil { + blockLockReadFailed.Inc() + return nil, errors.Wrapf(ErrorUnmarshalBlockLock, "block lcok file: %s, error: %v", lockFile, err.Error()) + } + return &blockLocker, nil +} + +func UpdateBlockLocker(ctx context.Context, bkt objstore.Bucket, blockID string, compactorID string, blockLockWriteFailed prometheus.Counter) error { + blockLockFilePath := path.Join(blockID, BlockLockFile) + blockLocker := BlockLocker{ + CompactorID: compactorID, + LockTime: time.Now(), + } + lockFileContent, err := json.Marshal(blockLocker) + if err != nil { + blockLockWriteFailed.Inc() + return err + } + err = bkt.Upload(ctx, blockLockFilePath, bytes.NewReader(lockFileContent)) + if err != nil { + blockLockWriteFailed.Inc() + return err + } + return nil +} diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 31b92bb18d8..903e8740b9f 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -53,7 +53,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.Bucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, _ prometheus.Gauge, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string) compact.Grouper { + DefaultBlocksGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.Bucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, _ prometheus.Gauge, _ prometheus.Counter, _ prometheus.Counter, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string) compact.Grouper { return compact.NewDefaultGrouper( logger, bkt, @@ -68,7 +68,7 @@ var ( cfg.BlocksFetchConcurrency) } - ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.Bucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string) compact.Grouper { + ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.Bucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, blockLockReadFailed prometheus.Counter, blockLockWriteFailed prometheus.Counter, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string) compact.Grouper { return NewShuffleShardingGrouper( ctx, logger, @@ -88,7 +88,11 @@ var ( limits, userID, cfg.BlockFilesConcurrency, - cfg.BlocksFetchConcurrency) + cfg.BlocksFetchConcurrency, + cfg.BlockLockTimeout, + cfg.BlockLockFileUpdateInterval, + blockLockReadFailed, + blockLockWriteFailed) } DefaultBlocksCompactorFactory = func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { @@ -97,7 +101,7 @@ var ( return nil, nil, err } - plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ prometheus.Counter) compact.Planner { return compact.NewPlanner(logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter) } @@ -110,9 +114,9 @@ var ( return nil, nil, err } - plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, blockLockReadFailed prometheus.Counter) compact.Planner { - return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID) + return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID, cfg.BlockLockTimeout, blockLockReadFailed) } return compactor, plannerFactory, nil } @@ -129,6 +133,8 @@ type BlocksGrouperFactory func( blocksMarkedForNoCompact prometheus.Counter, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, + blockLockReadFailed prometheus.Counter, + blockLockWriteFailed prometheus.Counter, ring *ring.Ring, ringLifecycler *ring.Lifecycler, limit Limits, @@ -150,6 +156,7 @@ type PlannerFactory func( cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, + blockLockReadFailed prometheus.Counter, ) compact.Planner // Limits defines limits used by the Compactor. @@ -195,6 +202,10 @@ type Config struct { // Allow downstream projects to customise the blocks compactor. BlocksGrouperFactory BlocksGrouperFactory `yaml:"-"` BlocksCompactorFactory BlocksCompactorFactory `yaml:"-"` + + // Block lock file config + BlockLockTimeout time.Duration `yaml:"block_lock_timeout"` + BlockLockFileUpdateInterval time.Duration `yaml:"block_lock_file_update_interval"` } // RegisterFlags registers the Compactor flags. @@ -228,6 +239,9 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.Var(&cfg.EnabledTenants, "compactor.enabled-tenants", "Comma separated list of tenants that can be compacted. If specified, only these tenants will be compacted by compactor, otherwise all tenants can be compacted. Subject to sharding.") f.Var(&cfg.DisabledTenants, "compactor.disabled-tenants", "Comma separated list of tenants that cannot be compacted by this compactor. If specified, and compactor would normally pick given tenant for compaction (via -compactor.enabled-tenants or sharding), it will be ignored instead.") + + f.DurationVar(&cfg.BlockLockTimeout, "compactor.block-lock-timeout", 5*time.Minute, "How long block lock file should be considered as expired and able to be picked up by compactor again.") + f.DurationVar(&cfg.BlockLockFileUpdateInterval, "compactor.block-lock-file-update-interval", 1*time.Minute, "How frequently block lock file should be updated duration compaction.") } func (cfg *Config) Validate(limits validation.Limits) error { @@ -311,6 +325,8 @@ type Compactor struct { blocksMarkedForNoCompaction prometheus.Counter garbageCollectedBlocks prometheus.Counter remainingPlannedCompactions prometheus.Gauge + blockLockReadFailed prometheus.Counter + blockLockWriteFailed prometheus.Counter // TSDB syncer metrics syncerMetrics *syncerMetrics @@ -428,6 +444,14 @@ func newCompactor( Name: "cortex_compactor_garbage_collected_blocks_total", Help: "Total number of blocks marked for deletion by compactor.", }), + blockLockReadFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_lock_read_failed", + Help: "Number of block lock file failed to be read.", + }), + blockLockWriteFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_lock_write_failed", + Help: "Number of block lock file failed to be written.", + }), remainingPlannedCompactions: remainingPlannedCompactions, limits: limits, } @@ -770,8 +794,8 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { compactor, err := compact.NewBucketCompactor( ulogger, syncer, - c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.ring, c.ringLifecycler, c.limits, userID), - c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler), + c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.blockLockReadFailed, c.blockLockWriteFailed, c.ring, c.ringLifecycler, c.limits, userID), + c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, c.blockLockReadFailed), c.blocksCompactor, path.Join(c.compactorCfg.DataDir, "compact"), bucket, diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index c8d72041dba..c13631ee23c 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1539,7 +1539,7 @@ func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, li blocksCompactorFactory := func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { return tsdbCompactor, - func(ctx context.Context, bkt objstore.Bucket, _ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler) compact.Planner { + func(ctx context.Context, bkt objstore.Bucket, _ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ prometheus.Counter) compact.Planner { tsdbPlanner.noCompactMarkFilters = append(tsdbPlanner.noCompactMarkFilters, noCompactMarkFilter) return tsdbPlanner }, diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index 7478f379a8d..6d3aa191777 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -1,13 +1,9 @@ package compactor import ( - "bytes" "context" - "encoding/json" "fmt" "hash/fnv" - "io/ioutil" - "path" "sort" "strings" "time" @@ -26,10 +22,6 @@ import ( "github.com/cortexproject/cortex/pkg/ring" ) -const BlockLockFile = "block.lock" -const DefaultTimeFormat = time.RFC3339 -const HeartBeatTimeout = 5 * time.Minute - type ShuffleShardingGrouper struct { ctx context.Context logger log.Logger @@ -56,6 +48,11 @@ type ShuffleShardingGrouper struct { ring ring.ReadRing ringLifecyclerAddr string ringLifecyclerID string + + blockLockTimeout time.Duration + blockLockFileUpdateInterval time.Duration + blockLockReadFailed prometheus.Counter + blockLockWriteFailed prometheus.Counter } func NewShuffleShardingGrouper( @@ -78,6 +75,10 @@ func NewShuffleShardingGrouper( userID string, blockFilesConcurrency int, blocksFetchConcurrency int, + blockLockTimeout time.Duration, + blockLockFileUpdateInterval time.Duration, + blockLockReadFailed prometheus.Counter, + blockLockWriteFailed prometheus.Counter, ) *ShuffleShardingGrouper { if logger == nil { logger = log.NewNopLogger() @@ -116,14 +117,18 @@ 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, + compactorCfg: compactorCfg, + ring: ring, + ringLifecyclerAddr: ringLifecyclerAddr, + ringLifecyclerID: ringLifecyclerID, + limits: limits, + userID: userID, + blockFilesConcurrency: blockFilesConcurrency, + blocksFetchConcurrency: blocksFetchConcurrency, + blockLockTimeout: blockLockTimeout, + blockLockFileUpdateInterval: blockLockFileUpdateInterval, + blockLockReadFailed: blockLockReadFailed, + blockLockWriteFailed: blockLockWriteFailed, } } @@ -269,26 +274,16 @@ mainLoop: func (g *ShuffleShardingGrouper) isGroupLocked(blocks []*metadata.Meta) (bool, error) { for _, block := range blocks { blockID := block.ULID.String() - lockFileReader, err := g.bkt.Get(g.ctx, path.Join(blockID, BlockLockFile)) + blockLocker, err := ReadBlockLocker(g.ctx, g.bkt, blockID, g.blockLockReadFailed) if err != nil { - if g.bkt.IsObjNotFoundErr(err) { - level.Debug(g.logger).Log("msg", fmt.Sprintf("no lock file for block: %s", blockID)) + if errors.Is(err, ErrorBlockLockNotFound) { + level.Debug(g.logger).Log("msg", "no lock file for block", "blockID", blockID) continue } + level.Error(g.logger).Log("msg", "unable to read block lock file", "blockID", blockID, "err", err) return true, err } - b, err := ioutil.ReadAll(lockFileReader) - if err != nil { - level.Error(g.logger).Log("msg", fmt.Sprintf("unable to read lock file for block: %s", blockID), "err", err) - return true, err - } - blockLocker := BlockLocker{} - err = json.Unmarshal(b, &blockLocker) - if err != nil { - level.Error(g.logger).Log("msg", fmt.Sprintf("unable to parse lock file for block: %s", blockID), "err", err) - return true, err - } - if time.Now().Before(blockLocker.LockTime.Add(HeartBeatTimeout)) { + if blockLocker.isLocked(g.blockLockTimeout) { level.Debug(g.logger).Log("msg", fmt.Sprintf("locked block: %s", blockID)) return true, nil } @@ -311,7 +306,7 @@ heartBeat: default: level.Debug(g.logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) g.lockBlocks(blocks) - time.Sleep(HeartBeatTimeout / 5) // it allows up to 5 failures on heart heat for single block + time.Sleep(g.blockLockFileUpdateInterval) } } level.Info(g.logger).Log("msg", fmt.Sprintf("stop heart beat for blocks: %s", blocksInfo)) @@ -320,18 +315,9 @@ heartBeat: func (g *ShuffleShardingGrouper) lockBlocks(blocks []*metadata.Meta) { for _, block := range blocks { blockID := block.ULID.String() - blockLockFilePath := path.Join(blockID, BlockLockFile) - blockLocker := BlockLocker{ - CompactorID: g.ringLifecyclerID, - LockTime: time.Now(), - } - lockFileContent, err := json.Marshal(blockLocker) - if err != nil { - level.Error(g.logger).Log("msg", "unable to create lock file content for block", "block_id", blockID, "err", err) - } - err = g.bkt.Upload(g.ctx, blockLockFilePath, bytes.NewReader(lockFileContent)) + err := UpdateBlockLocker(g.ctx, g.bkt, blockID, g.ringLifecyclerID, g.blockLockWriteFailed) if err != nil { - level.Error(g.logger).Log("msg", "unable to update heart beat for block", "block_id", blockID, "err", err) + level.Error(g.logger).Log("msg", "unable to upsert lock file content for block", "blockID", blockID, "err", err) } } } @@ -563,8 +549,3 @@ func sortMetasByMinTime(metas []*metadata.Meta) { return metas[i].BlockMeta.MinTime < metas[j].BlockMeta.MinTime }) } - -type BlockLocker struct { - CompactorID string `json:"compactorID"` - LockTime time.Time `json:"lockTime"` -} diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index 74c45c01b5b..cab447cb94d 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -281,13 +281,22 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { Name: "cortex_compactor_remaining_planned_compactions", Help: "Total number of plans that remain to be compacted.", }) + blockLockReadFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_lock_read_failed", + Help: "Number of block lock file failed to be read.", + }) + blockLockWriteFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_lock_write_failed", + Help: "Number of block lock file failed to be written.", + }) bkt := &bucket.ClientMock{} + blockLockTimeout := 5 * time.Minute for _, lockedBlock := range testData.lockedBlocks { lockFile := path.Join(lockedBlock.id.String(), BlockLockFile) expireTime := time.Now() if lockedBlock.isExpired { - expireTime = expireTime.Add(-1 * HeartBeatTimeout) + expireTime = expireTime.Add(-1 * blockLockTimeout) } blockLocker := BlockLocker{ CompactorID: "test-compactor", @@ -320,7 +329,12 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { overrides, "", 10, - 3) + 3, + blockLockTimeout, + time.Minute, + blockLockReadFailed, + blockLockWriteFailed, + ) actual, err := g.Groups(testData.blocks) require.NoError(t, err) require.Len(t, actual, len(testData.expected)) diff --git a/pkg/compactor/shuffle_sharding_planner.go b/pkg/compactor/shuffle_sharding_planner.go index b328676ac65..e8dafe5701e 100644 --- a/pkg/compactor/shuffle_sharding_planner.go +++ b/pkg/compactor/shuffle_sharding_planner.go @@ -2,25 +2,26 @@ package compactor import ( "context" - "encoding/json" "fmt" - "io/ioutil" - "path" "time" "github.com/go-kit/log" "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/objstore" ) type ShuffleShardingPlanner struct { - ctx context.Context - bkt objstore.Bucket - logger log.Logger - ranges []int64 - noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark - ringLifecyclerID string + ctx context.Context + bkt objstore.Bucket + logger log.Logger + ranges []int64 + noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark + ringLifecyclerID string + blockLockTimeout time.Duration + blockLockReadFailed prometheus.Counter } func NewShuffleShardingPlanner( @@ -30,14 +31,18 @@ func NewShuffleShardingPlanner( ranges []int64, noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark, ringLifecyclerID string, + blockLockTimeout time.Duration, + blockLockReadFailed prometheus.Counter, ) *ShuffleShardingPlanner { return &ShuffleShardingPlanner{ - ctx: ctx, - bkt: bkt, - logger: logger, - ranges: ranges, - noCompBlocksFunc: noCompBlocksFunc, - ringLifecyclerID: ringLifecyclerID, + ctx: ctx, + bkt: bkt, + logger: logger, + ranges: ranges, + noCompBlocksFunc: noCompBlocksFunc, + ringLifecyclerID: ringLifecyclerID, + blockLockTimeout: blockLockTimeout, + blockLockReadFailed: blockLockReadFailed, } } @@ -62,24 +67,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) } - lockFileReader, err := p.bkt.Get(p.ctx, path.Join(blockID, BlockLockFile)) + blockLocker, err := ReadBlockLocker(p.ctx, p.bkt, blockID, p.blockLockReadFailed) if err != nil { - if p.bkt.IsObjNotFoundErr(err) { + if errors.Is(err, ErrorBlockLockNotFound) { resultMetas = append(resultMetas, b) continue } return nil, fmt.Errorf("unable to get lock file for block %s: %s", blockID, err.Error()) } - bytes, err := ioutil.ReadAll(lockFileReader) - if err != nil { - return nil, fmt.Errorf("unable to read lock file for block %s: %s", blockID, err.Error()) - } - blockLocker := BlockLocker{} - err = json.Unmarshal(bytes, &blockLocker) - if err != nil { - return nil, fmt.Errorf("unable to parse lock file for block %s: %s", blockID, err.Error()) - } - if time.Now().Before(blockLocker.LockTime.Add(HeartBeatTimeout)) && blockLocker.CompactorID != p.ringLifecyclerID { + if blockLocker.isLockedForCompactor(p.blockLockTimeout, p.ringLifecyclerID) { return nil, fmt.Errorf("block %s is locked for compactor %s. but current compactor is %s", blockID, blockLocker.CompactorID, p.ringLifecyclerID) } diff --git a/pkg/compactor/shuffle_sharding_planner_test.go b/pkg/compactor/shuffle_sharding_planner_test.go index 904dfcb9f8b..d72f51ba470 100644 --- a/pkg/compactor/shuffle_sharding_planner_test.go +++ b/pkg/compactor/shuffle_sharding_planner_test.go @@ -9,6 +9,8 @@ import ( "time" "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" @@ -270,6 +272,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, } + blockLockTimeout := 5 * time.Minute for testName, testData := range tests { t.Run(testName, func(t *testing.T) { bkt := &bucket.ClientMock{} @@ -277,7 +280,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { lockFile := path.Join(lockedBlock.id.String(), BlockLockFile) expireTime := time.Now() if lockedBlock.isExpired { - expireTime = expireTime.Add(-1 * HeartBeatTimeout) + expireTime = expireTime.Add(-1 * blockLockTimeout) } blockLocker := BlockLocker{ CompactorID: lockedBlock.compactorID, @@ -288,6 +291,11 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { } bkt.MockGet(mock.Anything, "", nil) + blockLockReadFailed := promauto.With(prometheus.NewPedanticRegistry()).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_lock_read_failed", + Help: "Number of block lock file failed to be read.", + }) + p := NewShuffleShardingPlanner( context.Background(), bkt, @@ -297,6 +305,8 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { return testData.noCompactBlocks }, currentCompactor, + blockLockTimeout, + blockLockReadFailed, ) actual, err := p.Plan(context.Background(), testData.blocks) From 835251aadb39131ae7954e7f4693a7701c6e24c7 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Thu, 4 Aug 2022 15:23:02 -0700 Subject: [PATCH 12/17] Moved lock file heart beat into planner and refined planner logic to make sure blocks are locked by current compactor Signed-off-by: Alex Le --- pkg/compactor/block_locker.go | 44 ++++++- pkg/compactor/compactor.go | 10 +- pkg/compactor/compactor_test.go | 2 +- pkg/compactor/shuffle_sharding_grouper.go | 65 +++------- .../shuffle_sharding_grouper_test.go | 1 - pkg/compactor/shuffle_sharding_planner.go | 51 ++++---- .../shuffle_sharding_planner_test.go | 117 ++++++++++++++---- 7 files changed, 184 insertions(+), 106 deletions(-) diff --git a/pkg/compactor/block_locker.go b/pkg/compactor/block_locker.go index 455948374d1..903848f4539 100644 --- a/pkg/compactor/block_locker.go +++ b/pkg/compactor/block_locker.go @@ -4,12 +4,17 @@ import ( "bytes" "context" "encoding/json" + "fmt" "io/ioutil" "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/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/objstore" ) @@ -29,8 +34,8 @@ func (b *BlockLocker) isLocked(blockLockTimeout time.Duration) bool { return time.Now().Before(b.LockTime.Add(blockLockTimeout)) } -func (b *BlockLocker) isLockedForCompactor(blockLockTimeout time.Duration, compactorID string) bool { - return time.Now().Before(b.LockTime.Add(blockLockTimeout)) && b.CompactorID != compactorID +func (b *BlockLocker) isLockedByCompactor(blockLockTimeout time.Duration, compactorID string) bool { + return time.Now().Before(b.LockTime.Add(blockLockTimeout)) && b.CompactorID == compactorID } func ReadBlockLocker(ctx context.Context, bkt objstore.Bucket, blockID string, blockLockReadFailed prometheus.Counter) (*BlockLocker, error) { @@ -38,7 +43,7 @@ func ReadBlockLocker(ctx context.Context, bkt objstore.Bucket, blockID string, b lockFileReader, err := bkt.Get(ctx, lockFile) if err != nil { if bkt.IsObjNotFoundErr(err) { - return nil, errors.Wrapf(ErrorBlockLockNotFound, "block lcok file: %s", lockFile) + return nil, errors.Wrapf(ErrorBlockLockNotFound, "block lock file: %s", lockFile) } blockLockReadFailed.Inc() return nil, errors.Wrapf(err, "get block lock file: %s", lockFile) @@ -52,7 +57,7 @@ func ReadBlockLocker(ctx context.Context, bkt objstore.Bucket, blockID string, b err = json.Unmarshal(b, &blockLocker) if err != nil { blockLockReadFailed.Inc() - return nil, errors.Wrapf(ErrorUnmarshalBlockLock, "block lcok file: %s, error: %v", lockFile, err.Error()) + return nil, errors.Wrapf(ErrorUnmarshalBlockLock, "block lock file: %s, error: %v", lockFile, err.Error()) } return &blockLocker, nil } @@ -75,3 +80,34 @@ func UpdateBlockLocker(ctx context.Context, bkt objstore.Bucket, blockID string, } return nil } + +func LockBlocks(ctx context.Context, bkt objstore.Bucket, logger log.Logger, blocks []*metadata.Meta, compactorID string, blockLockWriteFailed prometheus.Counter) { + for _, block := range blocks { + blockID := block.ULID.String() + err := UpdateBlockLocker(ctx, bkt, blockID, compactorID, blockLockWriteFailed) + if err != nil { + level.Error(logger).Log("msg", "unable to upsert lock file content for block", "blockID", blockID, "err", err) + } + } +} + +func LockBlocksHeartBeat(ctx context.Context, bkt objstore.Bucket, logger log.Logger, blocks []*metadata.Meta, compactorID string, blockLockFileUpdateInterval time.Duration, blockLockWriteFailed 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)) +heartBeat: + for { + select { + case <-ctx.Done(): + break heartBeat + default: + level.Debug(logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) + LockBlocks(ctx, bkt, logger, blocks, compactorID, blockLockWriteFailed) + time.Sleep(blockLockFileUpdateInterval) + } + } + level.Info(logger).Log("msg", fmt.Sprintf("stop heart beat for blocks: %s", blocksInfo)) +} diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 903e8740b9f..bc3085b743a 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -90,7 +90,6 @@ var ( cfg.BlockFilesConcurrency, cfg.BlocksFetchConcurrency, cfg.BlockLockTimeout, - cfg.BlockLockFileUpdateInterval, blockLockReadFailed, blockLockWriteFailed) } @@ -101,7 +100,7 @@ var ( return nil, nil, err } - plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ prometheus.Counter) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ prometheus.Counter, _ prometheus.Counter) compact.Planner { return compact.NewPlanner(logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter) } @@ -114,9 +113,9 @@ var ( return nil, nil, err } - plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, blockLockReadFailed prometheus.Counter) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, blockLockReadFailed prometheus.Counter, blockLockWriteFailed prometheus.Counter) compact.Planner { - return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID, cfg.BlockLockTimeout, blockLockReadFailed) + return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID, cfg.BlockLockTimeout, cfg.BlockLockFileUpdateInterval, blockLockReadFailed, blockLockWriteFailed) } return compactor, plannerFactory, nil } @@ -157,6 +156,7 @@ type PlannerFactory func( noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, blockLockReadFailed prometheus.Counter, + blockLockWriteFailed prometheus.Counter, ) compact.Planner // Limits defines limits used by the Compactor. @@ -795,7 +795,7 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { ulogger, syncer, c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.blockLockReadFailed, c.blockLockWriteFailed, c.ring, c.ringLifecycler, c.limits, userID), - c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, c.blockLockReadFailed), + c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, c.blockLockReadFailed, c.blockLockWriteFailed), c.blocksCompactor, path.Join(c.compactorCfg.DataDir, "compact"), bucket, diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index c13631ee23c..b99f65a2b71 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1539,7 +1539,7 @@ func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, li blocksCompactorFactory := func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { return tsdbCompactor, - func(ctx context.Context, bkt objstore.Bucket, _ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ prometheus.Counter) compact.Planner { + func(ctx context.Context, bkt objstore.Bucket, _ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ prometheus.Counter, _ prometheus.Counter) compact.Planner { tsdbPlanner.noCompactMarkFilters = append(tsdbPlanner.noCompactMarkFilters, noCompactMarkFilter) return tsdbPlanner }, diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index 6d3aa191777..ae754a4f5c0 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -49,10 +49,9 @@ type ShuffleShardingGrouper struct { ringLifecyclerAddr string ringLifecyclerID string - blockLockTimeout time.Duration - blockLockFileUpdateInterval time.Duration - blockLockReadFailed prometheus.Counter - blockLockWriteFailed prometheus.Counter + blockLockTimeout time.Duration + blockLockReadFailed prometheus.Counter + blockLockWriteFailed prometheus.Counter } func NewShuffleShardingGrouper( @@ -76,7 +75,6 @@ func NewShuffleShardingGrouper( blockFilesConcurrency int, blocksFetchConcurrency int, blockLockTimeout time.Duration, - blockLockFileUpdateInterval time.Duration, blockLockReadFailed prometheus.Counter, blockLockWriteFailed prometheus.Counter, ) *ShuffleShardingGrouper { @@ -117,18 +115,17 @@ 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, - blockLockTimeout: blockLockTimeout, - blockLockFileUpdateInterval: blockLockFileUpdateInterval, - blockLockReadFailed: blockLockReadFailed, - blockLockWriteFailed: blockLockWriteFailed, + compactorCfg: compactorCfg, + ring: ring, + ringLifecyclerAddr: ringLifecyclerAddr, + ringLifecyclerID: ringLifecyclerID, + limits: limits, + userID: userID, + blockFilesConcurrency: blockFilesConcurrency, + blocksFetchConcurrency: blocksFetchConcurrency, + blockLockTimeout: blockLockTimeout, + blockLockReadFailed: blockLockReadFailed, + blockLockWriteFailed: blockLockWriteFailed, } } @@ -224,8 +221,7 @@ mainLoop: groupKey := createGroupKey(groupHash, group) level.Info(g.logger).Log("msg", "found compactable group for user", "group_hash", groupHash, "group", group.String()) - g.lockBlocks(group.blocks) - go g.groupLockHeartBeat(group.blocks) + LockBlocks(g.ctx, g.bkt, g.logger, group.blocks, g.ringLifecyclerID, g.blockLockWriteFailed) // All the blocks within the same group have the same downsample // resolution and external labels. @@ -291,37 +287,6 @@ func (g *ShuffleShardingGrouper) isGroupLocked(blocks []*metadata.Meta) (bool, e return false, nil } -func (g *ShuffleShardingGrouper) groupLockHeartBeat(blocks []*metadata.Meta) { - var blockIds []string - for _, block := range blocks { - blockIds = append(blockIds, block.ULID.String()) - } - blocksInfo := strings.Join(blockIds, ",") - level.Info(g.logger).Log("msg", fmt.Sprintf("start heart beat for blocks: %s", blocksInfo)) -heartBeat: - for { - select { - case <-g.ctx.Done(): - break heartBeat - default: - level.Debug(g.logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) - g.lockBlocks(blocks) - time.Sleep(g.blockLockFileUpdateInterval) - } - } - level.Info(g.logger).Log("msg", fmt.Sprintf("stop heart beat for blocks: %s", blocksInfo)) -} - -func (g *ShuffleShardingGrouper) lockBlocks(blocks []*metadata.Meta) { - for _, block := range blocks { - blockID := block.ULID.String() - err := UpdateBlockLocker(g.ctx, g.bkt, blockID, g.ringLifecyclerID, g.blockLockWriteFailed) - if err != nil { - level.Error(g.logger).Log("msg", "unable to upsert lock file content for block", "blockID", blockID, "err", err) - } - } -} - // Check whether this compactor exists on the subring based on user ID func (g *ShuffleShardingGrouper) checkSubringForCompactor() (bool, error) { subRing := g.ring.ShuffleShard(g.userID, g.limits.CompactorTenantShardSize(g.userID)) diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index cab447cb94d..688995a4f5e 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -331,7 +331,6 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { 10, 3, blockLockTimeout, - time.Minute, blockLockReadFailed, blockLockWriteFailed, ) diff --git a/pkg/compactor/shuffle_sharding_planner.go b/pkg/compactor/shuffle_sharding_planner.go index e8dafe5701e..1a35679c434 100644 --- a/pkg/compactor/shuffle_sharding_planner.go +++ b/pkg/compactor/shuffle_sharding_planner.go @@ -7,21 +7,22 @@ import ( "github.com/go-kit/log" "github.com/oklog/ulid" - "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/objstore" ) type ShuffleShardingPlanner struct { - ctx context.Context - bkt objstore.Bucket - logger log.Logger - ranges []int64 - noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark - ringLifecyclerID string - blockLockTimeout time.Duration - blockLockReadFailed prometheus.Counter + ctx context.Context + bkt objstore.Bucket + logger log.Logger + ranges []int64 + noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark + ringLifecyclerID string + blockLockTimeout time.Duration + blockLockFileUpdateInterval time.Duration + blockLockReadFailed prometheus.Counter + blockLockWriteFailed prometheus.Counter } func NewShuffleShardingPlanner( @@ -32,17 +33,21 @@ func NewShuffleShardingPlanner( noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark, ringLifecyclerID string, blockLockTimeout time.Duration, + blockLockFileUpdateInterval time.Duration, blockLockReadFailed prometheus.Counter, + blockLockWriteFailed prometheus.Counter, ) *ShuffleShardingPlanner { return &ShuffleShardingPlanner{ - ctx: ctx, - bkt: bkt, - logger: logger, - ranges: ranges, - noCompBlocksFunc: noCompBlocksFunc, - ringLifecyclerID: ringLifecyclerID, - blockLockTimeout: blockLockTimeout, - blockLockReadFailed: blockLockReadFailed, + ctx: ctx, + bkt: bkt, + logger: logger, + ranges: ranges, + noCompBlocksFunc: noCompBlocksFunc, + ringLifecyclerID: ringLifecyclerID, + blockLockTimeout: blockLockTimeout, + blockLockFileUpdateInterval: blockLockFileUpdateInterval, + blockLockReadFailed: blockLockReadFailed, + blockLockWriteFailed: blockLockWriteFailed, } } @@ -69,14 +74,12 @@ func (p *ShuffleShardingPlanner) Plan(_ context.Context, metasByMinTime []*metad blockLocker, err := ReadBlockLocker(p.ctx, p.bkt, blockID, p.blockLockReadFailed) if err != nil { - if errors.Is(err, ErrorBlockLockNotFound) { - resultMetas = append(resultMetas, b) - continue - } + // shuffle_sharding_grouper should put lock file for blocks ready for + // compaction. So error should be returned if lock file does not exist. return nil, fmt.Errorf("unable to get lock file for block %s: %s", blockID, err.Error()) } - if blockLocker.isLockedForCompactor(p.blockLockTimeout, p.ringLifecyclerID) { - return nil, fmt.Errorf("block %s is locked for compactor %s. but current compactor is %s", blockID, blockLocker.CompactorID, p.ringLifecyclerID) + if !blockLocker.isLockedByCompactor(p.blockLockTimeout, p.ringLifecyclerID) { + return nil, fmt.Errorf("block %s is not locked by current compactor %s", blockID, p.ringLifecyclerID) } resultMetas = append(resultMetas, b) @@ -86,5 +89,7 @@ func (p *ShuffleShardingPlanner) Plan(_ context.Context, metasByMinTime []*metad return nil, nil } + go LockBlocksHeartBeat(p.ctx, p.bkt, p.logger, resultMetas, p.ringLifecyclerID, p.blockLockFileUpdateInterval, p.blockLockWriteFailed) + return resultMetas, nil } diff --git a/pkg/compactor/shuffle_sharding_planner_test.go b/pkg/compactor/shuffle_sharding_planner_test.go index d72f51ba470..a407fedd2a6 100644 --- a/pkg/compactor/shuffle_sharding_planner_test.go +++ b/pkg/compactor/shuffle_sharding_planner_test.go @@ -8,6 +8,7 @@ import ( "testing" "time" + "github.com/go-kit/log" "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -18,6 +19,7 @@ import ( "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/cortexproject/cortex/pkg/storage/bucket" + "github.com/cortexproject/cortex/pkg/util/concurrency" ) func TestShuffleShardingPlanner_Plan(t *testing.T) { @@ -60,6 +62,18 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, + lockedBlocks: []LockedBlock{ + { + id: block1ulid, + isExpired: false, + compactorID: currentCompactor, + }, + { + id: block2ulid, + isExpired: false, + compactorID: currentCompactor, + }, + }, expected: []*metadata.Meta{ { BlockMeta: tsdb.BlockMeta{ @@ -95,6 +109,18 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, + lockedBlocks: []LockedBlock{ + { + id: block1ulid, + isExpired: false, + compactorID: currentCompactor, + }, + { + id: block2ulid, + 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()), }, "test blocks outside largest range 1": { @@ -115,6 +141,18 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, + lockedBlocks: []LockedBlock{ + { + id: block1ulid, + isExpired: false, + compactorID: currentCompactor, + }, + { + id: block2ulid, + 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()), }, "test blocks outside largest range 2": { @@ -135,6 +173,18 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, + lockedBlocks: []LockedBlock{ + { + id: block1ulid, + isExpired: false, + compactorID: currentCompactor, + }, + { + id: block2ulid, + 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()), }, "test should skip blocks marked for no compact": { @@ -163,6 +213,23 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, + lockedBlocks: []LockedBlock{ + { + id: block1ulid, + isExpired: false, + compactorID: currentCompactor, + }, + { + id: block2ulid, + isExpired: false, + compactorID: currentCompactor, + }, + { + id: block3ulid, + isExpired: false, + compactorID: currentCompactor, + }, + }, expected: []*metadata.Meta{ { BlockMeta: tsdb.BlockMeta{ @@ -199,6 +266,18 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, + lockedBlocks: []LockedBlock{ + { + id: block1ulid, + isExpired: false, + compactorID: currentCompactor, + }, + { + id: block2ulid, + isExpired: false, + compactorID: currentCompactor, + }, + }, expected: []*metadata.Meta{}, }, "test should not compact if lock file is not expired and locked by other compactor": { @@ -226,9 +305,9 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { compactorID: otherCompactor, }, }, - expectedErr: fmt.Errorf("block %s is locked for compactor %s. but current compactor is %s", block1ulid.String(), otherCompactor, currentCompactor), + expectedErr: fmt.Errorf("block %s is not locked by current compactor %s", block1ulid.String(), currentCompactor), }, - "test should compact if lock file is expired and was locked by other compactor": { + "test should not compact if lock file is expired": { ranges: []int64{2 * time.Hour.Milliseconds()}, blocks: []*metadata.Meta{ { @@ -246,29 +325,14 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - expected: []*metadata.Meta{ - { - BlockMeta: tsdb.BlockMeta{ - ULID: block1ulid, - MinTime: 1 * time.Hour.Milliseconds(), - MaxTime: 2 * time.Hour.Milliseconds(), - }, - }, - { - BlockMeta: tsdb.BlockMeta{ - ULID: block2ulid, - MinTime: 1 * time.Hour.Milliseconds(), - MaxTime: 2 * time.Hour.Milliseconds(), - }, - }, - }, lockedBlocks: []LockedBlock{ { id: block1ulid, isExpired: true, - compactorID: otherCompactor, + compactorID: currentCompactor, }, }, + expectedErr: fmt.Errorf("block %s is not locked by current compactor %s", block1ulid.String(), currentCompactor), }, } @@ -289,24 +353,33 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { lockFileContent, _ := json.Marshal(blockLocker) bkt.MockGet(lockFile, string(lockFileContent), nil) } - bkt.MockGet(mock.Anything, "", nil) + bkt.MockUpload(mock.Anything, nil) - blockLockReadFailed := promauto.With(prometheus.NewPedanticRegistry()).NewCounter(prometheus.CounterOpts{ + registerer := prometheus.NewPedanticRegistry() + blockLockReadFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_block_lock_read_failed", Help: "Number of block lock file failed to be read.", }) + blockLockWriteFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_lock_write_failed", + Help: "Number of block lock file failed to be written.", + }) + logs := &concurrency.SyncBuffer{} + logger := log.NewLogfmtLogger(logs) p := NewShuffleShardingPlanner( context.Background(), bkt, - nil, + logger, testData.ranges, func() map[ulid.ULID]*metadata.NoCompactMark { return testData.noCompactBlocks }, currentCompactor, blockLockTimeout, + time.Minute, blockLockReadFailed, + blockLockWriteFailed, ) actual, err := p.Plan(context.Background(), testData.blocks) From 3e5f65de1ebd2c7031d4b9d5ddbf8cf05de76bc4 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Mon, 22 Aug 2022 12:04:09 -0700 Subject: [PATCH 13/17] Updated documents Signed-off-by: Alex Le --- docs/blocks-storage/compactor.md | 9 +++++++++ docs/configuration/config-file-reference.md | 9 +++++++++ 2 files changed, 18 insertions(+) diff --git a/docs/blocks-storage/compactor.md b/docs/blocks-storage/compactor.md index 6156b0edd15..5c3e488cda7 100644 --- a/docs/blocks-storage/compactor.md +++ b/docs/blocks-storage/compactor.md @@ -254,4 +254,13 @@ compactor: # Timeout for waiting on compactor to become ACTIVE in the ring. # CLI flag: -compactor.ring.wait-active-instance-timeout [wait_active_instance_timeout: | default = 10m] + + # How long block lock file should be considered as expired and able to be + # picked up by compactor again. + # CLI flag: -compactor.block-lock-timeout + [block_lock_timeout: | default = 5m] + + # How frequently block lock file should be updated duration compaction. + # CLI flag: -compactor.block-lock-file-update-interval + [block_lock_file_update_interval: | default = 1m] ``` diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index f15c69e05b6..304d4321c39 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -5418,6 +5418,15 @@ sharding_ring: # Timeout for waiting on compactor to become ACTIVE in the ring. # CLI flag: -compactor.ring.wait-active-instance-timeout [wait_active_instance_timeout: | default = 10m] + +# How long block lock file should be considered as expired and able to be picked +# up by compactor again. +# CLI flag: -compactor.block-lock-timeout +[block_lock_timeout: | default = 5m] + +# How frequently block lock file should be updated duration compaction. +# CLI flag: -compactor.block-lock-file-update-interval +[block_lock_file_update_interval: | default = 1m] ``` ### `store_gateway_config` From d3530044599670accb8c18b4144c96d705756b92 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 30 Aug 2022 17:49:58 -0700 Subject: [PATCH 14/17] Return concurrency number of group. Use ticker for lock file heart beat Signed-off-by: Alex Le --- pkg/compactor/block_locker.go | 11 +-- pkg/compactor/compactor.go | 1 + pkg/compactor/shuffle_sharding_grouper.go | 7 +- .../shuffle_sharding_grouper_test.go | 69 +++++++++++++------ 4 files changed, 61 insertions(+), 27 deletions(-) diff --git a/pkg/compactor/block_locker.go b/pkg/compactor/block_locker.go index 903848f4539..f6abc526b92 100644 --- a/pkg/compactor/block_locker.go +++ b/pkg/compactor/block_locker.go @@ -98,15 +98,18 @@ func LockBlocksHeartBeat(ctx context.Context, bkt objstore.Bucket, logger log.Lo } blocksInfo := strings.Join(blockIds, ",") level.Info(logger).Log("msg", fmt.Sprintf("start heart beat for blocks: %s", blocksInfo)) + ticker := time.NewTicker(blockLockFileUpdateInterval) + defer ticker.Stop() heartBeat: for { + level.Debug(logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) + LockBlocks(ctx, bkt, logger, blocks, compactorID, blockLockWriteFailed) + select { case <-ctx.Done(): break heartBeat - default: - level.Debug(logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) - LockBlocks(ctx, bkt, logger, blocks, compactorID, blockLockWriteFailed) - time.Sleep(blockLockFileUpdateInterval) + case <-ticker.C: + continue } } level.Info(logger).Log("msg", fmt.Sprintf("stop heart beat for blocks: %s", blocksInfo)) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index a8b71149c86..046a33ddaee 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -89,6 +89,7 @@ var ( userID, cfg.BlockFilesConcurrency, cfg.BlocksFetchConcurrency, + cfg.CompactionConcurrency, cfg.BlockLockTimeout, blockLockReadFailed, blockLockWriteFailed) diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index ae754a4f5c0..c14e73d19ba 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -44,6 +44,7 @@ type ShuffleShardingGrouper struct { userID string blockFilesConcurrency int blocksFetchConcurrency int + compactionConcurrency int ring ring.ReadRing ringLifecyclerAddr string @@ -74,6 +75,7 @@ func NewShuffleShardingGrouper( userID string, blockFilesConcurrency int, blocksFetchConcurrency int, + compactionConcurrency int, blockLockTimeout time.Duration, blockLockReadFailed prometheus.Counter, blockLockWriteFailed prometheus.Counter, @@ -123,6 +125,7 @@ func NewShuffleShardingGrouper( userID: userID, blockFilesConcurrency: blockFilesConcurrency, blocksFetchConcurrency: blocksFetchConcurrency, + compactionConcurrency: compactionConcurrency, blockLockTimeout: blockLockTimeout, blockLockReadFailed: blockLockReadFailed, blockLockWriteFailed: blockLockWriteFailed, @@ -259,7 +262,9 @@ mainLoop: } outGroups = append(outGroups, thanosGroup) - break mainLoop + if len(outGroups) == g.compactionConcurrency { + break mainLoop + } } level.Info(g.logger).Log("msg", fmt.Sprintf("total groups for compaction: %d", len(outGroups))) diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index 688995a4f5e..bdaa075e0a4 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -116,6 +116,7 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { } tests := map[string]struct { + concurrency int ranges []time.Duration blocks map[ulid.ULID]*metadata.Meta lockedBlocks []struct { @@ -126,8 +127,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { metrics string }{ "test basic grouping": { - 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]}, + 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}, }, @@ -137,17 +139,19 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { `, }, "test no compaction": { - ranges: []time.Duration{2 * time.Hour, 4 * time.Hour}, - blocks: map[ulid.ULID]*metadata.Meta{block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block0hto1hExt2Ulid: blocks[block0hto1hExt2Ulid], block0to1hExt3Ulid: blocks[block0to1hExt3Ulid]}, - expected: [][]ulid.ULID{}, + concurrency: 1, + ranges: []time.Duration{2 * time.Hour, 4 * time.Hour}, + blocks: map[ulid.ULID]*metadata.Meta{block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block0hto1hExt2Ulid: blocks[block0hto1hExt2Ulid], block0to1hExt3Ulid: blocks[block0to1hExt3Ulid]}, + expected: [][]ulid.ULID{}, 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 0 `, }, "test smallest range first": { - 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], block4hto6hExt2Ulid: blocks[block4hto6hExt2Ulid], block6hto8hExt2Ulid: blocks[block6hto8hExt2Ulid]}, + 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], block4hto6hExt2Ulid: blocks[block4hto6hExt2Ulid], block6hto8hExt2Ulid: blocks[block6hto8hExt2Ulid]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, @@ -157,8 +161,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { `, }, "test oldest min time first": { - 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], block1hto2hExt1UlidCopy: blocks[block1hto2hExt1UlidCopy]}, + 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], block1hto2hExt1UlidCopy: blocks[block1hto2hExt1UlidCopy]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid, block1hto2hExt1UlidCopy}, }, @@ -168,8 +173,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { `, }, "test overlapping blocks": { - ranges: []time.Duration{20 * time.Hour, 40 * time.Hour}, - blocks: map[ulid.ULID]*metadata.Meta{block0hto20hExt1Ulid: blocks[block0hto20hExt1Ulid], block21hto40hExt1Ulid: blocks[block21hto40hExt1Ulid], block21hto40hExt1UlidCopy: blocks[block21hto40hExt1UlidCopy]}, + concurrency: 1, + ranges: []time.Duration{20 * time.Hour, 40 * time.Hour}, + blocks: map[ulid.ULID]*metadata.Meta{block0hto20hExt1Ulid: blocks[block0hto20hExt1Ulid], block21hto40hExt1Ulid: blocks[block21hto40hExt1Ulid], block21hto40hExt1UlidCopy: blocks[block21hto40hExt1UlidCopy]}, expected: [][]ulid.ULID{ {block21hto40hExt1Ulid, block21hto40hExt1UlidCopy}, }, @@ -179,8 +185,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { `, }, "test imperfect maxTime blocks": { - ranges: []time.Duration{2 * time.Hour}, - blocks: map[ulid.ULID]*metadata.Meta{block0hto1h30mExt1Ulid: blocks[block0hto1h30mExt1Ulid], block0hto45mExt1Ulid: blocks[block0hto45mExt1Ulid]}, + concurrency: 1, + ranges: []time.Duration{2 * time.Hour}, + blocks: map[ulid.ULID]*metadata.Meta{block0hto1h30mExt1Ulid: blocks[block0hto1h30mExt1Ulid], block0hto45mExt1Ulid: blocks[block0hto45mExt1Ulid]}, expected: [][]ulid.ULID{ {block0hto45mExt1Ulid, block0hto1h30mExt1Ulid}, }, @@ -190,17 +197,19 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { `, }, "test prematurely created blocks": { - ranges: []time.Duration{2 * time.Hour}, - blocks: map[ulid.ULID]*metadata.Meta{blocklast1hExt1UlidCopy: blocks[blocklast1hExt1UlidCopy], blocklast1hExt1Ulid: blocks[blocklast1hExt1Ulid]}, - expected: [][]ulid.ULID{}, + concurrency: 1, + ranges: []time.Duration{2 * time.Hour}, + blocks: map[ulid.ULID]*metadata.Meta{blocklast1hExt1UlidCopy: blocks[blocklast1hExt1UlidCopy], blocklast1hExt1Ulid: blocks[blocklast1hExt1Ulid]}, + expected: [][]ulid.ULID{}, 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 0 `, }, "test group with all blocks locked": { - 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]}, + 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{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, @@ -217,8 +226,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { `, }, "test group with one block locked": { - 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]}, + 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{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, @@ -234,8 +244,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { `, }, "test group block lock file expired": { - 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]}, + 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}, }, @@ -249,6 +260,19 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { 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 basic grouping with concurrency 2": { + concurrency: 2, + 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}, + {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, + }, + 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 `, }, } @@ -330,6 +354,7 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { "", 10, 3, + testData.concurrency, blockLockTimeout, blockLockReadFailed, blockLockWriteFailed, From b7b0663f3b1e6c0832e41ee030b4614da4c41e55 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Thu, 1 Sep 2022 14:16:29 -0700 Subject: [PATCH 15/17] Renamed lock file to be visit marker file Signed-off-by: Alex Le --- CHANGELOG.md | 2 +- docs/blocks-storage/compactor.md | 17 +-- docs/configuration/config-file-reference.md | 16 +-- pkg/compactor/block_locker.go | 116 ------------------ pkg/compactor/block_visit_marker.go | 116 ++++++++++++++++++ pkg/compactor/compactor.go | 50 ++++---- pkg/compactor/compactor_test.go | 4 +- pkg/compactor/shuffle_sharding_grouper.go | 60 ++++----- .../shuffle_sharding_grouper_test.go | 56 ++++----- pkg/compactor/shuffle_sharding_planner.go | 62 +++++----- .../shuffle_sharding_planner_test.go | 66 +++++----- 11 files changed, 283 insertions(+), 282 deletions(-) delete mode 100644 pkg/compactor/block_locker.go create mode 100644 pkg/compactor/block_visit_marker.go diff --git a/CHANGELOG.md b/CHANGELOG.md index eb45d6683b6..f27977a5ca0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -44,7 +44,7 @@ * [ENHANCEMENT] Ring: DoBatch prioritize 4xx errors when failing. #4783 * [ENHANCEMENT] Cortex now built with Go 1.18. #4829 * [ENHANCEMENT] Ingester: Prevent ingesters to become unhealthy during wall replay. #4847 -* [ENHANCEMENT] Compactor: Introduced lock file for blocks so blocks are under compaction will not be picked up by another compactor. #4805 +* [ENHANCEMENT] Compactor: Introduced visit marker file for blocks so blocks are under compaction will not be picked up by another compactor. #4805 * [FEATURE] Compactor: Added `-compactor.block-files-concurrency` allowing to configure number of go routines for download/upload block files during compaction. #4784 * [FEATURE] Compactor: Added -compactor.blocks-fetch-concurrency` allowing to configure number of go routines for blocks during compaction. #4787 * [FEATURE] Compactor: Added configurations for Azure MSI in blocks-storage, ruler-storage and alertmanager-storage. #4818 diff --git a/docs/blocks-storage/compactor.md b/docs/blocks-storage/compactor.md index 5c3e488cda7..2e7b40b8a2b 100644 --- a/docs/blocks-storage/compactor.md +++ b/docs/blocks-storage/compactor.md @@ -255,12 +255,13 @@ compactor: # CLI flag: -compactor.ring.wait-active-instance-timeout [wait_active_instance_timeout: | default = 10m] - # How long block lock file should be considered as expired and able to be - # picked up by compactor again. - # CLI flag: -compactor.block-lock-timeout - [block_lock_timeout: | default = 5m] - - # How frequently block lock file should be updated duration compaction. - # CLI flag: -compactor.block-lock-file-update-interval - [block_lock_file_update_interval: | default = 1m] + # How long block visit marker file should be considered as expired and able to + # be picked up by compactor again. + # CLI flag: -compactor.block-visit-marker-timeout + [block_visit_marker_timeout: | default = 5m] + + # How frequently block visit marker file should be updated duration + # compaction. + # CLI flag: -compactor.block-visit-marker-file-update-interval + [block_visit_marker_file_update_interval: | default = 1m] ``` diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index fbcad1490a8..6fb232f54f0 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -3847,14 +3847,14 @@ sharding_ring: # CLI flag: -compactor.ring.wait-active-instance-timeout [wait_active_instance_timeout: | default = 10m] -# How long block lock file should be considered as expired and able to be picked -# up by compactor again. -# CLI flag: -compactor.block-lock-timeout -[block_lock_timeout: | default = 5m] - -# How frequently block lock file should be updated duration compaction. -# CLI flag: -compactor.block-lock-file-update-interval -[block_lock_file_update_interval: | default = 1m] +# How long block visit marker file should be considered as expired and able to +# be picked up by compactor again. +# CLI flag: -compactor.block-visit-marker-timeout +[block_visit_marker_timeout: | default = 5m] + +# How frequently block visit marker file should be updated duration compaction. +# CLI flag: -compactor.block-visit-marker-file-update-interval +[block_visit_marker_file_update_interval: | default = 1m] ``` ### `store_gateway_config` diff --git a/pkg/compactor/block_locker.go b/pkg/compactor/block_locker.go deleted file mode 100644 index f6abc526b92..00000000000 --- a/pkg/compactor/block_locker.go +++ /dev/null @@ -1,116 +0,0 @@ -package compactor - -import ( - "bytes" - "context" - "encoding/json" - "fmt" - "io/ioutil" - "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/thanos/pkg/block/metadata" - "github.com/thanos-io/thanos/pkg/objstore" -) - -const BlockLockFile = "block.lock" - -var ( - ErrorBlockLockNotFound = errors.New("block lock not found") - ErrorUnmarshalBlockLock = errors.New("unmarshal block lock JSON") -) - -type BlockLocker struct { - CompactorID string `json:"compactorID"` - LockTime time.Time `json:"lockTime"` -} - -func (b *BlockLocker) isLocked(blockLockTimeout time.Duration) bool { - return time.Now().Before(b.LockTime.Add(blockLockTimeout)) -} - -func (b *BlockLocker) isLockedByCompactor(blockLockTimeout time.Duration, compactorID string) bool { - return time.Now().Before(b.LockTime.Add(blockLockTimeout)) && b.CompactorID == compactorID -} - -func ReadBlockLocker(ctx context.Context, bkt objstore.Bucket, blockID string, blockLockReadFailed prometheus.Counter) (*BlockLocker, error) { - lockFile := path.Join(blockID, BlockLockFile) - lockFileReader, err := bkt.Get(ctx, lockFile) - if err != nil { - if bkt.IsObjNotFoundErr(err) { - return nil, errors.Wrapf(ErrorBlockLockNotFound, "block lock file: %s", lockFile) - } - blockLockReadFailed.Inc() - return nil, errors.Wrapf(err, "get block lock file: %s", lockFile) - } - b, err := ioutil.ReadAll(lockFileReader) - if err != nil { - blockLockReadFailed.Inc() - return nil, errors.Wrapf(err, "read block lock file: %s", lockFile) - } - blockLocker := BlockLocker{} - err = json.Unmarshal(b, &blockLocker) - if err != nil { - blockLockReadFailed.Inc() - return nil, errors.Wrapf(ErrorUnmarshalBlockLock, "block lock file: %s, error: %v", lockFile, err.Error()) - } - return &blockLocker, nil -} - -func UpdateBlockLocker(ctx context.Context, bkt objstore.Bucket, blockID string, compactorID string, blockLockWriteFailed prometheus.Counter) error { - blockLockFilePath := path.Join(blockID, BlockLockFile) - blockLocker := BlockLocker{ - CompactorID: compactorID, - LockTime: time.Now(), - } - lockFileContent, err := json.Marshal(blockLocker) - if err != nil { - blockLockWriteFailed.Inc() - return err - } - err = bkt.Upload(ctx, blockLockFilePath, bytes.NewReader(lockFileContent)) - if err != nil { - blockLockWriteFailed.Inc() - return err - } - return nil -} - -func LockBlocks(ctx context.Context, bkt objstore.Bucket, logger log.Logger, blocks []*metadata.Meta, compactorID string, blockLockWriteFailed prometheus.Counter) { - for _, block := range blocks { - blockID := block.ULID.String() - err := UpdateBlockLocker(ctx, bkt, blockID, compactorID, blockLockWriteFailed) - if err != nil { - level.Error(logger).Log("msg", "unable to upsert lock file content for block", "blockID", blockID, "err", err) - } - } -} - -func LockBlocksHeartBeat(ctx context.Context, bkt objstore.Bucket, logger log.Logger, blocks []*metadata.Meta, compactorID string, blockLockFileUpdateInterval time.Duration, blockLockWriteFailed 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(blockLockFileUpdateInterval) - defer ticker.Stop() -heartBeat: - for { - level.Debug(logger).Log("msg", fmt.Sprintf("heart beat for blocks: %s", blocksInfo)) - LockBlocks(ctx, bkt, logger, blocks, compactorID, blockLockWriteFailed) - - select { - case <-ctx.Done(): - break heartBeat - case <-ticker.C: - continue - } - } - level.Info(logger).Log("msg", fmt.Sprintf("stop heart beat for blocks: %s", blocksInfo)) -} diff --git a/pkg/compactor/block_visit_marker.go b/pkg/compactor/block_visit_marker.go new file mode 100644 index 00000000000..155681dcad2 --- /dev/null +++ b/pkg/compactor/block_visit_marker.go @@ -0,0 +1,116 @@ +package compactor + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "io/ioutil" + "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/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/objstore" +) + +const BlockVisitMarkerFile = "block.visit" + +var ( + ErrorBlockVisitMarkerNotFound = errors.New("block visit marker not found") + ErrorUnmarshalBlockVisitMarker = errors.New("unmarshal block visit marker JSON") +) + +type BlockVisitMarker struct { + CompactorID string `json:"compactorID"` + VisitTime time.Time `json:"visitTime"` +} + +func (b *BlockVisitMarker) isVisited(blockVisitMarkerTimeout time.Duration) bool { + return time.Now().Before(b.VisitTime.Add(blockVisitMarkerTimeout)) +} + +func (b *BlockVisitMarker) isVisitedByCompactor(blockVisitMarkerTimeout time.Duration, compactorID string) bool { + return time.Now().Before(b.VisitTime.Add(blockVisitMarkerTimeout)) && b.CompactorID == compactorID +} + +func ReadBlockVisitMarker(ctx context.Context, bkt objstore.Bucket, blockID string, blockVisitMarkerReadFailed prometheus.Counter) (*BlockVisitMarker, error) { + visitMarkerFile := path.Join(blockID, BlockVisitMarkerFile) + visitMarkerFileReader, err := bkt.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) + } + b, err := ioutil.ReadAll(visitMarkerFileReader) + if err != nil { + blockVisitMarkerReadFailed.Inc() + return nil, errors.Wrapf(err, "read block visit marker file: %s", visitMarkerFile) + } + blockVisitMarker := BlockVisitMarker{} + err = json.Unmarshal(b, &blockVisitMarker) + if err != nil { + blockVisitMarkerReadFailed.Inc() + return nil, errors.Wrapf(ErrorUnmarshalBlockVisitMarker, "block visit marker file: %s, error: %v", visitMarkerFile, err.Error()) + } + return &blockVisitMarker, nil +} + +func UpdateBlockVisitMarker(ctx context.Context, bkt objstore.Bucket, blockID string, compactorID string, blockVisitMarkerWriteFailed prometheus.Counter) error { + blockVisitMarkerFilePath := path.Join(blockID, BlockVisitMarkerFile) + blockVisitMarker := BlockVisitMarker{ + CompactorID: compactorID, + VisitTime: time.Now(), + } + visitMarkerFileContent, err := json.Marshal(blockVisitMarker) + if err != nil { + blockVisitMarkerWriteFailed.Inc() + return err + } + err = bkt.Upload(ctx, blockVisitMarkerFilePath, bytes.NewReader(visitMarkerFileContent)) + if err != nil { + blockVisitMarkerWriteFailed.Inc() + return err + } + return nil +} + +func markBlocksVisited(ctx context.Context, bkt objstore.Bucket, logger log.Logger, blocks []*metadata.Meta, compactorID string, blockVisitMarkerWriteFailed prometheus.Counter) { + for _, block := range blocks { + blockID := block.ULID.String() + err := UpdateBlockVisitMarker(ctx, bkt, blockID, compactorID, blockVisitMarkerWriteFailed) + if err != nil { + level.Error(logger).Log("msg", "unable to upsert visit marker file content for block", "blockID", blockID, "err", err) + } + } +} + +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)) + markBlocksVisited(ctx, bkt, logger, blocks, compactorID, 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)) +} diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 046a33ddaee..bf4b2b871af 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -68,7 +68,7 @@ var ( cfg.BlocksFetchConcurrency) } - ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.Bucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, blockLockReadFailed prometheus.Counter, blockLockWriteFailed prometheus.Counter, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string) compact.Grouper { + ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.Bucket, 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) compact.Grouper { return NewShuffleShardingGrouper( ctx, logger, @@ -90,9 +90,9 @@ var ( cfg.BlockFilesConcurrency, cfg.BlocksFetchConcurrency, cfg.CompactionConcurrency, - cfg.BlockLockTimeout, - blockLockReadFailed, - blockLockWriteFailed) + cfg.BlockVisitMarkerTimeout, + blockVisitMarkerReadFailed, + blockVisitMarkerWriteFailed) } DefaultBlocksCompactorFactory = func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { @@ -114,9 +114,9 @@ var ( return nil, nil, err } - plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, blockLockReadFailed prometheus.Counter, blockLockWriteFailed prometheus.Counter) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.Bucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter) compact.Planner { - return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID, cfg.BlockLockTimeout, cfg.BlockLockFileUpdateInterval, blockLockReadFailed, blockLockWriteFailed) + return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID, cfg.BlockVisitMarkerTimeout, cfg.BlockVisitMarkerFileUpdateInterval, blockVisitMarkerReadFailed, blockVisitMarkerWriteFailed) } return compactor, plannerFactory, nil } @@ -133,8 +133,8 @@ type BlocksGrouperFactory func( blocksMarkedForNoCompact prometheus.Counter, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, - blockLockReadFailed prometheus.Counter, - blockLockWriteFailed prometheus.Counter, + blockVisitMarkerReadFailed prometheus.Counter, + blockVisitMarkerWriteFailed prometheus.Counter, ring *ring.Ring, ringLifecycler *ring.Lifecycler, limit Limits, @@ -156,8 +156,8 @@ type PlannerFactory func( cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, - blockLockReadFailed prometheus.Counter, - blockLockWriteFailed prometheus.Counter, + blockVisitMarkerReadFailed prometheus.Counter, + blockVisitMarkerWriteFailed prometheus.Counter, ) compact.Planner // Limits defines limits used by the Compactor. @@ -204,9 +204,9 @@ type Config struct { BlocksGrouperFactory BlocksGrouperFactory `yaml:"-"` BlocksCompactorFactory BlocksCompactorFactory `yaml:"-"` - // Block lock file config - BlockLockTimeout time.Duration `yaml:"block_lock_timeout"` - BlockLockFileUpdateInterval time.Duration `yaml:"block_lock_file_update_interval"` + // Block visit marker file config + BlockVisitMarkerTimeout time.Duration `yaml:"block_visit_marker_timeout"` + BlockVisitMarkerFileUpdateInterval time.Duration `yaml:"block_visit_marker_file_update_interval"` } // RegisterFlags registers the Compactor flags. @@ -241,8 +241,8 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.Var(&cfg.EnabledTenants, "compactor.enabled-tenants", "Comma separated list of tenants that can be compacted. If specified, only these tenants will be compacted by compactor, otherwise all tenants can be compacted. Subject to sharding.") f.Var(&cfg.DisabledTenants, "compactor.disabled-tenants", "Comma separated list of tenants that cannot be compacted by this compactor. If specified, and compactor would normally pick given tenant for compaction (via -compactor.enabled-tenants or sharding), it will be ignored instead.") - f.DurationVar(&cfg.BlockLockTimeout, "compactor.block-lock-timeout", 5*time.Minute, "How long block lock file should be considered as expired and able to be picked up by compactor again.") - f.DurationVar(&cfg.BlockLockFileUpdateInterval, "compactor.block-lock-file-update-interval", 1*time.Minute, "How frequently block lock file should be updated duration compaction.") + f.DurationVar(&cfg.BlockVisitMarkerTimeout, "compactor.block-visit-marker-timeout", 5*time.Minute, "How long block visit marker file should be considered as expired and able to be picked up by compactor again.") + f.DurationVar(&cfg.BlockVisitMarkerFileUpdateInterval, "compactor.block-visit-marker-file-update-interval", 1*time.Minute, "How frequently block visit marker file should be updated duration compaction.") } func (cfg *Config) Validate(limits validation.Limits) error { @@ -326,8 +326,8 @@ type Compactor struct { blocksMarkedForNoCompaction prometheus.Counter garbageCollectedBlocks prometheus.Counter remainingPlannedCompactions prometheus.Gauge - blockLockReadFailed prometheus.Counter - blockLockWriteFailed prometheus.Counter + blockVisitMarkerReadFailed prometheus.Counter + blockVisitMarkerWriteFailed prometheus.Counter // TSDB syncer metrics syncerMetrics *syncerMetrics @@ -445,13 +445,13 @@ func newCompactor( Name: "cortex_compactor_garbage_collected_blocks_total", Help: "Total number of blocks marked for deletion by compactor.", }), - blockLockReadFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_lock_read_failed", - Help: "Number of block lock file failed to be read.", + 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.", }), - blockLockWriteFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_lock_write_failed", - Help: "Number of block lock file failed to be written.", + 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, limits: limits, @@ -795,8 +795,8 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { compactor, err := compact.NewBucketCompactor( ulogger, syncer, - c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.blockLockReadFailed, c.blockLockWriteFailed, c.ring, c.ringLifecycler, c.limits, userID), - c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, c.blockLockReadFailed, c.blockLockWriteFailed), + 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), + c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed), c.blocksCompactor, path.Join(c.compactorCfg.DataDir, "compact"), bucket, diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index b99f65a2b71..9433b16ceb3 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1207,9 +1207,9 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit blockDirectory := []string{} for blockID, blockTimes := range blocks { - blockLocker := BlockLocker{ + blockLocker := BlockVisitMarker{ CompactorID: "test-compactor", - LockTime: time.Now(), + VisitTime: time.Now(), } lockFileContent, _ := json.Marshal(blockLocker) bucketClient.MockGet(userID+"/"+blockID+"/meta.json", mockBlockMetaJSONWithTime(blockID, userID, blockTimes["startTime"], blockTimes["endTime"]), nil) diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index c14e73d19ba..4b66b7c1170 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -50,9 +50,9 @@ type ShuffleShardingGrouper struct { ringLifecyclerAddr string ringLifecyclerID string - blockLockTimeout time.Duration - blockLockReadFailed prometheus.Counter - blockLockWriteFailed prometheus.Counter + blockVisitMarkerTimeout time.Duration + blockVisitMarkerReadFailed prometheus.Counter + blockVisitMarkerWriteFailed prometheus.Counter } func NewShuffleShardingGrouper( @@ -76,9 +76,9 @@ func NewShuffleShardingGrouper( blockFilesConcurrency int, blocksFetchConcurrency int, compactionConcurrency int, - blockLockTimeout time.Duration, - blockLockReadFailed prometheus.Counter, - blockLockWriteFailed prometheus.Counter, + blockVisitMarkerTimeout time.Duration, + blockVisitMarkerReadFailed prometheus.Counter, + blockVisitMarkerWriteFailed prometheus.Counter, ) *ShuffleShardingGrouper { if logger == nil { logger = log.NewNopLogger() @@ -117,18 +117,18 @@ 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, - blockLockTimeout: blockLockTimeout, - blockLockReadFailed: blockLockReadFailed, - blockLockWriteFailed: blockLockWriteFailed, + compactorCfg: compactorCfg, + ring: ring, + ringLifecyclerAddr: ringLifecyclerAddr, + ringLifecyclerID: ringLifecyclerID, + limits: limits, + userID: userID, + blockFilesConcurrency: blockFilesConcurrency, + blocksFetchConcurrency: blocksFetchConcurrency, + compactionConcurrency: compactionConcurrency, + blockVisitMarkerTimeout: blockVisitMarkerTimeout, + blockVisitMarkerReadFailed: blockVisitMarkerReadFailed, + blockVisitMarkerWriteFailed: blockVisitMarkerWriteFailed, } } @@ -212,11 +212,11 @@ mainLoop: groupHash := hashGroup(g.userID, group.rangeStart, group.rangeEnd) - if isLocked, err := g.isGroupLocked(group.blocks); err != nil { - level.Warn(g.logger).Log("msg", "unable to check if blocks in group are locked", "group hash", groupHash, "err", err, "group", group.String()) + if isVisited, err := g.isGroupVisited(group.blocks); 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 isLocked { - level.Info(g.logger).Log("msg", "skipping group because at least one block in group are locked", "group_hash", groupHash) + } else if isVisited { + level.Info(g.logger).Log("msg", "skipping group because at least one block in group is visited", "group_hash", groupHash) continue } @@ -224,7 +224,7 @@ mainLoop: groupKey := createGroupKey(groupHash, group) level.Info(g.logger).Log("msg", "found compactable group for user", "group_hash", groupHash, "group", group.String()) - LockBlocks(g.ctx, g.bkt, g.logger, group.blocks, g.ringLifecyclerID, g.blockLockWriteFailed) + markBlocksVisited(g.ctx, g.bkt, g.logger, group.blocks, g.ringLifecyclerID, g.blockVisitMarkerWriteFailed) // All the blocks within the same group have the same downsample // resolution and external labels. @@ -272,20 +272,20 @@ mainLoop: return outGroups, nil } -func (g *ShuffleShardingGrouper) isGroupLocked(blocks []*metadata.Meta) (bool, error) { +func (g *ShuffleShardingGrouper) isGroupVisited(blocks []*metadata.Meta) (bool, error) { for _, block := range blocks { blockID := block.ULID.String() - blockLocker, err := ReadBlockLocker(g.ctx, g.bkt, blockID, g.blockLockReadFailed) + blockVisitMarker, err := ReadBlockVisitMarker(g.ctx, g.bkt, blockID, g.blockVisitMarkerReadFailed) if err != nil { - if errors.Is(err, ErrorBlockLockNotFound) { - level.Debug(g.logger).Log("msg", "no lock file for block", "blockID", blockID) + if errors.Is(err, ErrorBlockVisitMarkerNotFound) { + level.Debug(g.logger).Log("msg", "no visit marker file for block", "blockID", blockID) continue } - level.Error(g.logger).Log("msg", "unable to read block lock file", "blockID", blockID, "err", err) + level.Error(g.logger).Log("msg", "unable to read block visit marker file", "blockID", blockID, "err", err) return true, err } - if blockLocker.isLocked(g.blockLockTimeout) { - level.Debug(g.logger).Log("msg", fmt.Sprintf("locked block: %s", blockID)) + if blockVisitMarker.isVisited(g.blockVisitMarkerTimeout) { + level.Debug(g.logger).Log("msg", fmt.Sprintf("visited block: %s", blockID)) return true, nil } } diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index bdaa075e0a4..6d1b90bc643 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -116,10 +116,10 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { } tests := map[string]struct { - concurrency int - ranges []time.Duration - blocks map[ulid.ULID]*metadata.Meta - lockedBlocks []struct { + concurrency int + ranges []time.Duration + blocks map[ulid.ULID]*metadata.Meta + visitedBlocks []struct { id ulid.ULID isExpired bool } @@ -206,14 +206,14 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { cortex_compactor_remaining_planned_compactions 0 `, }, - "test group with all blocks locked": { + "test group with all blocks 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]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, - lockedBlocks: []struct { + visitedBlocks: []struct { id ulid.ULID isExpired bool }{ @@ -225,14 +225,14 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { cortex_compactor_remaining_planned_compactions 1 `, }, - "test group with one block locked": { + "test group with one block 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]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, - lockedBlocks: []struct { + visitedBlocks: []struct { id ulid.ULID isExpired bool }{ @@ -243,14 +243,14 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { cortex_compactor_remaining_planned_compactions 1 `, }, - "test group block lock file expired": { + "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}, }, - lockedBlocks: []struct { + visitedBlocks: []struct { id ulid.ULID isExpired bool }{ @@ -305,29 +305,29 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { Name: "cortex_compactor_remaining_planned_compactions", Help: "Total number of plans that remain to be compacted.", }) - blockLockReadFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_lock_read_failed", - Help: "Number of block lock file failed to be read.", + 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.", }) - blockLockWriteFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_lock_write_failed", - Help: "Number of block lock file failed to be written.", + 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.", }) bkt := &bucket.ClientMock{} - blockLockTimeout := 5 * time.Minute - for _, lockedBlock := range testData.lockedBlocks { - lockFile := path.Join(lockedBlock.id.String(), BlockLockFile) + blockVisitMarkerTimeout := 5 * time.Minute + for _, visitedBlock := range testData.visitedBlocks { + visitMarkerFile := path.Join(visitedBlock.id.String(), BlockVisitMarkerFile) expireTime := time.Now() - if lockedBlock.isExpired { - expireTime = expireTime.Add(-1 * blockLockTimeout) + if visitedBlock.isExpired { + expireTime = expireTime.Add(-1 * blockVisitMarkerTimeout) } - blockLocker := BlockLocker{ + blockVisitMarker := BlockVisitMarker{ CompactorID: "test-compactor", - LockTime: expireTime, + VisitTime: expireTime, } - lockFileContent, _ := json.Marshal(blockLocker) - bkt.MockGet(lockFile, string(lockFileContent), nil) + visitMarkerFileContent, _ := json.Marshal(blockVisitMarker) + bkt.MockGet(visitMarkerFile, string(visitMarkerFileContent), nil) } bkt.MockUpload(mock.Anything, nil) bkt.MockGet(mock.Anything, "", nil) @@ -355,9 +355,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { 10, 3, testData.concurrency, - blockLockTimeout, - blockLockReadFailed, - blockLockWriteFailed, + blockVisitMarkerTimeout, + blockVisitMarkerReadFailed, + blockVisitMarkerWriteFailed, ) actual, err := g.Groups(testData.blocks) require.NoError(t, err) diff --git a/pkg/compactor/shuffle_sharding_planner.go b/pkg/compactor/shuffle_sharding_planner.go index 1a35679c434..9906a59e47d 100644 --- a/pkg/compactor/shuffle_sharding_planner.go +++ b/pkg/compactor/shuffle_sharding_planner.go @@ -13,16 +13,16 @@ import ( ) type ShuffleShardingPlanner struct { - ctx context.Context - bkt objstore.Bucket - logger log.Logger - ranges []int64 - noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark - ringLifecyclerID string - blockLockTimeout time.Duration - blockLockFileUpdateInterval time.Duration - blockLockReadFailed prometheus.Counter - blockLockWriteFailed prometheus.Counter + ctx context.Context + bkt objstore.Bucket + 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 } func NewShuffleShardingPlanner( @@ -32,22 +32,22 @@ func NewShuffleShardingPlanner( ranges []int64, noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark, ringLifecyclerID string, - blockLockTimeout time.Duration, - blockLockFileUpdateInterval time.Duration, - blockLockReadFailed prometheus.Counter, - blockLockWriteFailed prometheus.Counter, + blockVisitMarkerTimeout time.Duration, + blockVisitMarkerFileUpdateInterval time.Duration, + blockVisitMarkerReadFailed prometheus.Counter, + blockVisitMarkerWriteFailed prometheus.Counter, ) *ShuffleShardingPlanner { return &ShuffleShardingPlanner{ - ctx: ctx, - bkt: bkt, - logger: logger, - ranges: ranges, - noCompBlocksFunc: noCompBlocksFunc, - ringLifecyclerID: ringLifecyclerID, - blockLockTimeout: blockLockTimeout, - blockLockFileUpdateInterval: blockLockFileUpdateInterval, - blockLockReadFailed: blockLockReadFailed, - blockLockWriteFailed: blockLockWriteFailed, + ctx: ctx, + bkt: bkt, + logger: logger, + ranges: ranges, + noCompBlocksFunc: noCompBlocksFunc, + ringLifecyclerID: ringLifecyclerID, + blockVisitMarkerTimeout: blockVisitMarkerTimeout, + blockVisitMarkerFileUpdateInterval: blockVisitMarkerFileUpdateInterval, + blockVisitMarkerReadFailed: blockVisitMarkerReadFailed, + blockVisitMarkerWriteFailed: blockVisitMarkerWriteFailed, } } @@ -72,14 +72,14 @@ 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) } - blockLocker, err := ReadBlockLocker(p.ctx, p.bkt, blockID, p.blockLockReadFailed) + blockVisitMarker, err := ReadBlockVisitMarker(p.ctx, p.bkt, blockID, p.blockVisitMarkerReadFailed) if err != nil { - // shuffle_sharding_grouper should put lock file for blocks ready for - // compaction. So error should be returned if lock file does not exist. - return nil, fmt.Errorf("unable to get lock file for block %s: %s", blockID, err.Error()) + // 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 !blockLocker.isLockedByCompactor(p.blockLockTimeout, p.ringLifecyclerID) { - return nil, fmt.Errorf("block %s is not locked by current compactor %s", blockID, p.ringLifecyclerID) + if !blockVisitMarker.isVisitedByCompactor(p.blockVisitMarkerTimeout, p.ringLifecyclerID) { + return nil, fmt.Errorf("block %s is not visited by current compactor %s", blockID, p.ringLifecyclerID) } resultMetas = append(resultMetas, b) @@ -89,7 +89,7 @@ func (p *ShuffleShardingPlanner) Plan(_ context.Context, metasByMinTime []*metad return nil, nil } - go LockBlocksHeartBeat(p.ctx, p.bkt, p.logger, resultMetas, p.ringLifecyclerID, p.blockLockFileUpdateInterval, p.blockLockWriteFailed) + go markBlocksVisitedHeartBeat(p.ctx, p.bkt, p.logger, resultMetas, p.ringLifecyclerID, p.blockVisitMarkerFileUpdateInterval, p.blockVisitMarkerWriteFailed) return resultMetas, nil } diff --git a/pkg/compactor/shuffle_sharding_planner_test.go b/pkg/compactor/shuffle_sharding_planner_test.go index a407fedd2a6..b4d0a3f69e7 100644 --- a/pkg/compactor/shuffle_sharding_planner_test.go +++ b/pkg/compactor/shuffle_sharding_planner_test.go @@ -23,7 +23,7 @@ import ( ) func TestShuffleShardingPlanner_Plan(t *testing.T) { - type LockedBlock struct { + type VisitedBlock struct { id ulid.ULID isExpired bool compactorID string @@ -42,7 +42,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { blocks []*metadata.Meta expected []*metadata.Meta expectedErr error - lockedBlocks []LockedBlock + visitedBlocks []VisitedBlock }{ "test basic plan": { ranges: []int64{2 * time.Hour.Milliseconds()}, @@ -62,7 +62,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - lockedBlocks: []LockedBlock{ + visitedBlocks: []VisitedBlock{ { id: block1ulid, isExpired: false, @@ -109,7 +109,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - lockedBlocks: []LockedBlock{ + visitedBlocks: []VisitedBlock{ { id: block1ulid, isExpired: false, @@ -141,7 +141,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - lockedBlocks: []LockedBlock{ + visitedBlocks: []VisitedBlock{ { id: block1ulid, isExpired: false, @@ -173,7 +173,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - lockedBlocks: []LockedBlock{ + visitedBlocks: []VisitedBlock{ { id: block1ulid, isExpired: false, @@ -213,7 +213,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - lockedBlocks: []LockedBlock{ + visitedBlocks: []VisitedBlock{ { id: block1ulid, isExpired: false, @@ -266,7 +266,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - lockedBlocks: []LockedBlock{ + visitedBlocks: []VisitedBlock{ { id: block1ulid, isExpired: false, @@ -280,7 +280,7 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, expected: []*metadata.Meta{}, }, - "test should not compact if lock file is not expired and locked by other compactor": { + "test should not compact if visit marker file is not expired and visited by other compactor": { ranges: []int64{2 * time.Hour.Milliseconds()}, blocks: []*metadata.Meta{ { @@ -298,16 +298,16 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - lockedBlocks: []LockedBlock{ + visitedBlocks: []VisitedBlock{ { id: block1ulid, isExpired: false, compactorID: otherCompactor, }, }, - expectedErr: fmt.Errorf("block %s is not locked by current compactor %s", block1ulid.String(), currentCompactor), + expectedErr: fmt.Errorf("block %s is not visited by current compactor %s", block1ulid.String(), currentCompactor), }, - "test should not compact if lock file is expired": { + "test should not compact if visit marker file is expired": { ranges: []int64{2 * time.Hour.Milliseconds()}, blocks: []*metadata.Meta{ { @@ -325,44 +325,44 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { }, }, }, - lockedBlocks: []LockedBlock{ + visitedBlocks: []VisitedBlock{ { id: block1ulid, isExpired: true, compactorID: currentCompactor, }, }, - expectedErr: fmt.Errorf("block %s is not locked by current compactor %s", block1ulid.String(), currentCompactor), + expectedErr: fmt.Errorf("block %s is not visited by current compactor %s", block1ulid.String(), currentCompactor), }, } - blockLockTimeout := 5 * time.Minute + blockVisitMarkerTimeout := 5 * time.Minute for testName, testData := range tests { t.Run(testName, func(t *testing.T) { bkt := &bucket.ClientMock{} - for _, lockedBlock := range testData.lockedBlocks { - lockFile := path.Join(lockedBlock.id.String(), BlockLockFile) + for _, visitedBlock := range testData.visitedBlocks { + visitMarkerFile := path.Join(visitedBlock.id.String(), BlockVisitMarkerFile) expireTime := time.Now() - if lockedBlock.isExpired { - expireTime = expireTime.Add(-1 * blockLockTimeout) + if visitedBlock.isExpired { + expireTime = expireTime.Add(-1 * blockVisitMarkerTimeout) } - blockLocker := BlockLocker{ - CompactorID: lockedBlock.compactorID, - LockTime: expireTime, + blockVisitMarker := BlockVisitMarker{ + CompactorID: visitedBlock.compactorID, + VisitTime: expireTime, } - lockFileContent, _ := json.Marshal(blockLocker) - bkt.MockGet(lockFile, string(lockFileContent), nil) + visitMarkerFileContent, _ := json.Marshal(blockVisitMarker) + bkt.MockGet(visitMarkerFile, string(visitMarkerFileContent), nil) } bkt.MockUpload(mock.Anything, nil) registerer := prometheus.NewPedanticRegistry() - blockLockReadFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_lock_read_failed", - Help: "Number of block lock file failed to be read.", + 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.", }) - blockLockWriteFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_lock_write_failed", - Help: "Number of block lock file failed to be written.", + 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.", }) logs := &concurrency.SyncBuffer{} @@ -376,10 +376,10 @@ func TestShuffleShardingPlanner_Plan(t *testing.T) { return testData.noCompactBlocks }, currentCompactor, - blockLockTimeout, + blockVisitMarkerTimeout, time.Minute, - blockLockReadFailed, - blockLockWriteFailed, + blockVisitMarkerReadFailed, + blockVisitMarkerWriteFailed, ) actual, err := p.Plan(context.Background(), testData.blocks) From ad5963b0eedcea1cfe7bbb3108c45d86f6c92461 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Thu, 1 Sep 2022 14:33:12 -0700 Subject: [PATCH 16/17] Fixed unit test Signed-off-by: Alex Le --- pkg/compactor/compactor_test.go | 66 +++++++++---------- .../shuffle_sharding_grouper_test.go | 17 +++-- 2 files changed, 44 insertions(+), 39 deletions(-) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 9433b16ceb3..884c88e31a5 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -465,13 +465,13 @@ 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/block.lock", "", nil) - bucketClient.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/block.lock", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/block.visit", "", nil) + bucketClient.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/block.visit", 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/block.lock", "", nil) - bucketClient.MockUpload(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/block.lock", nil) + bucketClient.MockGet(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/block.visit", "", nil) + bucketClient.MockUpload(userID+"/01FN6CDF3PNEWWRY5MPGJPE3EX/block.visit", nil) bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) @@ -520,19 +520,19 @@ 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/block.lock", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/block.visit", "", 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/block.lock", "", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.visit", "", 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/block.lock", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.visit", "", 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/block.lock", "", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.visit", "", nil) bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) bucketClient.MockIter("user-1/markers/", nil, nil) @@ -775,24 +775,24 @@ 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/block.lock", "", nil) - bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.lock", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/block.visit", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.visit", 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/block.lock", "", nil) - bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.lock", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.visit", "", nil) + bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.visit", 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/block.lock", "", nil) - bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.lock", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.visit", "", nil) + bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.visit", 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/block.lock", "", nil) - bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.lock", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.visit", "", nil) + bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.visit", nil) bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) @@ -844,8 +844,8 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTVP434PA9VFXSW2JKB3392D/index"}, nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/index", "some index content", nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/block.lock", "", nil) - bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.lock", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/block.visit", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.visit", nil) bucketClient.MockExists("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", false, nil) bucketClient.MockDelete("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", nil) @@ -1006,23 +1006,23 @@ 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/block.lock", "", nil) - bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.lock", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/block.visit", "", nil) + bucketClient.MockUpload("user-1/01DTVP434PA9VFXSW2JKB3392D/block.visit", 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/block.lock", "", nil) - bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.lock", nil) + bucketClient.MockGet("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.visit", "", nil) + bucketClient.MockUpload("user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/block.visit", 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/block.lock", "", nil) - bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.lock", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.visit", "", nil) + bucketClient.MockUpload("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/block.visit", 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/block.lock", "", nil) - bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.lock", nil) + bucketClient.MockGet("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.visit", "", nil) + bucketClient.MockUpload("user-2/01FN3V83ABR9992RF8WRJZ76ZQ/block.visit", nil) bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) @@ -1104,8 +1104,8 @@ func TestCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndM 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/block.lock", "", nil) - bucketClient.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/block.lock", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/block.visit", "", nil) + bucketClient.MockUpload(userID+"/01DTVP434PA9VFXSW2JKB3392D/block.visit", nil) bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) } @@ -1207,17 +1207,17 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit blockDirectory := []string{} for blockID, blockTimes := range blocks { - blockLocker := BlockVisitMarker{ + blockVisitMarker := BlockVisitMarker{ CompactorID: "test-compactor", VisitTime: time.Now(), } - lockFileContent, _ := json.Marshal(blockLocker) + visitMarkerFileContent, _ := json.Marshal(blockVisitMarker) 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.MockGetTimes(userID+"/"+blockID+"/block.lock", "", nil, 1) - bucketClient.MockGet(userID+"/"+blockID+"/block.lock", string(lockFileContent), nil) - bucketClient.MockUpload(userID+"/"+blockID+"/block.lock", nil) + bucketClient.MockGetTimes(userID+"/"+blockID+"/block.visit", "", nil, 1) + bucketClient.MockGet(userID+"/"+blockID+"/block.visit", string(visitMarkerFileContent), nil) + bucketClient.MockUpload(userID+"/"+blockID+"/block.visit", nil) blockDirectory = append(blockDirectory, userID+"/"+blockID) // Get all of the unique group hashes so that they can be used to ensure all groups were compacted diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index 6d1b90bc643..ffcacece58d 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -127,15 +127,17 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { metrics string }{ "test basic grouping": { - concurrency: 1, + concurrency: 3, 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}, + {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 1 + cortex_compactor_remaining_planned_compactions 3 `, }, "test no compaction": { @@ -149,27 +151,30 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { `, }, "test smallest range first": { - concurrency: 1, + concurrency: 3, 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], block4hto6hExt2Ulid: blocks[block4hto6hExt2Ulid], block6hto8hExt2Ulid: blocks[block6hto8hExt2Ulid]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, + {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, + {block4hto6hExt2Ulid, block6hto8hExt2Ulid}, }, 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 3 `, }, "test oldest min time first": { - concurrency: 1, + concurrency: 2, 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], block1hto2hExt1UlidCopy: blocks[block1hto2hExt1UlidCopy]}, expected: [][]ulid.ULID{ {block1hto2hExt1Ulid, block0hto1hExt1Ulid, block1hto2hExt1UlidCopy}, + {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 1 + cortex_compactor_remaining_planned_compactions 2 `, }, "test overlapping blocks": { From c47eb5c2e7130a6fef2cf5da0f8b909d3e1b37eb Mon Sep 17 00:00:00 2001 From: Alex Le Date: Fri, 2 Sep 2022 16:45:26 -0700 Subject: [PATCH 17/17] Make sure visited block can be picked by compactor visited it Signed-off-by: Alex Le --- pkg/compactor/shuffle_sharding_grouper.go | 6 +- .../shuffle_sharding_grouper_test.go | 56 ++++++++++++++----- 2 files changed, 44 insertions(+), 18 deletions(-) diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index 4b66b7c1170..57b01436cd1 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -212,7 +212,7 @@ mainLoop: groupHash := hashGroup(g.userID, group.rangeStart, group.rangeEnd) - if isVisited, err := g.isGroupVisited(group.blocks); err != nil { + 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 { @@ -272,7 +272,7 @@ mainLoop: return outGroups, nil } -func (g *ShuffleShardingGrouper) isGroupVisited(blocks []*metadata.Meta) (bool, error) { +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, blockID, g.blockVisitMarkerReadFailed) @@ -284,7 +284,7 @@ func (g *ShuffleShardingGrouper) isGroupVisited(blocks []*metadata.Meta) (bool, level.Error(g.logger).Log("msg", "unable to read block visit marker file", "blockID", blockID, "err", err) return true, err } - if blockVisitMarker.isVisited(g.blockVisitMarkerTimeout) { + if compactorID != blockVisitMarker.CompactorID && blockVisitMarker.isVisited(g.blockVisitMarkerTimeout) { level.Debug(g.logger).Log("msg", fmt.Sprintf("visited block: %s", blockID)) return true, nil } diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index ffcacece58d..d618fbd9559 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -115,13 +115,17 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { }, } + testCompactorID := "test-compactor" + otherCompactorID := "other-compactor" + tests := map[string]struct { concurrency int ranges []time.Duration blocks map[ulid.ULID]*metadata.Meta visitedBlocks []struct { - id ulid.ULID - isExpired bool + id ulid.ULID + compactorID string + isExpired bool } expected [][]ulid.ULID metrics string @@ -219,11 +223,12 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, visitedBlocks: []struct { - id ulid.ULID - isExpired bool + id ulid.ULID + compactorID string + isExpired bool }{ - {id: block1hto2hExt2Ulid, isExpired: false}, - {id: block0hto1hExt2Ulid, isExpired: false}, + {id: block1hto2hExt2Ulid, compactorID: otherCompactorID, isExpired: false}, + {id: block0hto1hExt2Ulid, 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 @@ -238,10 +243,11 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, visitedBlocks: []struct { - id ulid.ULID - isExpired bool + id ulid.ULID + compactorID string + isExpired bool }{ - {id: block1hto2hExt2Ulid, isExpired: false}, + {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 @@ -256,11 +262,31 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {block1hto2hExt2Ulid, block0hto1hExt2Ulid}, }, visitedBlocks: []struct { - id ulid.ULID - isExpired bool + id ulid.ULID + compactorID string + isExpired bool + }{ + {id: block1hto2hExt2Ulid, compactorID: otherCompactorID, isExpired: true}, + {id: block0hto1hExt2Ulid, 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 +`, + }, + "test group with one block 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]}, + expected: [][]ulid.ULID{ + {block1hto2hExt2Ulid, block0hto1hExt2Ulid}, + }, + visitedBlocks: []struct { + id ulid.ULID + compactorID string + isExpired bool }{ - {id: block1hto2hExt2Ulid, isExpired: true}, - {id: block0hto1hExt2Ulid, isExpired: true}, + {id: block1hto2hExt2Ulid, 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 @@ -328,7 +354,7 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { expireTime = expireTime.Add(-1 * blockVisitMarkerTimeout) } blockVisitMarker := BlockVisitMarker{ - CompactorID: "test-compactor", + CompactorID: visitedBlock.compactorID, VisitTime: expireTime, } visitMarkerFileContent, _ := json.Marshal(blockVisitMarker) @@ -354,7 +380,7 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { *compactorCfg, ring, "test-addr", - "test-compactor", + testCompactorID, overrides, "", 10,