-
Notifications
You must be signed in to change notification settings - Fork 3.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
bloom blocks downloading queue #11201
Changes from all commits
73895e2
8c229b8
61ab0f0
7d2f0bf
6b05ec5
f65974f
3c4937b
d96c939
67db11d
a58a1db
c31ab4d
c9ee29d
ba28e04
75f3184
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||||
---|---|---|---|---|---|---|---|---|
|
@@ -82,7 +82,7 @@ func TestBloomGateway_StartStopService(t *testing.T) { | |||||||
}, | ||||||||
} | ||||||||
|
||||||||
gw, err := New(cfg, schemaCfg, storageCfg, ss, cm, logger, reg) | ||||||||
gw, err := New(cfg, schemaCfg, storageCfg, fakeLimits{}, ss, cm, logger, reg) | ||||||||
require.NoError(t, err) | ||||||||
|
||||||||
err = services.StartAndAwaitRunning(context.Background(), gw) | ||||||||
|
@@ -142,7 +142,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { | |||||||
|
||||||||
t.Run("returns unfiltered chunk refs if no filters provided", func(t *testing.T) { | ||||||||
reg := prometheus.NewRegistry() | ||||||||
gw, err := New(cfg, schemaCfg, storageCfg, ss, cm, logger, reg) | ||||||||
gw, err := New(cfg, schemaCfg, storageCfg, fakeLimits{}, ss, cm, logger, reg) | ||||||||
require.NoError(t, err) | ||||||||
|
||||||||
err = services.StartAndAwaitRunning(context.Background(), gw) | ||||||||
|
@@ -188,7 +188,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { | |||||||
|
||||||||
t.Run("returns error if chunk refs do not belong to tenant", func(t *testing.T) { | ||||||||
reg := prometheus.NewRegistry() | ||||||||
gw, err := New(cfg, schemaCfg, storageCfg, ss, cm, logger, reg) | ||||||||
gw, err := New(cfg, schemaCfg, storageCfg, fakeLimits{}, ss, cm, logger, reg) | ||||||||
require.NoError(t, err) | ||||||||
|
||||||||
ts, _ := time.Parse("2006-01-02 15:04", "2023-10-03 10:00") | ||||||||
|
@@ -212,7 +212,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { | |||||||
|
||||||||
t.Run("gateway tracks active users", func(t *testing.T) { | ||||||||
reg := prometheus.NewRegistry() | ||||||||
gw, err := New(cfg, schemaCfg, storageCfg, ss, cm, logger, reg) | ||||||||
gw, err := New(cfg, schemaCfg, storageCfg, fakeLimits{}, ss, cm, logger, reg) | ||||||||
require.NoError(t, err) | ||||||||
|
||||||||
err = services.StartAndAwaitRunning(context.Background(), gw) | ||||||||
|
@@ -248,3 +248,21 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { | |||||||
require.ElementsMatch(t, tenants, gw.activeUsers.ActiveUsers()) | ||||||||
}) | ||||||||
} | ||||||||
|
||||||||
type fakeLimits struct { | ||||||||
} | ||||||||
|
||||||||
func (f fakeLimits) BloomGatewayShardSize(_ string) int { | ||||||||
//TODO implement me | ||||||||
panic("implement me") | ||||||||
} | ||||||||
|
||||||||
func (f fakeLimits) BloomGatewayEnabled(_ string) bool { | ||||||||
//TODO implement me | ||||||||
panic("implement me") | ||||||||
Comment on lines
+261
to
+262
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||
} | ||||||||
|
||||||||
func (f fakeLimits) BloomGatewayBlocksDownloadingParallelism(_ string) int { | ||||||||
//TODO implement me | ||||||||
panic("implement me") | ||||||||
Comment on lines
+266
to
+267
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,230 @@ | ||
package bloomshipper | ||
|
||
import ( | ||
"context" | ||
"errors" | ||
"fmt" | ||
"io" | ||
"os" | ||
"path/filepath" | ||
"strconv" | ||
"strings" | ||
"time" | ||
|
||
"github.com/go-kit/log" | ||
"github.com/go-kit/log/level" | ||
"github.com/grafana/dskit/services" | ||
"github.com/prometheus/client_golang/prometheus" | ||
|
||
"github.com/grafana/loki/pkg/queue" | ||
v1 "github.com/grafana/loki/pkg/storage/bloom/v1" | ||
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" | ||
"github.com/grafana/loki/pkg/util" | ||
"github.com/grafana/loki/pkg/util/constants" | ||
) | ||
|
||
type blockDownloader struct { | ||
logger log.Logger | ||
|
||
workingDirectory string | ||
queueMetrics *queue.Metrics | ||
queue *queue.RequestQueue | ||
blockClient BlockClient | ||
limits Limits | ||
activeUsersService *util.ActiveUsersCleanupService | ||
|
||
ctx context.Context | ||
manager *services.Manager | ||
onWorkerStopCallback func() | ||
} | ||
|
||
func newBlockDownloader(config config.Config, blockClient BlockClient, limits Limits, logger log.Logger, reg prometheus.Registerer) (*blockDownloader, error) { | ||
queueMetrics := queue.NewMetrics(reg, constants.Loki, "bloom_blocks_downloader") | ||
//add cleanup service | ||
downloadingQueue := queue.NewRequestQueue(config.BlocksDownloadingQueue.MaxTasksEnqueuedPerTenant, time.Minute, queueMetrics) | ||
activeUsersService := util.NewActiveUsersCleanupWithDefaultValues(queueMetrics.Cleanup) | ||
|
||
ctx := context.Background() | ||
manager, err := services.NewManager(downloadingQueue, activeUsersService) | ||
if err != nil { | ||
return nil, fmt.Errorf("error creating service manager: %w", err) | ||
} | ||
err = services.StartManagerAndAwaitHealthy(ctx, manager) | ||
if err != nil { | ||
return nil, fmt.Errorf("error starting service manager: %w", err) | ||
} | ||
|
||
b := &blockDownloader{ | ||
ctx: ctx, | ||
logger: logger, | ||
workingDirectory: config.WorkingDirectory, | ||
queueMetrics: queueMetrics, | ||
queue: downloadingQueue, | ||
blockClient: blockClient, | ||
activeUsersService: activeUsersService, | ||
limits: limits, | ||
manager: manager, | ||
onWorkerStopCallback: onWorkerStopNoopCallback, | ||
} | ||
|
||
for i := 0; i < config.BlocksDownloadingQueue.WorkersCount; i++ { | ||
go b.serveDownloadingTasks(fmt.Sprintf("worker-%d", i)) | ||
vlad-diachenko marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
return b, nil | ||
} | ||
|
||
type BlockDownloadingTask struct { | ||
ctx context.Context | ||
block BlockRef | ||
// ErrCh is a send-only channel to write an error to | ||
ErrCh chan<- error | ||
// ResultsCh is a send-only channel to return the block querier for the downloaded block | ||
ResultsCh chan<- blockWithQuerier | ||
} | ||
|
||
func NewBlockDownloadingTask(ctx context.Context, block BlockRef, resCh chan<- blockWithQuerier, errCh chan<- error) *BlockDownloadingTask { | ||
return &BlockDownloadingTask{ | ||
ctx: ctx, | ||
block: block, | ||
ErrCh: errCh, | ||
ResultsCh: resCh, | ||
} | ||
} | ||
|
||
// noop implementation | ||
var onWorkerStopNoopCallback = func() {} | ||
|
||
func (d *blockDownloader) serveDownloadingTasks(workerID string) { | ||
logger := log.With(d.logger, "worker", workerID) | ||
level.Debug(logger).Log("msg", "starting worker") | ||
|
||
d.queue.RegisterConsumerConnection(workerID) | ||
defer d.queue.UnregisterConsumerConnection(workerID) | ||
//this callback is used only in the tests to assert that worker is stopped | ||
defer d.onWorkerStopCallback() | ||
|
||
idx := queue.StartIndexWithLocalQueue | ||
|
||
for { | ||
vlad-diachenko marked this conversation as resolved.
Show resolved
Hide resolved
|
||
item, newIdx, err := d.queue.Dequeue(d.ctx, idx, workerID) | ||
if err != nil { | ||
if !errors.Is(err, queue.ErrStopped) && !errors.Is(err, context.Canceled) { | ||
level.Error(logger).Log("msg", "failed to dequeue task", "err", err) | ||
continue | ||
} | ||
level.Info(logger).Log("msg", "stopping worker") | ||
return | ||
} | ||
task, ok := item.(*BlockDownloadingTask) | ||
if !ok { | ||
level.Error(logger).Log("msg", "failed to cast to BlockDownloadingTask", "item", fmt.Sprintf("%+v", item), "type", fmt.Sprintf("%T", item)) | ||
continue | ||
} | ||
|
||
idx = newIdx | ||
blockPath := task.block.BlockPath | ||
//todo add cache before downloading | ||
level.Debug(logger).Log("msg", "start downloading the block", "block", blockPath) | ||
block, err := d.blockClient.GetBlock(task.ctx, task.block) | ||
if err != nil { | ||
level.Error(logger).Log("msg", "error downloading the block", "block", blockPath, "err", err) | ||
task.ErrCh <- fmt.Errorf("error downloading the block %s : %w", blockPath, err) | ||
continue | ||
} | ||
directory, err := d.extractBlock(&block, time.Now()) | ||
if err != nil { | ||
level.Error(logger).Log("msg", "error extracting the block", "block", blockPath, "err", err) | ||
task.ErrCh <- fmt.Errorf("error extracting the block %s : %w", blockPath, err) | ||
continue | ||
} | ||
level.Debug(d.logger).Log("msg", "block has been downloaded and extracted", "block", task.block.BlockPath, "directory", directory) | ||
blockQuerier := d.createBlockQuerier(directory) | ||
task.ResultsCh <- blockWithQuerier{ | ||
BlockRef: task.block, | ||
BlockQuerier: blockQuerier, | ||
} | ||
} | ||
} | ||
|
||
func (d *blockDownloader) downloadBlocks(ctx context.Context, tenantID string, references []BlockRef) (chan blockWithQuerier, chan error) { | ||
d.activeUsersService.UpdateUserTimestamp(tenantID, time.Now()) | ||
// we need to have errCh with size that can keep max count of errors to prevent the case when | ||
// the queue worker reported the error to this channel before the current goroutine | ||
// and this goroutine will go to the deadlock because it won't be able to report an error | ||
// because nothing reads this channel at this moment. | ||
errCh := make(chan error, len(references)) | ||
Comment on lines
+151
to
+155
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not sure if I understand that correctly. On line 168 you return after sending the first and only error to the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this channel is also passed to the queue and queue workers will report errors to this channel also... |
||
blocksCh := make(chan blockWithQuerier, len(references)) | ||
|
||
downloadingParallelism := d.limits.BloomGatewayBlocksDownloadingParallelism(tenantID) | ||
for _, reference := range references { | ||
task := NewBlockDownloadingTask(ctx, reference, blocksCh, errCh) | ||
level.Debug(d.logger).Log("msg", "enqueuing task to download block", "block", reference.BlockPath) | ||
err := d.queue.Enqueue(tenantID, nil, task, downloadingParallelism, nil) | ||
if err != nil { | ||
errCh <- fmt.Errorf("error enquing downloading task for block %s : %w", reference.BlockPath, err) | ||
return blocksCh, errCh | ||
} | ||
} | ||
return blocksCh, errCh | ||
} | ||
|
||
type blockWithQuerier struct { | ||
BlockRef | ||
*v1.BlockQuerier | ||
} | ||
|
||
// extract the files into directory and returns absolute path to this directory. | ||
func (d *blockDownloader) extractBlock(block *Block, ts time.Time) (string, error) { | ||
workingDirectoryPath := filepath.Join(d.workingDirectory, block.BlockPath, strconv.FormatInt(ts.UnixMilli(), 10)) | ||
err := os.MkdirAll(workingDirectoryPath, os.ModePerm) | ||
if err != nil { | ||
return "", fmt.Errorf("can not create directory to extract the block: %w", err) | ||
} | ||
archivePath, err := writeDataToTempFile(workingDirectoryPath, block) | ||
if err != nil { | ||
return "", fmt.Errorf("error writing data to temp file: %w", err) | ||
} | ||
defer func() { | ||
os.Remove(archivePath) | ||
// todo log err | ||
}() | ||
err = extractArchive(archivePath, workingDirectoryPath) | ||
if err != nil { | ||
return "", fmt.Errorf("error extracting archive: %w", err) | ||
} | ||
return workingDirectoryPath, nil | ||
} | ||
|
||
func (d *blockDownloader) createBlockQuerier(directory string) *v1.BlockQuerier { | ||
reader := v1.NewDirectoryBlockReader(directory) | ||
block := v1.NewBlock(reader) | ||
return v1.NewBlockQuerier(block) | ||
} | ||
|
||
func (d *blockDownloader) stop() { | ||
_ = services.StopManagerAndAwaitStopped(d.ctx, d.manager) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe we should log the error? |
||
} | ||
|
||
func writeDataToTempFile(workingDirectoryPath string, block *Block) (string, error) { | ||
vlad-diachenko marked this conversation as resolved.
Show resolved
Hide resolved
|
||
defer block.Data.Close() | ||
archivePath := filepath.Join(workingDirectoryPath, block.BlockPath[strings.LastIndex(block.BlockPath, delimiter)+1:]) | ||
|
||
archiveFile, err := os.Create(archivePath) | ||
if err != nil { | ||
return "", fmt.Errorf("error creating empty file to store the archiver: %w", err) | ||
} | ||
defer archiveFile.Close() | ||
_, err = io.Copy(archiveFile, block.Data) | ||
if err != nil { | ||
return "", fmt.Errorf("error writing data to archive file: %w", err) | ||
} | ||
return archivePath, nil | ||
} | ||
|
||
func extractArchive(archivePath string, workingDirectoryPath string) error { | ||
file, err := os.Open(archivePath) | ||
if err != nil { | ||
return fmt.Errorf("error opening archive file %s: %w", file.Name(), err) | ||
} | ||
return v1.UnTarGz(workingDirectoryPath, file) | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.