From 54b9b7ef778da5d4edd2aed6adee3749241dbdf7 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 21 Oct 2024 10:26:52 -0700 Subject: [PATCH 01/56] blockbuilder pkg init Signed-off-by: Owen Diehl --- pkg/blockbuilder/blockbuilder.go | 1 + pkg/blockbuilder/plan.txt | 25 +++++++++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 pkg/blockbuilder/blockbuilder.go create mode 100644 pkg/blockbuilder/plan.txt diff --git a/pkg/blockbuilder/blockbuilder.go b/pkg/blockbuilder/blockbuilder.go new file mode 100644 index 0000000000000..2ffd21885f3ff --- /dev/null +++ b/pkg/blockbuilder/blockbuilder.go @@ -0,0 +1 @@ +package blockbuilder diff --git a/pkg/blockbuilder/plan.txt b/pkg/blockbuilder/plan.txt new file mode 100644 index 0000000000000..df28c3ee6927c --- /dev/null +++ b/pkg/blockbuilder/plan.txt @@ -0,0 +1,25 @@ +# Purpose +blockbuilder is responsible for consuming ingested data in the queue (kafka, etc) and writing it in an optimized form to long term storage. While this should always remain true, it can be built and iterated upon in phases. First, let's look at the simplest possible architecture: + +* [interface] loads "jobs": partitions+offset ranges in kafka +* For each job, process data, building the storage format +* [interface] Upon completion (inc flushing to storage), commit work + * e.g. update consumer group processed offset in kafka +* consumes + +# First Impl: Alongside existing multi-zone ingester writers +Goal: modify ingester architecture towards RF1, but don't actually write to storage yet, b/c we haven't solved coordinating interim reads/writes. +Deliverable: RF1 metrics proof +* run replicas==partitions (from ingesters) +* run every $INTERVAL (5m?), +* slim down ingester write path + * remove disk (all WALs). + * ignore limits if too complex (for now) + * /dev/null backend + +# Second Impl: Foo + +* metadata store + * include offsets committed for coordination b/w ingester-readers & long term storage +* scheduler architecture +* shuffle sharding \ No newline at end of file From 4edccef07cc2d97ff97a6c96f2210b7d10de6483 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 21 Oct 2024 15:56:53 -0700 Subject: [PATCH 02/56] specing out blockbuilder pkg Signed-off-by: Owen Diehl --- pkg/blockbuilder/blockbuilder.go | 74 ++++++++++++++++++++++++++++++++ pkg/blockbuilder/plan.txt | 15 +++++-- pkg/kafka/partition/reader.go | 10 ++--- 3 files changed, 91 insertions(+), 8 deletions(-) diff --git a/pkg/blockbuilder/blockbuilder.go b/pkg/blockbuilder/blockbuilder.go index 2ffd21885f3ff..1b77370f8cdf5 100644 --- a/pkg/blockbuilder/blockbuilder.go +++ b/pkg/blockbuilder/blockbuilder.go @@ -1 +1,75 @@ package blockbuilder + +import ( + "context" + + "github.com/grafana/loki/v3/pkg/kafka/partition" +) + +// [min,max) +type Offsets struct { + Min, Max int64 +} + +type Job struct { + Partition int32 + Offsets Offsets +} + +// PartitionJobController loads a single job a time, bound to a given +// * topic +// * partition +// * offset_step_len: the number of offsets each job to contain. e.g. "10" could yield a job w / min=15, max=25 +// +// At a high level, it watches a source topic/partition (where log data is ingested) and a "committed" topic/partition. +// The "comitted" partition corresponds to the offsets from the source partition which have been committed to object storage. +// In essence, the following loop is performed +// 1. load the most recent record from the "comitted" partition. This contains the highest msg offset in the "source" partition +// that has been committed to object storage. We'll call that $START_POS. +// 2. Create a job with `min=$START_POS+1,end=$START_POS+1+$STEP_LEN` +// 3. Sometime later when the job has been processed, we'll commit the final processed offset from the "source" partition +// (which will be <= $END_POS). +type PartitionJobController struct { + topic string + partition int32 + stepLen int64 + source partiotionReader + committed partitionRW +} + +// LoadJob(ctx) returns the next job by finding the most recent unconsumed offset in the partition +func (l *PartitionJobController) LoadJob(ctx context.Context) (Job, error) { + // Read the most recent committed offset + record, err := l.committed.ReadMostRecent(ctx) + if err != nil { + return Job{}, err + } + + // Calculate the start offset for the new job + startOffset := record.Offset + 1 + + // Create the job with the calculated offsets + job := Job{ + Partition: l.partition, + Offsets: Offsets{ + Min: startOffset, + Max: startOffset + l.stepLen, + }, + } + + return job, nil +} + +// CommitJob commits the processed offsets of a job to the committed partition +func (l *PartitionJobController) CommitJob(ctx context.Context, job Job) error { + return l.committed.Commit(ctx, job.Offsets) +} + +type partitionRW interface { + ReadMostRecent(context.Context) (partition.Record, error) + Commit(context.Context, Offsets) error + partiotionReader +} + +// TODO +type partiotionReader interface{} diff --git a/pkg/blockbuilder/plan.txt b/pkg/blockbuilder/plan.txt index df28c3ee6927c..a3a80a735fa4d 100644 --- a/pkg/blockbuilder/plan.txt +++ b/pkg/blockbuilder/plan.txt @@ -17,9 +17,18 @@ Deliverable: RF1 metrics proof * ignore limits if too complex (for now) * /dev/null backend -# Second Impl: Foo +# TODO improvements * metadata store * include offsets committed for coordination b/w ingester-readers & long term storage -* scheduler architecture -* shuffle sharding \ No newline at end of file +* planner/scheduler+worker architecture +* shuffle sharding + + +# Things to solve +* limits application +* job sizing -- coordinate kafka offsets w/ underlying bytes added? + * ideally we can ask kafka for "the next 1GB" in a partition, but to do this we'd need the kafka offsets (auto-incremented integers for messages within a partition) to be derived from the message size. Right now, different batch sizes can cause kafka msgs to have very different sizes. + * idea: another set of partitions to store offsets->datasize? Sounds shitty tbh & breaks the consistency bound on writes (what if kafka acks first write but doesnt ack the second?) + * what if we stored byte counter metadata in kafka records so we could O(log(n)) seek an offset range w/ the closest $SIZE + * Likely a reasonable perf tradeoff as this isn't called often (only in job planner in the future). \ No newline at end of file diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index e364f3bba748e..567a3553ce216 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -101,7 +101,7 @@ func (p *Reader) start(ctx context.Context) error { } // We manage our commits manually, so we must fetch the last offset for our consumer group to find out where to read from. - lastCommittedOffset := p.fetchLastCommittedOffset(ctx) + lastCommittedOffset := p.FetchLastCommittedOffset(ctx) p.client.AddConsumePartitions(map[string]map[int32]kgo.Offset{ p.kafkaCfg.Topic: {p.partitionID: kgo.NewOffset().At(lastCommittedOffset)}, }) @@ -157,7 +157,7 @@ func (p *Reader) run(ctx context.Context) error { return nil } -func (p *Reader) fetchLastCommittedOffset(ctx context.Context) int64 { +func (p *Reader) FetchLastCommittedOffset(ctx context.Context) int64 { // We manually create a request so that we can request the offset for a single partition // only, which is more performant than requesting the offsets for all partitions. req := kmsg.NewPtrOffsetFetchRequest() @@ -212,7 +212,7 @@ func (p *Reader) fetchLastCommittedOffset(ctx context.Context) int64 { return fetchRes.Groups[0].Topics[0].Partitions[0].Offset } -func (p *Reader) fetchPartitionOffset(ctx context.Context, position int64) (int64, error) { +func (p *Reader) FetchPartitionOffset(ctx context.Context, position int64) (int64, error) { // Create a custom request to fetch the latest offset of a specific partition. // We manually create a request so that we can request the offset for a single partition // only, which is more performant than requesting the offsets for all partitions. @@ -339,7 +339,7 @@ func (p *Reader) processNextFetchesUntilLagHonored(ctx context.Context, maxLag t for boff.Ongoing() { // Send a direct request to the Kafka backend to fetch the partition start offset. - partitionStartOffset, err := p.fetchPartitionOffset(ctx, kafkaStartOffset) + partitionStartOffset, err := p.FetchPartitionOffset(ctx, kafkaStartOffset) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch partition start offset", "err", err) boff.Wait() @@ -350,7 +350,7 @@ func (p *Reader) processNextFetchesUntilLagHonored(ctx context.Context, maxLag t // We intentionally don't use WaitNextFetchLastProducedOffset() to not introduce further // latency. lastProducedOffsetRequestedAt := time.Now() - lastProducedOffset, err := p.fetchPartitionOffset(ctx, kafkaEndOffset) + lastProducedOffset, err := p.FetchPartitionOffset(ctx, kafkaEndOffset) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch last produced offset", "err", err) boff.Wait() From cd65c1fae799888fffbe5a09eea577ca79c67061 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 21 Oct 2024 18:25:29 -0700 Subject: [PATCH 03/56] unexport Signed-off-by: Owen Diehl --- pkg/kafka/partition/reader.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index 567a3553ce216..e364f3bba748e 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -101,7 +101,7 @@ func (p *Reader) start(ctx context.Context) error { } // We manage our commits manually, so we must fetch the last offset for our consumer group to find out where to read from. - lastCommittedOffset := p.FetchLastCommittedOffset(ctx) + lastCommittedOffset := p.fetchLastCommittedOffset(ctx) p.client.AddConsumePartitions(map[string]map[int32]kgo.Offset{ p.kafkaCfg.Topic: {p.partitionID: kgo.NewOffset().At(lastCommittedOffset)}, }) @@ -157,7 +157,7 @@ func (p *Reader) run(ctx context.Context) error { return nil } -func (p *Reader) FetchLastCommittedOffset(ctx context.Context) int64 { +func (p *Reader) fetchLastCommittedOffset(ctx context.Context) int64 { // We manually create a request so that we can request the offset for a single partition // only, which is more performant than requesting the offsets for all partitions. req := kmsg.NewPtrOffsetFetchRequest() @@ -212,7 +212,7 @@ func (p *Reader) FetchLastCommittedOffset(ctx context.Context) int64 { return fetchRes.Groups[0].Topics[0].Partitions[0].Offset } -func (p *Reader) FetchPartitionOffset(ctx context.Context, position int64) (int64, error) { +func (p *Reader) fetchPartitionOffset(ctx context.Context, position int64) (int64, error) { // Create a custom request to fetch the latest offset of a specific partition. // We manually create a request so that we can request the offset for a single partition // only, which is more performant than requesting the offsets for all partitions. @@ -339,7 +339,7 @@ func (p *Reader) processNextFetchesUntilLagHonored(ctx context.Context, maxLag t for boff.Ongoing() { // Send a direct request to the Kafka backend to fetch the partition start offset. - partitionStartOffset, err := p.FetchPartitionOffset(ctx, kafkaStartOffset) + partitionStartOffset, err := p.fetchPartitionOffset(ctx, kafkaStartOffset) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch partition start offset", "err", err) boff.Wait() @@ -350,7 +350,7 @@ func (p *Reader) processNextFetchesUntilLagHonored(ctx context.Context, maxLag t // We intentionally don't use WaitNextFetchLastProducedOffset() to not introduce further // latency. lastProducedOffsetRequestedAt := time.Now() - lastProducedOffset, err := p.FetchPartitionOffset(ctx, kafkaEndOffset) + lastProducedOffset, err := p.fetchPartitionOffset(ctx, kafkaEndOffset) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch last produced offset", "err", err) boff.Wait() From 838f95fcc954e48629e412c0b03aa836cce76f40 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 21 Oct 2024 18:25:47 -0700 Subject: [PATCH 04/56] job building Signed-off-by: Owen Diehl --- pkg/blockbuilder/blockbuilder.go | 75 --------- pkg/blockbuilder/partition.go | 253 +++++++++++++++++++++++++++++++ 2 files changed, 253 insertions(+), 75 deletions(-) delete mode 100644 pkg/blockbuilder/blockbuilder.go create mode 100644 pkg/blockbuilder/partition.go diff --git a/pkg/blockbuilder/blockbuilder.go b/pkg/blockbuilder/blockbuilder.go deleted file mode 100644 index 1b77370f8cdf5..0000000000000 --- a/pkg/blockbuilder/blockbuilder.go +++ /dev/null @@ -1,75 +0,0 @@ -package blockbuilder - -import ( - "context" - - "github.com/grafana/loki/v3/pkg/kafka/partition" -) - -// [min,max) -type Offsets struct { - Min, Max int64 -} - -type Job struct { - Partition int32 - Offsets Offsets -} - -// PartitionJobController loads a single job a time, bound to a given -// * topic -// * partition -// * offset_step_len: the number of offsets each job to contain. e.g. "10" could yield a job w / min=15, max=25 -// -// At a high level, it watches a source topic/partition (where log data is ingested) and a "committed" topic/partition. -// The "comitted" partition corresponds to the offsets from the source partition which have been committed to object storage. -// In essence, the following loop is performed -// 1. load the most recent record from the "comitted" partition. This contains the highest msg offset in the "source" partition -// that has been committed to object storage. We'll call that $START_POS. -// 2. Create a job with `min=$START_POS+1,end=$START_POS+1+$STEP_LEN` -// 3. Sometime later when the job has been processed, we'll commit the final processed offset from the "source" partition -// (which will be <= $END_POS). -type PartitionJobController struct { - topic string - partition int32 - stepLen int64 - source partiotionReader - committed partitionRW -} - -// LoadJob(ctx) returns the next job by finding the most recent unconsumed offset in the partition -func (l *PartitionJobController) LoadJob(ctx context.Context) (Job, error) { - // Read the most recent committed offset - record, err := l.committed.ReadMostRecent(ctx) - if err != nil { - return Job{}, err - } - - // Calculate the start offset for the new job - startOffset := record.Offset + 1 - - // Create the job with the calculated offsets - job := Job{ - Partition: l.partition, - Offsets: Offsets{ - Min: startOffset, - Max: startOffset + l.stepLen, - }, - } - - return job, nil -} - -// CommitJob commits the processed offsets of a job to the committed partition -func (l *PartitionJobController) CommitJob(ctx context.Context, job Job) error { - return l.committed.Commit(ctx, job.Offsets) -} - -type partitionRW interface { - ReadMostRecent(context.Context) (partition.Record, error) - Commit(context.Context, Offsets) error - partiotionReader -} - -// TODO -type partiotionReader interface{} diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go new file mode 100644 index 0000000000000..be343f6444cdb --- /dev/null +++ b/pkg/blockbuilder/partition.go @@ -0,0 +1,253 @@ +package blockbuilder + +import ( + "context" + "errors" + "fmt" + "time" + + "github.com/grafana/dskit/backoff" + "github.com/grafana/loki/v3/pkg/kafka/partition" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + "github.com/twmb/franz-go/pkg/kadm" + "github.com/twmb/franz-go/pkg/kerr" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/kmsg" +) + +const ( + kafkaStartOffset = -2 + kafkaEndOffset = -1 +) + +var defaultBackoffConfig = backoff.Config{ + MinBackoff: 100 * time.Millisecond, + MaxBackoff: time.Second, + MaxRetries: 0, // Retry forever (unless context is canceled / deadline exceeded). +} + +// [min,max) +type Offsets struct { + Min, Max int64 +} + +type Job struct { + Partition int32 + Offsets Offsets +} + +// PartitionJobController loads a single job a time, bound to a given +// * topic +// * partition +// * offset_step_len: the number of offsets each job to contain. e.g. "10" could yield a job w / min=15, max=25 +// +// At a high level, it watches a source topic/partition (where log data is ingested) and a "committed" topic/partition. +// The "comitted" partition corresponds to the offsets from the source partition which have been committed to object storage. +// In essence, the following loop is performed +// 1. load the most recent record from the "comitted" partition. This contains the highest msg offset in the "source" partition +// that has been committed to object storage. We'll call that $START_POS. +// 2. Create a job with `min=$START_POS+1,end=$START_POS+1+$STEP_LEN` +// 3. Sometime later when the job has been processed, we'll commit the final processed offset from the "source" partition (which +// will be <= $END_POS) to the "committed" partition. +// +// NB(owen-d): In our case, "source" is the partition +// +// containing log data and "committed" is the consumer group +type PartitionJobController struct { + topic string + partition int32 + stepLen int64 + part PartitionController +} + +// LoadJob(ctx) returns the next job by finding the most recent unconsumed offset in the partition +func (l *PartitionJobController) LoadJob(ctx context.Context) (Job, error) { + // Read the most recent committed offset + startOffset, err := l.part.HighestCommittedOffset(ctx) + if err != nil { + return Job{}, err + } + + // Create the job with the calculated offsets + job := Job{ + Partition: l.partition, + Offsets: Offsets{ + Min: startOffset, + Max: startOffset + l.stepLen, + }, + } + + return job, nil +} + +type PartitionController interface { + // Returns the highest committed offset from the consumer group + HighestCommittedOffset(ctx context.Context) (int64, error) + // Commits the offset to the consumer group. + Commit(context.Context, int64) error + // Process will run load batches at a time and send them to channel, + // so it's advised to not buffer the channel for natural backpressure. + // As a convenience, it returns the last seen offset, which matches + // the final record sent on the channel. + Process(context.Context, Offsets, chan<- []partition.Record) (int64, error) +} + +type partitionReader struct { + topic string + group string + partitionID int32 + + logger log.Logger + client *kgo.Client + aClient *kadm.Client + reg prometheus.Registerer +} + +// Fetches the desired offset in the partition itself, not the consumer group +// NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` +func (r *partitionReader) fetchPartitionOffset(ctx context.Context, position int64) (int64, error) { + // Create a custom request to fetch the latest offset of a specific partition. + // We manually create a request so that we can request the offset for a single partition + // only, which is more performant than requesting the offsets for all partitions. + partitionReq := kmsg.NewListOffsetsRequestTopicPartition() + partitionReq.Partition = r.partitionID + partitionReq.Timestamp = position + + topicReq := kmsg.NewListOffsetsRequestTopic() + topicReq.Topic = r.topic + topicReq.Partitions = []kmsg.ListOffsetsRequestTopicPartition{partitionReq} + + req := kmsg.NewPtrListOffsetsRequest() + req.IsolationLevel = 0 // 0 means READ_UNCOMMITTED. + req.Topics = []kmsg.ListOffsetsRequestTopic{topicReq} + + // Even if we share the same client, other in-flight requests are not canceled once this context is canceled + // (or its deadline is exceeded). We've verified it with a unit test. + resps := r.client.RequestSharded(ctx, req) + + // Since we issued a request for only 1 partition, we expect exactly 1 response. + if expected := 1; len(resps) != 1 { + return 0, fmt.Errorf("unexpected number of responses (expected: %d, got: %d)", expected, len(resps)) + } + + // Ensure no error occurred. + res := resps[0] + if res.Err != nil { + return 0, res.Err + } + + // Parse the response. + listRes, ok := res.Resp.(*kmsg.ListOffsetsResponse) + if !ok { + return 0, errors.New("unexpected response type") + } + if expected, actual := 1, len(listRes.Topics); actual != expected { + return 0, fmt.Errorf("unexpected number of topics in the response (expected: %d, got: %d)", expected, actual) + } + if expected, actual := r.topic, listRes.Topics[0].Topic; expected != actual { + return 0, fmt.Errorf("unexpected topic in the response (expected: %s, got: %s)", expected, actual) + } + if expected, actual := 1, len(listRes.Topics[0].Partitions); actual != expected { + return 0, fmt.Errorf("unexpected number of partitions in the response (expected: %d, got: %d)", expected, actual) + } + if expected, actual := r.partitionID, listRes.Topics[0].Partitions[0].Partition; actual != expected { + return 0, fmt.Errorf("unexpected partition in the response (expected: %d, got: %d)", expected, actual) + } + if err := kerr.ErrorForCode(listRes.Topics[0].Partitions[0].ErrorCode); err != nil { + return 0, err + } + + return listRes.Topics[0].Partitions[0].Offset, nil +} + +// Fetches the highest committe offset in the consumer group +// NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` +// TODO(owen-d): expose errors: the failure case of restarting at +// the beginning of a partition is costly and duplicates data +func (r *partitionReader) fetchLastCommittedOffset(ctx context.Context) int64 { + // We manually create a request so that we can request the offset for a single partition + // only, which is more performant than requesting the offsets for all partitions. + req := kmsg.NewPtrOffsetFetchRequest() + req.Topics = []kmsg.OffsetFetchRequestTopic{{Topic: r.topic, Partitions: []int32{r.partitionID}}} + req.Group = r.group + + resps := r.client.RequestSharded(ctx, req) + + // Since we issued a request for only 1 partition, we expect exactly 1 response. + if expected, actual := 1, len(resps); actual != expected { + level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected number of responses (expected: %d, got: %d)", expected, actual), "expected", expected, "actual", len(resps)) + return kafkaStartOffset + } + // Ensure no error occurred. + res := resps[0] + if res.Err != nil { + level.Error(r.logger).Log("msg", "error fetching group offset for partition", "err", res.Err) + return kafkaStartOffset + } + + // Parse the response. + fetchRes, ok := res.Resp.(*kmsg.OffsetFetchResponse) + if !ok { + level.Error(r.logger).Log("msg", "unexpected response type") + return kafkaStartOffset + } + if expected, actual := 1, len(fetchRes.Groups); actual != expected { + level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected number of groups in the response (expected: %d, got: %d)", expected, actual)) + return kafkaStartOffset + } + if expected, actual := 1, len(fetchRes.Groups[0].Topics); actual != expected { + level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected number of topics in the response (expected: %d, got: %d)", expected, actual)) + return kafkaStartOffset + } + if expected, actual := r.topic, fetchRes.Groups[0].Topics[0].Topic; expected != actual { + level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected topic in the response (expected: %s, got: %s)", expected, actual)) + return kafkaStartOffset + } + if expected, actual := 1, len(fetchRes.Groups[0].Topics[0].Partitions); actual != expected { + level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected number of partitions in the response (expected: %d, got: %d)", expected, actual)) + return kafkaStartOffset + } + if expected, actual := r.partitionID, fetchRes.Groups[0].Topics[0].Partitions[0].Partition; actual != expected { + level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected partition in the response (expected: %d, got: %d)", expected, actual)) + return kafkaStartOffset + } + if err := kerr.ErrorForCode(fetchRes.Groups[0].Topics[0].Partitions[0].ErrorCode); err != nil { + level.Error(r.logger).Log("msg", "unexpected error in the response", "err", err) + return kafkaStartOffset + } + + return fetchRes.Groups[0].Topics[0].Partitions[0].Offset +} + +func (r *partitionReader) HighestCommittedOffset(ctx context.Context) (int64, error) { + return withBackoff( + ctx, + defaultBackoffConfig, + func() (int64, error) { + return r.fetchLastCommittedOffset(ctx), nil + }, + ) +} + +func withBackoff[T any]( + ctx context.Context, + config backoff.Config, + fn func() (T, error), +) (T, error) { + var zero T + + var boff = backoff.New(ctx, config) + for boff.Ongoing() { + res, err := fn() + if err != nil { + boff.Wait() + continue + } + return res, nil + } + + return zero, boff.ErrCause() +} From 62ca1d768ce10aa63f566bf5a7ca42b9712e7fcd Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 21 Oct 2024 19:01:23 -0700 Subject: [PATCH 05/56] polling/process Signed-off-by: Owen Diehl --- pkg/blockbuilder/partition.go | 141 ++++++++++++++++++++++++++++++++-- pkg/kafka/partition/reader.go | 56 +++++++------- 2 files changed, 163 insertions(+), 34 deletions(-) diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index be343f6444cdb..16a4e6d0836bc 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -6,16 +6,17 @@ import ( "fmt" "time" - "github.com/grafana/dskit/backoff" - "github.com/grafana/loki/v3/pkg/kafka/partition" - "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/dskit/backoff" + "github.com/grafana/dskit/multierror" "github.com/prometheus/client_golang/prometheus" "github.com/twmb/franz-go/pkg/kadm" "github.com/twmb/franz-go/pkg/kerr" "github.com/twmb/franz-go/pkg/kgo" "github.com/twmb/franz-go/pkg/kmsg" + + "github.com/grafana/loki/v3/pkg/kafka/partition" ) const ( @@ -64,11 +65,21 @@ type PartitionJobController struct { } // LoadJob(ctx) returns the next job by finding the most recent unconsumed offset in the partition -func (l *PartitionJobController) LoadJob(ctx context.Context) (Job, error) { +// Returns whether an applicable job exists, the job, and an error +func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) { // Read the most recent committed offset startOffset, err := l.part.HighestCommittedOffset(ctx) if err != nil { - return Job{}, err + return false, Job{}, err + } + + highestOffset, err := l.part.HighestPartitionOffset(ctx) + if err != nil { + return false, Job{}, err + } + + if highestOffset == startOffset { + return false, Job{}, nil } // Create the job with the calculated offsets @@ -80,12 +91,14 @@ func (l *PartitionJobController) LoadJob(ctx context.Context) (Job, error) { }, } - return job, nil + return true, job, nil } type PartitionController interface { // Returns the highest committed offset from the consumer group HighestCommittedOffset(ctx context.Context) (int64, error) + // Returns the highest available offset in the partition + HighestPartitionOffset(ctx context.Context) (int64, error) // Commits the offset to the consumer group. Commit(context.Context, int64) error // Process will run load batches at a time and send them to channel, @@ -93,6 +106,8 @@ type PartitionController interface { // As a convenience, it returns the last seen offset, which matches // the final record sent on the channel. Process(context.Context, Offsets, chan<- []partition.Record) (int64, error) + + Close() error } type partitionReader struct { @@ -100,6 +115,7 @@ type partitionReader struct { group string partitionID int32 + metrics partition.ReaderMetrics logger log.Logger client *kgo.Client aClient *kadm.Client @@ -222,6 +238,12 @@ func (r *partitionReader) fetchLastCommittedOffset(ctx context.Context) int64 { return fetchRes.Groups[0].Topics[0].Partitions[0].Offset } +func (r *partitionReader) updateReaderOffset(offset int64) { + r.client.AddConsumePartitions(map[string]map[int32]kgo.Offset{ + r.topic: {r.partitionID: kgo.NewOffset().At(offset)}, + }) +} + func (r *partitionReader) HighestCommittedOffset(ctx context.Context) (int64, error) { return withBackoff( ctx, @@ -232,6 +254,113 @@ func (r *partitionReader) HighestCommittedOffset(ctx context.Context) (int64, er ) } +func (r *partitionReader) HighestPartitionOffset(ctx context.Context) (int64, error) { + return withBackoff( + ctx, + defaultBackoffConfig, + func() (int64, error) { + return r.fetchPartitionOffset(ctx, kafkaEndOffset) + }, + ) +} + +// pollFetches retrieves the next batch of records from Kafka and measures the fetch duration. +// NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` +func (p *partitionReader) poll(ctx context.Context) []partition.Record { + defer func(start time.Time) { + p.metrics.FetchWaitDuration.Observe(time.Since(start).Seconds()) + }(time.Now()) + fetches := p.client.PollFetches(ctx) + p.recordFetchesMetrics(fetches) + p.logFetchErrors(fetches) + fetches = partition.FilterOutErrFetches(fetches) + if fetches.NumRecords() == 0 { + return nil + } + records := make([]partition.Record, 0, fetches.NumRecords()) + fetches.EachRecord(func(rec *kgo.Record) { + if rec.Partition != p.partitionID { + level.Error(p.logger).Log("msg", "wrong partition record received", "partition", rec.Partition, "expected_partition", p.partitionID) + return + } + records = append(records, partition.Record{ + // This context carries the tracing data for this individual record; + // kotel populates this data when it fetches the messages. + Ctx: rec.Context, + TenantID: string(rec.Key), + Content: rec.Value, + Offset: rec.Offset, + }) + }) + return records +} + +// logFetchErrors logs any errors encountered during the fetch operation. +func (p *partitionReader) logFetchErrors(fetches kgo.Fetches) { + mErr := multierror.New() + fetches.EachError(func(topic string, partition int32, err error) { + if errors.Is(err, context.Canceled) { + return + } + + // kgo advises to "restart" the kafka client if the returned error is a kerr.Error. + // Recreating the client would cause duplicate metrics registration, so we don't do it for now. + mErr.Add(fmt.Errorf("topic %q, partition %d: %w", topic, partition, err)) + }) + if len(mErr) == 0 { + return + } + p.metrics.FetchesErrors.Add(float64(len(mErr))) + level.Error(p.logger).Log("msg", "encountered error while fetching", "err", mErr.Err()) +} + +// recordFetchesMetrics updates various metrics related to the fetch operation. +// NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` +func (p *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { + var ( + now = time.Now() + numRecords = 0 + ) + fetches.EachRecord(func(record *kgo.Record) { + numRecords++ + delay := now.Sub(record.Timestamp).Seconds() + p.metrics.ReceiveDelayWhenRunning.Observe(delay) + }) + + p.metrics.FetchesTotal.Add(float64(len(fetches))) + p.metrics.RecordsPerFetch.Observe(float64(numRecords)) +} + +func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan<- []partition.Record) (int64, error) { + r.updateReaderOffset(offsets.Min) + + var lastOffset int64 = offsets.Min - 1 + + _, err := withBackoff( + ctx, + defaultBackoffConfig, + func() (struct{}, error) { + fetches := r.poll(ctx) + if len(fetches) > 0 { + lastOffset = fetches[len(fetches)-1].Offset + select { + case ch <- fetches: + case <-ctx.Done(): + } + } + return struct{}{}, nil + }, + ) + + return lastOffset, err +} + +func (r *partitionReader) Close() error { + r.aClient.Close() + r.client.Close() + return nil +} + func withBackoff[T any]( ctx context.Context, config backoff.Config, diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index e364f3bba748e..c5f09cd4bf8e5 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -48,7 +48,7 @@ type Reader struct { client *kgo.Client logger log.Logger - metrics readerMetrics + metrics ReaderMetrics reg prometheus.Registerer clock quartz.Clock } @@ -82,7 +82,7 @@ func NewReader( partitionID: partitionID, consumerGroup: kafkaCfg.GetConsumerGroup(instanceID, partitionID), logger: logger, - metrics: newReaderMetrics(reg), + metrics: NewReaderMetrics(reg), reg: reg, lastProcessedOffset: -1, consumerFactory: consumerFactory, @@ -433,19 +433,19 @@ func (p *Reader) logFetchErrors(fetches kgo.Fetches) { if len(mErr) == 0 { return } - p.metrics.fetchesErrors.Add(float64(len(mErr))) + p.metrics.FetchesErrors.Add(float64(len(mErr))) level.Error(p.logger).Log("msg", "encountered error while fetching", "err", mErr.Err()) } // pollFetches retrieves the next batch of records from Kafka and measures the fetch duration. func (p *Reader) poll(ctx context.Context) []Record { defer func(start time.Time) { - p.metrics.fetchWaitDuration.Observe(time.Since(start).Seconds()) + p.metrics.FetchWaitDuration.Observe(time.Since(start).Seconds()) }(time.Now()) fetches := p.client.PollFetches(ctx) p.recordFetchesMetrics(fetches) p.logFetchErrors(fetches) - fetches = filterOutErrFetches(fetches) + fetches = FilterOutErrFetches(fetches) if fetches.NumRecords() == 0 { return nil } @@ -478,18 +478,18 @@ func (p *Reader) recordFetchesMetrics(fetches kgo.Fetches) { numRecords++ delay := now.Sub(record.Timestamp).Seconds() if p.lastProcessedOffset == -1 { - p.metrics.receiveDelayWhenStarting.Observe(delay) + p.metrics.ReceiveDelayWhenStarting.Observe(delay) } else { - p.metrics.receiveDelayWhenRunning.Observe(delay) + p.metrics.ReceiveDelayWhenRunning.Observe(delay) } }) - p.metrics.fetchesTotal.Add(float64(len(fetches))) - p.metrics.recordsPerFetch.Observe(float64(numRecords)) + p.metrics.FetchesTotal.Add(float64(len(fetches))) + p.metrics.RecordsPerFetch.Observe(float64(numRecords)) } -// filterOutErrFetches removes any fetches that resulted in errors from the provided slice. -func filterOutErrFetches(fetches kgo.Fetches) kgo.Fetches { +// FilterOutErrFetches removes any fetches that resulted in errors from the provided slice. +func FilterOutErrFetches(fetches kgo.Fetches) kgo.Fetches { filtered := make(kgo.Fetches, 0, len(fetches)) for i, fetch := range fetches { if !isErrFetch(fetch) { @@ -512,21 +512,21 @@ func isErrFetch(fetch kgo.Fetch) bool { return false } -type readerMetrics struct { - receiveDelayWhenStarting prometheus.Observer - receiveDelayWhenRunning prometheus.Observer - recordsPerFetch prometheus.Histogram - fetchesErrors prometheus.Counter - fetchesTotal prometheus.Counter - fetchWaitDuration prometheus.Histogram +type ReaderMetrics struct { + ReceiveDelayWhenStarting prometheus.Observer + ReceiveDelayWhenRunning prometheus.Observer + RecordsPerFetch prometheus.Histogram + FetchesErrors prometheus.Counter + FetchesTotal prometheus.Counter + FetchWaitDuration prometheus.Histogram // strongConsistencyInstrumentation *StrongReadConsistencyInstrumentation[struct{}] // lastConsumedOffset prometheus.Gauge - consumeLatency prometheus.Histogram + ConsumeLatency prometheus.Histogram kprom *kprom.Metrics } -// newReaderMetrics initializes and returns a new set of metrics for the PartitionReader. -func newReaderMetrics(reg prometheus.Registerer) readerMetrics { +// NewReaderMetrics initializes and returns a new set of metrics for the PartitionReader. +func NewReaderMetrics(reg prometheus.Registerer) ReaderMetrics { receiveDelay := promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ Name: "loki_ingest_storage_reader_receive_delay_seconds", Help: "Delay between producing a record and receiving it in the consumer.", @@ -537,25 +537,25 @@ func newReaderMetrics(reg prometheus.Registerer) readerMetrics { Buckets: prometheus.ExponentialBuckets(0.125, 2, 18), // Buckets between 125ms and 9h. }, []string{"phase"}) - return readerMetrics{ - receiveDelayWhenStarting: receiveDelay.WithLabelValues("starting"), - receiveDelayWhenRunning: receiveDelay.WithLabelValues("running"), + return ReaderMetrics{ + ReceiveDelayWhenStarting: receiveDelay.WithLabelValues("starting"), + ReceiveDelayWhenRunning: receiveDelay.WithLabelValues("running"), kprom: client.NewReaderClientMetrics("partition-reader", reg), - fetchWaitDuration: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + FetchWaitDuration: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ Name: "loki_ingest_storage_reader_records_batch_wait_duration_seconds", Help: "How long a consumer spent waiting for a batch of records from the Kafka client. If fetching is faster than processing, then this will be close to 0.", NativeHistogramBucketFactor: 1.1, }), - recordsPerFetch: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + RecordsPerFetch: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ Name: "loki_ingest_storage_reader_records_per_fetch", Help: "The number of records received by the consumer in a single fetch operation.", Buckets: prometheus.ExponentialBuckets(1, 2, 15), }), - fetchesErrors: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + FetchesErrors: promauto.With(reg).NewCounter(prometheus.CounterOpts{ Name: "loki_ingest_storage_reader_fetch_errors_total", Help: "The number of fetch errors encountered by the consumer.", }), - fetchesTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + FetchesTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ Name: "loki_ingest_storage_reader_fetches_total", Help: "Total number of Kafka fetches received by the consumer.", }), From 79ff4da1bcd8d875eb10f64a37a97792dda4e91d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 22 Oct 2024 11:39:43 -0700 Subject: [PATCH 06/56] sequential polling handling Signed-off-by: Owen Diehl --- pkg/blockbuilder/partition.go | 60 ++++++++++++++++++++++------------- 1 file changed, 38 insertions(+), 22 deletions(-) diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index 16a4e6d0836bc..4255ea8e88cfc 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -265,8 +265,11 @@ func (r *partitionReader) HighestPartitionOffset(ctx context.Context) (int64, er } // pollFetches retrieves the next batch of records from Kafka and measures the fetch duration. -// NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` -func (p *partitionReader) poll(ctx context.Context) []partition.Record { +// NB(owen-d): originally lifted from `pkg/kafka/partition/reader.go:Reader` +func (p *partitionReader) poll( + ctx context.Context, + maxOffset int64, // exclusive +) ([]partition.Record, bool) { defer func(start time.Time) { p.metrics.FetchWaitDuration.Observe(time.Since(start).Seconds()) }(time.Now()) @@ -275,14 +278,22 @@ func (p *partitionReader) poll(ctx context.Context) []partition.Record { p.logFetchErrors(fetches) fetches = partition.FilterOutErrFetches(fetches) if fetches.NumRecords() == 0 { - return nil + return nil, false } records := make([]partition.Record, 0, fetches.NumRecords()) - fetches.EachRecord(func(rec *kgo.Record) { + + itr := fetches.RecordIter() + for !itr.Done() { + rec := itr.Next() if rec.Partition != p.partitionID { level.Error(p.logger).Log("msg", "wrong partition record received", "partition", rec.Partition, "expected_partition", p.partitionID) - return + continue } + + if rec.Offset >= maxOffset { + return records, true + } + records = append(records, partition.Record{ // This context carries the tracing data for this individual record; // kotel populates this data when it fetches the messages. @@ -291,8 +302,9 @@ func (p *partitionReader) poll(ctx context.Context) []partition.Record { Content: rec.Value, Offset: rec.Offset, }) - }) - return records + } + + return records, false } // logFetchErrors logs any errors encountered during the fetch operation. @@ -334,23 +346,27 @@ func (p *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan<- []partition.Record) (int64, error) { r.updateReaderOffset(offsets.Min) - var lastOffset int64 = offsets.Min - 1 + var ( + lastOffset int64 = offsets.Min - 1 + boff = backoff.New(ctx, defaultBackoffConfig) + err error + ) - _, err := withBackoff( - ctx, - defaultBackoffConfig, - func() (struct{}, error) { - fetches := r.poll(ctx) - if len(fetches) > 0 { - lastOffset = fetches[len(fetches)-1].Offset - select { - case ch <- fetches: - case <-ctx.Done(): - } + for boff.Ongoing() { + fetches, done := r.poll(ctx, offsets.Max) + if len(fetches) > 0 { + lastOffset = fetches[len(fetches)-1].Offset + select { + case ch <- fetches: + case <-ctx.Done(): + return lastOffset, ctx.Err() } - return struct{}{}, nil - }, - ) + } + + if done { + break + } + } return lastOffset, err } From a45e5be567a06484433e6d60ce2f20fa275729b0 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 22 Oct 2024 12:12:21 -0700 Subject: [PATCH 07/56] writermetrics kafka pkg Signed-off-by: Owen Diehl --- pkg/kafka/partition/committer.go | 78 ++++++++++++++++++-------------- pkg/kafka/partition/reader.go | 8 ++-- 2 files changed, 47 insertions(+), 39 deletions(-) diff --git a/pkg/kafka/partition/committer.go b/pkg/kafka/partition/committer.go index f9aeda3f0fc51..0a530b23bcd38 100644 --- a/pkg/kafka/partition/committer.go +++ b/pkg/kafka/partition/committer.go @@ -22,13 +22,46 @@ type Committer interface { EnqueueOffset(offset int64) } +type CommitterMetrics struct { + CommitRequestsTotal prometheus.Counter + CommitRequestsLatency prometheus.Histogram + CommitFailuresTotal prometheus.Counter + LastCommittedOffset prometheus.Gauge +} + +func NewCommitterMetrics(reg prometheus.Registerer, partitionID int32) *CommitterMetrics { + return &CommitterMetrics{ + CommitRequestsTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingest_storage_reader_offset_commit_requests_total", + Help: "Total number of requests issued to commit the last consumed offset (includes both successful and failed requests).", + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, + }), + CommitFailuresTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "loki_ingest_storage_reader_offset_commit_failures_total", + Help: "Total number of failed requests to commit the last consumed offset.", + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, + }), + CommitRequestsLatency: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "loki_ingest_storage_reader_offset_commit_request_duration_seconds", + Help: "The duration of requests to commit the last consumed offset.", + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, + NativeHistogramBucketFactor: 1.1, + NativeHistogramMaxBucketNumber: 100, + NativeHistogramMinResetDuration: time.Hour, + Buckets: prometheus.DefBuckets, + }), + LastCommittedOffset: promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Name: "loki_ingest_storage_reader_last_committed_offset", + Help: "The last consumed offset successfully committed by the partition reader. Set to -1 if not offset has been committed yet.", + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, + }), + } +} + // partitionCommitter is responsible for committing offsets for a specific Kafka partition // to the Kafka broker. It also tracks metrics related to the commit process. type partitionCommitter struct { - commitRequestsTotal prometheus.Counter - commitRequestsLatency prometheus.Histogram - commitFailuresTotal prometheus.Counter - lastCommittedOffset prometheus.Gauge + metrics *CommitterMetrics logger log.Logger admClient *kadm.Client @@ -51,35 +84,12 @@ func newCommitter(kafkaCfg kafka.Config, admClient *kadm.Client, partitionID int partitionID: partitionID, consumerGroup: consumerGroup, admClient: admClient, - commitRequestsTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "loki_ingest_storage_reader_offset_commit_requests_total", - Help: "Total number of requests issued to commit the last consumed offset (includes both successful and failed requests).", - ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, - }), - commitFailuresTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "loki_ingest_storage_reader_offset_commit_failures_total", - Help: "Total number of failed requests to commit the last consumed offset.", - ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, - }), - commitRequestsLatency: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ - Name: "loki_ingest_storage_reader_offset_commit_request_duration_seconds", - Help: "The duration of requests to commit the last consumed offset.", - ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, - NativeHistogramBucketFactor: 1.1, - NativeHistogramMaxBucketNumber: 100, - NativeHistogramMinResetDuration: time.Hour, - Buckets: prometheus.DefBuckets, - }), - lastCommittedOffset: promauto.With(reg).NewGauge(prometheus.GaugeOpts{ - Name: "loki_ingest_storage_reader_last_committed_offset", - Help: "The last consumed offset successfully committed by the partition reader. Set to -1 if not offset has been committed yet.", - ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, - }), - toCommit: atomic.NewInt64(-1), + metrics: NewCommitterMetrics(reg, partitionID), + toCommit: atomic.NewInt64(-1), } // Initialise the last committed offset metric to -1 to signal no offset has been committed yet (0 is a valid offset). - c.lastCommittedOffset.Set(-1) + c.metrics.LastCommittedOffset.Set(-1) if kafkaCfg.ConsumerGroupOffsetCommitInterval > 0 { c.wg.Add(1) @@ -124,14 +134,14 @@ func (r *partitionCommitter) EnqueueOffset(o int64) { // It updates relevant metrics and logs the result of the commit operation. func (r *partitionCommitter) Commit(ctx context.Context, offset int64) (returnErr error) { startTime := time.Now() - r.commitRequestsTotal.Inc() + r.metrics.CommitRequestsTotal.Inc() defer func() { - r.commitRequestsLatency.Observe(time.Since(startTime).Seconds()) + r.metrics.CommitRequestsLatency.Observe(time.Since(startTime).Seconds()) if returnErr != nil { level.Error(r.logger).Log("msg", "failed to commit last consumed offset to Kafka", "err", returnErr, "offset", offset) - r.commitFailuresTotal.Inc() + r.metrics.CommitFailuresTotal.Inc() } }() @@ -147,7 +157,7 @@ func (r *partitionCommitter) Commit(ctx context.Context, offset int64) (returnEr committedOffset, _ := committed.Lookup(r.kafkaCfg.Topic, r.partitionID) level.Debug(r.logger).Log("msg", "last commit offset successfully committed to Kafka", "offset", committedOffset.At) - r.lastCommittedOffset.Set(float64(committedOffset.At)) + r.metrics.LastCommittedOffset.Set(float64(committedOffset.At)) return nil } diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index c5f09cd4bf8e5..5625fbac7c7ae 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -48,7 +48,7 @@ type Reader struct { client *kgo.Client logger log.Logger - metrics ReaderMetrics + metrics *ReaderMetrics reg prometheus.Registerer clock quartz.Clock } @@ -314,7 +314,6 @@ func (p *Reader) processNextFetchesUntilTargetOrMaxLagHonored(ctx context.Contex if currLag <= targetLag { level.Info(logger).Log( "msg", "partition reader consumed partition and current lag is lower or equal to configured target consumer lag", - "last_consumed_offset", p.committer.lastCommittedOffset, "current_lag", currLag, ) return nil @@ -323,7 +322,6 @@ func (p *Reader) processNextFetchesUntilTargetOrMaxLagHonored(ctx context.Contex level.Warn(logger).Log( "msg", "partition reader consumed partition and current lag is lower than configured max consumer lag but higher than target consumer lag", - "last_consumed_offset", p.committer.lastCommittedOffset, "current_lag", currLag, ) return nil @@ -526,7 +524,7 @@ type ReaderMetrics struct { } // NewReaderMetrics initializes and returns a new set of metrics for the PartitionReader. -func NewReaderMetrics(reg prometheus.Registerer) ReaderMetrics { +func NewReaderMetrics(reg prometheus.Registerer) *ReaderMetrics { receiveDelay := promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ Name: "loki_ingest_storage_reader_receive_delay_seconds", Help: "Delay between producing a record and receiving it in the consumer.", @@ -537,7 +535,7 @@ func NewReaderMetrics(reg prometheus.Registerer) ReaderMetrics { Buckets: prometheus.ExponentialBuckets(0.125, 2, 18), // Buckets between 125ms and 9h. }, []string{"phase"}) - return ReaderMetrics{ + return &ReaderMetrics{ ReceiveDelayWhenStarting: receiveDelay.WithLabelValues("starting"), ReceiveDelayWhenRunning: receiveDelay.WithLabelValues("running"), kprom: client.NewReaderClientMetrics("partition-reader", reg), From 8299cf78ef50c1f52626a266315ebe8d240e272b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 22 Oct 2024 12:12:33 -0700 Subject: [PATCH 08/56] blockbuilder partition committer Signed-off-by: Owen Diehl --- pkg/blockbuilder/partition.go | 85 ++++++++++++++++++++++++++++++----- 1 file changed, 75 insertions(+), 10 deletions(-) diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index 4255ea8e88cfc..96b606b2fc246 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -115,11 +115,46 @@ type partitionReader struct { group string partitionID int32 - metrics partition.ReaderMetrics - logger log.Logger - client *kgo.Client - aClient *kadm.Client - reg prometheus.Registerer + readerMetrics *partition.ReaderMetrics + writerMetrics *partition.CommitterMetrics + logger log.Logger + client *kgo.Client + aClient *kadm.Client +} + +func NewPartitionReader( + topic string, + group string, + partitionID int32, + logger log.Logger, + r prometheus.Registerer, +) (*partitionReader, error) { + readerMetrics := partition.NewReaderMetrics(r) + writerMetrics := partition.NewCommitterMetrics(r, partitionID) + + opts := []kgo.Opt{ + kgo.SeedBrokers([]string{"localhost:9092"}...), + kgo.ConsumerGroup(group), + kgo.ConsumeTopics(topic), + } + + client, err := kgo.NewClient(opts...) + if err != nil { + return nil, err + } + + aClient := kadm.NewClient(client) + + return &partitionReader{ + topic: topic, + group: group, + partitionID: partitionID, + readerMetrics: readerMetrics, + writerMetrics: writerMetrics, + logger: logger, + client: client, + aClient: aClient, + }, nil } // Fetches the desired offset in the partition itself, not the consumer group @@ -271,7 +306,7 @@ func (p *partitionReader) poll( maxOffset int64, // exclusive ) ([]partition.Record, bool) { defer func(start time.Time) { - p.metrics.FetchWaitDuration.Observe(time.Since(start).Seconds()) + p.readerMetrics.FetchWaitDuration.Observe(time.Since(start).Seconds()) }(time.Now()) fetches := p.client.PollFetches(ctx) p.recordFetchesMetrics(fetches) @@ -322,7 +357,7 @@ func (p *partitionReader) logFetchErrors(fetches kgo.Fetches) { if len(mErr) == 0 { return } - p.metrics.FetchesErrors.Add(float64(len(mErr))) + p.readerMetrics.FetchesErrors.Add(float64(len(mErr))) level.Error(p.logger).Log("msg", "encountered error while fetching", "err", mErr.Err()) } @@ -336,11 +371,11 @@ func (p *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { fetches.EachRecord(func(record *kgo.Record) { numRecords++ delay := now.Sub(record.Timestamp).Seconds() - p.metrics.ReceiveDelayWhenRunning.Observe(delay) + p.readerMetrics.ReceiveDelayWhenRunning.Observe(delay) }) - p.metrics.FetchesTotal.Add(float64(len(fetches))) - p.metrics.RecordsPerFetch.Observe(float64(numRecords)) + p.readerMetrics.FetchesTotal.Add(float64(len(fetches))) + p.readerMetrics.RecordsPerFetch.Observe(float64(numRecords)) } func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan<- []partition.Record) (int64, error) { @@ -377,6 +412,36 @@ func (r *partitionReader) Close() error { return nil } +// Commits the offset to the consumer group. +func (r *partitionReader) Commit(ctx context.Context, offset int64) (err error) { + startTime := time.Now() + r.writerMetrics.CommitRequestsTotal.Inc() + + defer func() { + r.writerMetrics.CommitRequestsLatency.Observe(time.Since(startTime).Seconds()) + + if err != nil { + level.Error(r.logger).Log("msg", "failed to commit last consumed offset to Kafka", "err", err, "offset", offset) + r.writerMetrics.CommitFailuresTotal.Inc() + } + }() + + // Commit the last consumed offset. + toCommit := kadm.Offsets{} + toCommit.AddOffset(r.topic, r.partitionID, offset, -1) + committed, err := r.aClient.CommitOffsets(ctx, r.group, toCommit) + if err != nil { + return err + } else if !committed.Ok() { + return committed.Error() + } + + committedOffset, _ := committed.Lookup(r.topic, r.partitionID) + level.Debug(r.logger).Log("msg", "last commit offset successfully committed to Kafka", "offset", committedOffset.At) + r.writerMetrics.LastCommittedOffset.Set(float64(committedOffset.At)) + return nil +} + func withBackoff[T any]( ctx context.Context, config backoff.Config, From 324ed0ffb6853f8fef69f4f236834596bc3ed135 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 23 Oct 2024 11:12:35 -0700 Subject: [PATCH 09/56] speccing out slimgester Signed-off-by: Owen Diehl --- pkg/blockbuilder/controller.go | 88 ++++++++++++++++++++++++++++++++++ pkg/blockbuilder/partition.go | 80 ------------------------------- pkg/blockbuilder/slimgester.go | 36 ++++++++++++++ pkg/blockbuilder/writer.go | 23 +++++++++ 4 files changed, 147 insertions(+), 80 deletions(-) create mode 100644 pkg/blockbuilder/controller.go create mode 100644 pkg/blockbuilder/slimgester.go create mode 100644 pkg/blockbuilder/writer.go diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go new file mode 100644 index 0000000000000..61b5cd68b10fd --- /dev/null +++ b/pkg/blockbuilder/controller.go @@ -0,0 +1,88 @@ +package blockbuilder + +import ( + "context" + + "github.com/grafana/loki/v3/pkg/kafka/partition" +) + +// [min,max) +type Offsets struct { + Min, Max int64 +} + +type Job struct { + Partition int32 + Offsets Offsets +} + +// Interface required for interacting with queue partitions. +type PartitionController interface { + // Returns the highest committed offset from the consumer group + HighestCommittedOffset(ctx context.Context) (int64, error) + // Returns the highest available offset in the partition + HighestPartitionOffset(ctx context.Context) (int64, error) + // Commits the offset to the consumer group. + Commit(context.Context, int64) error + // Process will run load batches at a time and send them to channel, + // so it's advised to not buffer the channel for natural backpressure. + // As a convenience, it returns the last seen offset, which matches + // the final record sent on the channel. + Process(context.Context, Offsets, chan<- []partition.Record) (int64, error) + + Close() error +} + +// PartitionJobController loads a single job a time, bound to a given +// * topic +// * partition +// * offset_step_len: the number of offsets each job to contain. e.g. "10" could yield a job w / min=15, max=25 +// +// At a high level, it watches a source topic/partition (where log data is ingested) and a "committed" topic/partition. +// The "comitted" partition corresponds to the offsets from the source partition which have been committed to object storage. +// In essence, the following loop is performed +// 1. load the most recent record from the "comitted" partition. This contains the highest msg offset in the "source" partition +// that has been committed to object storage. We'll call that $START_POS. +// 2. Create a job with `min=$START_POS+1,end=$START_POS+1+$STEP_LEN` +// 3. Sometime later when the job has been processed, we'll commit the final processed offset from the "source" partition (which +// will be <= $END_POS) to the "committed" partition. +// +// NB(owen-d): In our case, "source" is the partition +// +// containing log data and "committed" is the consumer group +type PartitionJobController struct { + topic string + partition int32 + stepLen int64 + part PartitionController +} + +// LoadJob(ctx) returns the next job by finding the most recent unconsumed offset in the partition +// Returns whether an applicable job exists, the job, and an error +func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) { + // Read the most recent committed offset + startOffset, err := l.part.HighestCommittedOffset(ctx) + if err != nil { + return false, Job{}, err + } + + highestOffset, err := l.part.HighestPartitionOffset(ctx) + if err != nil { + return false, Job{}, err + } + + if highestOffset == startOffset { + return false, Job{}, nil + } + + // Create the job with the calculated offsets + job := Job{ + Partition: l.partition, + Offsets: Offsets{ + Min: startOffset, + Max: startOffset + l.stepLen, + }, + } + + return true, job, nil +} diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index 96b606b2fc246..52dbc38d6d43d 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -30,86 +30,6 @@ var defaultBackoffConfig = backoff.Config{ MaxRetries: 0, // Retry forever (unless context is canceled / deadline exceeded). } -// [min,max) -type Offsets struct { - Min, Max int64 -} - -type Job struct { - Partition int32 - Offsets Offsets -} - -// PartitionJobController loads a single job a time, bound to a given -// * topic -// * partition -// * offset_step_len: the number of offsets each job to contain. e.g. "10" could yield a job w / min=15, max=25 -// -// At a high level, it watches a source topic/partition (where log data is ingested) and a "committed" topic/partition. -// The "comitted" partition corresponds to the offsets from the source partition which have been committed to object storage. -// In essence, the following loop is performed -// 1. load the most recent record from the "comitted" partition. This contains the highest msg offset in the "source" partition -// that has been committed to object storage. We'll call that $START_POS. -// 2. Create a job with `min=$START_POS+1,end=$START_POS+1+$STEP_LEN` -// 3. Sometime later when the job has been processed, we'll commit the final processed offset from the "source" partition (which -// will be <= $END_POS) to the "committed" partition. -// -// NB(owen-d): In our case, "source" is the partition -// -// containing log data and "committed" is the consumer group -type PartitionJobController struct { - topic string - partition int32 - stepLen int64 - part PartitionController -} - -// LoadJob(ctx) returns the next job by finding the most recent unconsumed offset in the partition -// Returns whether an applicable job exists, the job, and an error -func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) { - // Read the most recent committed offset - startOffset, err := l.part.HighestCommittedOffset(ctx) - if err != nil { - return false, Job{}, err - } - - highestOffset, err := l.part.HighestPartitionOffset(ctx) - if err != nil { - return false, Job{}, err - } - - if highestOffset == startOffset { - return false, Job{}, nil - } - - // Create the job with the calculated offsets - job := Job{ - Partition: l.partition, - Offsets: Offsets{ - Min: startOffset, - Max: startOffset + l.stepLen, - }, - } - - return true, job, nil -} - -type PartitionController interface { - // Returns the highest committed offset from the consumer group - HighestCommittedOffset(ctx context.Context) (int64, error) - // Returns the highest available offset in the partition - HighestPartitionOffset(ctx context.Context) (int64, error) - // Commits the offset to the consumer group. - Commit(context.Context, int64) error - // Process will run load batches at a time and send them to channel, - // so it's advised to not buffer the channel for natural backpressure. - // As a convenience, it returns the last seen offset, which matches - // the final record sent on the channel. - Process(context.Context, Offsets, chan<- []partition.Record) (int64, error) - - Close() error -} - type partitionReader struct { topic string group string diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go new file mode 100644 index 0000000000000..433382719c076 --- /dev/null +++ b/pkg/blockbuilder/slimgester.go @@ -0,0 +1,36 @@ +package blockbuilder + +import ( + "sync" + + "github.com/go-kit/log" + "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/stores" +) + +type SlimgesterMetrics struct{} +type Config struct{} + +// Slimgester is a slimmed-down version of the ingester, intended to +// ingest logs without WALs. Broadly, it accumulates logs into per-tenant chunks in the same way the existing ingester does, +// without a WAL. Index (TSDB) creation is also not an out-of-band procedure and must be called directly. In essence, this +// allows us to buffer data, flushing chunks to storage as necessary, and then when ready to commit this, relevant TSDBs (one per period) are created and flushed to storage. This allows an external caller to prepare a batch of data, build relevant chunks+indices, ensure they're flushed, and then return. As long as chunk+index creation is deterministic, this operation is also +// idempotent, making retries simple and impossible to introduce duplicate data. +// It contains the following methods: +// - `Append(context.Context, logproto.PushRequest) error` +// Adds a push request to ingested data. May flush existing chunks when they're full/etc. +// - `Commit(context.Context) error` +// Serializes (cuts) any buffered data into chunks, flushes them to storage, then creates + flushes TSDB indices +// containing all chunk references. Finally, clears internal state. +type Slimgester struct { + cfg Config + periodicConfigs []config.PeriodConfig + + metrics *SlimgesterMetrics + logger log.Logger + + instances map[string]*instance + instancesMtx sync.RWMutex + + store stores.ChunkWriter +} diff --git a/pkg/blockbuilder/writer.go b/pkg/blockbuilder/writer.go new file mode 100644 index 0000000000000..c21a99685eb49 --- /dev/null +++ b/pkg/blockbuilder/writer.go @@ -0,0 +1,23 @@ +package blockbuilder + +import ( + "context" + + "github.com/go-kit/log" + + "github.com/grafana/loki/pkg/push" + "github.com/grafana/loki/v3/pkg/kafka" +) + +type kafkaConsumer struct { + logger log.Logger + decoder *kafka.Decoder +} + +func (c *kafkaConsumer) Write(ctx context.Context, req *push.PushRequest) error { + return nil +} + +func (c *kafkaConsumer) Commit(ctx context.Context) error { + return nil +} From 754d9615e94899ebd787b478340dd627a6fd3716 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 23 Oct 2024 12:57:48 -0700 Subject: [PATCH 10/56] slimgester instance work Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 168 ++++++++++++++++++++++++++++++++- 1 file changed, 167 insertions(+), 1 deletion(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 433382719c076..fbd8eb606020b 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -1,14 +1,31 @@ package blockbuilder import ( + "context" "sync" "github.com/go-kit/log" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/pkg/push" + "github.com/grafana/loki/v3/pkg/chunkenc" + "github.com/grafana/loki/v3/pkg/compression" + "github.com/grafana/loki/v3/pkg/ingester" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" ) -type SlimgesterMetrics struct{} +type SlimgesterMetrics struct { + chunksCreatedTotal prometheus.Counter + samplesPerChunk prometheus.Histogram + blocksPerChunk prometheus.Histogram +} + type Config struct{} // Slimgester is a slimmed-down version of the ingester, intended to @@ -34,3 +51,152 @@ type Slimgester struct { store stores.ChunkWriter } + +// instance is a slimmed down version from the ingester pkg +type instance struct { + buf []byte // buffer used to compute fps. + mapper *ingester.FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free + metrics *SlimgesterMetrics + streams *streamsMap + + schemaconfig *config.SchemaConfig +} + +type streamsMap struct { + // labels -> stream + m map[string]*stream + mtx sync.RWMutex +} + +// For performs an operation on an existing stream, creating it if it wasn't previously present. +func (m *streamsMap) For( + ls string, + createFn func() (*stream, error), + fn func(*stream) error, +) (existed bool, err error) { + // first use read lock in case the stream exists + m.mtx.RLock() + if s, ok := m.m[ls]; ok { + err := fn(s) + m.mtx.RUnlock() + return true, err + } + m.mtx.RUnlock() + + // Stream wasn't found, acquire write lock to create it + m.mtx.Lock() + defer m.mtx.Unlock() + + // Double check it wasn't created while we were upgrading the lock + if s, ok := m.m[ls]; ok { + return true, fn(s) + } + + // Create new stream + s, err := createFn() + if err != nil { + return false, err + } + + m.m[ls] = s + return false, fn(s) +} + +func (i *instance) getHashForLabels(ls labels.Labels) model.Fingerprint { + var fp uint64 + fp, i.buf = ls.HashWithoutLabels(i.buf, []string(nil)...) + return i.mapper.MapFP(model.Fingerprint(fp), ls) +} + +// Push will iterate over the given streams present in the PushRequest and attempt to store them. +func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error { + for _, s := range req.Streams { + i.streams.For( + s.Labels, + func() (*stream, error) { + ls, err := syntax.ParseLabels(s.Labels) + if err != nil { + return nil, err + } + fp := i.getHashForLabels(ls) + return newStream(fp, ls, i.metrics), nil + }, + func(stream *stream) error { + closed, err := stream.Push(s.Entries) + if err != nil { + return err + } + + // TODO: flush closed + panic(closed) + + return nil + }, + ) + } + + return nil +} + +type stream struct { + fp model.Fingerprint + ls labels.Labels + + chunkFormat byte + headFmt chunkenc.HeadBlockFmt + codec compression.Codec + blockSize int + targetChunkSize int + + chunkMtx sync.RWMutex + chunk *chunkenc.MemChunk + metrics *SlimgesterMetrics +} + +func newStream(fp model.Fingerprint, ls labels.Labels, metrics *SlimgesterMetrics) *stream { + return &stream{ + fp: fp, + ls: ls, + + chunkFormat: chunkenc.ChunkFormatV3, + metrics: metrics, + } +} + +func (s *stream) Push(entries []push.Entry) (closed []*chunkenc.MemChunk, err error) { + s.chunkMtx.Lock() + defer s.chunkMtx.Unlock() + + if s.chunk == nil { + s.chunk = s.NewChunk() + } + + // bytesAdded, err := s.storeEntries(ctx, toStore, usageTracker) + for i := 0; i < len(entries); i++ { + + // cut the chunk if the new addition overflows target size + if !s.chunk.SpaceFor(&entries[i]) { + if err = s.chunk.Close(); err != nil { + return closed, errors.Wrap(err, "closing chunk") + } + + s.metrics.samplesPerChunk.Observe(float64(s.chunk.Size())) + s.metrics.blocksPerChunk.Observe(float64(s.chunk.BlockCount())) + s.metrics.chunksCreatedTotal.Inc() + + // add a chunk + closed = append(closed, s.chunk) + s.chunk = s.NewChunk() + } + + if _, err = s.chunk.Append(&entries[i]); err != nil { + return closed, errors.Wrap(err, "appending entry") + } + } + + return closed, nil +} + +func (s *stream) NewChunk() *chunkenc.MemChunk { + return chunkenc.NewMemChunk(s.chunkFormat, s.codec, s.headFmt, s.blockSize, s.targetChunkSize) +} From 2129bd3d7dff740c37c6f240c7ca79030683f987 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 23 Oct 2024 17:36:26 -0700 Subject: [PATCH 11/56] tuning Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index fbd8eb606020b..c049616f4d947 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -73,13 +73,13 @@ func (m *streamsMap) For( ls string, createFn func() (*stream, error), fn func(*stream) error, -) (existed bool, err error) { +) error { // first use read lock in case the stream exists m.mtx.RLock() if s, ok := m.m[ls]; ok { err := fn(s) m.mtx.RUnlock() - return true, err + return err } m.mtx.RUnlock() @@ -89,17 +89,17 @@ func (m *streamsMap) For( // Double check it wasn't created while we were upgrading the lock if s, ok := m.m[ls]; ok { - return true, fn(s) + return fn(s) } // Create new stream s, err := createFn() if err != nil { - return false, err + return err } m.m[ls] = s - return false, fn(s) + return fn(s) } func (i *instance) getHashForLabels(ls labels.Labels) model.Fingerprint { @@ -109,9 +109,12 @@ func (i *instance) getHashForLabels(ls labels.Labels) model.Fingerprint { } // Push will iterate over the given streams present in the PushRequest and attempt to store them. -func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error { +func (i *instance) Push( + ctx context.Context, + req *logproto.PushRequest, +) (closed []*chunkenc.MemChunk, err error) { for _, s := range req.Streams { - i.streams.For( + err = i.streams.For( s.Labels, func() (*stream, error) { ls, err := syntax.ParseLabels(s.Labels) @@ -122,20 +125,17 @@ func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error { return newStream(fp, ls, i.metrics), nil }, func(stream *stream) error { - closed, err := stream.Push(s.Entries) + xs, err := stream.Push(s.Entries) if err != nil { return err } - - // TODO: flush closed - panic(closed) - - return nil + closed = append(closed, xs...) + return err }, ) } - return nil + return closed, err } type stream struct { From 239b517fcd3ecc0d64ac198b04dcc019f0060f4b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 23 Oct 2024 18:20:37 -0700 Subject: [PATCH 12/56] flush queue start Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 65 ++++++++++++++++++++++++++++++++-- 1 file changed, 62 insertions(+), 3 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index c049616f4d947..349b5d53fad40 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -16,8 +16,10 @@ import ( "github.com/grafana/loki/v3/pkg/ingester" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" + "github.com/grafana/loki/v3/pkg/util" ) type SlimgesterMetrics struct { @@ -50,16 +52,61 @@ type Slimgester struct { instancesMtx sync.RWMutex store stores.ChunkWriter + + queueMtx sync.Mutex + flushQueue []*chunk.Chunk +} + +func (i *Slimgester) Append(ctx context.Context, tenant string, req *logproto.PushRequest) error { + // use rlock so multiple appends can be called on same instance. + // re-check after using regular lock if it didnt exist. + i.instancesMtx.RLock() + inst, ok := i.instances[tenant] + i.instancesMtx.RUnlock() + if !ok { + i.instancesMtx.Lock() + inst, ok = i.instances[tenant] + if !ok { + inst = newInstance(tenant, i.metrics, i.periodicConfigs) + i.instances[tenant] = inst + } + i.instancesMtx.Unlock() + } + + closed, err := inst.Push(ctx, req) + if len(closed) > 0 { + i.queueMtx.Lock() + defer i.queueMtx.Unlock() + i.flushQueue = append(i.flushQueue, closed...) + } + + return err } // instance is a slimmed down version from the ingester pkg type instance struct { + tenant string buf []byte // buffer used to compute fps. mapper *ingester.FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free metrics *SlimgesterMetrics streams *streamsMap - schemaconfig *config.SchemaConfig + periods []config.PeriodConfig +} + +func newInstance( + tenant string, + metrics *SlimgesterMetrics, + periods []config.PeriodConfig, +) *instance { + return &instance{ + tenant: tenant, + buf: make([]byte, 0, 1024), + mapper: ingester.NewFPMapper(nil), // TODO: impl + metrics: metrics, + streams: &streamsMap{m: make(map[string]*stream)}, + periods: periods, + } } type streamsMap struct { @@ -112,7 +159,7 @@ func (i *instance) getHashForLabels(ls labels.Labels) model.Fingerprint { func (i *instance) Push( ctx context.Context, req *logproto.PushRequest, -) (closed []*chunkenc.MemChunk, err error) { +) (closed []*chunk.Chunk, err error) { for _, s := range req.Streams { err = i.streams.For( s.Labels, @@ -129,7 +176,19 @@ func (i *instance) Push( if err != nil { return err } - closed = append(closed, xs...) + + if len(xs) > 0 { + for _, x := range xs { + firstTime, lastTime := util.RoundToMilliseconds(x.Bounds()) + chk := chunk.NewChunk( + i.tenant, stream.fp, stream.ls, + chunkenc.NewFacade(x, stream.blockSize, stream.targetChunkSize), + firstTime, + lastTime, + ) + closed = append(closed, &chk) + } + } return err }, ) From 543094b59b821fb9bccf85d5d974de32997bd46b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 24 Oct 2024 12:57:19 -0700 Subject: [PATCH 13/56] fpmapper integration Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 35 +++++++++++++++++++++++++++------- 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 349b5d53fad40..1168cedcbd2c4 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -99,20 +99,29 @@ func newInstance( metrics *SlimgesterMetrics, periods []config.PeriodConfig, ) *instance { + streams := newStreamsMap() return &instance{ tenant: tenant, buf: make([]byte, 0, 1024), - mapper: ingester.NewFPMapper(nil), // TODO: impl + mapper: ingester.NewFPMapper(streams.getLabelsFromFingerprint), metrics: metrics, - streams: &streamsMap{m: make(map[string]*stream)}, + streams: streams, periods: periods, } } +func newStreamsMap() *streamsMap { + return &streamsMap{ + byLabels: make(map[string]*stream), + byFp: make(map[model.Fingerprint]*stream), + } +} + type streamsMap struct { // labels -> stream - m map[string]*stream - mtx sync.RWMutex + byLabels map[string]*stream + byFp map[model.Fingerprint]*stream + mtx sync.RWMutex } // For performs an operation on an existing stream, creating it if it wasn't previously present. @@ -123,7 +132,7 @@ func (m *streamsMap) For( ) error { // first use read lock in case the stream exists m.mtx.RLock() - if s, ok := m.m[ls]; ok { + if s, ok := m.byLabels[ls]; ok { err := fn(s) m.mtx.RUnlock() return err @@ -135,7 +144,7 @@ func (m *streamsMap) For( defer m.mtx.Unlock() // Double check it wasn't created while we were upgrading the lock - if s, ok := m.m[ls]; ok { + if s, ok := m.byLabels[ls]; ok { return fn(s) } @@ -145,10 +154,22 @@ func (m *streamsMap) For( return err } - m.m[ls] = s + m.byLabels[ls] = s + m.byFp[s.fp] = s return fn(s) } +// Return labels associated with given fingerprint. Used by fingerprint mapper. +func (m *streamsMap) getLabelsFromFingerprint(fp model.Fingerprint) labels.Labels { + m.mtx.RLock() + defer m.mtx.RUnlock() + + if s, ok := m.byFp[fp]; ok { + return s.ls + } + return nil +} + func (i *instance) getHashForLabels(ls labels.Labels) model.Fingerprint { var fp uint64 fp, i.buf = ls.HashWithoutLabels(i.buf, []string(nil)...) From 901c159a8c646b0ba40755cef6a8e1f744046a55 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 24 Oct 2024 14:48:43 -0700 Subject: [PATCH 14/56] svc scaffolding Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 129 +++++++++++++++++++++++++++++++-- 1 file changed, 121 insertions(+), 8 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 1168cedcbd2c4..f98151a5e61c3 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -2,11 +2,17 @@ package blockbuilder import ( "context" + "flag" + "fmt" + "runtime" "sync" + "time" "github.com/go-kit/log" + "github.com/grafana/dskit/services" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -28,7 +34,57 @@ type SlimgesterMetrics struct { blocksPerChunk prometheus.Histogram } -type Config struct{} +func NewSlimgesterMetrics(r prometheus.Registerer) *SlimgesterMetrics { + return &SlimgesterMetrics{ + chunksCreatedTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_slimgester_chunks_created_total", + Help: "The total number of chunks created in the slimgester.", + }), + samplesPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Name: "loki_slimgester_chunk_samples", + Help: "Number of samples in chunks at flush.", + Buckets: prometheus.ExponentialBuckets(10, 2, 8), // 10, 20, 40, 80, 160, 320, 640, 1280 + }), + blocksPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Name: "loki_slimgester_chunk_blocks", + Help: "Number of blocks in chunks at flush.", + Buckets: prometheus.ExponentialBuckets(2, 2, 6), // 2, 4, 8, 16, 32, 64 + }), + } +} + +type Config struct { + ConcurrentFlushes int `yaml:"concurrent_flushes"` + ConcurrentWriters int `yaml:"concurrent_writers"` + BlockSize int `yaml:"chunk_block_size"` + TargetChunkSize int `yaml:"chunk_target_size"` + ChunkEncoding string `yaml:"chunk_encoding"` + parsedEncoding compression.Codec `yaml:"-"` // placeholder for validated encoding + MaxChunkAge time.Duration `yaml:"max_chunk_age"` +} + +func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + f.IntVar(&cfg.ConcurrentFlushes, "ingester.concurrent-flushes", 16, "How many flushes can happen concurrently") + f.IntVar(&cfg.ConcurrentWriters, "ingester.concurrent-writers", runtime.NumCPU(), "How many workers to process writes, defaults to number of available cpus") + f.IntVar(&cfg.BlockSize, "ingester.chunks-block-size", 256*1024, "The targeted _uncompressed_ size in bytes of a chunk block When this threshold is exceeded the head block will be cut and compressed inside the chunk.") + f.IntVar(&cfg.TargetChunkSize, "ingester.chunk-target-size", 1572864, "A target _compressed_ size in bytes for chunks. This is a desired size not an exact size, chunks may be slightly bigger or significantly smaller if they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 creates chunks with a fixed 10 blocks, a non zero value will create chunks with a variable number of blocks to meet the target size.") // 1.5 MB + f.StringVar(&cfg.ChunkEncoding, "ingester.chunk-encoding", compression.GZIP.String(), fmt.Sprintf("The algorithm to use for compressing chunk. (%s)", compression.SupportedCodecs())) + f.DurationVar(&cfg.MaxChunkAge, "ingester.max-chunk-age", 2*time.Hour, "The maximum duration of a timeseries chunk in memory. If a timeseries runs for longer than this, the current chunk will be flushed to the store and a new chunk created.") +} + +// RegisterFlags registers flags. +func (c *Config) RegisterFlags(flags *flag.FlagSet) { + c.RegisterFlagsWithPrefix("slimgester", flags) +} + +func (cfg *Config) Validate() error { + enc, err := compression.ParseCodec(cfg.ChunkEncoding) + if err != nil { + return err + } + cfg.parsedEncoding = enc + return nil +} // Slimgester is a slimmed-down version of the ingester, intended to // ingest logs without WALs. Broadly, it accumulates logs into per-tenant chunks in the same way the existing ingester does, @@ -42,6 +98,8 @@ type Config struct{} // Serializes (cuts) any buffered data into chunks, flushes them to storage, then creates + flushes TSDB indices // containing all chunk references. Finally, clears internal state. type Slimgester struct { + services.Service + cfg Config periodicConfigs []config.PeriodConfig @@ -51,10 +109,67 @@ type Slimgester struct { instances map[string]*instance instancesMtx sync.RWMutex - store stores.ChunkWriter + store stores.ChunkWriter + flushQueue chan *chunk.Chunk + + wg sync.WaitGroup // for waiting on flusher + quit chan struct{} // for signaling flusher + closer sync.Once // for coordinating channel closure +} + +func NewSlimgester( + cfg Config, + periodicConfigs []config.PeriodConfig, + store stores.ChunkWriter, + logger log.Logger, + reg prometheus.Registerer, +) (*Slimgester, + error) { + i := &Slimgester{ + cfg: cfg, + periodicConfigs: periodicConfigs, + metrics: NewSlimgesterMetrics(reg), + logger: logger, + instances: make(map[string]*instance), + store: store, + + flushQueue: make(chan *chunk.Chunk), + quit: make(chan struct{}), + } + + i.Service = services.NewBasicService(i.starting, i.running, i.stopping) + return i, nil +} + +func (i *Slimgester) starting(_ context.Context) error { + go i.flushLoop() + return nil +} + +func (i *Slimgester) running(ctx context.Context) error { + select { + case <-ctx.Done(): + i.close() + } + + return nil +} + +func (i *Slimgester) stopping(err error) error { + i.wg.Wait() + return err +} + +func (i *Slimgester) flushLoop() { + i.wg.Add(1) + i.wg.Done() +} - queueMtx sync.Mutex - flushQueue []*chunk.Chunk +// behind sync.Once b/c it can be called from etiher `running` or `stopping`. +func (i *Slimgester) close() { + i.closer.Do(func() { + close(i.quit) + }) } func (i *Slimgester) Append(ctx context.Context, tenant string, req *logproto.PushRequest) error { @@ -74,10 +189,8 @@ func (i *Slimgester) Append(ctx context.Context, tenant string, req *logproto.Pu } closed, err := inst.Push(ctx, req) - if len(closed) > 0 { - i.queueMtx.Lock() - defer i.queueMtx.Unlock() - i.flushQueue = append(i.flushQueue, closed...) + for _, chk := range closed { + i.flushQueue <- chk } return err From 56f4e4197a88133385b06f5a53f62bb16d9831b1 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 24 Oct 2024 17:05:29 -0700 Subject: [PATCH 15/56] workers, metrics integration Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 312 ++++++++++++++++++++++++++++++--- 1 file changed, 284 insertions(+), 28 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index f98151a5e61c3..8ecb604155f16 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -1,6 +1,7 @@ package blockbuilder import ( + "bytes" "context" "flag" "fmt" @@ -9,14 +10,18 @@ import ( "time" "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/multierror" "github.com/grafana/dskit/services" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + "golang.org/x/sync/errgroup" "github.com/grafana/loki/pkg/push" + "github.com/grafana/loki/v3/pkg/analytics" "github.com/grafana/loki/v3/pkg/chunkenc" "github.com/grafana/loki/v3/pkg/compression" "github.com/grafana/loki/v3/pkg/ingester" @@ -26,30 +31,154 @@ import ( "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" "github.com/grafana/loki/v3/pkg/util" + "github.com/grafana/loki/v3/pkg/util/constants" +) + +const ( + flushReasonFull = "full" + flushReasonMaxAge = "max_age" ) type SlimgesterMetrics struct { + chunkUtilization prometheus.Histogram + chunkEntries prometheus.Histogram + chunkSize prometheus.Histogram + chunkCompressionRatio prometheus.Histogram + chunksPerTenant *prometheus.CounterVec + chunkSizePerTenant *prometheus.CounterVec + chunkAge prometheus.Histogram + chunkEncodeTime prometheus.Histogram + chunksFlushFailures prometheus.Counter + chunksFlushedPerReason *prometheus.CounterVec + chunkLifespan prometheus.Histogram + chunksEncoded *prometheus.CounterVec + chunkDecodeFailures *prometheus.CounterVec + flushedChunksStats *analytics.Counter + flushedChunksBytesStats *analytics.Statistics + flushedChunksLinesStats *analytics.Statistics + flushedChunksAgeStats *analytics.Statistics + flushedChunksLifespanStats *analytics.Statistics + flushedChunksUtilizationStats *analytics.Statistics + chunksCreatedTotal prometheus.Counter samplesPerChunk prometheus.Histogram blocksPerChunk prometheus.Histogram + chunkCreatedStats *analytics.Counter } func NewSlimgesterMetrics(r prometheus.Registerer) *SlimgesterMetrics { return &SlimgesterMetrics{ + chunkUtilization: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_utilization", + Help: "Distribution of stored chunk utilization (when stored).", + Buckets: prometheus.LinearBuckets(0, 0.2, 6), + }), + chunkEntries: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_entries", + Help: "Distribution of stored lines per chunk (when stored).", + Buckets: prometheus.ExponentialBuckets(200, 2, 9), // biggest bucket is 200*2^(9-1) = 51200 + }), + chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_size_bytes", + Help: "Distribution of stored chunk sizes (when stored).", + Buckets: prometheus.ExponentialBuckets(20000, 2, 10), // biggest bucket is 20000*2^(10-1) = 10,240,000 (~10.2MB) + }), + chunkCompressionRatio: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_compression_ratio", + Help: "Compression ratio of chunks (when stored).", + Buckets: prometheus.LinearBuckets(.75, 2, 10), + }), + chunksPerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_stored_total", + Help: "Total stored chunks per tenant.", + }, []string{"tenant"}), + chunkSizePerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_stored_bytes_total", + Help: "Total bytes stored in chunks per tenant.", + }, []string{"tenant"}), + chunkAge: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_age_seconds", + Help: "Distribution of chunk ages (when stored).", + // with default settings chunks should flush between 5 min and 12 hours + // so buckets at 1min, 5min, 10min, 30min, 1hr, 2hr, 4hr, 10hr, 12hr, 16hr + Buckets: []float64{60, 300, 600, 1800, 3600, 7200, 14400, 36000, 43200, 57600}, + }), + chunkEncodeTime: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_encode_time_seconds", + Help: "Distribution of chunk encode times.", + // 10ms to 10s. + Buckets: prometheus.ExponentialBuckets(0.01, 4, 6), + }), + chunksFlushFailures: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_flush_failures_total", + Help: "Total number of flush failures.", + }), + chunksFlushedPerReason: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_flushed_total", + Help: "Total flushed chunks per reason.", + }, []string{"reason"}), + chunkLifespan: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_bounds_hours", + Help: "Distribution of chunk end-start durations.", + // 1h -> 8hr + Buckets: prometheus.LinearBuckets(1, 1, 8), + }), + chunksEncoded: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_encoded_total", + Help: "The total number of chunks encoded in the ingester.", + }, []string{"user"}), + chunkDecodeFailures: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_decode_failures_total", + Help: "The number of freshly encoded chunks that failed to decode.", + }, []string{"user"}), + flushedChunksStats: analytics.NewCounter("slimgester_flushed_chunks"), + flushedChunksBytesStats: analytics.NewStatistics("slimgester_flushed_chunks_bytes"), + flushedChunksLinesStats: analytics.NewStatistics("slimgester_flushed_chunks_lines"), + flushedChunksAgeStats: analytics.NewStatistics( + "slimgester_flushed_chunks_age_seconds", + ), + flushedChunksLifespanStats: analytics.NewStatistics( + "slimgester_flushed_chunks_lifespan_seconds", + ), + flushedChunksUtilizationStats: analytics.NewStatistics( + "slimgester_flushed_chunks_utilization", + ), chunksCreatedTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Name: "loki_slimgester_chunks_created_total", - Help: "The total number of chunks created in the slimgester.", + Namespace: constants.Loki, + Name: "slimgester_chunks_created_total", + Help: "The total number of chunks created in the ingester.", }), samplesPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Name: "loki_slimgester_chunk_samples", - Help: "Number of samples in chunks at flush.", - Buckets: prometheus.ExponentialBuckets(10, 2, 8), // 10, 20, 40, 80, 160, 320, 640, 1280 + Namespace: constants.Loki, + Subsystem: "slimgester", + Name: "samples_per_chunk", + Help: "The number of samples in a chunk.", + + Buckets: prometheus.LinearBuckets(4096, 2048, 6), }), blocksPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Name: "loki_slimgester_chunk_blocks", - Help: "Number of blocks in chunks at flush.", - Buckets: prometheus.ExponentialBuckets(2, 2, 6), // 2, 4, 8, 16, 32, 64 + Namespace: constants.Loki, + Subsystem: "slimgester", + Name: "blocks_per_chunk", + Help: "The number of blocks in a chunk.", + + Buckets: prometheus.ExponentialBuckets(5, 2, 6), }), + + chunkCreatedStats: analytics.NewCounter("slimgester_chunk_created"), } } @@ -110,10 +239,11 @@ type Slimgester struct { instancesMtx sync.RWMutex store stores.ChunkWriter + input chan AppendInput // for processing flushQueue chan *chunk.Chunk - wg sync.WaitGroup // for waiting on flusher - quit chan struct{} // for signaling flusher + grp errgroup.Group // for waiting on flushers+workers + quit chan struct{} // for signaling flushers+workers closer sync.Once // for coordinating channel closure } @@ -133,36 +263,82 @@ func NewSlimgester( instances: make(map[string]*instance), store: store, + input: make(chan AppendInput), flushQueue: make(chan *chunk.Chunk), - quit: make(chan struct{}), + + quit: make(chan struct{}), } i.Service = services.NewBasicService(i.starting, i.running, i.stopping) return i, nil } -func (i *Slimgester) starting(_ context.Context) error { - go i.flushLoop() +func (i *Slimgester) starting(ctx context.Context) error { + // start flush goroutines + for j := 0; j < i.cfg.ConcurrentFlushes; j++ { + i.grp.Go(func() error { + for { + select { + case <-i.quit: + return nil + case <-ctx.Done(): + case chk := <-i.flushQueue: + if _, err := withBackoff( + ctx, + defaultBackoffConfig, // retry forever + func() (res struct{}, err error) { + err = i.store.PutOne(ctx, chk.From, chk.Through, *chk) + if err != nil { + i.metrics.chunksFlushFailures.Inc() + return + } + i.reportFlushedChunkStatistics(chk) + return + }, + ); err != nil { + return err + } + } + } + }) + } + + // start chunk building goroutines + for j := 0; j < i.cfg.ConcurrentWriters; j++ { + i.grp.Go(func() error { + for { + select { + case <-i.quit: + return nil + case <-ctx.Done(): + return ctx.Err() + case input := <-i.input: + if err := i.Append(ctx, input); err != nil { + level.Error(i.logger).Log("msg", "failed to append records", "err", err) + return err + } + } + } + }) + } return nil + } func (i *Slimgester) running(ctx context.Context) error { select { case <-ctx.Done(): i.close() + case <-i.quit: } - return nil } -func (i *Slimgester) stopping(err error) error { - i.wg.Wait() - return err -} - -func (i *Slimgester) flushLoop() { - i.wg.Add(1) - i.wg.Done() +func (i *Slimgester) stopping(e error) error { + i.close() + err := multierror.New(e) + err.Add(i.grp.Wait()) + return err.Err() } // behind sync.Once b/c it can be called from etiher `running` or `stopping`. @@ -172,23 +348,75 @@ func (i *Slimgester) close() { }) } -func (i *Slimgester) Append(ctx context.Context, tenant string, req *logproto.PushRequest) error { +// reportFlushedChunkStatistics calculate overall statistics of flushed chunks without compromising the flush process. +func (i *Slimgester) reportFlushedChunkStatistics( + ch *chunk.Chunk, +) { + byt, err := ch.Encoded() + if err != nil { + level.Error(i.logger).Log("msg", "failed to encode flushed wire chunk", "err", err) + return + } + sizePerTenant := i.metrics.chunkSizePerTenant.WithLabelValues(ch.UserID) + countPerTenant := i.metrics.chunksPerTenant.WithLabelValues(ch.UserID) + + reason := flushReasonFull + from, through := ch.From.Time(), ch.Through.Time() + if through.Sub(from) > i.cfg.MaxChunkAge { + reason = flushReasonMaxAge + } + + i.metrics.chunksFlushedPerReason.WithLabelValues(reason).Add(1) + + compressedSize := float64(len(byt)) + uncompressedSize, ok := chunkenc.UncompressedSize(ch.Data) + + if ok && compressedSize > 0 { + i.metrics.chunkCompressionRatio.Observe(float64(uncompressedSize) / compressedSize) + } + + utilization := ch.Data.Utilization() + i.metrics.chunkUtilization.Observe(utilization) + + numEntries := ch.Data.Entries() + i.metrics.chunkEntries.Observe(float64(numEntries)) + i.metrics.chunkSize.Observe(compressedSize) + sizePerTenant.Add(compressedSize) + countPerTenant.Inc() + + i.metrics.chunkAge.Observe(time.Since(from).Seconds()) + i.metrics.chunkLifespan.Observe(through.Sub(from).Hours()) + + i.metrics.flushedChunksBytesStats.Record(compressedSize) + i.metrics.flushedChunksLinesStats.Record(float64(numEntries)) + i.metrics.flushedChunksUtilizationStats.Record(utilization) + i.metrics.flushedChunksAgeStats.Record(time.Since(from).Seconds()) + i.metrics.flushedChunksLifespanStats.Record(through.Sub(from).Seconds()) + i.metrics.flushedChunksStats.Inc(1) +} + +type AppendInput struct { + tenant string + req *logproto.PushRequest +} + +func (i *Slimgester) Append(ctx context.Context, input AppendInput) error { // use rlock so multiple appends can be called on same instance. // re-check after using regular lock if it didnt exist. i.instancesMtx.RLock() - inst, ok := i.instances[tenant] + inst, ok := i.instances[input.tenant] i.instancesMtx.RUnlock() if !ok { i.instancesMtx.Lock() - inst, ok = i.instances[tenant] + inst, ok = i.instances[input.tenant] if !ok { - inst = newInstance(tenant, i.metrics, i.periodicConfigs) - i.instances[tenant] = inst + inst = newInstance(input.tenant, i.metrics, i.periodicConfigs, i.logger) + i.instances[input.tenant] = inst } i.instancesMtx.Unlock() } - closed, err := inst.Push(ctx, req) + closed, err := inst.Push(ctx, input.req) for _, chk := range closed { i.flushQueue <- chk } @@ -203,6 +431,7 @@ type instance struct { mapper *ingester.FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free metrics *SlimgesterMetrics streams *streamsMap + logger log.Logger periods []config.PeriodConfig } @@ -211,6 +440,7 @@ func newInstance( tenant string, metrics *SlimgesterMetrics, periods []config.PeriodConfig, + logger log.Logger, ) *instance { streams := newStreamsMap() return &instance{ @@ -219,6 +449,7 @@ func newInstance( mapper: ingester.NewFPMapper(streams.getLabelsFromFingerprint), metrics: metrics, streams: streams, + logger: logger, periods: periods, } } @@ -320,6 +551,11 @@ func (i *instance) Push( firstTime, lastTime, ) + // encodeChunk mutates the chunk so we must pass by reference + if err := i.encodeChunk(ctx, &chk, x); err != nil { + return err + } + closed = append(closed, &chk) } } @@ -331,6 +567,25 @@ func (i *instance) Push( return closed, err } +// encodeChunk encodes a chunk.Chunk. +func (i *instance) encodeChunk(ctx context.Context, ch *chunk.Chunk, mc *chunkenc.MemChunk) error { + if err := ctx.Err(); err != nil { + return err + } + start := time.Now() + chunkBytesSize := mc.BytesSize() + 4*1024 // size + 4kB should be enough room for cortex header + if err := ch.EncodeTo(bytes.NewBuffer(make([]byte, 0, chunkBytesSize)), i.logger); err != nil { + if !errors.Is(err, chunk.ErrChunkDecode) { + return fmt.Errorf("chunk encoding: %w", err) + } + + i.metrics.chunkDecodeFailures.WithLabelValues(ch.UserID).Inc() + } + i.metrics.chunkEncodeTime.Observe(time.Since(start).Seconds()) + i.metrics.chunksEncoded.WithLabelValues(ch.UserID).Inc() + return nil +} + type stream struct { fp model.Fingerprint ls labels.Labels @@ -376,6 +631,7 @@ func (s *stream) Push(entries []push.Entry) (closed []*chunkenc.MemChunk, err er s.metrics.samplesPerChunk.Observe(float64(s.chunk.Size())) s.metrics.blocksPerChunk.Observe(float64(s.chunk.BlockCount())) s.metrics.chunksCreatedTotal.Inc() + s.metrics.chunkCreatedStats.Inc(1) // add a chunk closed = append(closed, s.chunk) From 749455e7fed87fe5bcc918ba3892bf2a6825e62b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sat, 26 Oct 2024 10:05:42 -0700 Subject: [PATCH 16/56] start working on slimgester tsdb creator Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 4 +- pkg/blockbuilder/tsdb.go | 173 +++++++++++++++++++++++++++++++++ 2 files changed, 175 insertions(+), 2 deletions(-) create mode 100644 pkg/blockbuilder/tsdb.go diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 8ecb604155f16..0d5a51c638674 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -239,8 +239,8 @@ type Slimgester struct { instancesMtx sync.RWMutex store stores.ChunkWriter - input chan AppendInput // for processing - flushQueue chan *chunk.Chunk + input chan AppendInput // for processing + flushQueue chan *chunk.Chunk // for flushing grp errgroup.Group // for waiting on flushers+workers quit chan struct{} // for signaling flushers+workers diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go new file mode 100644 index 0000000000000..54360b5bb3fee --- /dev/null +++ b/pkg/blockbuilder/tsdb.go @@ -0,0 +1,173 @@ +package blockbuilder + +import ( + "sync" + + "github.com/cespare/xxhash" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" +) + +// TsdbCreator accepts writes and builds TSDBs. +type TsdbCreator struct { + // Function to build a TSDB from the current state + mkTsdb func() ([]byte, error) + + mtx sync.RWMutex + shards int + heads *tenantHeads +} + +// new creates a new HeadManager +func newTsdbCreator(mkTsdb func() ([]byte, error)) *TsdbCreator { + m := &TsdbCreator{ + mkTsdb: mkTsdb, + shards: 1 << 5, // 32 shards + } + + return m +} + +// Append adds a new series for the given user +func (m *TsdbCreator) Append(userID string, ls labels.Labels, fprint uint64, chks index.ChunkMetas) error { + m.mtx.RLock() + defer m.mtx.RUnlock() + + // TODO(owen-d): safe to remove? + // Remove __name__="logs" as it's not needed in TSDB + b := labels.NewBuilder(ls) + b.Del(labels.MetricName) + ls = b.Labels() + + // Just append to heads, no WAL needed + m.heads.Append(userID, ls, fprint, chks) + return nil +} + +// Create builds a TSDB from the current state using the provided mkTsdb function +func (m *TsdbCreator) Create() ([]byte, error) { + m.mtx.Lock() + defer m.mtx.Unlock() + + out, err := m.mkTsdb() + if err != nil { + return nil, errors.Wrap(err, "creating TSDB") + } + + m.heads = newTenantHeads(m.shards) + return out, nil +} + +// tenantHeads manages per-tenant series +type tenantHeads struct { + shards int + locks []sync.RWMutex + tenants []map[string]*Head +} + +func newTenantHeads(shards int) *tenantHeads { + t := &tenantHeads{ + shards: shards, + locks: make([]sync.RWMutex, shards), + tenants: make([]map[string]*Head, shards), + } + for i := range t.tenants { + t.tenants[i] = make(map[string]*Head) + } + return t +} + +func (t *tenantHeads) Append(userID string, ls labels.Labels, fprint uint64, chks index.ChunkMetas) { + head := t.getOrCreateTenantHead(userID) + head.Append(ls, fprint, chks) +} + +func (t *tenantHeads) getOrCreateTenantHead(userID string) *Head { + idx := t.shardForTenant(userID) + mtx := &t.locks[idx] + + // Fast path: return existing head + mtx.RLock() + head, ok := t.tenants[idx][userID] + mtx.RUnlock() + if ok { + return head + } + + // Slow path: create new head + mtx.Lock() + defer mtx.Unlock() + + head, ok = t.tenants[idx][userID] + if !ok { + head = NewHead(userID) + t.tenants[idx][userID] = head + } + return head +} + +func (t *tenantHeads) shardForTenant(userID string) uint64 { + return xxhash.Sum64String(userID) & uint64(t.shards-1) +} + +// forAll iterates through all series in all tenant heads +func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error) error { + for i, shard := range t.tenants { + t.locks[i].RLock() + defer t.locks[i].RUnlock() + + for user, tenant := range shard { + if err := tenant.forAll(func(ls labels.Labels, fp uint64, chks index.ChunkMetas) error { + return fn(user, ls, fp, chks) + }); err != nil { + return err + } + } + } + return nil +} + +// Head manages series for a single tenant +type Head struct { + userID string + series map[uint64]*series + mtx sync.RWMutex +} + +type series struct { + labels labels.Labels + chks []index.ChunkMeta +} + +func NewHead(userID string) *Head { + return &Head{ + userID: userID, + series: make(map[uint64]*series), + } +} + +func (h *Head) Append(ls labels.Labels, fp uint64, chks index.ChunkMetas) { + h.mtx.Lock() + defer h.mtx.Unlock() + + s, ok := h.series[fp] + if !ok { + s = &series{labels: ls} + h.series[fp] = s + } + s.chks = append(s.chks, chks...) +} + +func (h *Head) forAll(fn func(ls labels.Labels, fp uint64, chks index.ChunkMetas) error) error { + h.mtx.RLock() + defer h.mtx.RUnlock() + + for fp, s := range h.series { + if err := fn(s.labels, fp, s.chks); err != nil { + return err + } + } + return nil +} From deba52995aa1e3e46cdc16b2ff6b468d46ff7cb0 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 28 Oct 2024 12:55:42 -0700 Subject: [PATCH 17/56] decoding, input alignment Signed-off-by: Owen Diehl --- pkg/blockbuilder/controller.go | 4 +- pkg/blockbuilder/partition.go | 29 +++++- pkg/blockbuilder/slimgester.go | 161 ++++++++++++++++++--------------- pkg/blockbuilder/tsdb.go | 6 +- 4 files changed, 119 insertions(+), 81 deletions(-) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index 61b5cd68b10fd..d5f05271e0799 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -2,8 +2,6 @@ package blockbuilder import ( "context" - - "github.com/grafana/loki/v3/pkg/kafka/partition" ) // [min,max) @@ -28,7 +26,7 @@ type PartitionController interface { // so it's advised to not buffer the channel for natural backpressure. // As a convenience, it returns the last seen offset, which matches // the final record sent on the channel. - Process(context.Context, Offsets, chan<- []partition.Record) (int64, error) + Process(context.Context, Offsets, chan<- []AppendInput) (int64, error) Close() error } diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index 52dbc38d6d43d..3631c000bb5e7 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -16,6 +16,7 @@ import ( "github.com/twmb/franz-go/pkg/kgo" "github.com/twmb/franz-go/pkg/kmsg" + "github.com/grafana/loki/v3/pkg/kafka" "github.com/grafana/loki/v3/pkg/kafka/partition" ) @@ -34,6 +35,7 @@ type partitionReader struct { topic string group string partitionID int32 + decoder *kafka.Decoder readerMetrics *partition.ReaderMetrics writerMetrics *partition.CommitterMetrics @@ -46,6 +48,7 @@ func NewPartitionReader( topic string, group string, partitionID int32, + decoder *kafka.Decoder, logger log.Logger, r prometheus.Registerer, ) (*partitionReader, error) { @@ -72,6 +75,7 @@ func NewPartitionReader( readerMetrics: readerMetrics, writerMetrics: writerMetrics, logger: logger, + decoder: decoder, client: client, aClient: aClient, }, nil @@ -298,7 +302,7 @@ func (p *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { p.readerMetrics.RecordsPerFetch.Observe(float64(numRecords)) } -func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan<- []partition.Record) (int64, error) { +func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan<- []AppendInput) (int64, error) { r.updateReaderOffset(offsets.Min) var ( @@ -311,10 +315,29 @@ func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan< fetches, done := r.poll(ctx, offsets.Max) if len(fetches) > 0 { lastOffset = fetches[len(fetches)-1].Offset + converted := make([]AppendInput, 0, len(fetches)) + + for _, fetch := range fetches { + stream, labels, err := r.decoder.Decode(fetch.Content) + if err != nil { + return 0, fmt.Errorf("failed to decode record: %w", err) + } + if len(stream.Entries) == 0 { + continue + } + + converted = append(converted, AppendInput{ + tenant: fetch.TenantID, + labels: labels, + labelsStr: stream.Labels, + entries: stream.Entries, + }) + } + select { - case ch <- fetches: + case ch <- converted: case <-ctx.Done(): - return lastOffset, ctx.Err() + return 0, ctx.Err() } } diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 0d5a51c638674..1ee1a28937733 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -25,8 +25,6 @@ import ( "github.com/grafana/loki/v3/pkg/chunkenc" "github.com/grafana/loki/v3/pkg/compression" "github.com/grafana/loki/v3/pkg/ingester" - "github.com/grafana/loki/v3/pkg/logproto" - "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" @@ -239,8 +237,11 @@ type Slimgester struct { instancesMtx sync.RWMutex store stores.ChunkWriter - input chan AppendInput // for processing - flushQueue chan *chunk.Chunk // for flushing + input chan []AppendInput // for processing + flushQueue chan *chunk.Chunk // for flushing + + tsdbCreator *TsdbCreator + jobController *PartitionJobController grp errgroup.Group // for waiting on flushers+workers quit chan struct{} // for signaling flushers+workers @@ -253,6 +254,8 @@ func NewSlimgester( store stores.ChunkWriter, logger log.Logger, reg prometheus.Registerer, + tsdbCreator *TsdbCreator, + jobController *PartitionJobController, ) (*Slimgester, error) { i := &Slimgester{ @@ -262,18 +265,56 @@ func NewSlimgester( logger: logger, instances: make(map[string]*instance), store: store, + tsdbCreator: tsdbCreator, + jobController: jobController, - input: make(chan AppendInput), + input: make(chan []AppendInput), flushQueue: make(chan *chunk.Chunk), quit: make(chan struct{}), } - i.Service = services.NewBasicService(i.starting, i.running, i.stopping) + i.Service = services.NewBasicService(nil, i.running, i.stopping) return i, nil } -func (i *Slimgester) starting(ctx context.Context) error { +func (i *Slimgester) running(ctx context.Context) error { + select { + case <-ctx.Done(): + i.close() + case <-i.quit: + } + return nil +} + +func (i *Slimgester) stopping(e error) error { + i.close() + err := multierror.New(e) + err.Add(i.grp.Wait()) + return err.Err() +} + +// behind sync.Once b/c it can be called from etiher `running` or `stopping`. +func (i *Slimgester) close() { + i.closer.Do(func() { + close(i.quit) + }) +} + +// runOne performs a single +func (i *Slimgester) runOne(ctx context.Context) error { + exists, job, err := i.jobController.LoadJob(ctx) + if err != nil { + return err + } + + if !exists { + level.Info(i.logger).Log("msg", "no available job to process") + return nil + } + + i.jobController.part.Process(ctx, job.Offsets, i.input) + // start flush goroutines for j := 0; j < i.cfg.ConcurrentFlushes; j++ { i.grp.Go(func() error { @@ -312,10 +353,12 @@ func (i *Slimgester) starting(ctx context.Context) error { return nil case <-ctx.Done(): return ctx.Err() - case input := <-i.input: - if err := i.Append(ctx, input); err != nil { - level.Error(i.logger).Log("msg", "failed to append records", "err", err) - return err + case inputs := <-i.input: + for _, input := range inputs { + if err := i.Append(ctx, input); err != nil { + level.Error(i.logger).Log("msg", "failed to append records", "err", err) + return err + } } } } @@ -325,29 +368,6 @@ func (i *Slimgester) starting(ctx context.Context) error { } -func (i *Slimgester) running(ctx context.Context) error { - select { - case <-ctx.Done(): - i.close() - case <-i.quit: - } - return nil -} - -func (i *Slimgester) stopping(e error) error { - i.close() - err := multierror.New(e) - err.Add(i.grp.Wait()) - return err.Err() -} - -// behind sync.Once b/c it can be called from etiher `running` or `stopping`. -func (i *Slimgester) close() { - i.closer.Do(func() { - close(i.quit) - }) -} - // reportFlushedChunkStatistics calculate overall statistics of flushed chunks without compromising the flush process. func (i *Slimgester) reportFlushedChunkStatistics( ch *chunk.Chunk, @@ -397,7 +417,10 @@ func (i *Slimgester) reportFlushedChunkStatistics( type AppendInput struct { tenant string - req *logproto.PushRequest + // both labels & labelsStr are populated to prevent duplicating conversion work in mulitple places + labels labels.Labels + labelsStr string + entries []push.Entry } func (i *Slimgester) Append(ctx context.Context, input AppendInput) error { @@ -416,7 +439,7 @@ func (i *Slimgester) Append(ctx context.Context, input AppendInput) error { i.instancesMtx.Unlock() } - closed, err := inst.Push(ctx, input.req) + closed, err := inst.Push(ctx, input) for _, chk := range closed { i.flushQueue <- chk } @@ -523,46 +546,40 @@ func (i *instance) getHashForLabels(ls labels.Labels) model.Fingerprint { // Push will iterate over the given streams present in the PushRequest and attempt to store them. func (i *instance) Push( ctx context.Context, - req *logproto.PushRequest, + input AppendInput, ) (closed []*chunk.Chunk, err error) { - for _, s := range req.Streams { - err = i.streams.For( - s.Labels, - func() (*stream, error) { - ls, err := syntax.ParseLabels(s.Labels) - if err != nil { - return nil, err - } - fp := i.getHashForLabels(ls) - return newStream(fp, ls, i.metrics), nil - }, - func(stream *stream) error { - xs, err := stream.Push(s.Entries) - if err != nil { - return err - } - - if len(xs) > 0 { - for _, x := range xs { - firstTime, lastTime := util.RoundToMilliseconds(x.Bounds()) - chk := chunk.NewChunk( - i.tenant, stream.fp, stream.ls, - chunkenc.NewFacade(x, stream.blockSize, stream.targetChunkSize), - firstTime, - lastTime, - ) - // encodeChunk mutates the chunk so we must pass by reference - if err := i.encodeChunk(ctx, &chk, x); err != nil { - return err - } + err = i.streams.For( + input.labelsStr, + func() (*stream, error) { + fp := i.getHashForLabels(input.labels) + return newStream(fp, input.labels, i.metrics), nil + }, + func(stream *stream) error { + xs, err := stream.Push(input.entries) + if err != nil { + return err + } - closed = append(closed, &chk) + if len(xs) > 0 { + for _, x := range xs { + firstTime, lastTime := util.RoundToMilliseconds(x.Bounds()) + chk := chunk.NewChunk( + i.tenant, stream.fp, stream.ls, + chunkenc.NewFacade(x, stream.blockSize, stream.targetChunkSize), + firstTime, + lastTime, + ) + // encodeChunk mutates the chunk so we must pass by reference + if err := i.encodeChunk(ctx, &chk, x); err != nil { + return err } + + closed = append(closed, &chk) } - return err - }, - ) - } + } + return err + }, + ) return closed, err } diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go index 54360b5bb3fee..aca14add9f5c0 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/tsdb.go @@ -13,7 +13,7 @@ import ( // TsdbCreator accepts writes and builds TSDBs. type TsdbCreator struct { // Function to build a TSDB from the current state - mkTsdb func() ([]byte, error) + mkTsdb func(*tenantHeads) ([]byte, error) mtx sync.RWMutex shards int @@ -21,7 +21,7 @@ type TsdbCreator struct { } // new creates a new HeadManager -func newTsdbCreator(mkTsdb func() ([]byte, error)) *TsdbCreator { +func newTsdbCreator(mkTsdb func(*tenantHeads) ([]byte, error)) *TsdbCreator { m := &TsdbCreator{ mkTsdb: mkTsdb, shards: 1 << 5, // 32 shards @@ -51,7 +51,7 @@ func (m *TsdbCreator) Create() ([]byte, error) { m.mtx.Lock() defer m.mtx.Unlock() - out, err := m.mkTsdb() + out, err := m.mkTsdb(m.heads) if err != nil { return nil, errors.Wrap(err, "creating TSDB") } From 7974367b84fef4a499ed7b79c79f28536129722e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 28 Oct 2024 20:34:49 -0700 Subject: [PATCH 18/56] pipeline Signed-off-by: Owen Diehl --- pkg/blockbuilder/pipeline.go | 75 ++++++++++++++ pkg/blockbuilder/pipeline_test.go | 86 ++++++++++++++++ pkg/blockbuilder/slimgester.go | 161 +++++++++++++++--------------- 3 files changed, 244 insertions(+), 78 deletions(-) create mode 100644 pkg/blockbuilder/pipeline.go create mode 100644 pkg/blockbuilder/pipeline_test.go diff --git a/pkg/blockbuilder/pipeline.go b/pkg/blockbuilder/pipeline.go new file mode 100644 index 0000000000000..a70b5b2ee2285 --- /dev/null +++ b/pkg/blockbuilder/pipeline.go @@ -0,0 +1,75 @@ +package blockbuilder + +import ( + "context" + + "github.com/grafana/dskit/multierror" + "golang.org/x/sync/errgroup" +) + +type stage struct { + parallelism int + grp *errgroup.Group + ctx context.Context + fn func(context.Context) error + cleanup func() error // optional; will be called once the underlying group returns +} + +// pipeline is a sequence of n different stages. +type pipeline struct { + ctx context.Context // base context + stages []stage +} + +func newPipeline(ctx context.Context) *pipeline { + return &pipeline{ + ctx: ctx, + } +} + +func (p *pipeline) AddStageWithCleanup( + parallelism int, + fn func(context.Context) error, + cleanup func() error, +) { + grp, ctx := errgroup.WithContext(p.ctx) + p.stages = append(p.stages, stage{ + parallelism: parallelism, + fn: fn, + cleanup: cleanup, + ctx: ctx, + grp: grp, + }) +} + +func (p *pipeline) AddStage( + parallelism int, + fn func(context.Context) error, +) { + p.AddStageWithCleanup(parallelism, fn, nil) +} + +func (p *pipeline) Run() error { + var errs multierror.MultiError + + // begin all stages + for _, s := range p.stages { + for i := 0; i < s.parallelism; i++ { + s.grp.Go(func() error { + return s.fn(s.ctx) + }) + } + } + + // finish all stages + for _, s := range p.stages { + if err := s.grp.Wait(); err != nil { + errs.Add(err) + } + if s.cleanup != nil { + errs.Add(s.cleanup()) + } + } + + return errs.Err() +} diff --git a/pkg/blockbuilder/pipeline_test.go b/pkg/blockbuilder/pipeline_test.go new file mode 100644 index 0000000000000..a36616d2d95e7 --- /dev/null +++ b/pkg/blockbuilder/pipeline_test.go @@ -0,0 +1,86 @@ +package blockbuilder + +import ( + "context" + "errors" + "testing" + + "github.com/stretchr/testify/require" +) + +type testStage struct { + parallelism int + fn func(context.Context) error + cleanup func() error +} + +func TestPipeline(t *testing.T) { + tests := []struct { + name string + stages []testStage + expectedErr error + }{ + { + name: "single stage success", + stages: []testStage{ + { + parallelism: 1, + fn: func(ctx context.Context) error { + return nil + }, + }, + }, + }, + { + name: "multiple stages success", + stages: []testStage{ + { + parallelism: 2, + fn: func(ctx context.Context) error { + return nil + }, + }, + { + parallelism: 1, + fn: func(ctx context.Context) error { + return nil + }, + }, + }, + }, + { + name: "stage error propagates", + stages: []testStage{ + { + parallelism: 1, + fn: func(ctx context.Context) error { + return errors.New("stage error") + }, + }, + }, + expectedErr: errors.New("stage error"), + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + p := newPipeline(context.Background()) + + for _, stage := range tt.stages { + if stage.cleanup != nil { + p.AddStageWithCleanup(stage.parallelism, stage.fn, stage.cleanup) + } else { + p.AddStage(stage.parallelism, stage.fn) + } + } + + err := p.Run() + if tt.expectedErr != nil { + require.Error(t, err) + require.Equal(t, tt.expectedErr.Error(), err.Error()) + } else { + require.NoError(t, err) + } + }) + } +} diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 1ee1a28937733..6004967e210cd 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -11,14 +11,12 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" - "github.com/grafana/dskit/multierror" "github.com/grafana/dskit/services" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" - "golang.org/x/sync/errgroup" "github.com/grafana/loki/pkg/push" "github.com/grafana/loki/v3/pkg/analytics" @@ -236,16 +234,10 @@ type Slimgester struct { instances map[string]*instance instancesMtx sync.RWMutex - store stores.ChunkWriter - input chan []AppendInput // for processing - flushQueue chan *chunk.Chunk // for flushing + store stores.ChunkWriter tsdbCreator *TsdbCreator jobController *PartitionJobController - - grp errgroup.Group // for waiting on flushers+workers - quit chan struct{} // for signaling flushers+workers - closer sync.Once // for coordinating channel closure } func NewSlimgester( @@ -267,63 +259,100 @@ func NewSlimgester( store: store, tsdbCreator: tsdbCreator, jobController: jobController, - - input: make(chan []AppendInput), - flushQueue: make(chan *chunk.Chunk), - - quit: make(chan struct{}), } - i.Service = services.NewBasicService(nil, i.running, i.stopping) + i.Service = services.NewBasicService(nil, nil, nil) return i, nil } -func (i *Slimgester) running(ctx context.Context) error { - select { - case <-ctx.Done(): - i.close() - case <-i.quit: - } - return nil -} - -func (i *Slimgester) stopping(e error) error { - i.close() - err := multierror.New(e) - err.Add(i.grp.Wait()) - return err.Err() -} - -// behind sync.Once b/c it can be called from etiher `running` or `stopping`. -func (i *Slimgester) close() { - i.closer.Do(func() { - close(i.quit) - }) -} - // runOne performs a single -func (i *Slimgester) runOne(ctx context.Context) error { +func (i *Slimgester) runOne(ctx context.Context) (lastOffset int64, skipped bool, err error) { + exists, job, err := i.jobController.LoadJob(ctx) if err != nil { - return err + return 0, false, err } if !exists { level.Info(i.logger).Log("msg", "no available job to process") - return nil + return 0, true, nil } - i.jobController.part.Process(ctx, job.Offsets, i.input) + p := newPipeline(ctx) + + // Pipeline stage 1: Process the job offsets and write records to inputCh + // This stage reads from the partition and feeds records into the input channel + // When complete, it stores the last processed offset and closes the channel + inputCh := make(chan []AppendInput) + p.AddStageWithCleanup( + 1, + func(ctx context.Context) error { + lastOffset, err = i.jobController.part.Process(ctx, job.Offsets, inputCh) + return err + }, + func() error { + close(inputCh) + return nil + }, + ) + + // Stage 2: Process input records and generate chunks + // This stage receives AppendInput batches, appends them to appropriate instances, + // and forwards any cut chunks to the chunks channel for flushing. + // ConcurrentWriters workers process inputs in parallel to maximize throughput. + flush := make(chan *chunk.Chunk) + p.AddStageWithCleanup( + i.cfg.ConcurrentWriters, + func(ctx context.Context) error { - // start flush goroutines - for j := 0; j < i.cfg.ConcurrentFlushes; j++ { - i.grp.Go(func() error { for { select { - case <-i.quit: - return nil case <-ctx.Done(): - case chk := <-i.flushQueue: + return ctx.Err() + case inputs, ok := <-inputCh: + // inputs are finished; we're done + if !ok { + return nil + } + + for _, input := range inputs { + cut, err := i.Append(ctx, input) + if err != nil { + level.Error(i.logger).Log("msg", "failed to append records", "err", err) + return err + } + + for _, chk := range cut { + select { + case <-ctx.Done(): + return ctx.Err() + case flush <- chk: + } + } + } + } + } + }, + func() error { + close(flush) + return nil + }, + ) + + // Stage 3: Flush chunks to storage + // This stage receives chunks from the chunks channel and flushes them to storage + // using ConcurrentFlushes workers for parallel processing + p.AddStage( + i.cfg.ConcurrentFlushes, + func(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + case chk, ok := <-flush: + if !ok { + return nil + } if _, err := withBackoff( ctx, defaultBackoffConfig, // retry forever @@ -341,31 +370,11 @@ func (i *Slimgester) runOne(ctx context.Context) error { } } } - }) - } - - // start chunk building goroutines - for j := 0; j < i.cfg.ConcurrentWriters; j++ { - i.grp.Go(func() error { - for { - select { - case <-i.quit: - return nil - case <-ctx.Done(): - return ctx.Err() - case inputs := <-i.input: - for _, input := range inputs { - if err := i.Append(ctx, input); err != nil { - level.Error(i.logger).Log("msg", "failed to append records", "err", err) - return err - } - } - } - } - }) - } - return nil + }, + ) + err = p.Run() + return lastOffset, false, err } // reportFlushedChunkStatistics calculate overall statistics of flushed chunks without compromising the flush process. @@ -423,7 +432,7 @@ type AppendInput struct { entries []push.Entry } -func (i *Slimgester) Append(ctx context.Context, input AppendInput) error { +func (i *Slimgester) Append(ctx context.Context, input AppendInput) ([]*chunk.Chunk, error) { // use rlock so multiple appends can be called on same instance. // re-check after using regular lock if it didnt exist. i.instancesMtx.RLock() @@ -440,11 +449,7 @@ func (i *Slimgester) Append(ctx context.Context, input AppendInput) error { } closed, err := inst.Push(ctx, input) - for _, chk := range closed { - i.flushQueue <- chk - } - - return err + return closed, err } // instance is a slimmed down version from the ingester pkg From 6ff0be8ec05bb38a06fefb1f5ae200a9927f0e26 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 29 Oct 2024 11:02:55 -0700 Subject: [PATCH 19/56] commit offset Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 36 +++++++++++++++++++++++++++++----- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 6004967e210cd..e76b3b01ecfd8 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -261,23 +261,41 @@ func NewSlimgester( jobController: jobController, } - i.Service = services.NewBasicService(nil, nil, nil) + i.Service = services.NewBasicService(nil, i.running, nil) return i, nil } +func (i *Slimgester) running(ctx context.Context) error { + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() + + for { + select { + case <-ctx.Done(): + return nil + case <-ticker.C: + _, err := i.runOne(ctx) + if err != nil { + return err + } + } + } +} + // runOne performs a single -func (i *Slimgester) runOne(ctx context.Context) (lastOffset int64, skipped bool, err error) { +func (i *Slimgester) runOne(ctx context.Context) (skipped bool, err error) { exists, job, err := i.jobController.LoadJob(ctx) if err != nil { - return 0, false, err + return false, err } if !exists { level.Info(i.logger).Log("msg", "no available job to process") - return 0, true, nil + return true, nil } + var lastOffset int64 p := newPipeline(ctx) // Pipeline stage 1: Process the job offsets and write records to inputCh @@ -374,7 +392,15 @@ func (i *Slimgester) runOne(ctx context.Context) (lastOffset int64, skipped bool ) err = p.Run() - return lastOffset, false, err + if err != nil { + return false, err + } + + if err = i.jobController.part.Commit(ctx, lastOffset); err != nil { + return false, err + } + + return false, nil } // reportFlushedChunkStatistics calculate overall statistics of flushed chunks without compromising the flush process. From 5e56f5acbb60741f7ccbb261dcb1b741f2194254 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 29 Oct 2024 12:49:54 -0700 Subject: [PATCH 20/56] minor refactoring + wiring up Signed-off-by: Owen Diehl --- pkg/blockbuilder/controller.go | 19 +++- pkg/blockbuilder/metrics.go | 151 ++++++++++++++++++++++++++ pkg/blockbuilder/partition.go | 21 ++-- pkg/blockbuilder/slimgester.go | 187 ++++----------------------------- pkg/kafka/partition/reader.go | 6 +- pkg/loki/loki.go | 9 +- pkg/loki/modules.go | 43 ++++++++ 7 files changed, 252 insertions(+), 184 deletions(-) create mode 100644 pkg/blockbuilder/metrics.go diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index d5f05271e0799..146469d27bdbf 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -16,6 +16,8 @@ type Job struct { // Interface required for interacting with queue partitions. type PartitionController interface { + Topic() string + Partition() int32 // Returns the highest committed offset from the consumer group HighestCommittedOffset(ctx context.Context) (int64, error) // Returns the highest available offset in the partition @@ -49,10 +51,17 @@ type PartitionController interface { // // containing log data and "committed" is the consumer group type PartitionJobController struct { - topic string - partition int32 - stepLen int64 - part PartitionController + stepLen int64 + part PartitionController +} + +func NewPartitionJobController( + controller PartitionController, +) *PartitionJobController { + return &PartitionJobController{ + stepLen: 1000, // Default step length of 1000 offsets per job + part: controller, + } } // LoadJob(ctx) returns the next job by finding the most recent unconsumed offset in the partition @@ -75,7 +84,7 @@ func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) // Create the job with the calculated offsets job := Job{ - Partition: l.partition, + Partition: l.part.Partition(), Offsets: Offsets{ Min: startOffset, Max: startOffset + l.stepLen, diff --git a/pkg/blockbuilder/metrics.go b/pkg/blockbuilder/metrics.go new file mode 100644 index 0000000000000..5835bed1c303c --- /dev/null +++ b/pkg/blockbuilder/metrics.go @@ -0,0 +1,151 @@ +package blockbuilder + +import ( + "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/util/constants" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +type SlimgesterMetrics struct { + chunkUtilization prometheus.Histogram + chunkEntries prometheus.Histogram + chunkSize prometheus.Histogram + chunkCompressionRatio prometheus.Histogram + chunksPerTenant *prometheus.CounterVec + chunkSizePerTenant *prometheus.CounterVec + chunkAge prometheus.Histogram + chunkEncodeTime prometheus.Histogram + chunksFlushFailures prometheus.Counter + chunksFlushedPerReason *prometheus.CounterVec + chunkLifespan prometheus.Histogram + chunksEncoded *prometheus.CounterVec + chunkDecodeFailures *prometheus.CounterVec + flushedChunksStats *analytics.Counter + flushedChunksBytesStats *analytics.Statistics + flushedChunksLinesStats *analytics.Statistics + flushedChunksAgeStats *analytics.Statistics + flushedChunksLifespanStats *analytics.Statistics + flushedChunksUtilizationStats *analytics.Statistics + + chunksCreatedTotal prometheus.Counter + samplesPerChunk prometheus.Histogram + blocksPerChunk prometheus.Histogram + chunkCreatedStats *analytics.Counter +} + +func NewSlimgesterMetrics(r prometheus.Registerer) *SlimgesterMetrics { + return &SlimgesterMetrics{ + chunkUtilization: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_utilization", + Help: "Distribution of stored chunk utilization (when stored).", + Buckets: prometheus.LinearBuckets(0, 0.2, 6), + }), + chunkEntries: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_entries", + Help: "Distribution of stored lines per chunk (when stored).", + Buckets: prometheus.ExponentialBuckets(200, 2, 9), // biggest bucket is 200*2^(9-1) = 51200 + }), + chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_size_bytes", + Help: "Distribution of stored chunk sizes (when stored).", + Buckets: prometheus.ExponentialBuckets(20000, 2, 10), // biggest bucket is 20000*2^(10-1) = 10,240,000 (~10.2MB) + }), + chunkCompressionRatio: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_compression_ratio", + Help: "Compression ratio of chunks (when stored).", + Buckets: prometheus.LinearBuckets(.75, 2, 10), + }), + chunksPerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_stored_total", + Help: "Total stored chunks per tenant.", + }, []string{"tenant"}), + chunkSizePerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_stored_bytes_total", + Help: "Total bytes stored in chunks per tenant.", + }, []string{"tenant"}), + chunkAge: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_age_seconds", + Help: "Distribution of chunk ages (when stored).", + // with default settings chunks should flush between 5 min and 12 hours + // so buckets at 1min, 5min, 10min, 30min, 1hr, 2hr, 4hr, 10hr, 12hr, 16hr + Buckets: []float64{60, 300, 600, 1800, 3600, 7200, 14400, 36000, 43200, 57600}, + }), + chunkEncodeTime: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_encode_time_seconds", + Help: "Distribution of chunk encode times.", + // 10ms to 10s. + Buckets: prometheus.ExponentialBuckets(0.01, 4, 6), + }), + chunksFlushFailures: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_flush_failures_total", + Help: "Total number of flush failures.", + }), + chunksFlushedPerReason: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_flushed_total", + Help: "Total flushed chunks per reason.", + }, []string{"reason"}), + chunkLifespan: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_bounds_hours", + Help: "Distribution of chunk end-start durations.", + // 1h -> 8hr + Buckets: prometheus.LinearBuckets(1, 1, 8), + }), + chunksEncoded: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_encoded_total", + Help: "The total number of chunks encoded in the ingester.", + }, []string{"user"}), + chunkDecodeFailures: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunk_decode_failures_total", + Help: "The number of freshly encoded chunks that failed to decode.", + }, []string{"user"}), + flushedChunksStats: analytics.NewCounter("slimgester_flushed_chunks"), + flushedChunksBytesStats: analytics.NewStatistics("slimgester_flushed_chunks_bytes"), + flushedChunksLinesStats: analytics.NewStatistics("slimgester_flushed_chunks_lines"), + flushedChunksAgeStats: analytics.NewStatistics( + "slimgester_flushed_chunks_age_seconds", + ), + flushedChunksLifespanStats: analytics.NewStatistics( + "slimgester_flushed_chunks_lifespan_seconds", + ), + flushedChunksUtilizationStats: analytics.NewStatistics( + "slimgester_flushed_chunks_utilization", + ), + chunksCreatedTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "slimgester_chunks_created_total", + Help: "The total number of chunks created in the ingester.", + }), + samplesPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Subsystem: "slimgester", + Name: "samples_per_chunk", + Help: "The number of samples in a chunk.", + + Buckets: prometheus.LinearBuckets(4096, 2048, 6), + }), + blocksPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Subsystem: "slimgester", + Name: "blocks_per_chunk", + Help: "The number of blocks in a chunk.", + + Buckets: prometheus.ExponentialBuckets(5, 2, 6), + }), + + chunkCreatedStats: analytics.NewCounter("slimgester_chunk_created"), + } +} diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index 3631c000bb5e7..361ea10d51611 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -17,6 +17,7 @@ import ( "github.com/twmb/franz-go/pkg/kmsg" "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/kafka/client" "github.com/grafana/loki/v3/pkg/kafka/partition" ) @@ -45,23 +46,22 @@ type partitionReader struct { } func NewPartitionReader( - topic string, - group string, + kafkaCfg kafka.Config, partitionID int32, - decoder *kafka.Decoder, + instanceID string, logger log.Logger, r prometheus.Registerer, ) (*partitionReader, error) { readerMetrics := partition.NewReaderMetrics(r) writerMetrics := partition.NewCommitterMetrics(r, partitionID) + group := kafkaCfg.GetConsumerGroup(instanceID, partitionID) - opts := []kgo.Opt{ - kgo.SeedBrokers([]string{"localhost:9092"}...), - kgo.ConsumerGroup(group), - kgo.ConsumeTopics(topic), + decoder, err := kafka.NewDecoder() + if err != nil { + return nil, err } - client, err := kgo.NewClient(opts...) + client, err := client.NewReaderClient(kafkaCfg, readerMetrics.Kprom, logger) if err != nil { return nil, err } @@ -69,7 +69,7 @@ func NewPartitionReader( aClient := kadm.NewClient(client) return &partitionReader{ - topic: topic, + topic: kafkaCfg.Topic, group: group, partitionID: partitionID, readerMetrics: readerMetrics, @@ -81,6 +81,9 @@ func NewPartitionReader( }, nil } +func (r *partitionReader) Topic() string { return r.topic } +func (r *partitionReader) Partition() int32 { return r.partitionID } + // Fetches the desired offset in the partition itself, not the consumer group // NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` func (r *partitionReader) fetchPartitionOffset(ctx context.Context, position int64) (int64, error) { diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index e76b3b01ecfd8..0667f6baf7042 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -5,7 +5,6 @@ import ( "context" "flag" "fmt" - "runtime" "sync" "time" @@ -14,12 +13,10 @@ import ( "github.com/grafana/dskit/services" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/push" - "github.com/grafana/loki/v3/pkg/analytics" "github.com/grafana/loki/v3/pkg/chunkenc" "github.com/grafana/loki/v3/pkg/compression" "github.com/grafana/loki/v3/pkg/ingester" @@ -27,179 +24,39 @@ import ( "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" "github.com/grafana/loki/v3/pkg/util" - "github.com/grafana/loki/v3/pkg/util/constants" + "github.com/grafana/loki/v3/pkg/util/flagext" ) const ( flushReasonFull = "full" flushReasonMaxAge = "max_age" + onePointFiveMB = 3 << 19 ) -type SlimgesterMetrics struct { - chunkUtilization prometheus.Histogram - chunkEntries prometheus.Histogram - chunkSize prometheus.Histogram - chunkCompressionRatio prometheus.Histogram - chunksPerTenant *prometheus.CounterVec - chunkSizePerTenant *prometheus.CounterVec - chunkAge prometheus.Histogram - chunkEncodeTime prometheus.Histogram - chunksFlushFailures prometheus.Counter - chunksFlushedPerReason *prometheus.CounterVec - chunkLifespan prometheus.Histogram - chunksEncoded *prometheus.CounterVec - chunkDecodeFailures *prometheus.CounterVec - flushedChunksStats *analytics.Counter - flushedChunksBytesStats *analytics.Statistics - flushedChunksLinesStats *analytics.Statistics - flushedChunksAgeStats *analytics.Statistics - flushedChunksLifespanStats *analytics.Statistics - flushedChunksUtilizationStats *analytics.Statistics - - chunksCreatedTotal prometheus.Counter - samplesPerChunk prometheus.Histogram - blocksPerChunk prometheus.Histogram - chunkCreatedStats *analytics.Counter -} - -func NewSlimgesterMetrics(r prometheus.Registerer) *SlimgesterMetrics { - return &SlimgesterMetrics{ - chunkUtilization: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_utilization", - Help: "Distribution of stored chunk utilization (when stored).", - Buckets: prometheus.LinearBuckets(0, 0.2, 6), - }), - chunkEntries: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_entries", - Help: "Distribution of stored lines per chunk (when stored).", - Buckets: prometheus.ExponentialBuckets(200, 2, 9), // biggest bucket is 200*2^(9-1) = 51200 - }), - chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_size_bytes", - Help: "Distribution of stored chunk sizes (when stored).", - Buckets: prometheus.ExponentialBuckets(20000, 2, 10), // biggest bucket is 20000*2^(10-1) = 10,240,000 (~10.2MB) - }), - chunkCompressionRatio: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_compression_ratio", - Help: "Compression ratio of chunks (when stored).", - Buckets: prometheus.LinearBuckets(.75, 2, 10), - }), - chunksPerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunks_stored_total", - Help: "Total stored chunks per tenant.", - }, []string{"tenant"}), - chunkSizePerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_stored_bytes_total", - Help: "Total bytes stored in chunks per tenant.", - }, []string{"tenant"}), - chunkAge: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_age_seconds", - Help: "Distribution of chunk ages (when stored).", - // with default settings chunks should flush between 5 min and 12 hours - // so buckets at 1min, 5min, 10min, 30min, 1hr, 2hr, 4hr, 10hr, 12hr, 16hr - Buckets: []float64{60, 300, 600, 1800, 3600, 7200, 14400, 36000, 43200, 57600}, - }), - chunkEncodeTime: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_encode_time_seconds", - Help: "Distribution of chunk encode times.", - // 10ms to 10s. - Buckets: prometheus.ExponentialBuckets(0.01, 4, 6), - }), - chunksFlushFailures: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunks_flush_failures_total", - Help: "Total number of flush failures.", - }), - chunksFlushedPerReason: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunks_flushed_total", - Help: "Total flushed chunks per reason.", - }, []string{"reason"}), - chunkLifespan: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_bounds_hours", - Help: "Distribution of chunk end-start durations.", - // 1h -> 8hr - Buckets: prometheus.LinearBuckets(1, 1, 8), - }), - chunksEncoded: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunks_encoded_total", - Help: "The total number of chunks encoded in the ingester.", - }, []string{"user"}), - chunkDecodeFailures: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunk_decode_failures_total", - Help: "The number of freshly encoded chunks that failed to decode.", - }, []string{"user"}), - flushedChunksStats: analytics.NewCounter("slimgester_flushed_chunks"), - flushedChunksBytesStats: analytics.NewStatistics("slimgester_flushed_chunks_bytes"), - flushedChunksLinesStats: analytics.NewStatistics("slimgester_flushed_chunks_lines"), - flushedChunksAgeStats: analytics.NewStatistics( - "slimgester_flushed_chunks_age_seconds", - ), - flushedChunksLifespanStats: analytics.NewStatistics( - "slimgester_flushed_chunks_lifespan_seconds", - ), - flushedChunksUtilizationStats: analytics.NewStatistics( - "slimgester_flushed_chunks_utilization", - ), - chunksCreatedTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: constants.Loki, - Name: "slimgester_chunks_created_total", - Help: "The total number of chunks created in the ingester.", - }), - samplesPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Subsystem: "slimgester", - Name: "samples_per_chunk", - Help: "The number of samples in a chunk.", - - Buckets: prometheus.LinearBuckets(4096, 2048, 6), - }), - blocksPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Subsystem: "slimgester", - Name: "blocks_per_chunk", - Help: "The number of blocks in a chunk.", - - Buckets: prometheus.ExponentialBuckets(5, 2, 6), - }), - - chunkCreatedStats: analytics.NewCounter("slimgester_chunk_created"), - } -} - type Config struct { ConcurrentFlushes int `yaml:"concurrent_flushes"` ConcurrentWriters int `yaml:"concurrent_writers"` - BlockSize int `yaml:"chunk_block_size"` - TargetChunkSize int `yaml:"chunk_target_size"` + BlockSize flagext.ByteSize `yaml:"chunk_block_size"` + TargetChunkSize flagext.ByteSize `yaml:"chunk_target_size"` ChunkEncoding string `yaml:"chunk_encoding"` parsedEncoding compression.Codec `yaml:"-"` // placeholder for validated encoding MaxChunkAge time.Duration `yaml:"max_chunk_age"` } func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { - f.IntVar(&cfg.ConcurrentFlushes, "ingester.concurrent-flushes", 16, "How many flushes can happen concurrently") - f.IntVar(&cfg.ConcurrentWriters, "ingester.concurrent-writers", runtime.NumCPU(), "How many workers to process writes, defaults to number of available cpus") - f.IntVar(&cfg.BlockSize, "ingester.chunks-block-size", 256*1024, "The targeted _uncompressed_ size in bytes of a chunk block When this threshold is exceeded the head block will be cut and compressed inside the chunk.") - f.IntVar(&cfg.TargetChunkSize, "ingester.chunk-target-size", 1572864, "A target _compressed_ size in bytes for chunks. This is a desired size not an exact size, chunks may be slightly bigger or significantly smaller if they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 creates chunks with a fixed 10 blocks, a non zero value will create chunks with a variable number of blocks to meet the target size.") // 1.5 MB - f.StringVar(&cfg.ChunkEncoding, "ingester.chunk-encoding", compression.GZIP.String(), fmt.Sprintf("The algorithm to use for compressing chunk. (%s)", compression.SupportedCodecs())) - f.DurationVar(&cfg.MaxChunkAge, "ingester.max-chunk-age", 2*time.Hour, "The maximum duration of a timeseries chunk in memory. If a timeseries runs for longer than this, the current chunk will be flushed to the store and a new chunk created.") + f.IntVar(&cfg.ConcurrentFlushes, prefix+"concurrent-flushes", 16, "How many flushes can happen concurrently") + f.IntVar(&cfg.ConcurrentWriters, prefix+"concurrent-writers", 16, "How many workers to process writes, defaults to number of available cpus") + _ = cfg.BlockSize.Set("256KB") + f.Var(&cfg.BlockSize, prefix+"chunks-block-size", "The targeted _uncompressed_ size in bytes of a chunk block When this threshold is exceeded the head block will be cut and compressed inside the chunk.") + _ = cfg.TargetChunkSize.Set(fmt.Sprint(onePointFiveMB)) + f.Var(&cfg.TargetChunkSize, prefix+"chunk-target-size", "A target _compressed_ size in bytes for chunks. This is a desired size not an exact size, chunks may be slightly bigger or significantly smaller if they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 creates chunks with a fixed 10 blocks, a non zero value will create chunks with a variable number of blocks to meet the target size.") + f.StringVar(&cfg.ChunkEncoding, prefix+"chunk-encoding", compression.Snappy.String(), fmt.Sprintf("The algorithm to use for compressing chunk. (%s)", compression.SupportedCodecs())) + f.DurationVar(&cfg.MaxChunkAge, prefix+"max-chunk-age", 2*time.Hour, "The maximum duration of a timeseries chunk in memory. If a timeseries runs for longer than this, the current chunk will be flushed to the store and a new chunk created.") } // RegisterFlags registers flags. func (c *Config) RegisterFlags(flags *flag.FlagSet) { - c.RegisterFlagsWithPrefix("slimgester", flags) + c.RegisterFlagsWithPrefix("blockbuilder.", flags) } func (cfg *Config) Validate() error { @@ -211,7 +68,7 @@ func (cfg *Config) Validate() error { return nil } -// Slimgester is a slimmed-down version of the ingester, intended to +// BlockBuilder is a slimmed-down version of the ingester, intended to // ingest logs without WALs. Broadly, it accumulates logs into per-tenant chunks in the same way the existing ingester does, // without a WAL. Index (TSDB) creation is also not an out-of-band procedure and must be called directly. In essence, this // allows us to buffer data, flushing chunks to storage as necessary, and then when ready to commit this, relevant TSDBs (one per period) are created and flushed to storage. This allows an external caller to prepare a batch of data, build relevant chunks+indices, ensure they're flushed, and then return. As long as chunk+index creation is deterministic, this operation is also @@ -222,7 +79,7 @@ func (cfg *Config) Validate() error { // - `Commit(context.Context) error` // Serializes (cuts) any buffered data into chunks, flushes them to storage, then creates + flushes TSDB indices // containing all chunk references. Finally, clears internal state. -type Slimgester struct { +type BlockBuilder struct { services.Service cfg Config @@ -240,7 +97,7 @@ type Slimgester struct { jobController *PartitionJobController } -func NewSlimgester( +func NewBlockBuilder( cfg Config, periodicConfigs []config.PeriodConfig, store stores.ChunkWriter, @@ -248,9 +105,9 @@ func NewSlimgester( reg prometheus.Registerer, tsdbCreator *TsdbCreator, jobController *PartitionJobController, -) (*Slimgester, +) (*BlockBuilder, error) { - i := &Slimgester{ + i := &BlockBuilder{ cfg: cfg, periodicConfigs: periodicConfigs, metrics: NewSlimgesterMetrics(reg), @@ -265,7 +122,7 @@ func NewSlimgester( return i, nil } -func (i *Slimgester) running(ctx context.Context) error { +func (i *BlockBuilder) running(ctx context.Context) error { ticker := time.NewTicker(10 * time.Second) defer ticker.Stop() @@ -283,7 +140,7 @@ func (i *Slimgester) running(ctx context.Context) error { } // runOne performs a single -func (i *Slimgester) runOne(ctx context.Context) (skipped bool, err error) { +func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { exists, job, err := i.jobController.LoadJob(ctx) if err != nil { @@ -404,7 +261,7 @@ func (i *Slimgester) runOne(ctx context.Context) (skipped bool, err error) { } // reportFlushedChunkStatistics calculate overall statistics of flushed chunks without compromising the flush process. -func (i *Slimgester) reportFlushedChunkStatistics( +func (i *BlockBuilder) reportFlushedChunkStatistics( ch *chunk.Chunk, ) { byt, err := ch.Encoded() @@ -458,7 +315,7 @@ type AppendInput struct { entries []push.Entry } -func (i *Slimgester) Append(ctx context.Context, input AppendInput) ([]*chunk.Chunk, error) { +func (i *BlockBuilder) Append(ctx context.Context, input AppendInput) ([]*chunk.Chunk, error) { // use rlock so multiple appends can be called on same instance. // re-check after using regular lock if it didnt exist. i.instancesMtx.RLock() diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index 5625fbac7c7ae..3ea84293ac405 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -95,7 +95,7 @@ func NewReader( // This method is called when the PartitionReader service starts. func (p *Reader) start(ctx context.Context) error { var err error - p.client, err = client.NewReaderClient(p.kafkaCfg, p.metrics.kprom, p.logger) + p.client, err = client.NewReaderClient(p.kafkaCfg, p.metrics.Kprom, p.logger) if err != nil { return errors.Wrap(err, "creating kafka reader client") } @@ -520,7 +520,7 @@ type ReaderMetrics struct { // strongConsistencyInstrumentation *StrongReadConsistencyInstrumentation[struct{}] // lastConsumedOffset prometheus.Gauge ConsumeLatency prometheus.Histogram - kprom *kprom.Metrics + Kprom *kprom.Metrics } // NewReaderMetrics initializes and returns a new set of metrics for the PartitionReader. @@ -538,7 +538,7 @@ func NewReaderMetrics(reg prometheus.Registerer) *ReaderMetrics { return &ReaderMetrics{ ReceiveDelayWhenStarting: receiveDelay.WithLabelValues("starting"), ReceiveDelayWhenRunning: receiveDelay.WithLabelValues("running"), - kprom: client.NewReaderClientMetrics("partition-reader", reg), + Kprom: client.NewReaderClientMetrics("partition-reader", reg), FetchWaitDuration: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ Name: "loki_ingest_storage_reader_records_batch_wait_duration_seconds", Help: "How long a consumer spent waiting for a batch of records from the Kafka client. If fetching is faster than processing, then this will be close to 0.", diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index f59218307e7d7..b1e1ac62b551a 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -31,6 +31,7 @@ import ( "google.golang.org/grpc/health/grpc_health_v1" "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/blockbuilder" "github.com/grafana/loki/v3/pkg/bloombuild" "github.com/grafana/loki/v3/pkg/bloomgateway" "github.com/grafana/loki/v3/pkg/compactor" @@ -44,7 +45,6 @@ import ( metastoreclient "github.com/grafana/loki/v3/pkg/ingester-rf1/metastore/client" ingester_client "github.com/grafana/loki/v3/pkg/ingester/client" "github.com/grafana/loki/v3/pkg/kafka" - ingester_kafka "github.com/grafana/loki/v3/pkg/kafka/ingester" "github.com/grafana/loki/v3/pkg/loghttp/push" "github.com/grafana/loki/v3/pkg/loki/common" "github.com/grafana/loki/v3/pkg/lokifrontend" @@ -95,6 +95,7 @@ type Config struct { IngesterClient ingester_client.Config `yaml:"ingester_client,omitempty"` IngesterRF1Client ingester_client.Config `yaml:"ingester_rf1_client,omitempty"` Ingester ingester.Config `yaml:"ingester,omitempty"` + BlockBuilder blockbuilder.Config `yaml:"block_builder,omitempty"` Pattern pattern.Config `yaml:"pattern_ingester,omitempty"` IndexGateway indexgateway.Config `yaml:"index_gateway"` BloomBuild bloombuild.Config `yaml:"bloom_build,omitempty" category:"experimental"` @@ -265,6 +266,9 @@ func (c *Config) Validate() error { if err := c.Ingester.Validate(); err != nil { errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid ingester config")) } + if err := c.BlockBuilder.Validate(); err != nil { + errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid block_builder config")) + } if err := c.LimitsConfig.Validate(); err != nil { errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid limits_config config")) } @@ -382,7 +386,7 @@ type Loki struct { MetastoreClient *metastoreclient.Client partitionRingWatcher *ring.PartitionRingWatcher partitionRing *ring.PartitionInstanceRing - kafkaIngester *ingester_kafka.Ingester + blockBuilder *blockbuilder.BlockBuilder ClientMetrics storage.ClientMetrics deleteClientMetrics *deletion.DeleteRequestClientMetrics @@ -704,6 +708,7 @@ func (t *Loki) setupModuleManager() error { mm.RegisterModule(PatternIngesterTee, t.initPatternIngesterTee, modules.UserInvisibleModule) mm.RegisterModule(PatternIngester, t.initPatternIngester) mm.RegisterModule(PartitionRing, t.initPartitionRing, modules.UserInvisibleModule) + mm.RegisterModule(BlockBuilder, t.initBlockBuilder) mm.RegisterModule(All, nil) mm.RegisterModule(Read, nil) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 86b5e668167cb..e2d312a98d695 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -35,6 +35,7 @@ import ( "github.com/prometheus/common/model" "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/blockbuilder" "github.com/grafana/loki/v3/pkg/bloombuild/builder" "github.com/grafana/loki/v3/pkg/bloombuild/planner" bloomprotos "github.com/grafana/loki/v3/pkg/bloombuild/protos" @@ -48,6 +49,7 @@ import ( "github.com/grafana/loki/v3/pkg/indexgateway" "github.com/grafana/loki/v3/pkg/ingester" "github.com/grafana/loki/v3/pkg/ingester-rf1/objstore" + "github.com/grafana/loki/v3/pkg/kafka/partitionring" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logqlmodel/stats" @@ -136,6 +138,7 @@ const ( Analytics string = "analytics" InitCodec string = "init-codec" PartitionRing string = "partition-ring" + BlockBuilder string = "block-builder" ) const ( @@ -1776,6 +1779,46 @@ func (t *Loki) initPartitionRing() (services.Service, error) { return t.partitionRingWatcher, nil } +func (t *Loki) initBlockBuilder() (services.Service, error) { + logger := log.With(util_log.Logger, "component", "block_builder") + + id := t.Cfg.Ingester.LifecyclerConfig.ID + ingestPartitionID, err := partitionring.ExtractIngesterPartitionID(id) + if err != nil { + return nil, fmt.Errorf("calculating block builder partition ID: %w", err) + } + + reader, err := blockbuilder.NewPartitionReader( + t.Cfg.KafkaConfig, + ingestPartitionID, + id, + logger, + prometheus.DefaultRegisterer, + ) + if err != nil { + return nil, err + } + + bb, err := blockbuilder.NewBlockBuilder( + t.Cfg.BlockBuilder, + t.Cfg.SchemaConfig.Configs, + t.Store, + logger, + prometheus.DefaultRegisterer, + tsdbCreator, + blockbuilder.NewPartitionJobController( + reader, + ), + ) + + if err != nil { + return nil, err + } + + t.blockBuilder = bb + return t.blockBuilder, nil +} + func (t *Loki) deleteRequestsClient(clientType string, limits limiter.CombinedLimits) (deletion.DeleteRequestsClient, error) { if !t.supportIndexDeleteRequest() || !t.Cfg.CompactorConfig.RetentionEnabled { return deletion.NewNoOpDeleteRequestsStore(), nil From 9ae5bc5989f3b04329c166f8c92e77cfd5cce099 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 5 Nov 2024 08:49:19 -0800 Subject: [PATCH 21/56] merge cleanup Signed-off-by: Owen Diehl --- pkg/blockbuilder/controller.go | 4 +- pkg/blockbuilder/partition.go | 4 +- pkg/blockbuilder/slimgester.go | 5 ++- pkg/blockbuilder/writer.go | 3 +- pkg/kafka/partition/committer_test.go | 12 +++--- pkg/kafka/partition/metrics.go | 58 +++++++++++++-------------- pkg/kafka/partition/reader.go | 8 ++-- 7 files changed, 48 insertions(+), 46 deletions(-) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index 146469d27bdbf..45798e16614e2 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -39,9 +39,9 @@ type PartitionController interface { // * offset_step_len: the number of offsets each job to contain. e.g. "10" could yield a job w / min=15, max=25 // // At a high level, it watches a source topic/partition (where log data is ingested) and a "committed" topic/partition. -// The "comitted" partition corresponds to the offsets from the source partition which have been committed to object storage. +// The "committed" partition corresponds to the offsets from the source partition which have been committed to object storage. // In essence, the following loop is performed -// 1. load the most recent record from the "comitted" partition. This contains the highest msg offset in the "source" partition +// 1. load the most recent record from the "committed" partition. This contains the highest msg offset in the "source" partition // that has been committed to object storage. We'll call that $START_POS. // 2. Create a job with `min=$START_POS+1,end=$START_POS+1+$STEP_LEN` // 3. Sometime later when the job has been processed, we'll commit the final processed offset from the "source" partition (which diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index 361ea10d51611..c1beb940bc4fd 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -141,7 +141,7 @@ func (r *partitionReader) fetchPartitionOffset(ctx context.Context, position int return listRes.Topics[0].Partitions[0].Offset, nil } -// Fetches the highest committe offset in the consumer group +// Fetches the highest committee offset in the consumer group // NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` // TODO(owen-d): expose errors: the failure case of restarting at // the beginning of a partition is costly and duplicates data @@ -298,7 +298,7 @@ func (p *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { fetches.EachRecord(func(record *kgo.Record) { numRecords++ delay := now.Sub(record.Timestamp).Seconds() - p.readerMetrics.ReceiveDelayWhenRunning.Observe(delay) + p.readerMetrics.ReceiveDelay.WithLabelValues(partition.PhaseRunning).Observe(delay) }) p.readerMetrics.FetchesTotal.Add(float64(len(fetches))) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 0667f6baf7042..c20e3d843bd7c 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -16,7 +16,6 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" - "github.com/grafana/loki/pkg/push" "github.com/grafana/loki/v3/pkg/chunkenc" "github.com/grafana/loki/v3/pkg/compression" "github.com/grafana/loki/v3/pkg/ingester" @@ -25,6 +24,8 @@ import ( "github.com/grafana/loki/v3/pkg/storage/stores" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/flagext" + + "github.com/grafana/loki/pkg/push" ) const ( @@ -309,7 +310,7 @@ func (i *BlockBuilder) reportFlushedChunkStatistics( type AppendInput struct { tenant string - // both labels & labelsStr are populated to prevent duplicating conversion work in mulitple places + // both labels & labelsStr are populated to prevent duplicating conversion work in multiple places labels labels.Labels labelsStr string entries []push.Entry diff --git a/pkg/blockbuilder/writer.go b/pkg/blockbuilder/writer.go index c21a99685eb49..edf234f63be4d 100644 --- a/pkg/blockbuilder/writer.go +++ b/pkg/blockbuilder/writer.go @@ -5,8 +5,9 @@ import ( "github.com/go-kit/log" - "github.com/grafana/loki/pkg/push" "github.com/grafana/loki/v3/pkg/kafka" + + "github.com/grafana/loki/pkg/push" ) type kafkaConsumer struct { diff --git a/pkg/kafka/partition/committer_test.go b/pkg/kafka/partition/committer_test.go index 1739986cd66c8..55454c1229ba1 100644 --- a/pkg/kafka/partition/committer_test.go +++ b/pkg/kafka/partition/committer_test.go @@ -47,9 +47,9 @@ func TestPartitionCommitter(t *testing.T) { require.NoError(t, err) // Verify metrics - assert.Equal(t, float64(1), testutil.ToFloat64(committer.commitRequestsTotal)) - assert.Equal(t, float64(0), testutil.ToFloat64(committer.commitFailuresTotal)) - assert.Equal(t, float64(testOffset), testutil.ToFloat64(committer.lastCommittedOffset)) + assert.Equal(t, float64(1), testutil.ToFloat64(committer.metrics.CommitRequestsTotal)) + assert.Equal(t, float64(0), testutil.ToFloat64(committer.metrics.CommitFailuresTotal)) + assert.Equal(t, float64(testOffset), testutil.ToFloat64(committer.metrics.LastCommittedOffset)) // Verify committed offset offsets, err := admClient.FetchOffsets(context.Background(), consumerGroup) @@ -64,9 +64,9 @@ func TestPartitionCommitter(t *testing.T) { require.NoError(t, err) // Verify updated metrics - assert.Equal(t, float64(2), testutil.ToFloat64(committer.commitRequestsTotal)) - assert.Equal(t, float64(0), testutil.ToFloat64(committer.commitFailuresTotal)) - assert.Equal(t, float64(newTestOffset), testutil.ToFloat64(committer.lastCommittedOffset)) + assert.Equal(t, float64(2), testutil.ToFloat64(committer.metrics.CommitRequestsTotal)) + assert.Equal(t, float64(0), testutil.ToFloat64(committer.metrics.CommitFailuresTotal)) + assert.Equal(t, float64(newTestOffset), testutil.ToFloat64(committer.metrics.LastCommittedOffset)) // Verify updated committed offset offsets, err = admClient.FetchOffsets(context.Background(), consumerGroup) diff --git a/pkg/kafka/partition/metrics.go b/pkg/kafka/partition/metrics.go index 7979ea70a40c0..b89c540c22ab7 100644 --- a/pkg/kafka/partition/metrics.go +++ b/pkg/kafka/partition/metrics.go @@ -12,30 +12,30 @@ import ( "github.com/grafana/loki/v3/pkg/kafka/client" ) -type readerMetrics struct { - partition *prometheus.GaugeVec - phase *prometheus.GaugeVec - receiveDelay *prometheus.HistogramVec - recordsPerFetch prometheus.Histogram - fetchesErrors prometheus.Counter - fetchesTotal prometheus.Counter - fetchWaitDuration prometheus.Histogram - consumeLatency prometheus.Histogram - kprom *kprom.Metrics +type ReaderMetrics struct { + Partition *prometheus.GaugeVec + Phase *prometheus.GaugeVec + ReceiveDelay *prometheus.HistogramVec + RecordsPerFetch prometheus.Histogram + FetchesErrors prometheus.Counter + FetchesTotal prometheus.Counter + FetchWaitDuration prometheus.Histogram + ConsumeLatency prometheus.Histogram + Kprom *kprom.Metrics } -// newReaderMetrics initializes and returns a new set of metrics for the PartitionReader. -func newReaderMetrics(r prometheus.Registerer) readerMetrics { - return readerMetrics{ - partition: promauto.With(r).NewGaugeVec(prometheus.GaugeOpts{ +// NewReaderMetrics initializes and returns a new set of metrics for the PartitionReader. +func NewReaderMetrics(r prometheus.Registerer) *ReaderMetrics { + return &ReaderMetrics{ + Partition: promauto.With(r).NewGaugeVec(prometheus.GaugeOpts{ Name: "loki_ingest_storage_reader_partition", Help: "The partition ID assigned to this reader.", }, []string{"id"}), - phase: promauto.With(r).NewGaugeVec(prometheus.GaugeOpts{ + Phase: promauto.With(r).NewGaugeVec(prometheus.GaugeOpts{ Name: "loki_ingest_storage_reader_phase", Help: "The current phase of the consumer.", }, []string{"phase"}), - receiveDelay: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + ReceiveDelay: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Name: "loki_ingest_storage_reader_receive_delay_seconds", Help: "Delay between producing a record and receiving it in the consumer.", NativeHistogramZeroThreshold: math.Pow(2, -10), // Values below this will be considered to be 0. Equals to 0.0009765625, or about 1ms. @@ -44,36 +44,36 @@ func newReaderMetrics(r prometheus.Registerer) readerMetrics { NativeHistogramMinResetDuration: 1 * time.Hour, Buckets: prometheus.ExponentialBuckets(0.125, 2, 18), // Buckets between 125ms and 9h. }, []string{"phase"}), - kprom: client.NewReaderClientMetrics("partition-reader", r), - fetchWaitDuration: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Kprom: client.NewReaderClientMetrics("partition-reader", r), + FetchWaitDuration: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Name: "loki_ingest_storage_reader_records_batch_wait_duration_seconds", Help: "How long a consumer spent waiting for a batch of records from the Kafka client. If fetching is faster than processing, then this will be close to 0.", NativeHistogramBucketFactor: 1.1, }), - recordsPerFetch: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + RecordsPerFetch: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Name: "loki_ingest_storage_reader_records_per_fetch", Help: "The number of records received by the consumer in a single fetch operation.", Buckets: prometheus.ExponentialBuckets(1, 2, 15), }), - fetchesErrors: promauto.With(r).NewCounter(prometheus.CounterOpts{ + FetchesErrors: promauto.With(r).NewCounter(prometheus.CounterOpts{ Name: "loki_ingest_storage_reader_fetch_errors_total", Help: "The number of fetch errors encountered by the consumer.", }), - fetchesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + FetchesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ Name: "loki_ingest_storage_reader_fetches_total", Help: "Total number of Kafka fetches received by the consumer.", }), } } -func (m *readerMetrics) reportStarting(partitionID int32) { - m.partition.WithLabelValues(strconv.Itoa(int(partitionID))).Set(1) - m.phase.WithLabelValues(phaseStarting).Set(1) - m.phase.WithLabelValues(phaseRunning).Set(0) +func (m *ReaderMetrics) reportStarting(partitionID int32) { + m.Partition.WithLabelValues(strconv.Itoa(int(partitionID))).Set(1) + m.Phase.WithLabelValues(PhaseStarting).Set(1) + m.Phase.WithLabelValues(PhaseRunning).Set(0) } -func (m *readerMetrics) reportRunning(partitionID int32) { - m.partition.WithLabelValues(strconv.Itoa(int(partitionID))).Set(1) - m.phase.WithLabelValues(phaseStarting).Set(0) - m.phase.WithLabelValues(phaseRunning).Set(1) +func (m *ReaderMetrics) reportRunning(partitionID int32) { + m.Partition.WithLabelValues(strconv.Itoa(int(partitionID))).Set(1) + m.Phase.WithLabelValues(PhaseStarting).Set(0) + m.Phase.WithLabelValues(PhaseRunning).Set(1) } diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index 381b95345ffb1..e2ac6b350d1cd 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -28,8 +28,8 @@ const ( kafkaStartOffset = -2 kafkaEndOffset = -1 - phaseStarting = "starting" - phaseRunning = "running" + PhaseStarting = "starting" + PhaseRunning = "running" ) // Reader is responsible for reading data from a specific Kafka partition @@ -479,9 +479,9 @@ func (p *Reader) recordFetchesMetrics(fetches kgo.Fetches) { numRecords++ delay := now.Sub(record.Timestamp).Seconds() if p.Service.State() == services.Starting { - p.metrics.receiveDelay.WithLabelValues(phaseStarting).Observe(delay) + p.metrics.ReceiveDelay.WithLabelValues(PhaseStarting).Observe(delay) } else { - p.metrics.receiveDelay.WithLabelValues(phaseRunning).Observe(delay) + p.metrics.ReceiveDelay.WithLabelValues(PhaseRunning).Observe(delay) } }) From ef9f73d6e54ff20befc731df203b5ad4dfe9a856 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 5 Nov 2024 09:05:09 -0800 Subject: [PATCH 22/56] cosmetic linting Signed-off-by: Owen Diehl --- pkg/blockbuilder/partition.go | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index c1beb940bc4fd..b40bcb53461ac 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -228,16 +228,16 @@ func (r *partitionReader) HighestPartitionOffset(ctx context.Context) (int64, er // pollFetches retrieves the next batch of records from Kafka and measures the fetch duration. // NB(owen-d): originally lifted from `pkg/kafka/partition/reader.go:Reader` -func (p *partitionReader) poll( +func (r *partitionReader) poll( ctx context.Context, maxOffset int64, // exclusive ) ([]partition.Record, bool) { defer func(start time.Time) { - p.readerMetrics.FetchWaitDuration.Observe(time.Since(start).Seconds()) + r.readerMetrics.FetchWaitDuration.Observe(time.Since(start).Seconds()) }(time.Now()) - fetches := p.client.PollFetches(ctx) - p.recordFetchesMetrics(fetches) - p.logFetchErrors(fetches) + fetches := r.client.PollFetches(ctx) + r.recordFetchesMetrics(fetches) + r.logFetchErrors(fetches) fetches = partition.FilterOutErrFetches(fetches) if fetches.NumRecords() == 0 { return nil, false @@ -247,8 +247,8 @@ func (p *partitionReader) poll( itr := fetches.RecordIter() for !itr.Done() { rec := itr.Next() - if rec.Partition != p.partitionID { - level.Error(p.logger).Log("msg", "wrong partition record received", "partition", rec.Partition, "expected_partition", p.partitionID) + if rec.Partition != r.partitionID { + level.Error(r.logger).Log("msg", "wrong partition record received", "partition", rec.Partition, "expected_partition", r.partitionID) continue } @@ -270,7 +270,7 @@ func (p *partitionReader) poll( } // logFetchErrors logs any errors encountered during the fetch operation. -func (p *partitionReader) logFetchErrors(fetches kgo.Fetches) { +func (r *partitionReader) logFetchErrors(fetches kgo.Fetches) { mErr := multierror.New() fetches.EachError(func(topic string, partition int32, err error) { if errors.Is(err, context.Canceled) { @@ -284,13 +284,13 @@ func (p *partitionReader) logFetchErrors(fetches kgo.Fetches) { if len(mErr) == 0 { return } - p.readerMetrics.FetchesErrors.Add(float64(len(mErr))) - level.Error(p.logger).Log("msg", "encountered error while fetching", "err", mErr.Err()) + r.readerMetrics.FetchesErrors.Add(float64(len(mErr))) + level.Error(r.logger).Log("msg", "encountered error while fetching", "err", mErr.Err()) } // recordFetchesMetrics updates various metrics related to the fetch operation. // NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` -func (p *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { +func (r *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { var ( now = time.Now() numRecords = 0 @@ -298,19 +298,19 @@ func (p *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { fetches.EachRecord(func(record *kgo.Record) { numRecords++ delay := now.Sub(record.Timestamp).Seconds() - p.readerMetrics.ReceiveDelay.WithLabelValues(partition.PhaseRunning).Observe(delay) + r.readerMetrics.ReceiveDelay.WithLabelValues(partition.PhaseRunning).Observe(delay) }) - p.readerMetrics.FetchesTotal.Add(float64(len(fetches))) - p.readerMetrics.RecordsPerFetch.Observe(float64(numRecords)) + r.readerMetrics.FetchesTotal.Add(float64(len(fetches))) + r.readerMetrics.RecordsPerFetch.Observe(float64(numRecords)) } func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan<- []AppendInput) (int64, error) { r.updateReaderOffset(offsets.Min) var ( - lastOffset int64 = offsets.Min - 1 - boff = backoff.New(ctx, defaultBackoffConfig) + lastOffset = offsets.Min - 1 + boff = backoff.New(ctx, defaultBackoffConfig) err error ) From 51d54967868dd36dd398c189e21e92de8e3c58f5 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 5 Nov 2024 16:38:29 -0800 Subject: [PATCH 23/56] refactoring + CutRemainingChunks Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 159 ++++++++++++++++++++++++++------- 1 file changed, 128 insertions(+), 31 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index c20e3d843bd7c..ea4c1ca315054 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -5,6 +5,7 @@ import ( "context" "flag" "fmt" + "math" "sync" "time" @@ -22,6 +23,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/flagext" @@ -56,8 +58,8 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { } // RegisterFlags registers flags. -func (c *Config) RegisterFlags(flags *flag.FlagSet) { - c.RegisterFlagsWithPrefix("blockbuilder.", flags) +func (cfg *Config) RegisterFlags(flags *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix("blockbuilder.", flags) } func (cfg *Config) Validate() error { @@ -153,6 +155,8 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return true, nil } + indexer := newTsdbCreator(nil) + var lastOffset int64 p := newPipeline(ctx) @@ -210,7 +214,21 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { } }, func() error { - close(flush) + defer close(flush) + + // once we're done appending, cut all remaining chunks. + chks, err := i.CutRemainingChunks(ctx) + if err != nil { + return err + } + + for _, chk := range chks { + select { + case <-ctx.Done(): + return ctx.Err() + case flush <- chk: + } + } return nil }, ) @@ -239,6 +257,17 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return } i.reportFlushedChunkStatistics(chk) + + // write flushed chunk to index + approxKB := math.Round(float64(chk.Data.UncompressedSize()) / float64(1<<10)) + meta := index.ChunkMeta{ + Checksum: chk.ChunkRef.Checksum, + MinTime: int64(chk.ChunkRef.From), + MaxTime: int64(chk.ChunkRef.Through), + KB: uint32(approxKB), + Entries: uint32(chk.Data.Entries()), + } + err = indexer.Append(chk.UserID, chk.Metric, chk.ChunkRef.Fingerprint, index.ChunkMetas{meta}) return }, ); err != nil { @@ -254,6 +283,16 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return false, err } + built, err := indexer.Create() + if err != nil { + return false, err + } + + // ship built + fmt.Println(built) + + // TODO: build in mem tsdb; flush + if err = i.jobController.part.Commit(ctx, lastOffset); err != nil { return false, err } @@ -308,6 +347,53 @@ func (i *BlockBuilder) reportFlushedChunkStatistics( i.metrics.flushedChunksStats.Inc(1) } +func (i *BlockBuilder) CutRemainingChunks(ctx context.Context) ([]*chunk.Chunk, error) { + var chunks []*chunk.Chunk + i.instancesMtx.Lock() + defer i.instancesMtx.Unlock() + + for _, inst := range i.instances { + + // wrap in anonymous fn to make lock release more straightforward + if err := func() error { + inst.streams.mtx.Lock() + defer inst.streams.mtx.Unlock() + + for _, stream := range inst.streams.byLabels { + + // wrap in anonymous fn to make lock release more straightforward + if err := func() error { + stream.chunkMtx.Lock() + defer stream.chunkMtx.Unlock() + if stream.chunk != nil { + cut, err := stream.closeChunk() + if err != nil { + return err + } + encoded, err := inst.encodeChunk(ctx, stream, cut) + if err != nil { + return err + } + chunks = append(chunks, encoded) + } + return nil + + }(); err != nil { + return err + } + + } + return nil + + }(); err != nil { + return nil, err + } + + } + + return chunks, nil +} + type AppendInput struct { tenant string // both labels & labelsStr are populated to prevent duplicating conversion work in multiple places @@ -451,19 +537,12 @@ func (i *instance) Push( if len(xs) > 0 { for _, x := range xs { - firstTime, lastTime := util.RoundToMilliseconds(x.Bounds()) - chk := chunk.NewChunk( - i.tenant, stream.fp, stream.ls, - chunkenc.NewFacade(x, stream.blockSize, stream.targetChunkSize), - firstTime, - lastTime, - ) // encodeChunk mutates the chunk so we must pass by reference - if err := i.encodeChunk(ctx, &chk, x); err != nil { + chk, err := i.encodeChunk(ctx, stream, x) + if err != nil { return err } - - closed = append(closed, &chk) + closed = append(closed, chk) } } return err @@ -474,22 +553,31 @@ func (i *instance) Push( } // encodeChunk encodes a chunk.Chunk. -func (i *instance) encodeChunk(ctx context.Context, ch *chunk.Chunk, mc *chunkenc.MemChunk) error { +func (i *instance) encodeChunk(ctx context.Context, stream *stream, mc *chunkenc.MemChunk) (*chunk.Chunk, error) { if err := ctx.Err(); err != nil { - return err + return nil, err } start := time.Now() + + firstTime, lastTime := util.RoundToMilliseconds(mc.Bounds()) + chk := chunk.NewChunk( + i.tenant, stream.fp, stream.ls, + chunkenc.NewFacade(mc, stream.blockSize, stream.targetChunkSize), + firstTime, + lastTime, + ) + chunkBytesSize := mc.BytesSize() + 4*1024 // size + 4kB should be enough room for cortex header - if err := ch.EncodeTo(bytes.NewBuffer(make([]byte, 0, chunkBytesSize)), i.logger); err != nil { + if err := chk.EncodeTo(bytes.NewBuffer(make([]byte, 0, chunkBytesSize)), i.logger); err != nil { if !errors.Is(err, chunk.ErrChunkDecode) { - return fmt.Errorf("chunk encoding: %w", err) + return nil, fmt.Errorf("chunk encoding: %w", err) } - i.metrics.chunkDecodeFailures.WithLabelValues(ch.UserID).Inc() + i.metrics.chunkDecodeFailures.WithLabelValues(chk.UserID).Inc() } i.metrics.chunkEncodeTime.Observe(time.Since(start).Seconds()) - i.metrics.chunksEncoded.WithLabelValues(ch.UserID).Inc() - return nil + i.metrics.chunksEncoded.WithLabelValues(chk.UserID).Inc() + return &chk, nil } type stream struct { @@ -530,18 +618,11 @@ func (s *stream) Push(entries []push.Entry) (closed []*chunkenc.MemChunk, err er // cut the chunk if the new addition overflows target size if !s.chunk.SpaceFor(&entries[i]) { - if err = s.chunk.Close(); err != nil { - return closed, errors.Wrap(err, "closing chunk") + cut, err := s.closeChunk() + if err != nil { + return nil, err } - - s.metrics.samplesPerChunk.Observe(float64(s.chunk.Size())) - s.metrics.blocksPerChunk.Observe(float64(s.chunk.BlockCount())) - s.metrics.chunksCreatedTotal.Inc() - s.metrics.chunkCreatedStats.Inc(1) - - // add a chunk - closed = append(closed, s.chunk) - s.chunk = s.NewChunk() + closed = append(closed, cut) } if _, err = s.chunk.Append(&entries[i]); err != nil { @@ -552,6 +633,22 @@ func (s *stream) Push(entries []push.Entry) (closed []*chunkenc.MemChunk, err er return closed, nil } +func (s *stream) closeChunk() (*chunkenc.MemChunk, error) { + if err := s.chunk.Close(); err != nil { + return nil, errors.Wrap(err, "closing chunk") + } + + s.metrics.samplesPerChunk.Observe(float64(s.chunk.Size())) + s.metrics.blocksPerChunk.Observe(float64(s.chunk.BlockCount())) + s.metrics.chunksCreatedTotal.Inc() + s.metrics.chunkCreatedStats.Inc(1) + + // add a chunk + res := s.chunk + s.chunk = s.NewChunk() + return res, nil +} + func (s *stream) NewChunk() *chunkenc.MemChunk { return chunkenc.NewMemChunk(s.chunkFormat, s.codec, s.headFmt, s.blockSize, s.targetChunkSize) } From 9c961207d642e5a96543c5c6c99fc9151e23dfbc Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 5 Nov 2024 17:19:22 -0800 Subject: [PATCH 24/56] tsdb creator work + minor refactoring to tenant label address Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 4 +- pkg/blockbuilder/tsdb.go | 58 ++++++++++++++++--- .../shipper/indexshipper/tsdb/index/index.go | 4 ++ pkg/storage/wal/index/index.go | 2 +- 4 files changed, 57 insertions(+), 11 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index ea4c1ca315054..192d2e38e55cd 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -155,7 +155,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return true, nil } - indexer := newTsdbCreator(nil) + indexer := newTsdbCreator() var lastOffset int64 p := newPipeline(ctx) @@ -283,7 +283,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return false, err } - built, err := indexer.Create() + built, err := indexer.Create(ctx) if err != nil { return false, err } diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go index aca14add9f5c0..6ad1440269e8c 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/tsdb.go @@ -1,11 +1,15 @@ package blockbuilder import ( + "context" + "io" + "sort" "sync" "github.com/cespare/xxhash" - "github.com/pkg/errors" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) @@ -13,7 +17,6 @@ import ( // TsdbCreator accepts writes and builds TSDBs. type TsdbCreator struct { // Function to build a TSDB from the current state - mkTsdb func(*tenantHeads) ([]byte, error) mtx sync.RWMutex shards int @@ -21,9 +24,8 @@ type TsdbCreator struct { } // new creates a new HeadManager -func newTsdbCreator(mkTsdb func(*tenantHeads) ([]byte, error)) *TsdbCreator { +func newTsdbCreator() *TsdbCreator { m := &TsdbCreator{ - mkTsdb: mkTsdb, shards: 1 << 5, // 32 shards } @@ -47,17 +49,57 @@ func (m *TsdbCreator) Append(userID string, ls labels.Labels, fprint uint64, chk } // Create builds a TSDB from the current state using the provided mkTsdb function -func (m *TsdbCreator) Create() ([]byte, error) { +func (m *TsdbCreator) Create(ctx context.Context) ([]byte, error) { m.mtx.Lock() defer m.mtx.Unlock() - out, err := m.mkTsdb(m.heads) + builder, err := index.NewMemWriterWithVersion(ctx, index.FormatV3) if err != nil { - return nil, errors.Wrap(err, "creating TSDB") + return nil, err } + type seriesWithFP struct { + tenant string + fp uint64 + ls labels.Labels + chks index.ChunkMetas + } + var orderdByFprint []seriesWithFP + + if err := m.heads.forAll(func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error { + orderdByFprint = append( + orderdByFprint, + seriesWithFP{ + tenant: user, + fp: fp, + ls: ls, + chks: chks, + }, + ) + return nil + }); err != nil { + return nil, err + } + + sort.Slice(orderdByFprint, func(i, j int) bool { + return orderdByFprint[i].fp < orderdByFprint[j].fp + }) + + for i, s := range orderdByFprint { + // Must add tenantLabel initially to multitenant tsdbs. + ls := labels.NewBuilder(s.ls).Set(index.TenantLabel, s.tenant).Labels() + if err := builder.AddSeries(storage.SeriesRef(i), ls, model.Fingerprint(s.fp), s.chks...); err != nil { + return nil, err + } + } + + reader, err := builder.Close(true) + if err != nil { + return nil, err + } m.heads = newTenantHeads(m.shards) - return out, nil + + return io.ReadAll(reader) } // tenantHeads manages per-tenant series diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 756d354d1ed61..0766bd058fdf4 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -62,6 +62,10 @@ const ( fingerprintInterval = 1 << 10 millisecondsInHour = int64(time.Hour / time.Millisecond) + + // reserved; used in multitenant indices to signal the tenant. Eventually compacted away when + // single tenant indices are created. + TenantLabel = "__loki_tenant__" ) type indexWriterStage uint8 diff --git a/pkg/storage/wal/index/index.go b/pkg/storage/wal/index/index.go index 8959824c92780..c51e8fc84cc1f 100644 --- a/pkg/storage/wal/index/index.go +++ b/pkg/storage/wal/index/index.go @@ -54,7 +54,7 @@ const ( // checkContextEveryNIterations is used in some tight loops to check if the context is done. checkContextEveryNIterations = 128 - TenantLabel = "__loki_tenant__" + TenantLabel = index.TenantLabel ) // Bitmap used by func isRegexMetaCharacter to check whether a character needs to be escaped. From 312e2734f84a5fb03a89a21536b50104fe48ad69 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 6 Nov 2024 14:49:03 -0800 Subject: [PATCH 25/56] exposing some tsdb shipper fns+types Signed-off-by: Owen Diehl --- .../indexshipper/tsdb/compactor_test.go | 24 +++++++++---------- .../shipper/indexshipper/tsdb/identifier.go | 10 ++++---- .../tsdb/index_shipper_querier.go | 4 ++-- .../shipper/indexshipper/tsdb/manager.go | 22 ++++++++--------- 4 files changed, 30 insertions(+), 30 deletions(-) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go index 23a951deacbd6..be0a343309c5c 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go @@ -134,8 +134,8 @@ func setupMultiTenantIndex(t *testing.T, indexFormat int, userStreams map[string dst := NewPrefixedIdentifier( MultitenantTSDBIdentifier{ - nodeName: "test", - ts: ts, + NodeName: "test", + Ts: ts, }, destDir, "", @@ -239,7 +239,7 @@ func TestCompactor_Compact(t *testing.T) { PeriodicTableConfig: config.PeriodicTableConfig{Period: config.ObjectStorageIndexRequiredPeriod}}, Schema: "v12", } - indexBkts := indexBuckets(now, now, []config.TableRange{periodConfig.GetIndexTableNumberRange(config.DayTime{Time: now})}) + indexBkts := IndexBuckets(now, now, []config.TableRange{periodConfig.GetIndexTableNumberRange(config.DayTime{Time: now})}) tableName := indexBkts[0] lbls1 := mustParseLabels(`{foo="bar", a="b"}`) @@ -497,8 +497,8 @@ func TestCompactor_Compact(t *testing.T) { t.Run(name, func(t *testing.T) { tempDir := t.TempDir() objectStoragePath := filepath.Join(tempDir, objectsStorageDirName) - tablePathInStorage := filepath.Join(objectStoragePath, tableName.prefix) - tableWorkingDirectory := filepath.Join(tempDir, workingDirName, tableName.prefix) + tablePathInStorage := filepath.Join(objectStoragePath, tableName.Prefix) + tableWorkingDirectory := filepath.Join(tempDir, workingDirName, tableName.Prefix) require.NoError(t, util.EnsureDirectory(objectStoragePath)) require.NoError(t, util.EnsureDirectory(tablePathInStorage)) @@ -551,7 +551,7 @@ func TestCompactor_Compact(t *testing.T) { objectClient, err := local.NewFSObjectClient(local.FSConfig{Directory: objectStoragePath}) require.NoError(t, err) - _, commonPrefixes, err := objectClient.List(context.Background(), tableName.prefix, "/") + _, commonPrefixes, err := objectClient.List(context.Background(), tableName.Prefix, "/") require.NoError(t, err) initializedIndexSets := map[string]compactor.IndexSet{} @@ -559,19 +559,19 @@ func TestCompactor_Compact(t *testing.T) { existingUserIndexSets := make(map[string]compactor.IndexSet, len(commonPrefixes)) for _, commonPrefix := range commonPrefixes { userID := path.Base(string(commonPrefix)) - idxSet, err := newMockIndexSet(userID, tableName.prefix, filepath.Join(tableWorkingDirectory, userID), objectClient) + idxSet, err := newMockIndexSet(userID, tableName.Prefix, filepath.Join(tableWorkingDirectory, userID), objectClient) require.NoError(t, err) existingUserIndexSets[userID] = idxSet initializedIndexSets[userID] = idxSet } - commonIndexSet, err := newMockIndexSet("", tableName.prefix, tableWorkingDirectory, objectClient) + commonIndexSet, err := newMockIndexSet("", tableName.Prefix, tableWorkingDirectory, objectClient) require.NoError(t, err) // build TableCompactor and compact the index tCompactor := newTableCompactor(context.Background(), commonIndexSet, existingUserIndexSets, func(userID string) (compactor.IndexSet, error) { - idxSet, err := newMockIndexSet(userID, tableName.prefix, filepath.Join(tableWorkingDirectory, userID), objectClient) + idxSet, err := newMockIndexSet(userID, tableName.Prefix, filepath.Join(tableWorkingDirectory, userID), objectClient) require.NoError(t, err) initializedIndexSetsMtx.Lock() @@ -875,9 +875,9 @@ func setupCompactedIndex(t *testing.T) *testContext { schemaCfg := config.SchemaConfig{ Configs: []config.PeriodConfig{periodConfig}, } - indexBuckets := indexBuckets(now, now, []config.TableRange{periodConfig.GetIndexTableNumberRange(config.DayTime{Time: now})}) + indexBuckets := IndexBuckets(now, now, []config.TableRange{periodConfig.GetIndexTableNumberRange(config.DayTime{Time: now})}) tableName := indexBuckets[0] - tableInterval := retention.ExtractIntervalFromTableName(tableName.prefix) + tableInterval := retention.ExtractIntervalFromTableName(tableName.Prefix) // shiftTableStart shift tableInterval.Start by the given amount of milliseconds. // It is used for building chunkmetas relative to start time of the table. shiftTableStart := func(ms int64) int64 { @@ -900,7 +900,7 @@ func setupCompactedIndex(t *testing.T) *testContext { builder.FinalizeChunks() - return newCompactedIndex(context.Background(), tableName.prefix, buildUserID(0), t.TempDir(), periodConfig, builder) + return newCompactedIndex(context.Background(), tableName.Prefix, buildUserID(0), t.TempDir(), periodConfig, builder) } expectedChunkEntries := map[string][]retention.ChunkEntry{ diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go index 149d41bfa9441..eab26fe643d54 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go @@ -161,13 +161,13 @@ func ParseSingleTenantTSDBPath(p string) (id SingleTenantTSDBIdentifier, ok bool } type MultitenantTSDBIdentifier struct { - nodeName string - ts time.Time + NodeName string + Ts time.Time } // Name builds filename with format + `-` + ` func (id MultitenantTSDBIdentifier) Name() string { - return fmt.Sprintf("%d-%s.tsdb", id.ts.Unix(), id.nodeName) + return fmt.Sprintf("%d-%s.tsdb", id.Ts.Unix(), id.NodeName) } func (id MultitenantTSDBIdentifier) Path() string { @@ -200,7 +200,7 @@ func parseMultitenantTSDBNameFromBase(name string) (res MultitenantTSDBIdentifie } return MultitenantTSDBIdentifier{ - ts: time.Unix(int64(ts), 0), - nodeName: strings.Join(xs[1:], "-"), + Ts: time.Unix(int64(ts), 0), + NodeName: strings.Join(xs[1:], "-"), }, true } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go index b0d1824936d59..6ca252770169e 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go @@ -40,9 +40,9 @@ func (i indexIterFunc) For(_ context.Context, _ int, f func(context.Context, Ind func (i *indexShipperQuerier) indices(ctx context.Context, from, through model.Time, user string) (Index, error) { itr := indexIterFunc(func(f func(context.Context, Index) error) error { // Ensure we query both per tenant and multitenant TSDBs - idxBuckets := indexBuckets(from, through, []config.TableRange{i.tableRange}) + idxBuckets := IndexBuckets(from, through, []config.TableRange{i.tableRange}) for _, bkt := range idxBuckets { - if err := i.shipper.ForEachConcurrent(ctx, bkt.prefix, user, func(multitenant bool, idx shipperindex.Index) error { + if err := i.shipper.ForEachConcurrent(ctx, bkt.Prefix, user, func(multitenant bool, idx shipperindex.Index) error { impl, ok := idx.(Index) if !ok { return fmt.Errorf("unexpected shipper index type: %T", idx) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go index 96f56d7021f45..dd89911841d36 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go @@ -165,13 +165,13 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, indexShipper indexshippe // chunks may overlap index period bounds, in which case they're written to multiple pds := make(map[string]chunkInfo) for _, chk := range chks { - idxBuckets := indexBuckets(chk.From(), chk.Through(), tableRanges) + idxBuckets := IndexBuckets(chk.From(), chk.Through(), tableRanges) for _, bucket := range idxBuckets { - chkinfo := pds[bucket.prefix] + chkinfo := pds[bucket.Prefix] chkinfo.chunkMetas = append(chkinfo.chunkMetas, chk) - chkinfo.tsdbFormat = bucket.tsdbFormat - pds[bucket.prefix] = chkinfo + chkinfo.tsdbFormat = bucket.TsdbFormat + pds[bucket.Prefix] = chkinfo } } @@ -208,8 +208,8 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, indexShipper indexshippe dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) dst := NewPrefixedIdentifier( MultitenantTSDBIdentifier{ - nodeName: m.nodeName, - ts: heads.start, + NodeName: m.nodeName, + Ts: heads.start, }, dstDir, "", @@ -300,19 +300,19 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier, legacy boo return nil } -type indexInfo struct { - prefix string - tsdbFormat int +type IndexInfo struct { + Prefix string + TsdbFormat int } -func indexBuckets(from, through model.Time, tableRanges config.TableRanges) (res []indexInfo) { +func IndexBuckets(from, through model.Time, tableRanges config.TableRanges) (res []IndexInfo) { start := from.Time().UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) end := through.Time().UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) for cur := start; cur <= end; cur++ { cfg := tableRanges.ConfigForTableNumber(cur) if cfg != nil { tsdbFormat, _ := cfg.TSDBFormat() // Ignoring error, as any valid period config should return valid format. - res = append(res, indexInfo{prefix: cfg.IndexTables.Prefix + strconv.Itoa(int(cur)), tsdbFormat: tsdbFormat}) + res = append(res, IndexInfo{Prefix: cfg.IndexTables.Prefix + strconv.Itoa(int(cur)), TsdbFormat: tsdbFormat}) } } if len(res) == 0 { From 96880bed38f1f1b5f29eb32dcd13d9ada3b5375f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 6 Nov 2024 14:50:53 -0800 Subject: [PATCH 26/56] plumbing + tsdb uploader + building multiple tsdbs per cycle Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 29 ++++-- pkg/blockbuilder/tsdb.go | 171 +++++++++++++++++++++++++-------- pkg/loki/modules.go | 1 - 3 files changed, 154 insertions(+), 47 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 192d2e38e55cd..a40cc8367dc17 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -21,11 +21,13 @@ import ( "github.com/grafana/loki/v3/pkg/compression" "github.com/grafana/loki/v3/pkg/ingester" "github.com/grafana/loki/v3/pkg/storage/chunk" + "github.com/grafana/loki/v3/pkg/storage/chunk/client" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/flagext" + util_log "github.com/grafana/loki/v3/pkg/util/log" "github.com/grafana/loki/pkg/push" ) @@ -96,7 +98,6 @@ type BlockBuilder struct { store stores.ChunkWriter - tsdbCreator *TsdbCreator jobController *PartitionJobController } @@ -106,7 +107,6 @@ func NewBlockBuilder( store stores.ChunkWriter, logger log.Logger, reg prometheus.Registerer, - tsdbCreator *TsdbCreator, jobController *PartitionJobController, ) (*BlockBuilder, error) { @@ -117,7 +117,6 @@ func NewBlockBuilder( logger: logger, instances: make(map[string]*instance), store: store, - tsdbCreator: tsdbCreator, jobController: jobController, } @@ -283,15 +282,31 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return false, err } - built, err := indexer.Create(ctx) + var ( + // TODO(owen-d) + nodeName string // from lifecycler id + tabelRanges []config.TableRange // from periodconfigs + // TODO(owen-d): build uploaders based on table ranges, which can use different + // object clients + c client.ObjectClient + ) + + built, err := indexer.create(ctx, nodeName, tabelRanges) if err != nil { return false, err } - // ship built - fmt.Println(built) + for _, db := range built { + u := newUploader(c) + if err := u.Put(ctx, db); err != nil { + level.Error(util_log.Logger).Log( + "msg", "failed to upload tsdb", + "path", db.id.Path(), + ) - // TODO: build in mem tsdb; flush + return false, err + } + } if err = i.jobController.part.Commit(ctx, lastOffset); err != nil { return false, err diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go index 6ad1440269e8c..f822be199eac9 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/tsdb.go @@ -1,17 +1,24 @@ package blockbuilder import ( + "bytes" "context" + "fmt" "io" - "sort" "sync" + "time" "github.com/cespare/xxhash" + "github.com/go-kit/log/level" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/storage" + "github.com/grafana/loki/v3/pkg/compression" + "github.com/grafana/loki/v3/pkg/storage/chunk/client" + "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" + util_log "github.com/grafana/loki/v3/pkg/util/log" ) // TsdbCreator accepts writes and builds TSDBs. @@ -48,58 +55,112 @@ func (m *TsdbCreator) Append(userID string, ls labels.Labels, fprint uint64, chk return nil } +type chunkInfo struct { + chunkMetas index.ChunkMetas + tsdbFormat int +} + +type tsdbWithId struct { + data []byte + id tsdb.Identifier +} + // Create builds a TSDB from the current state using the provided mkTsdb function -func (m *TsdbCreator) Create(ctx context.Context) ([]byte, error) { +func (m *TsdbCreator) create(ctx context.Context, nodeName string, tableRanges []config.TableRange) ([]tsdbWithId, error) { m.mtx.Lock() defer m.mtx.Unlock() - builder, err := index.NewMemWriterWithVersion(ctx, index.FormatV3) - if err != nil { + periods := make(map[string]*tsdb.Builder) + + if err := m.heads.forAll( + func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error { + // chunks may overlap index period bounds, in which case they're written to multiple + pds := make(map[string]chunkInfo) + for _, chk := range chks { + idxBuckets := tsdb.IndexBuckets(chk.From(), chk.Through(), tableRanges) + + for _, bucket := range idxBuckets { + chkinfo := pds[bucket.Prefix] + chkinfo.chunkMetas = append(chkinfo.chunkMetas, chk) + chkinfo.tsdbFormat = bucket.TsdbFormat + pds[bucket.Prefix] = chkinfo + } + } + + // Embed the tenant label into TSDB + lb := labels.NewBuilder(ls) + lb.Set(index.TenantLabel, user) + withTenant := lb.Labels() + + // Add the chunks to all relevant builders + for pd, chkinfo := range pds { + matchingChks := chkinfo.chunkMetas + b, ok := periods[pd] + if !ok { + b = tsdb.NewBuilder(chkinfo.tsdbFormat) + periods[pd] = b + } + + b.AddSeries( + withTenant, + // use the fingerprint without the added tenant label + // so queries route to the chunks which actually exist. + model.Fingerprint(fp), + matchingChks, + ) + } + + return nil + }, + ); err != nil { + level.Error(util_log.Logger).Log("err", err.Error(), "msg", "building TSDB") return nil, err } - type seriesWithFP struct { - tenant string - fp uint64 - ls labels.Labels - chks index.ChunkMetas - } - var orderdByFprint []seriesWithFP - - if err := m.heads.forAll(func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error { - orderdByFprint = append( - orderdByFprint, - seriesWithFP{ - tenant: user, - fp: fp, - ls: ls, - chks: chks, - }, + now := time.Now() + res := make([]tsdbWithId, 0, len(periods)) + + for p, b := range periods { + + level.Debug(util_log.Logger).Log( + "msg", "building tsdb for period", + "pd", p, ) - return nil - }); err != nil { - return nil, err - } - sort.Slice(orderdByFprint, func(i, j int) bool { - return orderdByFprint[i].fp < orderdByFprint[j].fp - }) + // build+move tsdb to multitenant dir + start := time.Now() + dst, data, err := b.BuildInMemory( + ctx, + func(_, _ model.Time, _ uint32) tsdb.Identifier { + return tsdb.NewPrefixedIdentifier( + tsdb.MultitenantTSDBIdentifier{ + NodeName: nodeName, + Ts: now, + }, + p, + "", + ) + }, + ) - for i, s := range orderdByFprint { - // Must add tenantLabel initially to multitenant tsdbs. - ls := labels.NewBuilder(s.ls).Set(index.TenantLabel, s.tenant).Labels() - if err := builder.AddSeries(storage.SeriesRef(i), ls, model.Fingerprint(s.fp), s.chks...); err != nil { + if err != nil { return nil, err } - } - reader, err := builder.Close(true) - if err != nil { - return nil, err + level.Debug(util_log.Logger).Log( + "msg", "finished building tsdb for period", + "pd", p, + "dst", dst.Path(), + "duration", time.Since(start), + ) + res = append(res, tsdbWithId{ + id: dst, + data: data, + }) } - m.heads = newTenantHeads(m.shards) - return io.ReadAll(reader) + m.heads = newTenantHeads(m.shards) + return res, nil } // tenantHeads manages per-tenant series @@ -213,3 +274,35 @@ func (h *Head) forAll(fn func(ls labels.Labels, fp uint64, chks index.ChunkMetas } return nil } + +type uploader struct { + c client.ObjectClient +} + +func newUploader(client client.ObjectClient) *uploader { + return &uploader{c: client} +} + +func (u *uploader) Put(ctx context.Context, db tsdbWithId) error { + reader := bytes.NewReader(db.data) + gzipPool := compression.GetWriterPool(compression.GZIP) + buf := bytes.NewBuffer(make([]byte, 0, 1<<20)) + compressedWriter := gzipPool.GetWriter(buf) + defer gzipPool.PutWriter(compressedWriter) + + _, err := io.Copy(compressedWriter, reader) + if err != nil { + return err + } + + err = compressedWriter.Close() + if err != nil { + return err + } + + return u.c.PutObject(ctx, db.id.Path(), buf) +} + +func buildFileName(indexName string) string { + return fmt.Sprintf("%s.gz", indexName) +} diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index e2f27c0edc2cd..e9b4439147b3a 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1813,7 +1813,6 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { t.Store, logger, prometheus.DefaultRegisterer, - tsdbCreator, blockbuilder.NewPartitionJobController( reader, ), From ee6a62d4f6789bfb806ba90bffb9e8c9843701ee Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 7 Nov 2024 13:18:52 -0800 Subject: [PATCH 27/56] better object store integration for writing indices Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 14 +++++-- pkg/blockbuilder/tsdb.go | 38 ++++++++++++------- pkg/loki/modules.go | 6 +++ .../shipper/indexshipper/tsdb/manager.go | 11 ++++-- 4 files changed, 48 insertions(+), 21 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index a40cc8367dc17..05cdbeb233d18 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -20,8 +20,8 @@ import ( "github.com/grafana/loki/v3/pkg/chunkenc" "github.com/grafana/loki/v3/pkg/compression" "github.com/grafana/loki/v3/pkg/ingester" + "github.com/grafana/loki/v3/pkg/ingester-rf1/objstore" "github.com/grafana/loki/v3/pkg/storage/chunk" - "github.com/grafana/loki/v3/pkg/storage/chunk/client" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" @@ -96,7 +96,8 @@ type BlockBuilder struct { instances map[string]*instance instancesMtx sync.RWMutex - store stores.ChunkWriter + store stores.ChunkWriter + objStore *objstore.Multi jobController *PartitionJobController } @@ -105,6 +106,7 @@ func NewBlockBuilder( cfg Config, periodicConfigs []config.PeriodConfig, store stores.ChunkWriter, + objStore *objstore.Multi, logger log.Logger, reg prometheus.Registerer, jobController *PartitionJobController, @@ -117,6 +119,7 @@ func NewBlockBuilder( logger: logger, instances: make(map[string]*instance), store: store, + objStore: objStore, jobController: jobController, } @@ -288,7 +291,6 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { tabelRanges []config.TableRange // from periodconfigs // TODO(owen-d): build uploaders based on table ranges, which can use different // object clients - c client.ObjectClient ) built, err := indexer.create(ctx, nodeName, tabelRanges) @@ -297,7 +299,11 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { } for _, db := range built { - u := newUploader(c) + client, err := i.objStore.GetStoreFor(db.bucket) + if err != nil { + return false, err + } + u := newUploader(client) if err := u.Put(ctx, db); err != nil { level.Error(util_log.Logger).Log( "msg", "failed to upload tsdb", diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go index f822be199eac9..451238ef09986 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/tsdb.go @@ -60,30 +60,39 @@ type chunkInfo struct { tsdbFormat int } -type tsdbWithId struct { - data []byte - id tsdb.Identifier +type tsdbWithID struct { + bucket model.Time + data []byte + id tsdb.Identifier } // Create builds a TSDB from the current state using the provided mkTsdb function -func (m *TsdbCreator) create(ctx context.Context, nodeName string, tableRanges []config.TableRange) ([]tsdbWithId, error) { +func (m *TsdbCreator) create(ctx context.Context, nodeName string, tableRanges []config.TableRange) ([]tsdbWithID, error) { m.mtx.Lock() defer m.mtx.Unlock() - periods := make(map[string]*tsdb.Builder) + type key struct { + bucket model.Time + prefix string + } + periods := make(map[key]*tsdb.Builder) if err := m.heads.forAll( func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error { // chunks may overlap index period bounds, in which case they're written to multiple - pds := make(map[string]chunkInfo) + pds := make(map[key]chunkInfo) for _, chk := range chks { idxBuckets := tsdb.IndexBuckets(chk.From(), chk.Through(), tableRanges) for _, bucket := range idxBuckets { - chkinfo := pds[bucket.Prefix] + k := key{ + bucket: bucket.BucketStart, + prefix: bucket.Prefix, + } + chkinfo := pds[k] chkinfo.chunkMetas = append(chkinfo.chunkMetas, chk) chkinfo.tsdbFormat = bucket.TsdbFormat - pds[bucket.Prefix] = chkinfo + pds[k] = chkinfo } } @@ -118,7 +127,7 @@ func (m *TsdbCreator) create(ctx context.Context, nodeName string, tableRanges [ } now := time.Now() - res := make([]tsdbWithId, 0, len(periods)) + res := make([]tsdbWithID, 0, len(periods)) for p, b := range periods { @@ -137,7 +146,7 @@ func (m *TsdbCreator) create(ctx context.Context, nodeName string, tableRanges [ NodeName: nodeName, Ts: now, }, - p, + p.prefix, "", ) }, @@ -153,9 +162,10 @@ func (m *TsdbCreator) create(ctx context.Context, nodeName string, tableRanges [ "dst", dst.Path(), "duration", time.Since(start), ) - res = append(res, tsdbWithId{ - id: dst, - data: data, + res = append(res, tsdbWithID{ + bucket: p.bucket, + id: dst, + data: data, }) } @@ -283,7 +293,7 @@ func newUploader(client client.ObjectClient) *uploader { return &uploader{c: client} } -func (u *uploader) Put(ctx context.Context, db tsdbWithId) error { +func (u *uploader) Put(ctx context.Context, db tsdbWithID) error { reader := bytes.NewReader(db.data) gzipPool := compression.GetWriterPool(compression.GZIP) buf := bytes.NewBuffer(make([]byte, 0, 1<<20)) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index e9b4439147b3a..eb307f014aa98 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1807,10 +1807,16 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { return nil, err } + objectStore, err := objstore.New(t.Cfg.SchemaConfig.Configs, t.Cfg.StorageConfig, t.ClientMetrics) + if err != nil { + return nil, err + } + bb, err := blockbuilder.NewBlockBuilder( t.Cfg.BlockBuilder, t.Cfg.SchemaConfig.Configs, t.Store, + objectStore, logger, prometheus.DefaultRegisterer, blockbuilder.NewPartitionJobController( diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go index dd89911841d36..84c250eb7464c 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go @@ -301,8 +301,9 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier, legacy boo } type IndexInfo struct { - Prefix string - TsdbFormat int + BucketStart model.Time + Prefix string + TsdbFormat int } func IndexBuckets(from, through model.Time, tableRanges config.TableRanges) (res []IndexInfo) { @@ -312,7 +313,11 @@ func IndexBuckets(from, through model.Time, tableRanges config.TableRanges) (res cfg := tableRanges.ConfigForTableNumber(cur) if cfg != nil { tsdbFormat, _ := cfg.TSDBFormat() // Ignoring error, as any valid period config should return valid format. - res = append(res, IndexInfo{Prefix: cfg.IndexTables.Prefix + strconv.Itoa(int(cur)), TsdbFormat: tsdbFormat}) + res = append(res, IndexInfo{ + BucketStart: model.TimeFromUnixNano(cur * int64(config.ObjectStorageIndexRequiredPeriod)), + Prefix: cfg.IndexTables.Prefix + strconv.Itoa(int(cur)), + TsdbFormat: tsdbFormat, + }) } } if len(res) == 0 { From e3eefa360d9ca7fb42e909966d963105cc60d7c7 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 7 Nov 2024 14:11:15 -0800 Subject: [PATCH 28/56] some index bucket testing Signed-off-by: Owen Diehl --- .../shipper/indexshipper/tsdb/manager_test.go | 70 +++++++++++++++++++ 1 file changed, 70 insertions(+) create mode 100644 pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go new file mode 100644 index 0000000000000..870a6a9ef8041 --- /dev/null +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go @@ -0,0 +1,70 @@ +package tsdb + +import ( + "testing" + "time" + + "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/v3/pkg/storage/types" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" +) + +func TestIndexBuckets(t *testing.T) { + var ( + day0 = model.Time(0) + day1 = day0.Add(24 * time.Hour) + day2 = day1.Add(24 * time.Hour) + periods = []config.PeriodConfig{ + { + From: config.NewDayTime(day0), + Schema: "v12", + IndexType: "tsdb", + IndexTables: config.IndexPeriodicTableConfig{ + PeriodicTableConfig: config.PeriodicTableConfig{ + Prefix: "index/", + Period: 24 * time.Hour, + }, + }, + }, + { + From: config.NewDayTime(day2), + Schema: "v13", + IndexType: "tsdb", + IndexTables: config.IndexPeriodicTableConfig{ + PeriodicTableConfig: config.PeriodicTableConfig{ + Prefix: "index2/", + Period: 24 * time.Hour, + }, + }, + }, + } + + tableRanges = config.GetIndexStoreTableRanges(types.TSDBType, periods) + ) + tests := []struct { + name string + from model.Time + through model.Time + expectedInfo []IndexInfo + }{ + { + name: "single table range", + from: day0, + through: day2, + expectedInfo: []IndexInfo{ + {BucketStart: day0, TsdbFormat: index.FormatV2, Prefix: "index/0"}, + {BucketStart: day1, TsdbFormat: index.FormatV2, Prefix: "index/1"}, + {BucketStart: day2, TsdbFormat: index.FormatV3, Prefix: "index2/2"}, + }, + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + res := IndexBuckets(tc.from, tc.through, tableRanges) + require.Equal(t, tc.expectedInfo, res) + }) + } +} From 684c50dc3883da9c7cc372de9307f16f596a7279 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 7 Nov 2024 14:21:05 -0800 Subject: [PATCH 29/56] propagates lifecycler id Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 19 ++++++++----------- pkg/blockbuilder/tsdb.go | 17 +++++++++++------ pkg/loki/modules.go | 1 + 3 files changed, 20 insertions(+), 17 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 05cdbeb233d18..b02e285429b88 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -25,6 +25,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/v3/pkg/storage/types" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/flagext" util_log "github.com/grafana/loki/v3/pkg/util/log" @@ -87,6 +88,7 @@ func (cfg *Config) Validate() error { type BlockBuilder struct { services.Service + id string cfg Config periodicConfigs []config.PeriodConfig @@ -103,6 +105,7 @@ type BlockBuilder struct { } func NewBlockBuilder( + id string, cfg Config, periodicConfigs []config.PeriodConfig, store stores.ChunkWriter, @@ -113,6 +116,7 @@ func NewBlockBuilder( ) (*BlockBuilder, error) { i := &BlockBuilder{ + id: id, cfg: cfg, periodicConfigs: periodicConfigs, metrics: NewSlimgesterMetrics(reg), @@ -286,24 +290,17 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { } var ( - // TODO(owen-d) - nodeName string // from lifecycler id - tabelRanges []config.TableRange // from periodconfigs - // TODO(owen-d): build uploaders based on table ranges, which can use different - // object clients + nodeName = i.id + tableRanges = config.GetIndexStoreTableRanges(types.TSDBType, i.periodicConfigs) ) - built, err := indexer.create(ctx, nodeName, tabelRanges) + built, err := indexer.create(ctx, nodeName, tableRanges) if err != nil { return false, err } for _, db := range built { - client, err := i.objStore.GetStoreFor(db.bucket) - if err != nil { - return false, err - } - u := newUploader(client) + u := newUploader(i.objStore) if err := u.Put(ctx, db); err != nil { level.Error(util_log.Logger).Log( "msg", "failed to upload tsdb", diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go index 451238ef09986..7e3fab5d65e37 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/tsdb.go @@ -14,7 +14,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/v3/pkg/compression" - "github.com/grafana/loki/v3/pkg/storage/chunk/client" + "github.com/grafana/loki/v3/pkg/ingester-rf1/objstore" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" @@ -286,21 +286,26 @@ func (h *Head) forAll(fn func(ls labels.Labels, fp uint64, chks index.ChunkMetas } type uploader struct { - c client.ObjectClient + store *objstore.Multi } -func newUploader(client client.ObjectClient) *uploader { - return &uploader{c: client} +func newUploader(store *objstore.Multi) *uploader { + return &uploader{store: store} } func (u *uploader) Put(ctx context.Context, db tsdbWithID) error { + client, err := u.store.GetStoreFor(db.bucket) + if err != nil { + return err + } + reader := bytes.NewReader(db.data) gzipPool := compression.GetWriterPool(compression.GZIP) buf := bytes.NewBuffer(make([]byte, 0, 1<<20)) compressedWriter := gzipPool.GetWriter(buf) defer gzipPool.PutWriter(compressedWriter) - _, err := io.Copy(compressedWriter, reader) + _, err = io.Copy(compressedWriter, reader) if err != nil { return err } @@ -310,7 +315,7 @@ func (u *uploader) Put(ctx context.Context, db tsdbWithID) error { return err } - return u.c.PutObject(ctx, db.id.Path(), buf) + return client.PutObject(ctx, db.id.Path(), buf) } func buildFileName(indexName string) string { diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index eb307f014aa98..01a2c29360604 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1813,6 +1813,7 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { } bb, err := blockbuilder.NewBlockBuilder( + id, t.Cfg.BlockBuilder, t.Cfg.SchemaConfig.Configs, t.Store, From 569f4e8663728e3813100696b85c722cec6219cd Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Nov 2024 12:54:06 -0800 Subject: [PATCH 30/56] blockbuilder flag registration+deps defined Signed-off-by: Owen Diehl --- pkg/loki/loki.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index 7015f35cd04c5..2ac76b63a571d 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -196,6 +196,7 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.Metastore.RegisterFlags(f) c.MetastoreClient.RegisterFlags(f) c.KafkaConfig.RegisterFlags(f) + c.BlockBuilder.RegisterFlags(f) } func (c *Config) registerServerFlagsWithChangedDefaultValues(fs *flag.FlagSet) { @@ -751,6 +752,7 @@ func (t *Loki) setupModuleManager() error { IndexGatewayRing: {Overrides, MemberlistKV}, PartitionRing: {MemberlistKV, Server, Ring}, MemberlistKV: {Server}, + BlockBuilder: {PartitionRing, Store, Server}, Read: {QueryFrontend, Querier}, Write: {Ingester, Distributor, PatternIngester}, From 1ad155facd93fcf444d6f9f3ea6af4e1ab1da775 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Nov 2024 13:43:42 -0800 Subject: [PATCH 31/56] named pipeline stages Signed-off-by: Owen Diehl --- pkg/blockbuilder/pipeline.go | 6 +++++- pkg/blockbuilder/pipeline_test.go | 7 ++++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/pkg/blockbuilder/pipeline.go b/pkg/blockbuilder/pipeline.go index a70b5b2ee2285..5fb02e3b889ab 100644 --- a/pkg/blockbuilder/pipeline.go +++ b/pkg/blockbuilder/pipeline.go @@ -8,6 +8,7 @@ import ( ) type stage struct { + name string parallelism int grp *errgroup.Group ctx context.Context @@ -28,12 +29,14 @@ func newPipeline(ctx context.Context) *pipeline { } func (p *pipeline) AddStageWithCleanup( + name string, parallelism int, fn func(context.Context) error, cleanup func() error, ) { grp, ctx := errgroup.WithContext(p.ctx) p.stages = append(p.stages, stage{ + name: name, parallelism: parallelism, fn: fn, cleanup: cleanup, @@ -43,10 +46,11 @@ func (p *pipeline) AddStageWithCleanup( } func (p *pipeline) AddStage( + name string, parallelism int, fn func(context.Context) error, ) { - p.AddStageWithCleanup(parallelism, fn, nil) + p.AddStageWithCleanup(name, parallelism, fn, nil) } func (p *pipeline) Run() error { diff --git a/pkg/blockbuilder/pipeline_test.go b/pkg/blockbuilder/pipeline_test.go index a36616d2d95e7..82c36189d56a4 100644 --- a/pkg/blockbuilder/pipeline_test.go +++ b/pkg/blockbuilder/pipeline_test.go @@ -3,6 +3,7 @@ package blockbuilder import ( "context" "errors" + "fmt" "testing" "github.com/stretchr/testify/require" @@ -66,11 +67,11 @@ func TestPipeline(t *testing.T) { t.Run(tt.name, func(t *testing.T) { p := newPipeline(context.Background()) - for _, stage := range tt.stages { + for i, stage := range tt.stages { if stage.cleanup != nil { - p.AddStageWithCleanup(stage.parallelism, stage.fn, stage.cleanup) + p.AddStageWithCleanup(fmt.Sprint(i), stage.parallelism, stage.fn, stage.cleanup) } else { - p.AddStage(stage.parallelism, stage.fn) + p.AddStage(fmt.Sprint(i), stage.parallelism, stage.fn) } } From ba75ab71eb742fbb10d6052babf6822c461004b6 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Nov 2024 13:43:59 -0800 Subject: [PATCH 32/56] dummy partition controller Signed-off-by: Owen Diehl --- pkg/blockbuilder/controller.go | 95 ++++++++++++++++++++++++++++++++++ 1 file changed, 95 insertions(+) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index 45798e16614e2..1d72ba7ff7e80 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -2,6 +2,12 @@ package blockbuilder import ( "context" + "fmt" + "time" + + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/pkg/push" ) // [min,max) @@ -93,3 +99,92 @@ func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) return true, job, nil } + +// implement a dummy controller which can be parameterized to +// deterministically simulate partitions +type dummyPartitionController struct { + topic string + partition int32 + committed int64 + highest int64 + numTenants int // number of unique tenants to simulate + streamsPerTenant int // number of streams per tenant + entriesPerOffset int // coefficient for entries per offset +} + +func newDummyPartitionController(topic string, partition int32, highest int64) *dummyPartitionController { + return &dummyPartitionController{ + topic: topic, + partition: partition, + committed: 0, // always starts at zero + highest: highest, + numTenants: 2, // default number of tenants + streamsPerTenant: 1, // default streams per tenant + entriesPerOffset: 1, // default entries per offset coefficient + } +} + +func (d *dummyPartitionController) Topic() string { + return d.topic +} + +func (d *dummyPartitionController) Partition() int32 { + return d.partition +} + +func (d *dummyPartitionController) HighestCommittedOffset(ctx context.Context) (int64, error) { + return d.committed, nil +} + +func (d *dummyPartitionController) HighestPartitionOffset(ctx context.Context) (int64, error) { + return d.highest, nil +} + +func (d *dummyPartitionController) Commit(ctx context.Context, offset int64) error { + d.committed = offset + return nil +} + +func (d *dummyPartitionController) Process(ctx context.Context, offsets Offsets, ch chan<- []AppendInput) (int64, error) { + for i := int(offsets.Min); i < int(offsets.Max); i++ { + batch := d.createBatch(i) + select { + case <-ctx.Done(): + return int64(i - 1), ctx.Err() + case ch <- batch: + fmt.Println("sent", i) + } + } + return offsets.Max - 1, nil +} + +// creates (tenants*streams) inputs +func (d *dummyPartitionController) createBatch(offset int) []AppendInput { + result := make([]AppendInput, 0, d.numTenants*d.streamsPerTenant) + for i := 0; i < d.numTenants; i++ { + tenant := fmt.Sprintf("tenant-%d", i) + for j := 0; j < d.streamsPerTenant; j++ { + lbls := labels.Labels{ + {Name: "stream", Value: fmt.Sprintf("stream-%d", j)}, + } + entries := make([]push.Entry, d.entriesPerOffset) + for k := 0; k < d.entriesPerOffset; k++ { + entries[k] = push.Entry{ + Timestamp: time.Unix(int64(offset), 0), + Line: fmt.Sprintf("tenant=%d stream=%d line=%d", i, j, k), + } + } + result = append(result, AppendInput{ + tenant: tenant, + labels: lbls, + labelsStr: lbls.String(), + entries: entries, + }) + } + } + return result +} + +func (d *dummyPartitionController) Close() error { + return nil +} From bba20f0e2a231383c8b8ea072f4b9c6a02519b08 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Nov 2024 13:48:24 -0800 Subject: [PATCH 33/56] use named stages, reconfig default values, always run one at beginning Signed-off-by: Owen Diehl --- pkg/blockbuilder/controller.go | 2 +- pkg/blockbuilder/slimgester.go | 18 ++++++++++++++++-- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index 1d72ba7ff7e80..3de16c770c17e 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -112,7 +112,7 @@ type dummyPartitionController struct { entriesPerOffset int // coefficient for entries per offset } -func newDummyPartitionController(topic string, partition int32, highest int64) *dummyPartitionController { +func NewDummyPartitionController(topic string, partition int32, highest int64) *dummyPartitionController { return &dummyPartitionController{ topic: topic, partition: partition, diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index b02e285429b88..f1b9358014c02 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -50,8 +50,8 @@ type Config struct { } func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { - f.IntVar(&cfg.ConcurrentFlushes, prefix+"concurrent-flushes", 16, "How many flushes can happen concurrently") - f.IntVar(&cfg.ConcurrentWriters, prefix+"concurrent-writers", 16, "How many workers to process writes, defaults to number of available cpus") + f.IntVar(&cfg.ConcurrentFlushes, prefix+"concurrent-flushes", 1, "How many flushes can happen concurrently") + f.IntVar(&cfg.ConcurrentWriters, prefix+"concurrent-writers", 1, "How many workers to process writes, defaults to number of available cpus") _ = cfg.BlockSize.Set("256KB") f.Var(&cfg.BlockSize, prefix+"chunks-block-size", "The targeted _uncompressed_ size in bytes of a chunk block When this threshold is exceeded the head block will be cut and compressed inside the chunk.") _ = cfg.TargetChunkSize.Set(fmt.Sprint(onePointFiveMB)) @@ -135,6 +135,17 @@ func (i *BlockBuilder) running(ctx context.Context) error { ticker := time.NewTicker(10 * time.Second) defer ticker.Stop() + // run once in beginning + select { + case <-ctx.Done(): + return nil + default: + _, err := i.runOne(ctx) + if err != nil { + return err + } + } + for { select { case <-ctx.Done(): @@ -171,6 +182,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { // When complete, it stores the last processed offset and closes the channel inputCh := make(chan []AppendInput) p.AddStageWithCleanup( + "load records", 1, func(ctx context.Context) error { lastOffset, err = i.jobController.part.Process(ctx, job.Offsets, inputCh) @@ -188,6 +200,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { // ConcurrentWriters workers process inputs in parallel to maximize throughput. flush := make(chan *chunk.Chunk) p.AddStageWithCleanup( + "appender", i.cfg.ConcurrentWriters, func(ctx context.Context) error { @@ -243,6 +256,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { // This stage receives chunks from the chunks channel and flushes them to storage // using ConcurrentFlushes workers for parallel processing p.AddStage( + "flusher", i.cfg.ConcurrentFlushes, func(ctx context.Context) error { for { From 2a218e9ad42fcaa3aef97fe21d6326196b466c06 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Nov 2024 13:53:58 -0800 Subject: [PATCH 34/56] parameterize interval Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 4 +++- pkg/loki/modules.go | 8 +++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index f1b9358014c02..5648bd58dbb86 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -47,6 +47,7 @@ type Config struct { ChunkEncoding string `yaml:"chunk_encoding"` parsedEncoding compression.Codec `yaml:"-"` // placeholder for validated encoding MaxChunkAge time.Duration `yaml:"max_chunk_age"` + Interval time.Duration `yaml:"interval"` } func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { @@ -58,6 +59,7 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { f.Var(&cfg.TargetChunkSize, prefix+"chunk-target-size", "A target _compressed_ size in bytes for chunks. This is a desired size not an exact size, chunks may be slightly bigger or significantly smaller if they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 creates chunks with a fixed 10 blocks, a non zero value will create chunks with a variable number of blocks to meet the target size.") f.StringVar(&cfg.ChunkEncoding, prefix+"chunk-encoding", compression.Snappy.String(), fmt.Sprintf("The algorithm to use for compressing chunk. (%s)", compression.SupportedCodecs())) f.DurationVar(&cfg.MaxChunkAge, prefix+"max-chunk-age", 2*time.Hour, "The maximum duration of a timeseries chunk in memory. If a timeseries runs for longer than this, the current chunk will be flushed to the store and a new chunk created.") + f.DurationVar(&cfg.Interval, prefix+"interval", 10*time.Minute, "The interval at which to run.") } // RegisterFlags registers flags. @@ -132,7 +134,7 @@ func NewBlockBuilder( } func (i *BlockBuilder) running(ctx context.Context) error { - ticker := time.NewTicker(10 * time.Second) + ticker := time.NewTicker(i.cfg.Interval) defer ticker.Stop() // run once in beginning diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 01a2c29360604..ee2022c691ea4 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1790,13 +1790,14 @@ func (t *Loki) initPartitionRing() (services.Service, error) { func (t *Loki) initBlockBuilder() (services.Service, error) { logger := log.With(util_log.Logger, "component", "block_builder") - id := t.Cfg.Ingester.LifecyclerConfig.ID + // id := t.Cfg.Ingester.LifecyclerConfig.ID + id := "local-slimgester-0" // TODO(owen-d): remove ingestPartitionID, err := partitionring.ExtractIngesterPartitionID(id) if err != nil { return nil, fmt.Errorf("calculating block builder partition ID: %w", err) } - reader, err := blockbuilder.NewPartitionReader( + _, err = blockbuilder.NewPartitionReader( t.Cfg.KafkaConfig, ingestPartitionID, id, @@ -1821,7 +1822,8 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { logger, prometheus.DefaultRegisterer, blockbuilder.NewPartitionJobController( - reader, + // reader, + blockbuilder.NewDummyPartitionController("topic", 0, 10), // TODO(owen-d): undo ), ) From 559090252b7e0df63fda85b1a2af59ca7e5d758e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Nov 2024 14:24:04 -0800 Subject: [PATCH 35/56] minor fixes: tenant heads resetting & mtx access Signed-off-by: Owen Diehl --- pkg/blockbuilder/controller.go | 9 +++++---- pkg/blockbuilder/slimgester.go | 2 -- pkg/blockbuilder/tsdb.go | 8 +++++++- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index 3de16c770c17e..4edb19c33fcd7 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -112,13 +112,15 @@ type dummyPartitionController struct { entriesPerOffset int // coefficient for entries per offset } +// used in testing +// nolint:revive func NewDummyPartitionController(topic string, partition int32, highest int64) *dummyPartitionController { return &dummyPartitionController{ topic: topic, partition: partition, committed: 0, // always starts at zero highest: highest, - numTenants: 2, // default number of tenants + numTenants: 1, // default number of tenants streamsPerTenant: 1, // default streams per tenant entriesPerOffset: 1, // default entries per offset coefficient } @@ -152,7 +154,6 @@ func (d *dummyPartitionController) Process(ctx context.Context, offsets Offsets, case <-ctx.Done(): return int64(i - 1), ctx.Err() case ch <- batch: - fmt.Println("sent", i) } } return offsets.Max - 1, nil @@ -170,8 +171,8 @@ func (d *dummyPartitionController) createBatch(offset int) []AppendInput { entries := make([]push.Entry, d.entriesPerOffset) for k := 0; k < d.entriesPerOffset; k++ { entries[k] = push.Entry{ - Timestamp: time.Unix(int64(offset), 0), - Line: fmt.Sprintf("tenant=%d stream=%d line=%d", i, j, k), + Timestamp: time.Now(), + Line: fmt.Sprintf("tenant=%d stream=%d line=%d offset=%d", i, j, k, offset), } } result = append(result, AppendInput{ diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 5648bd58dbb86..22692a84998f4 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -537,8 +537,6 @@ func (m *streamsMap) For( // Return labels associated with given fingerprint. Used by fingerprint mapper. func (m *streamsMap) getLabelsFromFingerprint(fp model.Fingerprint) labels.Labels { - m.mtx.RLock() - defer m.mtx.RUnlock() if s, ok := m.byFp[fp]; ok { return s.ls diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go index 7e3fab5d65e37..8ff175933d48b 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/tsdb.go @@ -35,10 +35,16 @@ func newTsdbCreator() *TsdbCreator { m := &TsdbCreator{ shards: 1 << 5, // 32 shards } + m.reset() return m } +// reset updates heads +func (m *TsdbCreator) reset() { + m.heads = newTenantHeads(m.shards) +} + // Append adds a new series for the given user func (m *TsdbCreator) Append(userID string, ls labels.Labels, fprint uint64, chks index.ChunkMetas) error { m.mtx.RLock() @@ -169,7 +175,7 @@ func (m *TsdbCreator) create(ctx context.Context, nodeName string, tableRanges [ }) } - m.heads = newTenantHeads(m.shards) + m.reset() return res, nil } From f80e060e5999e429d144e2f4af66af47c6658150 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Nov 2024 14:32:59 -0800 Subject: [PATCH 36/56] parameterized chunk creation Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 22692a84998f4..5559e883c6fba 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -446,7 +446,7 @@ func (i *BlockBuilder) Append(ctx context.Context, input AppendInput) ([]*chunk. i.instancesMtx.Lock() inst, ok = i.instances[input.tenant] if !ok { - inst = newInstance(input.tenant, i.metrics, i.periodicConfigs, i.logger) + inst = newInstance(i.cfg, input.tenant, i.metrics, i.periodicConfigs, i.logger) i.instances[input.tenant] = inst } i.instancesMtx.Unlock() @@ -458,6 +458,7 @@ func (i *BlockBuilder) Append(ctx context.Context, input AppendInput) ([]*chunk. // instance is a slimmed down version from the ingester pkg type instance struct { + cfg Config tenant string buf []byte // buffer used to compute fps. mapper *ingester.FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free @@ -469,6 +470,7 @@ type instance struct { } func newInstance( + cfg Config, tenant string, metrics *SlimgesterMetrics, periods []config.PeriodConfig, @@ -476,6 +478,7 @@ func newInstance( ) *instance { streams := newStreamsMap() return &instance{ + cfg: cfg, tenant: tenant, buf: make([]byte, 0, 1024), mapper: ingester.NewFPMapper(streams.getLabelsFromFingerprint), @@ -559,7 +562,7 @@ func (i *instance) Push( input.labelsStr, func() (*stream, error) { fp := i.getHashForLabels(input.labels) - return newStream(fp, input.labels, i.metrics), nil + return newStream(fp, input.labels, i.cfg, i.metrics), nil }, func(stream *stream) error { xs, err := stream.Push(input.entries) @@ -617,7 +620,6 @@ type stream struct { ls labels.Labels chunkFormat byte - headFmt chunkenc.HeadBlockFmt codec compression.Codec blockSize int targetChunkSize int @@ -627,13 +629,17 @@ type stream struct { metrics *SlimgesterMetrics } -func newStream(fp model.Fingerprint, ls labels.Labels, metrics *SlimgesterMetrics) *stream { +func newStream(fp model.Fingerprint, ls labels.Labels, cfg Config, metrics *SlimgesterMetrics) *stream { return &stream{ fp: fp, ls: ls, - chunkFormat: chunkenc.ChunkFormatV3, - metrics: metrics, + chunkFormat: chunkenc.ChunkFormatV3, + codec: cfg.parsedEncoding, + blockSize: cfg.BlockSize.Val(), + targetChunkSize: cfg.TargetChunkSize.Val(), + + metrics: metrics, } } @@ -682,5 +688,11 @@ func (s *stream) closeChunk() (*chunkenc.MemChunk, error) { } func (s *stream) NewChunk() *chunkenc.MemChunk { - return chunkenc.NewMemChunk(s.chunkFormat, s.codec, s.headFmt, s.blockSize, s.targetChunkSize) + return chunkenc.NewMemChunk( + s.chunkFormat, + s.codec, + chunkenc.ChunkHeadFormatFor(s.chunkFormat), + s.blockSize, + s.targetChunkSize, + ) } From 08a5ee4454983684567348da527d98bf1ae3ca66 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Nov 2024 14:59:40 -0800 Subject: [PATCH 37/56] tenant tuning Signed-off-by: Owen Diehl --- pkg/blockbuilder/controller.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index 4edb19c33fcd7..bcc33923bac7c 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -120,8 +120,8 @@ func NewDummyPartitionController(topic string, partition int32, highest int64) * partition: partition, committed: 0, // always starts at zero highest: highest, - numTenants: 1, // default number of tenants - streamsPerTenant: 1, // default streams per tenant + numTenants: 2, // default number of tenants + streamsPerTenant: 2, // default streams per tenant entriesPerOffset: 1, // default entries per offset coefficient } } @@ -134,11 +134,11 @@ func (d *dummyPartitionController) Partition() int32 { return d.partition } -func (d *dummyPartitionController) HighestCommittedOffset(ctx context.Context) (int64, error) { +func (d *dummyPartitionController) HighestCommittedOffset(_ context.Context) (int64, error) { return d.committed, nil } -func (d *dummyPartitionController) HighestPartitionOffset(ctx context.Context) (int64, error) { +func (d *dummyPartitionController) HighestPartitionOffset(_ context.Context) (int64, error) { return d.highest, nil } From 63493fd2d925e9fb45da06d10c72626de41ab207 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Nov 2024 14:03:41 -0800 Subject: [PATCH 38/56] [revert-local] use kafka reader Signed-off-by: Owen Diehl --- pkg/loki/modules.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index ee2022c691ea4..903815075268b 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1790,14 +1790,15 @@ func (t *Loki) initPartitionRing() (services.Service, error) { func (t *Loki) initBlockBuilder() (services.Service, error) { logger := log.With(util_log.Logger, "component", "block_builder") - // id := t.Cfg.Ingester.LifecyclerConfig.ID - id := "local-slimgester-0" // TODO(owen-d): remove + // TODO(owen-d): perhaps refactor to not use the ingester config? + id := t.Cfg.Ingester.LifecyclerConfig.ID + ingestPartitionID, err := partitionring.ExtractIngesterPartitionID(id) if err != nil { return nil, fmt.Errorf("calculating block builder partition ID: %w", err) } - _, err = blockbuilder.NewPartitionReader( + reader, err := blockbuilder.NewPartitionReader( t.Cfg.KafkaConfig, ingestPartitionID, id, @@ -1822,8 +1823,7 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { logger, prometheus.DefaultRegisterer, blockbuilder.NewPartitionJobController( - // reader, - blockbuilder.NewDummyPartitionController("topic", 0, 10), // TODO(owen-d): undo + reader, ), ) From edc24b7d4f62d85afd64b5741ed21081366fd18d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Nov 2024 14:20:38 -0800 Subject: [PATCH 39/56] isolates slimgester state across jobs Signed-off-by: Owen Diehl --- pkg/blockbuilder/slimgester.go | 124 ++++++++++++++++++++++----------- 1 file changed, 83 insertions(+), 41 deletions(-) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 5559e883c6fba..66518ddc5542e 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -97,12 +97,8 @@ type BlockBuilder struct { metrics *SlimgesterMetrics logger log.Logger - instances map[string]*instance - instancesMtx sync.RWMutex - - store stores.ChunkWriter - objStore *objstore.Multi - + store stores.ChunkWriter + objStore *objstore.Multi jobController *PartitionJobController } @@ -123,7 +119,6 @@ func NewBlockBuilder( periodicConfigs: periodicConfigs, metrics: NewSlimgesterMetrics(reg), logger: logger, - instances: make(map[string]*instance), store: store, objStore: objStore, jobController: jobController, @@ -175,6 +170,14 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { } indexer := newTsdbCreator() + appender := newAppender(i.id, + i.cfg, + i.periodicConfigs, + i.store, + i.objStore, + i.logger, + i.metrics, + ) var lastOffset int64 p := newPipeline(ctx) @@ -217,7 +220,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { } for _, input := range inputs { - cut, err := i.Append(ctx, input) + cut, err := appender.Append(ctx, input) if err != nil { level.Error(i.logger).Log("msg", "failed to append records", "err", err) return err @@ -238,7 +241,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { defer close(flush) // once we're done appending, cut all remaining chunks. - chks, err := i.CutRemainingChunks(ctx) + chks, err := appender.CutRemainingChunks(ctx) if err != nil { return err } @@ -278,7 +281,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { i.metrics.chunksFlushFailures.Inc() return } - i.reportFlushedChunkStatistics(chk) + appender.reportFlushedChunkStatistics(chk) // write flushed chunk to index approxKB := math.Round(float64(chk.Data.UncompressedSize()) / float64(1<<10)) @@ -334,59 +337,98 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return false, nil } +type Appender struct { + id string + cfg Config + periodicConfigs []config.PeriodConfig + + metrics *SlimgesterMetrics + logger log.Logger + + instances map[string]*instance + instancesMtx sync.RWMutex + + store stores.ChunkWriter + objStore *objstore.Multi +} + +// Writer is a single use construct for building chunks +// for from a set of records. It's an independent struct to ensure its +// state is not reused across jobs. +func newAppender( + id string, + cfg Config, + periodicConfigs []config.PeriodConfig, + store stores.ChunkWriter, + objStore *objstore.Multi, + logger log.Logger, + metrics *SlimgesterMetrics, +) *Appender { + return &Appender{ + id: id, + cfg: cfg, + periodicConfigs: periodicConfigs, + metrics: metrics, + logger: logger, + instances: make(map[string]*instance), + store: store, + objStore: objStore, + } +} + // reportFlushedChunkStatistics calculate overall statistics of flushed chunks without compromising the flush process. -func (i *BlockBuilder) reportFlushedChunkStatistics( +func (w *Appender) reportFlushedChunkStatistics( ch *chunk.Chunk, ) { byt, err := ch.Encoded() if err != nil { - level.Error(i.logger).Log("msg", "failed to encode flushed wire chunk", "err", err) + level.Error(w.logger).Log("msg", "failed to encode flushed wire chunk", "err", err) return } - sizePerTenant := i.metrics.chunkSizePerTenant.WithLabelValues(ch.UserID) - countPerTenant := i.metrics.chunksPerTenant.WithLabelValues(ch.UserID) + sizePerTenant := w.metrics.chunkSizePerTenant.WithLabelValues(ch.UserID) + countPerTenant := w.metrics.chunksPerTenant.WithLabelValues(ch.UserID) reason := flushReasonFull from, through := ch.From.Time(), ch.Through.Time() - if through.Sub(from) > i.cfg.MaxChunkAge { + if through.Sub(from) > w.cfg.MaxChunkAge { reason = flushReasonMaxAge } - i.metrics.chunksFlushedPerReason.WithLabelValues(reason).Add(1) + w.metrics.chunksFlushedPerReason.WithLabelValues(reason).Add(1) compressedSize := float64(len(byt)) uncompressedSize, ok := chunkenc.UncompressedSize(ch.Data) if ok && compressedSize > 0 { - i.metrics.chunkCompressionRatio.Observe(float64(uncompressedSize) / compressedSize) + w.metrics.chunkCompressionRatio.Observe(float64(uncompressedSize) / compressedSize) } utilization := ch.Data.Utilization() - i.metrics.chunkUtilization.Observe(utilization) + w.metrics.chunkUtilization.Observe(utilization) numEntries := ch.Data.Entries() - i.metrics.chunkEntries.Observe(float64(numEntries)) - i.metrics.chunkSize.Observe(compressedSize) + w.metrics.chunkEntries.Observe(float64(numEntries)) + w.metrics.chunkSize.Observe(compressedSize) sizePerTenant.Add(compressedSize) countPerTenant.Inc() - i.metrics.chunkAge.Observe(time.Since(from).Seconds()) - i.metrics.chunkLifespan.Observe(through.Sub(from).Hours()) + w.metrics.chunkAge.Observe(time.Since(from).Seconds()) + w.metrics.chunkLifespan.Observe(through.Sub(from).Hours()) - i.metrics.flushedChunksBytesStats.Record(compressedSize) - i.metrics.flushedChunksLinesStats.Record(float64(numEntries)) - i.metrics.flushedChunksUtilizationStats.Record(utilization) - i.metrics.flushedChunksAgeStats.Record(time.Since(from).Seconds()) - i.metrics.flushedChunksLifespanStats.Record(through.Sub(from).Seconds()) - i.metrics.flushedChunksStats.Inc(1) + w.metrics.flushedChunksBytesStats.Record(compressedSize) + w.metrics.flushedChunksLinesStats.Record(float64(numEntries)) + w.metrics.flushedChunksUtilizationStats.Record(utilization) + w.metrics.flushedChunksAgeStats.Record(time.Since(from).Seconds()) + w.metrics.flushedChunksLifespanStats.Record(through.Sub(from).Seconds()) + w.metrics.flushedChunksStats.Inc(1) } -func (i *BlockBuilder) CutRemainingChunks(ctx context.Context) ([]*chunk.Chunk, error) { +func (w *Appender) CutRemainingChunks(ctx context.Context) ([]*chunk.Chunk, error) { var chunks []*chunk.Chunk - i.instancesMtx.Lock() - defer i.instancesMtx.Unlock() + w.instancesMtx.Lock() + defer w.instancesMtx.Unlock() - for _, inst := range i.instances { + for _, inst := range w.instances { // wrap in anonymous fn to make lock release more straightforward if err := func() error { @@ -436,20 +478,20 @@ type AppendInput struct { entries []push.Entry } -func (i *BlockBuilder) Append(ctx context.Context, input AppendInput) ([]*chunk.Chunk, error) { +func (w *Appender) Append(ctx context.Context, input AppendInput) ([]*chunk.Chunk, error) { // use rlock so multiple appends can be called on same instance. // re-check after using regular lock if it didnt exist. - i.instancesMtx.RLock() - inst, ok := i.instances[input.tenant] - i.instancesMtx.RUnlock() + w.instancesMtx.RLock() + inst, ok := w.instances[input.tenant] + w.instancesMtx.RUnlock() if !ok { - i.instancesMtx.Lock() - inst, ok = i.instances[input.tenant] + w.instancesMtx.Lock() + inst, ok = w.instances[input.tenant] if !ok { - inst = newInstance(i.cfg, input.tenant, i.metrics, i.periodicConfigs, i.logger) - i.instances[input.tenant] = inst + inst = newInstance(w.cfg, input.tenant, w.metrics, w.periodicConfigs, w.logger) + w.instances[input.tenant] = inst } - i.instancesMtx.Unlock() + w.instancesMtx.Unlock() } closed, err := inst.Push(ctx, input) From 554d142bc3db8f5d48d86a4912b88841ae316ea9 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Nov 2024 16:33:55 -0800 Subject: [PATCH 40/56] better pipelining Signed-off-by: Owen Diehl --- pkg/blockbuilder/pipeline.go | 50 ++++++++++++++++++++++------------ pkg/blockbuilder/slimgester.go | 4 +-- 2 files changed, 35 insertions(+), 19 deletions(-) diff --git a/pkg/blockbuilder/pipeline.go b/pkg/blockbuilder/pipeline.go index 5fb02e3b889ab..db63b08c7af16 100644 --- a/pkg/blockbuilder/pipeline.go +++ b/pkg/blockbuilder/pipeline.go @@ -13,18 +13,24 @@ type stage struct { grp *errgroup.Group ctx context.Context fn func(context.Context) error - cleanup func() error // optional; will be called once the underlying group returns + cleanup func(context.Context) error // optional; will be called once the underlying group returns } // pipeline is a sequence of n different stages. type pipeline struct { - ctx context.Context // base context + ctx context.Context // base context + // we use a separate errgroup for stage dispatch/collection + // and inherit stage-specific groups from this ctx to + // propagate cancellation + grp *errgroup.Group stages []stage } func newPipeline(ctx context.Context) *pipeline { + stagesGrp, ctx := errgroup.WithContext(ctx) return &pipeline{ ctx: ctx, + grp: stagesGrp, } } @@ -32,7 +38,7 @@ func (p *pipeline) AddStageWithCleanup( name string, parallelism int, fn func(context.Context) error, - cleanup func() error, + cleanup func(context.Context) error, ) { grp, ctx := errgroup.WithContext(p.ctx) p.stages = append(p.stages, stage{ @@ -54,26 +60,36 @@ func (p *pipeline) AddStage( } func (p *pipeline) Run() error { - var errs multierror.MultiError - // begin all stages - for _, s := range p.stages { - for i := 0; i < s.parallelism; i++ { + for i := range p.stages { + // we're using this in subsequent async closures; + // assign it directly in-loop + s := p.stages[i] + + // spin up n workers for each stage using that stage's + // error group. + for j := 0; j < s.parallelism; j++ { s.grp.Go(func() error { return s.fn(s.ctx) }) } - } - // finish all stages - for _, s := range p.stages { - if err := s.grp.Wait(); err != nil { - errs.Add(err) - } - if s.cleanup != nil { - errs.Add(s.cleanup()) - } + // Using the pipeline's err group, await the stage finish, + // calling any necessary cleanup fn + // NB: by using the pipeline's errgroup here, we propagate + // failures to downstream stage contexts, so once a single stage + // fails, the others will be notified. + p.grp.Go(func() error { + var errs multierror.MultiError + errs.Add(s.grp.Wait()) + if s.cleanup != nil { + errs.Add(s.cleanup(s.ctx)) + } + + return errs.Err() + }) } - return errs.Err() + // finish all stages + return p.grp.Wait() } diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 66518ddc5542e..68890aaf06eda 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -193,7 +193,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { lastOffset, err = i.jobController.part.Process(ctx, job.Offsets, inputCh) return err }, - func() error { + func(_ context.Context) error { close(inputCh) return nil }, @@ -237,7 +237,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { } } }, - func() error { + func(ctx context.Context) error { defer close(flush) // once we're done appending, cut all remaining chunks. From 63f598addb7ea60c8357207d6242791bfa8c3554 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Nov 2024 17:38:57 -0800 Subject: [PATCH 41/56] cleanup uses pipeline ctx instead of stage ctx which is already cancelled Signed-off-by: Owen Diehl --- pkg/blockbuilder/pipeline.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pkg/blockbuilder/pipeline.go b/pkg/blockbuilder/pipeline.go index db63b08c7af16..494763d8c83f3 100644 --- a/pkg/blockbuilder/pipeline.go +++ b/pkg/blockbuilder/pipeline.go @@ -83,7 +83,11 @@ func (p *pipeline) Run() error { var errs multierror.MultiError errs.Add(s.grp.Wait()) if s.cleanup != nil { - errs.Add(s.cleanup(s.ctx)) + // NB: we use the pipeline's context for the cleanup call b/c + // the stage's context is cancelled once `Wait` returns. + // That's ok. cleanup is always called for a relevant stage + // and just needs to know if _other_ stages failed at this point + errs.Add(s.cleanup(p.ctx)) } return errs.Err() From 6ad5e83bb0535dadabff9d88bedeca355c730a54 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 13 Nov 2024 09:32:54 -0800 Subject: [PATCH 42/56] indexshipper.ModeDisabled for explicit indexing operations Signed-off-by: Owen Diehl --- pkg/storage/stores/shipper/indexshipper/shipper.go | 5 +++++ .../stores/shipper/indexshipper/tsdb/store.go | 13 +++++++++++++ 2 files changed, 18 insertions(+) diff --git a/pkg/storage/stores/shipper/indexshipper/shipper.go b/pkg/storage/stores/shipper/indexshipper/shipper.go index 5b3037c45b086..2917b1fc7974f 100644 --- a/pkg/storage/stores/shipper/indexshipper/shipper.go +++ b/pkg/storage/stores/shipper/indexshipper/shipper.go @@ -33,6 +33,9 @@ const ( ModeReadOnly = Mode("RO") // ModeWriteOnly is to allow only write operations ModeWriteOnly = Mode("WO") + // ModeDisabled is a no-op implementation which does nothing & does not error. + // It's used by the blockbuilder which handles index operations independently. + ModeDisabled = Mode("NO") // FilesystemObjectStoreType holds the periodic config type for the filesystem store FilesystemObjectStoreType = "filesystem" @@ -142,6 +145,8 @@ type indexShipper struct { func NewIndexShipper(prefix string, cfg Config, storageClient client.ObjectClient, limits downloads.Limits, tenantFilter downloads.TenantFilter, open index.OpenIndexFileFunc, tableRangeToHandle config.TableRange, reg prometheus.Registerer, logger log.Logger) (IndexShipper, error) { switch cfg.Mode { + case ModeDisabled: + return Noop{}, nil case ModeReadOnly, ModeWriteOnly, ModeReadWrite: default: return nil, fmt.Errorf("invalid mode: %v", cfg.Mode) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go index 1ef58c32a1e56..8ca8a2489e6d3 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go @@ -85,6 +85,13 @@ func (s *store) init(name, prefix string, indexShipperCfg indexshipper.Config, s var indices []Index opts := DefaultIndexClientOptions() + // early return in case index shipper is disabled. + if indexShipperCfg.Mode == indexshipper.ModeDisabled { + s.indexWriter = noopIndexWriter{} + s.Reader = NewIndexClient(NoopIndex{}, opts, limits) + return nil + } + if indexShipperCfg.Mode == indexshipper.ModeWriteOnly { // We disable bloom filters on write nodes // for the Stats() methods as it's of relatively little @@ -172,3 +179,9 @@ type failingIndexWriter struct{} func (f failingIndexWriter) Append(_ string, _ labels.Labels, _ uint64, _ tsdbindex.ChunkMetas) error { return fmt.Errorf("index writer is not initialized due to tsdb store being initialized in read-only mode") } + +type noopIndexWriter struct{} + +func (f noopIndexWriter) Append(_ string, _ labels.Labels, _ uint64, _ tsdbindex.ChunkMetas) error { + return nil +} From 029f91ac9b0460e82f76814576851a391de214d3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 13 Nov 2024 09:45:15 -0800 Subject: [PATCH 43/56] blockbuilder parameterized backoff Signed-off-by: Owen Diehl --- pkg/blockbuilder/partition.go | 15 ++++++--------- pkg/blockbuilder/slimgester.go | 5 ++++- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index b40bcb53461ac..d2be138a3ccbe 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -26,17 +26,12 @@ const ( kafkaEndOffset = -1 ) -var defaultBackoffConfig = backoff.Config{ - MinBackoff: 100 * time.Millisecond, - MaxBackoff: time.Second, - MaxRetries: 0, // Retry forever (unless context is canceled / deadline exceeded). -} - type partitionReader struct { topic string group string partitionID int32 decoder *kafka.Decoder + backoff backoff.Config readerMetrics *partition.ReaderMetrics writerMetrics *partition.CommitterMetrics @@ -47,6 +42,7 @@ type partitionReader struct { func NewPartitionReader( kafkaCfg kafka.Config, + backoff backoff.Config, partitionID int32, instanceID string, logger log.Logger, @@ -72,6 +68,7 @@ func NewPartitionReader( topic: kafkaCfg.Topic, group: group, partitionID: partitionID, + backoff: backoff, readerMetrics: readerMetrics, writerMetrics: writerMetrics, logger: logger, @@ -209,7 +206,7 @@ func (r *partitionReader) updateReaderOffset(offset int64) { func (r *partitionReader) HighestCommittedOffset(ctx context.Context) (int64, error) { return withBackoff( ctx, - defaultBackoffConfig, + r.backoff, func() (int64, error) { return r.fetchLastCommittedOffset(ctx), nil }, @@ -219,7 +216,7 @@ func (r *partitionReader) HighestCommittedOffset(ctx context.Context) (int64, er func (r *partitionReader) HighestPartitionOffset(ctx context.Context) (int64, error) { return withBackoff( ctx, - defaultBackoffConfig, + r.backoff, func() (int64, error) { return r.fetchPartitionOffset(ctx, kafkaEndOffset) }, @@ -310,7 +307,7 @@ func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan< var ( lastOffset = offsets.Min - 1 - boff = backoff.New(ctx, defaultBackoffConfig) + boff = backoff.New(ctx, r.backoff) err error ) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 68890aaf06eda..82c59285d8568 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -11,6 +11,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/dskit/backoff" "github.com/grafana/dskit/services" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -48,6 +49,7 @@ type Config struct { parsedEncoding compression.Codec `yaml:"-"` // placeholder for validated encoding MaxChunkAge time.Duration `yaml:"max_chunk_age"` Interval time.Duration `yaml:"interval"` + Backoff backoff.Config `yaml:"backoff_config"` } func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { @@ -60,6 +62,7 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { f.StringVar(&cfg.ChunkEncoding, prefix+"chunk-encoding", compression.Snappy.String(), fmt.Sprintf("The algorithm to use for compressing chunk. (%s)", compression.SupportedCodecs())) f.DurationVar(&cfg.MaxChunkAge, prefix+"max-chunk-age", 2*time.Hour, "The maximum duration of a timeseries chunk in memory. If a timeseries runs for longer than this, the current chunk will be flushed to the store and a new chunk created.") f.DurationVar(&cfg.Interval, prefix+"interval", 10*time.Minute, "The interval at which to run.") + cfg.Backoff.RegisterFlagsWithPrefix(prefix+"backoff.", f) } // RegisterFlags registers flags. @@ -274,7 +277,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { } if _, err := withBackoff( ctx, - defaultBackoffConfig, // retry forever + i.cfg.Backoff, // retry forever func() (res struct{}, err error) { err = i.store.PutOne(ctx, chk.From, chk.Through, *chk) if err != nil { From 735ab08ee699b0c37d4ad5d65c43f34284913064 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 13 Nov 2024 09:48:39 -0800 Subject: [PATCH 44/56] hooks up blockbuilder backoff & disables indexshipper when block-builder is the target Signed-off-by: Owen Diehl --- pkg/loki/modules.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 903815075268b..0e87327f56abd 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -892,6 +892,10 @@ func (t *Loki) updateConfigForShipperStore() { t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadOnly t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeReadOnly + case t.Cfg.isTarget(BlockBuilder): + // Blockbuilder handles index creation independently of the shipper. + t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeDisabled + default: // All other targets use the shipper store in RW mode t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadWrite @@ -1800,6 +1804,7 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { reader, err := blockbuilder.NewPartitionReader( t.Cfg.KafkaConfig, + t.Cfg.BlockBuilder.Backoff, ingestPartitionID, id, logger, From b150592377cd404c1b82947c0db14e9a5d642028 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Nov 2024 09:27:58 -0800 Subject: [PATCH 45/56] move multistore into blockbuilder pkg --- pkg/blockbuilder/pipeline_test.go | 2 +- pkg/blockbuilder/slimgester.go | 9 +- pkg/blockbuilder/storage.go | 154 ++++++++++++++++++++++++++++++ pkg/blockbuilder/storage_test.go | 37 +++++++ pkg/blockbuilder/tsdb.go | 5 +- pkg/loki/modules.go | 2 +- 6 files changed, 199 insertions(+), 10 deletions(-) create mode 100644 pkg/blockbuilder/storage.go create mode 100644 pkg/blockbuilder/storage_test.go diff --git a/pkg/blockbuilder/pipeline_test.go b/pkg/blockbuilder/pipeline_test.go index 82c36189d56a4..d9c62fd778ee4 100644 --- a/pkg/blockbuilder/pipeline_test.go +++ b/pkg/blockbuilder/pipeline_test.go @@ -12,7 +12,7 @@ import ( type testStage struct { parallelism int fn func(context.Context) error - cleanup func() error + cleanup func(context.Context) error } func TestPipeline(t *testing.T) { diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 82c59285d8568..bb9690bf2d7cf 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -21,7 +21,6 @@ import ( "github.com/grafana/loki/v3/pkg/chunkenc" "github.com/grafana/loki/v3/pkg/compression" "github.com/grafana/loki/v3/pkg/ingester" - "github.com/grafana/loki/v3/pkg/ingester-rf1/objstore" "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" @@ -101,7 +100,7 @@ type BlockBuilder struct { logger log.Logger store stores.ChunkWriter - objStore *objstore.Multi + objStore *MultiStore jobController *PartitionJobController } @@ -110,7 +109,7 @@ func NewBlockBuilder( cfg Config, periodicConfigs []config.PeriodConfig, store stores.ChunkWriter, - objStore *objstore.Multi, + objStore *MultiStore, logger log.Logger, reg prometheus.Registerer, jobController *PartitionJobController, @@ -352,7 +351,7 @@ type Appender struct { instancesMtx sync.RWMutex store stores.ChunkWriter - objStore *objstore.Multi + objStore *MultiStore } // Writer is a single use construct for building chunks @@ -363,7 +362,7 @@ func newAppender( cfg Config, periodicConfigs []config.PeriodConfig, store stores.ChunkWriter, - objStore *objstore.Multi, + objStore *MultiStore, logger log.Logger, metrics *SlimgesterMetrics, ) *Appender { diff --git a/pkg/blockbuilder/storage.go b/pkg/blockbuilder/storage.go new file mode 100644 index 0000000000000..2815b9b97ad86 --- /dev/null +++ b/pkg/blockbuilder/storage.go @@ -0,0 +1,154 @@ +package blockbuilder + +import ( + "context" + "fmt" + "io" + "sort" + + "github.com/opentracing/opentracing-go" + "github.com/prometheus/common/model" + + "github.com/grafana/loki/v3/pkg/storage" + "github.com/grafana/loki/v3/pkg/storage/chunk/client" + "github.com/grafana/loki/v3/pkg/storage/config" +) + +type MultiStore struct { + stores []*storeEntry + storageConfig storage.Config +} + +type storeEntry struct { + start model.Time + cfg config.PeriodConfig + objectClient client.ObjectClient +} + +var _ client.ObjectClient = (*MultiStore)(nil) + +func NewMultiStore( + periodicConfigs []config.PeriodConfig, + storageConfig storage.Config, + clientMetrics storage.ClientMetrics, +) (*MultiStore, error) { + store := &MultiStore{ + storageConfig: storageConfig, + } + // sort by From time + sort.Slice(periodicConfigs, func(i, j int) bool { + return periodicConfigs[i].From.Time.Before(periodicConfigs[j].From.Time) + }) + for _, periodicConfig := range periodicConfigs { + objectClient, err := storage.NewObjectClient(periodicConfig.ObjectType, "storage-rf1", storageConfig, clientMetrics) + if err != nil { + return nil, fmt.Errorf("creating object client for period %s: %w ", periodicConfig.From, err) + } + store.stores = append(store.stores, &storeEntry{ + start: periodicConfig.From.Time, + cfg: periodicConfig, + objectClient: objectClient, + }) + } + return store, nil +} + +func (m *MultiStore) GetStoreFor(ts model.Time) (client.ObjectClient, error) { + // find the schema with the lowest start _after_ tm + j := sort.Search(len(m.stores), func(j int) bool { + return m.stores[j].start > ts + }) + + // reduce it by 1 because we want a schema with start <= tm + j-- + + if 0 <= j && j < len(m.stores) { + return m.stores[j].objectClient, nil + } + + // should in theory never happen + return nil, fmt.Errorf("no store found for timestamp %s", ts) +} + +func (m *MultiStore) GetAttributes(ctx context.Context, objectKey string) (client.ObjectAttributes, error) { + s, err := m.GetStoreFor(model.Now()) + if err != nil { + return client.ObjectAttributes{}, err + } + return s.GetAttributes(ctx, objectKey) +} + +func (m *MultiStore) ObjectExists(ctx context.Context, objectKey string) (bool, error) { + s, err := m.GetStoreFor(model.Now()) + if err != nil { + return false, err + } + return s.ObjectExists(ctx, objectKey) +} + +func (m *MultiStore) PutObject(ctx context.Context, objectKey string, object io.Reader) error { + s, err := m.GetStoreFor(model.Now()) + if err != nil { + return err + } + return s.PutObject(ctx, objectKey, object) +} + +func (m *MultiStore) GetObject(ctx context.Context, objectKey string) (io.ReadCloser, int64, error) { + s, err := m.GetStoreFor(model.Now()) + if err != nil { + return nil, 0, err + } + return s.GetObject(ctx, objectKey) +} + +func (m *MultiStore) GetObjectRange(ctx context.Context, objectKey string, off, length int64) (io.ReadCloser, error) { + sp, _ := opentracing.StartSpanFromContext(ctx, "GetObjectRange") + if sp != nil { + sp.LogKV("objectKey", objectKey, "off", off, "length", length) + } + defer sp.Finish() + s, err := m.GetStoreFor(model.Now()) + if err != nil { + return nil, err + } + return s.GetObjectRange(ctx, objectKey, off, length) +} + +func (m *MultiStore) List(ctx context.Context, prefix string, delimiter string) ([]client.StorageObject, []client.StorageCommonPrefix, error) { + s, err := m.GetStoreFor(model.Now()) + if err != nil { + return nil, nil, err + } + return s.List(ctx, prefix, delimiter) +} + +func (m *MultiStore) DeleteObject(ctx context.Context, objectKey string) error { + s, err := m.GetStoreFor(model.Now()) + if err != nil { + return err + } + return s.DeleteObject(ctx, objectKey) +} + +func (m *MultiStore) IsObjectNotFoundErr(err error) bool { + s, _ := m.GetStoreFor(model.Now()) + if s == nil { + return false + } + return s.IsObjectNotFoundErr(err) +} + +func (m *MultiStore) IsRetryableErr(err error) bool { + s, _ := m.GetStoreFor(model.Now()) + if s == nil { + return false + } + return s.IsRetryableErr(err) +} + +func (m *MultiStore) Stop() { + for _, s := range m.stores { + s.objectClient.Stop() + } +} diff --git a/pkg/blockbuilder/storage_test.go b/pkg/blockbuilder/storage_test.go new file mode 100644 index 0000000000000..8fc6b237e132d --- /dev/null +++ b/pkg/blockbuilder/storage_test.go @@ -0,0 +1,37 @@ +package blockbuilder + +import ( + "os" + "testing" + + "github.com/prometheus/common/model" + + "github.com/grafana/loki/v3/pkg/storage" + "github.com/grafana/loki/v3/pkg/storage/chunk/client/local" + "github.com/grafana/loki/v3/pkg/storage/config" +) + +var metrics *storage.ClientMetrics + +func NewTestStorage(t testing.TB) (*MultiStore, error) { + if metrics == nil { + m := storage.NewClientMetrics() + metrics = &m + } + dir := t.TempDir() + t.Cleanup(func() { + os.RemoveAll(dir) + metrics.Unregister() + }) + cfg := storage.Config{ + FSConfig: local.FSConfig{ + Directory: dir, + }, + } + return NewMultiStore([]config.PeriodConfig{ + { + From: config.DayTime{Time: model.Now()}, + ObjectType: "filesystem", + }, + }, cfg, *metrics) +} diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go index 8ff175933d48b..3e32f4ed05b1f 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/tsdb.go @@ -14,7 +14,6 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/v3/pkg/compression" - "github.com/grafana/loki/v3/pkg/ingester-rf1/objstore" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" @@ -292,10 +291,10 @@ func (h *Head) forAll(fn func(ls labels.Labels, fp uint64, chks index.ChunkMetas } type uploader struct { - store *objstore.Multi + store *MultiStore } -func newUploader(store *objstore.Multi) *uploader { +func newUploader(store *MultiStore) *uploader { return &uploader{store: store} } diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 0e87327f56abd..04df8d41b11b8 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1814,7 +1814,7 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { return nil, err } - objectStore, err := objstore.New(t.Cfg.SchemaConfig.Configs, t.Cfg.StorageConfig, t.ClientMetrics) + objectStore, err := blockbuilder.NewMultiStore(t.Cfg.SchemaConfig.Configs, t.Cfg.StorageConfig, t.ClientMetrics) if err != nil { return nil, err } From 66bfd92231cd0e452980b200c342a43f2d0c2bd1 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Nov 2024 09:38:16 -0800 Subject: [PATCH 46/56] make format --- pkg/blockbuilder/metrics.go | 5 +++-- pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/pkg/blockbuilder/metrics.go b/pkg/blockbuilder/metrics.go index 5835bed1c303c..31679e34f4466 100644 --- a/pkg/blockbuilder/metrics.go +++ b/pkg/blockbuilder/metrics.go @@ -1,10 +1,11 @@ package blockbuilder import ( - "github.com/grafana/loki/v3/pkg/analytics" - "github.com/grafana/loki/v3/pkg/util/constants" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/util/constants" ) type SlimgesterMetrics struct { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go index 870a6a9ef8041..6816b0aeb2085 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/manager_test.go @@ -4,11 +4,12 @@ import ( "testing" "time" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/types" - "github.com/prometheus/common/model" - "github.com/stretchr/testify/require" ) func TestIndexBuckets(t *testing.T) { From 295a310f4752d68f16b279e42111de8496280302 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Nov 2024 09:43:11 -0800 Subject: [PATCH 47/56] config.md --- docs/sources/shared/configuration.md | 51 ++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index ec2fc098afbaa..6e029e6e7637f 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -137,6 +137,57 @@ Pass the `-config.expand-env` flag at the command line to enable this way of set # itself to a key value store. [ingester: ] +block_builder: + # How many flushes can happen concurrently + # CLI flag: -blockbuilder.concurrent-flushes + [concurrent_flushes: | default = 1] + + # How many workers to process writes, defaults to number of available cpus + # CLI flag: -blockbuilder.concurrent-writers + [concurrent_writers: | default = 1] + + # The targeted _uncompressed_ size in bytes of a chunk block When this + # threshold is exceeded the head block will be cut and compressed inside the + # chunk. + # CLI flag: -blockbuilder.chunks-block-size + [chunk_block_size: | default = 256KB] + + # A target _compressed_ size in bytes for chunks. This is a desired size not + # an exact size, chunks may be slightly bigger or significantly smaller if + # they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 + # creates chunks with a fixed 10 blocks, a non zero value will create chunks + # with a variable number of blocks to meet the target size. + # CLI flag: -blockbuilder.chunk-target-size + [chunk_target_size: | default = 1536KB] + + # The algorithm to use for compressing chunk. (none, gzip, lz4-64k, snappy, + # lz4-256k, lz4-1M, lz4, flate, zstd) + # CLI flag: -blockbuilder.chunk-encoding + [chunk_encoding: | default = "snappy"] + + # The maximum duration of a timeseries chunk in memory. If a timeseries runs + # for longer than this, the current chunk will be flushed to the store and a + # new chunk created. + # CLI flag: -blockbuilder.max-chunk-age + [max_chunk_age: | default = 2h] + + # The interval at which to run. + # CLI flag: -blockbuilder.interval + [interval: | default = 10m] + + backoff_config: + # Minimum delay when backing off. + # CLI flag: -blockbuilder.backoff..backoff-min-period + [min_period: | default = 100ms] + + # Maximum delay when backing off. + # CLI flag: -blockbuilder.backoff..backoff-max-period + [max_period: | default = 10s] + + # Number of times to backoff and retry before failing. + # CLI flag: -blockbuilder.backoff..backoff-retries + [max_retries: | default = 10] + pattern_ingester: # Whether the pattern ingester is enabled. # CLI flag: -pattern-ingester.enabled From bebb944fa6fce13383f69063412f3c7af6f38e28 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Nov 2024 15:58:17 -0800 Subject: [PATCH 48/56] more blockbuilder logging --- pkg/blockbuilder/partition.go | 28 +++++++++++++++++- pkg/blockbuilder/slimgester.go | 53 ++++++++++++++++++++++++++++++---- 2 files changed, 75 insertions(+), 6 deletions(-) diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index d2be138a3ccbe..16ea3b6d805bb 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -52,6 +52,7 @@ func NewPartitionReader( writerMetrics := partition.NewCommitterMetrics(r, partitionID) group := kafkaCfg.GetConsumerGroup(instanceID, partitionID) + logger = log.With(logger, "component", "partition_reader") decoder, err := kafka.NewDecoder() if err != nil { return nil, err @@ -110,6 +111,14 @@ func (r *partitionReader) fetchPartitionOffset(ctx context.Context, position int // Ensure no error occurred. res := resps[0] + + level.Debug(r.logger).Log( + "msg", "fetched partition offset", + "partition", r.partitionID, + "position", position, + "topic", r.topic, + "err", res.Err, + ) if res.Err != nil { return 0, res.Err } @@ -194,7 +203,18 @@ func (r *partitionReader) fetchLastCommittedOffset(ctx context.Context) int64 { return kafkaStartOffset } - return fetchRes.Groups[0].Topics[0].Partitions[0].Offset + position := fetchRes.Groups[0].Topics[0].Partitions[0].Offset + + level.Debug(r.logger).Log( + "msg", "fetched last committed offset", + "partition", r.partitionID, + "position", position, + "topic", r.topic, + "group", r.group, + "err", res.Err, + ) + + return position } func (r *partitionReader) updateReaderOffset(offset int64) { @@ -313,6 +333,12 @@ func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan< for boff.Ongoing() { fetches, done := r.poll(ctx, offsets.Max) + level.Debug(r.logger).Log( + "msg", "polling kafka", + "records", len(fetches), + "done", done, + "max_offset", offsets.Max, + ) if len(fetches) > 0 { lastOffset = fetches[len(fetches)-1].Offset converted := make([]AppendInput, 0, len(fetches)) diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index bb9690bf2d7cf..37e622ad081a4 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -150,7 +150,12 @@ func (i *BlockBuilder) running(ctx context.Context) error { case <-ctx.Done(): return nil case <-ticker.C: - _, err := i.runOne(ctx) + skipped, err := i.runOne(ctx) + level.Info(i.logger).Log( + "msg", "completed block builder run", "skipped", + "skipped", skipped, + "err", err, + ) if err != nil { return err } @@ -171,13 +176,20 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return true, nil } + logger := log.With( + i.logger, + "partition", job.Partition, + "job_min_offset", job.Offsets.Min, + "job_max_offset", job.Offsets.Max, + ) + indexer := newTsdbCreator() appender := newAppender(i.id, i.cfg, i.periodicConfigs, i.store, i.objStore, - i.logger, + logger, i.metrics, ) @@ -195,7 +207,11 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { lastOffset, err = i.jobController.part.Process(ctx, job.Offsets, inputCh) return err }, - func(_ context.Context) error { + func(ctx context.Context) error { + level.Debug(logger).Log( + "msg", "finished loading records", + "ctx_error", ctx.Err(), + ) close(inputCh) return nil }, @@ -224,7 +240,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { for _, input := range inputs { cut, err := appender.Append(ctx, input) if err != nil { - level.Error(i.logger).Log("msg", "failed to append records", "err", err) + level.Error(logger).Log("msg", "failed to append records", "err", err) return err } @@ -239,7 +255,14 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { } } }, - func(ctx context.Context) error { + func(ctx context.Context) (err error) { + defer func() { + level.Debug(logger).Log( + "msg", "finished appender", + "err", err, + "ctx_error", ctx.Err(), + ) + }() defer close(flush) // once we're done appending, cut all remaining chunks. @@ -306,6 +329,10 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { ) err = p.Run() + level.Debug(logger).Log( + "msg", "finished chunk creation", + "err", err, + ) if err != nil { return false, err } @@ -330,12 +357,28 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return false, err } + + level.Debug(logger).Log( + "msg", "uploaded tsdb", + "name", db.id.Name(), + ) } if err = i.jobController.part.Commit(ctx, lastOffset); err != nil { + level.Error(logger).Log( + "msg", "failed to commit offset", + "last_offset", lastOffset, + "err", err, + ) return false, err } + // log success + level.Info(logger).Log( + "msg", "successfully processed and committed batch", + "last_offset", lastOffset, + ) + return false, nil } From 6714603210502cebe2672b8e4a2d76b78887e73a Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Nov 2024 16:27:28 -0800 Subject: [PATCH 49/56] lint --- pkg/blockbuilder/pipeline_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/blockbuilder/pipeline_test.go b/pkg/blockbuilder/pipeline_test.go index d9c62fd778ee4..9ec69d2006ebe 100644 --- a/pkg/blockbuilder/pipeline_test.go +++ b/pkg/blockbuilder/pipeline_test.go @@ -26,7 +26,7 @@ func TestPipeline(t *testing.T) { stages: []testStage{ { parallelism: 1, - fn: func(ctx context.Context) error { + fn: func(_ context.Context) error { return nil }, }, @@ -37,13 +37,13 @@ func TestPipeline(t *testing.T) { stages: []testStage{ { parallelism: 2, - fn: func(ctx context.Context) error { + fn: func(_ context.Context) error { return nil }, }, { parallelism: 1, - fn: func(ctx context.Context) error { + fn: func(_ context.Context) error { return nil }, }, @@ -54,7 +54,7 @@ func TestPipeline(t *testing.T) { stages: []testStage{ { parallelism: 1, - fn: func(ctx context.Context) error { + fn: func(_ context.Context) error { return errors.New("stage error") }, }, From b734482d7ea4e8660c2046439c20eddde69e25b8 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Nov 2024 14:00:39 -0800 Subject: [PATCH 50/56] more instrumentation --- pkg/blockbuilder/partition.go | 24 ++++++++++++++++++------ pkg/blockbuilder/slimgester.go | 2 ++ 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index 16ea3b6d805bb..3fa099b2f406d 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -72,10 +72,16 @@ func NewPartitionReader( backoff: backoff, readerMetrics: readerMetrics, writerMetrics: writerMetrics, - logger: logger, - decoder: decoder, - client: client, - aClient: aClient, + logger: log.With( + logger, + "component", "partitionReader", + "topic", kafkaCfg.Topic, + "partition_id", partitionID, + "group", group, + ), + decoder: decoder, + client: client, + aClient: aClient, }, nil } @@ -386,11 +392,17 @@ func (r *partitionReader) Commit(ctx context.Context, offset int64) (err error) startTime := time.Now() r.writerMetrics.CommitRequestsTotal.Inc() + logger := log.With( + r.logger, + "offset", offset, + "phase", "committer", + ) + defer func() { r.writerMetrics.CommitRequestsLatency.Observe(time.Since(startTime).Seconds()) if err != nil { - level.Error(r.logger).Log("msg", "failed to commit last consumed offset to Kafka", "err", err, "offset", offset) + level.Error(logger).Log("msg", "failed to commit last consumed offset to Kafka", "err", err, "offset", offset) r.writerMetrics.CommitFailuresTotal.Inc() } }() @@ -406,7 +418,7 @@ func (r *partitionReader) Commit(ctx context.Context, offset int64) (err error) } committedOffset, _ := committed.Lookup(r.topic, r.partitionID) - level.Debug(r.logger).Log("msg", "last commit offset successfully committed to Kafka", "offset", committedOffset.At) + level.Debug(logger).Log("msg", "last commit offset successfully committed to Kafka", "committed", committedOffset.At) r.writerMetrics.LastCommittedOffset.Set(float64(committedOffset.At)) return nil } diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index 37e622ad081a4..af1f1e84a02da 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -183,6 +183,8 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { "job_max_offset", job.Offsets.Max, ) + level.Debug(logger).Log("msg", "beginning job") + indexer := newTsdbCreator() appender := newAppender(i.id, i.cfg, From 26acba1872853cccd14408480f2f08003b0a1b31 Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 21 Nov 2024 13:27:45 +0530 Subject: [PATCH 51/56] fixes to load job with valid offsets on the first run --- pkg/blockbuilder/controller.go | 19 +++++++++++++++---- pkg/blockbuilder/partition.go | 27 +++++++++++++++++++-------- pkg/blockbuilder/slimgester.go | 6 +++++- pkg/loki/modules.go | 2 ++ 4 files changed, 41 insertions(+), 13 deletions(-) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index bcc33923bac7c..0b1e7f2235340 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -28,6 +28,8 @@ type PartitionController interface { HighestCommittedOffset(ctx context.Context) (int64, error) // Returns the highest available offset in the partition HighestPartitionOffset(ctx context.Context) (int64, error) + // Returns the earliest available offset in the partition + EarliestPartitionOffset(ctx context.Context) (int64, error) // Commits the offset to the consumer group. Commit(context.Context, int64) error // Process will run load batches at a time and send them to channel, @@ -74,17 +76,26 @@ func NewPartitionJobController( // Returns whether an applicable job exists, the job, and an error func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) { // Read the most recent committed offset - startOffset, err := l.part.HighestCommittedOffset(ctx) + committedOffset, err := l.part.HighestCommittedOffset(ctx) if err != nil { return false, Job{}, err } - highestOffset, err := l.part.HighestPartitionOffset(ctx) + earliestOffset, err := l.part.EarliestPartitionOffset(ctx) if err != nil { return false, Job{}, err } - if highestOffset == startOffset { + startOffset := committedOffset + 1 + if startOffset < earliestOffset { + startOffset = earliestOffset + } + + highestOffset, err := l.part.HighestPartitionOffset(ctx) + if err != nil { + return false, Job{}, err + } + if highestOffset == committedOffset { return false, Job{}, nil } @@ -93,7 +104,7 @@ func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) Partition: l.part.Partition(), Offsets: Offsets{ Min: startOffset, - Max: startOffset + l.stepLen, + Max: min(startOffset+l.stepLen, highestOffset), }, } diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go index 3fa099b2f406d..4918e0907d650 100644 --- a/pkg/blockbuilder/partition.go +++ b/pkg/blockbuilder/partition.go @@ -117,14 +117,6 @@ func (r *partitionReader) fetchPartitionOffset(ctx context.Context, position int // Ensure no error occurred. res := resps[0] - - level.Debug(r.logger).Log( - "msg", "fetched partition offset", - "partition", r.partitionID, - "position", position, - "topic", r.topic, - "err", res.Err, - ) if res.Err != nil { return 0, res.Err } @@ -150,6 +142,15 @@ func (r *partitionReader) fetchPartitionOffset(ctx context.Context, position int return 0, err } + level.Debug(r.logger).Log( + "msg", "fetched partition offset", + "partition", r.partitionID, + "position", position, + "topic", r.topic, + "err", res.Err, + "offset", listRes.Topics[0].Partitions[0].Offset, + ) + return listRes.Topics[0].Partitions[0].Offset, nil } @@ -249,6 +250,16 @@ func (r *partitionReader) HighestPartitionOffset(ctx context.Context) (int64, er ) } +func (r *partitionReader) EarliestPartitionOffset(ctx context.Context) (int64, error) { + return withBackoff( + ctx, + r.backoff, + func() (int64, error) { + return r.fetchPartitionOffset(ctx, kafkaStartOffset) + }, + ) +} + // pollFetches retrieves the next batch of records from Kafka and measures the fetch duration. // NB(owen-d): originally lifted from `pkg/kafka/partition/reader.go:Reader` func (r *partitionReader) poll( diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index af1f1e84a02da..dff643d9ddddd 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -366,6 +366,10 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { ) } + if lastOffset <= 0 { + return false, nil + } + if err = i.jobController.part.Commit(ctx, lastOffset); err != nil { level.Error(logger).Log( "msg", "failed to commit offset", @@ -723,7 +727,7 @@ func newStream(fp model.Fingerprint, ls labels.Labels, cfg Config, metrics *Slim fp: fp, ls: ls, - chunkFormat: chunkenc.ChunkFormatV3, + chunkFormat: chunkenc.ChunkFormatV4, codec: cfg.parsedEncoding, blockSize: cfg.BlockSize.Val(), targetChunkSize: cfg.TargetChunkSize.Val(), diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 66dcf7b3a5a25..3e82399e9ff13 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -880,6 +880,8 @@ func (t *Loki) updateConfigForShipperStore() { case t.Cfg.isTarget(BlockBuilder): // Blockbuilder handles index creation independently of the shipper. + // TODO: introduce Disabled mode for boltdb shipper and set it here. + t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadOnly t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeDisabled default: From 6a951c02597710de451211be1946268b808881aa Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Nov 2024 13:26:10 +0800 Subject: [PATCH 52/56] blockbuilder uses refactored partition lib --- pkg/blockbuilder/controller.go | 114 ++++++++- pkg/blockbuilder/partition.go | 455 --------------------------------- pkg/blockbuilder/slimgester.go | 24 +- pkg/blockbuilder/tsdb.go | 2 +- 4 files changed, 131 insertions(+), 464 deletions(-) delete mode 100644 pkg/blockbuilder/partition.go diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index 0b1e7f2235340..b68ee47210423 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -7,6 +7,10 @@ import ( "github.com/prometheus/prometheus/model/labels" + "github.com/grafana/dskit/backoff" + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/kafka/partition" + "github.com/grafana/loki/pkg/push" ) @@ -60,28 +64,126 @@ type PartitionController interface { // containing log data and "committed" is the consumer group type PartitionJobController struct { stepLen int64 - part PartitionController + part partition.ReaderIfc + backoff backoff.Config + decoder *kafka.Decoder } func NewPartitionJobController( - controller PartitionController, -) *PartitionJobController { + controller partition.ReaderIfc, + backoff backoff.Config, +) (*PartitionJobController, error) { + decoder, err := kafka.NewDecoder() + if err != nil { + return nil, err + } return &PartitionJobController{ stepLen: 1000, // Default step length of 1000 offsets per job part: controller, + backoff: backoff, + decoder: decoder, + }, nil +} + +func (l *PartitionJobController) HighestCommittedOffset(ctx context.Context) (int64, error) { + return withBackoff( + ctx, + l.backoff, + func() (int64, error) { + return l.part.FetchLastCommittedOffset(ctx) + }, + ) +} + +func (l *PartitionJobController) HighestPartitionOffset(ctx context.Context) (int64, error) { + return withBackoff( + ctx, + l.backoff, + func() (int64, error) { + return l.part.FetchPartitionOffset(ctx, partition.KafkaEndOffset) + }, + ) +} + +func (l *PartitionJobController) EarliestPartitionOffset(ctx context.Context) (int64, error) { + return withBackoff( + ctx, + l.backoff, + func() (int64, error) { + return l.part.FetchPartitionOffset(ctx, partition.KafkaStartOffset) + }, + ) +} + +func (l *PartitionJobController) Process(ctx context.Context, offsets Offsets, ch chan<- []AppendInput) (int64, error) { + l.part.SetOffsetForConsumption(offsets.Min) + + var ( + lastOffset = offsets.Min - 1 + boff = backoff.New(ctx, l.backoff) + err error + ) + + for boff.Ongoing() { + var records []partition.Record + records, err = l.part.Poll(ctx) + if err != nil { + boff.Wait() + continue + } + + if len(records) == 0 { + // No more records available + break + } + + // Reset backoff on successful poll + boff.Reset() + + converted := make([]AppendInput, 0, len(records)) + for _, record := range records { + offset := records[len(records)-1].Offset + if offset >= offsets.Max { + break + } + lastOffset = offset + + stream, labels, err := l.decoder.Decode(record.Content) + if err != nil { + return 0, fmt.Errorf("failed to decode record: %w", err) + } + if len(stream.Entries) == 0 { + continue + } + + converted = append(converted, AppendInput{ + tenant: record.TenantID, + labels: labels, + labelsStr: stream.Labels, + entries: stream.Entries, + }) + + select { + case ch <- converted: + case <-ctx.Done(): + return 0, ctx.Err() + } + } } + + return lastOffset, err } // LoadJob(ctx) returns the next job by finding the most recent unconsumed offset in the partition // Returns whether an applicable job exists, the job, and an error func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) { // Read the most recent committed offset - committedOffset, err := l.part.HighestCommittedOffset(ctx) + committedOffset, err := l.HighestCommittedOffset(ctx) if err != nil { return false, Job{}, err } - earliestOffset, err := l.part.EarliestPartitionOffset(ctx) + earliestOffset, err := l.EarliestPartitionOffset(ctx) if err != nil { return false, Job{}, err } @@ -91,7 +193,7 @@ func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) startOffset = earliestOffset } - highestOffset, err := l.part.HighestPartitionOffset(ctx) + highestOffset, err := l.HighestPartitionOffset(ctx) if err != nil { return false, Job{}, err } diff --git a/pkg/blockbuilder/partition.go b/pkg/blockbuilder/partition.go deleted file mode 100644 index 4918e0907d650..0000000000000 --- a/pkg/blockbuilder/partition.go +++ /dev/null @@ -1,455 +0,0 @@ -package blockbuilder - -import ( - "context" - "errors" - "fmt" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/dskit/backoff" - "github.com/grafana/dskit/multierror" - "github.com/prometheus/client_golang/prometheus" - "github.com/twmb/franz-go/pkg/kadm" - "github.com/twmb/franz-go/pkg/kerr" - "github.com/twmb/franz-go/pkg/kgo" - "github.com/twmb/franz-go/pkg/kmsg" - - "github.com/grafana/loki/v3/pkg/kafka" - "github.com/grafana/loki/v3/pkg/kafka/client" - "github.com/grafana/loki/v3/pkg/kafka/partition" -) - -const ( - kafkaStartOffset = -2 - kafkaEndOffset = -1 -) - -type partitionReader struct { - topic string - group string - partitionID int32 - decoder *kafka.Decoder - backoff backoff.Config - - readerMetrics *partition.ReaderMetrics - writerMetrics *partition.CommitterMetrics - logger log.Logger - client *kgo.Client - aClient *kadm.Client -} - -func NewPartitionReader( - kafkaCfg kafka.Config, - backoff backoff.Config, - partitionID int32, - instanceID string, - logger log.Logger, - r prometheus.Registerer, -) (*partitionReader, error) { - readerMetrics := partition.NewReaderMetrics(r) - writerMetrics := partition.NewCommitterMetrics(r, partitionID) - group := kafkaCfg.GetConsumerGroup(instanceID, partitionID) - - logger = log.With(logger, "component", "partition_reader") - decoder, err := kafka.NewDecoder() - if err != nil { - return nil, err - } - - client, err := client.NewReaderClient(kafkaCfg, readerMetrics.Kprom, logger) - if err != nil { - return nil, err - } - - aClient := kadm.NewClient(client) - - return &partitionReader{ - topic: kafkaCfg.Topic, - group: group, - partitionID: partitionID, - backoff: backoff, - readerMetrics: readerMetrics, - writerMetrics: writerMetrics, - logger: log.With( - logger, - "component", "partitionReader", - "topic", kafkaCfg.Topic, - "partition_id", partitionID, - "group", group, - ), - decoder: decoder, - client: client, - aClient: aClient, - }, nil -} - -func (r *partitionReader) Topic() string { return r.topic } -func (r *partitionReader) Partition() int32 { return r.partitionID } - -// Fetches the desired offset in the partition itself, not the consumer group -// NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` -func (r *partitionReader) fetchPartitionOffset(ctx context.Context, position int64) (int64, error) { - // Create a custom request to fetch the latest offset of a specific partition. - // We manually create a request so that we can request the offset for a single partition - // only, which is more performant than requesting the offsets for all partitions. - partitionReq := kmsg.NewListOffsetsRequestTopicPartition() - partitionReq.Partition = r.partitionID - partitionReq.Timestamp = position - - topicReq := kmsg.NewListOffsetsRequestTopic() - topicReq.Topic = r.topic - topicReq.Partitions = []kmsg.ListOffsetsRequestTopicPartition{partitionReq} - - req := kmsg.NewPtrListOffsetsRequest() - req.IsolationLevel = 0 // 0 means READ_UNCOMMITTED. - req.Topics = []kmsg.ListOffsetsRequestTopic{topicReq} - - // Even if we share the same client, other in-flight requests are not canceled once this context is canceled - // (or its deadline is exceeded). We've verified it with a unit test. - resps := r.client.RequestSharded(ctx, req) - - // Since we issued a request for only 1 partition, we expect exactly 1 response. - if expected := 1; len(resps) != 1 { - return 0, fmt.Errorf("unexpected number of responses (expected: %d, got: %d)", expected, len(resps)) - } - - // Ensure no error occurred. - res := resps[0] - if res.Err != nil { - return 0, res.Err - } - - // Parse the response. - listRes, ok := res.Resp.(*kmsg.ListOffsetsResponse) - if !ok { - return 0, errors.New("unexpected response type") - } - if expected, actual := 1, len(listRes.Topics); actual != expected { - return 0, fmt.Errorf("unexpected number of topics in the response (expected: %d, got: %d)", expected, actual) - } - if expected, actual := r.topic, listRes.Topics[0].Topic; expected != actual { - return 0, fmt.Errorf("unexpected topic in the response (expected: %s, got: %s)", expected, actual) - } - if expected, actual := 1, len(listRes.Topics[0].Partitions); actual != expected { - return 0, fmt.Errorf("unexpected number of partitions in the response (expected: %d, got: %d)", expected, actual) - } - if expected, actual := r.partitionID, listRes.Topics[0].Partitions[0].Partition; actual != expected { - return 0, fmt.Errorf("unexpected partition in the response (expected: %d, got: %d)", expected, actual) - } - if err := kerr.ErrorForCode(listRes.Topics[0].Partitions[0].ErrorCode); err != nil { - return 0, err - } - - level.Debug(r.logger).Log( - "msg", "fetched partition offset", - "partition", r.partitionID, - "position", position, - "topic", r.topic, - "err", res.Err, - "offset", listRes.Topics[0].Partitions[0].Offset, - ) - - return listRes.Topics[0].Partitions[0].Offset, nil -} - -// Fetches the highest committee offset in the consumer group -// NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` -// TODO(owen-d): expose errors: the failure case of restarting at -// the beginning of a partition is costly and duplicates data -func (r *partitionReader) fetchLastCommittedOffset(ctx context.Context) int64 { - // We manually create a request so that we can request the offset for a single partition - // only, which is more performant than requesting the offsets for all partitions. - req := kmsg.NewPtrOffsetFetchRequest() - req.Topics = []kmsg.OffsetFetchRequestTopic{{Topic: r.topic, Partitions: []int32{r.partitionID}}} - req.Group = r.group - - resps := r.client.RequestSharded(ctx, req) - - // Since we issued a request for only 1 partition, we expect exactly 1 response. - if expected, actual := 1, len(resps); actual != expected { - level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected number of responses (expected: %d, got: %d)", expected, actual), "expected", expected, "actual", len(resps)) - return kafkaStartOffset - } - // Ensure no error occurred. - res := resps[0] - if res.Err != nil { - level.Error(r.logger).Log("msg", "error fetching group offset for partition", "err", res.Err) - return kafkaStartOffset - } - - // Parse the response. - fetchRes, ok := res.Resp.(*kmsg.OffsetFetchResponse) - if !ok { - level.Error(r.logger).Log("msg", "unexpected response type") - return kafkaStartOffset - } - if expected, actual := 1, len(fetchRes.Groups); actual != expected { - level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected number of groups in the response (expected: %d, got: %d)", expected, actual)) - return kafkaStartOffset - } - if expected, actual := 1, len(fetchRes.Groups[0].Topics); actual != expected { - level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected number of topics in the response (expected: %d, got: %d)", expected, actual)) - return kafkaStartOffset - } - if expected, actual := r.topic, fetchRes.Groups[0].Topics[0].Topic; expected != actual { - level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected topic in the response (expected: %s, got: %s)", expected, actual)) - return kafkaStartOffset - } - if expected, actual := 1, len(fetchRes.Groups[0].Topics[0].Partitions); actual != expected { - level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected number of partitions in the response (expected: %d, got: %d)", expected, actual)) - return kafkaStartOffset - } - if expected, actual := r.partitionID, fetchRes.Groups[0].Topics[0].Partitions[0].Partition; actual != expected { - level.Error(r.logger).Log("msg", fmt.Sprintf("unexpected partition in the response (expected: %d, got: %d)", expected, actual)) - return kafkaStartOffset - } - if err := kerr.ErrorForCode(fetchRes.Groups[0].Topics[0].Partitions[0].ErrorCode); err != nil { - level.Error(r.logger).Log("msg", "unexpected error in the response", "err", err) - return kafkaStartOffset - } - - position := fetchRes.Groups[0].Topics[0].Partitions[0].Offset - - level.Debug(r.logger).Log( - "msg", "fetched last committed offset", - "partition", r.partitionID, - "position", position, - "topic", r.topic, - "group", r.group, - "err", res.Err, - ) - - return position -} - -func (r *partitionReader) updateReaderOffset(offset int64) { - r.client.AddConsumePartitions(map[string]map[int32]kgo.Offset{ - r.topic: {r.partitionID: kgo.NewOffset().At(offset)}, - }) -} - -func (r *partitionReader) HighestCommittedOffset(ctx context.Context) (int64, error) { - return withBackoff( - ctx, - r.backoff, - func() (int64, error) { - return r.fetchLastCommittedOffset(ctx), nil - }, - ) -} - -func (r *partitionReader) HighestPartitionOffset(ctx context.Context) (int64, error) { - return withBackoff( - ctx, - r.backoff, - func() (int64, error) { - return r.fetchPartitionOffset(ctx, kafkaEndOffset) - }, - ) -} - -func (r *partitionReader) EarliestPartitionOffset(ctx context.Context) (int64, error) { - return withBackoff( - ctx, - r.backoff, - func() (int64, error) { - return r.fetchPartitionOffset(ctx, kafkaStartOffset) - }, - ) -} - -// pollFetches retrieves the next batch of records from Kafka and measures the fetch duration. -// NB(owen-d): originally lifted from `pkg/kafka/partition/reader.go:Reader` -func (r *partitionReader) poll( - ctx context.Context, - maxOffset int64, // exclusive -) ([]partition.Record, bool) { - defer func(start time.Time) { - r.readerMetrics.FetchWaitDuration.Observe(time.Since(start).Seconds()) - }(time.Now()) - fetches := r.client.PollFetches(ctx) - r.recordFetchesMetrics(fetches) - r.logFetchErrors(fetches) - fetches = partition.FilterOutErrFetches(fetches) - if fetches.NumRecords() == 0 { - return nil, false - } - records := make([]partition.Record, 0, fetches.NumRecords()) - - itr := fetches.RecordIter() - for !itr.Done() { - rec := itr.Next() - if rec.Partition != r.partitionID { - level.Error(r.logger).Log("msg", "wrong partition record received", "partition", rec.Partition, "expected_partition", r.partitionID) - continue - } - - if rec.Offset >= maxOffset { - return records, true - } - - records = append(records, partition.Record{ - // This context carries the tracing data for this individual record; - // kotel populates this data when it fetches the messages. - Ctx: rec.Context, - TenantID: string(rec.Key), - Content: rec.Value, - Offset: rec.Offset, - }) - } - - return records, false -} - -// logFetchErrors logs any errors encountered during the fetch operation. -func (r *partitionReader) logFetchErrors(fetches kgo.Fetches) { - mErr := multierror.New() - fetches.EachError(func(topic string, partition int32, err error) { - if errors.Is(err, context.Canceled) { - return - } - - // kgo advises to "restart" the kafka client if the returned error is a kerr.Error. - // Recreating the client would cause duplicate metrics registration, so we don't do it for now. - mErr.Add(fmt.Errorf("topic %q, partition %d: %w", topic, partition, err)) - }) - if len(mErr) == 0 { - return - } - r.readerMetrics.FetchesErrors.Add(float64(len(mErr))) - level.Error(r.logger).Log("msg", "encountered error while fetching", "err", mErr.Err()) -} - -// recordFetchesMetrics updates various metrics related to the fetch operation. -// NB(owen-d): lifted from `pkg/kafka/partition/reader.go:Reader` -func (r *partitionReader) recordFetchesMetrics(fetches kgo.Fetches) { - var ( - now = time.Now() - numRecords = 0 - ) - fetches.EachRecord(func(record *kgo.Record) { - numRecords++ - delay := now.Sub(record.Timestamp).Seconds() - r.readerMetrics.ReceiveDelay.WithLabelValues(partition.PhaseRunning).Observe(delay) - }) - - r.readerMetrics.FetchesTotal.Add(float64(len(fetches))) - r.readerMetrics.RecordsPerFetch.Observe(float64(numRecords)) -} - -func (r *partitionReader) Process(ctx context.Context, offsets Offsets, ch chan<- []AppendInput) (int64, error) { - r.updateReaderOffset(offsets.Min) - - var ( - lastOffset = offsets.Min - 1 - boff = backoff.New(ctx, r.backoff) - err error - ) - - for boff.Ongoing() { - fetches, done := r.poll(ctx, offsets.Max) - level.Debug(r.logger).Log( - "msg", "polling kafka", - "records", len(fetches), - "done", done, - "max_offset", offsets.Max, - ) - if len(fetches) > 0 { - lastOffset = fetches[len(fetches)-1].Offset - converted := make([]AppendInput, 0, len(fetches)) - - for _, fetch := range fetches { - stream, labels, err := r.decoder.Decode(fetch.Content) - if err != nil { - return 0, fmt.Errorf("failed to decode record: %w", err) - } - if len(stream.Entries) == 0 { - continue - } - - converted = append(converted, AppendInput{ - tenant: fetch.TenantID, - labels: labels, - labelsStr: stream.Labels, - entries: stream.Entries, - }) - } - - select { - case ch <- converted: - case <-ctx.Done(): - return 0, ctx.Err() - } - } - - if done { - break - } - } - - return lastOffset, err -} - -func (r *partitionReader) Close() error { - r.aClient.Close() - r.client.Close() - return nil -} - -// Commits the offset to the consumer group. -func (r *partitionReader) Commit(ctx context.Context, offset int64) (err error) { - startTime := time.Now() - r.writerMetrics.CommitRequestsTotal.Inc() - - logger := log.With( - r.logger, - "offset", offset, - "phase", "committer", - ) - - defer func() { - r.writerMetrics.CommitRequestsLatency.Observe(time.Since(startTime).Seconds()) - - if err != nil { - level.Error(logger).Log("msg", "failed to commit last consumed offset to Kafka", "err", err, "offset", offset) - r.writerMetrics.CommitFailuresTotal.Inc() - } - }() - - // Commit the last consumed offset. - toCommit := kadm.Offsets{} - toCommit.AddOffset(r.topic, r.partitionID, offset, -1) - committed, err := r.aClient.CommitOffsets(ctx, r.group, toCommit) - if err != nil { - return err - } else if !committed.Ok() { - return committed.Error() - } - - committedOffset, _ := committed.Lookup(r.topic, r.partitionID) - level.Debug(logger).Log("msg", "last commit offset successfully committed to Kafka", "committed", committedOffset.At) - r.writerMetrics.LastCommittedOffset.Set(float64(committedOffset.At)) - return nil -} - -func withBackoff[T any]( - ctx context.Context, - config backoff.Config, - fn func() (T, error), -) (T, error) { - var zero T - - var boff = backoff.New(ctx, config) - for boff.Ongoing() { - res, err := fn() - if err != nil { - boff.Wait() - continue - } - return res, nil - } - - return zero, boff.ErrCause() -} diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/slimgester.go index dff643d9ddddd..705b47444a6f2 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/slimgester.go @@ -206,7 +206,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { "load records", 1, func(ctx context.Context) error { - lastOffset, err = i.jobController.part.Process(ctx, job.Offsets, inputCh) + lastOffset, err = i.jobController.Process(ctx, job.Offsets, inputCh) return err }, func(ctx context.Context) error { @@ -366,7 +366,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { ) } - if lastOffset <= 0 { + if lastOffset <= job.Offsets.Min { return false, nil } @@ -789,3 +789,23 @@ func (s *stream) NewChunk() *chunkenc.MemChunk { s.targetChunkSize, ) } + +func withBackoff[T any]( + ctx context.Context, + config backoff.Config, + fn func() (T, error), +) (T, error) { + var zero T + + var boff = backoff.New(ctx, config) + for boff.Ongoing() { + res, err := fn() + if err != nil { + boff.Wait() + continue + } + return res, nil + } + + return zero, boff.ErrCause() +} diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/tsdb.go index 3e32f4ed05b1f..8af463fcd27da 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/tsdb.go @@ -8,7 +8,7 @@ import ( "sync" "time" - "github.com/cespare/xxhash" + "github.com/cespare/xxhash/v2" "github.com/go-kit/log/level" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" From 6c7233ba9282d8c49c49e4f61cf99dc6c1645941 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Nov 2024 13:40:10 +0800 Subject: [PATCH 53/56] moves partition client creation into reader from readersvc + reintegrates blockbuilder --- pkg/kafka/partition/committer_test.go | 12 +++++----- pkg/kafka/partition/reader.go | 32 +++++++++++++++++++++++++++ pkg/kafka/partition/reader_service.go | 22 ++++++------------ pkg/loki/modules.go | 17 +++++++++----- 4 files changed, 57 insertions(+), 26 deletions(-) diff --git a/pkg/kafka/partition/committer_test.go b/pkg/kafka/partition/committer_test.go index 244665b9cf0d7..dc3d2748c0788 100644 --- a/pkg/kafka/partition/committer_test.go +++ b/pkg/kafka/partition/committer_test.go @@ -55,9 +55,9 @@ func TestPartitionCommitter(t *testing.T) { require.NoError(t, err) // Verify metrics - assert.Equal(t, float64(1), testutil.ToFloat64(committer.metrics.CommitRequestsTotal)) - assert.Equal(t, float64(0), testutil.ToFloat64(committer.metrics.CommitFailuresTotal)) - assert.Equal(t, float64(testOffset), testutil.ToFloat64(committer.metrics.LastCommittedOffset)) + assert.Equal(t, float64(1), testutil.ToFloat64(committer.commitRequestsTotal)) + assert.Equal(t, float64(0), testutil.ToFloat64(committer.commitFailuresTotal)) + assert.Equal(t, float64(testOffset), testutil.ToFloat64(committer.lastCommittedOffset)) // Verify committed offset offsets, err := admClient.FetchOffsets(context.Background(), consumerGroup) @@ -72,9 +72,9 @@ func TestPartitionCommitter(t *testing.T) { require.NoError(t, err) // Verify updated metrics - assert.Equal(t, float64(2), testutil.ToFloat64(committer.metrics.CommitRequestsTotal)) - assert.Equal(t, float64(0), testutil.ToFloat64(committer.metrics.CommitFailuresTotal)) - assert.Equal(t, float64(newTestOffset), testutil.ToFloat64(committer.metrics.LastCommittedOffset)) + assert.Equal(t, float64(2), testutil.ToFloat64(committer.commitRequestsTotal)) + assert.Equal(t, float64(0), testutil.ToFloat64(committer.commitFailuresTotal)) + assert.Equal(t, float64(newTestOffset), testutil.ToFloat64(committer.lastCommittedOffset)) // Verify updated committed offset offsets, err = admClient.FetchOffsets(context.Background(), consumerGroup) diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index 827ed079d56a0..9488dacb640d1 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -9,6 +9,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/grafana/dskit/multierror" + "github.com/grafana/loki/v3/pkg/kafka" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -16,6 +17,8 @@ import ( "github.com/twmb/franz-go/pkg/kerr" "github.com/twmb/franz-go/pkg/kgo" "github.com/twmb/franz-go/pkg/kmsg" + + "github.com/grafana/loki/v3/pkg/kafka/client" ) type SpecialOffset int @@ -97,6 +100,35 @@ type Reader struct { logger log.Logger } +func NewReader( + cfg kafka.Config, + partitionID int32, + instanceID string, + logger log.Logger, + reg prometheus.Registerer, +) (*Reader, error) { + // Create a new Kafka client for this reader + clientMetrics := client.NewReaderClientMetrics("partition-reader", reg) + c, err := client.NewReaderClient( + cfg, + clientMetrics, + log.With(logger, "component", "kafka-client"), + ) + if err != nil { + return nil, fmt.Errorf("creating kafka client: %w", err) + } + + // Create the reader + return newReader( + c, + cfg.Topic, + partitionID, + cfg.GetConsumerGroup(instanceID, partitionID), + logger, + reg, + ), nil +} + // NewReader creates a new Reader instance func newReader( client *kgo.Client, diff --git a/pkg/kafka/partition/reader_service.go b/pkg/kafka/partition/reader_service.go index 76898f1b9ef45..49dcda1c928b3 100644 --- a/pkg/kafka/partition/reader_service.go +++ b/pkg/kafka/partition/reader_service.go @@ -15,7 +15,6 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/grafana/loki/v3/pkg/kafka" - "github.com/grafana/loki/v3/pkg/kafka/client" ) var errWaitTargetLagDeadlineExceeded = errors.New("waiting for target lag deadline exceeded") @@ -81,27 +80,20 @@ func NewReaderService( logger log.Logger, reg prometheus.Registerer, ) (*ReaderService, error) { - // Create a new Kafka client for this reader - clientMetrics := client.NewReaderClientMetrics("partition-reader", reg) - c, err := client.NewReaderClient( - kafkaCfg, - clientMetrics, - log.With(logger, "component", "kafka-client"), - ) - if err != nil { - return nil, fmt.Errorf("creating kafka client: %w", err) - } // Create the reader - reader := newReader( - c, - kafkaCfg.Topic, + reader, err := NewReader( + kafkaCfg, partitionID, - kafkaCfg.GetConsumerGroup(instanceID, partitionID), + instanceID, logger, reg, ) + if err != nil { + return nil, errors.Wrap(err, "creating kafka reader") + } + return newReaderServiceFromIfc( ReaderConfig{ TargetConsumerLagAtStartup: kafkaCfg.TargetConsumerLagAtStartup, diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 3e82399e9ff13..726c8cfe52a69 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -48,6 +48,7 @@ import ( "github.com/grafana/loki/v3/pkg/distributor" "github.com/grafana/loki/v3/pkg/indexgateway" "github.com/grafana/loki/v3/pkg/ingester" + "github.com/grafana/loki/v3/pkg/kafka/partition" "github.com/grafana/loki/v3/pkg/kafka/partitionring" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql" @@ -1790,9 +1791,8 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { return nil, fmt.Errorf("calculating block builder partition ID: %w", err) } - reader, err := blockbuilder.NewPartitionReader( + reader, err := partition.NewReader( t.Cfg.KafkaConfig, - t.Cfg.BlockBuilder.Backoff, ingestPartitionID, id, logger, @@ -1802,6 +1802,15 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { return nil, err } + controller, err := blockbuilder.NewPartitionJobController( + reader, + t.Cfg.BlockBuilder.Backoff, + ) + + if err != nil { + return nil, err + } + objectStore, err := blockbuilder.NewMultiStore(t.Cfg.SchemaConfig.Configs, t.Cfg.StorageConfig, t.ClientMetrics) if err != nil { return nil, err @@ -1815,9 +1824,7 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { objectStore, logger, prometheus.DefaultRegisterer, - blockbuilder.NewPartitionJobController( - reader, - ), + controller, ) if err != nil { From 1df329756f86dce5d802dd58f06d7f08a7f7ca33 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Nov 2024 13:44:42 +0800 Subject: [PATCH 54/56] make format --- pkg/blockbuilder/controller.go | 1 + pkg/kafka/partition/reader.go | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index b68ee47210423..f655761b1629c 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -8,6 +8,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/dskit/backoff" + "github.com/grafana/loki/v3/pkg/kafka" "github.com/grafana/loki/v3/pkg/kafka/partition" diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index 9488dacb640d1..a0d360c4a065f 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -9,7 +9,6 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/grafana/dskit/multierror" - "github.com/grafana/loki/v3/pkg/kafka" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -18,6 +17,8 @@ import ( "github.com/twmb/franz-go/pkg/kgo" "github.com/twmb/franz-go/pkg/kmsg" + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/kafka/client" ) From 7c475e513e54bbd2effdc38a1ea6aaecd5001da6 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Nov 2024 14:38:14 +0800 Subject: [PATCH 55/56] more linting --- pkg/blockbuilder/controller.go | 2 +- pkg/blockbuilder/writer.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/controller.go index f655761b1629c..f252b3d657444 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/controller.go @@ -256,7 +256,7 @@ func (d *dummyPartitionController) HighestPartitionOffset(_ context.Context) (in return d.highest, nil } -func (d *dummyPartitionController) Commit(ctx context.Context, offset int64) error { +func (d *dummyPartitionController) Commit(_ context.Context, offset int64) error { d.committed = offset return nil } diff --git a/pkg/blockbuilder/writer.go b/pkg/blockbuilder/writer.go index edf234f63be4d..e60ab8d9fb9dc 100644 --- a/pkg/blockbuilder/writer.go +++ b/pkg/blockbuilder/writer.go @@ -15,10 +15,10 @@ type kafkaConsumer struct { decoder *kafka.Decoder } -func (c *kafkaConsumer) Write(ctx context.Context, req *push.PushRequest) error { +func (c *kafkaConsumer) Write(_ context.Context, req *push.PushRequest) error { return nil } -func (c *kafkaConsumer) Commit(ctx context.Context) error { +func (c *kafkaConsumer) Commit(_ context.Context) error { return nil } From e8a93c93bd91d39a5d89f2ef76e949c9189d3db6 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Nov 2024 14:45:09 +0800 Subject: [PATCH 56/56] removes unused file --- pkg/blockbuilder/writer.go | 24 ------------------------ 1 file changed, 24 deletions(-) delete mode 100644 pkg/blockbuilder/writer.go diff --git a/pkg/blockbuilder/writer.go b/pkg/blockbuilder/writer.go deleted file mode 100644 index e60ab8d9fb9dc..0000000000000 --- a/pkg/blockbuilder/writer.go +++ /dev/null @@ -1,24 +0,0 @@ -package blockbuilder - -import ( - "context" - - "github.com/go-kit/log" - - "github.com/grafana/loki/v3/pkg/kafka" - - "github.com/grafana/loki/pkg/push" -) - -type kafkaConsumer struct { - logger log.Logger - decoder *kafka.Decoder -} - -func (c *kafkaConsumer) Write(_ context.Context, req *push.PushRequest) error { - return nil -} - -func (c *kafkaConsumer) Commit(_ context.Context) error { - return nil -}