From a10140df3daf86566e2575170c6a8477053aff3b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sun, 1 Dec 2024 21:23:10 -0800 Subject: [PATCH 01/44] feat: Block scheduler scaffolding (#15198) --- pkg/blockbuilder/architecture.md | 159 ++++++++++++++++++ pkg/blockbuilder/builder/builder_test.go | 16 ++ pkg/blockbuilder/{ => builder}/controller.go | 45 ++--- pkg/blockbuilder/{ => builder}/metrics.go | 2 +- pkg/blockbuilder/{ => builder}/pipeline.go | 2 +- .../{ => builder}/pipeline_test.go | 2 +- pkg/blockbuilder/{ => builder}/slimgester.go | 2 +- pkg/blockbuilder/{ => builder}/storage.go | 2 +- .../{ => builder}/storage_test.go | 2 +- pkg/blockbuilder/builder/transport.go | 58 +++++++ pkg/blockbuilder/{ => builder}/tsdb.go | 2 +- pkg/blockbuilder/builder/worker.go | 66 ++++++++ pkg/blockbuilder/scheduler/queue.go | 106 ++++++++++++ pkg/blockbuilder/scheduler/scheduler.go | 56 ++++++ pkg/blockbuilder/scheduler/scheduler_test.go | 148 ++++++++++++++++ pkg/blockbuilder/types/interfaces.go | 53 ++++++ pkg/blockbuilder/types/job.go | 42 +++++ pkg/loki/loki.go | 2 +- pkg/loki/modules.go | 2 +- 19 files changed, 730 insertions(+), 37 deletions(-) create mode 100644 pkg/blockbuilder/architecture.md create mode 100644 pkg/blockbuilder/builder/builder_test.go rename pkg/blockbuilder/{ => builder}/controller.go (92%) rename pkg/blockbuilder/{ => builder}/metrics.go (99%) rename pkg/blockbuilder/{ => builder}/pipeline.go (99%) rename pkg/blockbuilder/{ => builder}/pipeline_test.go (98%) rename pkg/blockbuilder/{ => builder}/slimgester.go (99%) rename pkg/blockbuilder/{ => builder}/storage.go (99%) rename pkg/blockbuilder/{ => builder}/storage_test.go (97%) create mode 100644 pkg/blockbuilder/builder/transport.go rename pkg/blockbuilder/{ => builder}/tsdb.go (99%) create mode 100644 pkg/blockbuilder/builder/worker.go create mode 100644 pkg/blockbuilder/scheduler/queue.go create mode 100644 pkg/blockbuilder/scheduler/scheduler.go create mode 100644 pkg/blockbuilder/scheduler/scheduler_test.go create mode 100644 pkg/blockbuilder/types/interfaces.go create mode 100644 pkg/blockbuilder/types/job.go diff --git a/pkg/blockbuilder/architecture.md b/pkg/blockbuilder/architecture.md new file mode 100644 index 0000000000000..633d7340ccca3 --- /dev/null +++ b/pkg/blockbuilder/architecture.md @@ -0,0 +1,159 @@ +# Block Builder Architecture + +## Overview + +The Block Builder and Block Scheduler are separate components designed to build storage formats from ingested Kafka data. The Block Scheduler coordinates job distribution to multiple Block Builder instances, implementing a pull-based architecture that decouples read and write operations, allowing for independent scaling and simpler operational management. This document describes the architecture and interaction between components. + +## Package Structure + +The Block Builder system is organized into three main packages: + +### pkg/blockbuilder/types +- Contains shared type definitions and interfaces +- Defines core data structures like `Job` and `Offsets` +- Provides interface definitions for: + - `Worker`: Interface for processing jobs and reporting status + - `Scheduler`: Interface for job scheduling and worker management + - `Transport`: Interface for communication between components + +### pkg/blockbuilder/scheduler +- Implements the job queue and scheduling logic +- Manages job distribution to block builders +- Tracks job progress and ensures exactly-once processing +- Handles job state management and offset tracking + +### pkg/blockbuilder/builder +- Implements the block builder worker functionality +- Processes assigned jobs and builds storage formats +- Manages transport layer communication +- Handles data processing and object storage interactions + +## Component Diagram + +```mermaid +graph TB + subgraph Kafka + KP[Kafka Partitions] + end + + subgraph Block Scheduler + S[Scheduler] + Q[Job Queue] + PC[Partition Controller] + + subgraph Transport Layer + T[gRPC/Transport Interface] + end + end + + subgraph Block Builders + BB1[Block Builder 1] + BB2[Block Builder 2] + BB3[Block Builder N] + end + + subgraph Storage + OS[Object Storage] + end + + KP --> PC + PC --> S + S <--> Q + S <--> T + T <--> BB1 + T <--> BB2 + T <--> BB3 + BB1 --> OS + BB2 --> OS + BB3 --> OS +``` + +## Job Processing Sequence + +```mermaid +sequenceDiagram + participant PC as Partition Controller + participant S as Block Scheduler + participant Q as Queue + participant T as Transport + participant BB as Block Builder + participant OS as Object Storage + + loop Monitor Partitions + PC->>PC: Check for new offsets + PC->>S: Create Job (partition, offset range) + S->>Q: Enqueue Job + end + + BB->>T: Request Job + T->>S: Forward Request + S->>Q: Dequeue Job + Q-->>S: Return Job (or empty) + alt Has Job + S->>T: Send Job + T->>BB: Forward Job + BB->>OS: Process & Write Data + BB->>T: Report Success + T->>S: Forward Status + S->>PC: Commit Offset + else No Job + S->>T: Send No Job Available + T->>BB: Forward Response + end +``` + +## Core Components + +### Job and Offsets +- `Job`: Represents a unit of work for processing Kafka data + - Contains a partition ID and an offset range + - Immutable data structure that can be safely passed between components +- `Offsets`: Defines a half-open range [min,max) of Kafka offsets to process + - Used to track progress and ensure exactly-once processing + +### Block Scheduler +- Central component responsible for: + - Managing the job queue + - Coordinating Block Builder assignments + - Tracking job progress +- Implements a pull-based model where Block Builders request jobs +- Decoupled from specific transport mechanisms through the Transport interface + +### Block Builder +- Processes jobs assigned by the Block Scheduler +- Responsible for: + - Building storage formats from Kafka data + - Writing completed blocks to object storage + - Reporting job status back to scheduler +- Implements the Worker interface for job processing + +### Transport Layer +- Provides communication between Block Builders and Scheduler +- Abstracts transport mechanism (currently in-memory & gRPC) +- Defines message types for: + - Job requests + - Job completion notifications + - Job synchronization + +## Design Principles + +### Decoupled I/O +- Business logic is separated from I/O operations +- Transport interface allows for different communication mechanisms +- Enables easier testing through mock implementations + +### Stateless Design +- Block Builders are stateless workers +- All state is managed by the Scheduler +- Allows for easy scaling and failover + +### Pull-Based Architecture +- Block Builders pull jobs when ready +- Natural load balancing +- Prevents overloading of workers + + +### Interface-Driven Development +- Core components defined by interfaces +- Allows for multiple implementations +- Facilitates testing and modularity diff --git a/pkg/blockbuilder/builder/builder_test.go b/pkg/blockbuilder/builder/builder_test.go new file mode 100644 index 0000000000000..ac9890526f1d7 --- /dev/null +++ b/pkg/blockbuilder/builder/builder_test.go @@ -0,0 +1,16 @@ +package builder + +import ( + "github.com/grafana/loki/v3/pkg/blockbuilder/types" +) + +// TestBuilder implements Worker interface for testing +type TestBuilder struct { + *Worker +} + +func NewTestBuilder(builderID string, transport types.Transport) *TestBuilder { + return &TestBuilder{ + Worker: NewWorker(builderID, transport), + } +} diff --git a/pkg/blockbuilder/controller.go b/pkg/blockbuilder/builder/controller.go similarity index 92% rename from pkg/blockbuilder/controller.go rename to pkg/blockbuilder/builder/controller.go index 1193119525bd2..60570daa912cb 100644 --- a/pkg/blockbuilder/controller.go +++ b/pkg/blockbuilder/builder/controller.go @@ -1,4 +1,4 @@ -package blockbuilder +package builder import ( "context" @@ -7,26 +7,16 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" - "github.com/prometheus/prometheus/model/labels" - "github.com/grafana/dskit/backoff" + "github.com/prometheus/prometheus/model/labels" + "github.com/grafana/loki/v3/pkg/blockbuilder/types" "github.com/grafana/loki/v3/pkg/kafka" "github.com/grafana/loki/v3/pkg/kafka/partition" "github.com/grafana/loki/pkg/push" ) -// [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 { Topic() string @@ -43,7 +33,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<- []AppendInput) (int64, error) + Process(context.Context, types.Offsets, chan<- []AppendInput) (int64, error) Close() error } @@ -125,7 +115,7 @@ func (l *PartitionJobController) EarliestPartitionOffset(ctx context.Context) (i ) } -func (l *PartitionJobController) Process(ctx context.Context, offsets Offsets, ch chan<- []AppendInput) (int64, error) { +func (l *PartitionJobController) Process(ctx context.Context, offsets types.Offsets, ch chan<- []AppendInput) (int64, error) { l.part.SetOffsetForConsumption(offsets.Min) var ( @@ -188,16 +178,16 @@ func (l *PartitionJobController) Process(ctx context.Context, offsets Offsets, c // 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) { +func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, *types.Job, error) { // Read the most recent committed offset committedOffset, err := l.HighestCommittedOffset(ctx) if err != nil { - return false, Job{}, err + return false, nil, err } earliestOffset, err := l.EarliestPartitionOffset(ctx) if err != nil { - return false, Job{}, err + return false, nil, err } startOffset := committedOffset + 1 @@ -207,28 +197,27 @@ func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, Job, error) highestOffset, err := l.HighestPartitionOffset(ctx) if err != nil { - return false, Job{}, err + return false, nil, err } if highestOffset < committedOffset { level.Error(l.logger).Log("msg", "partition highest offset is less than committed offset", "highest", highestOffset, "committed", committedOffset) - return false, Job{}, fmt.Errorf("partition highest offset is less than committed offset") + return false, nil, fmt.Errorf("partition highest offset is less than committed offset") } if highestOffset == committedOffset { level.Info(l.logger).Log("msg", "no pending records to process") - return false, Job{}, nil + return false, nil, nil } // Create the job with the calculated offsets - job := Job{ - Partition: l.part.Partition(), - Offsets: Offsets{ - Min: startOffset, - Max: min(startOffset+l.stepLen, highestOffset), - }, + offsets := types.Offsets{ + Min: startOffset, + Max: min(startOffset+l.stepLen, highestOffset), } + // Convert partition from int32 to int + job := types.NewJob(int(l.part.Partition()), offsets) return true, job, nil } @@ -279,7 +268,7 @@ func (d *dummyPartitionController) Commit(_ context.Context, offset int64) error return nil } -func (d *dummyPartitionController) Process(ctx context.Context, offsets Offsets, ch chan<- []AppendInput) (int64, error) { +func (d *dummyPartitionController) Process(ctx context.Context, offsets types.Offsets, ch chan<- []AppendInput) (int64, error) { for i := int(offsets.Min); i < int(offsets.Max); i++ { batch := d.createBatch(i) select { diff --git a/pkg/blockbuilder/metrics.go b/pkg/blockbuilder/builder/metrics.go similarity index 99% rename from pkg/blockbuilder/metrics.go rename to pkg/blockbuilder/builder/metrics.go index 31679e34f4466..3411985209533 100644 --- a/pkg/blockbuilder/metrics.go +++ b/pkg/blockbuilder/builder/metrics.go @@ -1,4 +1,4 @@ -package blockbuilder +package builder import ( "github.com/prometheus/client_golang/prometheus" diff --git a/pkg/blockbuilder/pipeline.go b/pkg/blockbuilder/builder/pipeline.go similarity index 99% rename from pkg/blockbuilder/pipeline.go rename to pkg/blockbuilder/builder/pipeline.go index 494763d8c83f3..eadefb5f9bb43 100644 --- a/pkg/blockbuilder/pipeline.go +++ b/pkg/blockbuilder/builder/pipeline.go @@ -1,4 +1,4 @@ -package blockbuilder +package builder import ( "context" diff --git a/pkg/blockbuilder/pipeline_test.go b/pkg/blockbuilder/builder/pipeline_test.go similarity index 98% rename from pkg/blockbuilder/pipeline_test.go rename to pkg/blockbuilder/builder/pipeline_test.go index 9ec69d2006ebe..be3246318b20e 100644 --- a/pkg/blockbuilder/pipeline_test.go +++ b/pkg/blockbuilder/builder/pipeline_test.go @@ -1,4 +1,4 @@ -package blockbuilder +package builder import ( "context" diff --git a/pkg/blockbuilder/slimgester.go b/pkg/blockbuilder/builder/slimgester.go similarity index 99% rename from pkg/blockbuilder/slimgester.go rename to pkg/blockbuilder/builder/slimgester.go index 249a061aa73df..1807892cbec05 100644 --- a/pkg/blockbuilder/slimgester.go +++ b/pkg/blockbuilder/builder/slimgester.go @@ -1,4 +1,4 @@ -package blockbuilder +package builder import ( "bytes" diff --git a/pkg/blockbuilder/storage.go b/pkg/blockbuilder/builder/storage.go similarity index 99% rename from pkg/blockbuilder/storage.go rename to pkg/blockbuilder/builder/storage.go index 2815b9b97ad86..859f541b5daf2 100644 --- a/pkg/blockbuilder/storage.go +++ b/pkg/blockbuilder/builder/storage.go @@ -1,4 +1,4 @@ -package blockbuilder +package builder import ( "context" diff --git a/pkg/blockbuilder/storage_test.go b/pkg/blockbuilder/builder/storage_test.go similarity index 97% rename from pkg/blockbuilder/storage_test.go rename to pkg/blockbuilder/builder/storage_test.go index 8fc6b237e132d..7f281da48ec62 100644 --- a/pkg/blockbuilder/storage_test.go +++ b/pkg/blockbuilder/builder/storage_test.go @@ -1,4 +1,4 @@ -package blockbuilder +package builder import ( "os" diff --git a/pkg/blockbuilder/builder/transport.go b/pkg/blockbuilder/builder/transport.go new file mode 100644 index 0000000000000..ae498459cb667 --- /dev/null +++ b/pkg/blockbuilder/builder/transport.go @@ -0,0 +1,58 @@ +package builder + +import ( + "context" + + "github.com/grafana/loki/v3/pkg/blockbuilder/types" +) + +var ( + _ types.Transport = unimplementedTransport{} + _ types.Transport = &MemoryTransport{} +) + +// unimplementedTransport provides default implementations that panic +type unimplementedTransport struct{} + +func (t unimplementedTransport) SendGetJobRequest(_ context.Context, _ *types.GetJobRequest) (*types.GetJobResponse, error) { + panic("unimplemented") +} + +func (t unimplementedTransport) SendCompleteJob(_ context.Context, _ *types.CompleteJobRequest) error { + panic("unimplemented") +} + +func (t unimplementedTransport) SendSyncJob(_ context.Context, _ *types.SyncJobRequest) error { + panic("unimplemented") +} + +// MemoryTransport implements Transport interface for in-memory communication +type MemoryTransport struct { + scheduler types.Scheduler +} + +// NewMemoryTransport creates a new in-memory transport instance +func NewMemoryTransport(scheduler types.Scheduler) *MemoryTransport { + return &MemoryTransport{ + scheduler: scheduler, + } +} + +func (t *MemoryTransport) SendGetJobRequest(ctx context.Context, req *types.GetJobRequest) (*types.GetJobResponse, error) { + job, ok, err := t.scheduler.HandleGetJob(ctx, req.BuilderID) + if err != nil { + return nil, err + } + return &types.GetJobResponse{ + Job: job, + OK: ok, + }, nil +} + +func (t *MemoryTransport) SendCompleteJob(ctx context.Context, req *types.CompleteJobRequest) error { + return t.scheduler.HandleCompleteJob(ctx, req.BuilderID, req.Job) +} + +func (t *MemoryTransport) SendSyncJob(ctx context.Context, req *types.SyncJobRequest) error { + return t.scheduler.HandleSyncJob(ctx, req.BuilderID, req.Job) +} diff --git a/pkg/blockbuilder/tsdb.go b/pkg/blockbuilder/builder/tsdb.go similarity index 99% rename from pkg/blockbuilder/tsdb.go rename to pkg/blockbuilder/builder/tsdb.go index 8af463fcd27da..e90bedb3815ad 100644 --- a/pkg/blockbuilder/tsdb.go +++ b/pkg/blockbuilder/builder/tsdb.go @@ -1,4 +1,4 @@ -package blockbuilder +package builder import ( "bytes" diff --git a/pkg/blockbuilder/builder/worker.go b/pkg/blockbuilder/builder/worker.go new file mode 100644 index 0000000000000..41dc0cb7563ec --- /dev/null +++ b/pkg/blockbuilder/builder/worker.go @@ -0,0 +1,66 @@ +package builder + +import ( + "context" + + "github.com/grafana/loki/v3/pkg/blockbuilder/types" +) + +var ( + _ types.Worker = unimplementedWorker{} + _ types.Worker = &Worker{} +) + +// unimplementedWorker provides default implementations for the Worker interface. +type unimplementedWorker struct{} + +func (u unimplementedWorker) GetJob(_ context.Context) (*types.Job, bool, error) { + panic("unimplemented") +} + +func (u unimplementedWorker) CompleteJob(_ context.Context, _ *types.Job) error { + panic("unimplemented") +} + +func (u unimplementedWorker) SyncJob(_ context.Context, _ *types.Job) error { + panic("unimplemented") +} + +// Worker is the implementation of the Worker interface. +type Worker struct { + unimplementedWorker + transport types.Transport + builderID string +} + +// NewWorker creates a new Worker instance. +func NewWorker(builderID string, transport types.Transport) *Worker { + return &Worker{ + transport: transport, + builderID: builderID, + } +} + +func (w *Worker) GetJob(ctx context.Context) (*types.Job, bool, error) { + resp, err := w.transport.SendGetJobRequest(ctx, &types.GetJobRequest{ + BuilderID: w.builderID, + }) + if err != nil { + return nil, false, err + } + return resp.Job, resp.OK, nil +} + +func (w *Worker) CompleteJob(ctx context.Context, job *types.Job) error { + return w.transport.SendCompleteJob(ctx, &types.CompleteJobRequest{ + BuilderID: w.builderID, + Job: job, + }) +} + +func (w *Worker) SyncJob(ctx context.Context, job *types.Job) error { + return w.transport.SendSyncJob(ctx, &types.SyncJobRequest{ + BuilderID: w.builderID, + Job: job, + }) +} diff --git a/pkg/blockbuilder/scheduler/queue.go b/pkg/blockbuilder/scheduler/queue.go new file mode 100644 index 0000000000000..3e9cf087c6792 --- /dev/null +++ b/pkg/blockbuilder/scheduler/queue.go @@ -0,0 +1,106 @@ +package scheduler + +import ( + "fmt" + "sync" + + "github.com/grafana/loki/v3/pkg/blockbuilder/types" +) + +// jobAssignment tracks a job and its assigned builder +type jobAssignment struct { + job *types.Job + builderID string +} + +// JobQueue manages the queue of pending jobs and tracks their state. +type JobQueue struct { + pending map[string]*types.Job // Jobs waiting to be processed, key is job ID + inProgress map[string]*jobAssignment // job ID -> assignment info + completed map[string]*types.Job // Completed jobs, key is job ID + mu sync.RWMutex +} + +// NewJobQueue creates a new job queue instance +func NewJobQueue() *JobQueue { + return &JobQueue{ + pending: make(map[string]*types.Job), + inProgress: make(map[string]*jobAssignment), + completed: make(map[string]*types.Job), + } +} + +// Enqueue adds a new job to the pending queue +// This is a naive implementation, intended to be refactored +func (q *JobQueue) Enqueue(job *types.Job) error { + q.mu.Lock() + defer q.mu.Unlock() + + if _, exists := q.pending[job.ID]; exists { + return fmt.Errorf("job %s already exists in pending queue", job.ID) + } + if _, exists := q.inProgress[job.ID]; exists { + return fmt.Errorf("job %s already exists in progress", job.ID) + } + if _, exists := q.completed[job.ID]; exists { + return fmt.Errorf("job %s already completed", job.ID) + } + + q.pending[job.ID] = job + return nil +} + +// Dequeue gets the next available job and assigns it to a builder +func (q *JobQueue) Dequeue(builderID string) (*types.Job, bool, error) { + q.mu.Lock() + defer q.mu.Unlock() + + // Simple FIFO for now + for id, job := range q.pending { + delete(q.pending, id) + q.inProgress[id] = &jobAssignment{ + job: job, + builderID: builderID, + } + return job, true, nil + } + + return nil, false, nil +} + +// MarkComplete moves a job from in-progress to completed +func (q *JobQueue) MarkComplete(jobID string, builderID string) error { + q.mu.Lock() + defer q.mu.Unlock() + + assignment, exists := q.inProgress[jobID] + if !exists { + return fmt.Errorf("job %s not found in progress", jobID) + } + + if assignment.builderID != builderID { + return fmt.Errorf("job %s not assigned to builder %s", jobID, builderID) + } + + delete(q.inProgress, jobID) + q.completed[jobID] = assignment.job + return nil +} + +// SyncJob updates the state of an in-progress job +func (q *JobQueue) SyncJob(jobID string, builderID string, job *types.Job) error { + q.mu.Lock() + defer q.mu.Unlock() + + assignment, exists := q.inProgress[jobID] + if !exists { + return fmt.Errorf("job %s not found in progress", jobID) + } + + if assignment.builderID != builderID { + return fmt.Errorf("job %s not assigned to builder %s", jobID, builderID) + } + + assignment.job = job + return nil +} diff --git a/pkg/blockbuilder/scheduler/scheduler.go b/pkg/blockbuilder/scheduler/scheduler.go new file mode 100644 index 0000000000000..274713b5b1c36 --- /dev/null +++ b/pkg/blockbuilder/scheduler/scheduler.go @@ -0,0 +1,56 @@ +package scheduler + +import ( + "context" + + "github.com/grafana/loki/v3/pkg/blockbuilder/types" +) + +var ( + _ types.Scheduler = unimplementedScheduler{} + _ types.Scheduler = &QueueScheduler{} +) + +// unimplementedScheduler provides default implementations that panic. +type unimplementedScheduler struct{} + +func (s unimplementedScheduler) HandleGetJob(_ context.Context, _ string) (*types.Job, bool, error) { + panic("unimplemented") +} + +func (s unimplementedScheduler) HandleCompleteJob(_ context.Context, _ string, _ *types.Job) error { + panic("unimplemented") +} + +func (s unimplementedScheduler) HandleSyncJob(_ context.Context, _ string, _ *types.Job) error { + panic("unimplemented") +} + +// QueueScheduler implements the Scheduler interface +type QueueScheduler struct { + queue *JobQueue +} + +// NewScheduler creates a new scheduler instance +func NewScheduler(queue *JobQueue) *QueueScheduler { + return &QueueScheduler{ + queue: queue, + } +} + +func (s *QueueScheduler) HandleGetJob(ctx context.Context, builderID string) (*types.Job, bool, error) { + select { + case <-ctx.Done(): + return nil, false, ctx.Err() + default: + return s.queue.Dequeue(builderID) + } +} + +func (s *QueueScheduler) HandleCompleteJob(_ context.Context, builderID string, job *types.Job) error { + return s.queue.MarkComplete(job.ID, builderID) +} + +func (s *QueueScheduler) HandleSyncJob(_ context.Context, builderID string, job *types.Job) error { + return s.queue.SyncJob(job.ID, builderID, job) +} diff --git a/pkg/blockbuilder/scheduler/scheduler_test.go b/pkg/blockbuilder/scheduler/scheduler_test.go new file mode 100644 index 0000000000000..ad6829bc8fe69 --- /dev/null +++ b/pkg/blockbuilder/scheduler/scheduler_test.go @@ -0,0 +1,148 @@ +package scheduler + +import ( + "context" + "testing" + "time" + + "github.com/grafana/loki/v3/pkg/blockbuilder/builder" + "github.com/grafana/loki/v3/pkg/blockbuilder/types" +) + +type testEnv struct { + queue *JobQueue + scheduler *QueueScheduler + transport *builder.MemoryTransport + builder *builder.Worker +} + +func newTestEnv(builderID string) *testEnv { + queue := NewJobQueue() + scheduler := NewScheduler(queue) + transport := builder.NewMemoryTransport(scheduler) + builder := builder.NewWorker(builderID, builder.NewMemoryTransport(scheduler)) + + return &testEnv{ + queue: queue, + scheduler: scheduler, + transport: transport, + builder: builder, + } +} + +func TestScheduleAndProcessJob(t *testing.T) { + env := newTestEnv("test-builder-1") + ctx := context.Background() + + // Create and enqueue a test job + job := types.NewJob(1, types.Offsets{Min: 100, Max: 200}) + err := env.queue.Enqueue(job) + if err != nil { + t.Fatalf("failed to enqueue job: %v", err) + } + + // Builder gets job + receivedJob, ok, err := env.builder.GetJob(ctx) + if err != nil { + t.Fatalf("failed to get job: %v", err) + } + if !ok { + t.Fatal("expected to receive job") + } + if receivedJob.ID != job.ID { + t.Errorf("got job ID %s, want %s", receivedJob.ID, job.ID) + } + + // Builder completes job + err = env.builder.CompleteJob(ctx, receivedJob) + if err != nil { + t.Fatalf("failed to complete job: %v", err) + } + + // Try to get another job (should be none available) + _, ok, err = env.builder.GetJob(ctx) + if err != nil { + t.Fatalf("failed to get second job: %v", err) + } + if ok { + t.Error("got unexpected second job") + } +} + +func TestContextCancellation(t *testing.T) { + env := newTestEnv("test-builder-1") + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) + defer cancel() + + // Try to get job after context timeout + time.Sleep(20 * time.Millisecond) + _, _, err := env.builder.GetJob(ctx) + if err == nil { + t.Error("expected error from cancelled context") + } +} + +func TestMultipleBuilders(t *testing.T) { + // Create first environment + env1 := newTestEnv("test-builder-1") + // Create second builder using same scheduler + builder2 := builder.NewWorker("test-builder-2", builder.NewMemoryTransport(env1.scheduler)) + + ctx := context.Background() + + // Create test jobs + job1 := types.NewJob(1, types.Offsets{Min: 100, Max: 200}) + job2 := types.NewJob(2, types.Offsets{Min: 300, Max: 400}) + + // Enqueue jobs + err := env1.queue.Enqueue(job1) + if err != nil { + t.Fatalf("failed to enqueue job1: %v", err) + } + err = env1.queue.Enqueue(job2) + if err != nil { + t.Fatalf("failed to enqueue job2: %v", err) + } + + // Builders get jobs + receivedJob1, ok, err := env1.builder.GetJob(ctx) + if err != nil { + t.Fatalf("builder1 failed to get job: %v", err) + } + if !ok { + t.Fatal("builder1 expected to receive job") + } + + receivedJob2, ok, err := builder2.GetJob(ctx) + if err != nil { + t.Fatalf("builder2 failed to get job: %v", err) + } + if !ok { + t.Fatal("builder2 expected to receive job") + } + + // Verify different jobs were assigned + if receivedJob1.ID == receivedJob2.ID { + t.Error("builders received same job") + } + + // Complete jobs + err = env1.builder.CompleteJob(ctx, receivedJob1) + if err != nil { + t.Fatalf("builder1 failed to complete job: %v", err) + } + + err = builder2.CompleteJob(ctx, receivedJob2) + if err != nil { + t.Fatalf("builder2 failed to complete job: %v", err) + } + + // Try to get more jobs (should be none available) + _, ok, err = env1.builder.GetJob(ctx) + if err != nil { + t.Fatalf("builder1 failed to get second job: %v", err) + } + if ok { + t.Error("builder1 got unexpected second job") + } +} diff --git a/pkg/blockbuilder/types/interfaces.go b/pkg/blockbuilder/types/interfaces.go new file mode 100644 index 0000000000000..74267f912fd7e --- /dev/null +++ b/pkg/blockbuilder/types/interfaces.go @@ -0,0 +1,53 @@ +package types + +import "context" + +// Worker interface defines the methods for processing jobs and reporting status. +type Worker interface { + // GetJob requests a new job from the scheduler + GetJob(ctx context.Context) (*Job, bool, error) + // CompleteJob marks a job as finished + CompleteJob(ctx context.Context, job *Job) error + // SyncJob informs the scheduler about an in-progress job + SyncJob(ctx context.Context, job *Job) error +} + +// Scheduler interface defines the methods for scheduling jobs and managing worker pools. +type Scheduler interface { + // HandleGetJob processes a job request from a block builder + HandleGetJob(ctx context.Context, builderID string) (*Job, bool, error) + // HandleCompleteJob processes a job completion notification + HandleCompleteJob(ctx context.Context, builderID string, job *Job) error + // HandleSyncJob processes a job sync request + HandleSyncJob(ctx context.Context, builderID string, job *Job) error +} + +// Transport defines the interface for communication between block builders and scheduler +type Transport interface { + // SendGetJobRequest sends a request to get a new job + SendGetJobRequest(ctx context.Context, req *GetJobRequest) (*GetJobResponse, error) + // SendCompleteJob sends a job completion notification + SendCompleteJob(ctx context.Context, req *CompleteJobRequest) error + // SendSyncJob sends a job sync request + SendSyncJob(ctx context.Context, req *SyncJobRequest) error +} + +// Request/Response message types +type GetJobRequest struct { + BuilderID string +} + +type GetJobResponse struct { + Job *Job + OK bool +} + +type CompleteJobRequest struct { + BuilderID string + Job *Job +} + +type SyncJobRequest struct { + BuilderID string + Job *Job +} diff --git a/pkg/blockbuilder/types/job.go b/pkg/blockbuilder/types/job.go new file mode 100644 index 0000000000000..d6ed42b598906 --- /dev/null +++ b/pkg/blockbuilder/types/job.go @@ -0,0 +1,42 @@ +package types + +import "fmt" + +// Job represents a block building task. +type Job struct { + ID string + Status JobStatus + // Partition and offset information + Partition int + Offsets Offsets +} + +// JobStatus represents the current state of a job +type JobStatus int + +const ( + JobStatusPending JobStatus = iota + JobStatusInProgress + JobStatusComplete +) + +// Offsets represents the range of offsets to process +type Offsets struct { + Min int64 + Max int64 +} + +// NewJob creates a new job with the given partition and offsets +func NewJob(partition int, offsets Offsets) *Job { + return &Job{ + ID: GenerateJobID(partition, offsets), + Status: JobStatusPending, + Partition: partition, + Offsets: offsets, + } +} + +// GenerateJobID creates a deterministic job ID from partition and offsets +func GenerateJobID(partition int, offsets Offsets) string { + return fmt.Sprintf("job-%d-%d-%d", partition, offsets.Min, offsets.Max) +} diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index ff5c6de1565bd..153387035d6b5 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -31,7 +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" + blockbuilder "github.com/grafana/loki/v3/pkg/blockbuilder/builder" "github.com/grafana/loki/v3/pkg/bloombuild" "github.com/grafana/loki/v3/pkg/bloomgateway" "github.com/grafana/loki/v3/pkg/compactor" diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 3c76434ef852c..994576076af3e 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -36,7 +36,7 @@ import ( "github.com/prometheus/common/model" "github.com/grafana/loki/v3/pkg/analytics" - "github.com/grafana/loki/v3/pkg/blockbuilder" + blockbuilder "github.com/grafana/loki/v3/pkg/blockbuilder/builder" "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" From 3a5bed438a22d39170b04ca49b0c1467c140772a Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 2 Dec 2024 02:29:02 -0800 Subject: [PATCH 02/44] refactor: kafka offset signals (#15201) --- pkg/kafka/partition/reader_service.go | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/pkg/kafka/partition/reader_service.go b/pkg/kafka/partition/reader_service.go index b9d92fc7c5d6c..40c1b64b3b790 100644 --- a/pkg/kafka/partition/reader_service.go +++ b/pkg/kafka/partition/reader_service.go @@ -17,9 +17,6 @@ import ( ) const ( - kafkaStartOffset = -2 - kafkaEndOffset = -1 - phaseStarting = "starting" phaseRunning = "running" ) @@ -111,7 +108,7 @@ func newReaderService( consumerFactory: consumerFactory, logger: log.With(logger, "partition", reader.Partition(), "consumer_group", reader.ConsumerGroup()), metrics: newServiceMetrics(reg), - lastProcessedOffset: kafkaEndOffset, + lastProcessedOffset: int64(KafkaEndOffset), } // Create the committer @@ -135,12 +132,12 @@ func (s *ReaderService) starting(ctx context.Context) error { } if lastCommittedOffset == int64(KafkaEndOffset) { - level.Warn(logger).Log("msg", fmt.Sprintf("no committed offset found, starting from %d", kafkaStartOffset)) + level.Warn(logger).Log("msg", fmt.Sprintf("no committed offset found, starting from %d", KafkaStartOffset)) } else { level.Debug(logger).Log("msg", "last committed offset", "offset", lastCommittedOffset) } - consumeOffset := int64(kafkaStartOffset) + consumeOffset := int64(KafkaStartOffset) if lastCommittedOffset >= 0 { // Read from the next offset. consumeOffset = lastCommittedOffset + 1 @@ -222,7 +219,7 @@ func (s *ReaderService) fetchUntilLagSatisfied( for b.Ongoing() { // Send a direct request to the Kafka backend to fetch the partition start offset. - partitionStartOffset, err := s.reader.FetchPartitionOffset(ctx, kafkaStartOffset) + partitionStartOffset, err := s.reader.FetchPartitionOffset(ctx, KafkaStartOffset) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch partition start offset", "err", err) b.Wait() @@ -240,7 +237,7 @@ func (s *ReaderService) fetchUntilLagSatisfied( // We intentionally don't use WaitNextFetchLastProducedOffset() to not introduce further // latency. lastProducedOffsetRequestedAt := time.Now() - lastProducedOffset, err := s.reader.FetchPartitionOffset(ctx, kafkaEndOffset) + lastProducedOffset, err := s.reader.FetchPartitionOffset(ctx, KafkaEndOffset) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch last produced offset", "err", err) b.Wait() From 84e1204a7abb121fccc9c93f0788870324e5deeb Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 09:12:09 -0500 Subject: [PATCH 03/44] fix(deps): update module github.com/aws/aws-sdk-go-v2 to v1.32.6 (#15199) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: Paul Rogers --- nix/packages/loki.nix | 2 +- tools/lambda-promtail/go.mod | 2 +- tools/lambda-promtail/go.sum | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/nix/packages/loki.nix b/nix/packages/loki.nix index b0b22e106c21b..03c17fc74f8c0 100644 --- a/nix/packages/loki.nix +++ b/nix/packages/loki.nix @@ -5,7 +5,7 @@ let pname = "lambda-promtail"; src = ./../../tools/lambda-promtail; - vendorHash = "sha256-d4EMPbUuw3m+M/9lkxQOJGhN8qvvmeoIwfBGJX1Xrl8="; + vendorHash = "sha256-90pY7nXU92K3HC2tr1oHT2AYpiaimbcdnIhyP9awkTk="; doCheck = false; diff --git a/tools/lambda-promtail/go.mod b/tools/lambda-promtail/go.mod index c28517895ce52..b73cd9736837a 100644 --- a/tools/lambda-promtail/go.mod +++ b/tools/lambda-promtail/go.mod @@ -4,7 +4,7 @@ go 1.22 require ( github.com/aws/aws-lambda-go v1.47.0 - github.com/aws/aws-sdk-go-v2 v1.32.5 + github.com/aws/aws-sdk-go-v2 v1.32.6 github.com/aws/aws-sdk-go-v2/config v1.28.5 github.com/aws/aws-sdk-go-v2/service/s3 v1.69.0 github.com/go-kit/log v0.2.1 diff --git a/tools/lambda-promtail/go.sum b/tools/lambda-promtail/go.sum index 3609501ea11b6..34e76123cdaac 100644 --- a/tools/lambda-promtail/go.sum +++ b/tools/lambda-promtail/go.sum @@ -48,8 +48,8 @@ github.com/aws/aws-lambda-go v1.47.0 h1:0H8s0vumYx/YKs4sE7YM0ktwL2eWse+kfopsRI1s github.com/aws/aws-lambda-go v1.47.0/go.mod h1:dpMpZgvWx5vuQJfBt0zqBha60q7Dd7RfgJv23DymV8A= github.com/aws/aws-sdk-go v1.54.19 h1:tyWV+07jagrNiCcGRzRhdtVjQs7Vy41NwsuOcl0IbVI= github.com/aws/aws-sdk-go v1.54.19/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= -github.com/aws/aws-sdk-go-v2 v1.32.5 h1:U8vdWJuY7ruAkzaOdD7guwJjD06YSKmnKCJs7s3IkIo= -github.com/aws/aws-sdk-go-v2 v1.32.5/go.mod h1:P5WJBrYqqbWVaOxgH0X/FYYD47/nooaPOZPlQdmiN2U= +github.com/aws/aws-sdk-go-v2 v1.32.6 h1:7BokKRgRPuGmKkFMhEg/jSul+tB9VvXhcViILtfG8b4= +github.com/aws/aws-sdk-go-v2 v1.32.6/go.mod h1:P5WJBrYqqbWVaOxgH0X/FYYD47/nooaPOZPlQdmiN2U= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.7 h1:lL7IfaFzngfx0ZwUGOZdsFFnQ5uLvR0hWqqhyE7Q9M8= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.7/go.mod h1:QraP0UcVlQJsmHfioCrveWOC1nbiWUl3ej08h4mXWoc= github.com/aws/aws-sdk-go-v2/config v1.28.5 h1:Za41twdCXbuyyWv9LndXxZZv3QhTG1DinqlFsSuvtI0= From 61c3efdcb89a1f324a3def540637558c1db5989b Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 09:41:39 -0500 Subject: [PATCH 04/44] fix(deps): update module github.com/shirou/gopsutil/v4 to v4.24.11 (#15194) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- vendor/modules.txt | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index 466ea2e299542..5174a10e73330 100644 --- a/go.mod +++ b/go.mod @@ -133,7 +133,7 @@ require ( github.com/prometheus/common/sigv4 v0.1.0 github.com/richardartoul/molecule v1.0.0 github.com/schollz/progressbar/v3 v3.17.1 - github.com/shirou/gopsutil/v4 v4.24.10 + github.com/shirou/gopsutil/v4 v4.24.11 github.com/thanos-io/objstore v0.0.0-20241111205755-d1dd89d41f97 github.com/twmb/franz-go v1.18.0 github.com/twmb/franz-go/pkg/kadm v1.14.0 diff --git a/go.sum b/go.sum index ff695ca82846f..9d7fe1bb3c307 100644 --- a/go.sum +++ b/go.sum @@ -2507,8 +2507,8 @@ github.com/sercand/kuberesolver/v5 v5.1.1/go.mod h1:Fs1KbKhVRnB2aDWN12NjKCB+RgYM github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.20.9+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil/v3 v3.22.9/go.mod h1:bBYl1kjgEJpWpxeHmLI+dVHWtyAwfcmSBLDsp2TNT8A= -github.com/shirou/gopsutil/v4 v4.24.10 h1:7VOzPtfw/5YDU+jLEoBwXwxJbQetULywoSV4RYY7HkM= -github.com/shirou/gopsutil/v4 v4.24.10/go.mod h1:s4D/wg+ag4rG0WO7AiTj2BeYCRhym0vM7DHbZRxnIT8= +github.com/shirou/gopsutil/v4 v4.24.11 h1:WaU9xqGFKvFfsUv94SXcUPD7rCkU0vr/asVdQOBZNj8= +github.com/shirou/gopsutil/v4 v4.24.11/go.mod h1:s4D/wg+ag4rG0WO7AiTj2BeYCRhym0vM7DHbZRxnIT8= github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= github.com/shopspring/decimal v0.0.0-20200105231215-408a2507e114/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= diff --git a/vendor/modules.txt b/vendor/modules.txt index 743a9dbefb253..e7e389624a74a 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1507,7 +1507,7 @@ github.com/segmentio/fasthash/fnv1a # github.com/sercand/kuberesolver/v5 v5.1.1 ## explicit; go 1.18 github.com/sercand/kuberesolver/v5 -# github.com/shirou/gopsutil/v4 v4.24.10 +# github.com/shirou/gopsutil/v4 v4.24.11 ## explicit; go 1.18 github.com/shirou/gopsutil/v4/common github.com/shirou/gopsutil/v4/cpu From bc5687e772c1302310c4078b89962e8432e2a600 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 09:54:57 -0500 Subject: [PATCH 05/44] fix(deps): update module github.com/aws/aws-sdk-go-v2/config to v1.28.6 (#15202) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: Paul Rogers --- nix/packages/loki.nix | 2 +- tools/lambda-promtail/go.mod | 18 +++++++++--------- tools/lambda-promtail/go.sum | 36 ++++++++++++++++++------------------ 3 files changed, 28 insertions(+), 28 deletions(-) diff --git a/nix/packages/loki.nix b/nix/packages/loki.nix index 03c17fc74f8c0..c2f0fda208376 100644 --- a/nix/packages/loki.nix +++ b/nix/packages/loki.nix @@ -5,7 +5,7 @@ let pname = "lambda-promtail"; src = ./../../tools/lambda-promtail; - vendorHash = "sha256-90pY7nXU92K3HC2tr1oHT2AYpiaimbcdnIhyP9awkTk="; + vendorHash = "sha256-HfIjAMaS9L3hRuKQo99964e9GH4fsdZQ27Awk9Vzwpo="; doCheck = false; diff --git a/tools/lambda-promtail/go.mod b/tools/lambda-promtail/go.mod index b73cd9736837a..5564da4268043 100644 --- a/tools/lambda-promtail/go.mod +++ b/tools/lambda-promtail/go.mod @@ -5,7 +5,7 @@ go 1.22 require ( github.com/aws/aws-lambda-go v1.47.0 github.com/aws/aws-sdk-go-v2 v1.32.6 - github.com/aws/aws-sdk-go-v2/config v1.28.5 + github.com/aws/aws-sdk-go-v2/config v1.28.6 github.com/aws/aws-sdk-go-v2/service/s3 v1.69.0 github.com/go-kit/log v0.2.1 github.com/gogo/protobuf v1.3.2 @@ -24,19 +24,19 @@ require ( github.com/alecthomas/units v0.0.0-20240626203959-61d1e3462e30 // indirect github.com/armon/go-metrics v0.4.1 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.7 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.17.46 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.20 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.24 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.24 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.17.47 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.21 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.25 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.25 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.24 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.1 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.5 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.24.6 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.33.1 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.24.7 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.6 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.33.2 // indirect github.com/aws/smithy-go v1.22.1 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/c2h5oh/datasize v0.0.0-20231215233829-aa82cc1e6500 // indirect diff --git a/tools/lambda-promtail/go.sum b/tools/lambda-promtail/go.sum index 34e76123cdaac..1e9cb2e9c3eda 100644 --- a/tools/lambda-promtail/go.sum +++ b/tools/lambda-promtail/go.sum @@ -52,16 +52,16 @@ github.com/aws/aws-sdk-go-v2 v1.32.6 h1:7BokKRgRPuGmKkFMhEg/jSul+tB9VvXhcViILtfG github.com/aws/aws-sdk-go-v2 v1.32.6/go.mod h1:P5WJBrYqqbWVaOxgH0X/FYYD47/nooaPOZPlQdmiN2U= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.7 h1:lL7IfaFzngfx0ZwUGOZdsFFnQ5uLvR0hWqqhyE7Q9M8= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.7/go.mod h1:QraP0UcVlQJsmHfioCrveWOC1nbiWUl3ej08h4mXWoc= -github.com/aws/aws-sdk-go-v2/config v1.28.5 h1:Za41twdCXbuyyWv9LndXxZZv3QhTG1DinqlFsSuvtI0= -github.com/aws/aws-sdk-go-v2/config v1.28.5/go.mod h1:4VsPbHP8JdcdUDmbTVgNL/8w9SqOkM5jyY8ljIxLO3o= -github.com/aws/aws-sdk-go-v2/credentials v1.17.46 h1:AU7RcriIo2lXjUfHFnFKYsLCwgbz1E7Mm95ieIRDNUg= -github.com/aws/aws-sdk-go-v2/credentials v1.17.46/go.mod h1:1FmYyLGL08KQXQ6mcTlifyFXfJVCNJTVGuQP4m0d/UA= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.20 h1:sDSXIrlsFSFJtWKLQS4PUWRvrT580rrnuLydJrCQ/yA= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.20/go.mod h1:WZ/c+w0ofps+/OUqMwWgnfrgzZH1DZO1RIkktICsqnY= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.24 h1:4usbeaes3yJnCFC7kfeyhkdkPtoRYPa/hTmCqMpKpLI= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.24/go.mod h1:5CI1JemjVwde8m2WG3cz23qHKPOxbpkq0HaoreEgLIY= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.24 h1:N1zsICrQglfzaBnrfM0Ys00860C+QFwu6u/5+LomP+o= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.24/go.mod h1:dCn9HbJ8+K31i8IQ8EWmWj0EiIk0+vKiHNMxTTYveAg= +github.com/aws/aws-sdk-go-v2/config v1.28.6 h1:D89IKtGrs/I3QXOLNTH93NJYtDhm8SYa9Q5CsPShmyo= +github.com/aws/aws-sdk-go-v2/config v1.28.6/go.mod h1:GDzxJ5wyyFSCoLkS+UhGB0dArhb9mI+Co4dHtoTxbko= +github.com/aws/aws-sdk-go-v2/credentials v1.17.47 h1:48bA+3/fCdi2yAwVt+3COvmatZ6jUDNkDTIsqDiMUdw= +github.com/aws/aws-sdk-go-v2/credentials v1.17.47/go.mod h1:+KdckOejLW3Ks3b0E3b5rHsr2f9yuORBum0WPnE5o5w= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.21 h1:AmoU1pziydclFT/xRV+xXE/Vb8fttJCLRPv8oAkprc0= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.21/go.mod h1:AjUdLYe4Tgs6kpH4Bv7uMZo7pottoyHMn4eTcIcneaY= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.25 h1:s/fF4+yDQDoElYhfIVvSNyeCydfbuTKzhxSXDXCPasU= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.25/go.mod h1:IgPfDv5jqFIzQSNbUEMoitNooSMXjRSDkhXv8jiROvU= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.25 h1:ZntTCl5EsYnhN/IygQEUugpdwbhdkom9uHcbCftiGgA= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.25/go.mod h1:DBdPrgeocww+CSl1C8cEV8PN1mHMBhuCDLpXezyvWkE= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 h1:VaRN3TlFdd6KxX1x3ILT5ynH6HvKgqdiXoTxAF4HQcQ= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1/go.mod h1:FbtygfRFze9usAadmnGJNc8KsP346kEe+y2/oyhGAGc= github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.24 h1:JX70yGKLj25+lMC5Yyh8wBtvB01GDilyRuJvXJ4piD0= @@ -70,18 +70,18 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.1 h1:iXtILhv github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.1/go.mod h1:9nu0fVANtYiAePIBh2/pFUSwtJ402hLnp854CNoDOeE= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.5 h1:gvZOjQKPxFXy1ft3QnEyXmT+IqneM9QAUWlM3r0mfqw= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.5/go.mod h1:DLWnfvIcm9IET/mmjdxeXbBKmTCm0ZB8p1za9BVteM8= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.5 h1:wtpJ4zcwrSbwhECWQoI/g6WM9zqCcSpHDJIWSbMLOu4= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.5/go.mod h1:qu/W9HXQbbQ4+1+JcZp0ZNPV31ym537ZJN+fiS7Ti8E= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6 h1:50+XsN70RS7dwJ2CkVNXzj7U2L1HKP8nqTd3XWEXBN4= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6/go.mod h1:WqgLmwY7so32kG01zD8CPTJWVWM+TzJoOVHwTg4aPug= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.5 h1:P1doBzv5VEg1ONxnJss1Kh5ZG/ewoIE4MQtKKc6Crgg= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.5/go.mod h1:NOP+euMW7W3Ukt28tAxPuoWao4rhhqJD3QEBk7oCg7w= github.com/aws/aws-sdk-go-v2/service/s3 v1.69.0 h1:Q2ax8S21clKOnHhhr933xm3JxdJebql+R7aNo7p7GBQ= github.com/aws/aws-sdk-go-v2/service/s3 v1.69.0/go.mod h1:ralv4XawHjEMaHOWnTFushl0WRqim/gQWesAMF6hTow= -github.com/aws/aws-sdk-go-v2/service/sso v1.24.6 h1:3zu537oLmsPfDMyjnUS2g+F2vITgy5pB74tHI+JBNoM= -github.com/aws/aws-sdk-go-v2/service/sso v1.24.6/go.mod h1:WJSZH2ZvepM6t6jwu4w/Z45Eoi75lPN7DcydSRtJg6Y= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.5 h1:K0OQAsDywb0ltlFrZm0JHPY3yZp/S9OaoLU33S7vPS8= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.5/go.mod h1:ORITg+fyuMoeiQFiVGoqB3OydVTLkClw/ljbblMq6Cc= -github.com/aws/aws-sdk-go-v2/service/sts v1.33.1 h1:6SZUVRQNvExYlMLbHdlKB48x0fLbc2iVROyaNEwBHbU= -github.com/aws/aws-sdk-go-v2/service/sts v1.33.1/go.mod h1:GqWyYCwLXnlUB1lOAXQyNSPqPLQJvmo8J0DWBzp9mtg= +github.com/aws/aws-sdk-go-v2/service/sso v1.24.7 h1:rLnYAfXQ3YAccocshIH5mzNNwZBkBo+bP6EhIxak6Hw= +github.com/aws/aws-sdk-go-v2/service/sso v1.24.7/go.mod h1:ZHtuQJ6t9A/+YDuxOLnbryAmITtr8UysSny3qcyvJTc= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.6 h1:JnhTZR3PiYDNKlXy50/pNeix9aGMo6lLpXwJ1mw8MD4= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.6/go.mod h1:URronUEGfXZN1VpdktPSD1EkAL9mfrV+2F4sjH38qOY= +github.com/aws/aws-sdk-go-v2/service/sts v1.33.2 h1:s4074ZO1Hk8qv65GqNXqDjmkf4HSQqJukaLuuW0TpDA= +github.com/aws/aws-sdk-go-v2/service/sts v1.33.2/go.mod h1:mVggCnIWoM09jP71Wh+ea7+5gAp53q+49wDFs1SW5z8= github.com/aws/smithy-go v1.22.1 h1:/HPHZQ0g7f4eUeK6HKglFz8uwVfZKgoI25rb/J+dnro= github.com/aws/smithy-go v1.22.1/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3 h1:6df1vn4bBlDDo4tARvBm7l6KA9iVMnE3NWizDeWSrps= From ae8b29f5640b36887b33beef58c0ec8e273b6085 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 10:19:04 -0500 Subject: [PATCH 06/44] fix(deps): update module github.com/aws/aws-sdk-go-v2/service/s3 to v1.70.0 (#15205) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: Paul Rogers --- nix/packages/loki.nix | 2 +- tools/lambda-promtail/go.mod | 8 ++++---- tools/lambda-promtail/go.sum | 16 ++++++++-------- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/nix/packages/loki.nix b/nix/packages/loki.nix index c2f0fda208376..a4f5b4b9feb28 100644 --- a/nix/packages/loki.nix +++ b/nix/packages/loki.nix @@ -5,7 +5,7 @@ let pname = "lambda-promtail"; src = ./../../tools/lambda-promtail; - vendorHash = "sha256-HfIjAMaS9L3hRuKQo99964e9GH4fsdZQ27Awk9Vzwpo="; + vendorHash = "sha256-qsgaWkuSVo5Clb9FpS4SUIHVZZjpfjv0dKtwSUIX7a8="; doCheck = false; diff --git a/tools/lambda-promtail/go.mod b/tools/lambda-promtail/go.mod index 5564da4268043..4662711dbf32c 100644 --- a/tools/lambda-promtail/go.mod +++ b/tools/lambda-promtail/go.mod @@ -6,7 +6,7 @@ require ( github.com/aws/aws-lambda-go v1.47.0 github.com/aws/aws-sdk-go-v2 v1.32.6 github.com/aws/aws-sdk-go-v2/config v1.28.6 - github.com/aws/aws-sdk-go-v2/service/s3 v1.69.0 + github.com/aws/aws-sdk-go-v2/service/s3 v1.70.0 github.com/go-kit/log v0.2.1 github.com/gogo/protobuf v1.3.2 github.com/golang/snappy v0.0.4 @@ -29,11 +29,11 @@ require ( github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.25 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.25 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.24 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.25 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.1 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.6 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.6 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.24.7 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.6 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.33.2 // indirect diff --git a/tools/lambda-promtail/go.sum b/tools/lambda-promtail/go.sum index 1e9cb2e9c3eda..382e523491456 100644 --- a/tools/lambda-promtail/go.sum +++ b/tools/lambda-promtail/go.sum @@ -64,18 +64,18 @@ github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.25 h1:ZntTCl5EsYnhN/IygQ github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.25/go.mod h1:DBdPrgeocww+CSl1C8cEV8PN1mHMBhuCDLpXezyvWkE= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 h1:VaRN3TlFdd6KxX1x3ILT5ynH6HvKgqdiXoTxAF4HQcQ= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1/go.mod h1:FbtygfRFze9usAadmnGJNc8KsP346kEe+y2/oyhGAGc= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.24 h1:JX70yGKLj25+lMC5Yyh8wBtvB01GDilyRuJvXJ4piD0= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.24/go.mod h1:+Ln60j9SUTD0LEwnhEB0Xhg61DHqplBrbZpLgyjoEHg= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.25 h1:r67ps7oHCYnflpgDy2LZU0MAQtQbYIOqNNnqGO6xQkE= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.25/go.mod h1:GrGY+Q4fIokYLtjCVB/aFfCVL6hhGUFl8inD18fDalE= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.1 h1:iXtILhvDxB6kPvEXgsDhGaZCSC6LQET5ZHSdJozeI0Y= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.1/go.mod h1:9nu0fVANtYiAePIBh2/pFUSwtJ402hLnp854CNoDOeE= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.5 h1:gvZOjQKPxFXy1ft3QnEyXmT+IqneM9QAUWlM3r0mfqw= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.5/go.mod h1:DLWnfvIcm9IET/mmjdxeXbBKmTCm0ZB8p1za9BVteM8= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.6 h1:HCpPsWqmYQieU7SS6E9HXfdAMSud0pteVXieJmcpIRI= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.6/go.mod h1:ngUiVRCco++u+soRRVBIvBZxSMMvOVMXA4PJ36JLfSw= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6 h1:50+XsN70RS7dwJ2CkVNXzj7U2L1HKP8nqTd3XWEXBN4= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6/go.mod h1:WqgLmwY7so32kG01zD8CPTJWVWM+TzJoOVHwTg4aPug= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.5 h1:P1doBzv5VEg1ONxnJss1Kh5ZG/ewoIE4MQtKKc6Crgg= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.5/go.mod h1:NOP+euMW7W3Ukt28tAxPuoWao4rhhqJD3QEBk7oCg7w= -github.com/aws/aws-sdk-go-v2/service/s3 v1.69.0 h1:Q2ax8S21clKOnHhhr933xm3JxdJebql+R7aNo7p7GBQ= -github.com/aws/aws-sdk-go-v2/service/s3 v1.69.0/go.mod h1:ralv4XawHjEMaHOWnTFushl0WRqim/gQWesAMF6hTow= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.6 h1:BbGDtTi0T1DYlmjBiCr/le3wzhA37O8QTC5/Ab8+EXk= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.6/go.mod h1:hLMJt7Q8ePgViKupeymbqI0la+t9/iYFBjxQCFwuAwI= +github.com/aws/aws-sdk-go-v2/service/s3 v1.70.0 h1:HrHFR8RoS4l4EvodRMFcJMYQ8o3UhmALn2nbInXaxZA= +github.com/aws/aws-sdk-go-v2/service/s3 v1.70.0/go.mod h1:sT/iQz8JK3u/5gZkT+Hmr7GzVZehUMkRZpOaAwYXeGY= github.com/aws/aws-sdk-go-v2/service/sso v1.24.7 h1:rLnYAfXQ3YAccocshIH5mzNNwZBkBo+bP6EhIxak6Hw= github.com/aws/aws-sdk-go-v2/service/sso v1.24.7/go.mod h1:ZHtuQJ6t9A/+YDuxOLnbryAmITtr8UysSny3qcyvJTc= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.6 h1:JnhTZR3PiYDNKlXy50/pNeix9aGMo6lLpXwJ1mw8MD4= From 5f670a4b183dbf0066b5b941cac0330402f51e6e Mon Sep 17 00:00:00 2001 From: Paul Rogers <129207811+paul1r@users.noreply.github.com> Date: Mon, 2 Dec 2024 11:00:37 -0500 Subject: [PATCH 07/44] chore(deps): Move from base-nossl to static (#15203) --- cmd/logcli/Dockerfile | 4 ++-- cmd/logql-analyzer/Dockerfile | 4 ++-- cmd/loki-canary/Dockerfile | 4 ++-- cmd/loki-canary/Dockerfile.cross | 4 ++-- cmd/loki/Dockerfile | 4 ++-- cmd/loki/Dockerfile.cross | 4 ++-- cmd/migrate/Dockerfile | 4 ++-- cmd/querytee/Dockerfile | 2 +- cmd/querytee/Dockerfile.cross | 4 ++-- production/helm/loki/src/helm-test/Dockerfile | 2 +- 10 files changed, 18 insertions(+), 18 deletions(-) diff --git a/cmd/logcli/Dockerfile b/cmd/logcli/Dockerfile index 52a66fea9a0cd..ddccaff2a7813 100644 --- a/cmd/logcli/Dockerfile +++ b/cmd/logcli/Dockerfile @@ -1,12 +1,12 @@ ARG GO_VERSION=1.23 -FROM golang:${GO_VERSION} as build +FROM golang:${GO_VERSION} AS build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false logcli -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/cmd/logcli/logcli /usr/bin/logcli SHELL [ "/busybox/sh", "-c" ] diff --git a/cmd/logql-analyzer/Dockerfile b/cmd/logql-analyzer/Dockerfile index 6cfb7ad795e4c..838aa9027778b 100644 --- a/cmd/logql-analyzer/Dockerfile +++ b/cmd/logql-analyzer/Dockerfile @@ -1,11 +1,11 @@ ARG GO_VERSION=1.23 -FROM golang:${GO_VERSION} as build +FROM golang:${GO_VERSION} AS build COPY . /src/loki WORKDIR /src/loki RUN make clean && CGO_ENABLED=0 go build ./cmd/logql-analyzer/ -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/logql-analyzer /usr/bin/logql-analyzer SHELL [ "/busybox/sh", "-c" ] diff --git a/cmd/loki-canary/Dockerfile b/cmd/loki-canary/Dockerfile index 2833d60590ca0..bfb97ba22bb80 100644 --- a/cmd/loki-canary/Dockerfile +++ b/cmd/loki-canary/Dockerfile @@ -1,11 +1,11 @@ ARG GO_VERSION=1.23 -FROM golang:${GO_VERSION} as build +FROM golang:${GO_VERSION} AS build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false loki-canary -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/cmd/loki-canary/loki-canary /usr/bin/loki-canary SHELL [ "/busybox/sh", "-c" ] diff --git a/cmd/loki-canary/Dockerfile.cross b/cmd/loki-canary/Dockerfile.cross index deaafce0ddbad..9471ea46924fd 100644 --- a/cmd/loki-canary/Dockerfile.cross +++ b/cmd/loki-canary/Dockerfile.cross @@ -3,7 +3,7 @@ ARG GO_VERSION=1.23 # Directories in this file are referenced from the root of the project not this folder # This file is intended to be called from the root like so: # docker build -t grafana/promtail -f cmd/promtail/Dockerfile . -FROM golang:${GO_VERSION} as goenv +FROM golang:${GO_VERSION} AS goenv RUN go env GOARCH > /goarch && \ go env GOARM > /goarm @@ -13,7 +13,7 @@ COPY . /src/loki WORKDIR /src/loki RUN make clean && GOARCH=$(cat /goarch) GOARM=$(cat /goarm) make BUILD_IN_CONTAINER=false loki-canary -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/cmd/loki-canary/loki-canary /usr/bin/loki-canary SHELL [ "/busybox/sh", "-c" ] RUN ln -s /busybox/sh /bin/sh diff --git a/cmd/loki/Dockerfile b/cmd/loki/Dockerfile index 521a897d645dc..4e7ea13081695 100644 --- a/cmd/loki/Dockerfile +++ b/cmd/loki/Dockerfile @@ -1,11 +1,11 @@ ARG GO_VERSION=1.23 -FROM golang:${GO_VERSION} as build +FROM golang:${GO_VERSION} AS build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false loki -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/cmd/loki/loki /usr/bin/loki COPY cmd/loki/loki-docker-config.yaml /etc/loki/local-config.yaml diff --git a/cmd/loki/Dockerfile.cross b/cmd/loki/Dockerfile.cross index 97fba445ef2f6..9562c6539ab18 100644 --- a/cmd/loki/Dockerfile.cross +++ b/cmd/loki/Dockerfile.cross @@ -2,7 +2,7 @@ ARG GO_VERSION=1.23 # Directories in this file are referenced from the root of the project not this folder # This file is intended to be called from the root like so: # docker build -t grafana/loki -f cmd/loki/Dockerfile . -FROM golang:${GO_VERSION} as goenv +FROM golang:${GO_VERSION} AS goenv RUN go env GOARCH > /goarch && \ go env GOARM > /goarm @@ -10,7 +10,7 @@ COPY . /src/loki WORKDIR /src/loki RUN make clean && GOARCH=$(cat /goarch) GOARM=$(cat /goarm) make BUILD_IN_CONTAINER=false loki -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=goenv /src/loki/cmd/loki/loki /usr/bin/loki COPY cmd/loki/loki-local-config.yaml /etc/loki/local-config.yaml diff --git a/cmd/migrate/Dockerfile b/cmd/migrate/Dockerfile index 82a78a4782d17..120602d9c4d66 100644 --- a/cmd/migrate/Dockerfile +++ b/cmd/migrate/Dockerfile @@ -1,10 +1,10 @@ ARG GO_VERSION=1.23 -FROM golang:${GO_VERSION} as build +FROM golang:${GO_VERSION} AS build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false migrate -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/cmd/migrate/migrate /usr/bin/migrate SHELL [ "/busybox/sh", "-c" ] diff --git a/cmd/querytee/Dockerfile b/cmd/querytee/Dockerfile index f2403d8df0e73..9fc23f8ce316b 100644 --- a/cmd/querytee/Dockerfile +++ b/cmd/querytee/Dockerfile @@ -5,7 +5,7 @@ COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false loki-querytee -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/cmd/querytee/querytee /usr/bin/querytee SHELL [ "/busybox/sh", "-c" ] diff --git a/cmd/querytee/Dockerfile.cross b/cmd/querytee/Dockerfile.cross index 83795cd3dc285..f514ba84941a8 100644 --- a/cmd/querytee/Dockerfile.cross +++ b/cmd/querytee/Dockerfile.cross @@ -3,7 +3,7 @@ ARG BUILD_IMAGE=grafana/loki-build-image:0.34.0 # This file is intended to be called from the root like so: # docker build -t grafana/promtail -f cmd/promtail/Dockerfile . ARG GO_VERSION=1.23 -FROM golang:${GO_VERSION} as goenv +FROM golang:${GO_VERSION} AS goenv RUN go env GOARCH > /goarch && \ go env GOARM > /goarm @@ -13,7 +13,7 @@ COPY . /src/loki WORKDIR /src/loki RUN make clean && GOARCH=$(cat /goarch) GOARM=$(cat /goarm) make BUILD_IN_CONTAINER=false loki-querytee -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/cmd/querytee/querytee /usr/bin/querytee SHELL [ "/busybox/sh", "-c" ] RUN ln -s /busybox/sh /bin/sh diff --git a/production/helm/loki/src/helm-test/Dockerfile b/production/helm/loki/src/helm-test/Dockerfile index 9645b206b1057..9378d53aa8ba5 100644 --- a/production/helm/loki/src/helm-test/Dockerfile +++ b/production/helm/loki/src/helm-test/Dockerfile @@ -8,6 +8,6 @@ COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false helm-test -FROM gcr.io/distroless/base-nossl:debug +FROM gcr.io/distroless/static:debug COPY --from=build /src/loki/production/helm/loki/src/helm-test/helm-test /usr/bin/helm-test ENTRYPOINT [ "/usr/bin/helm-test" ] From 4d9c5bb20aba82087d87836d88d2260f7dd913ac Mon Sep 17 00:00:00 2001 From: Callum Styan Date: Mon, 2 Dec 2024 10:39:31 -0800 Subject: [PATCH 08/44] chore: switch promtails base image from debian to ubuntu (#15195) Signed-off-by: Callum Styan --- Makefile | 1 + clients/cmd/promtail/Dockerfile | 11 ++++++----- clients/cmd/promtail/Dockerfile.arm32 | 4 ++-- clients/cmd/promtail/Dockerfile.cross | 4 ++-- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/Makefile b/Makefile index a9977fbbb102f..2c21110493827 100644 --- a/Makefile +++ b/Makefile @@ -865,6 +865,7 @@ trivy: loki-image build-image snyk: loki-image build-image snyk container test $(IMAGE_PREFIX)/loki:$(IMAGE_TAG) --file=cmd/loki/Dockerfile snyk container test $(IMAGE_PREFIX)/loki-build-image:$(IMAGE_TAG) --file=loki-build-image/Dockerfile + snyk container test $(IMAGE_PREFIX)/promtail:$(IMAGE_TAG) --file=clients/cmd/promtail/Dockerfile snyk code test .PHONY: scan-vulnerabilities diff --git a/clients/cmd/promtail/Dockerfile b/clients/cmd/promtail/Dockerfile index 1f1825f1a5563..3738e16569cf3 100644 --- a/clients/cmd/promtail/Dockerfile +++ b/clients/cmd/promtail/Dockerfile @@ -6,12 +6,13 @@ WORKDIR /src/loki RUN apt-get update && apt-get install -qy libsystemd-dev RUN make clean && make BUILD_IN_CONTAINER=false PROMTAIL_JOURNAL_ENABLED=true promtail -# Promtail requires debian as the base image to support systemd journal reading -FROM debian:12.8-slim +# Promtail requires debian or ubuntu as the base image to support systemd journal reading +FROM public.ecr.aws/ubuntu/ubuntu:noble # tzdata required for the timestamp stage to work -RUN apt-get update && \ - apt-get install -qy tzdata ca-certificates libsystemd-dev && \ - rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/* +# Install dependencies needed at runtime. +RUN apt-get update \ + && apt-get install -qy libsystemd-dev tzdata ca-certificates \ + && rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/* COPY --from=build /src/loki/clients/cmd/promtail/promtail /usr/bin/promtail COPY clients/cmd/promtail/promtail-docker-config.yaml /etc/promtail/config.yml ENTRYPOINT ["/usr/bin/promtail"] diff --git a/clients/cmd/promtail/Dockerfile.arm32 b/clients/cmd/promtail/Dockerfile.arm32 index c5d2127d4d54c..308cb53f648ee 100644 --- a/clients/cmd/promtail/Dockerfile.arm32 +++ b/clients/cmd/promtail/Dockerfile.arm32 @@ -5,8 +5,8 @@ WORKDIR /src/loki RUN apt-get update && apt-get install -qy libsystemd-dev RUN make clean && make BUILD_IN_CONTAINER=false PROMTAIL_JOURNAL_ENABLED=true promtail -# Promtail requires debian as the base image to support systemd journal reading -FROM debian:12.8-slim +# Promtail requires debian or ubuntu as the base image to support systemd journal reading +FROM public.ecr.aws/ubuntu/ubuntu:noble # tzdata required for the timestamp stage to work RUN apt-get update && \ apt-get install -qy tzdata ca-certificates wget libsystemd-dev && \ diff --git a/clients/cmd/promtail/Dockerfile.cross b/clients/cmd/promtail/Dockerfile.cross index 206bdd665d2e6..73a7dac649b60 100644 --- a/clients/cmd/promtail/Dockerfile.cross +++ b/clients/cmd/promtail/Dockerfile.cross @@ -13,8 +13,8 @@ COPY . /src/loki WORKDIR /src/loki RUN make clean && GOARCH=$(cat /goarch) GOARM=$(cat /goarm) make BUILD_IN_CONTAINER=false PROMTAIL_JOURNAL_ENABLED=true promtail -# Promtail requires debian as the base image to support systemd journal reading -FROM debian:12.8-slim +# Promtail requires debian or ubuntu as the base image to support systemd journal reading +FROM public.ecr.aws/ubuntu/ubuntu:noble # tzdata required for the timestamp stage to work RUN apt-get update && \ apt-get install -qy tzdata ca-certificates wget libsystemd-dev && \ From fa748270ff60b8257fb085e374b490c7623e946c Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 14:21:04 -0500 Subject: [PATCH 09/44] chore(deps): update prom/prometheus docker tag to v3.0.1 (#15183) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- production/docker/docker-compose.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/production/docker/docker-compose.yaml b/production/docker/docker-compose.yaml index a29e5efbf4db2..eb0d5a7b0c1ca 100644 --- a/production/docker/docker-compose.yaml +++ b/production/docker/docker-compose.yaml @@ -38,7 +38,7 @@ services: - loki prometheus: - image: prom/prometheus:v3.0.0 + image: prom/prometheus:v3.0.1 ports: - 9090 volumes: From 8e9ebf08a9ae339459a91a1a9abab83d3deb5e38 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 14:25:25 -0500 Subject: [PATCH 10/44] chore(deps): update fluent/fluentd docker tag to v1.18 (#15204) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- clients/cmd/fluentd/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/cmd/fluentd/Dockerfile b/clients/cmd/fluentd/Dockerfile index 77470966c10d7..b9bc8f4740fcc 100644 --- a/clients/cmd/fluentd/Dockerfile +++ b/clients/cmd/fluentd/Dockerfile @@ -9,7 +9,7 @@ COPY . /src/loki WORKDIR /src/loki RUN make BUILD_IN_CONTAINER=false fluentd-plugin -FROM fluent/fluentd:v1.17-debian-1 +FROM fluent/fluentd:v1.18-debian-1 ENV LOKI_URL="https://logs-prod-us-central1.grafana.net" COPY --from=build /src/loki/clients/cmd/fluentd/lib/fluent/plugin/out_loki.rb /fluentd/plugins/out_loki.rb From f59838981e7c5229411d5df783ac5920d3c255d6 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 14:25:47 -0500 Subject: [PATCH 11/44] chore(deps): update dependency fluentd to v1.18.0 (#15185) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- clients/cmd/fluentd/docker/Gemfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/cmd/fluentd/docker/Gemfile b/clients/cmd/fluentd/docker/Gemfile index d3033ffc3350d..428c18b840894 100644 --- a/clients/cmd/fluentd/docker/Gemfile +++ b/clients/cmd/fluentd/docker/Gemfile @@ -2,5 +2,5 @@ source 'https://rubygems.org' -gem 'fluentd', '1.17.1' +gem 'fluentd', '1.18.0' gem 'fluent-plugin-multi-format-parser', '~>1.1.0' From ad322c0fc22bbb99128001b81ebb384bd778066c Mon Sep 17 00:00:00 2001 From: Ashwanth Date: Tue, 3 Dec 2024 06:58:55 +0530 Subject: [PATCH 12/44] feat(block-scheduler): adds service and basic planner support for scheduler (#15200) --- docs/sources/shared/configuration.md | 20 +++ pkg/blockbuilder/scheduler/kafkautil.go | 80 +++++++++ pkg/blockbuilder/scheduler/kafkautil_test.go | 164 +++++++++++++++++++ pkg/blockbuilder/scheduler/metrics.go | 24 +++ pkg/blockbuilder/scheduler/offsets_reader.go | 62 +++++++ pkg/blockbuilder/scheduler/queue.go | 19 +++ pkg/blockbuilder/scheduler/scheduler.go | 149 +++++++++++++++-- pkg/blockbuilder/scheduler/scheduler_test.go | 7 +- pkg/blockbuilder/scheduler/strategy.go | 142 ++++++++++++++++ pkg/blockbuilder/scheduler/strategy_test.go | 159 ++++++++++++++++++ pkg/loki/loki.go | 9 + pkg/loki/modules.go | 19 +++ 12 files changed, 834 insertions(+), 20 deletions(-) create mode 100644 pkg/blockbuilder/scheduler/kafkautil.go create mode 100644 pkg/blockbuilder/scheduler/kafkautil_test.go create mode 100644 pkg/blockbuilder/scheduler/metrics.go create mode 100644 pkg/blockbuilder/scheduler/offsets_reader.go create mode 100644 pkg/blockbuilder/scheduler/strategy.go create mode 100644 pkg/blockbuilder/scheduler/strategy_test.go diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 6c8ed01c5c0c7..18c4cdceb649e 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -188,6 +188,26 @@ block_builder: # CLI flag: -blockbuilder.backoff..backoff-retries [max_retries: | default = 10] +block_scheduler: + # Consumer group used by block scheduler to track the last consumed offset. + # CLI flag: -block-scheduler.consumer-group + [consumer_group: | default = "block-scheduler"] + + # How often the scheduler should plan jobs. + # CLI flag: -block-scheduler.interval + [interval: | default = 5m] + + # Period used by the planner to calculate the start and end offset such that + # each job consumes records spanning the target period. + # CLI flag: -block-scheduler.target-records-spanning-period + [target_records_spanning_period: | default = 1h] + + # Lookback period in milliseconds used by the scheduler to plan jobs when the + # consumer group has no commits. -1 consumes from the latest offset. -2 + # consumes from the start of the partition. + # CLI flag: -block-scheduler.lookback-period + [lookback_period: | default = -2] + pattern_ingester: # Whether the pattern ingester is enabled. # CLI flag: -pattern-ingester.enabled diff --git a/pkg/blockbuilder/scheduler/kafkautil.go b/pkg/blockbuilder/scheduler/kafkautil.go new file mode 100644 index 0000000000000..f746f2a9fd4e0 --- /dev/null +++ b/pkg/blockbuilder/scheduler/kafkautil.go @@ -0,0 +1,80 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package scheduler + +import ( + "context" + "errors" + "fmt" + "sync" + + "github.com/twmb/franz-go/pkg/kadm" + "github.com/twmb/franz-go/pkg/kerr" +) + +// GetGroupLag is similar to `kadm.Client.Lag` but works when the group doesn't have live participants. +// Similar to `kadm.CalculateGroupLagWithStartOffsets`, it takes into account that the group may not have any commits. +// +// The lag is the difference between the last produced offset (high watermark) and an offset in the "past". +// If the block builder committed an offset for a given partition to the consumer group at least once, then +// the lag is the difference between the last produced offset and the offset committed in the consumer group. +// Otherwise, if the block builder didn't commit an offset for a given partition yet (e.g. block builder is +// running for the first time), then the lag is the difference between the last produced offset and fallbackOffsetMillis. +func GetGroupLag(ctx context.Context, admClient *kadm.Client, topic, group string, fallbackOffsetMillis int64) (kadm.GroupLag, error) { + offsets, err := admClient.FetchOffsets(ctx, group) + if err != nil { + if !errors.Is(err, kerr.GroupIDNotFound) { + return nil, fmt.Errorf("fetch offsets: %w", err) + } + } + if err := offsets.Error(); err != nil { + return nil, fmt.Errorf("fetch offsets got error in response: %w", err) + } + + startOffsets, err := admClient.ListStartOffsets(ctx, topic) + if err != nil { + return nil, err + } + endOffsets, err := admClient.ListEndOffsets(ctx, topic) + if err != nil { + return nil, err + } + + resolveFallbackOffsets := sync.OnceValues(func() (kadm.ListedOffsets, error) { + return admClient.ListOffsetsAfterMilli(ctx, fallbackOffsetMillis, topic) + }) + // If the group-partition in offsets doesn't have a commit, fall back depending on where fallbackOffsetMillis points at. + for topic, pt := range startOffsets.Offsets() { + for partition, startOffset := range pt { + if _, ok := offsets.Lookup(topic, partition); ok { + continue + } + fallbackOffsets, err := resolveFallbackOffsets() + if err != nil { + return nil, fmt.Errorf("resolve fallback offsets: %w", err) + } + o, ok := fallbackOffsets.Lookup(topic, partition) + if !ok { + return nil, fmt.Errorf("partition %d not found in fallback offsets for topic %s", partition, topic) + } + if o.Offset < startOffset.At { + // Skip the resolved fallback offset if it's before the partition's start offset (i.e. before the earliest offset of the partition). + // This should not happen in Kafka, but can happen in Kafka-compatible systems, e.g. Warpstream. + continue + } + offsets.Add(kadm.OffsetResponse{Offset: kadm.Offset{ + Topic: o.Topic, + Partition: o.Partition, + At: o.Offset, + LeaderEpoch: o.LeaderEpoch, + }}) + } + } + + descrGroup := kadm.DescribedGroup{ + // "Empty" is the state that indicates that the group doesn't have active consumer members; this is always the case for block-builder, + // because we don't use group consumption. + State: "Empty", + } + return kadm.CalculateGroupLagWithStartOffsets(descrGroup, offsets, startOffsets, endOffsets), nil +} diff --git a/pkg/blockbuilder/scheduler/kafkautil_test.go b/pkg/blockbuilder/scheduler/kafkautil_test.go new file mode 100644 index 0000000000000..d2a865702a808 --- /dev/null +++ b/pkg/blockbuilder/scheduler/kafkautil_test.go @@ -0,0 +1,164 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package scheduler + +import ( + "context" + "errors" + "testing" + "time" + + "github.com/stretchr/testify/require" + "github.com/twmb/franz-go/pkg/kadm" + "github.com/twmb/franz-go/pkg/kgo" + + "github.com/grafana/loki/v3/pkg/kafka/testkafka" +) + +const ( + testTopic = "test" + testGroup = "testgroup" +) + +func TestKafkaGetGroupLag(t *testing.T) { + ctx, cancel := context.WithCancelCause(context.Background()) + t.Cleanup(func() { cancel(errors.New("test done")) }) + + _, addr := testkafka.CreateClusterWithoutCustomConsumerGroupsSupport(t, 3, testTopic) + kafkaClient := mustKafkaClient(t, addr) + admClient := kadm.NewClient(kafkaClient) + + const numRecords = 5 + + var producedRecords []kgo.Record + kafkaTime := time.Now().Add(-12 * time.Hour) + for i := int64(0); i < numRecords; i++ { + kafkaTime = kafkaTime.Add(time.Minute) + + // Produce and keep records to partition 0. + res := produceRecords(ctx, t, kafkaClient, kafkaTime, "1", testTopic, 0, []byte(`test value`)) + rec, err := res.First() + require.NoError(t, err) + require.NotNil(t, rec) + + producedRecords = append(producedRecords, *rec) + + // Produce same records to partition 1 (this partition won't have any commits). + produceRecords(ctx, t, kafkaClient, kafkaTime, "1", testTopic, 1, []byte(`test value`)) + } + require.Len(t, producedRecords, numRecords) + + // Commit last produced record from partition 0. + rec := producedRecords[len(producedRecords)-1] + offsets := make(kadm.Offsets) + offsets.Add(kadm.Offset{ + Topic: rec.Topic, + Partition: rec.Partition, + At: rec.Offset + 1, + LeaderEpoch: rec.LeaderEpoch, + }) + err := admClient.CommitAllOffsets(ctx, testGroup, offsets) + require.NoError(t, err) + + // Truncate partition 1 after second to last record to emulate the retention + // Note Kafka sets partition's start offset to the requested offset. Any records within the segment before the requested offset can no longer be read. + // Note the difference between DeleteRecords and DeleteOffsets in kadm docs. + deleteRecOffsets := make(kadm.Offsets) + deleteRecOffsets.Add(kadm.Offset{ + Topic: testTopic, + Partition: 1, + At: numRecords - 2, + }) + _, err = admClient.DeleteRecords(ctx, deleteRecOffsets) + require.NoError(t, err) + + getTopicPartitionLag := func(t *testing.T, lag kadm.GroupLag, topic string, part int32) int64 { + l, ok := lag.Lookup(topic, part) + require.True(t, ok) + return l.Lag + } + + t.Run("fallbackOffset=milliseconds", func(t *testing.T) { + // get the timestamp of the last produced record + rec := producedRecords[len(producedRecords)-1] + fallbackOffset := rec.Timestamp.Add(-time.Millisecond).UnixMilli() + groupLag, err := GetGroupLag(ctx, admClient, testTopic, testGroup, fallbackOffset) + require.NoError(t, err) + + require.EqualValues(t, 0, getTopicPartitionLag(t, groupLag, testTopic, 0), "partition 0 must have no lag") + require.EqualValues(t, 1, getTopicPartitionLag(t, groupLag, testTopic, 1), "partition 1 must fall back to known record and get its lag from there") + require.EqualValues(t, 0, getTopicPartitionLag(t, groupLag, testTopic, 2), "partition 2 has no data and must have no lag") + }) + + t.Run("fallbackOffset=before-earliest", func(t *testing.T) { + // get the timestamp of third to last produced record (record before earliest in partition 1) + rec := producedRecords[len(producedRecords)-3] + fallbackOffset := rec.Timestamp.Add(-time.Millisecond).UnixMilli() + groupLag, err := GetGroupLag(ctx, admClient, testTopic, testGroup, fallbackOffset) + require.NoError(t, err) + + require.EqualValues(t, 0, getTopicPartitionLag(t, groupLag, testTopic, 0), "partition 0 must have no lag") + require.EqualValues(t, 2, getTopicPartitionLag(t, groupLag, testTopic, 1), "partition 1 must fall back to earliest and get its lag from there") + require.EqualValues(t, 0, getTopicPartitionLag(t, groupLag, testTopic, 2), "partition 2 has no data and must have no lag") + }) + + t.Run("fallbackOffset=0", func(t *testing.T) { + groupLag, err := GetGroupLag(ctx, admClient, testTopic, testGroup, 0) + require.NoError(t, err) + + require.EqualValues(t, 0, getTopicPartitionLag(t, groupLag, testTopic, 0), "partition 0 must have no lag") + require.EqualValues(t, 2, getTopicPartitionLag(t, groupLag, testTopic, 1), "partition 1 must fall back to the earliest and get its lag from there") + require.EqualValues(t, 0, getTopicPartitionLag(t, groupLag, testTopic, 2), "partition 2 has no data and must have no lag") + }) + + t.Run("group=unknown", func(t *testing.T) { + groupLag, err := GetGroupLag(ctx, admClient, testTopic, "unknown", 0) + require.NoError(t, err) + + // This group doesn't have any commits, so it must calc its lag from the fallback. + require.EqualValues(t, numRecords, getTopicPartitionLag(t, groupLag, testTopic, 0)) + require.EqualValues(t, 2, getTopicPartitionLag(t, groupLag, testTopic, 1), "partition 1 must fall back to the earliest and get its lag from there") + require.EqualValues(t, 0, getTopicPartitionLag(t, groupLag, testTopic, 2), "partition 2 has no data and must have no lag") + }) +} + +func mustKafkaClient(t *testing.T, addrs ...string) *kgo.Client { + writeClient, err := kgo.NewClient( + kgo.SeedBrokers(addrs...), + kgo.AllowAutoTopicCreation(), + // We will choose the partition of each record. + kgo.RecordPartitioner(kgo.ManualPartitioner()), + ) + require.NoError(t, err) + t.Cleanup(writeClient.Close) + return writeClient +} + +func produceRecords( + ctx context.Context, + t *testing.T, + kafkaClient *kgo.Client, + ts time.Time, + userID string, + topic string, + part int32, + val []byte, +) kgo.ProduceResults { + rec := &kgo.Record{ + Timestamp: ts, + Key: []byte(userID), + Value: val, + Topic: topic, + Partition: part, // samples in this batch are split between N partitions + } + produceResult := kafkaClient.ProduceSync(ctx, rec) + require.NoError(t, produceResult.FirstErr()) + return produceResult +} + +func commitOffset(ctx context.Context, t *testing.T, kafkaClient *kgo.Client, group string, offset kadm.Offset) { + offsets := make(kadm.Offsets) + offsets.Add(offset) + err := kadm.NewClient(kafkaClient).CommitAllOffsets(ctx, group, offsets) + require.NoError(t, err) +} diff --git a/pkg/blockbuilder/scheduler/metrics.go b/pkg/blockbuilder/scheduler/metrics.go new file mode 100644 index 0000000000000..4e1dbfa2afa1c --- /dev/null +++ b/pkg/blockbuilder/scheduler/metrics.go @@ -0,0 +1,24 @@ +package scheduler + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +type Metrics struct { + lag *prometheus.GaugeVec + committedOffset *prometheus.GaugeVec +} + +func NewMetrics(reg prometheus.Registerer) *Metrics { + return &Metrics{ + lag: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "loki_block_scheduler_group_lag", + Help: "How far behind the block scheduler consumer group is from the latest offset.", + }, []string{"partition"}), + committedOffset: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "loki_block_scheduler_group_committed_offset", + Help: "The current offset the block scheduler consumer group is at.", + }, []string{"partition"}), + } +} diff --git a/pkg/blockbuilder/scheduler/offsets_reader.go b/pkg/blockbuilder/scheduler/offsets_reader.go new file mode 100644 index 0000000000000..742185dba817f --- /dev/null +++ b/pkg/blockbuilder/scheduler/offsets_reader.go @@ -0,0 +1,62 @@ +package scheduler + +import ( + "context" + "errors" + "time" + + "github.com/twmb/franz-go/pkg/kadm" + "github.com/twmb/franz-go/pkg/kgo" +) + +type offsetReader struct { + topic string + consumerGroup string + fallbackOffsetMillis int64 + + adminClient *kadm.Client +} + +func NewOffsetReader(topic, consumerGroup string, lookbackPeriodInMs int64, client *kgo.Client) OffsetReader { + var fallbackOffsetMillis int64 + if lookbackPeriodInMs >= 0 { + fallbackOffsetMillis = time.Now().UnixMilli() - lookbackPeriodInMs + } else { + fallbackOffsetMillis = lookbackPeriodInMs + } + + return &offsetReader{ + topic: topic, + consumerGroup: consumerGroup, + adminClient: kadm.NewClient(client), + fallbackOffsetMillis: fallbackOffsetMillis, + } +} + +func (r *offsetReader) GroupLag(ctx context.Context) (map[int32]kadm.GroupMemberLag, error) { + lag, err := GetGroupLag(ctx, r.adminClient, r.topic, r.consumerGroup, r.fallbackOffsetMillis) + if err != nil { + return nil, err + } + + offsets, ok := lag[r.topic] + if !ok { + return nil, errors.New("no lag found for the topic") + } + + return offsets, nil +} + +func (r *offsetReader) ListOffsetsAfterMilli(ctx context.Context, ts int64) (map[int32]kadm.ListedOffset, error) { + offsets, err := r.adminClient.ListOffsetsAfterMilli(ctx, ts, r.topic) + if err != nil { + return nil, err + } + + resp, ok := offsets[r.topic] + if !ok { + return nil, errors.New("no offsets found for the topic") + } + + return resp, nil +} diff --git a/pkg/blockbuilder/scheduler/queue.go b/pkg/blockbuilder/scheduler/queue.go index 3e9cf087c6792..e2f125ad70a07 100644 --- a/pkg/blockbuilder/scheduler/queue.go +++ b/pkg/blockbuilder/scheduler/queue.go @@ -30,6 +30,25 @@ func NewJobQueue() *JobQueue { } } +func (q *JobQueue) Exists(job *types.Job) (types.JobStatus, bool) { + q.mu.RLock() + defer q.mu.RUnlock() + + if _, ok := q.inProgress[job.ID]; ok { + return types.JobStatusInProgress, true + } + + if _, ok := q.pending[job.ID]; ok { + return types.JobStatusPending, true + } + + if _, ok := q.completed[job.ID]; ok { + return types.JobStatusComplete, true + } + + return -1, false +} + // Enqueue adds a new job to the pending queue // This is a naive implementation, intended to be refactored func (q *JobQueue) Enqueue(job *types.Job) error { diff --git a/pkg/blockbuilder/scheduler/scheduler.go b/pkg/blockbuilder/scheduler/scheduler.go index 274713b5b1c36..dbf732742de39 100644 --- a/pkg/blockbuilder/scheduler/scheduler.go +++ b/pkg/blockbuilder/scheduler/scheduler.go @@ -2,43 +2,140 @@ package scheduler import ( "context" + "errors" + "flag" + "strconv" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + "github.com/twmb/franz-go/pkg/kadm" "github.com/grafana/loki/v3/pkg/blockbuilder/types" ) var ( _ types.Scheduler = unimplementedScheduler{} - _ types.Scheduler = &QueueScheduler{} + _ types.Scheduler = &BlockScheduler{} ) -// unimplementedScheduler provides default implementations that panic. -type unimplementedScheduler struct{} +type Config struct { + ConsumerGroup string `yaml:"consumer_group"` + Interval time.Duration `yaml:"interval"` + TargetRecordConsumptionPeriod time.Duration `yaml:"target_records_spanning_period"` + LookbackPeriod int64 `yaml:"lookback_period"` +} -func (s unimplementedScheduler) HandleGetJob(_ context.Context, _ string) (*types.Job, bool, error) { - panic("unimplemented") +func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + f.DurationVar(&cfg.Interval, prefix+"interval", 5*time.Minute, "How often the scheduler should plan jobs.") + f.DurationVar(&cfg.TargetRecordConsumptionPeriod, prefix+"target-records-spanning-period", time.Hour, "Period used by the planner to calculate the start and end offset such that each job consumes records spanning the target period.") + f.StringVar(&cfg.ConsumerGroup, prefix+"consumer-group", "block-scheduler", "Consumer group used by block scheduler to track the last consumed offset.") + f.Int64Var(&cfg.LookbackPeriod, prefix+"lookback-period", -2, "Lookback period in milliseconds used by the scheduler to plan jobs when the consumer group has no commits. -1 consumes from the latest offset. -2 consumes from the start of the partition.") } -func (s unimplementedScheduler) HandleCompleteJob(_ context.Context, _ string, _ *types.Job) error { - panic("unimplemented") +func (cfg *Config) RegisterFlags(f *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix("block-scheduler.", f) } -func (s unimplementedScheduler) HandleSyncJob(_ context.Context, _ string, _ *types.Job) error { - panic("unimplemented") +func (cfg *Config) Validate() error { + if cfg.Interval <= 0 { + return errors.New("interval must be a non-zero value") + } + + if cfg.LookbackPeriod < -2 { + return errors.New("only -1(latest) and -2(earliest) are valid as negative values for lookback_period") + } + + return nil } -// QueueScheduler implements the Scheduler interface -type QueueScheduler struct { - queue *JobQueue +// BlockScheduler implements the Scheduler interface +type BlockScheduler struct { + services.Service + + cfg Config + logger log.Logger + queue *JobQueue + metrics *Metrics + + offsetReader OffsetReader + planner Planner } // NewScheduler creates a new scheduler instance -func NewScheduler(queue *JobQueue) *QueueScheduler { - return &QueueScheduler{ - queue: queue, +func NewScheduler(cfg Config, queue *JobQueue, offsetReader OffsetReader, logger log.Logger, r prometheus.Registerer) *BlockScheduler { + planner := NewTimeRangePlanner(cfg.TargetRecordConsumptionPeriod, offsetReader, func() time.Time { return time.Now().UTC() }, logger) + s := &BlockScheduler{ + cfg: cfg, + planner: planner, + offsetReader: offsetReader, + logger: logger, + metrics: NewMetrics(r), + queue: queue, + } + s.Service = services.NewBasicService(nil, s.running, nil) + return s +} + +func (s *BlockScheduler) running(ctx context.Context) error { + if err := s.runOnce(ctx); err != nil { + level.Error(s.logger).Log("msg", "failed to schedule jobs", "err", err) + } + + ticker := time.NewTicker(s.cfg.Interval) + for { + select { + case <-ticker.C: + if err := s.runOnce(ctx); err != nil { + // TODO: add metrics + level.Error(s.logger).Log("msg", "failed to schedule jobs", "err", err) + } + case <-ctx.Done(): + return nil + } } } -func (s *QueueScheduler) HandleGetJob(ctx context.Context, builderID string) (*types.Job, bool, error) { +func (s *BlockScheduler) runOnce(ctx context.Context) error { + lag, err := s.offsetReader.GroupLag(ctx) + if err != nil { + level.Error(s.logger).Log("msg", "failed to get group lag", "err", err) + return err + } + + s.publishLagMetrics(lag) + + jobs, err := s.planner.Plan(ctx) + if err != nil { + level.Error(s.logger).Log("msg", "failed to plan jobs", "err", err) + } + + for _, job := range jobs { + // TODO: end offset keeps moving each time we plan jobs, maybe we should not use it as part of the job ID + if status, ok := s.queue.Exists(&job); ok { + level.Debug(s.logger).Log("msg", "job already exists", "job", job, "status", status) + continue + } + + if err := s.queue.Enqueue(&job); err != nil { + level.Error(s.logger).Log("msg", "failed to enqueue job", "job", job, "err", err) + } + } + + return nil +} + +func (s *BlockScheduler) publishLagMetrics(lag map[int32]kadm.GroupMemberLag) { + for partition, offsets := range lag { + // useful for scaling builders + s.metrics.lag.WithLabelValues(strconv.Itoa(int(partition))).Set(float64(offsets.Lag)) + s.metrics.committedOffset.WithLabelValues(strconv.Itoa(int(partition))).Set(float64(offsets.Commit.At)) + } +} + +func (s *BlockScheduler) HandleGetJob(ctx context.Context, builderID string) (*types.Job, bool, error) { select { case <-ctx.Done(): return nil, false, ctx.Err() @@ -47,10 +144,26 @@ func (s *QueueScheduler) HandleGetJob(ctx context.Context, builderID string) (*t } } -func (s *QueueScheduler) HandleCompleteJob(_ context.Context, builderID string, job *types.Job) error { +func (s *BlockScheduler) HandleCompleteJob(_ context.Context, builderID string, job *types.Job) error { + // TODO: handle commits return s.queue.MarkComplete(job.ID, builderID) } -func (s *QueueScheduler) HandleSyncJob(_ context.Context, builderID string, job *types.Job) error { +func (s *BlockScheduler) HandleSyncJob(_ context.Context, builderID string, job *types.Job) error { return s.queue.SyncJob(job.ID, builderID, job) } + +// unimplementedScheduler provides default implementations that panic. +type unimplementedScheduler struct{} + +func (s unimplementedScheduler) HandleGetJob(_ context.Context, _ string) (*types.Job, bool, error) { + panic("unimplemented") +} + +func (s unimplementedScheduler) HandleCompleteJob(_ context.Context, _ string, _ *types.Job) error { + panic("unimplemented") +} + +func (s unimplementedScheduler) HandleSyncJob(_ context.Context, _ string, _ *types.Job) error { + panic("unimplemented") +} diff --git a/pkg/blockbuilder/scheduler/scheduler_test.go b/pkg/blockbuilder/scheduler/scheduler_test.go index ad6829bc8fe69..bd9e00450dfa7 100644 --- a/pkg/blockbuilder/scheduler/scheduler_test.go +++ b/pkg/blockbuilder/scheduler/scheduler_test.go @@ -5,20 +5,23 @@ import ( "testing" "time" + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/grafana/loki/v3/pkg/blockbuilder/builder" "github.com/grafana/loki/v3/pkg/blockbuilder/types" ) type testEnv struct { queue *JobQueue - scheduler *QueueScheduler + scheduler *BlockScheduler transport *builder.MemoryTransport builder *builder.Worker } func newTestEnv(builderID string) *testEnv { queue := NewJobQueue() - scheduler := NewScheduler(queue) + scheduler := NewScheduler(Config{}, queue, nil, log.NewNopLogger(), prometheus.NewRegistry()) transport := builder.NewMemoryTransport(scheduler) builder := builder.NewWorker(builderID, builder.NewMemoryTransport(scheduler)) diff --git a/pkg/blockbuilder/scheduler/strategy.go b/pkg/blockbuilder/scheduler/strategy.go new file mode 100644 index 0000000000000..5ea1fb6db2d9c --- /dev/null +++ b/pkg/blockbuilder/scheduler/strategy.go @@ -0,0 +1,142 @@ +package scheduler + +import ( + "context" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/twmb/franz-go/pkg/kadm" + + "github.com/grafana/loki/v3/pkg/blockbuilder/types" +) + +// OffsetReader is an interface to list offsets for all partitions of a topic from Kafka. +type OffsetReader interface { + ListOffsetsAfterMilli(context.Context, int64) (map[int32]kadm.ListedOffset, error) + GroupLag(context.Context) (map[int32]kadm.GroupMemberLag, error) +} + +type Planner interface { + Name() string + Plan(ctx context.Context) ([]types.Job, error) +} + +const ( + RecordCountStrategy = "record_count" + TimeRangeStrategy = "time_range" +) + +// tries to consume upto targetRecordCount records per partition +type RecordCountPlanner struct { + targetRecordCount int64 + offsetReader OffsetReader + logger log.Logger +} + +func NewRecordCountPlanner(targetRecordCount int64) *RecordCountPlanner { + return &RecordCountPlanner{ + targetRecordCount: targetRecordCount, + } +} + +func (p *RecordCountPlanner) Name() string { + return RecordCountStrategy +} + +func (p *RecordCountPlanner) Plan(ctx context.Context) ([]types.Job, error) { + offsets, err := p.offsetReader.GroupLag(ctx) + if err != nil { + level.Error(p.logger).Log("msg", "failed to get group lag", "err", err) + return nil, err + } + + jobs := make([]types.Job, 0, len(offsets)) + for _, partition := range offsets { + // kadm.GroupMemberLag contains valid Commit.At even when consumer group never committed any offset. + // no additional validation is needed here + startOffset := partition.Commit.At + 1 + endOffset := min(startOffset+p.targetRecordCount, partition.End.Offset) + + job := types.Job{ + Partition: int(partition.Partition), + Offsets: types.Offsets{ + Min: startOffset, + Max: endOffset, + }, + } + + jobs = append(jobs, job) + } + + return jobs, nil +} + +// Targets consuming records spanning a configured period. +// This is a stateless planner, it is upto the caller to deduplicate or update jobs that are already in queue or progress. +type TimeRangePlanner struct { + offsetReader OffsetReader + + buffer time.Duration + targetPeriod time.Duration + now func() time.Time + + logger log.Logger +} + +func NewTimeRangePlanner(interval time.Duration, offsetReader OffsetReader, now func() time.Time, logger log.Logger) *TimeRangePlanner { + return &TimeRangePlanner{ + targetPeriod: interval, + buffer: interval, + offsetReader: offsetReader, + now: now, + logger: logger, + } +} + +func (p *TimeRangePlanner) Name() string { + return TimeRangeStrategy +} + +func (p *TimeRangePlanner) Plan(ctx context.Context) ([]types.Job, error) { + // truncate to the nearest Interval + consumeUptoTS := p.now().Add(-p.buffer).Truncate(p.targetPeriod) + + // this will return the latest offset in the partition if no records are produced after this ts. + consumeUptoOffsets, err := p.offsetReader.ListOffsetsAfterMilli(ctx, consumeUptoTS.UnixMilli()) + if err != nil { + level.Error(p.logger).Log("msg", "failed to list offsets after timestamp", "err", err) + return nil, err + } + + offsets, err := p.offsetReader.GroupLag(ctx) + if err != nil { + level.Error(p.logger).Log("msg", "failed to get group lag", "err", err) + return nil, err + } + + var jobs []types.Job + for _, partitionOffset := range offsets { + startOffset := partitionOffset.Commit.At + 1 + // TODO: we could further break down the work into Interval sized chunks if this partition has pending records spanning a long time range + // or have the builder consume in chunks and commit the job status back to scheduler. + endOffset := consumeUptoOffsets[partitionOffset.Partition].Offset + + if startOffset >= endOffset { + level.Info(p.logger).Log("msg", "no pending records to process", "partition", partitionOffset.Partition, + "commitOffset", partitionOffset.Commit.At, + "consumeUptoOffset", consumeUptoOffsets[partitionOffset.Partition].Offset) + continue + } + + jobs = append(jobs, types.Job{ + Partition: int(partitionOffset.Partition), + Offsets: types.Offsets{ + Min: startOffset, + Max: endOffset, + }, + }) + } + + return jobs, nil +} diff --git a/pkg/blockbuilder/scheduler/strategy_test.go b/pkg/blockbuilder/scheduler/strategy_test.go new file mode 100644 index 0000000000000..eb4704f268c74 --- /dev/null +++ b/pkg/blockbuilder/scheduler/strategy_test.go @@ -0,0 +1,159 @@ +package scheduler + +import ( + "context" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/stretchr/testify/require" + "github.com/twmb/franz-go/pkg/kadm" + + "github.com/grafana/loki/v3/pkg/blockbuilder/types" +) + +func TestTimeRangePlanner_Plan(t *testing.T) { + interval := 15 * time.Minute + for _, tc := range []struct { + name string + now time.Time + expectedJobs []types.Job + groupLag map[int32]kadm.GroupMemberLag + consumeUpto map[int32]kadm.ListedOffset + }{ + { + // Interval 1 + // now: 00:42:00. consume until 00:15:00 + // last consumed offset 100 with record ts: 00:10:00 + // record offset with ts after 00:15:00 - offset 200 + // resulting jobs: [100, 200] + name: "normal case. schedule first interval", + now: time.Date(0, 0, 0, 0, 42, 0, 0, time.UTC), // 00:42:00 + groupLag: map[int32]kadm.GroupMemberLag{ + 0: { + Commit: kadm.Offset{ + At: 100, + }, + Partition: 0, + }, + }, + consumeUpto: map[int32]kadm.ListedOffset{ + 0: { + Offset: 200, + }, + }, + expectedJobs: []types.Job{ + { + Partition: 0, + Offsets: types.Offsets{Min: 101, Max: 200}, + }, + }, + }, + { + // Interval 2 + // now: 00:46:00. consume until 00:30:00 + // last consumed offset 199 with record ts: 00:11:00 + // record offset with ts after 00:30:00 - offset 300 + // resulting jobs: [200, 300] + name: "normal case. schedule second interval", + now: time.Date(0, 0, 0, 0, 46, 0, 0, time.UTC), // 00:46:00 + groupLag: map[int32]kadm.GroupMemberLag{ + 0: { + Commit: kadm.Offset{ + At: 199, + }, + Partition: 0, + }, + 1: { + Commit: kadm.Offset{ + At: 11, + }, + Partition: 1, + }, + }, + consumeUpto: map[int32]kadm.ListedOffset{ + 0: { + Offset: 300, + }, + 1: { + Offset: 123, + }, + }, + expectedJobs: []types.Job{ + { + Partition: 0, + Offsets: types.Offsets{Min: 200, Max: 300}, + }, + { + Partition: 1, + Offsets: types.Offsets{Min: 12, Max: 123}, + }, + }, + }, + { + // Interval 2 - run scheduling again + // now: 00:48:00. consume until 00:30:00 + // last consumed offset 299 + // record offset with ts after 00:30:00 - offset 300 + // no jobs to schedule for partition 0 + name: "no pending records to consume. schedule second interval once more time", + now: time.Date(0, 0, 0, 0, 48, 0, 0, time.UTC), // 00:48:00 + groupLag: map[int32]kadm.GroupMemberLag{ + 0: { + Commit: kadm.Offset{ + At: 299, + }, + Partition: 0, + }, + 1: { + Commit: kadm.Offset{ + At: 11, + }, + Partition: 1, + }, + }, + consumeUpto: map[int32]kadm.ListedOffset{ + 0: { + Offset: 300, + }, + // still pending. assume no builder were assigned + 1: { + Offset: 123, + }, + }, + expectedJobs: []types.Job{ + { + Partition: 1, + Offsets: types.Offsets{Min: 12, Max: 123}, + }, + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + mockOffsetReader := &mockOffsetReader{ + offsetsAfterMilli: tc.consumeUpto, + groupLag: tc.groupLag, + } + planner := NewTimeRangePlanner(interval, mockOffsetReader, func() time.Time { return tc.now }, log.NewNopLogger()) + + jobs, err := planner.Plan(context.Background()) + require.NoError(t, err) + + require.Equal(t, len(tc.expectedJobs), len(jobs)) + require.Equal(t, tc.expectedJobs, jobs) + }) + } +} + +type mockOffsetReader struct { + offsetsAfterMilli map[int32]kadm.ListedOffset + groupLag map[int32]kadm.GroupMemberLag +} + +func (m *mockOffsetReader) ListOffsetsAfterMilli(_ context.Context, _ int64) (map[int32]kadm.ListedOffset, error) { + return m.offsetsAfterMilli, nil +} + +func (m *mockOffsetReader) GroupLag(_ context.Context) (map[int32]kadm.GroupMemberLag, error) { + return m.groupLag, nil +} diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index 153387035d6b5..9747a8f231f7e 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -32,6 +32,7 @@ import ( "github.com/grafana/loki/v3/pkg/analytics" blockbuilder "github.com/grafana/loki/v3/pkg/blockbuilder/builder" + blockscheduler "github.com/grafana/loki/v3/pkg/blockbuilder/scheduler" "github.com/grafana/loki/v3/pkg/bloombuild" "github.com/grafana/loki/v3/pkg/bloomgateway" "github.com/grafana/loki/v3/pkg/compactor" @@ -91,6 +92,7 @@ type Config struct { IngesterClient ingester_client.Config `yaml:"ingester_client,omitempty"` Ingester ingester.Config `yaml:"ingester,omitempty"` BlockBuilder blockbuilder.Config `yaml:"block_builder,omitempty"` + BlockScheduler blockscheduler.Config `yaml:"block_scheduler,omitempty"` Pattern pattern.Config `yaml:"pattern_ingester,omitempty"` IndexGateway indexgateway.Config `yaml:"index_gateway"` BloomBuild bloombuild.Config `yaml:"bloom_build,omitempty" category:"experimental"` @@ -186,6 +188,7 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.Profiling.RegisterFlags(f) c.KafkaConfig.RegisterFlags(f) c.BlockBuilder.RegisterFlags(f) + c.BlockScheduler.RegisterFlags(f) } func (c *Config) registerServerFlagsWithChangedDefaultValues(fs *flag.FlagSet) { @@ -264,6 +267,9 @@ func (c *Config) Validate() error { if err := c.BlockBuilder.Validate(); err != nil { errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid block_builder config")) } + if err := c.BlockScheduler.Validate(); err != nil { + errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid block_scheduler config")) + } if err := c.LimitsConfig.Validate(); err != nil { errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid limits_config config")) } @@ -379,6 +385,7 @@ type Loki struct { partitionRingWatcher *ring.PartitionRingWatcher partitionRing *ring.PartitionInstanceRing blockBuilder *blockbuilder.BlockBuilder + blockScheduler *blockscheduler.BlockScheduler ClientMetrics storage.ClientMetrics deleteClientMetrics *deletion.DeleteRequestClientMetrics @@ -690,6 +697,7 @@ func (t *Loki) setupModuleManager() error { mm.RegisterModule(PatternIngester, t.initPatternIngester) mm.RegisterModule(PartitionRing, t.initPartitionRing, modules.UserInvisibleModule) mm.RegisterModule(BlockBuilder, t.initBlockBuilder) + mm.RegisterModule(BlockScheduler, t.initBlockScheduler) mm.RegisterModule(All, nil) mm.RegisterModule(Read, nil) @@ -728,6 +736,7 @@ func (t *Loki) setupModuleManager() error { PartitionRing: {MemberlistKV, Server, Ring}, MemberlistKV: {Server}, BlockBuilder: {PartitionRing, Store, Server}, + BlockScheduler: {Server}, Read: {QueryFrontend, Querier}, Write: {Ingester, Distributor, PatternIngester}, diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 994576076af3e..c4449f3c51134 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -37,6 +37,7 @@ import ( "github.com/grafana/loki/v3/pkg/analytics" blockbuilder "github.com/grafana/loki/v3/pkg/blockbuilder/builder" + blockscheduler "github.com/grafana/loki/v3/pkg/blockbuilder/scheduler" "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" @@ -49,6 +50,7 @@ import ( "github.com/grafana/loki/v3/pkg/distributor" "github.com/grafana/loki/v3/pkg/indexgateway" "github.com/grafana/loki/v3/pkg/ingester" + kclient "github.com/grafana/loki/v3/pkg/kafka/client" "github.com/grafana/loki/v3/pkg/kafka/partition" "github.com/grafana/loki/v3/pkg/kafka/partitionring" "github.com/grafana/loki/v3/pkg/logproto" @@ -139,6 +141,7 @@ const ( InitCodec string = "init-codec" PartitionRing string = "partition-ring" BlockBuilder string = "block-builder" + BlockScheduler string = "block-scheduler" ) const ( @@ -1863,6 +1866,22 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { return t.blockBuilder, nil } +func (t *Loki) initBlockScheduler() (services.Service, error) { + logger := log.With(util_log.Logger, "component", "block_scheduler") + + clientMetrics := kclient.NewReaderClientMetrics("block-scheduler", prometheus.DefaultRegisterer) + c, err := kclient.NewReaderClient( + t.Cfg.KafkaConfig, + clientMetrics, + log.With(logger, "component", "kafka-client"), + ) + if err != nil { + return nil, fmt.Errorf("creating kafka client: %w", err) + } + offsetReader := blockscheduler.NewOffsetReader(t.Cfg.KafkaConfig.Topic, t.Cfg.BlockScheduler.ConsumerGroup, t.Cfg.BlockScheduler.LookbackPeriod, c) + return blockscheduler.NewScheduler(t.Cfg.BlockScheduler, blockscheduler.NewJobQueue(), offsetReader, logger, prometheus.DefaultRegisterer), 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 339ba1a5b7a4e1f37e991c401fc233b5cb19744b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 2 Dec 2024 19:37:20 -0800 Subject: [PATCH 13/44] feat(blockbuilder): grpc transport (#15218) --- pkg/blockbuilder/scheduler/scheduler_test.go | 8 +- pkg/blockbuilder/types/grpc_transport.go | 147 ++ pkg/blockbuilder/types/interfaces.go | 9 + pkg/blockbuilder/types/job.go | 4 +- .../types/proto/blockbuilder.pb.go | 2317 +++++++++++++++++ .../types/proto/blockbuilder.proto | 57 + .../{builder => types}/transport.go | 26 +- 7 files changed, 2547 insertions(+), 21 deletions(-) create mode 100644 pkg/blockbuilder/types/grpc_transport.go create mode 100644 pkg/blockbuilder/types/proto/blockbuilder.pb.go create mode 100644 pkg/blockbuilder/types/proto/blockbuilder.proto rename pkg/blockbuilder/{builder => types}/transport.go (66%) diff --git a/pkg/blockbuilder/scheduler/scheduler_test.go b/pkg/blockbuilder/scheduler/scheduler_test.go index bd9e00450dfa7..35e53ee255993 100644 --- a/pkg/blockbuilder/scheduler/scheduler_test.go +++ b/pkg/blockbuilder/scheduler/scheduler_test.go @@ -15,15 +15,15 @@ import ( type testEnv struct { queue *JobQueue scheduler *BlockScheduler - transport *builder.MemoryTransport + transport *types.MemoryTransport builder *builder.Worker } func newTestEnv(builderID string) *testEnv { queue := NewJobQueue() scheduler := NewScheduler(Config{}, queue, nil, log.NewNopLogger(), prometheus.NewRegistry()) - transport := builder.NewMemoryTransport(scheduler) - builder := builder.NewWorker(builderID, builder.NewMemoryTransport(scheduler)) + transport := types.NewMemoryTransport(scheduler) + builder := builder.NewWorker(builderID, transport) return &testEnv{ queue: queue, @@ -89,7 +89,7 @@ func TestMultipleBuilders(t *testing.T) { // Create first environment env1 := newTestEnv("test-builder-1") // Create second builder using same scheduler - builder2 := builder.NewWorker("test-builder-2", builder.NewMemoryTransport(env1.scheduler)) + builder2 := builder.NewWorker("test-builder-2", env1.transport) ctx := context.Background() diff --git a/pkg/blockbuilder/types/grpc_transport.go b/pkg/blockbuilder/types/grpc_transport.go new file mode 100644 index 0000000000000..675eb92413ac7 --- /dev/null +++ b/pkg/blockbuilder/types/grpc_transport.go @@ -0,0 +1,147 @@ +package types + +import ( + "context" + "flag" + "io" + + "github.com/grafana/dskit/grpcclient" + "github.com/grafana/dskit/instrument" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "google.golang.org/grpc" + "google.golang.org/grpc/health/grpc_health_v1" + + "github.com/grafana/loki/v3/pkg/blockbuilder/types/proto" + "github.com/grafana/loki/v3/pkg/util/constants" +) + +var _ Transport = &GRPCTransport{} + +type GRPCTransportConfig struct { + Address string `yaml:"address,omitempty"` + + // GRPCClientConfig configures the gRPC connection between the Bloom Gateway client and the server. + GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config"` +} + +func (cfg *GRPCTransportConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + f.StringVar(&cfg.Address, prefix+"address", "", "address in DNS Service Discovery format: https://grafana.com/docs/mimir/latest/configure/about-dns-service-discovery/#supported-discovery-modes") +} + +type grpcTransportMetrics struct { + requestLatency *prometheus.HistogramVec +} + +func newGRPCTransportMetrics(registerer prometheus.Registerer) *grpcTransportMetrics { + return &grpcTransportMetrics{ + requestLatency: promauto.With(registerer).NewHistogramVec(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Subsystem: "block_builder_grpc", + Name: "request_duration_seconds", + Help: "Time (in seconds) spent serving requests when using the block builder grpc transport", + Buckets: instrument.DefBuckets, + }, []string{"operation", "status_code"}), + } +} + +// GRPCTransport implements the Transport interface using gRPC +type GRPCTransport struct { + grpc_health_v1.HealthClient + io.Closer + proto.BlockBuilderServiceClient +} + +// NewGRPCTransportFromAddress creates a new gRPC transport instance from an address and dial options +func NewGRPCTransportFromAddress( + metrics *grpcTransportMetrics, + cfg GRPCTransportConfig, +) (*GRPCTransport, error) { + + dialOpts, err := cfg.GRPCClientConfig.DialOption(grpcclient.Instrument(metrics.requestLatency)) + if err != nil { + return nil, err + } + + // nolint:staticcheck // grpc.Dial() has been deprecated; we'll address it before upgrading to gRPC 2. + conn, err := grpc.Dial(cfg.Address, dialOpts...) + if err != nil { + return nil, errors.Wrap(err, "new grpc pool dial") + } + + return &GRPCTransport{ + Closer: conn, + HealthClient: grpc_health_v1.NewHealthClient(conn), + BlockBuilderServiceClient: proto.NewBlockBuilderServiceClient(conn), + }, nil +} + +// SendGetJobRequest implements Transport +func (t *GRPCTransport) SendGetJobRequest(ctx context.Context, req *GetJobRequest) (*GetJobResponse, error) { + protoReq := &proto.GetJobRequest{ + BuilderId: req.BuilderID, + } + + resp, err := t.GetJob(ctx, protoReq) + if err != nil { + return nil, err + } + + return &GetJobResponse{ + Job: protoToJob(resp.GetJob()), + OK: resp.GetOk(), + }, nil +} + +// SendCompleteJob implements Transport +func (t *GRPCTransport) SendCompleteJob(ctx context.Context, req *CompleteJobRequest) error { + protoReq := &proto.CompleteJobRequest{ + BuilderId: req.BuilderID, + Job: jobToProto(req.Job), + } + + _, err := t.CompleteJob(ctx, protoReq) + return err +} + +// SendSyncJob implements Transport +func (t *GRPCTransport) SendSyncJob(ctx context.Context, req *SyncJobRequest) error { + protoReq := &proto.SyncJobRequest{ + BuilderId: req.BuilderID, + Job: jobToProto(req.Job), + } + + _, err := t.SyncJob(ctx, protoReq) + return err +} + +// protoToJob converts a proto Job to a types.Job +func protoToJob(p *proto.Job) *Job { + if p == nil { + return nil + } + return &Job{ + ID: p.GetId(), + Partition: int(p.GetPartition()), + Offsets: Offsets{ + Min: p.GetOffsets().GetMin(), + Max: p.GetOffsets().GetMax(), + }, + } +} + +// jobToProto converts a types.Job to a proto Job +func jobToProto(j *Job) *proto.Job { + if j == nil { + return nil + } + return &proto.Job{ + Id: j.ID, + Partition: int32(j.Partition), + Offsets: &proto.Offsets{ + Min: j.Offsets.Min, + Max: j.Offsets.Max, + }, + } +} diff --git a/pkg/blockbuilder/types/interfaces.go b/pkg/blockbuilder/types/interfaces.go index 74267f912fd7e..dd719757ba6a1 100644 --- a/pkg/blockbuilder/types/interfaces.go +++ b/pkg/blockbuilder/types/interfaces.go @@ -24,6 +24,15 @@ type Scheduler interface { // Transport defines the interface for communication between block builders and scheduler type Transport interface { + BuilderTransport + SchedulerTransport +} + +// SchedulerTransport is for calls originating from the scheduler +type SchedulerTransport interface{} + +// BuilderTransport is for calls originating from the builder +type BuilderTransport interface { // SendGetJobRequest sends a request to get a new job SendGetJobRequest(ctx context.Context, req *GetJobRequest) (*GetJobResponse, error) // SendCompleteJob sends a job completion notification diff --git a/pkg/blockbuilder/types/job.go b/pkg/blockbuilder/types/job.go index d6ed42b598906..2c06fec4d48cd 100644 --- a/pkg/blockbuilder/types/job.go +++ b/pkg/blockbuilder/types/job.go @@ -4,8 +4,7 @@ import "fmt" // Job represents a block building task. type Job struct { - ID string - Status JobStatus + ID string // Partition and offset information Partition int Offsets Offsets @@ -30,7 +29,6 @@ type Offsets struct { func NewJob(partition int, offsets Offsets) *Job { return &Job{ ID: GenerateJobID(partition, offsets), - Status: JobStatusPending, Partition: partition, Offsets: offsets, } diff --git a/pkg/blockbuilder/types/proto/blockbuilder.pb.go b/pkg/blockbuilder/types/proto/blockbuilder.pb.go new file mode 100644 index 0000000000000..c5c4b05d38604 --- /dev/null +++ b/pkg/blockbuilder/types/proto/blockbuilder.pb.go @@ -0,0 +1,2317 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: pkg/blockbuilder/types/proto/blockbuilder.proto + +package proto + +import ( + context "context" + fmt "fmt" + proto "github.com/gogo/protobuf/proto" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + io "io" + math "math" + math_bits "math/bits" + reflect "reflect" + strings "strings" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +// GetJobRequest represents a request for a new job +type GetJobRequest struct { + BuilderId string `protobuf:"bytes,1,opt,name=builder_id,json=builderId,proto3" json:"builder_id,omitempty"` +} + +func (m *GetJobRequest) Reset() { *m = GetJobRequest{} } +func (*GetJobRequest) ProtoMessage() {} +func (*GetJobRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_04968622516f7b79, []int{0} +} +func (m *GetJobRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_GetJobRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *GetJobRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetJobRequest.Merge(m, src) +} +func (m *GetJobRequest) XXX_Size() int { + return m.Size() +} +func (m *GetJobRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetJobRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetJobRequest proto.InternalMessageInfo + +func (m *GetJobRequest) GetBuilderId() string { + if m != nil { + return m.BuilderId + } + return "" +} + +// GetJobResponse contains the response for a job request +type GetJobResponse struct { + Job *Job `protobuf:"bytes,1,opt,name=job,proto3" json:"job,omitempty"` + Ok bool `protobuf:"varint,2,opt,name=ok,proto3" json:"ok,omitempty"` +} + +func (m *GetJobResponse) Reset() { *m = GetJobResponse{} } +func (*GetJobResponse) ProtoMessage() {} +func (*GetJobResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_04968622516f7b79, []int{1} +} +func (m *GetJobResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_GetJobResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *GetJobResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetJobResponse.Merge(m, src) +} +func (m *GetJobResponse) XXX_Size() int { + return m.Size() +} +func (m *GetJobResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetJobResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GetJobResponse proto.InternalMessageInfo + +func (m *GetJobResponse) GetJob() *Job { + if m != nil { + return m.Job + } + return nil +} + +func (m *GetJobResponse) GetOk() bool { + if m != nil { + return m.Ok + } + return false +} + +// CompleteJobRequest represents a job completion notification +type CompleteJobRequest struct { + BuilderId string `protobuf:"bytes,1,opt,name=builder_id,json=builderId,proto3" json:"builder_id,omitempty"` + Job *Job `protobuf:"bytes,2,opt,name=job,proto3" json:"job,omitempty"` +} + +func (m *CompleteJobRequest) Reset() { *m = CompleteJobRequest{} } +func (*CompleteJobRequest) ProtoMessage() {} +func (*CompleteJobRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_04968622516f7b79, []int{2} +} +func (m *CompleteJobRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CompleteJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CompleteJobRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *CompleteJobRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CompleteJobRequest.Merge(m, src) +} +func (m *CompleteJobRequest) XXX_Size() int { + return m.Size() +} +func (m *CompleteJobRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CompleteJobRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_CompleteJobRequest proto.InternalMessageInfo + +func (m *CompleteJobRequest) GetBuilderId() string { + if m != nil { + return m.BuilderId + } + return "" +} + +func (m *CompleteJobRequest) GetJob() *Job { + if m != nil { + return m.Job + } + return nil +} + +// CompleteJobResponse is an empty response for job completion +type CompleteJobResponse struct { +} + +func (m *CompleteJobResponse) Reset() { *m = CompleteJobResponse{} } +func (*CompleteJobResponse) ProtoMessage() {} +func (*CompleteJobResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_04968622516f7b79, []int{3} +} +func (m *CompleteJobResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CompleteJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CompleteJobResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *CompleteJobResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CompleteJobResponse.Merge(m, src) +} +func (m *CompleteJobResponse) XXX_Size() int { + return m.Size() +} +func (m *CompleteJobResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CompleteJobResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_CompleteJobResponse proto.InternalMessageInfo + +// SyncJobRequest represents a job sync request +type SyncJobRequest struct { + BuilderId string `protobuf:"bytes,1,opt,name=builder_id,json=builderId,proto3" json:"builder_id,omitempty"` + Job *Job `protobuf:"bytes,2,opt,name=job,proto3" json:"job,omitempty"` +} + +func (m *SyncJobRequest) Reset() { *m = SyncJobRequest{} } +func (*SyncJobRequest) ProtoMessage() {} +func (*SyncJobRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_04968622516f7b79, []int{4} +} +func (m *SyncJobRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SyncJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SyncJobRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SyncJobRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncJobRequest.Merge(m, src) +} +func (m *SyncJobRequest) XXX_Size() int { + return m.Size() +} +func (m *SyncJobRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SyncJobRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SyncJobRequest proto.InternalMessageInfo + +func (m *SyncJobRequest) GetBuilderId() string { + if m != nil { + return m.BuilderId + } + return "" +} + +func (m *SyncJobRequest) GetJob() *Job { + if m != nil { + return m.Job + } + return nil +} + +// SyncJobResponse is an empty response for job sync +type SyncJobResponse struct { +} + +func (m *SyncJobResponse) Reset() { *m = SyncJobResponse{} } +func (*SyncJobResponse) ProtoMessage() {} +func (*SyncJobResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_04968622516f7b79, []int{5} +} +func (m *SyncJobResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SyncJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SyncJobResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SyncJobResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncJobResponse.Merge(m, src) +} +func (m *SyncJobResponse) XXX_Size() int { + return m.Size() +} +func (m *SyncJobResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SyncJobResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SyncJobResponse proto.InternalMessageInfo + +// Offsets represents the start and end offsets for a job +type Offsets struct { + Min int64 `protobuf:"varint,1,opt,name=min,proto3" json:"min,omitempty"` + Max int64 `protobuf:"varint,2,opt,name=max,proto3" json:"max,omitempty"` +} + +func (m *Offsets) Reset() { *m = Offsets{} } +func (*Offsets) ProtoMessage() {} +func (*Offsets) Descriptor() ([]byte, []int) { + return fileDescriptor_04968622516f7b79, []int{6} +} +func (m *Offsets) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Offsets) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Offsets.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Offsets) XXX_Merge(src proto.Message) { + xxx_messageInfo_Offsets.Merge(m, src) +} +func (m *Offsets) XXX_Size() int { + return m.Size() +} +func (m *Offsets) XXX_DiscardUnknown() { + xxx_messageInfo_Offsets.DiscardUnknown(m) +} + +var xxx_messageInfo_Offsets proto.InternalMessageInfo + +func (m *Offsets) GetMin() int64 { + if m != nil { + return m.Min + } + return 0 +} + +func (m *Offsets) GetMax() int64 { + if m != nil { + return m.Max + } + return 0 +} + +// Job represents a block building job +type Job struct { + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Partition int32 `protobuf:"varint,2,opt,name=partition,proto3" json:"partition,omitempty"` + Offsets *Offsets `protobuf:"bytes,3,opt,name=offsets,proto3" json:"offsets,omitempty"` +} + +func (m *Job) Reset() { *m = Job{} } +func (*Job) ProtoMessage() {} +func (*Job) Descriptor() ([]byte, []int) { + return fileDescriptor_04968622516f7b79, []int{7} +} +func (m *Job) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Job) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Job.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Job) XXX_Merge(src proto.Message) { + xxx_messageInfo_Job.Merge(m, src) +} +func (m *Job) XXX_Size() int { + return m.Size() +} +func (m *Job) XXX_DiscardUnknown() { + xxx_messageInfo_Job.DiscardUnknown(m) +} + +var xxx_messageInfo_Job proto.InternalMessageInfo + +func (m *Job) GetId() string { + if m != nil { + return m.Id + } + return "" +} + +func (m *Job) GetPartition() int32 { + if m != nil { + return m.Partition + } + return 0 +} + +func (m *Job) GetOffsets() *Offsets { + if m != nil { + return m.Offsets + } + return nil +} + +func init() { + proto.RegisterType((*GetJobRequest)(nil), "blockbuilder.types.GetJobRequest") + proto.RegisterType((*GetJobResponse)(nil), "blockbuilder.types.GetJobResponse") + proto.RegisterType((*CompleteJobRequest)(nil), "blockbuilder.types.CompleteJobRequest") + proto.RegisterType((*CompleteJobResponse)(nil), "blockbuilder.types.CompleteJobResponse") + proto.RegisterType((*SyncJobRequest)(nil), "blockbuilder.types.SyncJobRequest") + proto.RegisterType((*SyncJobResponse)(nil), "blockbuilder.types.SyncJobResponse") + proto.RegisterType((*Offsets)(nil), "blockbuilder.types.Offsets") + proto.RegisterType((*Job)(nil), "blockbuilder.types.Job") +} + +func init() { + proto.RegisterFile("pkg/blockbuilder/types/proto/blockbuilder.proto", fileDescriptor_04968622516f7b79) +} + +var fileDescriptor_04968622516f7b79 = []byte{ + // 438 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x93, 0xcf, 0xae, 0xd2, 0x40, + 0x18, 0xc5, 0x3b, 0x6d, 0xbc, 0xc8, 0x77, 0x23, 0xea, 0xdc, 0x18, 0x09, 0xea, 0xe4, 0x5a, 0x13, + 0xbd, 0x2e, 0x6c, 0x13, 0xd4, 0x17, 0xc0, 0x85, 0x11, 0x17, 0xc6, 0xe2, 0x8a, 0x85, 0xda, 0x3f, + 0x03, 0x0e, 0x2d, 0x9d, 0xda, 0x0e, 0x06, 0x76, 0x3e, 0x82, 0x8f, 0xe0, 0xd2, 0x47, 0x71, 0xc9, + 0x92, 0xa5, 0x94, 0x8d, 0x4b, 0x1e, 0xc1, 0x74, 0xda, 0xa2, 0x0d, 0x0d, 0xb8, 0xb9, 0xab, 0x36, + 0xa7, 0xbf, 0x9e, 0x73, 0xf2, 0x7d, 0x33, 0x60, 0x46, 0xfe, 0xd8, 0x74, 0x02, 0xee, 0xfa, 0xce, + 0x8c, 0x05, 0x1e, 0x8d, 0x4d, 0xb1, 0x88, 0x68, 0x62, 0x46, 0x31, 0x17, 0xbc, 0xf2, 0xc1, 0x90, + 0x12, 0xc6, 0x15, 0x4d, 0xc2, 0xba, 0x01, 0xd7, 0x5e, 0x52, 0xd1, 0xe7, 0x8e, 0x45, 0x3f, 0xcf, + 0x68, 0x22, 0xf0, 0x3d, 0x80, 0x82, 0xf8, 0xc0, 0xbc, 0x36, 0x3a, 0x47, 0x17, 0x4d, 0xab, 0x59, + 0x28, 0xaf, 0x3c, 0xfd, 0x35, 0xb4, 0x4a, 0x3e, 0x89, 0x78, 0x98, 0x50, 0xfc, 0x18, 0xb4, 0x09, + 0x77, 0x24, 0x79, 0xda, 0xbd, 0x6d, 0xec, 0x67, 0x18, 0x19, 0x9d, 0x31, 0xb8, 0x05, 0x2a, 0xf7, + 0xdb, 0xea, 0x39, 0xba, 0xb8, 0x6a, 0xa9, 0xdc, 0xd7, 0xdf, 0x03, 0x7e, 0xc1, 0xa7, 0x51, 0x40, + 0x05, 0xfd, 0xef, 0x06, 0x65, 0x9e, 0x7a, 0x3c, 0x4f, 0xbf, 0x05, 0x67, 0x15, 0xff, 0xbc, 0xb1, + 0x3e, 0x84, 0xd6, 0x60, 0x11, 0xba, 0x97, 0x12, 0x79, 0x13, 0xae, 0xef, 0xbc, 0x8b, 0xb8, 0x27, + 0xd0, 0x78, 0x33, 0x1a, 0x25, 0x54, 0x24, 0xf8, 0x06, 0x68, 0x53, 0x16, 0xca, 0x00, 0xcd, 0xca, + 0x5e, 0xa5, 0x62, 0xcf, 0xa5, 0x75, 0xa6, 0xd8, 0x73, 0x7d, 0x02, 0x5a, 0x3f, 0x9f, 0xd5, 0xae, + 0x8a, 0xca, 0x3c, 0x7c, 0x17, 0x9a, 0x91, 0x1d, 0x0b, 0x26, 0x18, 0x0f, 0x25, 0x7e, 0xc5, 0xfa, + 0x2b, 0xe0, 0xe7, 0xd0, 0xe0, 0x79, 0x46, 0x5b, 0x93, 0x2d, 0xef, 0xd4, 0xb5, 0x2c, 0x6a, 0x58, + 0x25, 0xdb, 0xfd, 0xae, 0xc2, 0x59, 0x2f, 0xe3, 0x7a, 0x39, 0x37, 0xa0, 0xf1, 0x17, 0xe6, 0x52, + 0xfc, 0x16, 0x4e, 0xf2, 0x2d, 0xe3, 0xfb, 0x75, 0x3e, 0x95, 0x13, 0xd3, 0xd1, 0x0f, 0x21, 0xc5, + 0x0c, 0x14, 0xfc, 0x11, 0x4e, 0xff, 0xd9, 0x05, 0x7e, 0x58, 0xf7, 0xd3, 0xfe, 0x61, 0xe8, 0x3c, + 0x3a, 0xca, 0xed, 0x12, 0xde, 0x41, 0xa3, 0x18, 0x3d, 0xae, 0xad, 0x54, 0xdd, 0x79, 0xe7, 0xc1, + 0x41, 0xa6, 0x74, 0xed, 0x4d, 0x96, 0x6b, 0xa2, 0xac, 0xd6, 0x44, 0xd9, 0xae, 0x09, 0xfa, 0x9a, + 0x12, 0xf4, 0x23, 0x25, 0xe8, 0x67, 0x4a, 0xd0, 0x32, 0x25, 0xe8, 0x57, 0x4a, 0xd0, 0xef, 0x94, + 0x28, 0xdb, 0x94, 0xa0, 0x6f, 0x1b, 0xa2, 0x2c, 0x37, 0x44, 0x59, 0x6d, 0x88, 0x32, 0x7c, 0x36, + 0x66, 0xe2, 0xd3, 0xcc, 0x31, 0x5c, 0x3e, 0x35, 0xc7, 0xb1, 0x3d, 0xb2, 0x43, 0xdb, 0x0c, 0xb8, + 0xcf, 0x0e, 0xde, 0x59, 0xe7, 0x44, 0x3e, 0x9e, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, 0x6b, 0x42, + 0xf6, 0xf1, 0xda, 0x03, 0x00, 0x00, +} + +func (this *GetJobRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*GetJobRequest) + if !ok { + that2, ok := that.(GetJobRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.BuilderId != that1.BuilderId { + return false + } + return true +} +func (this *GetJobResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*GetJobResponse) + if !ok { + that2, ok := that.(GetJobResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Job.Equal(that1.Job) { + return false + } + if this.Ok != that1.Ok { + return false + } + return true +} +func (this *CompleteJobRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*CompleteJobRequest) + if !ok { + that2, ok := that.(CompleteJobRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.BuilderId != that1.BuilderId { + return false + } + if !this.Job.Equal(that1.Job) { + return false + } + return true +} +func (this *CompleteJobResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*CompleteJobResponse) + if !ok { + that2, ok := that.(CompleteJobResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + return true +} +func (this *SyncJobRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SyncJobRequest) + if !ok { + that2, ok := that.(SyncJobRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.BuilderId != that1.BuilderId { + return false + } + if !this.Job.Equal(that1.Job) { + return false + } + return true +} +func (this *SyncJobResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SyncJobResponse) + if !ok { + that2, ok := that.(SyncJobResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + return true +} +func (this *Offsets) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Offsets) + if !ok { + that2, ok := that.(Offsets) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Min != that1.Min { + return false + } + if this.Max != that1.Max { + return false + } + return true +} +func (this *Job) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Job) + if !ok { + that2, ok := that.(Job) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Id != that1.Id { + return false + } + if this.Partition != that1.Partition { + return false + } + if !this.Offsets.Equal(that1.Offsets) { + return false + } + return true +} +func (this *GetJobRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&proto.GetJobRequest{") + s = append(s, "BuilderId: "+fmt.Sprintf("%#v", this.BuilderId)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *GetJobResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&proto.GetJobResponse{") + if this.Job != nil { + s = append(s, "Job: "+fmt.Sprintf("%#v", this.Job)+",\n") + } + s = append(s, "Ok: "+fmt.Sprintf("%#v", this.Ok)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *CompleteJobRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&proto.CompleteJobRequest{") + s = append(s, "BuilderId: "+fmt.Sprintf("%#v", this.BuilderId)+",\n") + if this.Job != nil { + s = append(s, "Job: "+fmt.Sprintf("%#v", this.Job)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *CompleteJobResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 4) + s = append(s, "&proto.CompleteJobResponse{") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SyncJobRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&proto.SyncJobRequest{") + s = append(s, "BuilderId: "+fmt.Sprintf("%#v", this.BuilderId)+",\n") + if this.Job != nil { + s = append(s, "Job: "+fmt.Sprintf("%#v", this.Job)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SyncJobResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 4) + s = append(s, "&proto.SyncJobResponse{") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Offsets) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&proto.Offsets{") + s = append(s, "Min: "+fmt.Sprintf("%#v", this.Min)+",\n") + s = append(s, "Max: "+fmt.Sprintf("%#v", this.Max)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Job) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&proto.Job{") + s = append(s, "Id: "+fmt.Sprintf("%#v", this.Id)+",\n") + s = append(s, "Partition: "+fmt.Sprintf("%#v", this.Partition)+",\n") + if this.Offsets != nil { + s = append(s, "Offsets: "+fmt.Sprintf("%#v", this.Offsets)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringBlockbuilder(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// BlockBuilderServiceClient is the client API for BlockBuilderService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type BlockBuilderServiceClient interface { + // GetJob requests a new job from the scheduler + GetJob(ctx context.Context, in *GetJobRequest, opts ...grpc.CallOption) (*GetJobResponse, error) + // CompleteJob notifies the scheduler that a job has been completed + CompleteJob(ctx context.Context, in *CompleteJobRequest, opts ...grpc.CallOption) (*CompleteJobResponse, error) + // SyncJob syncs job state with the scheduler + SyncJob(ctx context.Context, in *SyncJobRequest, opts ...grpc.CallOption) (*SyncJobResponse, error) +} + +type blockBuilderServiceClient struct { + cc *grpc.ClientConn +} + +func NewBlockBuilderServiceClient(cc *grpc.ClientConn) BlockBuilderServiceClient { + return &blockBuilderServiceClient{cc} +} + +func (c *blockBuilderServiceClient) GetJob(ctx context.Context, in *GetJobRequest, opts ...grpc.CallOption) (*GetJobResponse, error) { + out := new(GetJobResponse) + err := c.cc.Invoke(ctx, "/blockbuilder.types.BlockBuilderService/GetJob", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *blockBuilderServiceClient) CompleteJob(ctx context.Context, in *CompleteJobRequest, opts ...grpc.CallOption) (*CompleteJobResponse, error) { + out := new(CompleteJobResponse) + err := c.cc.Invoke(ctx, "/blockbuilder.types.BlockBuilderService/CompleteJob", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *blockBuilderServiceClient) SyncJob(ctx context.Context, in *SyncJobRequest, opts ...grpc.CallOption) (*SyncJobResponse, error) { + out := new(SyncJobResponse) + err := c.cc.Invoke(ctx, "/blockbuilder.types.BlockBuilderService/SyncJob", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// BlockBuilderServiceServer is the server API for BlockBuilderService service. +type BlockBuilderServiceServer interface { + // GetJob requests a new job from the scheduler + GetJob(context.Context, *GetJobRequest) (*GetJobResponse, error) + // CompleteJob notifies the scheduler that a job has been completed + CompleteJob(context.Context, *CompleteJobRequest) (*CompleteJobResponse, error) + // SyncJob syncs job state with the scheduler + SyncJob(context.Context, *SyncJobRequest) (*SyncJobResponse, error) +} + +// UnimplementedBlockBuilderServiceServer can be embedded to have forward compatible implementations. +type UnimplementedBlockBuilderServiceServer struct { +} + +func (*UnimplementedBlockBuilderServiceServer) GetJob(ctx context.Context, req *GetJobRequest) (*GetJobResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetJob not implemented") +} +func (*UnimplementedBlockBuilderServiceServer) CompleteJob(ctx context.Context, req *CompleteJobRequest) (*CompleteJobResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CompleteJob not implemented") +} +func (*UnimplementedBlockBuilderServiceServer) SyncJob(ctx context.Context, req *SyncJobRequest) (*SyncJobResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SyncJob not implemented") +} + +func RegisterBlockBuilderServiceServer(s *grpc.Server, srv BlockBuilderServiceServer) { + s.RegisterService(&_BlockBuilderService_serviceDesc, srv) +} + +func _BlockBuilderService_GetJob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetJobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(BlockBuilderServiceServer).GetJob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/blockbuilder.types.BlockBuilderService/GetJob", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(BlockBuilderServiceServer).GetJob(ctx, req.(*GetJobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _BlockBuilderService_CompleteJob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CompleteJobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(BlockBuilderServiceServer).CompleteJob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/blockbuilder.types.BlockBuilderService/CompleteJob", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(BlockBuilderServiceServer).CompleteJob(ctx, req.(*CompleteJobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _BlockBuilderService_SyncJob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SyncJobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(BlockBuilderServiceServer).SyncJob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/blockbuilder.types.BlockBuilderService/SyncJob", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(BlockBuilderServiceServer).SyncJob(ctx, req.(*SyncJobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +var _BlockBuilderService_serviceDesc = grpc.ServiceDesc{ + ServiceName: "blockbuilder.types.BlockBuilderService", + HandlerType: (*BlockBuilderServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetJob", + Handler: _BlockBuilderService_GetJob_Handler, + }, + { + MethodName: "CompleteJob", + Handler: _BlockBuilderService_CompleteJob_Handler, + }, + { + MethodName: "SyncJob", + Handler: _BlockBuilderService_SyncJob_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "pkg/blockbuilder/types/proto/blockbuilder.proto", +} + +func (m *GetJobRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetJobRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetJobRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.BuilderId) > 0 { + i -= len(m.BuilderId) + copy(dAtA[i:], m.BuilderId) + i = encodeVarintBlockbuilder(dAtA, i, uint64(len(m.BuilderId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *GetJobResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GetJobResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GetJobResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Ok { + i-- + if m.Ok { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if m.Job != nil { + { + size, err := m.Job.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintBlockbuilder(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CompleteJobRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CompleteJobRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CompleteJobRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Job != nil { + { + size, err := m.Job.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintBlockbuilder(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.BuilderId) > 0 { + i -= len(m.BuilderId) + copy(dAtA[i:], m.BuilderId) + i = encodeVarintBlockbuilder(dAtA, i, uint64(len(m.BuilderId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CompleteJobResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CompleteJobResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CompleteJobResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + return len(dAtA) - i, nil +} + +func (m *SyncJobRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncJobRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncJobRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Job != nil { + { + size, err := m.Job.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintBlockbuilder(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.BuilderId) > 0 { + i -= len(m.BuilderId) + copy(dAtA[i:], m.BuilderId) + i = encodeVarintBlockbuilder(dAtA, i, uint64(len(m.BuilderId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SyncJobResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SyncJobResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SyncJobResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + return len(dAtA) - i, nil +} + +func (m *Offsets) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Offsets) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Offsets) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Max != 0 { + i = encodeVarintBlockbuilder(dAtA, i, uint64(m.Max)) + i-- + dAtA[i] = 0x10 + } + if m.Min != 0 { + i = encodeVarintBlockbuilder(dAtA, i, uint64(m.Min)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Job) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Job) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Job) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Offsets != nil { + { + size, err := m.Offsets.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintBlockbuilder(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.Partition != 0 { + i = encodeVarintBlockbuilder(dAtA, i, uint64(m.Partition)) + i-- + dAtA[i] = 0x10 + } + if len(m.Id) > 0 { + i -= len(m.Id) + copy(dAtA[i:], m.Id) + i = encodeVarintBlockbuilder(dAtA, i, uint64(len(m.Id))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func encodeVarintBlockbuilder(dAtA []byte, offset int, v uint64) int { + offset -= sovBlockbuilder(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *GetJobRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.BuilderId) + if l > 0 { + n += 1 + l + sovBlockbuilder(uint64(l)) + } + return n +} + +func (m *GetJobResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Job != nil { + l = m.Job.Size() + n += 1 + l + sovBlockbuilder(uint64(l)) + } + if m.Ok { + n += 2 + } + return n +} + +func (m *CompleteJobRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.BuilderId) + if l > 0 { + n += 1 + l + sovBlockbuilder(uint64(l)) + } + if m.Job != nil { + l = m.Job.Size() + n += 1 + l + sovBlockbuilder(uint64(l)) + } + return n +} + +func (m *CompleteJobResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *SyncJobRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.BuilderId) + if l > 0 { + n += 1 + l + sovBlockbuilder(uint64(l)) + } + if m.Job != nil { + l = m.Job.Size() + n += 1 + l + sovBlockbuilder(uint64(l)) + } + return n +} + +func (m *SyncJobResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *Offsets) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Min != 0 { + n += 1 + sovBlockbuilder(uint64(m.Min)) + } + if m.Max != 0 { + n += 1 + sovBlockbuilder(uint64(m.Max)) + } + return n +} + +func (m *Job) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Id) + if l > 0 { + n += 1 + l + sovBlockbuilder(uint64(l)) + } + if m.Partition != 0 { + n += 1 + sovBlockbuilder(uint64(m.Partition)) + } + if m.Offsets != nil { + l = m.Offsets.Size() + n += 1 + l + sovBlockbuilder(uint64(l)) + } + return n +} + +func sovBlockbuilder(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozBlockbuilder(x uint64) (n int) { + return sovBlockbuilder(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *GetJobRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&GetJobRequest{`, + `BuilderId:` + fmt.Sprintf("%v", this.BuilderId) + `,`, + `}`, + }, "") + return s +} +func (this *GetJobResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&GetJobResponse{`, + `Job:` + strings.Replace(this.Job.String(), "Job", "Job", 1) + `,`, + `Ok:` + fmt.Sprintf("%v", this.Ok) + `,`, + `}`, + }, "") + return s +} +func (this *CompleteJobRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&CompleteJobRequest{`, + `BuilderId:` + fmt.Sprintf("%v", this.BuilderId) + `,`, + `Job:` + strings.Replace(this.Job.String(), "Job", "Job", 1) + `,`, + `}`, + }, "") + return s +} +func (this *CompleteJobResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&CompleteJobResponse{`, + `}`, + }, "") + return s +} +func (this *SyncJobRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SyncJobRequest{`, + `BuilderId:` + fmt.Sprintf("%v", this.BuilderId) + `,`, + `Job:` + strings.Replace(this.Job.String(), "Job", "Job", 1) + `,`, + `}`, + }, "") + return s +} +func (this *SyncJobResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SyncJobResponse{`, + `}`, + }, "") + return s +} +func (this *Offsets) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Offsets{`, + `Min:` + fmt.Sprintf("%v", this.Min) + `,`, + `Max:` + fmt.Sprintf("%v", this.Max) + `,`, + `}`, + }, "") + return s +} +func (this *Job) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Job{`, + `Id:` + fmt.Sprintf("%v", this.Id) + `,`, + `Partition:` + fmt.Sprintf("%v", this.Partition) + `,`, + `Offsets:` + strings.Replace(this.Offsets.String(), "Offsets", "Offsets", 1) + `,`, + `}`, + }, "") + return s +} +func valueToStringBlockbuilder(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *GetJobRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetJobRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetJobRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BuilderId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthBlockbuilder + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthBlockbuilder + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BuilderId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipBlockbuilder(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetJobResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GetJobResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetJobResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Job", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthBlockbuilder + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthBlockbuilder + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Job == nil { + m.Job = &Job{} + } + if err := m.Job.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Ok", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Ok = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipBlockbuilder(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CompleteJobRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CompleteJobRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CompleteJobRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BuilderId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthBlockbuilder + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthBlockbuilder + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BuilderId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Job", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthBlockbuilder + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthBlockbuilder + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Job == nil { + m.Job = &Job{} + } + if err := m.Job.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipBlockbuilder(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CompleteJobResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CompleteJobResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CompleteJobResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipBlockbuilder(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyncJobRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncJobRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncJobRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BuilderId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthBlockbuilder + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthBlockbuilder + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BuilderId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Job", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthBlockbuilder + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthBlockbuilder + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Job == nil { + m.Job = &Job{} + } + if err := m.Job.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipBlockbuilder(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SyncJobResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SyncJobResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncJobResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipBlockbuilder(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Offsets) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Offsets: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Offsets: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Min", wireType) + } + m.Min = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Min |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Max", wireType) + } + m.Max = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Max |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipBlockbuilder(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Job) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Job: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Job: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthBlockbuilder + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthBlockbuilder + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Id = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Partition", wireType) + } + m.Partition = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Partition |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Offsets", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthBlockbuilder + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthBlockbuilder + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Offsets == nil { + m.Offsets = &Offsets{} + } + if err := m.Offsets.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipBlockbuilder(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthBlockbuilder + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipBlockbuilder(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthBlockbuilder + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthBlockbuilder + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipBlockbuilder(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthBlockbuilder + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthBlockbuilder = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowBlockbuilder = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/blockbuilder/types/proto/blockbuilder.proto b/pkg/blockbuilder/types/proto/blockbuilder.proto new file mode 100644 index 0000000000000..89811989b821c --- /dev/null +++ b/pkg/blockbuilder/types/proto/blockbuilder.proto @@ -0,0 +1,57 @@ +syntax = "proto3"; + +package blockbuilder.types; + +option go_package = "github.com/grafana/loki/pkg/blockbuilder/types/proto"; + +// BlockBuilderService defines the gRPC service for block builder communication +service BlockBuilderService { + // GetJob requests a new job from the scheduler + rpc GetJob(GetJobRequest) returns (GetJobResponse) {} + // CompleteJob notifies the scheduler that a job has been completed + rpc CompleteJob(CompleteJobRequest) returns (CompleteJobResponse) {} + // SyncJob syncs job state with the scheduler + rpc SyncJob(SyncJobRequest) returns (SyncJobResponse) {} +} + +// GetJobRequest represents a request for a new job +message GetJobRequest { + string builder_id = 1; +} + +// GetJobResponse contains the response for a job request +message GetJobResponse { + Job job = 1; + bool ok = 2; +} + +// CompleteJobRequest represents a job completion notification +message CompleteJobRequest { + string builder_id = 1; + Job job = 2; +} + +// CompleteJobResponse is an empty response for job completion +message CompleteJobResponse {} + +// SyncJobRequest represents a job sync request +message SyncJobRequest { + string builder_id = 1; + Job job = 2; +} + +// SyncJobResponse is an empty response for job sync +message SyncJobResponse {} + +// Offsets represents the start and end offsets for a job +message Offsets { + int64 min = 1; + int64 max = 2; +} + +// Job represents a block building job +message Job { + string id = 1; + int32 partition = 2; + Offsets offsets = 3; +} diff --git a/pkg/blockbuilder/builder/transport.go b/pkg/blockbuilder/types/transport.go similarity index 66% rename from pkg/blockbuilder/builder/transport.go rename to pkg/blockbuilder/types/transport.go index ae498459cb667..5659ffb48a4b4 100644 --- a/pkg/blockbuilder/builder/transport.go +++ b/pkg/blockbuilder/types/transport.go @@ -1,58 +1,56 @@ -package builder +package types import ( "context" - - "github.com/grafana/loki/v3/pkg/blockbuilder/types" ) var ( - _ types.Transport = unimplementedTransport{} - _ types.Transport = &MemoryTransport{} + _ Transport = unimplementedTransport{} + _ Transport = &MemoryTransport{} ) // unimplementedTransport provides default implementations that panic type unimplementedTransport struct{} -func (t unimplementedTransport) SendGetJobRequest(_ context.Context, _ *types.GetJobRequest) (*types.GetJobResponse, error) { +func (t unimplementedTransport) SendGetJobRequest(_ context.Context, _ *GetJobRequest) (*GetJobResponse, error) { panic("unimplemented") } -func (t unimplementedTransport) SendCompleteJob(_ context.Context, _ *types.CompleteJobRequest) error { +func (t unimplementedTransport) SendCompleteJob(_ context.Context, _ *CompleteJobRequest) error { panic("unimplemented") } -func (t unimplementedTransport) SendSyncJob(_ context.Context, _ *types.SyncJobRequest) error { +func (t unimplementedTransport) SendSyncJob(_ context.Context, _ *SyncJobRequest) error { panic("unimplemented") } // MemoryTransport implements Transport interface for in-memory communication type MemoryTransport struct { - scheduler types.Scheduler + scheduler Scheduler } // NewMemoryTransport creates a new in-memory transport instance -func NewMemoryTransport(scheduler types.Scheduler) *MemoryTransport { +func NewMemoryTransport(scheduler Scheduler) *MemoryTransport { return &MemoryTransport{ scheduler: scheduler, } } -func (t *MemoryTransport) SendGetJobRequest(ctx context.Context, req *types.GetJobRequest) (*types.GetJobResponse, error) { +func (t *MemoryTransport) SendGetJobRequest(ctx context.Context, req *GetJobRequest) (*GetJobResponse, error) { job, ok, err := t.scheduler.HandleGetJob(ctx, req.BuilderID) if err != nil { return nil, err } - return &types.GetJobResponse{ + return &GetJobResponse{ Job: job, OK: ok, }, nil } -func (t *MemoryTransport) SendCompleteJob(ctx context.Context, req *types.CompleteJobRequest) error { +func (t *MemoryTransport) SendCompleteJob(ctx context.Context, req *CompleteJobRequest) error { return t.scheduler.HandleCompleteJob(ctx, req.BuilderID, req.Job) } -func (t *MemoryTransport) SendSyncJob(ctx context.Context, req *types.SyncJobRequest) error { +func (t *MemoryTransport) SendSyncJob(ctx context.Context, req *SyncJobRequest) error { return t.scheduler.HandleSyncJob(ctx, req.BuilderID, req.Job) } From 50dedb1c3dd731f7c1f600a0cdbd75c9c0db7874 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 2 Dec 2024 20:09:57 -0800 Subject: [PATCH 14/44] chore(blockbuilder): updates comments (#15220) --- pkg/blockbuilder/types/grpc_transport.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/blockbuilder/types/grpc_transport.go b/pkg/blockbuilder/types/grpc_transport.go index 675eb92413ac7..0d132e2d07bd7 100644 --- a/pkg/blockbuilder/types/grpc_transport.go +++ b/pkg/blockbuilder/types/grpc_transport.go @@ -22,7 +22,7 @@ var _ Transport = &GRPCTransport{} type GRPCTransportConfig struct { Address string `yaml:"address,omitempty"` - // GRPCClientConfig configures the gRPC connection between the Bloom Gateway client and the server. + // GRPCClientConfig configures the gRPC connection between the block-builder and its scheduler. GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config"` } From a6ba3c05ab5eb2bf1f5caa76cb85cfa25c508d5e Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Tue, 3 Dec 2024 07:50:41 -0500 Subject: [PATCH 15/44] fix(deps): update module github.com/baidubce/bce-sdk-go to v0.9.203 (#15219) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- vendor/github.com/baidubce/bce-sdk-go/bce/config.go | 2 +- vendor/modules.txt | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 5174a10e73330..b206bf405bcbb 100644 --- a/go.mod +++ b/go.mod @@ -21,7 +21,7 @@ require ( github.com/alicebob/miniredis/v2 v2.33.0 github.com/aliyun/aliyun-oss-go-sdk v3.0.2+incompatible github.com/aws/aws-sdk-go v1.55.5 - github.com/baidubce/bce-sdk-go v0.9.202 + github.com/baidubce/bce-sdk-go v0.9.203 github.com/bmatcuk/doublestar/v4 v4.7.1 github.com/c2h5oh/datasize v0.0.0-20231215233829-aa82cc1e6500 github.com/cespare/xxhash/v2 v2.3.0 diff --git a/go.sum b/go.sum index 9d7fe1bb3c307..d9b51fe611825 100644 --- a/go.sum +++ b/go.sum @@ -1008,8 +1008,8 @@ github.com/aws/smithy-go v1.11.1 h1:IQ+lPZVkSM3FRtyaDox41R8YS6iwPMYIreejOgPW49g= github.com/aws/smithy-go v1.11.1/go.mod h1:3xHYmszWVx2c0kIwQeEVf9uSm4fYZt67FBJnwub1bgM= github.com/axiomhq/hyperloglog v0.2.0 h1:u1XT3yyY1rjzlWuP6NQIrV4bRYHOaqZaovqjcBEvZJo= github.com/axiomhq/hyperloglog v0.2.0/go.mod h1:GcgMjz9gaDKZ3G0UMS6Fq/VkZ4l7uGgcJyxA7M+omIM= -github.com/baidubce/bce-sdk-go v0.9.202 h1:TGRdO4g4CtiI2IZ6MxeUmkbKe6l8kq+mYH6SbxczO3g= -github.com/baidubce/bce-sdk-go v0.9.202/go.mod h1:zbYJMQwE4IZuyrJiFO8tO8NbtYiKTFTbwh4eIsqjVdg= +github.com/baidubce/bce-sdk-go v0.9.203 h1:D4YBk4prtlIjrnwrh5nvsSSjLjataApDmeL0fxvI/KU= +github.com/baidubce/bce-sdk-go v0.9.203/go.mod h1:zbYJMQwE4IZuyrJiFO8tO8NbtYiKTFTbwh4eIsqjVdg= github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3 h1:6df1vn4bBlDDo4tARvBm7l6KA9iVMnE3NWizDeWSrps= github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3/go.mod h1:CIWtjkly68+yqLPbvwwR/fjNJA/idrtULjZWh2v1ys0= github.com/benbjohnson/clock v1.0.3/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= diff --git a/vendor/github.com/baidubce/bce-sdk-go/bce/config.go b/vendor/github.com/baidubce/bce-sdk-go/bce/config.go index 243b4c81ce93a..cd632ec2cd89a 100644 --- a/vendor/github.com/baidubce/bce-sdk-go/bce/config.go +++ b/vendor/github.com/baidubce/bce-sdk-go/bce/config.go @@ -26,7 +26,7 @@ import ( // Constants and default values for the package bce const ( - SDK_VERSION = "0.9.202" + SDK_VERSION = "0.9.203" URI_PREFIX = "/" // now support uri without prefix "v1" so just set root path DEFAULT_DOMAIN = "baidubce.com" DEFAULT_PROTOCOL = "http" diff --git a/vendor/modules.txt b/vendor/modules.txt index e7e389624a74a..88462c29e7d15 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -465,7 +465,7 @@ github.com/aws/smithy-go/transport/http/internal/io # github.com/axiomhq/hyperloglog v0.2.0 ## explicit; go 1.21 github.com/axiomhq/hyperloglog -# github.com/baidubce/bce-sdk-go v0.9.202 +# github.com/baidubce/bce-sdk-go v0.9.203 ## explicit; go 1.11 github.com/baidubce/bce-sdk-go/auth github.com/baidubce/bce-sdk-go/bce From 9f7892a55f9e0952e248ff820628681851d8511f Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Tue, 3 Dec 2024 07:51:17 -0500 Subject: [PATCH 16/44] fix(deps): update module github.com/opentracing-contrib/go-stdlib to v1.1.0 (#15221) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 +- .../go-stdlib/nethttp/client.go | 86 +++++++++++-------- ...tus-code-tracker.go => metrics-tracker.go} | 16 ++-- .../go-stdlib/nethttp/server.go | 49 ++++++----- vendor/modules.txt | 2 +- 6 files changed, 94 insertions(+), 65 deletions(-) rename vendor/github.com/opentracing-contrib/go-stdlib/nethttp/{status-code-tracker.go => metrics-tracker.go} (92%) diff --git a/go.mod b/go.mod index b206bf405bcbb..0db5700d14b03 100644 --- a/go.mod +++ b/go.mod @@ -76,7 +76,7 @@ require ( github.com/oklog/run v1.1.0 github.com/oklog/ulid v1.3.1 // indirect github.com/opentracing-contrib/go-grpc v0.1.0 - github.com/opentracing-contrib/go-stdlib v1.0.0 + github.com/opentracing-contrib/go-stdlib v1.1.0 github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b github.com/oschwald/geoip2-golang v1.11.0 // github.com/pierrec/lz4 v2.0.5+incompatible diff --git a/go.sum b/go.sum index d9b51fe611825..dbf5091d71167 100644 --- a/go.sum +++ b/go.sum @@ -2309,8 +2309,8 @@ github.com/opencontainers/image-spec v1.1.0/go.mod h1:W4s4sFTMaBeK1BQLXbG4AdM2sz github.com/opentracing-contrib/go-grpc v0.1.0 h1:9JHDtQXv6UL0tFF8KJB/4ApJgeOcaHp1h07d0PJjESc= github.com/opentracing-contrib/go-grpc v0.1.0/go.mod h1:i3/jx/TvJZ/HKidtT4XGIi/NosUEpzS9xjVJctbKZzI= github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= -github.com/opentracing-contrib/go-stdlib v1.0.0 h1:TBS7YuVotp8myLon4Pv7BtCBzOTo1DeZCld0Z63mW2w= -github.com/opentracing-contrib/go-stdlib v1.0.0/go.mod h1:qtI1ogk+2JhVPIXVc6q+NHziSmy2W5GbdQZFUHADCBU= +github.com/opentracing-contrib/go-stdlib v1.1.0 h1:cZBWc4pA4e65tqTJddbflK435S0tDImj6c9BMvkdUH0= +github.com/opentracing-contrib/go-stdlib v1.1.0/go.mod h1:S0p+X9p6dcBkoMTL+Qq2VOvxKs9ys5PpYWXWqlCS0bQ= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= diff --git a/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/client.go b/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/client.go index bfb305ffa463a..d4a8db5c5c33f 100644 --- a/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/client.go +++ b/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/client.go @@ -1,3 +1,4 @@ +//go:build go1.7 // +build go1.7 package nethttp @@ -32,12 +33,12 @@ type Transport struct { } type clientOptions struct { + urlTagFunc func(u *url.URL) string + spanObserver func(span opentracing.Span, r *http.Request) operationName string componentName string - urlTagFunc func(u *url.URL) string disableClientTrace bool disableInjectSpanContext bool - spanObserver func(span opentracing.Span, r *http.Request) } // ClientOption contols the behavior of TraceRequest. @@ -78,7 +79,7 @@ func ClientTrace(enabled bool) ClientOption { // InjectSpanContext returns a ClientOption that turns on or off // injection of the Span context in the request HTTP headers. -// If this option is not used, the default behaviour is to +// If this option is not used, the default behavior is to // inject the span context. func InjectSpanContext(enabled bool) ClientOption { return func(options *clientOptions) { @@ -100,24 +101,24 @@ func ClientSpanObserver(f func(span opentracing.Span, r *http.Request)) ClientOp // // Example: // -// func AskGoogle(ctx context.Context) error { -// client := &http.Client{Transport: &nethttp.Transport{}} -// req, err := http.NewRequest("GET", "http://google.com", nil) -// if err != nil { -// return err -// } -// req = req.WithContext(ctx) // extend existing trace, if any +// func AskGoogle(ctx context.Context) error { +// client := &http.Client{Transport: &nethttp.Transport{}} +// req, err := http.NewRequest("GET", "http://google.com", nil) +// if err != nil { +// return err +// } +// req = req.WithContext(ctx) // extend existing trace, if any // -// req, ht := nethttp.TraceRequest(tracer, req) -// defer ht.Finish() +// req, ht := nethttp.TraceRequest(tracer, req) +// defer ht.Finish() // -// res, err := client.Do(req) -// if err != nil { -// return err -// } -// res.Body.Close() -// return nil -// } +// res, err := client.Do(req) +// if err != nil { +// return err +// } +// res.Body.Close() +// return nil +// } func TraceRequest(tr opentracing.Tracer, req *http.Request, options ...ClientOption) (*http.Request, *Tracer) { opts := &clientOptions{ urlTagFunc: func(u *url.URL) string { @@ -149,6 +150,18 @@ func (c closeTracker) Close() error { return err } +type writerCloseTracker struct { + io.ReadWriteCloser + sp opentracing.Span +} + +func (c writerCloseTracker) Close() error { + err := c.ReadWriteCloser.Close() + c.sp.LogFields(log.String("event", "ClosedBody")) + c.sp.Finish() + return err +} + // TracerFromRequest retrieves the Tracer from the request. If the request does // not have a Tracer it will return nil. func TracerFromRequest(req *http.Request) *Tracer { @@ -170,31 +183,36 @@ func (t *Transport) RoundTrip(req *http.Request) (*http.Response, error) { return rt.RoundTrip(req) } - tracer.start(req) + sp := tracer.start(req) - ext.HTTPMethod.Set(tracer.sp, req.Method) - ext.HTTPUrl.Set(tracer.sp, tracer.opts.urlTagFunc(req.URL)) - tracer.opts.spanObserver(tracer.sp, req) + ext.HTTPMethod.Set(sp, req.Method) + ext.HTTPUrl.Set(sp, tracer.opts.urlTagFunc(req.URL)) + ext.PeerAddress.Set(sp, req.URL.Host) + tracer.opts.spanObserver(sp, req) if !tracer.opts.disableInjectSpanContext { carrier := opentracing.HTTPHeadersCarrier(req.Header) - tracer.sp.Tracer().Inject(tracer.sp.Context(), opentracing.HTTPHeaders, carrier) + sp.Tracer().Inject(sp.Context(), opentracing.HTTPHeaders, carrier) //nolint:errcheck // TODO: should we check the error? Returning it makes the tests fail } resp, err := rt.RoundTrip(req) - if err != nil { - tracer.sp.Finish() + sp.Finish() return resp, err } - ext.HTTPStatusCode.Set(tracer.sp, uint16(resp.StatusCode)) + ext.HTTPStatusCode.Set(sp, uint16(resp.StatusCode)) //nolint:gosec // can't have integer overflow with status code if resp.StatusCode >= http.StatusInternalServerError { - ext.Error.Set(tracer.sp, true) + ext.Error.Set(sp, true) } - if req.Method == "HEAD" { - tracer.sp.Finish() + if req.Method == http.MethodHead { + sp.Finish() } else { - resp.Body = closeTracker{resp.Body, tracer.sp} + readWriteCloser, ok := resp.Body.(io.ReadWriteCloser) + if ok { + resp.Body = writerCloseTracker{readWriteCloser, sp} + } else { + resp.Body = closeTracker{resp.Body, sp} + } } return resp, nil } @@ -223,8 +241,7 @@ func (h *Tracer) start(req *http.Request) opentracing.Span { } ctx := h.root.Context() - h.sp = h.tr.StartSpan("HTTP "+req.Method, opentracing.ChildOf(ctx)) - ext.SpanKindRPCClient.Set(h.sp) + h.sp = h.tr.StartSpan("HTTP "+req.Method, opentracing.ChildOf(ctx), ext.SpanKindRPCClient) componentName := h.opts.componentName if componentName == "" { @@ -266,8 +283,7 @@ func (h *Tracer) clientTrace() *httptrace.ClientTrace { } func (h *Tracer) getConn(hostPort string) { - ext.HTTPUrl.Set(h.sp, hostPort) - h.sp.LogFields(log.String("event", "GetConn")) + h.sp.LogFields(log.String("event", "GetConn"), log.String("hostPort", hostPort)) } func (h *Tracer) gotConn(info httptrace.GotConnInfo) { diff --git a/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/status-code-tracker.go b/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/metrics-tracker.go similarity index 92% rename from vendor/github.com/opentracing-contrib/go-stdlib/nethttp/status-code-tracker.go rename to vendor/github.com/opentracing-contrib/go-stdlib/nethttp/metrics-tracker.go index 80a5ce08645b3..f4250cbb9b226 100644 --- a/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/status-code-tracker.go +++ b/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/metrics-tracker.go @@ -1,3 +1,4 @@ +//go:build go1.8 // +build go1.8 package nethttp @@ -7,28 +8,31 @@ import ( "net/http" ) -type statusCodeTracker struct { +type metricsTracker struct { http.ResponseWriter status int + size int } -func (w *statusCodeTracker) WriteHeader(status int) { +func (w *metricsTracker) WriteHeader(status int) { w.status = status w.ResponseWriter.WriteHeader(status) } -func (w *statusCodeTracker) Write(b []byte) (int, error) { - return w.ResponseWriter.Write(b) +func (w *metricsTracker) Write(b []byte) (int, error) { + size, err := w.ResponseWriter.Write(b) + w.size += size + return size, err } // wrappedResponseWriter returns a wrapped version of the original // ResponseWriter and only implements the same combination of additional // interfaces as the original. This implementation is based on // https://github.com/felixge/httpsnoop. -func (w *statusCodeTracker) wrappedResponseWriter() http.ResponseWriter { +func (w *metricsTracker) wrappedResponseWriter() http.ResponseWriter { var ( hj, i0 = w.ResponseWriter.(http.Hijacker) - cn, i1 = w.ResponseWriter.(http.CloseNotifier) + cn, i1 = w.ResponseWriter.(http.CloseNotifier) //nolint:staticcheck // TODO: Replace deprecated CloseNotifier pu, i2 = w.ResponseWriter.(http.Pusher) fl, i3 = w.ResponseWriter.(http.Flusher) rf, i4 = w.ResponseWriter.(io.ReaderFrom) diff --git a/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/server.go b/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/server.go index db2df6620412b..322e88fca65ef 100644 --- a/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/server.go +++ b/vendor/github.com/opentracing-contrib/go-stdlib/nethttp/server.go @@ -1,3 +1,4 @@ +//go:build go1.7 // +build go1.7 package nethttp @@ -10,6 +11,8 @@ import ( "github.com/opentracing/opentracing-go/ext" ) +var responseSizeKey = "http.response_size" + type mwOptions struct { opNameFunc func(r *http.Request) string spanFilter func(r *http.Request) bool @@ -67,24 +70,26 @@ func MWURLTagFunc(f func(u *url.URL) string) MWOption { // Additionally, it adds the span to the request's context. // // By default, the operation name of the spans is set to "HTTP {method}". -// This can be overriden with options. +// This can be overridden with options. // // Example: -// http.ListenAndServe("localhost:80", nethttp.Middleware(tracer, http.DefaultServeMux)) +// +// http.ListenAndServe("localhost:80", nethttp.Middleware(tracer, http.DefaultServeMux)) // // The options allow fine tuning the behavior of the middleware. // // Example: -// mw := nethttp.Middleware( -// tracer, -// http.DefaultServeMux, -// nethttp.OperationNameFunc(func(r *http.Request) string { -// return "HTTP " + r.Method + ":/api/customers" -// }), -// nethttp.MWSpanObserver(func(sp opentracing.Span, r *http.Request) { -// sp.SetTag("http.uri", r.URL.EscapedPath()) -// }), -// ) +// +// mw := nethttp.Middleware( +// tracer, +// http.DefaultServeMux, +// nethttp.OperationNameFunc(func(r *http.Request) string { +// return "HTTP " + r.Method + ":/api/customers" +// }), +// nethttp.MWSpanObserver(func(sp opentracing.Span, r *http.Request) { +// sp.SetTag("http.uri", r.URL.EscapedPath()) +// }), +// ) func Middleware(tr opentracing.Tracer, h http.Handler, options ...MWOption) http.Handler { return MiddlewareFunc(tr, h.ServeHTTP, options...) } @@ -93,7 +98,8 @@ func Middleware(tr opentracing.Tracer, h http.Handler, options ...MWOption) http // It behaves identically to the Middleware function above. // // Example: -// http.ListenAndServe("localhost:80", nethttp.MiddlewareFunc(tracer, MyHandler)) +// +// http.ListenAndServe("localhost:80", nethttp.MiddlewareFunc(tracer, MyHandler)) func MiddlewareFunc(tr opentracing.Tracer, h http.HandlerFunc, options ...MWOption) http.HandlerFunc { opts := mwOptions{ opNameFunc: func(r *http.Request) string { @@ -126,22 +132,25 @@ func MiddlewareFunc(tr opentracing.Tracer, h http.HandlerFunc, options ...MWOpti ext.Component.Set(sp, componentName) opts.spanObserver(sp, r) - sct := &statusCodeTracker{ResponseWriter: w} + mt := &metricsTracker{ResponseWriter: w} r = r.WithContext(opentracing.ContextWithSpan(r.Context(), sp)) defer func() { panicErr := recover() didPanic := panicErr != nil - if sct.status == 0 && !didPanic { + if mt.status == 0 && !didPanic { // Standard behavior of http.Server is to assume status code 200 if one was not written by a handler that returned successfully. // https://github.com/golang/go/blob/fca286bed3ed0e12336532cc711875ae5b3cb02a/src/net/http/server.go#L120 - sct.status = 200 + mt.status = 200 + } + if mt.status > 0 { + ext.HTTPStatusCode.Set(sp, uint16(mt.status)) //nolint:gosec // can't have integer overflow with status code } - if sct.status > 0 { - ext.HTTPStatusCode.Set(sp, uint16(sct.status)) + if mt.size > 0 { + sp.SetTag(responseSizeKey, mt.size) } - if sct.status >= http.StatusInternalServerError || didPanic { + if mt.status >= http.StatusInternalServerError || didPanic { ext.Error.Set(sp, true) } sp.Finish() @@ -151,7 +160,7 @@ func MiddlewareFunc(tr opentracing.Tracer, h http.HandlerFunc, options ...MWOpti } }() - h(sct.wrappedResponseWriter(), r) + h(mt.wrappedResponseWriter(), r) } return http.HandlerFunc(fn) } diff --git a/vendor/modules.txt b/vendor/modules.txt index 88462c29e7d15..ad64ef0e776b9 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1311,7 +1311,7 @@ github.com/opencontainers/image-spec/specs-go/v1 # github.com/opentracing-contrib/go-grpc v0.1.0 ## explicit; go 1.22.7 github.com/opentracing-contrib/go-grpc -# github.com/opentracing-contrib/go-stdlib v1.0.0 +# github.com/opentracing-contrib/go-stdlib v1.1.0 ## explicit; go 1.14 github.com/opentracing-contrib/go-stdlib/nethttp # github.com/opentracing/opentracing-go v1.2.1-0.20220228012449-10b1cf09e00b From d73e112a6ae136ced9f1ee821ec6f142a4da22c3 Mon Sep 17 00:00:00 2001 From: J Stickler Date: Tue, 3 Dec 2024 10:11:57 -0500 Subject: [PATCH 17/44] docs: add another third-party client to the list (#15216) --- docs/sources/send-data/_index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sources/send-data/_index.md b/docs/sources/send-data/_index.md index 411f0f4367042..58c8a0f2ce0d5 100644 --- a/docs/sources/send-data/_index.md +++ b/docs/sources/send-data/_index.md @@ -56,6 +56,7 @@ By adding our output plugin you can quickly try Loki without doing big configura These third-party clients also enable sending logs to Loki: - [Cribl Loki Destination](https://docs.cribl.io/stream/destinations-loki) +- [GrafanaLokiLogger](https://github.com/antoniojmsjr/GrafanaLokiLogger) (Delphi/Lazarus) - [ilogtail](https://github.com/alibaba/ilogtail) (Go) - [Log4j2 appender for Loki](https://github.com/tkowalcz/tjahzi) (Java) - [loki-logback-appender](https://github.com/loki4j/loki-logback-appender) (Java) From 2fb07a35e9ef28ae70f9372d7fbaaa2773746b1a Mon Sep 17 00:00:00 2001 From: J Stickler Date: Tue, 3 Dec 2024 10:12:22 -0500 Subject: [PATCH 18/44] docs: small clarification re: structured metadata (#15215) --- docs/sources/send-data/otel/_index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/send-data/otel/_index.md b/docs/sources/send-data/otel/_index.md index ce0b1de64483e..88d099a4c3165 100644 --- a/docs/sources/send-data/otel/_index.md +++ b/docs/sources/send-data/otel/_index.md @@ -18,7 +18,7 @@ For ingesting logs to Loki using the OpenTelemetry Collector, you must use the [ When logs are ingested by Loki using an OpenTelemetry protocol (OTLP) ingestion endpoint, some of the data is stored as [Structured Metadata]({{< relref "../../get-started/labels/structured-metadata" >}}). -You must set `allow_structured_metadata` to `true` within your Loki config file. Otherwise, Loki will reject the log payload as malformed. +You must set `allow_structured_metadata` to `true` within your Loki config file. Otherwise, Loki will reject the log payload as malformed. Note that Structured Metadata is enabled by default in Loki 3.0 and later. ```yaml limits_config: From 658fb24311d57dfcdca783f4cf2b64a7a19fa97f Mon Sep 17 00:00:00 2001 From: Alex Richard Westhaver-Ford Date: Tue, 3 Dec 2024 17:01:37 -0500 Subject: [PATCH 19/44] docs: fixed typos/grammatical mistakes in metrics.md (#15166) Co-authored-by: J Stickler --- docs/sources/send-data/promtail/stages/metrics.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/sources/send-data/promtail/stages/metrics.md b/docs/sources/send-data/promtail/stages/metrics.md index b034bd6d6d6a1..ea1c7b78150c5 100644 --- a/docs/sources/send-data/promtail/stages/metrics.md +++ b/docs/sources/send-data/promtail/stages/metrics.md @@ -51,8 +51,8 @@ type: Counter [max_idle_duration: ] config: - # If present and true all log lines will be counted without - # attempting to match the source to the extract map. + # If present and true all log lines will be counted without attempting + # to match the `value` to the field specified by `source` in the extracted map. # It is an error to specify `match_all: true` and also specify a `value` [match_all: ] @@ -231,7 +231,7 @@ This pipeline first tries to find text in the format `order_status=` in the log line, pulling out the `` into the extracted map with the key `order_status`. -The metric stages creates `successful_orders_total` and `failed_orders_total` +The metrics stage creates `successful_orders_total` and `failed_orders_total` metrics that only increment when the value of `order_status` in the extracted map is `success` or `fail` respectively. @@ -265,7 +265,7 @@ number in the `retries` field from the extracted map. - metrics: http_response_time_seconds: type: Histogram - description: "length of each log line" + description: "distribution of log response time" source: response_time config: buckets: [0.001,0.0025,0.005,0.010,0.025,0.050] From e9d0c3ee527597d1db61a20f01efca1189ac0079 Mon Sep 17 00:00:00 2001 From: Callum Styan Date: Tue, 3 Dec 2024 19:36:30 -0800 Subject: [PATCH 20/44] feat: Add a metric ingestion time SM sanitization (#15222) Signed-off-by: Callum Styan --- pkg/distributor/distributor.go | 22 +++++++++++++++++----- pkg/distributor/distributor_test.go | 8 ++++++++ 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 88c71dd62886a..f871336e26e9d 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -167,10 +167,11 @@ type Distributor struct { RequestParserWrapper push.RequestParserWrapper // metrics - ingesterAppends *prometheus.CounterVec - ingesterAppendTimeouts *prometheus.CounterVec - replicationFactor prometheus.Gauge - streamShardCount prometheus.Counter + ingesterAppends *prometheus.CounterVec + ingesterAppendTimeouts *prometheus.CounterVec + replicationFactor prometheus.Gauge + streamShardCount prometheus.Counter + tenantPushSanitizedStructuredMetadata *prometheus.CounterVec usageTracker push.UsageTracker ingesterTasks chan pushIngesterTask @@ -284,6 +285,11 @@ func New( Name: "stream_sharding_count", Help: "Total number of times the distributor has sharded streams", }), + tenantPushSanitizedStructuredMetadata: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "distributor_push_structured_metadata_sanitized_total", + Help: "The total number of times we've had to sanitize structured metadata (names or values) at ingestion time per tenant.", + }, []string{"tenant"}), kafkaAppends: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{ Namespace: constants.Loki, Name: "distributor_kafka_appends_total", @@ -527,11 +533,17 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log continue } + var normalized string structuredMetadata := logproto.FromLabelAdaptersToLabels(entry.StructuredMetadata) for i := range entry.StructuredMetadata { - structuredMetadata[i].Name = otlptranslate.NormalizeLabel(structuredMetadata[i].Name) + normalized = otlptranslate.NormalizeLabel(structuredMetadata[i].Name) + if normalized != structuredMetadata[i].Name { + structuredMetadata[i].Name = normalized + d.tenantPushSanitizedStructuredMetadata.WithLabelValues(tenantID).Inc() + } if strings.ContainsRune(structuredMetadata[i].Value, utf8.RuneError) { structuredMetadata[i].Value = strings.Map(removeInvalidUtf, structuredMetadata[i].Value) + d.tenantPushSanitizedStructuredMetadata.WithLabelValues(tenantID).Inc() } } if shouldDiscoverLevels { diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index e03c675ebb1b5..3f735b42371ac 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -13,6 +13,8 @@ import ( "time" "unicode/utf8" + "github.com/prometheus/client_golang/prometheus/testutil" + otlptranslate "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus" "github.com/grafana/loki/pkg/push" @@ -1961,22 +1963,27 @@ func TestDistributor_StructuredMetadataSanitization(t *testing.T) { for _, tc := range []struct { req *logproto.PushRequest expectedResponse *logproto.PushResponse + numSanitizations float64 }{ { makeWriteRequestWithLabels(10, 10, []string{`{foo="bar"}`}, true, false, false), success, + 0, }, { makeWriteRequestWithLabels(10, 10, []string{`{foo="bar"}`}, true, true, false), success, + 10, }, { makeWriteRequestWithLabels(10, 10, []string{`{foo="bar"}`}, true, false, true), success, + 10, }, { makeWriteRequestWithLabels(10, 10, []string{`{foo="bar"}`}, true, true, true), success, + 20, }, } { distributors, _ := prepare(t, 1, 5, limits, nil) @@ -1988,5 +1995,6 @@ func TestDistributor_StructuredMetadataSanitization(t *testing.T) { response, err := distributors[0].Push(ctx, &request) require.NoError(t, err) assert.Equal(t, tc.expectedResponse, response) + assert.Equal(t, tc.numSanitizations, testutil.ToFloat64(distributors[0].tenantPushSanitizedStructuredMetadata.WithLabelValues("test"))) } } From e0cf6da4664fe0521cc9b43b76229701726e14e5 Mon Sep 17 00:00:00 2001 From: Ashwanth Date: Wed, 4 Dec 2024 10:10:33 +0530 Subject: [PATCH 21/44] chore: split kafka reader ifc into reader and offset manager (#15223) --- pkg/blockbuilder/builder/controller.go | 39 ++-- pkg/blockbuilder/builder/slimgester.go | 2 +- pkg/blockbuilder/scheduler/strategy_test.go | 2 +- pkg/kafka/partition/committer.go | 24 +-- pkg/kafka/partition/committer_test.go | 3 +- pkg/kafka/partition/offset_manager.go | 209 ++++++++++++++++++++ pkg/kafka/partition/reader.go | 165 ++-------------- pkg/kafka/partition/reader_service.go | 31 ++- pkg/loki/modules.go | 11 ++ 9 files changed, 291 insertions(+), 195 deletions(-) create mode 100644 pkg/kafka/partition/offset_manager.go diff --git a/pkg/blockbuilder/builder/controller.go b/pkg/blockbuilder/builder/controller.go index 60570daa912cb..29248e16d9525 100644 --- a/pkg/blockbuilder/builder/controller.go +++ b/pkg/blockbuilder/builder/controller.go @@ -56,15 +56,17 @@ type PartitionController interface { // // containing log data and "committed" is the consumer group type PartitionJobController struct { - stepLen int64 - part partition.Reader - backoff backoff.Config - decoder *kafka.Decoder - logger log.Logger + stepLen int64 + reader partition.Reader + offsetManager partition.OffsetManager + backoff backoff.Config + decoder *kafka.Decoder + logger log.Logger } func NewPartitionJobController( - controller partition.Reader, + reader partition.Reader, + offsetManager partition.OffsetManager, backoff backoff.Config, logger log.Logger, ) (*PartitionJobController, error) { @@ -73,14 +75,15 @@ func NewPartitionJobController( return nil, err } return &PartitionJobController{ - stepLen: 1000, // Default step length of 1000 offsets per job - part: controller, - backoff: backoff, - decoder: decoder, + stepLen: 1000, // Default step length of 1000 offsets per job + reader: reader, + offsetManager: offsetManager, + backoff: backoff, + decoder: decoder, logger: log.With(logger, "component", "job-controller", - "topic", controller.Topic(), - "partition", controller.Partition(), + "topic", offsetManager.Topic(), + "partition", offsetManager.Partition(), ), }, nil } @@ -90,7 +93,7 @@ func (l *PartitionJobController) HighestCommittedOffset(ctx context.Context) (in ctx, l.backoff, func() (int64, error) { - return l.part.FetchLastCommittedOffset(ctx) + return l.offsetManager.FetchLastCommittedOffset(ctx) }, ) } @@ -100,7 +103,7 @@ func (l *PartitionJobController) HighestPartitionOffset(ctx context.Context) (in ctx, l.backoff, func() (int64, error) { - return l.part.FetchPartitionOffset(ctx, partition.KafkaEndOffset) + return l.offsetManager.FetchPartitionOffset(ctx, partition.KafkaEndOffset) }, ) } @@ -110,13 +113,13 @@ func (l *PartitionJobController) EarliestPartitionOffset(ctx context.Context) (i ctx, l.backoff, func() (int64, error) { - return l.part.FetchPartitionOffset(ctx, partition.KafkaStartOffset) + return l.offsetManager.FetchPartitionOffset(ctx, partition.KafkaStartOffset) }, ) } func (l *PartitionJobController) Process(ctx context.Context, offsets types.Offsets, ch chan<- []AppendInput) (int64, error) { - l.part.SetOffsetForConsumption(offsets.Min) + l.reader.SetOffsetForConsumption(offsets.Min) var ( lastOffset = offsets.Min - 1 @@ -126,7 +129,7 @@ func (l *PartitionJobController) Process(ctx context.Context, offsets types.Offs for lastOffset < offsets.Max && boff.Ongoing() { var records []partition.Record - records, err = l.part.Poll(ctx, int(offsets.Max-lastOffset)) + records, err = l.reader.Poll(ctx, int(offsets.Max-lastOffset)) if err != nil { boff.Wait() continue @@ -217,7 +220,7 @@ func (l *PartitionJobController) LoadJob(ctx context.Context) (bool, *types.Job, } // Convert partition from int32 to int - job := types.NewJob(int(l.part.Partition()), offsets) + job := types.NewJob(int(l.reader.Partition()), offsets) return true, job, nil } diff --git a/pkg/blockbuilder/builder/slimgester.go b/pkg/blockbuilder/builder/slimgester.go index 1807892cbec05..510c2f9bd71d4 100644 --- a/pkg/blockbuilder/builder/slimgester.go +++ b/pkg/blockbuilder/builder/slimgester.go @@ -383,7 +383,7 @@ func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { return false, nil } - if err = i.jobController.part.Commit(ctx, lastOffset); err != nil { + if err = i.jobController.offsetManager.Commit(ctx, lastOffset); err != nil { level.Error(logger).Log( "msg", "failed to commit offset", "last_offset", lastOffset, diff --git a/pkg/blockbuilder/scheduler/strategy_test.go b/pkg/blockbuilder/scheduler/strategy_test.go index eb4704f268c74..d777113433f35 100644 --- a/pkg/blockbuilder/scheduler/strategy_test.go +++ b/pkg/blockbuilder/scheduler/strategy_test.go @@ -140,7 +140,7 @@ func TestTimeRangePlanner_Plan(t *testing.T) { require.NoError(t, err) require.Equal(t, len(tc.expectedJobs), len(jobs)) - require.Equal(t, tc.expectedJobs, jobs) + require.ElementsMatch(t, tc.expectedJobs, jobs) }) } } diff --git a/pkg/kafka/partition/committer.go b/pkg/kafka/partition/committer.go index eb401c47dafb1..6535c9c148b9c 100644 --- a/pkg/kafka/partition/committer.go +++ b/pkg/kafka/partition/committer.go @@ -25,34 +25,34 @@ type partitionCommitter struct { commitFailuresTotal prometheus.Counter lastCommittedOffset prometheus.Gauge - logger log.Logger - reader Reader - commitFreq time.Duration + logger log.Logger + offsetManager OffsetManager + commitFreq time.Duration toCommit *atomic.Int64 wg sync.WaitGroup cancel context.CancelFunc } -func newCommitter(reader Reader, commitFreq time.Duration, logger log.Logger, reg prometheus.Registerer) *partitionCommitter { +func newCommitter(offsetManager OffsetManager, commitFreq time.Duration, logger log.Logger, reg prometheus.Registerer) *partitionCommitter { c := &partitionCommitter{ - logger: logger, - reader: reader, - commitFreq: commitFreq, + logger: logger, + offsetManager: offsetManager, + commitFreq: commitFreq, 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(reader.Partition()))}, + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(offsetManager.Partition()))}, }), 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(reader.Partition()))}, + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(offsetManager.Partition()))}, }), 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(reader.Partition()))}, + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(offsetManager.Partition()))}, NativeHistogramBucketFactor: 1.1, NativeHistogramMaxBucketNumber: 100, NativeHistogramMinResetDuration: time.Hour, @@ -61,7 +61,7 @@ func newCommitter(reader Reader, commitFreq time.Duration, logger log.Logger, re 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(reader.Partition()))}, + ConstLabels: prometheus.Labels{"partition": strconv.Itoa(int(offsetManager.Partition()))}, }), toCommit: atomic.NewInt64(-1), } @@ -116,7 +116,7 @@ func (c *partitionCommitter) Commit(ctx context.Context, offset int64) error { startTime := time.Now() c.commitRequestsTotal.Inc() - if err := c.reader.Commit(ctx, offset); err != nil { + if err := c.offsetManager.Commit(ctx, offset); err != nil { level.Error(c.logger).Log("msg", "failed to commit offset", "err", err, "offset", offset) c.commitFailuresTotal.Inc() c.commitRequestsLatency.Observe(time.Since(startTime).Seconds()) diff --git a/pkg/kafka/partition/committer_test.go b/pkg/kafka/partition/committer_test.go index c67f8bb3a6237..0ca2bd39214f5 100644 --- a/pkg/kafka/partition/committer_test.go +++ b/pkg/kafka/partition/committer_test.go @@ -36,13 +36,12 @@ func TestPartitionCommitter(t *testing.T) { reg := prometheus.NewRegistry() partitionID := int32(1) consumerGroup := "test-consumer-group" - reader := newKafkaReader( + reader := newKafkaOffsetManager( client, kafkaCfg.Topic, partitionID, consumerGroup, logger, - reg, ) committer := newCommitter(reader, kafkaCfg.ConsumerGroupOffsetCommitInterval, logger, reg) diff --git a/pkg/kafka/partition/offset_manager.go b/pkg/kafka/partition/offset_manager.go new file mode 100644 index 0000000000000..89bff6f6e82a6 --- /dev/null +++ b/pkg/kafka/partition/offset_manager.go @@ -0,0 +1,209 @@ +package partition + +import ( + "context" + "errors" + "fmt" + + "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" + + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/kafka/client" +) + +type OffsetManager interface { + Topic() string + Partition() int32 + ConsumerGroup() string + + FetchLastCommittedOffset(ctx context.Context) (int64, error) + FetchPartitionOffset(ctx context.Context, position SpecialOffset) (int64, error) + Commit(ctx context.Context, offset int64) error +} + +var _ OffsetManager = &KafkaOffsetManager{} + +type KafkaOffsetManager struct { + client *kgo.Client + topic string + partitionID int32 + consumerGroup string + logger log.Logger +} + +func NewKafkaOffsetManager( + cfg kafka.Config, + partitionID int32, + instanceID string, + logger log.Logger, + reg prometheus.Registerer, +) (*KafkaOffsetManager, error) { + // Create a new Kafka client for the partition manager. + clientMetrics := client.NewReaderClientMetrics("partition-manager", 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) + } + + return newKafkaOffsetManager( + c, + cfg.Topic, + partitionID, + cfg.GetConsumerGroup(instanceID, partitionID), + logger, + ), nil +} + +// newKafkaReader creates a new KafkaReader instance +func newKafkaOffsetManager( + client *kgo.Client, + topic string, + partitionID int32, + consumerGroup string, + logger log.Logger, +) *KafkaOffsetManager { + return &KafkaOffsetManager{ + client: client, + topic: topic, + partitionID: partitionID, + consumerGroup: consumerGroup, + logger: logger, + } +} + +// Topic returns the topic being read +func (r *KafkaOffsetManager) Topic() string { + return r.topic +} + +// Partition returns the partition being read +func (r *KafkaOffsetManager) Partition() int32 { + return r.partitionID +} + +// ConsumerGroup returns the consumer group +func (r *KafkaOffsetManager) ConsumerGroup() string { + return r.consumerGroup +} + +// FetchLastCommittedOffset retrieves the last committed offset for this partition +func (r *KafkaOffsetManager) FetchLastCommittedOffset(ctx context.Context) (int64, error) { + req := kmsg.NewPtrOffsetFetchRequest() + req.Topics = []kmsg.OffsetFetchRequestTopic{{ + Topic: r.topic, + Partitions: []int32{r.partitionID}, + }} + req.Group = r.consumerGroup + + 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 { + return 0, fmt.Errorf("unexpected number of responses: %d", len(resps)) + } + + // Ensure no error occurred. + res := resps[0] + if res.Err != nil { + return 0, res.Err + } + + // Parse the response. + fetchRes, ok := res.Resp.(*kmsg.OffsetFetchResponse) + if !ok { + return 0, errors.New("unexpected response type") + } + + if len(fetchRes.Groups) != 1 || + len(fetchRes.Groups[0].Topics) != 1 || + len(fetchRes.Groups[0].Topics[0].Partitions) != 1 { + level.Debug(r.logger).Log( + "msg", "malformed response, setting to start offset", + ) + return int64(KafkaStartOffset), nil + } + + partition := fetchRes.Groups[0].Topics[0].Partitions[0] + if err := kerr.ErrorForCode(partition.ErrorCode); err != nil { + return 0, err + } + + return partition.Offset, nil +} + +// FetchPartitionOffset retrieves the offset for a specific position +func (r *KafkaOffsetManager) FetchPartitionOffset(ctx context.Context, position SpecialOffset) (int64, error) { + partitionReq := kmsg.NewListOffsetsRequestTopicPartition() + partitionReq.Partition = r.partitionID + partitionReq.Timestamp = int64(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 len(resps) != 1 { + return 0, fmt.Errorf("unexpected number of responses: %d", len(resps)) + } + + // Ensure no error occurred. + res := resps[0] + if res.Err != nil { + return 0, res.Err + } + + listRes, ok := res.Resp.(*kmsg.ListOffsetsResponse) + if !ok { + return 0, errors.New("unexpected response type") + } + + if len(listRes.Topics) != 1 || + len(listRes.Topics[0].Partitions) != 1 { + return 0, errors.New("malformed response") + } + + partition := listRes.Topics[0].Partitions[0] + if err := kerr.ErrorForCode(partition.ErrorCode); err != nil { + return 0, err + } + + return partition.Offset, nil +} + +// Commit commits an offset to the consumer group +func (r *KafkaOffsetManager) Commit(ctx context.Context, offset int64) error { + admin := kadm.NewClient(r.client) + + // Commit the last consumed offset. + toCommit := kadm.Offsets{} + toCommit.AddOffset(r.topic, r.partitionID, offset, -1) + + committed, err := admin.CommitOffsets(ctx, r.consumerGroup, 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) + return nil +} diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index 1aa79c08fdf79..15e78c1946dc3 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -7,15 +7,11 @@ import ( "time" "github.com/go-kit/log" - "github.com/go-kit/log/level" "github.com/grafana/dskit/multierror" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "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" @@ -29,6 +25,12 @@ const ( KafkaEndOffset SpecialOffset = -1 ) +var rm *readerMetrics + +func init() { + rm = newReaderMetrics(prometheus.DefaultRegisterer) +} + type Record struct { // Context holds the tracing (and potentially other) info, that the record was enriched with on fetch from Kafka. Ctx context.Context @@ -40,11 +42,7 @@ type Record struct { type Reader interface { Topic() string Partition() int32 - ConsumerGroup() string - FetchLastCommittedOffset(ctx context.Context) (int64, error) - FetchPartitionOffset(ctx context.Context, position SpecialOffset) (int64, error) Poll(ctx context.Context, maxPollRecords int) ([]Record, error) - Commit(ctx context.Context, offset int64) error // Set the target offset for consumption. reads will begin from here. SetOffsetForConsumption(offset int64) } @@ -104,7 +102,6 @@ type KafkaReader struct { func NewKafkaReader( cfg kafka.Config, partitionID int32, - instanceID string, logger log.Logger, reg prometheus.Registerer, ) (*KafkaReader, error) { @@ -119,34 +116,13 @@ func NewKafkaReader( return nil, fmt.Errorf("creating kafka client: %w", err) } - // Create the reader - return newKafkaReader( - c, - cfg.Topic, - partitionID, - cfg.GetConsumerGroup(instanceID, partitionID), - logger, - reg, - ), nil -} - -// newKafkaReader creates a new KafkaReader instance -func newKafkaReader( - client *kgo.Client, - topic string, - partitionID int32, - consumerGroup string, - logger log.Logger, - reg prometheus.Registerer, -) *KafkaReader { return &KafkaReader{ - client: client, - topic: topic, - partitionID: partitionID, - consumerGroup: consumerGroup, - metrics: newReaderMetrics(reg), - logger: logger, - } + client: c, + topic: cfg.Topic, + partitionID: partitionID, + metrics: rm, + logger: logger, + }, nil } // Topic returns the topic being read @@ -159,103 +135,6 @@ func (r *KafkaReader) Partition() int32 { return r.partitionID } -// ConsumerGroup returns the consumer group -func (r *KafkaReader) ConsumerGroup() string { - return r.consumerGroup -} - -// FetchLastCommittedOffset retrieves the last committed offset for this partition -func (r *KafkaReader) FetchLastCommittedOffset(ctx context.Context) (int64, error) { - req := kmsg.NewPtrOffsetFetchRequest() - req.Topics = []kmsg.OffsetFetchRequestTopic{{ - Topic: r.topic, - Partitions: []int32{r.partitionID}, - }} - req.Group = r.consumerGroup - - 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 { - return 0, fmt.Errorf("unexpected number of responses: %d", len(resps)) - } - - // Ensure no error occurred. - res := resps[0] - if res.Err != nil { - return 0, res.Err - } - - // Parse the response. - fetchRes, ok := res.Resp.(*kmsg.OffsetFetchResponse) - if !ok { - return 0, errors.New("unexpected response type") - } - - if len(fetchRes.Groups) != 1 || - len(fetchRes.Groups[0].Topics) != 1 || - len(fetchRes.Groups[0].Topics[0].Partitions) != 1 { - level.Debug(r.logger).Log( - "msg", "malformed response, setting to start offset", - ) - return int64(KafkaStartOffset), nil - } - - partition := fetchRes.Groups[0].Topics[0].Partitions[0] - if err := kerr.ErrorForCode(partition.ErrorCode); err != nil { - return 0, err - } - - return partition.Offset, nil -} - -// FetchPartitionOffset retrieves the offset for a specific position -func (r *KafkaReader) FetchPartitionOffset(ctx context.Context, position SpecialOffset) (int64, error) { - partitionReq := kmsg.NewListOffsetsRequestTopicPartition() - partitionReq.Partition = r.partitionID - partitionReq.Timestamp = int64(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 len(resps) != 1 { - return 0, fmt.Errorf("unexpected number of responses: %d", len(resps)) - } - - // Ensure no error occurred. - res := resps[0] - if res.Err != nil { - return 0, res.Err - } - - listRes, ok := res.Resp.(*kmsg.ListOffsetsResponse) - if !ok { - return 0, errors.New("unexpected response type") - } - - if len(listRes.Topics) != 1 || - len(listRes.Topics[0].Partitions) != 1 { - return 0, errors.New("malformed response") - } - - partition := listRes.Topics[0].Partitions[0] - if err := kerr.ErrorForCode(partition.ErrorCode); err != nil { - return 0, err - } - - return partition.Offset, nil -} - // Poll retrieves the next batch of records from Kafka // Number of records fetched can be limited by configuring maxPollRecords to a non-zero value. func (r *KafkaReader) Poll(ctx context.Context, maxPollRecords int) ([]Record, error) { @@ -309,23 +188,3 @@ func (r *KafkaReader) SetOffsetForConsumption(offset int64) { r.topic: {r.partitionID: kgo.NewOffset().At(offset)}, }) } - -// Commit commits an offset to the consumer group -func (r *KafkaReader) Commit(ctx context.Context, offset int64) error { - admin := kadm.NewClient(r.client) - - // Commit the last consumed offset. - toCommit := kadm.Offsets{} - toCommit.AddOffset(r.topic, r.partitionID, offset, -1) - - committed, err := admin.CommitOffsets(ctx, r.consumerGroup, 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) - return nil -} diff --git a/pkg/kafka/partition/reader_service.go b/pkg/kafka/partition/reader_service.go index 40c1b64b3b790..4fba065c94d1d 100644 --- a/pkg/kafka/partition/reader_service.go +++ b/pkg/kafka/partition/reader_service.go @@ -50,6 +50,7 @@ type ReaderService struct { cfg ReaderConfig reader Reader + offsetManager OffsetManager consumerFactory ConsumerFactory logger log.Logger metrics *serviceMetrics @@ -76,19 +77,31 @@ func NewReaderService( reader, err := NewKafkaReader( kafkaCfg, partitionID, - instanceID, logger, reg, ) if err != nil { return nil, fmt.Errorf("creating kafka reader: %w", err) } + + offsetManager, err := NewKafkaOffsetManager( + kafkaCfg, + partitionID, + instanceID, + logger, + reg, + ) + if err != nil { + return nil, fmt.Errorf("creating kafka offset manager: %w", err) + } + return newReaderService( ReaderConfig{ MaxConsumerLagAtStartup: kafkaCfg.MaxConsumerLagAtStartup, ConsumerGroupOffsetCommitFreq: kafkaCfg.ConsumerGroupOffsetCommitInterval, }, reader, + offsetManager, consumerFactory, logger, reg, @@ -98,6 +111,7 @@ func NewReaderService( func newReaderService( cfg ReaderConfig, reader Reader, + offsetManager OffsetManager, consumerFactory ConsumerFactory, logger log.Logger, reg prometheus.Registerer, @@ -105,14 +119,15 @@ func newReaderService( s := &ReaderService{ cfg: cfg, reader: reader, + offsetManager: offsetManager, consumerFactory: consumerFactory, - logger: log.With(logger, "partition", reader.Partition(), "consumer_group", reader.ConsumerGroup()), + logger: log.With(logger, "partition", offsetManager.Partition(), "consumer_group", offsetManager.ConsumerGroup()), metrics: newServiceMetrics(reg), lastProcessedOffset: int64(KafkaEndOffset), } // Create the committer - s.committer = newCommitter(reader, cfg.ConsumerGroupOffsetCommitFreq, logger, reg) + s.committer = newCommitter(offsetManager, cfg.ConsumerGroupOffsetCommitFreq, logger, reg) s.Service = services.NewBasicService(s.starting, s.running, nil) return s @@ -120,13 +135,13 @@ func newReaderService( func (s *ReaderService) starting(ctx context.Context) error { level.Info(s.logger).Log("msg", "starting reader service") - s.metrics.reportOwnerOfPartition(s.reader.Partition()) + s.metrics.reportOwnerOfPartition(s.offsetManager.Partition()) s.metrics.reportStarting() logger := log.With(s.logger, "phase", phaseStarting) // Fetch the last committed offset to determine where to start reading - lastCommittedOffset, err := s.reader.FetchLastCommittedOffset(ctx) + lastCommittedOffset, err := s.offsetManager.FetchLastCommittedOffset(ctx) if err != nil { return fmt.Errorf("fetching last committed offset: %w", err) } @@ -219,14 +234,14 @@ func (s *ReaderService) fetchUntilLagSatisfied( for b.Ongoing() { // Send a direct request to the Kafka backend to fetch the partition start offset. - partitionStartOffset, err := s.reader.FetchPartitionOffset(ctx, KafkaStartOffset) + partitionStartOffset, err := s.offsetManager.FetchPartitionOffset(ctx, KafkaStartOffset) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch partition start offset", "err", err) b.Wait() continue } - consumerGroupLastCommittedOffset, err := s.reader.FetchLastCommittedOffset(ctx) + consumerGroupLastCommittedOffset, err := s.offsetManager.FetchLastCommittedOffset(ctx) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch last committed offset", "err", err) b.Wait() @@ -237,7 +252,7 @@ func (s *ReaderService) fetchUntilLagSatisfied( // We intentionally don't use WaitNextFetchLastProducedOffset() to not introduce further // latency. lastProducedOffsetRequestedAt := time.Now() - lastProducedOffset, err := s.reader.FetchPartitionOffset(ctx, KafkaEndOffset) + lastProducedOffset, err := s.offsetManager.FetchPartitionOffset(ctx, KafkaEndOffset) if err != nil { level.Warn(logger).Log("msg", "partition reader failed to fetch last produced offset", "err", err) b.Wait() diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index c4449f3c51134..6a88f719c629a 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1822,6 +1822,16 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { } reader, err := partition.NewKafkaReader( + t.Cfg.KafkaConfig, + ingestPartitionID, + logger, + prometheus.DefaultRegisterer, + ) + if err != nil { + return nil, err + } + + offsetManager, err := partition.NewKafkaOffsetManager( t.Cfg.KafkaConfig, ingestPartitionID, id, @@ -1834,6 +1844,7 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { controller, err := blockbuilder.NewPartitionJobController( reader, + offsetManager, t.Cfg.BlockBuilder.Backoff, logger, ) From d59a5e2cfbf5dab3d2af1d2d4f3dcb9f272cc48b Mon Sep 17 00:00:00 2001 From: Ashwanth Date: Wed, 4 Dec 2024 13:46:24 +0530 Subject: [PATCH 22/44] chore: pass down reader metrics to avoid duplicate registration (#15246) --- pkg/kafka/partition/reader.go | 26 +++++++++++--------------- pkg/kafka/partition/reader_service.go | 3 ++- pkg/loki/modules.go | 3 ++- 3 files changed, 15 insertions(+), 17 deletions(-) diff --git a/pkg/kafka/partition/reader.go b/pkg/kafka/partition/reader.go index 15e78c1946dc3..58f9e77c4b05e 100644 --- a/pkg/kafka/partition/reader.go +++ b/pkg/kafka/partition/reader.go @@ -12,6 +12,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/plugin/kprom" "github.com/grafana/loki/v3/pkg/kafka" @@ -25,12 +26,6 @@ const ( KafkaEndOffset SpecialOffset = -1 ) -var rm *readerMetrics - -func init() { - rm = newReaderMetrics(prometheus.DefaultRegisterer) -} - type Record struct { // Context holds the tracing (and potentially other) info, that the record was enriched with on fetch from Kafka. Ctx context.Context @@ -47,18 +42,19 @@ type Reader interface { SetOffsetForConsumption(offset int64) } -// readerMetrics contains metrics specific to Kafka reading operations -type readerMetrics struct { +// ReaderMetrics contains metrics specific to Kafka reading operations +type ReaderMetrics struct { recordsPerFetch prometheus.Histogram fetchesErrors prometheus.Counter fetchesTotal prometheus.Counter fetchWaitDuration prometheus.Histogram receiveDelay prometheus.Histogram lastCommittedOffset prometheus.Gauge + kprom *kprom.Metrics } -func newReaderMetrics(r prometheus.Registerer) *readerMetrics { - return &readerMetrics{ +func NewReaderMetrics(r prometheus.Registerer) *ReaderMetrics { + return &ReaderMetrics{ fetchWaitDuration: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Name: "loki_kafka_reader_fetch_wait_duration_seconds", Help: "How long the reader spent waiting for a batch of records from Kafka.", @@ -86,6 +82,7 @@ func newReaderMetrics(r prometheus.Registerer) *readerMetrics { NativeHistogramMinResetDuration: 1 * time.Hour, Buckets: prometheus.ExponentialBuckets(0.125, 2, 18), }), + kprom: client.NewReaderClientMetrics("partition-reader", r), } } @@ -95,7 +92,7 @@ type KafkaReader struct { topic string partitionID int32 consumerGroup string - metrics *readerMetrics + metrics *ReaderMetrics logger log.Logger } @@ -103,13 +100,12 @@ func NewKafkaReader( cfg kafka.Config, partitionID int32, logger log.Logger, - reg prometheus.Registerer, + metrics *ReaderMetrics, ) (*KafkaReader, error) { // Create a new Kafka client for this reader - clientMetrics := client.NewReaderClientMetrics("partition-reader", reg) c, err := client.NewReaderClient( cfg, - clientMetrics, + metrics.kprom, log.With(logger, "component", "kafka-client"), ) if err != nil { @@ -120,7 +116,7 @@ func NewKafkaReader( client: c, topic: cfg.Topic, partitionID: partitionID, - metrics: rm, + metrics: metrics, logger: logger, }, nil } diff --git a/pkg/kafka/partition/reader_service.go b/pkg/kafka/partition/reader_service.go index 4fba065c94d1d..5c67cde5fde2a 100644 --- a/pkg/kafka/partition/reader_service.go +++ b/pkg/kafka/partition/reader_service.go @@ -74,11 +74,12 @@ func NewReaderService( logger log.Logger, reg prometheus.Registerer, ) (*ReaderService, error) { + readerMetrics := NewReaderMetrics(reg) reader, err := NewKafkaReader( kafkaCfg, partitionID, logger, - reg, + readerMetrics, ) if err != nil { return nil, fmt.Errorf("creating kafka reader: %w", err) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 6a88f719c629a..fb5c800cc03e1 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1821,11 +1821,12 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { return nil, fmt.Errorf("calculating block builder partition ID: %w", err) } + readerMetrics := partition.NewReaderMetrics(prometheus.DefaultRegisterer) reader, err := partition.NewKafkaReader( t.Cfg.KafkaConfig, ingestPartitionID, logger, - prometheus.DefaultRegisterer, + readerMetrics, ) if err != nil { return nil, err From 098127390fa89b5aebaa52f5076a9b95bbe1083c Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 Dec 2024 00:33:45 -0800 Subject: [PATCH 23/44] feat(blockbuilder): priority queue for job dispatching (#15245) --- .../scheduler/prioritiy_queue_test.go | 193 ++++++++++++++++++ pkg/blockbuilder/scheduler/priority_queue.go | 126 ++++++++++++ pkg/blockbuilder/scheduler/queue.go | 149 ++++++++------ pkg/blockbuilder/scheduler/scheduler.go | 12 +- pkg/blockbuilder/scheduler/scheduler_test.go | 6 +- pkg/blockbuilder/scheduler/strategy.go | 47 +++-- pkg/blockbuilder/scheduler/strategy_test.go | 56 ++--- 7 files changed, 461 insertions(+), 128 deletions(-) create mode 100644 pkg/blockbuilder/scheduler/prioritiy_queue_test.go create mode 100644 pkg/blockbuilder/scheduler/priority_queue.go diff --git a/pkg/blockbuilder/scheduler/prioritiy_queue_test.go b/pkg/blockbuilder/scheduler/prioritiy_queue_test.go new file mode 100644 index 0000000000000..b27d950aa04b0 --- /dev/null +++ b/pkg/blockbuilder/scheduler/prioritiy_queue_test.go @@ -0,0 +1,193 @@ +package scheduler + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestPriorityQueue(t *testing.T) { + t.Run("operations", func(t *testing.T) { + tests := []struct { + name string + input []int + wantPops []int + }{ + { + name: "empty queue", + input: []int{}, + wantPops: []int{}, + }, + { + name: "single element", + input: []int{1}, + wantPops: []int{1}, + }, + { + name: "multiple elements in order", + input: []int{1, 2, 3}, + wantPops: []int{1, 2, 3}, + }, + { + name: "multiple elements out of order", + input: []int{3, 1, 2}, + wantPops: []int{1, 2, 3}, + }, + { + name: "duplicate elements", + input: []int{2, 1, 2, 1}, + wantPops: []int{1, 1, 2, 2}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + pq := NewPriorityQueue[int](func(a, b int) bool { return a < b }) + require.Equal(t, 0, pq.Len()) + + // Push all elements + for _, v := range tt.input { + pq.Push(v) + } + require.Equal(t, len(tt.input), pq.Len()) + + // Pop all elements and verify order + got := make([]int, 0, len(tt.input)) + for range tt.input { + v, ok := pq.Pop() + require.True(t, ok) + got = append(got, v) + } + require.Equal(t, tt.wantPops, got) + + // Verify empty queue behavior + v, ok := pq.Pop() + require.False(t, ok) + require.Zero(t, v) + require.Equal(t, 0, pq.Len()) + }) + } + }) + + t.Run("custom type", func(t *testing.T) { + type Job struct { + ID string + Priority int + } + + pq := NewPriorityQueue[Job](func(a, b Job) bool { + return a.Priority < b.Priority + }) + + jobs := []Job{ + {ID: "high", Priority: 3}, + {ID: "low", Priority: 1}, + {ID: "medium", Priority: 2}, + } + + // Push all jobs + for _, j := range jobs { + pq.Push(j) + } + + // Verify they come out in priority order + want := []string{"low", "medium", "high"} + got := make([]string, 0, len(jobs)) + for range jobs { + j, ok := pq.Pop() + require.True(t, ok) + got = append(got, j.ID) + } + require.Equal(t, want, got) + }) + + t.Run("mixed operations", func(t *testing.T) { + pq := NewPriorityQueue[int](func(a, b int) bool { return a < b }) + + // Push some elements + pq.Push(3) + pq.Push(1) + require.Equal(t, 2, pq.Len()) + + // Pop lowest + v, ok := pq.Pop() + require.True(t, ok) + require.Equal(t, 1, v) + + // Push more elements + pq.Push(2) + pq.Push(4) + + // Verify remaining elements come out in order + want := []int{2, 3, 4} + got := make([]int, 0, 3) + for range want { + v, ok := pq.Pop() + require.True(t, ok) + got = append(got, v) + } + require.Equal(t, want, got) + }) +} + +func TestCircularBuffer(t *testing.T) { + tests := []struct { + name string + capacity int + input []int + wantPops []int + }{ + { + name: "empty buffer", + capacity: 5, + input: []int{}, + wantPops: []int{}, + }, + { + name: "partial fill", + capacity: 5, + input: []int{1, 2, 3}, + wantPops: []int{1, 2, 3}, + }, + { + name: "full buffer", + capacity: 3, + input: []int{1, 2, 3}, + wantPops: []int{1, 2, 3}, + }, + { + name: "overflow buffer", + capacity: 3, + input: []int{1, 2, 3, 4, 5}, + wantPops: []int{3, 4, 5}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + cb := NewCircularBuffer[int](tt.capacity) + require.Equal(t, 0, cb.Len()) + + // Push all elements + for _, v := range tt.input { + cb.Push(v) + } + require.Equal(t, min(tt.capacity, len(tt.input)), cb.Len()) + + // Pop all elements and verify order + got := make([]int, 0, cb.Len()) + for cb.Len() > 0 { + v, ok := cb.Pop() + require.True(t, ok) + got = append(got, v) + } + require.Equal(t, tt.wantPops, got) + + // Verify empty buffer behavior + v, ok := cb.Pop() + require.False(t, ok) + require.Zero(t, v) + require.Equal(t, 0, cb.Len()) + }) + } +} diff --git a/pkg/blockbuilder/scheduler/priority_queue.go b/pkg/blockbuilder/scheduler/priority_queue.go new file mode 100644 index 0000000000000..3b488716cabe8 --- /dev/null +++ b/pkg/blockbuilder/scheduler/priority_queue.go @@ -0,0 +1,126 @@ +package scheduler + +import ( + "container/heap" +) + +// PriorityQueue is a generic priority queue. +type PriorityQueue[T any] struct { + h *priorityHeap[T] +} + +// NewPriorityQueue creates a new priority queue. +func NewPriorityQueue[T any](less func(T, T) bool) *PriorityQueue[T] { + h := &priorityHeap[T]{ + less: less, + heap: make([]T, 0), + } + heap.Init(h) + return &PriorityQueue[T]{h: h} +} + +// Push adds an element to the queue. +func (pq *PriorityQueue[T]) Push(v T) { + heap.Push(pq.h, v) +} + +// Pop removes and returns the element with the highest priority from the queue. +func (pq *PriorityQueue[T]) Pop() (T, bool) { + if pq.Len() == 0 { + var zero T + return zero, false + } + return heap.Pop(pq.h).(T), true +} + +// Len returns the number of elements in the queue. +func (pq *PriorityQueue[T]) Len() int { + return pq.h.Len() +} + +// priorityHeap is the internal heap implementation that satisfies heap.Interface. +type priorityHeap[T any] struct { + less func(T, T) bool + heap []T +} + +func (h *priorityHeap[T]) Len() int { + return len(h.heap) +} + +func (h *priorityHeap[T]) Less(i, j int) bool { + return h.less(h.heap[i], h.heap[j]) +} + +func (h *priorityHeap[T]) Swap(i, j int) { + h.heap[i], h.heap[j] = h.heap[j], h.heap[i] +} + +func (h *priorityHeap[T]) Push(x any) { + h.heap = append(h.heap, x.(T)) +} + +func (h *priorityHeap[T]) Pop() any { + old := h.heap + n := len(old) + x := old[n-1] + h.heap = old[0 : n-1] + return x +} + +// CircularBuffer is a generic circular buffer. +type CircularBuffer[T any] struct { + buffer []T + size int + head int + tail int +} + +// NewCircularBuffer creates a new circular buffer with the given capacity. +func NewCircularBuffer[T any](capacity int) *CircularBuffer[T] { + return &CircularBuffer[T]{ + buffer: make([]T, capacity), + size: 0, + head: 0, + tail: 0, + } +} + +// Push adds an element to the circular buffer and returns the evicted element if any +func (b *CircularBuffer[T]) Push(v T) (T, bool) { + var evicted T + hasEvicted := false + + if b.size == len(b.buffer) { + // If buffer is full, evict the oldest element (at head) + evicted = b.buffer[b.head] + hasEvicted = true + b.head = (b.head + 1) % len(b.buffer) + } else { + b.size++ + } + + b.buffer[b.tail] = v + b.tail = (b.tail + 1) % len(b.buffer) + + return evicted, hasEvicted +} + +// Pop removes and returns the oldest element from the buffer +func (b *CircularBuffer[T]) Pop() (T, bool) { + if b.size == 0 { + var zero T + return zero, false + } + + v := b.buffer[b.head] + b.head = (b.head + 1) % len(b.buffer) + b.size-- + + return v, true +} + +// Len returns the number of elements in the buffer +func (b *CircularBuffer[T]) Len() int { + return b.size +} diff --git a/pkg/blockbuilder/scheduler/queue.go b/pkg/blockbuilder/scheduler/queue.go index e2f125ad70a07..dab46f164908d 100644 --- a/pkg/blockbuilder/scheduler/queue.go +++ b/pkg/blockbuilder/scheduler/queue.go @@ -3,30 +3,58 @@ package scheduler import ( "fmt" "sync" + "time" "github.com/grafana/loki/v3/pkg/blockbuilder/types" ) -// jobAssignment tracks a job and its assigned builder -type jobAssignment struct { +const ( + defaultCompletedJobsCapacity = 100 +) + +// JobWithPriority wraps a job with a priority value +type JobWithPriority[T comparable] struct { + Job *types.Job + Priority T +} + +// NewJobWithPriority creates a new JobWithPriority instance +func NewJobWithPriority[T comparable](job *types.Job, priority T) *JobWithPriority[T] { + return &JobWithPriority[T]{ + Job: job, + Priority: priority, + } +} + +// inProgressJob contains a job and its start time +type inProgressJob struct { job *types.Job - builderID string + startTime time.Time +} + +// Duration returns how long the job has been running +func (j *inProgressJob) Duration() time.Duration { + return time.Since(j.startTime) } // JobQueue manages the queue of pending jobs and tracks their state. type JobQueue struct { - pending map[string]*types.Job // Jobs waiting to be processed, key is job ID - inProgress map[string]*jobAssignment // job ID -> assignment info - completed map[string]*types.Job // Completed jobs, key is job ID + pending *PriorityQueue[*JobWithPriority[int]] // Jobs waiting to be processed, ordered by priority + inProgress map[string]*inProgressJob // Jobs currently being processed, key is job ID + completed *CircularBuffer[*types.Job] // Last N completed jobs + statusMap map[string]types.JobStatus // Maps job ID to its current status mu sync.RWMutex } // NewJobQueue creates a new job queue instance func NewJobQueue() *JobQueue { return &JobQueue{ - pending: make(map[string]*types.Job), - inProgress: make(map[string]*jobAssignment), - completed: make(map[string]*types.Job), + pending: NewPriorityQueue[*JobWithPriority[int]](func(a, b *JobWithPriority[int]) bool { + return a.Priority > b.Priority // Higher priority first + }), + inProgress: make(map[string]*inProgressJob), + completed: NewCircularBuffer[*types.Job](defaultCompletedJobsCapacity), + statusMap: make(map[string]types.JobStatus), } } @@ -34,92 +62,81 @@ func (q *JobQueue) Exists(job *types.Job) (types.JobStatus, bool) { q.mu.RLock() defer q.mu.RUnlock() - if _, ok := q.inProgress[job.ID]; ok { - return types.JobStatusInProgress, true - } - - if _, ok := q.pending[job.ID]; ok { - return types.JobStatusPending, true - } - - if _, ok := q.completed[job.ID]; ok { - return types.JobStatusComplete, true - } - - return -1, false + status, exists := q.statusMap[job.ID] + return status, exists } -// Enqueue adds a new job to the pending queue -// This is a naive implementation, intended to be refactored -func (q *JobQueue) Enqueue(job *types.Job) error { +// Enqueue adds a new job to the pending queue with a priority +func (q *JobQueue) Enqueue(job *types.Job, priority int) error { q.mu.Lock() defer q.mu.Unlock() - if _, exists := q.pending[job.ID]; exists { - return fmt.Errorf("job %s already exists in pending queue", job.ID) - } - if _, exists := q.inProgress[job.ID]; exists { - return fmt.Errorf("job %s already exists in progress", job.ID) - } - if _, exists := q.completed[job.ID]; exists { - return fmt.Errorf("job %s already completed", job.ID) + // Check if job already exists + if status, exists := q.statusMap[job.ID]; exists { + return fmt.Errorf("job %s already exists with status %v", job.ID, status) } - q.pending[job.ID] = job + jobWithPriority := NewJobWithPriority(job, priority) + q.pending.Push(jobWithPriority) + q.statusMap[job.ID] = types.JobStatusPending return nil } // Dequeue gets the next available job and assigns it to a builder -func (q *JobQueue) Dequeue(builderID string) (*types.Job, bool, error) { +func (q *JobQueue) Dequeue(_ string) (*types.Job, bool, error) { q.mu.Lock() defer q.mu.Unlock() - // Simple FIFO for now - for id, job := range q.pending { - delete(q.pending, id) - q.inProgress[id] = &jobAssignment{ - job: job, - builderID: builderID, - } - return job, true, nil + if q.pending.Len() == 0 { + return nil, false, nil + } + + jobWithPriority, ok := q.pending.Pop() + if !ok { + return nil, false, nil + } + + // Add to in-progress with current time + q.inProgress[jobWithPriority.Job.ID] = &inProgressJob{ + job: jobWithPriority.Job, + startTime: time.Now(), } + q.statusMap[jobWithPriority.Job.ID] = types.JobStatusInProgress - return nil, false, nil + return jobWithPriority.Job, true, nil } // MarkComplete moves a job from in-progress to completed -func (q *JobQueue) MarkComplete(jobID string, builderID string) error { +func (q *JobQueue) MarkComplete(jobID string) { q.mu.Lock() defer q.mu.Unlock() - assignment, exists := q.inProgress[jobID] - if !exists { - return fmt.Errorf("job %s not found in progress", jobID) + // Find job in in-progress map + inProgressJob, exists := q.inProgress[jobID] + // if it doesn't exist, it could be previously removed (duplicate job execution) + // or the scheduler may have restarted and not have the job state anymore. + if exists { + // Remove from in-progress + delete(q.inProgress, jobID) } - if assignment.builderID != builderID { - return fmt.Errorf("job %s not assigned to builder %s", jobID, builderID) + // Add to completed buffer and handle evicted job + if evictedJob, hasEvicted := q.completed.Push(inProgressJob.job); hasEvicted { + // Remove evicted job from status map + delete(q.statusMap, evictedJob.ID) } - - delete(q.inProgress, jobID) - q.completed[jobID] = assignment.job - return nil + q.statusMap[jobID] = types.JobStatusComplete } -// SyncJob updates the state of an in-progress job -func (q *JobQueue) SyncJob(jobID string, builderID string, job *types.Job) error { +// SyncJob registers a job as in-progress, used for restoring state after scheduler restarts +func (q *JobQueue) SyncJob(jobID string, _ string, job *types.Job) { q.mu.Lock() defer q.mu.Unlock() - assignment, exists := q.inProgress[jobID] - if !exists { - return fmt.Errorf("job %s not found in progress", jobID) - } - - if assignment.builderID != builderID { - return fmt.Errorf("job %s not assigned to builder %s", jobID, builderID) + // Add directly to in-progress + q.inProgress[jobID] = &inProgressJob{ + job: job, + startTime: time.Now(), } - - assignment.job = job - return nil + q.statusMap[jobID] = types.JobStatusInProgress } diff --git a/pkg/blockbuilder/scheduler/scheduler.go b/pkg/blockbuilder/scheduler/scheduler.go index dbf732742de39..96356515a921f 100644 --- a/pkg/blockbuilder/scheduler/scheduler.go +++ b/pkg/blockbuilder/scheduler/scheduler.go @@ -114,12 +114,12 @@ func (s *BlockScheduler) runOnce(ctx context.Context) error { for _, job := range jobs { // TODO: end offset keeps moving each time we plan jobs, maybe we should not use it as part of the job ID - if status, ok := s.queue.Exists(&job); ok { + if status, ok := s.queue.Exists(job.Job); ok { level.Debug(s.logger).Log("msg", "job already exists", "job", job, "status", status) continue } - if err := s.queue.Enqueue(&job); err != nil { + if err := s.queue.Enqueue(job.Job, job.Priority); err != nil { level.Error(s.logger).Log("msg", "failed to enqueue job", "job", job, "err", err) } } @@ -144,13 +144,15 @@ func (s *BlockScheduler) HandleGetJob(ctx context.Context, builderID string) (*t } } -func (s *BlockScheduler) HandleCompleteJob(_ context.Context, builderID string, job *types.Job) error { +func (s *BlockScheduler) HandleCompleteJob(_ context.Context, _ string, job *types.Job) error { // TODO: handle commits - return s.queue.MarkComplete(job.ID, builderID) + s.queue.MarkComplete(job.ID) + return nil } func (s *BlockScheduler) HandleSyncJob(_ context.Context, builderID string, job *types.Job) error { - return s.queue.SyncJob(job.ID, builderID, job) + s.queue.SyncJob(job.ID, builderID, job) + return nil } // unimplementedScheduler provides default implementations that panic. diff --git a/pkg/blockbuilder/scheduler/scheduler_test.go b/pkg/blockbuilder/scheduler/scheduler_test.go index 35e53ee255993..2d857d06a2fe9 100644 --- a/pkg/blockbuilder/scheduler/scheduler_test.go +++ b/pkg/blockbuilder/scheduler/scheduler_test.go @@ -39,7 +39,7 @@ func TestScheduleAndProcessJob(t *testing.T) { // Create and enqueue a test job job := types.NewJob(1, types.Offsets{Min: 100, Max: 200}) - err := env.queue.Enqueue(job) + err := env.queue.Enqueue(job, 100) if err != nil { t.Fatalf("failed to enqueue job: %v", err) } @@ -98,11 +98,11 @@ func TestMultipleBuilders(t *testing.T) { job2 := types.NewJob(2, types.Offsets{Min: 300, Max: 400}) // Enqueue jobs - err := env1.queue.Enqueue(job1) + err := env1.queue.Enqueue(job1, 100) if err != nil { t.Fatalf("failed to enqueue job1: %v", err) } - err = env1.queue.Enqueue(job2) + err = env1.queue.Enqueue(job2, 100) if err != nil { t.Fatalf("failed to enqueue job2: %v", err) } diff --git a/pkg/blockbuilder/scheduler/strategy.go b/pkg/blockbuilder/scheduler/strategy.go index 5ea1fb6db2d9c..8824c16f510ea 100644 --- a/pkg/blockbuilder/scheduler/strategy.go +++ b/pkg/blockbuilder/scheduler/strategy.go @@ -2,6 +2,7 @@ package scheduler import ( "context" + "sort" "time" "github.com/go-kit/log" @@ -19,7 +20,7 @@ type OffsetReader interface { type Planner interface { Name() string - Plan(ctx context.Context) ([]types.Job, error) + Plan(ctx context.Context) ([]*JobWithPriority[int], error) } const ( @@ -44,31 +45,35 @@ func (p *RecordCountPlanner) Name() string { return RecordCountStrategy } -func (p *RecordCountPlanner) Plan(ctx context.Context) ([]types.Job, error) { +func (p *RecordCountPlanner) Plan(ctx context.Context) ([]*JobWithPriority[int], error) { offsets, err := p.offsetReader.GroupLag(ctx) if err != nil { level.Error(p.logger).Log("msg", "failed to get group lag", "err", err) return nil, err } - jobs := make([]types.Job, 0, len(offsets)) - for _, partition := range offsets { + var jobs []*JobWithPriority[int] + for _, partitionOffset := range offsets { // kadm.GroupMemberLag contains valid Commit.At even when consumer group never committed any offset. // no additional validation is needed here - startOffset := partition.Commit.At + 1 - endOffset := min(startOffset+p.targetRecordCount, partition.End.Offset) + startOffset := partitionOffset.Commit.At + 1 + endOffset := min(startOffset+p.targetRecordCount, partitionOffset.End.Offset) - job := types.Job{ - Partition: int(partition.Partition), - Offsets: types.Offsets{ + job := NewJobWithPriority( + types.NewJob(int(partitionOffset.Partition), types.Offsets{ Min: startOffset, Max: endOffset, - }, - } + }), int(partitionOffset.End.Offset-startOffset), + ) jobs = append(jobs, job) } + // Sort jobs by partition number to ensure consistent ordering + sort.Slice(jobs, func(i, j int) bool { + return jobs[i].Job.Partition < jobs[j].Job.Partition + }) + return jobs, nil } @@ -98,7 +103,7 @@ func (p *TimeRangePlanner) Name() string { return TimeRangeStrategy } -func (p *TimeRangePlanner) Plan(ctx context.Context) ([]types.Job, error) { +func (p *TimeRangePlanner) Plan(ctx context.Context) ([]*JobWithPriority[int], error) { // truncate to the nearest Interval consumeUptoTS := p.now().Add(-p.buffer).Truncate(p.targetPeriod) @@ -115,7 +120,7 @@ func (p *TimeRangePlanner) Plan(ctx context.Context) ([]types.Job, error) { return nil, err } - var jobs []types.Job + var jobs []*JobWithPriority[int] for _, partitionOffset := range offsets { startOffset := partitionOffset.Commit.At + 1 // TODO: we could further break down the work into Interval sized chunks if this partition has pending records spanning a long time range @@ -129,14 +134,20 @@ func (p *TimeRangePlanner) Plan(ctx context.Context) ([]types.Job, error) { continue } - jobs = append(jobs, types.Job{ - Partition: int(partitionOffset.Partition), - Offsets: types.Offsets{ + job := NewJobWithPriority( + types.NewJob(int(partitionOffset.Partition), types.Offsets{ Min: startOffset, Max: endOffset, - }, - }) + }), int(endOffset-startOffset), + ) + + jobs = append(jobs, job) } + // Sort jobs by partition number to ensure consistent ordering + sort.Slice(jobs, func(i, j int) bool { + return jobs[i].Job.Partition < jobs[j].Job.Partition + }) + return jobs, nil } diff --git a/pkg/blockbuilder/scheduler/strategy_test.go b/pkg/blockbuilder/scheduler/strategy_test.go index d777113433f35..30cbd1ee8a172 100644 --- a/pkg/blockbuilder/scheduler/strategy_test.go +++ b/pkg/blockbuilder/scheduler/strategy_test.go @@ -17,16 +17,11 @@ func TestTimeRangePlanner_Plan(t *testing.T) { for _, tc := range []struct { name string now time.Time - expectedJobs []types.Job + expectedJobs []*JobWithPriority[int] groupLag map[int32]kadm.GroupMemberLag consumeUpto map[int32]kadm.ListedOffset }{ { - // Interval 1 - // now: 00:42:00. consume until 00:15:00 - // last consumed offset 100 with record ts: 00:10:00 - // record offset with ts after 00:15:00 - offset 200 - // resulting jobs: [100, 200] name: "normal case. schedule first interval", now: time.Date(0, 0, 0, 0, 42, 0, 0, time.UTC), // 00:42:00 groupLag: map[int32]kadm.GroupMemberLag{ @@ -42,19 +37,14 @@ func TestTimeRangePlanner_Plan(t *testing.T) { Offset: 200, }, }, - expectedJobs: []types.Job{ - { - Partition: 0, - Offsets: types.Offsets{Min: 101, Max: 200}, - }, + expectedJobs: []*JobWithPriority[int]{ + NewJobWithPriority( + types.NewJob(0, types.Offsets{Min: 101, Max: 200}), + 99, // 200-101 + ), }, }, { - // Interval 2 - // now: 00:46:00. consume until 00:30:00 - // last consumed offset 199 with record ts: 00:11:00 - // record offset with ts after 00:30:00 - offset 300 - // resulting jobs: [200, 300] name: "normal case. schedule second interval", now: time.Date(0, 0, 0, 0, 46, 0, 0, time.UTC), // 00:46:00 groupLag: map[int32]kadm.GroupMemberLag{ @@ -79,23 +69,18 @@ func TestTimeRangePlanner_Plan(t *testing.T) { Offset: 123, }, }, - expectedJobs: []types.Job{ - { - Partition: 0, - Offsets: types.Offsets{Min: 200, Max: 300}, - }, - { - Partition: 1, - Offsets: types.Offsets{Min: 12, Max: 123}, - }, + expectedJobs: []*JobWithPriority[int]{ + NewJobWithPriority( + types.NewJob(0, types.Offsets{Min: 200, Max: 300}), + 100, // 300-200 + ), + NewJobWithPriority( + types.NewJob(1, types.Offsets{Min: 12, Max: 123}), + 111, // 123-12 + ), }, }, { - // Interval 2 - run scheduling again - // now: 00:48:00. consume until 00:30:00 - // last consumed offset 299 - // record offset with ts after 00:30:00 - offset 300 - // no jobs to schedule for partition 0 name: "no pending records to consume. schedule second interval once more time", now: time.Date(0, 0, 0, 0, 48, 0, 0, time.UTC), // 00:48:00 groupLag: map[int32]kadm.GroupMemberLag{ @@ -116,16 +101,15 @@ func TestTimeRangePlanner_Plan(t *testing.T) { 0: { Offset: 300, }, - // still pending. assume no builder were assigned 1: { Offset: 123, }, }, - expectedJobs: []types.Job{ - { - Partition: 1, - Offsets: types.Offsets{Min: 12, Max: 123}, - }, + expectedJobs: []*JobWithPriority[int]{ + NewJobWithPriority( + types.NewJob(1, types.Offsets{Min: 12, Max: 123}), + 111, // 123-12 + ), }, }, } { From 0d678311dee56d2d0b40f82dc3a1ff447798ff87 Mon Sep 17 00:00:00 2001 From: Ashwanth Date: Wed, 4 Dec 2024 15:14:20 +0530 Subject: [PATCH 24/44] feat: update blockbuilder to use scheduler for fetching jobs (#15224) --- docs/sources/shared/configuration.md | 24 +- pkg/blockbuilder/builder/appender.go | 433 +++++++++ pkg/blockbuilder/builder/builder.go | 569 ++++++++++++ pkg/blockbuilder/builder/controller.go | 315 ------- pkg/blockbuilder/builder/metrics.go | 59 +- pkg/blockbuilder/builder/slimgester.go | 824 ------------------ pkg/blockbuilder/types/grpc_transport.go | 28 +- pkg/blockbuilder/types/interfaces.go | 5 +- .../types/proto/blockbuilder.pb.go | 105 ++- .../types/proto/blockbuilder.proto | 1 + pkg/loki/modules.go | 49 +- 11 files changed, 1150 insertions(+), 1262 deletions(-) create mode 100644 pkg/blockbuilder/builder/appender.go create mode 100644 pkg/blockbuilder/builder/builder.go delete mode 100644 pkg/blockbuilder/builder/controller.go delete mode 100644 pkg/blockbuilder/builder/slimgester.go diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 18c4cdceb649e..8cc10f9b9be3f 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -171,10 +171,6 @@ block_builder: # 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 @@ -188,6 +184,25 @@ block_builder: # CLI flag: -blockbuilder.backoff..backoff-retries [max_retries: | default = 10] + # The number of workers to run in parallel to process jobs. + # CLI flag: -blockbuilder.worker-parallelism + [worker_parallelism: | default = 1] + + # The interval at which to sync job status with the scheduler. + # CLI flag: -blockbuilder.sync-interval + [sync_interval: | default = 30s] + + # Address of the scheduler in the format described here: + # https://github.com/grpc/grpc/blob/master/doc/naming.md + # CLI flag: -blockbuilder.scheduler-address + [scheduler_address: | default = ""] + + # The grpc_client block configures the gRPC client used to communicate between + # a client and server component in Loki. + # The CLI flags prefix for this block configuration is: + # blockbuilder.scheduler-grpc-client. + [scheduler_grpc_client_config: ] + block_scheduler: # Consumer group used by block scheduler to track the last consumed offset. # CLI flag: -block-scheduler.consumer-group @@ -2632,6 +2647,7 @@ The `gcs_storage_config` block configures the connection to Google Cloud Storage The `grpc_client` block configures the gRPC client used to communicate between a client and server component in Loki. The supported CLI flags `` used to reference this configuration block are: - `bigtable` +- `blockbuilder.scheduler-grpc-client.` - `bloom-build.builder.grpc` - `bloom-gateway-client.grpc` - `boltdb.shipper.index-gateway-client.grpc` diff --git a/pkg/blockbuilder/builder/appender.go b/pkg/blockbuilder/builder/appender.go new file mode 100644 index 0000000000000..d956eb7931c43 --- /dev/null +++ b/pkg/blockbuilder/builder/appender.go @@ -0,0 +1,433 @@ +package builder + +import ( + "bytes" + "context" + "errors" + "fmt" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + + "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/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" + + "github.com/grafana/loki/pkg/push" +) + +const ( + flushReasonFull = "full" + flushReasonMaxAge = "max_age" + onePointFiveMB = 3 << 19 +) + +type Appender struct { + id string + cfg Config + periodicConfigs []config.PeriodConfig + + metrics *builderMetrics + logger log.Logger + + instances map[string]*instance + instancesMtx sync.RWMutex + + store stores.ChunkWriter + objStore *MultiStore +} + +// 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 *MultiStore, + logger log.Logger, + metrics *builderMetrics, +) *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 (w *Appender) reportFlushedChunkStatistics( + ch *chunk.Chunk, +) { + byt, err := ch.Encoded() + if err != nil { + level.Error(w.logger).Log("msg", "failed to encode flushed wire chunk", "err", err) + return + } + 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) > w.cfg.MaxChunkAge { + reason = flushReasonMaxAge + } + + w.metrics.chunksFlushedPerReason.WithLabelValues(reason).Add(1) + + compressedSize := float64(len(byt)) + uncompressedSize, ok := chunkenc.UncompressedSize(ch.Data) + + if ok && compressedSize > 0 { + w.metrics.chunkCompressionRatio.Observe(float64(uncompressedSize) / compressedSize) + } + + utilization := ch.Data.Utilization() + w.metrics.chunkUtilization.Observe(utilization) + + numEntries := ch.Data.Entries() + w.metrics.chunkEntries.Observe(float64(numEntries)) + w.metrics.chunkSize.Observe(compressedSize) + sizePerTenant.Add(compressedSize) + countPerTenant.Inc() + + w.metrics.chunkAge.Observe(time.Since(from).Seconds()) + w.metrics.chunkLifespan.Observe(through.Sub(from).Hours()) + + 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 (w *Appender) CutRemainingChunks(ctx context.Context) ([]*chunk.Chunk, error) { + var chunks []*chunk.Chunk + w.instancesMtx.Lock() + defer w.instancesMtx.Unlock() + + for _, inst := range w.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 + labels labels.Labels + labelsStr string + entries []push.Entry +} + +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. + w.instancesMtx.RLock() + inst, ok := w.instances[input.tenant] + w.instancesMtx.RUnlock() + if !ok { + w.instancesMtx.Lock() + inst, ok = w.instances[input.tenant] + if !ok { + inst = newInstance(w.cfg, input.tenant, w.metrics, w.periodicConfigs, w.logger) + w.instances[input.tenant] = inst + } + w.instancesMtx.Unlock() + } + + closed, err := inst.Push(ctx, input) + return closed, err +} + +// 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 + metrics *builderMetrics + streams *streamsMap + logger log.Logger + + periods []config.PeriodConfig +} + +func newInstance( + cfg Config, + tenant string, + metrics *builderMetrics, + periods []config.PeriodConfig, + logger log.Logger, +) *instance { + streams := newStreamsMap() + return &instance{ + cfg: cfg, + tenant: tenant, + buf: make([]byte, 0, 1024), + mapper: ingester.NewFPMapper(streams.getLabelsFromFingerprint), + metrics: metrics, + streams: streams, + logger: logger, + periods: periods, + } +} + +func newStreamsMap() *streamsMap { + return &streamsMap{ + byLabels: make(map[string]*stream), + byFp: make(map[model.Fingerprint]*stream), + } +} + +type streamsMap struct { + // labels -> stream + 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. +func (m *streamsMap) For( + ls string, + createFn func() (*stream, error), + fn func(*stream) error, +) error { + // first use read lock in case the stream exists + m.mtx.RLock() + if s, ok := m.byLabels[ls]; ok { + err := fn(s) + m.mtx.RUnlock() + return 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.byLabels[ls]; ok { + return fn(s) + } + + // Create new stream + s, err := createFn() + if err != nil { + return err + } + + 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 { + + 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)...) + 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, + input AppendInput, +) (closed []*chunk.Chunk, err error) { + err = i.streams.For( + input.labelsStr, + func() (*stream, error) { + fp := i.getHashForLabels(input.labels) + return newStream(fp, input.labels, i.cfg, i.metrics), nil + }, + func(stream *stream) error { + xs, err := stream.Push(input.entries) + if err != nil { + return err + } + + if len(xs) > 0 { + for _, x := range xs { + // encodeChunk mutates the chunk so we must pass by reference + chk, err := i.encodeChunk(ctx, stream, x) + if err != nil { + return err + } + closed = append(closed, chk) + } + } + return err + }, + ) + + return closed, err +} + +// encodeChunk encodes a chunk.Chunk. +func (i *instance) encodeChunk(ctx context.Context, stream *stream, mc *chunkenc.MemChunk) (*chunk.Chunk, error) { + if err := ctx.Err(); err != nil { + 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 := chk.EncodeTo(bytes.NewBuffer(make([]byte, 0, chunkBytesSize)), i.logger); err != nil { + if !errors.Is(err, chunk.ErrChunkDecode) { + return nil, fmt.Errorf("chunk encoding: %w", err) + } + + i.metrics.chunkDecodeFailures.WithLabelValues(chk.UserID).Inc() + } + i.metrics.chunkEncodeTime.Observe(time.Since(start).Seconds()) + i.metrics.chunksEncoded.WithLabelValues(chk.UserID).Inc() + return &chk, nil +} + +type stream struct { + fp model.Fingerprint + ls labels.Labels + + chunkFormat byte + codec compression.Codec + blockSize int + targetChunkSize int + + chunkMtx sync.RWMutex + chunk *chunkenc.MemChunk + metrics *builderMetrics +} + +func newStream(fp model.Fingerprint, ls labels.Labels, cfg Config, metrics *builderMetrics) *stream { + return &stream{ + fp: fp, + ls: ls, + + chunkFormat: chunkenc.ChunkFormatV4, + codec: cfg.parsedEncoding, + blockSize: cfg.BlockSize.Val(), + targetChunkSize: cfg.TargetChunkSize.Val(), + + 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]) { + cut, err := s.closeChunk() + if err != nil { + return nil, err + } + closed = append(closed, cut) + } + + if _, err = s.chunk.Append(&entries[i]); err != nil { + return closed, fmt.Errorf("appending entry: %w", err) + } + } + + return closed, nil +} + +func (s *stream) closeChunk() (*chunkenc.MemChunk, error) { + if err := s.chunk.Close(); err != nil { + return nil, fmt.Errorf("closing chunk: %w", 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 + res := s.chunk + s.chunk = s.NewChunk() + return res, nil +} + +func (s *stream) NewChunk() *chunkenc.MemChunk { + return chunkenc.NewMemChunk( + s.chunkFormat, + s.codec, + chunkenc.ChunkHeadFormatFor(s.chunkFormat), + s.blockSize, + s.targetChunkSize, + ) +} diff --git a/pkg/blockbuilder/builder/builder.go b/pkg/blockbuilder/builder/builder.go new file mode 100644 index 0000000000000..e3c1420b3cd81 --- /dev/null +++ b/pkg/blockbuilder/builder/builder.go @@ -0,0 +1,569 @@ +package builder + +import ( + "context" + "errors" + "flag" + "fmt" + "math" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/backoff" + "github.com/grafana/dskit/grpcclient" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + + "github.com/grafana/loki/v3/pkg/blockbuilder/types" + "github.com/grafana/loki/v3/pkg/compression" + "github.com/grafana/loki/v3/pkg/kafka" + "github.com/grafana/loki/v3/pkg/kafka/partition" + "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" + storagetypes "github.com/grafana/loki/v3/pkg/storage/types" + "github.com/grafana/loki/v3/pkg/util/flagext" + util_log "github.com/grafana/loki/v3/pkg/util/log" +) + +type Config struct { + ConcurrentFlushes int `yaml:"concurrent_flushes"` + ConcurrentWriters int `yaml:"concurrent_writers"` + + 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"` + + Backoff backoff.Config `yaml:"backoff_config"` + WorkerParallelism int `yaml:"worker_parallelism"` + SyncInterval time.Duration `yaml:"sync_interval"` + + SchedulerAddress string `yaml:"scheduler_address"` + // SchedulerGRPCClientConfig configures the gRPC connection between the block-builder and its scheduler. + SchedulerGRPCClientConfig grpcclient.Config `yaml:"scheduler_grpc_client_config"` +} + +func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + 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)) + 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.SyncInterval, prefix+"sync-interval", 30*time.Second, "The interval at which to sync job status with the scheduler.") + f.IntVar(&cfg.WorkerParallelism, prefix+"worker-parallelism", 1, "The number of workers to run in parallel to process jobs.") + f.StringVar(&cfg.SchedulerAddress, prefix+"scheduler-address", "", "Address of the scheduler in the format described here: https://github.com/grpc/grpc/blob/master/doc/naming.md") + + cfg.SchedulerGRPCClientConfig.RegisterFlagsWithPrefix(prefix+"scheduler-grpc-client.", f) + cfg.Backoff.RegisterFlagsWithPrefix(prefix+"backoff.", f) +} + +// RegisterFlags registers flags. +func (cfg *Config) RegisterFlags(flags *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix("blockbuilder.", flags) +} + +func (cfg *Config) Validate() error { + enc, err := compression.ParseCodec(cfg.ChunkEncoding) + if err != nil { + return err + } + cfg.parsedEncoding = enc + + if cfg.SyncInterval <= 0 { + return errors.New("sync interval must be greater than 0") + } + + if cfg.WorkerParallelism < 1 { + return errors.New("worker parallelism must be greater than 0") + } + + return nil +} + +// 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 +// 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 BlockBuilder struct { + services.Service + types.BuilderTransport + + id string + cfg Config + periodicConfigs []config.PeriodConfig + metrics *builderMetrics + logger log.Logger + + decoder *kafka.Decoder + readerFactory func(partition int32) (partition.Reader, error) + + store stores.ChunkWriter + objStore *MultiStore + + jobsMtx sync.RWMutex + inflightJobs map[string]*types.Job +} + +func NewBlockBuilder( + id string, + cfg Config, + periodicConfigs []config.PeriodConfig, + readerFactory func(partition int32) (partition.Reader, error), + store stores.ChunkWriter, + objStore *MultiStore, + logger log.Logger, + reg prometheus.Registerer, +) (*BlockBuilder, + error) { + decoder, err := kafka.NewDecoder() + if err != nil { + return nil, err + } + + t, err := types.NewGRPCTransportFromAddress(cfg.SchedulerAddress, cfg.SchedulerGRPCClientConfig, reg) + if err != nil { + return nil, fmt.Errorf("create grpc transport: %w", err) + } + + i := &BlockBuilder{ + id: id, + cfg: cfg, + periodicConfigs: periodicConfigs, + metrics: newBuilderMetrics(reg), + logger: logger, + decoder: decoder, + readerFactory: readerFactory, + store: store, + objStore: objStore, + BuilderTransport: t, + } + + i.Service = services.NewBasicService(nil, i.running, nil) + return i, nil +} + +func (i *BlockBuilder) running(ctx context.Context) error { + wg := sync.WaitGroup{} + + for j := 0; j < i.cfg.WorkerParallelism; j++ { + wg.Add(1) + go func(id string) { + defer wg.Done() + + for { + select { + case <-ctx.Done(): + return + default: + err := i.runOne(ctx, id) + if err != nil { + level.Error(i.logger).Log("msg", "block builder run failed", "err", err) + } + } + } + }(fmt.Sprintf("worker-%d", j)) + } + + wg.Add(1) + go func() { + defer wg.Done() + + ticker := time.NewTicker(i.cfg.SyncInterval) + defer ticker.Stop() + + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + if err := i.syncJobs(ctx); err != nil { + level.Error(i.logger).Log("msg", "failed to sync jobs", "err", err) + } + } + } + }() + + wg.Wait() + return nil +} + +func (i *BlockBuilder) syncJobs(ctx context.Context) error { + i.jobsMtx.RLock() + defer i.jobsMtx.RUnlock() + + for _, job := range i.inflightJobs { + if err := i.SendSyncJob(ctx, &types.SyncJobRequest{ + BuilderID: i.id, + Job: job, + }); err != nil { + level.Error(i.logger).Log("msg", "failed to sync job", "err", err) + } + } + + return nil +} + +func (i *BlockBuilder) runOne(ctx context.Context, workerID string) error { + // assuming GetJob blocks/polls until a job is available + resp, err := i.SendGetJobRequest(ctx, &types.GetJobRequest{ + BuilderID: workerID, + }) + if err != nil { + return err + } + + if !resp.OK { + level.Info(i.logger).Log("msg", "no available job to process") + return nil + } + + job := resp.Job + logger := log.With( + i.logger, + "worker_id", workerID, + "partition", job.Partition, + "job_min_offset", job.Offsets.Min, + "job_max_offset", job.Offsets.Max, + ) + + i.jobsMtx.Lock() + i.inflightJobs[job.ID] = job + i.metrics.inflightJobs.Set(float64(len(i.inflightJobs))) + i.jobsMtx.Unlock() + + lastConsumedOffset, err := i.processJob(ctx, job, logger) + + if _, err := withBackoff( + ctx, + i.cfg.Backoff, + func() (res struct{}, err error) { + if err = i.SendCompleteJob(ctx, &types.CompleteJobRequest{ + BuilderID: workerID, + Job: job, + LastConsumedOffset: lastConsumedOffset, + }); err != nil { + level.Error(i.logger).Log("msg", "failed to mark the job as complete", "err", err) + } + return + }, + ); err != nil { + return err + } + + i.jobsMtx.Lock() + delete(i.inflightJobs, job.ID) + i.metrics.inflightJobs.Set(float64(len(i.inflightJobs))) + i.jobsMtx.Unlock() + + return err +} + +func (i *BlockBuilder) processJob(ctx context.Context, job *types.Job, logger log.Logger) (lastOffsetConsumed int64, err error) { + level.Debug(logger).Log("msg", "beginning job") + + indexer := newTsdbCreator() + appender := newAppender(i.id, + i.cfg, + i.periodicConfigs, + i.store, + i.objStore, + logger, + i.metrics, + ) + + var lastOffset int64 + 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( + "load records", + 1, + func(ctx context.Context) error { + lastOffset, err = i.loadRecords(ctx, int32(job.Partition), job.Offsets, inputCh) + return err + }, + func(ctx context.Context) error { + level.Debug(logger).Log( + "msg", "finished loading records", + "ctx_error", ctx.Err(), + "last_offset", lastOffset, + "total_records", lastOffset-job.Offsets.Min, + ) + 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( + "appender", + i.cfg.ConcurrentWriters, + func(ctx context.Context) error { + + for { + select { + case <-ctx.Done(): + return ctx.Err() + case inputs, ok := <-inputCh: + // inputs are finished; we're done + if !ok { + return nil + } + + for _, input := range inputs { + cut, err := appender.Append(ctx, input) + if err != nil { + level.Error(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(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. + chks, err := appender.CutRemainingChunks(ctx) + if err != nil { + return err + } + + for _, chk := range chks { + select { + case <-ctx.Done(): + return ctx.Err() + case flush <- chk: + } + } + 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( + "flusher", + 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, + i.cfg.Backoff, // retry forever + func() (res struct{}, err error) { + err = i.store.PutOne(ctx, chk.From, chk.Through, *chk) + if err != nil { + level.Error(logger).Log("msg", "failed to flush chunk", "err", err) + i.metrics.chunksFlushFailures.Inc() + return + } + appender.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}) + if err != nil { + level.Error(logger).Log("msg", "failed to append chunk to index", "err", err) + } + + return + }, + ); err != nil { + return err + } + } + } + }, + ) + + err = p.Run() + level.Debug(logger).Log( + "msg", "finished chunk creation", + "err", err, + ) + if err != nil { + return 0, err + } + + var ( + nodeName = i.id + tableRanges = config.GetIndexStoreTableRanges(storagetypes.TSDBType, i.periodicConfigs) + ) + + built, err := indexer.create(ctx, nodeName, tableRanges) + if err != nil { + level.Error(logger).Log("msg", "failed to build index", "err", err) + return 0, err + } + + u := newUploader(i.objStore) + for _, db := range built { + if _, err := withBackoff(ctx, i.cfg.Backoff, func() (res struct{}, err error) { + err = u.Put(ctx, db) + if err != nil { + level.Error(util_log.Logger).Log( + "msg", "failed to upload tsdb", + "path", db.id.Path(), + ) + return + } + + level.Debug(logger).Log( + "msg", "uploaded tsdb", + "name", db.id.Name(), + ) + return + }); err != nil { + return 0, err + } + } + + if lastOffset <= job.Offsets.Min { + return lastOffset, nil + } + + // log success + level.Info(logger).Log( + "msg", "successfully processed job", + "last_offset", lastOffset, + ) + + return lastOffset, nil +} + +func (i *BlockBuilder) loadRecords(ctx context.Context, partitionID int32, offsets types.Offsets, ch chan<- []AppendInput) (int64, error) { + f, err := i.readerFactory(partitionID) + if err != nil { + return 0, err + } + + f.SetOffsetForConsumption(offsets.Min) + + var ( + lastOffset = offsets.Min - 1 + boff = backoff.New(ctx, i.cfg.Backoff) + ) + + for lastOffset < offsets.Max && boff.Ongoing() { + var records []partition.Record + records, err = f.Poll(ctx, int(offsets.Max-lastOffset)) + 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 { + if record.Offset >= offsets.Max { + level.Debug(i.logger).Log("msg", "record offset exceeds job max offset. stop processing", "record offset", record.Offset, "max offset", offsets.Max) + break + } + lastOffset = record.Offset + + stream, labels, err := i.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, + }) + } + + if len(converted) > 0 { + select { + case ch <- converted: + case <-ctx.Done(): + return 0, ctx.Err() + } + } + } + + return lastOffset, err +} + +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/builder/controller.go b/pkg/blockbuilder/builder/controller.go deleted file mode 100644 index 29248e16d9525..0000000000000 --- a/pkg/blockbuilder/builder/controller.go +++ /dev/null @@ -1,315 +0,0 @@ -package builder - -import ( - "context" - "fmt" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/dskit/backoff" - "github.com/prometheus/prometheus/model/labels" - - "github.com/grafana/loki/v3/pkg/blockbuilder/types" - "github.com/grafana/loki/v3/pkg/kafka" - "github.com/grafana/loki/v3/pkg/kafka/partition" - - "github.com/grafana/loki/pkg/push" -) - -// 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 - 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, - // 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, types.Offsets, chan<- []AppendInput) (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 "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 "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 -// 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 { - stepLen int64 - reader partition.Reader - offsetManager partition.OffsetManager - backoff backoff.Config - decoder *kafka.Decoder - logger log.Logger -} - -func NewPartitionJobController( - reader partition.Reader, - offsetManager partition.OffsetManager, - backoff backoff.Config, - logger log.Logger, -) (*PartitionJobController, error) { - decoder, err := kafka.NewDecoder() - if err != nil { - return nil, err - } - return &PartitionJobController{ - stepLen: 1000, // Default step length of 1000 offsets per job - reader: reader, - offsetManager: offsetManager, - backoff: backoff, - decoder: decoder, - logger: log.With(logger, - "component", "job-controller", - "topic", offsetManager.Topic(), - "partition", offsetManager.Partition(), - ), - }, nil -} - -func (l *PartitionJobController) HighestCommittedOffset(ctx context.Context) (int64, error) { - return withBackoff( - ctx, - l.backoff, - func() (int64, error) { - return l.offsetManager.FetchLastCommittedOffset(ctx) - }, - ) -} - -func (l *PartitionJobController) HighestPartitionOffset(ctx context.Context) (int64, error) { - return withBackoff( - ctx, - l.backoff, - func() (int64, error) { - return l.offsetManager.FetchPartitionOffset(ctx, partition.KafkaEndOffset) - }, - ) -} - -func (l *PartitionJobController) EarliestPartitionOffset(ctx context.Context) (int64, error) { - return withBackoff( - ctx, - l.backoff, - func() (int64, error) { - return l.offsetManager.FetchPartitionOffset(ctx, partition.KafkaStartOffset) - }, - ) -} - -func (l *PartitionJobController) Process(ctx context.Context, offsets types.Offsets, ch chan<- []AppendInput) (int64, error) { - l.reader.SetOffsetForConsumption(offsets.Min) - - var ( - lastOffset = offsets.Min - 1 - boff = backoff.New(ctx, l.backoff) - err error - ) - - for lastOffset < offsets.Max && boff.Ongoing() { - var records []partition.Record - records, err = l.reader.Poll(ctx, int(offsets.Max-lastOffset)) - 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 { - if record.Offset >= offsets.Max { - level.Debug(l.logger).Log("msg", "record offset exceeds job max offset. stop processing", "record offset", record.Offset, "max offset", offsets.Max) - break - } - lastOffset = record.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, - }) - } - - if len(converted) > 0 { - 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, *types.Job, error) { - // Read the most recent committed offset - committedOffset, err := l.HighestCommittedOffset(ctx) - if err != nil { - return false, nil, err - } - - earliestOffset, err := l.EarliestPartitionOffset(ctx) - if err != nil { - return false, nil, err - } - - startOffset := committedOffset + 1 - if startOffset < earliestOffset { - startOffset = earliestOffset - } - - highestOffset, err := l.HighestPartitionOffset(ctx) - if err != nil { - return false, nil, err - } - - if highestOffset < committedOffset { - level.Error(l.logger).Log("msg", "partition highest offset is less than committed offset", "highest", highestOffset, "committed", committedOffset) - return false, nil, fmt.Errorf("partition highest offset is less than committed offset") - } - - if highestOffset == committedOffset { - level.Info(l.logger).Log("msg", "no pending records to process") - return false, nil, nil - } - - // Create the job with the calculated offsets - offsets := types.Offsets{ - Min: startOffset, - Max: min(startOffset+l.stepLen, highestOffset), - } - - // Convert partition from int32 to int - job := types.NewJob(int(l.reader.Partition()), offsets) - 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 -} - -// 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 - streamsPerTenant: 2, // 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(_ context.Context) (int64, error) { - return d.committed, nil -} - -func (d *dummyPartitionController) HighestPartitionOffset(_ context.Context) (int64, error) { - return d.highest, nil -} - -func (d *dummyPartitionController) Commit(_ context.Context, offset int64) error { - d.committed = offset - return nil -} - -func (d *dummyPartitionController) Process(ctx context.Context, offsets types.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: - } - } - 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.Now(), - Line: fmt.Sprintf("tenant=%d stream=%d line=%d offset=%d", i, j, k, offset), - } - } - result = append(result, AppendInput{ - tenant: tenant, - labels: lbls, - labelsStr: lbls.String(), - entries: entries, - }) - } - } - return result -} - -func (d *dummyPartitionController) Close() error { - return nil -} diff --git a/pkg/blockbuilder/builder/metrics.go b/pkg/blockbuilder/builder/metrics.go index 3411985209533..27a89f195d07f 100644 --- a/pkg/blockbuilder/builder/metrics.go +++ b/pkg/blockbuilder/builder/metrics.go @@ -8,7 +8,7 @@ import ( "github.com/grafana/loki/v3/pkg/util/constants" ) -type SlimgesterMetrics struct { +type builderMetrics struct { chunkUtilization prometheus.Histogram chunkEntries prometheus.Histogram chunkSize prometheus.Histogram @@ -33,47 +33,49 @@ type SlimgesterMetrics struct { samplesPerChunk prometheus.Histogram blocksPerChunk prometheus.Histogram chunkCreatedStats *analytics.Counter + + inflightJobs prometheus.Gauge } -func NewSlimgesterMetrics(r prometheus.Registerer) *SlimgesterMetrics { - return &SlimgesterMetrics{ +func newBuilderMetrics(r prometheus.Registerer) *builderMetrics { + return &builderMetrics{ chunkUtilization: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Namespace: constants.Loki, - Name: "slimgester_chunk_utilization", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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 @@ -81,58 +83,58 @@ func NewSlimgesterMetrics(r prometheus.Registerer) *SlimgesterMetrics { }), chunkEncodeTime: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Namespace: constants.Loki, - Name: "slimgester_chunk_encode_time_seconds", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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", + Name: "block_builder_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"), + flushedChunksStats: analytics.NewCounter("block_builder_flushed_chunks"), + flushedChunksBytesStats: analytics.NewStatistics("block_builder_flushed_chunks_bytes"), + flushedChunksLinesStats: analytics.NewStatistics("block_builder_flushed_chunks_lines"), flushedChunksAgeStats: analytics.NewStatistics( - "slimgester_flushed_chunks_age_seconds", + "block_builder_flushed_chunks_age_seconds", ), flushedChunksLifespanStats: analytics.NewStatistics( - "slimgester_flushed_chunks_lifespan_seconds", + "block_builder_flushed_chunks_lifespan_seconds", ), flushedChunksUtilizationStats: analytics.NewStatistics( - "slimgester_flushed_chunks_utilization", + "block_builder_flushed_chunks_utilization", ), chunksCreatedTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: constants.Loki, - Name: "slimgester_chunks_created_total", + Name: "block_builder_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", + Subsystem: "block_builder", Name: "samples_per_chunk", Help: "The number of samples in a chunk.", @@ -140,13 +142,18 @@ func NewSlimgesterMetrics(r prometheus.Registerer) *SlimgesterMetrics { }), blocksPerChunk: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Namespace: constants.Loki, - Subsystem: "slimgester", + Subsystem: "block_builder", Name: "blocks_per_chunk", Help: "The number of blocks in a chunk.", Buckets: prometheus.ExponentialBuckets(5, 2, 6), }), - chunkCreatedStats: analytics.NewCounter("slimgester_chunk_created"), + chunkCreatedStats: analytics.NewCounter("block_builder_chunk_created"), + inflightJobs: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: constants.Loki, + Name: "block_builder_inflight_jobs", + Help: "The number of jobs currently being processed by the block builder.", + }), } } diff --git a/pkg/blockbuilder/builder/slimgester.go b/pkg/blockbuilder/builder/slimgester.go deleted file mode 100644 index 510c2f9bd71d4..0000000000000 --- a/pkg/blockbuilder/builder/slimgester.go +++ /dev/null @@ -1,824 +0,0 @@ -package builder - -import ( - "bytes" - "context" - "flag" - "fmt" - "math" - "sync" - "time" - - "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" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - - "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/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/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" - - "github.com/grafana/loki/pkg/push" -) - -const ( - flushReasonFull = "full" - flushReasonMaxAge = "max_age" - onePointFiveMB = 3 << 19 -) - -type Config struct { - ConcurrentFlushes int `yaml:"concurrent_flushes"` - ConcurrentWriters int `yaml:"concurrent_writers"` - 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"` - Interval time.Duration `yaml:"interval"` - Backoff backoff.Config `yaml:"backoff_config"` -} - -func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { - 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)) - 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.") - cfg.Backoff.RegisterFlagsWithPrefix(prefix+"backoff.", f) -} - -// RegisterFlags registers flags. -func (cfg *Config) RegisterFlags(flags *flag.FlagSet) { - cfg.RegisterFlagsWithPrefix("blockbuilder.", flags) -} - -func (cfg *Config) Validate() error { - enc, err := compression.ParseCodec(cfg.ChunkEncoding) - if err != nil { - return err - } - cfg.parsedEncoding = enc - return nil -} - -// 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 -// 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 BlockBuilder struct { - services.Service - - id string - cfg Config - periodicConfigs []config.PeriodConfig - - metrics *SlimgesterMetrics - logger log.Logger - - store stores.ChunkWriter - objStore *MultiStore - jobController *PartitionJobController -} - -func NewBlockBuilder( - id string, - cfg Config, - periodicConfigs []config.PeriodConfig, - store stores.ChunkWriter, - objStore *MultiStore, - logger log.Logger, - reg prometheus.Registerer, - jobController *PartitionJobController, -) (*BlockBuilder, - error) { - i := &BlockBuilder{ - id: id, - cfg: cfg, - periodicConfigs: periodicConfigs, - metrics: NewSlimgesterMetrics(reg), - logger: logger, - store: store, - objStore: objStore, - jobController: jobController, - } - - i.Service = services.NewBasicService(nil, i.running, nil) - return i, nil -} - -func (i *BlockBuilder) running(ctx context.Context) error { - ticker := time.NewTicker(i.cfg.Interval) - defer ticker.Stop() - - // run once in beginning - select { - case <-ctx.Done(): - return nil - default: - _, err := i.runOne(ctx) - if err != nil { - level.Error(i.logger).Log("msg", "block builder run failed", "err", err) - } - } - - for { - select { - case <-ctx.Done(): - return nil - case <-ticker.C: - skipped, err := i.runOne(ctx) - level.Info(i.logger).Log( - "msg", "completed block builder run", "skipped", - "skipped", skipped, - "err", err, - ) - if err != nil { - level.Error(i.logger).Log("msg", "block builder run failed", "err", err) - } - } - } -} - -// runOne performs a single -func (i *BlockBuilder) runOne(ctx context.Context) (skipped bool, err error) { - - exists, job, err := i.jobController.LoadJob(ctx) - if err != nil { - return false, err - } - - if !exists { - level.Info(i.logger).Log("msg", "no available job to process") - return true, nil - } - - logger := log.With( - i.logger, - "partition", job.Partition, - "job_min_offset", job.Offsets.Min, - "job_max_offset", job.Offsets.Max, - ) - - level.Debug(logger).Log("msg", "beginning job") - - indexer := newTsdbCreator() - appender := newAppender(i.id, - i.cfg, - i.periodicConfigs, - i.store, - i.objStore, - logger, - i.metrics, - ) - - var lastOffset int64 - 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( - "load records", - 1, - func(ctx context.Context) error { - lastOffset, err = i.jobController.Process(ctx, job.Offsets, inputCh) - return err - }, - func(ctx context.Context) error { - level.Debug(logger).Log( - "msg", "finished loading records", - "ctx_error", ctx.Err(), - "last_offset", lastOffset, - "total_records", lastOffset-job.Offsets.Min, - ) - 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( - "appender", - i.cfg.ConcurrentWriters, - func(ctx context.Context) error { - - for { - select { - case <-ctx.Done(): - return ctx.Err() - case inputs, ok := <-inputCh: - // inputs are finished; we're done - if !ok { - return nil - } - - for _, input := range inputs { - cut, err := appender.Append(ctx, input) - if err != nil { - level.Error(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(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. - chks, err := appender.CutRemainingChunks(ctx) - if err != nil { - return err - } - - for _, chk := range chks { - select { - case <-ctx.Done(): - return ctx.Err() - case flush <- chk: - } - } - 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( - "flusher", - 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, - i.cfg.Backoff, // retry forever - func() (res struct{}, err error) { - err = i.store.PutOne(ctx, chk.From, chk.Through, *chk) - if err != nil { - level.Error(logger).Log("msg", "failed to flush chunk", "err", err) - i.metrics.chunksFlushFailures.Inc() - return - } - appender.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}) - if err != nil { - level.Error(logger).Log("msg", "failed to append chunk to index", "err", err) - } - - return - }, - ); err != nil { - return err - } - } - } - }, - ) - - err = p.Run() - level.Debug(logger).Log( - "msg", "finished chunk creation", - "err", err, - ) - if err != nil { - return false, err - } - - var ( - nodeName = i.id - tableRanges = config.GetIndexStoreTableRanges(types.TSDBType, i.periodicConfigs) - ) - - built, err := indexer.create(ctx, nodeName, tableRanges) - if err != nil { - level.Error(logger).Log("msg", "failed to build index", "err", err) - return false, err - } - - u := newUploader(i.objStore) - for _, db := range built { - if _, err := withBackoff(ctx, i.cfg.Backoff, func() (res struct{}, err error) { - err = u.Put(ctx, db) - if err != nil { - level.Error(util_log.Logger).Log( - "msg", "failed to upload tsdb", - "path", db.id.Path(), - ) - return - } - - level.Debug(logger).Log( - "msg", "uploaded tsdb", - "name", db.id.Name(), - ) - return - }); err != nil { - return false, err - } - } - - if lastOffset <= job.Offsets.Min { - return false, nil - } - - if err = i.jobController.offsetManager.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 -} - -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 *MultiStore -} - -// 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 *MultiStore, - 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 (w *Appender) reportFlushedChunkStatistics( - ch *chunk.Chunk, -) { - byt, err := ch.Encoded() - if err != nil { - level.Error(w.logger).Log("msg", "failed to encode flushed wire chunk", "err", err) - return - } - 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) > w.cfg.MaxChunkAge { - reason = flushReasonMaxAge - } - - w.metrics.chunksFlushedPerReason.WithLabelValues(reason).Add(1) - - compressedSize := float64(len(byt)) - uncompressedSize, ok := chunkenc.UncompressedSize(ch.Data) - - if ok && compressedSize > 0 { - w.metrics.chunkCompressionRatio.Observe(float64(uncompressedSize) / compressedSize) - } - - utilization := ch.Data.Utilization() - w.metrics.chunkUtilization.Observe(utilization) - - numEntries := ch.Data.Entries() - w.metrics.chunkEntries.Observe(float64(numEntries)) - w.metrics.chunkSize.Observe(compressedSize) - sizePerTenant.Add(compressedSize) - countPerTenant.Inc() - - w.metrics.chunkAge.Observe(time.Since(from).Seconds()) - w.metrics.chunkLifespan.Observe(through.Sub(from).Hours()) - - 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 (w *Appender) CutRemainingChunks(ctx context.Context) ([]*chunk.Chunk, error) { - var chunks []*chunk.Chunk - w.instancesMtx.Lock() - defer w.instancesMtx.Unlock() - - for _, inst := range w.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 - labels labels.Labels - labelsStr string - entries []push.Entry -} - -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. - w.instancesMtx.RLock() - inst, ok := w.instances[input.tenant] - w.instancesMtx.RUnlock() - if !ok { - w.instancesMtx.Lock() - inst, ok = w.instances[input.tenant] - if !ok { - inst = newInstance(w.cfg, input.tenant, w.metrics, w.periodicConfigs, w.logger) - w.instances[input.tenant] = inst - } - w.instancesMtx.Unlock() - } - - closed, err := inst.Push(ctx, input) - return closed, err -} - -// 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 - metrics *SlimgesterMetrics - streams *streamsMap - logger log.Logger - - periods []config.PeriodConfig -} - -func newInstance( - cfg Config, - tenant string, - metrics *SlimgesterMetrics, - periods []config.PeriodConfig, - logger log.Logger, -) *instance { - streams := newStreamsMap() - return &instance{ - cfg: cfg, - tenant: tenant, - buf: make([]byte, 0, 1024), - mapper: ingester.NewFPMapper(streams.getLabelsFromFingerprint), - metrics: metrics, - streams: streams, - logger: logger, - periods: periods, - } -} - -func newStreamsMap() *streamsMap { - return &streamsMap{ - byLabels: make(map[string]*stream), - byFp: make(map[model.Fingerprint]*stream), - } -} - -type streamsMap struct { - // labels -> stream - 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. -func (m *streamsMap) For( - ls string, - createFn func() (*stream, error), - fn func(*stream) error, -) error { - // first use read lock in case the stream exists - m.mtx.RLock() - if s, ok := m.byLabels[ls]; ok { - err := fn(s) - m.mtx.RUnlock() - return 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.byLabels[ls]; ok { - return fn(s) - } - - // Create new stream - s, err := createFn() - if err != nil { - return err - } - - 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 { - - 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)...) - 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, - input AppendInput, -) (closed []*chunk.Chunk, err error) { - err = i.streams.For( - input.labelsStr, - func() (*stream, error) { - fp := i.getHashForLabels(input.labels) - return newStream(fp, input.labels, i.cfg, i.metrics), nil - }, - func(stream *stream) error { - xs, err := stream.Push(input.entries) - if err != nil { - return err - } - - if len(xs) > 0 { - for _, x := range xs { - // encodeChunk mutates the chunk so we must pass by reference - chk, err := i.encodeChunk(ctx, stream, x) - if err != nil { - return err - } - closed = append(closed, chk) - } - } - return err - }, - ) - - return closed, err -} - -// encodeChunk encodes a chunk.Chunk. -func (i *instance) encodeChunk(ctx context.Context, stream *stream, mc *chunkenc.MemChunk) (*chunk.Chunk, error) { - if err := ctx.Err(); err != nil { - 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 := chk.EncodeTo(bytes.NewBuffer(make([]byte, 0, chunkBytesSize)), i.logger); err != nil { - if !errors.Is(err, chunk.ErrChunkDecode) { - return nil, fmt.Errorf("chunk encoding: %w", err) - } - - i.metrics.chunkDecodeFailures.WithLabelValues(chk.UserID).Inc() - } - i.metrics.chunkEncodeTime.Observe(time.Since(start).Seconds()) - i.metrics.chunksEncoded.WithLabelValues(chk.UserID).Inc() - return &chk, nil -} - -type stream struct { - fp model.Fingerprint - ls labels.Labels - - chunkFormat byte - codec compression.Codec - blockSize int - targetChunkSize int - - chunkMtx sync.RWMutex - chunk *chunkenc.MemChunk - metrics *SlimgesterMetrics -} - -func newStream(fp model.Fingerprint, ls labels.Labels, cfg Config, metrics *SlimgesterMetrics) *stream { - return &stream{ - fp: fp, - ls: ls, - - chunkFormat: chunkenc.ChunkFormatV4, - codec: cfg.parsedEncoding, - blockSize: cfg.BlockSize.Val(), - targetChunkSize: cfg.TargetChunkSize.Val(), - - 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]) { - cut, err := s.closeChunk() - if err != nil { - return nil, err - } - closed = append(closed, cut) - } - - if _, err = s.chunk.Append(&entries[i]); err != nil { - return closed, errors.Wrap(err, "appending entry") - } - } - - 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, - chunkenc.ChunkHeadFormatFor(s.chunkFormat), - s.blockSize, - 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/types/grpc_transport.go b/pkg/blockbuilder/types/grpc_transport.go index 0d132e2d07bd7..a5a9fa843671e 100644 --- a/pkg/blockbuilder/types/grpc_transport.go +++ b/pkg/blockbuilder/types/grpc_transport.go @@ -2,7 +2,6 @@ package types import ( "context" - "flag" "io" "github.com/grafana/dskit/grpcclient" @@ -19,17 +18,6 @@ import ( var _ Transport = &GRPCTransport{} -type GRPCTransportConfig struct { - Address string `yaml:"address,omitempty"` - - // GRPCClientConfig configures the gRPC connection between the block-builder and its scheduler. - GRPCClientConfig grpcclient.Config `yaml:"grpc_client_config"` -} - -func (cfg *GRPCTransportConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { - f.StringVar(&cfg.Address, prefix+"address", "", "address in DNS Service Discovery format: https://grafana.com/docs/mimir/latest/configure/about-dns-service-discovery/#supported-discovery-modes") -} - type grpcTransportMetrics struct { requestLatency *prometheus.HistogramVec } @@ -55,17 +43,16 @@ type GRPCTransport struct { // NewGRPCTransportFromAddress creates a new gRPC transport instance from an address and dial options func NewGRPCTransportFromAddress( - metrics *grpcTransportMetrics, - cfg GRPCTransportConfig, + address string, + cfg grpcclient.Config, + reg prometheus.Registerer, ) (*GRPCTransport, error) { - - dialOpts, err := cfg.GRPCClientConfig.DialOption(grpcclient.Instrument(metrics.requestLatency)) + dialOpts, err := cfg.DialOption(grpcclient.Instrument(newGRPCTransportMetrics(reg).requestLatency)) if err != nil { return nil, err } - // nolint:staticcheck // grpc.Dial() has been deprecated; we'll address it before upgrading to gRPC 2. - conn, err := grpc.Dial(cfg.Address, dialOpts...) + conn, err := grpc.NewClient(address, dialOpts...) if err != nil { return nil, errors.Wrap(err, "new grpc pool dial") } @@ -97,8 +84,9 @@ func (t *GRPCTransport) SendGetJobRequest(ctx context.Context, req *GetJobReques // SendCompleteJob implements Transport func (t *GRPCTransport) SendCompleteJob(ctx context.Context, req *CompleteJobRequest) error { protoReq := &proto.CompleteJobRequest{ - BuilderId: req.BuilderID, - Job: jobToProto(req.Job), + BuilderId: req.BuilderID, + Job: jobToProto(req.Job), + LastConsumedOffset: req.LastConsumedOffset, } _, err := t.CompleteJob(ctx, protoReq) diff --git a/pkg/blockbuilder/types/interfaces.go b/pkg/blockbuilder/types/interfaces.go index dd719757ba6a1..e3e4a8476e3ca 100644 --- a/pkg/blockbuilder/types/interfaces.go +++ b/pkg/blockbuilder/types/interfaces.go @@ -52,8 +52,9 @@ type GetJobResponse struct { } type CompleteJobRequest struct { - BuilderID string - Job *Job + BuilderID string + Job *Job + LastConsumedOffset int64 } type SyncJobRequest struct { diff --git a/pkg/blockbuilder/types/proto/blockbuilder.pb.go b/pkg/blockbuilder/types/proto/blockbuilder.pb.go index c5c4b05d38604..b6a743a4b571f 100644 --- a/pkg/blockbuilder/types/proto/blockbuilder.pb.go +++ b/pkg/blockbuilder/types/proto/blockbuilder.pb.go @@ -126,8 +126,9 @@ func (m *GetJobResponse) GetOk() bool { // CompleteJobRequest represents a job completion notification type CompleteJobRequest struct { - BuilderId string `protobuf:"bytes,1,opt,name=builder_id,json=builderId,proto3" json:"builder_id,omitempty"` - Job *Job `protobuf:"bytes,2,opt,name=job,proto3" json:"job,omitempty"` + BuilderId string `protobuf:"bytes,1,opt,name=builder_id,json=builderId,proto3" json:"builder_id,omitempty"` + Job *Job `protobuf:"bytes,2,opt,name=job,proto3" json:"job,omitempty"` + LastConsumedOffset int64 `protobuf:"varint,3,opt,name=LastConsumedOffset,proto3" json:"LastConsumedOffset,omitempty"` } func (m *CompleteJobRequest) Reset() { *m = CompleteJobRequest{} } @@ -176,6 +177,13 @@ func (m *CompleteJobRequest) GetJob() *Job { return nil } +func (m *CompleteJobRequest) GetLastConsumedOffset() int64 { + if m != nil { + return m.LastConsumedOffset + } + return 0 +} + // CompleteJobResponse is an empty response for job completion type CompleteJobResponse struct { } @@ -428,35 +436,36 @@ func init() { } var fileDescriptor_04968622516f7b79 = []byte{ - // 438 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x93, 0xcf, 0xae, 0xd2, 0x40, - 0x18, 0xc5, 0x3b, 0x6d, 0xbc, 0xc8, 0x77, 0x23, 0xea, 0xdc, 0x18, 0x09, 0xea, 0xe4, 0x5a, 0x13, - 0xbd, 0x2e, 0x6c, 0x13, 0xd4, 0x17, 0xc0, 0x85, 0x11, 0x17, 0xc6, 0xe2, 0x8a, 0x85, 0xda, 0x3f, - 0x03, 0x0e, 0x2d, 0x9d, 0xda, 0x0e, 0x06, 0x76, 0x3e, 0x82, 0x8f, 0xe0, 0xd2, 0x47, 0x71, 0xc9, - 0x92, 0xa5, 0x94, 0x8d, 0x4b, 0x1e, 0xc1, 0x74, 0xda, 0xa2, 0x0d, 0x0d, 0xb8, 0xb9, 0xab, 0x36, - 0xa7, 0xbf, 0x9e, 0x73, 0xf2, 0x7d, 0x33, 0x60, 0x46, 0xfe, 0xd8, 0x74, 0x02, 0xee, 0xfa, 0xce, - 0x8c, 0x05, 0x1e, 0x8d, 0x4d, 0xb1, 0x88, 0x68, 0x62, 0x46, 0x31, 0x17, 0xbc, 0xf2, 0xc1, 0x90, - 0x12, 0xc6, 0x15, 0x4d, 0xc2, 0xba, 0x01, 0xd7, 0x5e, 0x52, 0xd1, 0xe7, 0x8e, 0x45, 0x3f, 0xcf, - 0x68, 0x22, 0xf0, 0x3d, 0x80, 0x82, 0xf8, 0xc0, 0xbc, 0x36, 0x3a, 0x47, 0x17, 0x4d, 0xab, 0x59, - 0x28, 0xaf, 0x3c, 0xfd, 0x35, 0xb4, 0x4a, 0x3e, 0x89, 0x78, 0x98, 0x50, 0xfc, 0x18, 0xb4, 0x09, - 0x77, 0x24, 0x79, 0xda, 0xbd, 0x6d, 0xec, 0x67, 0x18, 0x19, 0x9d, 0x31, 0xb8, 0x05, 0x2a, 0xf7, - 0xdb, 0xea, 0x39, 0xba, 0xb8, 0x6a, 0xa9, 0xdc, 0xd7, 0xdf, 0x03, 0x7e, 0xc1, 0xa7, 0x51, 0x40, - 0x05, 0xfd, 0xef, 0x06, 0x65, 0x9e, 0x7a, 0x3c, 0x4f, 0xbf, 0x05, 0x67, 0x15, 0xff, 0xbc, 0xb1, - 0x3e, 0x84, 0xd6, 0x60, 0x11, 0xba, 0x97, 0x12, 0x79, 0x13, 0xae, 0xef, 0xbc, 0x8b, 0xb8, 0x27, - 0xd0, 0x78, 0x33, 0x1a, 0x25, 0x54, 0x24, 0xf8, 0x06, 0x68, 0x53, 0x16, 0xca, 0x00, 0xcd, 0xca, - 0x5e, 0xa5, 0x62, 0xcf, 0xa5, 0x75, 0xa6, 0xd8, 0x73, 0x7d, 0x02, 0x5a, 0x3f, 0x9f, 0xd5, 0xae, - 0x8a, 0xca, 0x3c, 0x7c, 0x17, 0x9a, 0x91, 0x1d, 0x0b, 0x26, 0x18, 0x0f, 0x25, 0x7e, 0xc5, 0xfa, - 0x2b, 0xe0, 0xe7, 0xd0, 0xe0, 0x79, 0x46, 0x5b, 0x93, 0x2d, 0xef, 0xd4, 0xb5, 0x2c, 0x6a, 0x58, - 0x25, 0xdb, 0xfd, 0xae, 0xc2, 0x59, 0x2f, 0xe3, 0x7a, 0x39, 0x37, 0xa0, 0xf1, 0x17, 0xe6, 0x52, - 0xfc, 0x16, 0x4e, 0xf2, 0x2d, 0xe3, 0xfb, 0x75, 0x3e, 0x95, 0x13, 0xd3, 0xd1, 0x0f, 0x21, 0xc5, - 0x0c, 0x14, 0xfc, 0x11, 0x4e, 0xff, 0xd9, 0x05, 0x7e, 0x58, 0xf7, 0xd3, 0xfe, 0x61, 0xe8, 0x3c, - 0x3a, 0xca, 0xed, 0x12, 0xde, 0x41, 0xa3, 0x18, 0x3d, 0xae, 0xad, 0x54, 0xdd, 0x79, 0xe7, 0xc1, - 0x41, 0xa6, 0x74, 0xed, 0x4d, 0x96, 0x6b, 0xa2, 0xac, 0xd6, 0x44, 0xd9, 0xae, 0x09, 0xfa, 0x9a, - 0x12, 0xf4, 0x23, 0x25, 0xe8, 0x67, 0x4a, 0xd0, 0x32, 0x25, 0xe8, 0x57, 0x4a, 0xd0, 0xef, 0x94, - 0x28, 0xdb, 0x94, 0xa0, 0x6f, 0x1b, 0xa2, 0x2c, 0x37, 0x44, 0x59, 0x6d, 0x88, 0x32, 0x7c, 0x36, - 0x66, 0xe2, 0xd3, 0xcc, 0x31, 0x5c, 0x3e, 0x35, 0xc7, 0xb1, 0x3d, 0xb2, 0x43, 0xdb, 0x0c, 0xb8, - 0xcf, 0x0e, 0xde, 0x59, 0xe7, 0x44, 0x3e, 0x9e, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, 0x6b, 0x42, - 0xf6, 0xf1, 0xda, 0x03, 0x00, 0x00, + // 464 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x93, 0x31, 0x8e, 0xd3, 0x40, + 0x14, 0x86, 0x3d, 0xb6, 0xd8, 0x90, 0x17, 0x11, 0x60, 0x56, 0x88, 0x28, 0xc0, 0x28, 0x0c, 0x12, + 0x84, 0x02, 0x5b, 0x0a, 0x70, 0x81, 0x6c, 0x81, 0x58, 0x90, 0x10, 0x5e, 0xaa, 0x6d, 0xc0, 0x8e, + 0x27, 0x61, 0xe2, 0xc4, 0x63, 0x3c, 0x13, 0xb4, 0xdb, 0x71, 0x02, 0xc4, 0x11, 0x28, 0x39, 0x0a, + 0x65, 0xca, 0x2d, 0x89, 0xd3, 0x50, 0xee, 0x11, 0x90, 0xc7, 0x76, 0xc0, 0x5a, 0x2b, 0xa4, 0xd9, + 0xca, 0xd6, 0x3f, 0x9f, 0xdf, 0xff, 0xfb, 0xbd, 0x37, 0xe0, 0xc4, 0xe1, 0xc4, 0xf1, 0x67, 0x62, + 0x14, 0xfa, 0x0b, 0x3e, 0x0b, 0x58, 0xe2, 0xa8, 0xd3, 0x98, 0x49, 0x27, 0x4e, 0x84, 0x12, 0x95, + 0x03, 0x5b, 0x4b, 0x18, 0x57, 0x34, 0x0d, 0x53, 0x1b, 0xae, 0xbd, 0x60, 0xea, 0x50, 0xf8, 0x2e, + 0xfb, 0xb4, 0x60, 0x52, 0xe1, 0x7b, 0x00, 0x05, 0xf1, 0x9e, 0x07, 0x1d, 0xd4, 0x43, 0xfd, 0xa6, + 0xdb, 0x2c, 0x94, 0x97, 0x01, 0x7d, 0x05, 0xed, 0x92, 0x97, 0xb1, 0x88, 0x24, 0xc3, 0x8f, 0xc1, + 0x9a, 0x0a, 0x5f, 0x93, 0xad, 0xc1, 0x6d, 0xfb, 0xa2, 0x87, 0x9d, 0xd1, 0x19, 0x83, 0xdb, 0x60, + 0x8a, 0xb0, 0x63, 0xf6, 0x50, 0xff, 0xaa, 0x6b, 0x8a, 0x90, 0x7e, 0x45, 0x80, 0x0f, 0xc4, 0x3c, + 0x9e, 0x31, 0xc5, 0x76, 0x8e, 0x50, 0x1a, 0x9a, 0x3b, 0x18, 0xda, 0x80, 0x5f, 0x7b, 0x52, 0x1d, + 0x88, 0x48, 0x2e, 0xe6, 0x2c, 0x78, 0x33, 0x1e, 0x4b, 0xa6, 0x3a, 0x56, 0x0f, 0xf5, 0x2d, 0xb7, + 0xe6, 0x84, 0xde, 0x82, 0xfd, 0x4a, 0x9e, 0xfc, 0x17, 0xe9, 0x31, 0xb4, 0x8f, 0x4e, 0xa3, 0xd1, + 0x65, 0x44, 0xa4, 0x37, 0xe1, 0xfa, 0xa6, 0x76, 0x61, 0xf7, 0x04, 0x1a, 0x79, 0x1e, 0x89, 0x6f, + 0x80, 0x35, 0xe7, 0x91, 0x36, 0xb0, 0xdc, 0xec, 0x55, 0x2b, 0xde, 0x89, 0x2e, 0x9d, 0x29, 0xde, + 0x09, 0x9d, 0x82, 0x75, 0x98, 0x37, 0x77, 0x13, 0xc5, 0xe4, 0x01, 0xbe, 0x0b, 0xcd, 0xd8, 0x4b, + 0x14, 0x57, 0x5c, 0x44, 0x1a, 0xbf, 0xe2, 0xfe, 0x15, 0xf0, 0x73, 0x68, 0x88, 0xdc, 0x43, 0xb7, + 0xa3, 0x35, 0xb8, 0x53, 0x97, 0xb2, 0x88, 0xe1, 0x96, 0xec, 0xe0, 0xbb, 0x09, 0xfb, 0xc3, 0x8c, + 0x1b, 0xe6, 0xdc, 0x11, 0x4b, 0x3e, 0xf3, 0x11, 0xc3, 0x6f, 0x61, 0x2f, 0x5f, 0x0b, 0x7c, 0xbf, + 0xae, 0x4e, 0x65, 0xc5, 0xba, 0x74, 0x1b, 0x52, 0xf4, 0xc0, 0xc0, 0x1f, 0xa0, 0xf5, 0xcf, 0x2c, + 0xf0, 0xc3, 0xba, 0x8f, 0x2e, 0x2e, 0x4f, 0xf7, 0xd1, 0x7f, 0xb9, 0x8d, 0xc3, 0x3b, 0x68, 0x14, + 0xad, 0xc7, 0xb5, 0x91, 0xaa, 0x33, 0xef, 0x3e, 0xd8, 0xca, 0x94, 0x55, 0x87, 0xd3, 0xe5, 0x8a, + 0x18, 0x67, 0x2b, 0x62, 0x9c, 0xaf, 0x08, 0xfa, 0x92, 0x12, 0xf4, 0x23, 0x25, 0xe8, 0x67, 0x4a, + 0xd0, 0x32, 0x25, 0xe8, 0x57, 0x4a, 0xd0, 0xef, 0x94, 0x18, 0xe7, 0x29, 0x41, 0xdf, 0xd6, 0xc4, + 0x58, 0xae, 0x89, 0x71, 0xb6, 0x26, 0xc6, 0xf1, 0xb3, 0x09, 0x57, 0x1f, 0x17, 0xbe, 0x3d, 0x12, + 0x73, 0x67, 0x92, 0x78, 0x63, 0x2f, 0xf2, 0x9c, 0x99, 0x08, 0xf9, 0xd6, 0x4b, 0xee, 0xef, 0xe9, + 0xc7, 0xd3, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x20, 0x0d, 0x30, 0x7c, 0x0b, 0x04, 0x00, 0x00, } func (this *GetJobRequest) Equal(that interface{}) bool { @@ -535,6 +544,9 @@ func (this *CompleteJobRequest) Equal(that interface{}) bool { if !this.Job.Equal(that1.Job) { return false } + if this.LastConsumedOffset != that1.LastConsumedOffset { + return false + } return true } func (this *CompleteJobResponse) Equal(that interface{}) bool { @@ -690,12 +702,13 @@ func (this *CompleteJobRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 6) + s := make([]string, 0, 7) s = append(s, "&proto.CompleteJobRequest{") s = append(s, "BuilderId: "+fmt.Sprintf("%#v", this.BuilderId)+",\n") if this.Job != nil { s = append(s, "Job: "+fmt.Sprintf("%#v", this.Job)+",\n") } + s = append(s, "LastConsumedOffset: "+fmt.Sprintf("%#v", this.LastConsumedOffset)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -1017,6 +1030,11 @@ func (m *CompleteJobRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.LastConsumedOffset != 0 { + i = encodeVarintBlockbuilder(dAtA, i, uint64(m.LastConsumedOffset)) + i-- + dAtA[i] = 0x18 + } if m.Job != nil { { size, err := m.Job.MarshalToSizedBuffer(dAtA[:i]) @@ -1261,6 +1279,9 @@ func (m *CompleteJobRequest) Size() (n int) { l = m.Job.Size() n += 1 + l + sovBlockbuilder(uint64(l)) } + if m.LastConsumedOffset != 0 { + n += 1 + sovBlockbuilder(uint64(m.LastConsumedOffset)) + } return n } @@ -1368,6 +1389,7 @@ func (this *CompleteJobRequest) String() string { s := strings.Join([]string{`&CompleteJobRequest{`, `BuilderId:` + fmt.Sprintf("%v", this.BuilderId) + `,`, `Job:` + strings.Replace(this.Job.String(), "Job", "Job", 1) + `,`, + `LastConsumedOffset:` + fmt.Sprintf("%v", this.LastConsumedOffset) + `,`, `}`, }, "") return s @@ -1723,6 +1745,25 @@ func (m *CompleteJobRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LastConsumedOffset", wireType) + } + m.LastConsumedOffset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBlockbuilder + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LastConsumedOffset |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipBlockbuilder(dAtA[iNdEx:]) diff --git a/pkg/blockbuilder/types/proto/blockbuilder.proto b/pkg/blockbuilder/types/proto/blockbuilder.proto index 89811989b821c..56160d336d336 100644 --- a/pkg/blockbuilder/types/proto/blockbuilder.proto +++ b/pkg/blockbuilder/types/proto/blockbuilder.proto @@ -29,6 +29,7 @@ message GetJobResponse { message CompleteJobRequest { string builder_id = 1; Job job = 2; + int64 LastConsumedOffset = 3; } // CompleteJobResponse is an empty response for job completion diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index fb5c800cc03e1..c4c4a6b0412c5 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -52,7 +52,6 @@ import ( "github.com/grafana/loki/v3/pkg/ingester" kclient "github.com/grafana/loki/v3/pkg/kafka/client" "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" "github.com/grafana/loki/v3/pkg/logqlmodel/stats" @@ -1816,58 +1815,30 @@ func (t *Loki) initBlockBuilder() (services.Service, error) { // 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) - } - - readerMetrics := partition.NewReaderMetrics(prometheus.DefaultRegisterer) - reader, err := partition.NewKafkaReader( - t.Cfg.KafkaConfig, - ingestPartitionID, - logger, - readerMetrics, - ) - if err != nil { - return nil, err - } - - offsetManager, err := partition.NewKafkaOffsetManager( - t.Cfg.KafkaConfig, - ingestPartitionID, - id, - logger, - prometheus.DefaultRegisterer, - ) - if err != nil { - return nil, err - } - - controller, err := blockbuilder.NewPartitionJobController( - reader, - offsetManager, - t.Cfg.BlockBuilder.Backoff, - logger, - ) - + objectStore, err := blockbuilder.NewMultiStore(t.Cfg.SchemaConfig.Configs, t.Cfg.StorageConfig, t.ClientMetrics) 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 + readerMetrics := partition.NewReaderMetrics(prometheus.DefaultRegisterer) + readerFactory := func(partitionID int32) (partition.Reader, error) { + return partition.NewKafkaReader( + t.Cfg.KafkaConfig, + partitionID, + logger, + readerMetrics, + ) } bb, err := blockbuilder.NewBlockBuilder( id, t.Cfg.BlockBuilder, t.Cfg.SchemaConfig.Configs, + readerFactory, t.Store, objectStore, logger, prometheus.DefaultRegisterer, - controller, ) if err != nil { From 532bdbc15643445bcd770d8d45df8b25b2c926d3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 Dec 2024 03:29:41 -0800 Subject: [PATCH 25/44] feat(blockbuilder): consolidate on record counting planner (#15247) --- docs/sources/shared/configuration.md | 14 +- pkg/blockbuilder/scheduler/scheduler.go | 44 +++++- pkg/blockbuilder/scheduler/scheduler_test.go | 101 +++++++++++++ pkg/blockbuilder/scheduler/strategy.go | 112 ++++----------- pkg/blockbuilder/scheduler/strategy_test.go | 140 ++++++++++--------- 5 files changed, 247 insertions(+), 164 deletions(-) diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 8cc10f9b9be3f..1160adff8f8ba 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -212,17 +212,21 @@ block_scheduler: # CLI flag: -block-scheduler.interval [interval: | default = 5m] - # Period used by the planner to calculate the start and end offset such that - # each job consumes records spanning the target period. - # CLI flag: -block-scheduler.target-records-spanning-period - [target_records_spanning_period: | default = 1h] - # Lookback period in milliseconds used by the scheduler to plan jobs when the # consumer group has no commits. -1 consumes from the latest offset. -2 # consumes from the start of the partition. # CLI flag: -block-scheduler.lookback-period [lookback_period: | default = -2] + # Strategy used by the planner to plan jobs. One of record-count + # CLI flag: -block-scheduler.strategy + [strategy: | default = "record-count"] + + # Target record count used by the planner to plan jobs. Only used when + # strategy is record-count + # CLI flag: -block-scheduler.target-record-count + [target_record_count: | default = 1000] + pattern_ingester: # Whether the pattern ingester is enabled. # CLI flag: -pattern-ingester.enabled diff --git a/pkg/blockbuilder/scheduler/scheduler.go b/pkg/blockbuilder/scheduler/scheduler.go index 96356515a921f..632e6842993a5 100644 --- a/pkg/blockbuilder/scheduler/scheduler.go +++ b/pkg/blockbuilder/scheduler/scheduler.go @@ -4,7 +4,9 @@ import ( "context" "errors" "flag" + "fmt" "strconv" + "strings" "time" "github.com/go-kit/log" @@ -22,17 +24,36 @@ var ( ) type Config struct { - ConsumerGroup string `yaml:"consumer_group"` - Interval time.Duration `yaml:"interval"` - TargetRecordConsumptionPeriod time.Duration `yaml:"target_records_spanning_period"` - LookbackPeriod int64 `yaml:"lookback_period"` + ConsumerGroup string `yaml:"consumer_group"` + Interval time.Duration `yaml:"interval"` + LookbackPeriod int64 `yaml:"lookback_period"` + Strategy string `yaml:"strategy"` + planner Planner `yaml:"-"` // validated planner + TargetRecordCount int64 `yaml:"target_record_count"` } func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { f.DurationVar(&cfg.Interval, prefix+"interval", 5*time.Minute, "How often the scheduler should plan jobs.") - f.DurationVar(&cfg.TargetRecordConsumptionPeriod, prefix+"target-records-spanning-period", time.Hour, "Period used by the planner to calculate the start and end offset such that each job consumes records spanning the target period.") f.StringVar(&cfg.ConsumerGroup, prefix+"consumer-group", "block-scheduler", "Consumer group used by block scheduler to track the last consumed offset.") f.Int64Var(&cfg.LookbackPeriod, prefix+"lookback-period", -2, "Lookback period in milliseconds used by the scheduler to plan jobs when the consumer group has no commits. -1 consumes from the latest offset. -2 consumes from the start of the partition.") + f.StringVar( + &cfg.Strategy, + prefix+"strategy", + RecordCountStrategy, + fmt.Sprintf( + "Strategy used by the planner to plan jobs. One of %s", + strings.Join(validStrategies, ", "), + ), + ) + f.Int64Var( + &cfg.TargetRecordCount, + prefix+"target-record-count", + 1000, + fmt.Sprintf( + "Target record count used by the planner to plan jobs. Only used when strategy is %s", + RecordCountStrategy, + ), + ) } func (cfg *Config) RegisterFlags(f *flag.FlagSet) { @@ -48,6 +69,16 @@ func (cfg *Config) Validate() error { return errors.New("only -1(latest) and -2(earliest) are valid as negative values for lookback_period") } + switch cfg.Strategy { + case RecordCountStrategy: + if cfg.TargetRecordCount <= 0 { + return errors.New("target record count must be a non-zero value") + } + cfg.planner = NewRecordCountPlanner(cfg.TargetRecordCount) + default: + return fmt.Errorf("invalid strategy: %s", cfg.Strategy) + } + return nil } @@ -66,10 +97,9 @@ type BlockScheduler struct { // NewScheduler creates a new scheduler instance func NewScheduler(cfg Config, queue *JobQueue, offsetReader OffsetReader, logger log.Logger, r prometheus.Registerer) *BlockScheduler { - planner := NewTimeRangePlanner(cfg.TargetRecordConsumptionPeriod, offsetReader, func() time.Time { return time.Now().UTC() }, logger) s := &BlockScheduler{ cfg: cfg, - planner: planner, + planner: cfg.planner, offsetReader: offsetReader, logger: logger, metrics: NewMetrics(r), diff --git a/pkg/blockbuilder/scheduler/scheduler_test.go b/pkg/blockbuilder/scheduler/scheduler_test.go index 2d857d06a2fe9..977dd1d556d1e 100644 --- a/pkg/blockbuilder/scheduler/scheduler_test.go +++ b/pkg/blockbuilder/scheduler/scheduler_test.go @@ -149,3 +149,104 @@ func TestMultipleBuilders(t *testing.T) { t.Error("builder1 got unexpected second job") } } + +func TestConfig_Validate(t *testing.T) { + tests := []struct { + name string + cfg Config + wantErr string + }{ + { + name: "valid config with record count strategy", + cfg: Config{ + Interval: time.Minute, + LookbackPeriod: -1, + Strategy: RecordCountStrategy, + TargetRecordCount: 1000, + }, + }, + { + name: "zero interval", + cfg: Config{ + Interval: 0, + LookbackPeriod: -1, + Strategy: RecordCountStrategy, + TargetRecordCount: 1000, + }, + wantErr: "interval must be a non-zero value", + }, + { + name: "negative interval", + cfg: Config{ + Interval: -time.Minute, + LookbackPeriod: -1, + Strategy: RecordCountStrategy, + TargetRecordCount: 1000, + }, + wantErr: "interval must be a non-zero value", + }, + { + name: "invalid lookback period", + cfg: Config{ + Interval: time.Minute, + LookbackPeriod: -3, + Strategy: RecordCountStrategy, + TargetRecordCount: 1000, + }, + wantErr: "only -1(latest) and -2(earliest) are valid as negative values for lookback_period", + }, + { + name: "invalid strategy", + cfg: Config{ + Interval: time.Minute, + LookbackPeriod: -1, + Strategy: "invalid", + TargetRecordCount: 1000, + }, + wantErr: "invalid strategy: invalid", + }, + { + name: "zero target record count", + cfg: Config{ + Interval: time.Minute, + LookbackPeriod: -1, + Strategy: RecordCountStrategy, + TargetRecordCount: 0, + }, + wantErr: "target record count must be a non-zero value", + }, + { + name: "negative target record count", + cfg: Config{ + Interval: time.Minute, + LookbackPeriod: -1, + Strategy: RecordCountStrategy, + TargetRecordCount: -1000, + }, + wantErr: "target record count must be a non-zero value", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := tt.cfg.Validate() + if tt.wantErr != "" { + if err == nil { + t.Errorf("Validate() error = nil, wantErr %v", tt.wantErr) + return + } + if err.Error() != tt.wantErr { + t.Errorf("Validate() error = %v, wantErr %v", err, tt.wantErr) + } + return + } + if err != nil { + t.Errorf("Validate() error = %v, wantErr nil", err) + } + // Check that planner is set for valid configs + if tt.cfg.planner == nil { + t.Error("Validate() did not set planner for valid config") + } + }) + } +} diff --git a/pkg/blockbuilder/scheduler/strategy.go b/pkg/blockbuilder/scheduler/strategy.go index 8824c16f510ea..0b8c38aa64d11 100644 --- a/pkg/blockbuilder/scheduler/strategy.go +++ b/pkg/blockbuilder/scheduler/strategy.go @@ -3,7 +3,6 @@ package scheduler import ( "context" "sort" - "time" "github.com/go-kit/log" "github.com/go-kit/log/level" @@ -14,7 +13,6 @@ import ( // OffsetReader is an interface to list offsets for all partitions of a topic from Kafka. type OffsetReader interface { - ListOffsetsAfterMilli(context.Context, int64) (map[int32]kadm.ListedOffset, error) GroupLag(context.Context) (map[int32]kadm.GroupMemberLag, error) } @@ -24,10 +22,13 @@ type Planner interface { } const ( - RecordCountStrategy = "record_count" - TimeRangeStrategy = "time_range" + RecordCountStrategy = "record-count" ) +var validStrategies = []string{ + RecordCountStrategy, +} + // tries to consume upto targetRecordCount records per partition type RecordCountPlanner struct { targetRecordCount int64 @@ -52,101 +53,40 @@ func (p *RecordCountPlanner) Plan(ctx context.Context) ([]*JobWithPriority[int], return nil, err } - var jobs []*JobWithPriority[int] + jobs := make([]*JobWithPriority[int], 0, len(offsets)) for _, partitionOffset := range offsets { // kadm.GroupMemberLag contains valid Commit.At even when consumer group never committed any offset. // no additional validation is needed here startOffset := partitionOffset.Commit.At + 1 - endOffset := min(startOffset+p.targetRecordCount, partitionOffset.End.Offset) - - job := NewJobWithPriority( - types.NewJob(int(partitionOffset.Partition), types.Offsets{ - Min: startOffset, - Max: endOffset, - }), int(partitionOffset.End.Offset-startOffset), - ) - - jobs = append(jobs, job) - } - - // Sort jobs by partition number to ensure consistent ordering - sort.Slice(jobs, func(i, j int) bool { - return jobs[i].Job.Partition < jobs[j].Job.Partition - }) - - return jobs, nil -} - -// Targets consuming records spanning a configured period. -// This is a stateless planner, it is upto the caller to deduplicate or update jobs that are already in queue or progress. -type TimeRangePlanner struct { - offsetReader OffsetReader - - buffer time.Duration - targetPeriod time.Duration - now func() time.Time - - logger log.Logger -} - -func NewTimeRangePlanner(interval time.Duration, offsetReader OffsetReader, now func() time.Time, logger log.Logger) *TimeRangePlanner { - return &TimeRangePlanner{ - targetPeriod: interval, - buffer: interval, - offsetReader: offsetReader, - now: now, - logger: logger, - } -} - -func (p *TimeRangePlanner) Name() string { - return TimeRangeStrategy -} - -func (p *TimeRangePlanner) Plan(ctx context.Context) ([]*JobWithPriority[int], error) { - // truncate to the nearest Interval - consumeUptoTS := p.now().Add(-p.buffer).Truncate(p.targetPeriod) - - // this will return the latest offset in the partition if no records are produced after this ts. - consumeUptoOffsets, err := p.offsetReader.ListOffsetsAfterMilli(ctx, consumeUptoTS.UnixMilli()) - if err != nil { - level.Error(p.logger).Log("msg", "failed to list offsets after timestamp", "err", err) - return nil, err - } - - offsets, err := p.offsetReader.GroupLag(ctx) - if err != nil { - level.Error(p.logger).Log("msg", "failed to get group lag", "err", err) - return nil, err - } - - var jobs []*JobWithPriority[int] - for _, partitionOffset := range offsets { - startOffset := partitionOffset.Commit.At + 1 - // TODO: we could further break down the work into Interval sized chunks if this partition has pending records spanning a long time range - // or have the builder consume in chunks and commit the job status back to scheduler. - endOffset := consumeUptoOffsets[partitionOffset.Partition].Offset + endOffset := partitionOffset.End.Offset + // Skip if there's no lag if startOffset >= endOffset { - level.Info(p.logger).Log("msg", "no pending records to process", "partition", partitionOffset.Partition, - "commitOffset", partitionOffset.Commit.At, - "consumeUptoOffset", consumeUptoOffsets[partitionOffset.Partition].Offset) continue } - job := NewJobWithPriority( - types.NewJob(int(partitionOffset.Partition), types.Offsets{ - Min: startOffset, - Max: endOffset, - }), int(endOffset-startOffset), - ) + // Create jobs of size targetRecordCount until we reach endOffset + for currentStart := startOffset; currentStart < endOffset; { + currentEnd := min(currentStart+p.targetRecordCount, endOffset) + + job := NewJobWithPriority( + types.NewJob(int(partitionOffset.Partition), types.Offsets{ + Min: currentStart, + Max: currentEnd, + }), int(endOffset-currentStart), // priority is remaining records to process + ) + jobs = append(jobs, job) - jobs = append(jobs, job) + currentStart = currentEnd + } } - // Sort jobs by partition number to ensure consistent ordering + // Sort jobs by partition then priority sort.Slice(jobs, func(i, j int) bool { - return jobs[i].Job.Partition < jobs[j].Job.Partition + if jobs[i].Job.Partition != jobs[j].Job.Partition { + return jobs[i].Job.Partition < jobs[j].Job.Partition + } + return jobs[i].Priority > jobs[j].Priority }) return jobs, nil diff --git a/pkg/blockbuilder/scheduler/strategy_test.go b/pkg/blockbuilder/scheduler/strategy_test.go index 30cbd1ee8a172..9c7b732fb4e08 100644 --- a/pkg/blockbuilder/scheduler/strategy_test.go +++ b/pkg/blockbuilder/scheduler/strategy_test.go @@ -5,120 +5,141 @@ import ( "testing" "time" - "github.com/go-kit/log" "github.com/stretchr/testify/require" "github.com/twmb/franz-go/pkg/kadm" "github.com/grafana/loki/v3/pkg/blockbuilder/types" ) -func TestTimeRangePlanner_Plan(t *testing.T) { - interval := 15 * time.Minute +type mockOffsetReader struct { + groupLag map[int32]kadm.GroupMemberLag +} + +func (m *mockOffsetReader) GroupLag(_ context.Context) (map[int32]kadm.GroupMemberLag, error) { + return m.groupLag, nil +} + +func TestRecordCountPlanner_Plan(t *testing.T) { for _, tc := range []struct { name string - now time.Time + recordCount int64 expectedJobs []*JobWithPriority[int] groupLag map[int32]kadm.GroupMemberLag - consumeUpto map[int32]kadm.ListedOffset }{ { - name: "normal case. schedule first interval", - now: time.Date(0, 0, 0, 0, 42, 0, 0, time.UTC), // 00:42:00 + name: "single partition, single job", + recordCount: 100, groupLag: map[int32]kadm.GroupMemberLag{ 0: { Commit: kadm.Offset{ At: 100, }, + End: kadm.ListedOffset{ + Offset: 150, + }, Partition: 0, }, }, - consumeUpto: map[int32]kadm.ListedOffset{ - 0: { - Offset: 200, - }, - }, expectedJobs: []*JobWithPriority[int]{ NewJobWithPriority( - types.NewJob(0, types.Offsets{Min: 101, Max: 200}), - 99, // 200-101 + types.NewJob(0, types.Offsets{Min: 101, Max: 150}), + 49, // 150-101 ), }, }, { - name: "normal case. schedule second interval", - now: time.Date(0, 0, 0, 0, 46, 0, 0, time.UTC), // 00:46:00 + name: "single partition, multiple jobs", + recordCount: 50, groupLag: map[int32]kadm.GroupMemberLag{ 0: { Commit: kadm.Offset{ - At: 199, + At: 100, }, - Partition: 0, - }, - 1: { - Commit: kadm.Offset{ - At: 11, + End: kadm.ListedOffset{ + Offset: 200, }, - Partition: 1, - }, - }, - consumeUpto: map[int32]kadm.ListedOffset{ - 0: { - Offset: 300, - }, - 1: { - Offset: 123, + Partition: 0, }, }, expectedJobs: []*JobWithPriority[int]{ NewJobWithPriority( - types.NewJob(0, types.Offsets{Min: 200, Max: 300}), - 100, // 300-200 + types.NewJob(0, types.Offsets{Min: 101, Max: 151}), + 99, // priority is total remaining: 200-101 ), NewJobWithPriority( - types.NewJob(1, types.Offsets{Min: 12, Max: 123}), - 111, // 123-12 + types.NewJob(0, types.Offsets{Min: 151, Max: 200}), + 49, // priority is total remaining: 200-151 ), }, }, { - name: "no pending records to consume. schedule second interval once more time", - now: time.Date(0, 0, 0, 0, 48, 0, 0, time.UTC), // 00:48:00 + name: "multiple partitions", + recordCount: 100, groupLag: map[int32]kadm.GroupMemberLag{ 0: { Commit: kadm.Offset{ - At: 299, + At: 100, + }, + End: kadm.ListedOffset{ + Offset: 150, }, Partition: 0, }, 1: { Commit: kadm.Offset{ - At: 11, + At: 200, + }, + End: kadm.ListedOffset{ + Offset: 400, }, Partition: 1, }, }, - consumeUpto: map[int32]kadm.ListedOffset{ - 0: { - Offset: 300, - }, - 1: { - Offset: 123, - }, - }, expectedJobs: []*JobWithPriority[int]{ NewJobWithPriority( - types.NewJob(1, types.Offsets{Min: 12, Max: 123}), - 111, // 123-12 + types.NewJob(1, types.Offsets{Min: 201, Max: 301}), + 199, // priority is total remaining: 400-201 + ), + NewJobWithPriority( + types.NewJob(1, types.Offsets{Min: 301, Max: 400}), + 99, // priority is total remaining: 400-301 ), + NewJobWithPriority( + types.NewJob(0, types.Offsets{Min: 101, Max: 150}), + 49, // priority is total remaining: 150-101 + ), + }, + }, + { + name: "no lag", + recordCount: 100, + groupLag: map[int32]kadm.GroupMemberLag{ + 0: { + Commit: kadm.Offset{ + At: 100, + }, + End: kadm.ListedOffset{ + Offset: 100, + }, + Partition: 0, + }, }, + expectedJobs: nil, }, } { t.Run(tc.name, func(t *testing.T) { - mockOffsetReader := &mockOffsetReader{ - offsetsAfterMilli: tc.consumeUpto, - groupLag: tc.groupLag, + mockReader := &mockOffsetReader{ + groupLag: tc.groupLag, + } + cfg := Config{ + Interval: time.Second, // foced > 0 in validation + Strategy: RecordCountStrategy, + TargetRecordCount: tc.recordCount, } - planner := NewTimeRangePlanner(interval, mockOffsetReader, func() time.Time { return tc.now }, log.NewNopLogger()) + require.NoError(t, cfg.Validate()) + + planner := NewRecordCountPlanner(tc.recordCount) + planner.offsetReader = mockReader jobs, err := planner.Plan(context.Background()) require.NoError(t, err) @@ -128,16 +149,3 @@ func TestTimeRangePlanner_Plan(t *testing.T) { }) } } - -type mockOffsetReader struct { - offsetsAfterMilli map[int32]kadm.ListedOffset - groupLag map[int32]kadm.GroupMemberLag -} - -func (m *mockOffsetReader) ListOffsetsAfterMilli(_ context.Context, _ int64) (map[int32]kadm.ListedOffset, error) { - return m.offsetsAfterMilli, nil -} - -func (m *mockOffsetReader) GroupLag(_ context.Context) (map[int32]kadm.GroupMemberLag, error) { - return m.groupLag, nil -} From 70d9587863410b5610606bf20210402a2438ffd3 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 4 Dec 2024 12:46:45 +0100 Subject: [PATCH 26/44] feat: Use context propagation to call the same ingester in GetChunksID as Query (#15186) Co-authored-by: Ben Clive --- pkg/querier/ingester_querier.go | 137 +++++++++++++++++++----- pkg/querier/ingester_querier_test.go | 153 ++++++++++++++++++++++++--- pkg/querier/querier.go | 6 ++ pkg/querier/querier_mock_test.go | 13 +++ 4 files changed, 267 insertions(+), 42 deletions(-) diff --git a/pkg/querier/ingester_querier.go b/pkg/querier/ingester_querier.go index 830ef53961ad3..76c8b8bfc42db 100644 --- a/pkg/querier/ingester_querier.go +++ b/pkg/querier/ingester_querier.go @@ -5,6 +5,7 @@ import ( "net/http" "slices" "strings" + "sync" "time" "github.com/go-kit/log" @@ -82,10 +83,91 @@ func newIngesterQuerier(querierConfig Config, clientCfg client.Config, ring ring return &iq, nil } +type ctxKeyType string + +const ( + partitionCtxKey ctxKeyType = "partitionCtx" +) + +type PartitionContext struct { + isPartitioned bool + ingestersUsed map[string]PartitionIngesterUsed + mtx sync.Mutex +} + +type PartitionIngesterUsed struct { + client logproto.QuerierClient + addr string +} + +func (p *PartitionContext) AddClient(client logproto.QuerierClient, addr string) { + p.mtx.Lock() + defer p.mtx.Unlock() + if !p.isPartitioned { + return + } + p.ingestersUsed[addr] = PartitionIngesterUsed{client: client, addr: addr} +} + +func (p *PartitionContext) RemoveClient(addr string) { + p.mtx.Lock() + defer p.mtx.Unlock() + if !p.isPartitioned { + return + } + delete(p.ingestersUsed, addr) +} + +func (p *PartitionContext) SetIsPartitioned(isPartitioned bool) { + p.mtx.Lock() + defer p.mtx.Unlock() + p.isPartitioned = isPartitioned +} + +func (p *PartitionContext) IsPartitioned() bool { + return p.isPartitioned +} + +func (p *PartitionContext) forQueriedIngesters(ctx context.Context, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { + p.mtx.Lock() + defer p.mtx.Unlock() + + ingestersUsed := make([]PartitionIngesterUsed, 0, len(p.ingestersUsed)) + for _, ingester := range p.ingestersUsed { + ingestersUsed = append(ingestersUsed, ingester) + } + + return concurrency.ForEachJobMergeResults(ctx, ingestersUsed, 0, func(ctx context.Context, job PartitionIngesterUsed) ([]responseFromIngesters, error) { + resp, err := f(ctx, job.client) + if err != nil { + return nil, err + } + return []responseFromIngesters{{addr: job.addr, response: resp}}, nil + }) +} + +// NewPartitionContext creates a new partition context +// This is used to track which ingesters were used in the query and reuse the same ingesters for consecutive queries +func NewPartitionContext(ctx context.Context) context.Context { + return context.WithValue(ctx, partitionCtxKey, &PartitionContext{ + ingestersUsed: make(map[string]PartitionIngesterUsed), + }) +} + +func ExtractPartitionContext(ctx context.Context) *PartitionContext { + v, ok := ctx.Value(partitionCtxKey).(*PartitionContext) + if !ok { + return &PartitionContext{ + ingestersUsed: make(map[string]PartitionIngesterUsed), + } + } + return v +} + // forAllIngesters runs f, in parallel, for all ingesters -// waitForAllResponses param can be used to require results from all ingesters in the replication set. If this is set to false, the call will return as soon as we have a quorum by zone. Only valid for partition-ingesters. -func (q *IngesterQuerier) forAllIngesters(ctx context.Context, waitForAllResponses bool, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { +func (q *IngesterQuerier) forAllIngesters(ctx context.Context, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { if q.querierConfig.QueryPartitionIngesters { + ExtractPartitionContext(ctx).SetIsPartitioned(true) tenantID, err := user.ExtractOrgID(ctx) if err != nil { return nil, err @@ -99,7 +181,7 @@ func (q *IngesterQuerier) forAllIngesters(ctx context.Context, waitForAllRespons if err != nil { return nil, err } - return q.forGivenIngesterSets(ctx, waitForAllResponses, replicationSets, f) + return q.forGivenIngesterSets(ctx, replicationSets, f) } replicationSet, err := q.ring.GetReplicationSetForOperation(ring.Read) @@ -111,19 +193,13 @@ func (q *IngesterQuerier) forAllIngesters(ctx context.Context, waitForAllRespons } // forGivenIngesterSets runs f, in parallel, for given ingester sets -// waitForAllResponses param can be used to require results from all ingesters in all replication sets. If this is set to false, the call will return as soon as we have a quorum by zone. -func (q *IngesterQuerier) forGivenIngesterSets(ctx context.Context, waitForAllResponses bool, replicationSet []ring.ReplicationSet, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { +func (q *IngesterQuerier) forGivenIngesterSets(ctx context.Context, replicationSet []ring.ReplicationSet, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { // Enable minimize requests if we can, so we initially query a single ingester per replication set, as each replication-set is one partition. // Ingesters must supply zone information for this to have an effect. config := ring.DoUntilQuorumConfig{ - MinimizeRequests: !waitForAllResponses, + MinimizeRequests: true, } return concurrency.ForEachJobMergeResults[ring.ReplicationSet, responseFromIngesters](ctx, replicationSet, 0, func(ctx context.Context, set ring.ReplicationSet) ([]responseFromIngesters, error) { - if waitForAllResponses { - // Tell the ring we need to return all responses from all zones - set.MaxErrors = 0 - set.MaxUnavailableZones = 0 - } return q.forGivenIngesters(ctx, set, config, f) }) } @@ -135,17 +211,16 @@ func (q *IngesterQuerier) forGivenIngesters(ctx context.Context, replicationSet if err != nil { return responseFromIngesters{addr: ingester.Addr}, err } - resp, err := f(ctx, client.(logproto.QuerierClient)) if err != nil { return responseFromIngesters{addr: ingester.Addr}, err } + ExtractPartitionContext(ctx).AddClient(client.(logproto.QuerierClient), ingester.Addr) return responseFromIngesters{ingester.Addr, resp}, nil - }, func(responseFromIngesters) { - // Nothing to do + }, func(cleanup responseFromIngesters) { + ExtractPartitionContext(ctx).RemoveClient(cleanup.addr) }) - if err != nil { return nil, err } @@ -157,7 +232,7 @@ func (q *IngesterQuerier) forGivenIngesters(ctx context.Context, replicationSet } func (q *IngesterQuerier) SelectLogs(ctx context.Context, params logql.SelectLogParams) ([]iter.EntryIterator, error) { - resps, err := q.forAllIngesters(ctx, false, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { + resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { stats.FromContext(ctx).AddIngesterReached(1) return client.Query(ctx, params.QueryRequest) }) @@ -173,7 +248,7 @@ func (q *IngesterQuerier) SelectLogs(ctx context.Context, params logql.SelectLog } func (q *IngesterQuerier) SelectSample(ctx context.Context, params logql.SelectSampleParams) ([]iter.SampleIterator, error) { - resps, err := q.forAllIngesters(ctx, false, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { + resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { stats.FromContext(ctx).AddIngesterReached(1) return client.QuerySample(ctx, params.SampleQueryRequest) }) @@ -189,7 +264,7 @@ func (q *IngesterQuerier) SelectSample(ctx context.Context, params logql.SelectS } func (q *IngesterQuerier) Label(ctx context.Context, req *logproto.LabelRequest) ([][]string, error) { - resps, err := q.forAllIngesters(ctx, false, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { + resps, err := q.forAllIngesters(ctx, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { return client.Label(ctx, req) }) if err != nil { @@ -205,7 +280,7 @@ func (q *IngesterQuerier) Label(ctx context.Context, req *logproto.LabelRequest) } func (q *IngesterQuerier) Tail(ctx context.Context, req *logproto.TailRequest) (map[string]logproto.Querier_TailClient, error) { - resps, err := q.forAllIngesters(ctx, false, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { + resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { return client.Tail(ctx, req) }) if err != nil { @@ -270,7 +345,7 @@ func (q *IngesterQuerier) TailDisconnectedIngesters(ctx context.Context, req *lo } func (q *IngesterQuerier) Series(ctx context.Context, req *logproto.SeriesRequest) ([][]logproto.SeriesIdentifier, error) { - resps, err := q.forAllIngesters(ctx, false, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { + resps, err := q.forAllIngesters(ctx, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { return client.Series(ctx, req) }) if err != nil { @@ -325,15 +400,22 @@ func (q *IngesterQuerier) TailersCount(ctx context.Context) ([]uint32, error) { } func (q *IngesterQuerier) GetChunkIDs(ctx context.Context, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { - // We must wait for all responses when using partition-ingesters to avoid a race between Query and GetChunkIDs calls. - // This occurs if call Query on an ingester after a recent flush then call GetChunkIDs on a different, unflushed ingester in the same partition. - resps, err := q.forAllIngesters(ctx, q.querierConfig.QueryPartitionIngesters, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { + ingesterQueryFn := q.forAllIngesters + + partitionCtx := ExtractPartitionContext(ctx) + if partitionCtx.IsPartitioned() { + // We need to query the same ingesters as the previous query + ingesterQueryFn = partitionCtx.forQueriedIngesters + } + + resps, err := ingesterQueryFn(ctx, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { return querierClient.GetChunkIDs(ctx, &logproto.GetChunkIDsRequest{ Matchers: convertMatchersToString(matchers), Start: from.Time(), End: through.Time(), }) }) + if err != nil { return nil, err } @@ -347,14 +429,13 @@ func (q *IngesterQuerier) GetChunkIDs(ctx context.Context, from, through model.T } func (q *IngesterQuerier) Stats(ctx context.Context, _ string, from, through model.Time, matchers ...*labels.Matcher) (*index_stats.Stats, error) { - resps, err := q.forAllIngesters(ctx, false, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { + resps, err := q.forAllIngesters(ctx, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { return querierClient.GetStats(ctx, &logproto.IndexStatsRequest{ From: from, Through: through, Matchers: syntax.MatchersString(matchers), }) }) - if err != nil { if isUnimplementedCallError(err) { // Handle communication with older ingesters gracefully @@ -378,7 +459,7 @@ func (q *IngesterQuerier) Volume(ctx context.Context, _ string, from, through mo matcherString = syntax.MatchersString(matchers) } - resps, err := q.forAllIngesters(ctx, false, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { + resps, err := q.forAllIngesters(ctx, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { return querierClient.GetVolume(ctx, &logproto.VolumeRequest{ From: from, Through: through, @@ -388,7 +469,6 @@ func (q *IngesterQuerier) Volume(ctx context.Context, _ string, from, through mo AggregateBy: aggregateBy, }) }) - if err != nil { if isUnimplementedCallError(err) { // Handle communication with older ingesters gracefully @@ -407,10 +487,9 @@ func (q *IngesterQuerier) Volume(ctx context.Context, _ string, from, through mo } func (q *IngesterQuerier) DetectedLabel(ctx context.Context, req *logproto.DetectedLabelsRequest) (*logproto.LabelToValuesResponse, error) { - ingesterResponses, err := q.forAllIngesters(ctx, false, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { + ingesterResponses, err := q.forAllIngesters(ctx, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { return client.GetDetectedLabels(ctx, req) }) - if err != nil { level.Error(q.logger).Log("msg", "error getting detected labels", "err", err) return nil, err diff --git a/pkg/querier/ingester_querier_test.go b/pkg/querier/ingester_querier_test.go index a5066176ad785..268191bd17a72 100644 --- a/pkg/querier/ingester_querier_test.go +++ b/pkg/querier/ingester_querier_test.go @@ -8,6 +8,7 @@ import ( "time" "github.com/go-kit/log" + "github.com/grafana/dskit/ring/client" "github.com/grafana/dskit/user" "go.uber.org/atomic" @@ -241,11 +242,10 @@ func TestIngesterQuerierFetchesResponsesFromPartitionIngesters(t *testing.T) { } tests := map[string]struct { - method string - testFn func(*IngesterQuerier) error - retVal interface{} - shards int - expectAllResponses bool + method string + testFn func(*IngesterQuerier) error + retVal interface{} + shards int }{ "label": { method: "Label", @@ -269,8 +269,7 @@ func TestIngesterQuerierFetchesResponsesFromPartitionIngesters(t *testing.T) { _, err := ingesterQuerier.GetChunkIDs(ctx, model.Time(0), model.Time(0)) return err }, - retVal: new(logproto.GetChunkIDsResponse), - expectAllResponses: true, + retVal: new(logproto.GetChunkIDsResponse), }, "select_logs": { method: "Query", @@ -330,7 +329,7 @@ func TestIngesterQuerierFetchesResponsesFromPartitionIngesters(t *testing.T) { ingestersPerPartition := len(ingesters) / partitions assert.Greaterf(t, ingestersPerPartition, 1, "must have more than one ingester per partition") - ingesterQuerier, err := newTestPartitionIngesterQuerier(ingesterClient, instanceRing, newPartitionInstanceRingMock(instanceRing, ingesters, partitions, ingestersPerPartition), testData.shards) + ingesterQuerier, err := newTestPartitionIngesterQuerier(newIngesterClientMockFactory(ingesterClient), instanceRing, newPartitionInstanceRingMock(instanceRing, ingesters, partitions, ingestersPerPartition), testData.shards) require.NoError(t, err) ingesterQuerier.querierConfig.QueryPartitionIngesters = true @@ -342,9 +341,6 @@ func TestIngesterQuerierFetchesResponsesFromPartitionIngesters(t *testing.T) { testData.shards = partitions } expectedCalls := min(testData.shards, partitions) - if testData.expectAllResponses { - expectedCalls = expectedCalls * ingestersPerPartition - } // Wait for responses: We expect one request per queried partition because we have request minimization enabled & ingesters are in multiple zones. // If shuffle sharding is enabled, we expect one query per shard as we write to a subset of partitions. require.Eventually(t, func() bool { return cnt.Load() >= int32(expectedCalls) }, time.Millisecond*100, time.Millisecond*1, "expected all ingesters to respond") @@ -353,6 +349,137 @@ func TestIngesterQuerierFetchesResponsesFromPartitionIngesters(t *testing.T) { } } +func TestIngesterQuerier_QueriesSameIngestersWithPartitionContext(t *testing.T) { + t.Parallel() + userCtx := user.InjectOrgID(context.Background(), "test-user") + testCtx, cancel := context.WithTimeout(userCtx, time.Second*10) + defer cancel() + + ingesters := []ring.InstanceDesc{ + mockInstanceDescWithZone("1.1.1.1", ring.ACTIVE, "A"), + mockInstanceDescWithZone("2.2.2.2", ring.ACTIVE, "B"), + mockInstanceDescWithZone("3.3.3.3", ring.ACTIVE, "A"), + mockInstanceDescWithZone("4.4.4.4", ring.ACTIVE, "B"), + mockInstanceDescWithZone("5.5.5.5", ring.ACTIVE, "A"), + mockInstanceDescWithZone("6.6.6.6", ring.ACTIVE, "B"), + } + + tests := map[string]struct { + method string + testFn func(context.Context, *IngesterQuerier) error + retVal interface{} + shards int + }{ + "select_logs": { + method: "Query", + testFn: func(ctx context.Context, ingesterQuerier *IngesterQuerier) error { + _, err := ingesterQuerier.SelectLogs(ctx, logql.SelectLogParams{ + QueryRequest: new(logproto.QueryRequest), + }) + return err + }, + retVal: newQueryClientMock(), + }, + "select_sample": { + method: "QuerySample", + testFn: func(ctx context.Context, ingesterQuerier *IngesterQuerier) error { + _, err := ingesterQuerier.SelectSample(ctx, logql.SelectSampleParams{ + SampleQueryRequest: new(logproto.SampleQueryRequest), + }) + return err + }, + retVal: newQuerySampleClientMock(), + }, + "select_logs_shuffle_sharded": { + method: "Query", + testFn: func(ctx context.Context, ingesterQuerier *IngesterQuerier) error { + _, err := ingesterQuerier.SelectLogs(ctx, logql.SelectLogParams{ + QueryRequest: new(logproto.QueryRequest), + }) + return err + }, + retVal: newQueryClientMock(), + shards: 2, // Must be less than number of partitions + }, + } + + for testName, testData := range tests { + cnt := atomic.NewInt32(0) + ctx := NewPartitionContext(testCtx) + + t.Run(testName, func(t *testing.T) { + cnt.Store(0) + runFn := func(args mock.Arguments) { + ctx := args[0].(context.Context) + + select { + case <-ctx.Done(): + // should not be cancelled by the tracker + require.NoErrorf(t, ctx.Err(), "tracker should not cancel ctx: %v", context.Cause(ctx)) + default: + cnt.Add(1) + } + } + + instanceRing := newReadRingMock(ingesters, 0) + ingesterClient := newQuerierClientMock() + ingesterClient.On(testData.method, mock.Anything, mock.Anything, mock.Anything).Return(testData.retVal, nil).Run(runFn) + ingesterClient.On("GetChunkIDs", mock.Anything, mock.Anything, mock.Anything).Return(new(logproto.GetChunkIDsResponse), nil).Run(runFn) + + partitions := 3 + ingestersPerPartition := len(ingesters) / partitions + assert.Greaterf(t, ingestersPerPartition, 1, "must have more than one ingester per partition") + + mockClientFactory := mockIngesterClientFactory{ + requestedClients: make(map[string]int), + } + + ingesterQuerier, err := newTestPartitionIngesterQuerier(mockClientFactory.newIngesterClientMockFactory(ingesterClient), instanceRing, newPartitionInstanceRingMock(instanceRing, ingesters, partitions, ingestersPerPartition), testData.shards) + require.NoError(t, err) + + ingesterQuerier.querierConfig.QueryPartitionIngesters = true + + err = testData.testFn(ctx, ingesterQuerier) + require.NoError(t, err) + + if testData.shards == 0 { + testData.shards = partitions + } + expectedCalls := min(testData.shards, partitions) + expectedIngesterCalls := expectedCalls + // Wait for responses: We expect one request per queried partition because we have request minimization enabled & ingesters are in multiple zones. + // If shuffle sharding is enabled, we expect one query per shard as we write to a subset of partitions. + require.Eventually(t, func() bool { return cnt.Load() >= int32(expectedCalls) }, time.Millisecond*100, time.Millisecond*1, "expected ingesters to respond") + ingesterClient.AssertNumberOfCalls(t, testData.method, expectedCalls) + + partitionCtx := ExtractPartitionContext(ctx) + require.Equal(t, expectedIngesterCalls, len(partitionCtx.ingestersUsed)) + require.Equal(t, expectedIngesterCalls, len(mockClientFactory.requestedClients)) + + for _, ingester := range partitionCtx.ingestersUsed { + count, ok := mockClientFactory.requestedClients[ingester.addr] + require.True(t, ok) + require.Equal(t, count, 1) + } + + // Now call getChunkIDs to ensure we only call the same ingesters as before. + _, err = ingesterQuerier.GetChunkIDs(ctx, model.Time(0), model.Time(1)) + require.NoError(t, err) + + require.Eventually(t, func() bool { return cnt.Load() >= int32(expectedCalls) }, time.Millisecond*100, time.Millisecond*1, "expected ingesters to respond") + ingesterClient.AssertNumberOfCalls(t, "GetChunkIDs", expectedCalls) + + // Finally, confirm we called the same ingesters again and didn't ask for any new clients + require.Equal(t, expectedIngesterCalls, len(mockClientFactory.requestedClients)) + for _, ingester := range partitionCtx.ingestersUsed { + count, ok := mockClientFactory.requestedClients[ingester.addr] + require.True(t, ok) + require.Equal(t, count, 1) + } + }) + } +} + func TestQuerier_tailDisconnectedIngesters(t *testing.T) { t.Parallel() @@ -540,14 +667,14 @@ func newTestIngesterQuerier(readRingMock *readRingMock, ingesterClient *querierC ) } -func newTestPartitionIngesterQuerier(ingesterClient *querierClientMock, instanceRing *readRingMock, partitionRing *ring.PartitionInstanceRing, tenantShards int) (*IngesterQuerier, error) { +func newTestPartitionIngesterQuerier(clientFactory client.PoolFactory, instanceRing *readRingMock, partitionRing *ring.PartitionInstanceRing, tenantShards int) (*IngesterQuerier, error) { return newIngesterQuerier( mockQuerierConfig(), mockIngesterClientConfig(), instanceRing, partitionRing, func(string) int { return tenantShards }, - newIngesterClientMockFactory(ingesterClient), + clientFactory, constants.Loki, log.NewNopLogger(), ) diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 947101c0aa2ff..3f03d1e037aad 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -152,6 +152,9 @@ func New(cfg Config, store Store, ingesterQuerier *IngesterQuerier, limits Limit // Select Implements logql.Querier which select logs via matchers and regex filters. func (q *SingleTenantQuerier) SelectLogs(ctx context.Context, params logql.SelectLogParams) (iter.EntryIterator, error) { + // Create a new partition context for the query + // This is used to track which ingesters were used in the query and reuse the same ingesters for consecutive queries + ctx = NewPartitionContext(ctx) var err error params.Start, params.End, err = q.validateQueryRequest(ctx, params) if err != nil { @@ -211,6 +214,9 @@ func (q *SingleTenantQuerier) SelectLogs(ctx context.Context, params logql.Selec } func (q *SingleTenantQuerier) SelectSamples(ctx context.Context, params logql.SelectSampleParams) (iter.SampleIterator, error) { + // Create a new partition context for the query + // This is used to track which ingesters were used in the query and reuse the same ingesters for consecutive queries + ctx = NewPartitionContext(ctx) var err error params.Start, params.End, err = q.validateQueryRequest(ctx, params) if err != nil { diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index ab70de4baacea..0fd9b421de000 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -142,6 +142,19 @@ func (c *querierClientMock) Close() error { return nil } +type mockIngesterClientFactory struct { + requestedClients map[string]int +} + +// newIngesterClientMockFactory creates a factory function always returning +// the input querierClientMock +func (f mockIngesterClientFactory) newIngesterClientMockFactory(c *querierClientMock) ring_client.PoolFactory { + return ring_client.PoolAddrFunc(func(addr string) (ring_client.PoolClient, error) { + f.requestedClients[addr]++ + return c, nil + }) +} + // newIngesterClientMockFactory creates a factory function always returning // the input querierClientMock func newIngesterClientMockFactory(c *querierClientMock) ring_client.PoolFactory { From 44b70f0e5262b7d3e70a4bb54838c14017f77ea1 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Wed, 4 Dec 2024 08:11:19 -0500 Subject: [PATCH 27/44] fix(deps): update module go.opentelemetry.io/collector/pdata to v1.21.0 (#15244) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 +- .../profiles_service.pb.go | 98 +- .../data/protogen/metrics/v1/metrics.pb.go | 25 +- .../profiles.pb.go} | 3518 ++++++++++------- .../profiles/v1experimental/profiles.pb.go | 1477 ------- .../internal/generated_wrapper_int32slice.go | 34 + .../internal/generated_wrapper_intslice.go | 34 + .../collector/pdata/internal/otlp/profiles.go | 2 +- .../pdata/internal/wrapper_profiles.go | 4 +- .../pdata/pcommon/generated_int32slice.go | 108 + .../pdata/pcommon/generated_intslice.go | 108 + vendor/modules.txt | 6 +- 13 files changed, 2450 insertions(+), 2970 deletions(-) rename vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/{v1experimental => v1development}/profiles_service.pb.go (84%) rename vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/{v1experimental/pprofextended.pb.go => v1development/profiles.pb.go} (56%) delete mode 100644 vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental/profiles.pb.go create mode 100644 vendor/go.opentelemetry.io/collector/pdata/internal/generated_wrapper_int32slice.go create mode 100644 vendor/go.opentelemetry.io/collector/pdata/internal/generated_wrapper_intslice.go create mode 100644 vendor/go.opentelemetry.io/collector/pdata/pcommon/generated_int32slice.go create mode 100644 vendor/go.opentelemetry.io/collector/pdata/pcommon/generated_intslice.go diff --git a/go.mod b/go.mod index 0db5700d14b03..d242de7e31902 100644 --- a/go.mod +++ b/go.mod @@ -142,7 +142,7 @@ require ( github.com/twmb/franz-go/plugin/kotel v1.5.0 github.com/twmb/franz-go/plugin/kprom v1.1.0 github.com/willf/bloom v2.0.3+incompatible - go.opentelemetry.io/collector/pdata v1.20.0 + go.opentelemetry.io/collector/pdata v1.21.0 go4.org/netipx v0.0.0-20230125063823-8449b0a6169f golang.org/x/oauth2 v0.24.0 golang.org/x/text v0.20.0 diff --git a/go.sum b/go.sum index dbf5091d71167..d27c746629d85 100644 --- a/go.sum +++ b/go.sum @@ -2750,8 +2750,8 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/collector/pdata v1.20.0 h1:ePcwt4bdtISP0loHaE+C9xYoU2ZkIvWv89Fob16o9SM= -go.opentelemetry.io/collector/pdata v1.20.0/go.mod h1:Ox1YVLe87cZDB/TL30i4SUz1cA5s6AM6SpFMfY61ICs= +go.opentelemetry.io/collector/pdata v1.21.0 h1:PG+UbiFMJ35X/WcAR7Rf/PWmWtRdW0aHlOidsR6c5MA= +go.opentelemetry.io/collector/pdata v1.21.0/go.mod h1:GKb1/zocKJMvxKbS+sl0W85lxhYBTFJ6h6I1tphVyDU= go.opentelemetry.io/collector/semconv v0.105.0 h1:8p6dZ3JfxFTjbY38d8xlQGB1TQ3nPUvs+D0RERniZ1g= go.opentelemetry.io/collector/semconv v0.105.0/go.mod h1:yMVUCNoQPZVq/IPfrHrnntZTWsLf5YGZ7qwKulIl5hw= go.opentelemetry.io/contrib/detectors/gcp v1.29.0 h1:TiaiXB4DpGD3sdzNlYQxruQngn5Apwzi1X0DRhuGvDQ= diff --git a/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1experimental/profiles_service.pb.go b/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1development/profiles_service.pb.go similarity index 84% rename from vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1experimental/profiles_service.pb.go rename to vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1development/profiles_service.pb.go index f998cd3f8c546..9e0475e68759b 100644 --- a/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1experimental/profiles_service.pb.go +++ b/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1development/profiles_service.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: opentelemetry/proto/collector/profiles/v1experimental/profiles_service.proto +// source: opentelemetry/proto/collector/profiles/v1development/profiles_service.proto -package v1experimental +package v1development import ( context "context" @@ -16,7 +16,7 @@ import ( codes "google.golang.org/grpc/codes" status "google.golang.org/grpc/status" - v1experimental "go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental" + v1development "go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1development" ) // Reference imports to suppress errors if they are not otherwise used. @@ -36,14 +36,14 @@ type ExportProfilesServiceRequest struct { // element. Intermediary nodes (such as OpenTelemetry Collector) that receive // data from multiple origins typically batch the data before forwarding further and // in that case this array will contain multiple elements. - ResourceProfiles []*v1experimental.ResourceProfiles `protobuf:"bytes,1,rep,name=resource_profiles,json=resourceProfiles,proto3" json:"resource_profiles,omitempty"` + ResourceProfiles []*v1development.ResourceProfiles `protobuf:"bytes,1,rep,name=resource_profiles,json=resourceProfiles,proto3" json:"resource_profiles,omitempty"` } func (m *ExportProfilesServiceRequest) Reset() { *m = ExportProfilesServiceRequest{} } func (m *ExportProfilesServiceRequest) String() string { return proto.CompactTextString(m) } func (*ExportProfilesServiceRequest) ProtoMessage() {} func (*ExportProfilesServiceRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_3d903b74e05b443d, []int{0} + return fileDescriptor_ad3943ce836e7720, []int{0} } func (m *ExportProfilesServiceRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -72,7 +72,7 @@ func (m *ExportProfilesServiceRequest) XXX_DiscardUnknown() { var xxx_messageInfo_ExportProfilesServiceRequest proto.InternalMessageInfo -func (m *ExportProfilesServiceRequest) GetResourceProfiles() []*v1experimental.ResourceProfiles { +func (m *ExportProfilesServiceRequest) GetResourceProfiles() []*v1development.ResourceProfiles { if m != nil { return m.ResourceProfiles } @@ -102,7 +102,7 @@ func (m *ExportProfilesServiceResponse) Reset() { *m = ExportProfilesSer func (m *ExportProfilesServiceResponse) String() string { return proto.CompactTextString(m) } func (*ExportProfilesServiceResponse) ProtoMessage() {} func (*ExportProfilesServiceResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_3d903b74e05b443d, []int{1} + return fileDescriptor_ad3943ce836e7720, []int{1} } func (m *ExportProfilesServiceResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -158,7 +158,7 @@ func (m *ExportProfilesPartialSuccess) Reset() { *m = ExportProfilesPart func (m *ExportProfilesPartialSuccess) String() string { return proto.CompactTextString(m) } func (*ExportProfilesPartialSuccess) ProtoMessage() {} func (*ExportProfilesPartialSuccess) Descriptor() ([]byte, []int) { - return fileDescriptor_3d903b74e05b443d, []int{2} + return fileDescriptor_ad3943ce836e7720, []int{2} } func (m *ExportProfilesPartialSuccess) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -202,45 +202,45 @@ func (m *ExportProfilesPartialSuccess) GetErrorMessage() string { } func init() { - proto.RegisterType((*ExportProfilesServiceRequest)(nil), "opentelemetry.proto.collector.profiles.v1experimental.ExportProfilesServiceRequest") - proto.RegisterType((*ExportProfilesServiceResponse)(nil), "opentelemetry.proto.collector.profiles.v1experimental.ExportProfilesServiceResponse") - proto.RegisterType((*ExportProfilesPartialSuccess)(nil), "opentelemetry.proto.collector.profiles.v1experimental.ExportProfilesPartialSuccess") + proto.RegisterType((*ExportProfilesServiceRequest)(nil), "opentelemetry.proto.collector.profiles.v1development.ExportProfilesServiceRequest") + proto.RegisterType((*ExportProfilesServiceResponse)(nil), "opentelemetry.proto.collector.profiles.v1development.ExportProfilesServiceResponse") + proto.RegisterType((*ExportProfilesPartialSuccess)(nil), "opentelemetry.proto.collector.profiles.v1development.ExportProfilesPartialSuccess") } func init() { - proto.RegisterFile("opentelemetry/proto/collector/profiles/v1experimental/profiles_service.proto", fileDescriptor_3d903b74e05b443d) -} - -var fileDescriptor_3d903b74e05b443d = []byte{ - // 437 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x53, 0x3f, 0xcb, 0xd3, 0x40, - 0x18, 0xcf, 0xf5, 0x95, 0x17, 0xbc, 0xaa, 0xd5, 0xd0, 0xa1, 0x14, 0x8d, 0x25, 0x2e, 0x01, 0xe1, - 0x42, 0x2b, 0x05, 0x11, 0x5c, 0x2a, 0xdd, 0x14, 0x43, 0x5a, 0x1c, 0x44, 0x08, 0x31, 0x7d, 0x0c, - 0x29, 0x69, 0xee, 0xbc, 0xbb, 0x96, 0xba, 0x89, 0xa3, 0x93, 0x1f, 0xc2, 0xc9, 0xdd, 0xef, 0x50, - 0xb7, 0x8e, 0x4e, 0x22, 0xed, 0x17, 0x79, 0x49, 0xae, 0x09, 0x4d, 0x68, 0x29, 0x94, 0x6e, 0x77, - 0xbf, 0xe3, 0xf7, 0xe7, 0xf9, 0x1d, 0x0f, 0x7e, 0x4d, 0x19, 0x24, 0x12, 0x62, 0x98, 0x81, 0xe4, - 0x5f, 0x6c, 0xc6, 0xa9, 0xa4, 0x76, 0x40, 0xe3, 0x18, 0x02, 0x49, 0x79, 0x7a, 0xff, 0x14, 0xc5, - 0x20, 0xec, 0x45, 0x17, 0x96, 0x0c, 0x78, 0x34, 0x83, 0x44, 0xfa, 0x71, 0x81, 0x7b, 0x02, 0xf8, - 0x22, 0x0a, 0x80, 0x64, 0x44, 0xbd, 0x5f, 0x52, 0x53, 0x20, 0x29, 0xd4, 0x48, 0xce, 0x22, 0x65, - 0xb5, 0x76, 0x33, 0xa4, 0x21, 0x55, 0xd6, 0xe9, 0x49, 0xf1, 0xda, 0x2f, 0x0e, 0x45, 0x3b, 0x15, - 0x48, 0x71, 0xcd, 0xef, 0x08, 0x3f, 0x1c, 0x2e, 0x19, 0xe5, 0xd2, 0xd9, 0x3d, 0x8c, 0x54, 0x50, - 0x17, 0x3e, 0xcf, 0x41, 0x48, 0x7d, 0x8a, 0x1f, 0x70, 0x10, 0x74, 0xce, 0x03, 0xf0, 0x72, 0x6e, - 0x0b, 0x75, 0xae, 0xac, 0x7a, 0xef, 0x25, 0x39, 0x34, 0xc5, 0x91, 0xec, 0xc4, 0xdd, 0xa9, 0xe4, - 0x3e, 0xee, 0x7d, 0x5e, 0x41, 0xcc, 0x9f, 0x08, 0x3f, 0x3a, 0x12, 0x46, 0x30, 0x9a, 0x08, 0xd0, - 0xbf, 0x21, 0xdc, 0x60, 0x3e, 0x97, 0x91, 0x1f, 0x7b, 0x62, 0x1e, 0x04, 0x20, 0xd2, 0x30, 0xc8, - 0xaa, 0xf7, 0x46, 0xe4, 0xac, 0x4a, 0x49, 0xd9, 0xcf, 0x51, 0xda, 0x23, 0x25, 0x3d, 0xb8, 0xb5, - 0xfa, 0xf7, 0x58, 0x73, 0xef, 0xb1, 0x12, 0x6a, 0xb2, 0x6a, 0x65, 0x65, 0x96, 0xfe, 0x34, 0xad, - 0x6c, 0x0a, 0x81, 0x84, 0xc9, 0x7e, 0x65, 0xc8, 0xba, 0x4a, 0x67, 0x56, 0x0f, 0x39, 0x55, 0x7f, - 0x82, 0xef, 0x02, 0xe7, 0x94, 0x7b, 0x33, 0x10, 0xc2, 0x0f, 0xa1, 0x55, 0xeb, 0x20, 0xeb, 0xb6, - 0x7b, 0x27, 0x03, 0xdf, 0x28, 0xac, 0xf7, 0x07, 0xe1, 0x46, 0xa5, 0x12, 0xfd, 0x37, 0xc2, 0xd7, - 0x2a, 0x86, 0x7e, 0x99, 0xd9, 0xcb, 0x1f, 0xdf, 0x1e, 0x5f, 0x56, 0x54, 0x7d, 0xa0, 0xa9, 0x0d, - 0xbe, 0xd6, 0x56, 0x1b, 0x03, 0xad, 0x37, 0x06, 0xfa, 0xbf, 0x31, 0xd0, 0x8f, 0xad, 0xa1, 0xad, - 0xb7, 0x86, 0xf6, 0x77, 0x6b, 0x68, 0xf8, 0x79, 0x44, 0xcf, 0x33, 0x1d, 0x34, 0x2b, 0x7e, 0x4e, - 0xca, 0x73, 0xd0, 0xfb, 0x0f, 0x61, 0x55, 0x31, 0x2a, 0x6d, 0xed, 0xc4, 0x97, 0xbe, 0x1d, 0x25, - 0x12, 0x78, 0xe2, 0xc7, 0x76, 0x76, 0xcb, 0x2c, 0x43, 0x48, 0x4e, 0x2f, 0xf7, 0xaf, 0x5a, 0xff, - 0x2d, 0x83, 0x64, 0x5c, 0x68, 0x67, 0xae, 0xe4, 0x55, 0x91, 0x36, 0x0f, 0x45, 0xde, 0x75, 0x87, - 0x7b, 0xbc, 0x8f, 0xd7, 0x99, 0xc7, 0xb3, 0x9b, 0x00, 0x00, 0x00, 0xff, 0xff, 0x8e, 0x3d, 0x57, - 0xb0, 0x54, 0x04, 0x00, 0x00, + proto.RegisterFile("opentelemetry/proto/collector/profiles/v1development/profiles_service.proto", fileDescriptor_ad3943ce836e7720) +} + +var fileDescriptor_ad3943ce836e7720 = []byte{ + // 438 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x53, 0x4d, 0x8b, 0xd3, 0x40, + 0x18, 0xce, 0xb4, 0x52, 0x70, 0xaa, 0x56, 0x43, 0x0f, 0xa5, 0x68, 0x2c, 0xf1, 0x12, 0x10, 0x26, + 0xb4, 0x16, 0x44, 0xf0, 0x54, 0xf5, 0x24, 0x62, 0x48, 0xc5, 0x83, 0x1e, 0x42, 0x4c, 0x5f, 0x43, + 0x24, 0xcd, 0x8c, 0x33, 0xd3, 0xa2, 0x47, 0x8f, 0xde, 0xf6, 0x3f, 0xec, 0x6d, 0xaf, 0xfb, 0x23, + 0xb6, 0xc7, 0x1e, 0xf7, 0xb4, 0x2c, 0xed, 0xef, 0x58, 0x58, 0x92, 0x69, 0xb2, 0x9b, 0xd0, 0xa5, + 0x50, 0x7a, 0x9b, 0x79, 0x86, 0xe7, 0xe3, 0x7d, 0x86, 0x17, 0x7f, 0xa4, 0x0c, 0x12, 0x09, 0x31, + 0x4c, 0x41, 0xf2, 0xbf, 0x36, 0xe3, 0x54, 0x52, 0x3b, 0xa0, 0x71, 0x0c, 0x81, 0xa4, 0x3c, 0xbd, + 0xff, 0x8c, 0x62, 0x10, 0xf6, 0xbc, 0x3f, 0x81, 0x39, 0xc4, 0x94, 0x4d, 0x21, 0x91, 0x05, 0xec, + 0x09, 0xe0, 0xf3, 0x28, 0x00, 0x92, 0xf1, 0xf4, 0x61, 0x49, 0x4c, 0x81, 0xa4, 0x10, 0x23, 0x39, + 0x8b, 0x94, 0xc4, 0xba, 0xed, 0x90, 0x86, 0x54, 0x19, 0xa7, 0x27, 0x45, 0xeb, 0xbe, 0xd9, 0x16, + 0x6c, 0x47, 0x1c, 0x45, 0x35, 0xff, 0x23, 0xfc, 0xf4, 0xc3, 0x1f, 0x46, 0xb9, 0x74, 0x36, 0x0f, + 0x63, 0x15, 0xd3, 0x85, 0xdf, 0x33, 0x10, 0x52, 0x8f, 0xf0, 0x13, 0x0e, 0x82, 0xce, 0x78, 0x00, + 0x5e, 0xce, 0xed, 0xa0, 0x5e, 0xdd, 0x6a, 0x0e, 0xde, 0x92, 0x6d, 0x33, 0x6c, 0x4f, 0x4e, 0xdc, + 0x8d, 0x48, 0x6e, 0xe3, 0x3e, 0xe6, 0x15, 0xc4, 0x3c, 0x46, 0xf8, 0xd9, 0x1d, 0x59, 0x04, 0xa3, + 0x89, 0x00, 0xfd, 0x1f, 0xc2, 0x2d, 0xe6, 0x73, 0x19, 0xf9, 0xb1, 0x27, 0x66, 0x41, 0x00, 0x22, + 0xcd, 0x82, 0xac, 0xe6, 0xc0, 0x25, 0xfb, 0xf4, 0x49, 0xca, 0x76, 0x8e, 0x92, 0x1e, 0x2b, 0xe5, + 0xd1, 0xbd, 0xc5, 0xc5, 0x73, 0xcd, 0x7d, 0xc4, 0x4a, 0xa8, 0xc9, 0xaa, 0x85, 0x95, 0x59, 0xfa, + 0xcb, 0xb4, 0xb0, 0x5f, 0x10, 0x48, 0x98, 0xdc, 0x2e, 0x0c, 0x59, 0xf5, 0x74, 0x64, 0xf5, 0x90, + 0x53, 0xf5, 0x17, 0xf8, 0x21, 0x70, 0x4e, 0xb9, 0x37, 0x05, 0x21, 0xfc, 0x10, 0x3a, 0xb5, 0x1e, + 0xb2, 0xee, 0xbb, 0x0f, 0x32, 0xf0, 0x93, 0xc2, 0x06, 0x67, 0x08, 0xb7, 0x2a, 0x8d, 0xe8, 0xa7, + 0x08, 0x37, 0x54, 0x0c, 0xfd, 0x20, 0xa3, 0x97, 0x7f, 0xbd, 0x3b, 0x3e, 0xa8, 0xa6, 0xfa, 0x3d, + 0x53, 0x1b, 0x5d, 0xa1, 0xc5, 0xca, 0x40, 0xcb, 0x95, 0x81, 0x2e, 0x57, 0x06, 0x3a, 0x5a, 0x1b, + 0xda, 0x72, 0x6d, 0x68, 0xe7, 0x6b, 0x43, 0xc3, 0xaf, 0x23, 0xba, 0x97, 0xe7, 0xa8, 0x5d, 0xb1, + 0x73, 0x52, 0x9a, 0x83, 0xbe, 0x7d, 0x0f, 0xab, 0x82, 0x51, 0x69, 0x5b, 0x27, 0xbe, 0xf4, 0xed, + 0x28, 0x91, 0xc0, 0x13, 0x3f, 0xb6, 0xb3, 0x5b, 0xe6, 0x18, 0x42, 0xb2, 0x73, 0xa9, 0x4f, 0x6a, + 0xc3, 0xcf, 0x0c, 0x92, 0x2f, 0x85, 0x74, 0x66, 0x4a, 0xde, 0x15, 0x59, 0xf3, 0x4c, 0xe4, 0x6b, + 0xff, 0xfd, 0x0d, 0xed, 0x47, 0x23, 0x73, 0x78, 0x75, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x8f, 0x35, + 0x8c, 0xea, 0x4a, 0x04, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -270,7 +270,7 @@ func NewProfilesServiceClient(cc *grpc.ClientConn) ProfilesServiceClient { func (c *profilesServiceClient) Export(ctx context.Context, in *ExportProfilesServiceRequest, opts ...grpc.CallOption) (*ExportProfilesServiceResponse, error) { out := new(ExportProfilesServiceResponse) - err := c.cc.Invoke(ctx, "/opentelemetry.proto.collector.profiles.v1experimental.ProfilesService/Export", in, out, opts...) + err := c.cc.Invoke(ctx, "/opentelemetry.proto.collector.profiles.v1development.ProfilesService/Export", in, out, opts...) if err != nil { return nil, err } @@ -306,7 +306,7 @@ func _ProfilesService_Export_Handler(srv interface{}, ctx context.Context, dec f } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/opentelemetry.proto.collector.profiles.v1experimental.ProfilesService/Export", + FullMethod: "/opentelemetry.proto.collector.profiles.v1development.ProfilesService/Export", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(ProfilesServiceServer).Export(ctx, req.(*ExportProfilesServiceRequest)) @@ -315,7 +315,7 @@ func _ProfilesService_Export_Handler(srv interface{}, ctx context.Context, dec f } var _ProfilesService_serviceDesc = grpc.ServiceDesc{ - ServiceName: "opentelemetry.proto.collector.profiles.v1experimental.ProfilesService", + ServiceName: "opentelemetry.proto.collector.profiles.v1development.ProfilesService", HandlerType: (*ProfilesServiceServer)(nil), Methods: []grpc.MethodDesc{ { @@ -324,7 +324,7 @@ var _ProfilesService_serviceDesc = grpc.ServiceDesc{ }, }, Streams: []grpc.StreamDesc{}, - Metadata: "opentelemetry/proto/collector/profiles/v1experimental/profiles_service.proto", + Metadata: "opentelemetry/proto/collector/profiles/v1development/profiles_service.proto", } func (m *ExportProfilesServiceRequest) Marshal() (dAtA []byte, err error) { @@ -549,7 +549,7 @@ func (m *ExportProfilesServiceRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ResourceProfiles = append(m.ResourceProfiles, &v1experimental.ResourceProfiles{}) + m.ResourceProfiles = append(m.ResourceProfiles, &v1development.ResourceProfiles{}) if err := m.ResourceProfiles[len(m.ResourceProfiles)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/metrics/v1/metrics.pb.go b/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/metrics/v1/metrics.pb.go index 3649bd83f81ba..6614333796419 100644 --- a/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/metrics/v1/metrics.pb.go +++ b/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/metrics/v1/metrics.pb.go @@ -160,6 +160,25 @@ func (DataPointFlags) EnumDescriptor() ([]byte, []int) { // storage, OR can be embedded by other protocols that transfer OTLP metrics // data but do not implement the OTLP protocol. // +// MetricsData +// └─── ResourceMetrics +// +// ├── Resource +// ├── SchemaURL +// └── ScopeMetrics +// ├── Scope +// ├── SchemaURL +// └── Metric +// ├── Name +// ├── Description +// ├── Unit +// └── data +// ├── Gauge +// ├── Sum +// ├── Histogram +// ├── ExponentialHistogram +// └── Summary +// // The main difference between this message and collector protocol is that // in this message there will not be any "control" or "metadata" specific to // OTLP protocol. @@ -855,6 +874,9 @@ func (m *ExponentialHistogram) GetAggregationTemporality() AggregationTemporalit // data type. These data points cannot always be merged in a meaningful way. // While they can be useful in some applications, histogram data points are // recommended for new applications. +// Summary metrics do not have an aggregation temporality field. This is +// because the count and sum fields of a SummaryDataPoint are assumed to be +// cumulative values. type Summary struct { DataPoints []*SummaryDataPoint `protobuf:"bytes,1,rep,name=data_points,json=dataPoints,proto3" json:"data_points,omitempty"` } @@ -1644,7 +1666,8 @@ func (m *ExponentialHistogramDataPoint_Buckets) GetBucketCounts() []uint64 { } // SummaryDataPoint is a single data point in a timeseries that describes the -// time-varying values of a Summary metric. +// time-varying values of a Summary metric. The count and sum fields represent +// cumulative values. type SummaryDataPoint struct { // The set of key/value pairs that uniquely identify the timeseries from // where this point belongs. The list may be empty (may contain 0 elements). diff --git a/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental/pprofextended.pb.go b/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1development/profiles.pb.go similarity index 56% rename from vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental/pprofextended.pb.go rename to vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1development/profiles.pb.go index 7b90b35373fb1..faa8630469c8b 100644 --- a/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental/pprofextended.pb.go +++ b/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1development/profiles.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: opentelemetry/proto/profiles/v1experimental/pprofextended.proto +// source: opentelemetry/proto/profiles/v1development/profiles.proto -package v1experimental +package v1development import ( fmt "fmt" @@ -13,7 +13,8 @@ import ( proto "github.com/gogo/protobuf/proto" go_opentelemetry_io_collector_pdata_internal_data "go.opentelemetry.io/collector/pdata/internal/data" - v1 "go.opentelemetry.io/collector/pdata/internal/data/protogen/common/v1" + v11 "go.opentelemetry.io/collector/pdata/internal/data/protogen/common/v1" + v1 "go.opentelemetry.io/collector/pdata/internal/data/protogen/resource/v1" ) // Reference imports to suppress errors if they are not otherwise used. @@ -88,7 +89,7 @@ const ( //11. A request is received, the system measures 1 request. //12. The 1 second collection cycle ends. A metric is exported for the //number of requests received over the interval of time t_1 to - //t_0+1 with a value of 1. + //t_1+1 with a value of 1. // //Note: Even though, when reporting changes since last report time, using //CUMULATIVE is valid, it is not recommended. @@ -112,42 +113,213 @@ func (x AggregationTemporality) String() string { } func (AggregationTemporality) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{0} + return fileDescriptor_ddd0cf081a2fe76f, []int{0} +} + +// ProfilesData represents the profiles data that can be stored in persistent storage, +// OR can be embedded by other protocols that transfer OTLP profiles data but do not +// implement the OTLP protocol. +// +// The main difference between this message and collector protocol is that +// in this message there will not be any "control" or "metadata" specific to +// OTLP protocol. +// +// When new fields are added into this message, the OTLP request MUST be updated +// as well. +type ProfilesData struct { + // An array of ResourceProfiles. + // For data coming from a single resource this array will typically contain + // one element. Intermediary nodes that receive data from multiple origins + // typically batch the data before forwarding further and in that case this + // array will contain multiple elements. + ResourceProfiles []*ResourceProfiles `protobuf:"bytes,1,rep,name=resource_profiles,json=resourceProfiles,proto3" json:"resource_profiles,omitempty"` +} + +func (m *ProfilesData) Reset() { *m = ProfilesData{} } +func (m *ProfilesData) String() string { return proto.CompactTextString(m) } +func (*ProfilesData) ProtoMessage() {} +func (*ProfilesData) Descriptor() ([]byte, []int) { + return fileDescriptor_ddd0cf081a2fe76f, []int{0} +} +func (m *ProfilesData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProfilesData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ProfilesData.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ProfilesData) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProfilesData.Merge(m, src) +} +func (m *ProfilesData) XXX_Size() int { + return m.Size() +} +func (m *ProfilesData) XXX_DiscardUnknown() { + xxx_messageInfo_ProfilesData.DiscardUnknown(m) } -// Indicates the semantics of the build_id field. -type BuildIdKind int32 +var xxx_messageInfo_ProfilesData proto.InternalMessageInfo -const ( - // Linker-generated build ID, stored in the ELF binary notes. - BuildIdKind_BUILD_ID_LINKER BuildIdKind = 0 - // Build ID based on the content hash of the binary. Currently no particular - // hashing approach is standardized, so a given producer needs to define it - // themselves and thus unlike BUILD_ID_LINKER this kind of hash is producer-specific. - // We may choose to provide a standardized stable hash recommendation later. - BuildIdKind_BUILD_ID_BINARY_HASH BuildIdKind = 1 -) +func (m *ProfilesData) GetResourceProfiles() []*ResourceProfiles { + if m != nil { + return m.ResourceProfiles + } + return nil +} + +// A collection of ScopeProfiles from a Resource. +type ResourceProfiles struct { + // The resource for the profiles in this message. + // If this field is not set then no resource info is known. + Resource v1.Resource `protobuf:"bytes,1,opt,name=resource,proto3" json:"resource"` + // A list of ScopeProfiles that originate from a resource. + ScopeProfiles []*ScopeProfiles `protobuf:"bytes,2,rep,name=scope_profiles,json=scopeProfiles,proto3" json:"scope_profiles,omitempty"` + // The Schema URL, if known. This is the identifier of the Schema that the resource data + // is recorded in. To learn more about Schema URL see + // https://opentelemetry.io/docs/specs/otel/schemas/#schema-url + // This schema_url applies to the data in the "resource" field. It does not apply + // to the data in the "scope_profiles" field which have their own schema_url field. + SchemaUrl string `protobuf:"bytes,3,opt,name=schema_url,json=schemaUrl,proto3" json:"schema_url,omitempty"` +} + +func (m *ResourceProfiles) Reset() { *m = ResourceProfiles{} } +func (m *ResourceProfiles) String() string { return proto.CompactTextString(m) } +func (*ResourceProfiles) ProtoMessage() {} +func (*ResourceProfiles) Descriptor() ([]byte, []int) { + return fileDescriptor_ddd0cf081a2fe76f, []int{1} +} +func (m *ResourceProfiles) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResourceProfiles) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ResourceProfiles.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ResourceProfiles) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResourceProfiles.Merge(m, src) +} +func (m *ResourceProfiles) XXX_Size() int { + return m.Size() +} +func (m *ResourceProfiles) XXX_DiscardUnknown() { + xxx_messageInfo_ResourceProfiles.DiscardUnknown(m) +} + +var xxx_messageInfo_ResourceProfiles proto.InternalMessageInfo + +func (m *ResourceProfiles) GetResource() v1.Resource { + if m != nil { + return m.Resource + } + return v1.Resource{} +} + +func (m *ResourceProfiles) GetScopeProfiles() []*ScopeProfiles { + if m != nil { + return m.ScopeProfiles + } + return nil +} -var BuildIdKind_name = map[int32]string{ - 0: "BUILD_ID_LINKER", - 1: "BUILD_ID_BINARY_HASH", +func (m *ResourceProfiles) GetSchemaUrl() string { + if m != nil { + return m.SchemaUrl + } + return "" +} + +// A collection of Profiles produced by an InstrumentationScope. +type ScopeProfiles struct { + // The instrumentation scope information for the profiles in this message. + // Semantically when InstrumentationScope isn't set, it is equivalent with + // an empty instrumentation scope name (unknown). + Scope v11.InstrumentationScope `protobuf:"bytes,1,opt,name=scope,proto3" json:"scope"` + // A list of Profiles that originate from an instrumentation scope. + Profiles []*Profile `protobuf:"bytes,2,rep,name=profiles,proto3" json:"profiles,omitempty"` + // The Schema URL, if known. This is the identifier of the Schema that the profile data + // is recorded in. To learn more about Schema URL see + // https://opentelemetry.io/docs/specs/otel/schemas/#schema-url + // This schema_url applies to all profiles in the "profiles" field. + SchemaUrl string `protobuf:"bytes,3,opt,name=schema_url,json=schemaUrl,proto3" json:"schema_url,omitempty"` +} + +func (m *ScopeProfiles) Reset() { *m = ScopeProfiles{} } +func (m *ScopeProfiles) String() string { return proto.CompactTextString(m) } +func (*ScopeProfiles) ProtoMessage() {} +func (*ScopeProfiles) Descriptor() ([]byte, []int) { + return fileDescriptor_ddd0cf081a2fe76f, []int{2} +} +func (m *ScopeProfiles) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ScopeProfiles) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ScopeProfiles.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ScopeProfiles) XXX_Merge(src proto.Message) { + xxx_messageInfo_ScopeProfiles.Merge(m, src) +} +func (m *ScopeProfiles) XXX_Size() int { + return m.Size() +} +func (m *ScopeProfiles) XXX_DiscardUnknown() { + xxx_messageInfo_ScopeProfiles.DiscardUnknown(m) } -var BuildIdKind_value = map[string]int32{ - "BUILD_ID_LINKER": 0, - "BUILD_ID_BINARY_HASH": 1, +var xxx_messageInfo_ScopeProfiles proto.InternalMessageInfo + +func (m *ScopeProfiles) GetScope() v11.InstrumentationScope { + if m != nil { + return m.Scope + } + return v11.InstrumentationScope{} } -func (x BuildIdKind) String() string { - return proto.EnumName(BuildIdKind_name, int32(x)) +func (m *ScopeProfiles) GetProfiles() []*Profile { + if m != nil { + return m.Profiles + } + return nil } -func (BuildIdKind) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{1} +func (m *ScopeProfiles) GetSchemaUrl() string { + if m != nil { + return m.SchemaUrl + } + return "" } // Represents a complete profile, including sample types, samples, // mappings to binaries, locations, functions, string table, and additional metadata. +// It modifies and annotates pprof Profile with OpenTelemetry specific fields. +// +// Note that whilst fields in this message retain the name and field id from pprof in most cases +// for ease of understanding data migration, it is not intended that pprof:Profile and +// OpenTelemetry:Profile encoding be wire compatible. type Profile struct { // A description of the samples associated with each Sample.value. // For a cpu profile this might be: @@ -162,53 +334,80 @@ type Profile struct { Sample []*Sample `protobuf:"bytes,2,rep,name=sample,proto3" json:"sample,omitempty"` // Mapping from address ranges to the image/binary/library mapped // into that address range. mapping[0] will be the main binary. - Mapping []*Mapping `protobuf:"bytes,3,rep,name=mapping,proto3" json:"mapping,omitempty"` + // If multiple binaries contribute to the Profile and no main + // binary can be identified, mapping[0] has no special meaning. + MappingTable []*Mapping `protobuf:"bytes,3,rep,name=mapping_table,json=mappingTable,proto3" json:"mapping_table,omitempty"` // Locations referenced by samples via location_indices. - Location []*Location `protobuf:"bytes,4,rep,name=location,proto3" json:"location,omitempty"` + LocationTable []*Location `protobuf:"bytes,4,rep,name=location_table,json=locationTable,proto3" json:"location_table,omitempty"` // Array of locations referenced by samples. - LocationIndices []int64 `protobuf:"varint,15,rep,packed,name=location_indices,json=locationIndices,proto3" json:"location_indices,omitempty"` + LocationIndices []int32 `protobuf:"varint,5,rep,packed,name=location_indices,json=locationIndices,proto3" json:"location_indices,omitempty"` // Functions referenced by locations. - Function []*Function `protobuf:"bytes,5,rep,name=function,proto3" json:"function,omitempty"` + FunctionTable []*Function `protobuf:"bytes,6,rep,name=function_table,json=functionTable,proto3" json:"function_table,omitempty"` // Lookup table for attributes. - AttributeTable []v1.KeyValue `protobuf:"bytes,16,rep,name=attribute_table,json=attributeTable,proto3" json:"attribute_table"` + AttributeTable []v11.KeyValue `protobuf:"bytes,7,rep,name=attribute_table,json=attributeTable,proto3" json:"attribute_table"` // Represents a mapping between Attribute Keys and Units. - AttributeUnits []*AttributeUnit `protobuf:"bytes,17,rep,name=attribute_units,json=attributeUnits,proto3" json:"attribute_units,omitempty"` + AttributeUnits []*AttributeUnit `protobuf:"bytes,8,rep,name=attribute_units,json=attributeUnits,proto3" json:"attribute_units,omitempty"` // Lookup table for links. - LinkTable []*Link `protobuf:"bytes,18,rep,name=link_table,json=linkTable,proto3" json:"link_table,omitempty"` + LinkTable []*Link `protobuf:"bytes,9,rep,name=link_table,json=linkTable,proto3" json:"link_table,omitempty"` // A common table for strings referenced by various messages. // string_table[0] must always be "". - StringTable []string `protobuf:"bytes,6,rep,name=string_table,json=stringTable,proto3" json:"string_table,omitempty"` - // frames with Function.function_name fully matching the following - // regexp will be dropped from the samples, along with their successors. - DropFrames int64 `protobuf:"varint,7,opt,name=drop_frames,json=dropFrames,proto3" json:"drop_frames,omitempty"` - // frames with Function.function_name fully matching the following - // regexp will be kept, even if it matches drop_frames. - KeepFrames int64 `protobuf:"varint,8,opt,name=keep_frames,json=keepFrames,proto3" json:"keep_frames,omitempty"` + StringTable []string `protobuf:"bytes,10,rep,name=string_table,json=stringTable,proto3" json:"string_table,omitempty"` // Time of collection (UTC) represented as nanoseconds past the epoch. - TimeNanos int64 `protobuf:"varint,9,opt,name=time_nanos,json=timeNanos,proto3" json:"time_nanos,omitempty"` + TimeNanos int64 `protobuf:"varint,11,opt,name=time_nanos,json=timeNanos,proto3" json:"time_nanos,omitempty"` // Duration of the profile, if a duration makes sense. - DurationNanos int64 `protobuf:"varint,10,opt,name=duration_nanos,json=durationNanos,proto3" json:"duration_nanos,omitempty"` + DurationNanos int64 `protobuf:"varint,12,opt,name=duration_nanos,json=durationNanos,proto3" json:"duration_nanos,omitempty"` // The kind of events between sampled occurrences. // e.g [ "cpu","cycles" ] or [ "heap","bytes" ] - PeriodType ValueType `protobuf:"bytes,11,opt,name=period_type,json=periodType,proto3" json:"period_type"` + PeriodType ValueType `protobuf:"bytes,13,opt,name=period_type,json=periodType,proto3" json:"period_type"` // The number of events between sampled occurrences. - Period int64 `protobuf:"varint,12,opt,name=period,proto3" json:"period,omitempty"` + Period int64 `protobuf:"varint,14,opt,name=period,proto3" json:"period,omitempty"` // Free-form text associated with the profile. The text is displayed as is // to the user by the tools that read profiles (e.g. by pprof). This field // should not be used to store any machine-readable information, it is only // for human-friendly content. The profile must stay functional if this field // is cleaned. - Comment []int64 `protobuf:"varint,13,rep,packed,name=comment,proto3" json:"comment,omitempty"` + CommentStrindices []int32 `protobuf:"varint,15,rep,packed,name=comment_strindices,json=commentStrindices,proto3" json:"comment_strindices,omitempty"` // Index into the string table of the type of the preferred sample // value. If unset, clients should default to the last sample value. - DefaultSampleType int64 `protobuf:"varint,14,opt,name=default_sample_type,json=defaultSampleType,proto3" json:"default_sample_type,omitempty"` + DefaultSampleTypeStrindex int32 `protobuf:"varint,16,opt,name=default_sample_type_strindex,json=defaultSampleTypeStrindex,proto3" json:"default_sample_type_strindex,omitempty"` + // A globally unique identifier for a profile. The ID is a 16-byte array. An ID with + // all zeroes is considered invalid. + // + // This field is required. + ProfileId go_opentelemetry_io_collector_pdata_internal_data.ProfileID `protobuf:"bytes,17,opt,name=profile_id,json=profileId,proto3,customtype=go.opentelemetry.io/collector/pdata/internal/data.ProfileID" json:"profile_id"` + // attributes is a collection of key/value pairs. Note, global attributes + // like server name can be set using the resource API. Examples of attributes: + // + // "/http/user_agent": "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_14_2) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/71.0.3578.98 Safari/537.36" + // "/http/server_latency": 300 + // "abc.com/myattribute": true + // "abc.com/score": 10.239 + // + // The OpenTelemetry API specification further restricts the allowed value types: + // https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/common/README.md#attribute + // Attribute keys MUST be unique (it is not allowed to have more than one + // attribute with the same key). + Attributes []v11.KeyValue `protobuf:"bytes,18,rep,name=attributes,proto3" json:"attributes"` + // dropped_attributes_count is the number of attributes that were discarded. Attributes + // can be discarded because their keys are too long or because there are too many + // attributes. If this value is 0, then no attributes were dropped. + DroppedAttributesCount uint32 `protobuf:"varint,19,opt,name=dropped_attributes_count,json=droppedAttributesCount,proto3" json:"dropped_attributes_count,omitempty"` + // Specifies format of the original payload. Common values are defined in semantic conventions. [required if original_payload is present] + OriginalPayloadFormat string `protobuf:"bytes,20,opt,name=original_payload_format,json=originalPayloadFormat,proto3" json:"original_payload_format,omitempty"` + // Original payload can be stored in this field. This can be useful for users who want to get the original payload. + // Formats such as JFR are highly extensible and can contain more information than what is defined in this spec. + // Inclusion of original payload should be configurable by the user. Default behavior should be to not include the original payload. + // If the original payload is in pprof format, it SHOULD not be included in this field. + // The field is optional, however if it is present then equivalent converted data should be populated in other fields + // of this message as far as is practicable. + OriginalPayload []byte `protobuf:"bytes,21,opt,name=original_payload,json=originalPayload,proto3" json:"original_payload,omitempty"` } func (m *Profile) Reset() { *m = Profile{} } func (m *Profile) String() string { return proto.CompactTextString(m) } func (*Profile) ProtoMessage() {} func (*Profile) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{0} + return fileDescriptor_ddd0cf081a2fe76f, []int{3} } func (m *Profile) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -251,35 +450,35 @@ func (m *Profile) GetSample() []*Sample { return nil } -func (m *Profile) GetMapping() []*Mapping { +func (m *Profile) GetMappingTable() []*Mapping { if m != nil { - return m.Mapping + return m.MappingTable } return nil } -func (m *Profile) GetLocation() []*Location { +func (m *Profile) GetLocationTable() []*Location { if m != nil { - return m.Location + return m.LocationTable } return nil } -func (m *Profile) GetLocationIndices() []int64 { +func (m *Profile) GetLocationIndices() []int32 { if m != nil { return m.LocationIndices } return nil } -func (m *Profile) GetFunction() []*Function { +func (m *Profile) GetFunctionTable() []*Function { if m != nil { - return m.Function + return m.FunctionTable } return nil } -func (m *Profile) GetAttributeTable() []v1.KeyValue { +func (m *Profile) GetAttributeTable() []v11.KeyValue { if m != nil { return m.AttributeTable } @@ -307,20 +506,6 @@ func (m *Profile) GetStringTable() []string { return nil } -func (m *Profile) GetDropFrames() int64 { - if m != nil { - return m.DropFrames - } - return 0 -} - -func (m *Profile) GetKeepFrames() int64 { - if m != nil { - return m.KeepFrames - } - return 0 -} - func (m *Profile) GetTimeNanos() int64 { if m != nil { return m.TimeNanos @@ -349,33 +534,61 @@ func (m *Profile) GetPeriod() int64 { return 0 } -func (m *Profile) GetComment() []int64 { +func (m *Profile) GetCommentStrindices() []int32 { + if m != nil { + return m.CommentStrindices + } + return nil +} + +func (m *Profile) GetDefaultSampleTypeStrindex() int32 { + if m != nil { + return m.DefaultSampleTypeStrindex + } + return 0 +} + +func (m *Profile) GetAttributes() []v11.KeyValue { if m != nil { - return m.Comment + return m.Attributes } return nil } -func (m *Profile) GetDefaultSampleType() int64 { +func (m *Profile) GetDroppedAttributesCount() uint32 { if m != nil { - return m.DefaultSampleType + return m.DroppedAttributesCount } return 0 } +func (m *Profile) GetOriginalPayloadFormat() string { + if m != nil { + return m.OriginalPayloadFormat + } + return "" +} + +func (m *Profile) GetOriginalPayload() []byte { + if m != nil { + return m.OriginalPayload + } + return nil +} + // Represents a mapping between Attribute Keys and Units. type AttributeUnit struct { // Index into string table. - AttributeKey int64 `protobuf:"varint,1,opt,name=attribute_key,json=attributeKey,proto3" json:"attribute_key,omitempty"` + AttributeKeyStrindex int32 `protobuf:"varint,1,opt,name=attribute_key_strindex,json=attributeKeyStrindex,proto3" json:"attribute_key_strindex,omitempty"` // Index into string table. - Unit int64 `protobuf:"varint,2,opt,name=unit,proto3" json:"unit,omitempty"` + UnitStrindex int32 `protobuf:"varint,2,opt,name=unit_strindex,json=unitStrindex,proto3" json:"unit_strindex,omitempty"` } func (m *AttributeUnit) Reset() { *m = AttributeUnit{} } func (m *AttributeUnit) String() string { return proto.CompactTextString(m) } func (*AttributeUnit) ProtoMessage() {} func (*AttributeUnit) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{1} + return fileDescriptor_ddd0cf081a2fe76f, []int{4} } func (m *AttributeUnit) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -404,16 +617,16 @@ func (m *AttributeUnit) XXX_DiscardUnknown() { var xxx_messageInfo_AttributeUnit proto.InternalMessageInfo -func (m *AttributeUnit) GetAttributeKey() int64 { +func (m *AttributeUnit) GetAttributeKeyStrindex() int32 { if m != nil { - return m.AttributeKey + return m.AttributeKeyStrindex } return 0 } -func (m *AttributeUnit) GetUnit() int64 { +func (m *AttributeUnit) GetUnitStrindex() int32 { if m != nil { - return m.Unit + return m.UnitStrindex } return 0 } @@ -432,7 +645,7 @@ func (m *Link) Reset() { *m = Link{} } func (m *Link) String() string { return proto.CompactTextString(m) } func (*Link) ProtoMessage() {} func (*Link) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{2} + return fileDescriptor_ddd0cf081a2fe76f, []int{5} } func (m *Link) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -463,16 +676,16 @@ var xxx_messageInfo_Link proto.InternalMessageInfo // ValueType describes the type and units of a value, with an optional aggregation temporality. type ValueType struct { - Type int64 `protobuf:"varint,1,opt,name=type,proto3" json:"type,omitempty"` - Unit int64 `protobuf:"varint,2,opt,name=unit,proto3" json:"unit,omitempty"` - AggregationTemporality AggregationTemporality `protobuf:"varint,3,opt,name=aggregation_temporality,json=aggregationTemporality,proto3,enum=opentelemetry.proto.profiles.v1experimental.AggregationTemporality" json:"aggregation_temporality,omitempty"` + TypeStrindex int32 `protobuf:"varint,1,opt,name=type_strindex,json=typeStrindex,proto3" json:"type_strindex,omitempty"` + UnitStrindex int32 `protobuf:"varint,2,opt,name=unit_strindex,json=unitStrindex,proto3" json:"unit_strindex,omitempty"` + AggregationTemporality AggregationTemporality `protobuf:"varint,3,opt,name=aggregation_temporality,json=aggregationTemporality,proto3,enum=opentelemetry.proto.profiles.v1development.AggregationTemporality" json:"aggregation_temporality,omitempty"` } func (m *ValueType) Reset() { *m = ValueType{} } func (m *ValueType) String() string { return proto.CompactTextString(m) } func (*ValueType) ProtoMessage() {} func (*ValueType) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{3} + return fileDescriptor_ddd0cf081a2fe76f, []int{6} } func (m *ValueType) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -501,16 +714,16 @@ func (m *ValueType) XXX_DiscardUnknown() { var xxx_messageInfo_ValueType proto.InternalMessageInfo -func (m *ValueType) GetType() int64 { +func (m *ValueType) GetTypeStrindex() int32 { if m != nil { - return m.Type + return m.TypeStrindex } return 0 } -func (m *ValueType) GetUnit() int64 { +func (m *ValueType) GetUnitStrindex() int32 { if m != nil { - return m.Unit + return m.UnitStrindex } return 0 } @@ -527,48 +740,35 @@ func (m *ValueType) GetAggregationTemporality() AggregationTemporality { // augmented with auxiliary information like the thread-id, some // indicator of a higher level request being handled etc. type Sample struct { - // The indices recorded here correspond to locations in Profile.location. - // The leaf is at location_index[0]. [deprecated, superseded by locations_start_index / locations_length] - LocationIndex []uint64 `protobuf:"varint,1,rep,packed,name=location_index,json=locationIndex,proto3" json:"location_index,omitempty"` - // locations_start_index along with locations_length refers to to a slice of locations in Profile.location. - // Supersedes location_index. - LocationsStartIndex uint64 `protobuf:"varint,7,opt,name=locations_start_index,json=locationsStartIndex,proto3" json:"locations_start_index,omitempty"` - // locations_length along with locations_start_index refers to a slice of locations in Profile.location. + // locations_start_index along with locations_length refers to to a slice of locations in Profile.location_indices. + LocationsStartIndex int32 `protobuf:"varint,1,opt,name=locations_start_index,json=locationsStartIndex,proto3" json:"locations_start_index,omitempty"` + // locations_length along with locations_start_index refers to a slice of locations in Profile.location_indices. // Supersedes location_index. - LocationsLength uint64 `protobuf:"varint,8,opt,name=locations_length,json=locationsLength,proto3" json:"locations_length,omitempty"` - // A 128bit id that uniquely identifies this stacktrace, globally. Index into string table. [optional] - StacktraceIdIndex uint32 `protobuf:"varint,9,opt,name=stacktrace_id_index,json=stacktraceIdIndex,proto3" json:"stacktrace_id_index,omitempty"` + LocationsLength int32 `protobuf:"varint,2,opt,name=locations_length,json=locationsLength,proto3" json:"locations_length,omitempty"` // The type and unit of each value is defined by the corresponding // entry in Profile.sample_type. All samples must have the same // number of values, the same as the length of Profile.sample_type. // When aggregating multiple samples into a single sample, the // result has a list of values that is the element-wise sum of the // lists of the originals. - Value []int64 `protobuf:"varint,2,rep,packed,name=value,proto3" json:"value,omitempty"` - // label includes additional context for this sample. It can include - // things like a thread id, allocation size, etc. - // - // NOTE: While possible, having multiple values for the same label key is - // strongly discouraged and should never be used. Most tools (e.g. pprof) do - // not have good (or any) support for multi-value labels. And an even more - // discouraged case is having a string label and a numeric label of the same - // name on a sample. Again, possible to express, but should not be used. - // [deprecated, superseded by attributes] - Label []*Label `protobuf:"bytes,3,rep,name=label,proto3" json:"label,omitempty"` + Value []int64 `protobuf:"varint,3,rep,packed,name=value,proto3" json:"value,omitempty"` // References to attributes in Profile.attribute_table. [optional] - Attributes []uint64 `protobuf:"varint,10,rep,packed,name=attributes,proto3" json:"attributes,omitempty"` + AttributeIndices []int32 `protobuf:"varint,4,rep,packed,name=attribute_indices,json=attributeIndices,proto3" json:"attribute_indices,omitempty"` // Reference to link in Profile.link_table. [optional] - Link uint64 `protobuf:"varint,12,opt,name=link,proto3" json:"link,omitempty"` + // + // Types that are valid to be assigned to LinkIndex_: + // *Sample_LinkIndex + LinkIndex_ isSample_LinkIndex_ `protobuf_oneof:"link_index_"` // Timestamps associated with Sample represented in nanoseconds. These timestamps are expected // to fall within the Profile's time range. [optional] - TimestampsUnixNano []uint64 `protobuf:"varint,13,rep,packed,name=timestamps_unix_nano,json=timestampsUnixNano,proto3" json:"timestamps_unix_nano,omitempty"` + TimestampsUnixNano []uint64 `protobuf:"varint,6,rep,packed,name=timestamps_unix_nano,json=timestampsUnixNano,proto3" json:"timestamps_unix_nano,omitempty"` } func (m *Sample) Reset() { *m = Sample{} } func (m *Sample) String() string { return proto.CompactTextString(m) } func (*Sample) ProtoMessage() {} func (*Sample) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{4} + return fileDescriptor_ddd0cf081a2fe76f, []int{7} } func (m *Sample) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -597,34 +797,39 @@ func (m *Sample) XXX_DiscardUnknown() { var xxx_messageInfo_Sample proto.InternalMessageInfo -func (m *Sample) GetLocationIndex() []uint64 { +type isSample_LinkIndex_ interface { + isSample_LinkIndex_() + MarshalTo([]byte) (int, error) + Size() int +} + +type Sample_LinkIndex struct { + LinkIndex int32 `protobuf:"varint,5,opt,name=link_index,json=linkIndex,proto3,oneof" json:"link_index,omitempty"` +} + +func (*Sample_LinkIndex) isSample_LinkIndex_() {} + +func (m *Sample) GetLinkIndex_() isSample_LinkIndex_ { if m != nil { - return m.LocationIndex + return m.LinkIndex_ } return nil } -func (m *Sample) GetLocationsStartIndex() uint64 { +func (m *Sample) GetLocationsStartIndex() int32 { if m != nil { return m.LocationsStartIndex } return 0 } -func (m *Sample) GetLocationsLength() uint64 { +func (m *Sample) GetLocationsLength() int32 { if m != nil { return m.LocationsLength } return 0 } -func (m *Sample) GetStacktraceIdIndex() uint32 { - if m != nil { - return m.StacktraceIdIndex - } - return 0 -} - func (m *Sample) GetValue() []int64 { if m != nil { return m.Value @@ -632,23 +837,16 @@ func (m *Sample) GetValue() []int64 { return nil } -func (m *Sample) GetLabel() []*Label { - if m != nil { - return m.Label - } - return nil -} - -func (m *Sample) GetAttributes() []uint64 { +func (m *Sample) GetAttributeIndices() []int32 { if m != nil { - return m.Attributes + return m.AttributeIndices } return nil } -func (m *Sample) GetLink() uint64 { - if m != nil { - return m.Link +func (m *Sample) GetLinkIndex() int32 { + if x, ok := m.GetLinkIndex_().(*Sample_LinkIndex); ok { + return x.LinkIndex } return 0 } @@ -660,13 +858,20 @@ func (m *Sample) GetTimestampsUnixNano() []uint64 { return nil } +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Sample) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Sample_LinkIndex)(nil), + } +} + // Provides additional context for a sample, // such as thread ID or allocation size, with optional units. [deprecated] type Label struct { - Key int64 `protobuf:"varint,1,opt,name=key,proto3" json:"key,omitempty"` + KeyStrindex int32 `protobuf:"varint,1,opt,name=key_strindex,json=keyStrindex,proto3" json:"key_strindex,omitempty"` // At most one of the following must be present - Str int64 `protobuf:"varint,2,opt,name=str,proto3" json:"str,omitempty"` - Num int64 `protobuf:"varint,3,opt,name=num,proto3" json:"num,omitempty"` + StrStrindex int32 `protobuf:"varint,2,opt,name=str_strindex,json=strStrindex,proto3" json:"str_strindex,omitempty"` + Num int64 `protobuf:"varint,3,opt,name=num,proto3" json:"num,omitempty"` // Should only be present when num is present. // Specifies the units of num. // Use arbitrary string (for example, "requests") as a custom count unit. @@ -674,14 +879,14 @@ type Label struct { // Consumers may also interpret units like "bytes" and "kilobytes" as memory // units and units like "seconds" and "nanoseconds" as time units, // and apply appropriate unit conversions to these. - NumUnit int64 `protobuf:"varint,4,opt,name=num_unit,json=numUnit,proto3" json:"num_unit,omitempty"` + NumUnitStrindex int32 `protobuf:"varint,4,opt,name=num_unit_strindex,json=numUnitStrindex,proto3" json:"num_unit_strindex,omitempty"` } func (m *Label) Reset() { *m = Label{} } func (m *Label) String() string { return proto.CompactTextString(m) } func (*Label) ProtoMessage() {} func (*Label) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{5} + return fileDescriptor_ddd0cf081a2fe76f, []int{8} } func (m *Label) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -710,16 +915,16 @@ func (m *Label) XXX_DiscardUnknown() { var xxx_messageInfo_Label proto.InternalMessageInfo -func (m *Label) GetKey() int64 { +func (m *Label) GetKeyStrindex() int32 { if m != nil { - return m.Key + return m.KeyStrindex } return 0 } -func (m *Label) GetStr() int64 { +func (m *Label) GetStrStrindex() int32 { if m != nil { - return m.Str + return m.StrStrindex } return 0 } @@ -731,9 +936,9 @@ func (m *Label) GetNum() int64 { return 0 } -func (m *Label) GetNumUnit() int64 { +func (m *Label) GetNumUnitStrindex() int32 { if m != nil { - return m.NumUnit + return m.NumUnitStrindex } return 0 } @@ -741,38 +946,30 @@ func (m *Label) GetNumUnit() int64 { // Describes the mapping of a binary in memory, including its address range, // file offset, and metadata like build ID type Mapping struct { - // Unique nonzero id for the mapping. [deprecated] - Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` // Address at which the binary (or DLL) is loaded into memory. - MemoryStart uint64 `protobuf:"varint,2,opt,name=memory_start,json=memoryStart,proto3" json:"memory_start,omitempty"` + MemoryStart uint64 `protobuf:"varint,1,opt,name=memory_start,json=memoryStart,proto3" json:"memory_start,omitempty"` // The limit of the address range occupied by this mapping. - MemoryLimit uint64 `protobuf:"varint,3,opt,name=memory_limit,json=memoryLimit,proto3" json:"memory_limit,omitempty"` + MemoryLimit uint64 `protobuf:"varint,2,opt,name=memory_limit,json=memoryLimit,proto3" json:"memory_limit,omitempty"` // Offset in the binary that corresponds to the first mapped address. - FileOffset uint64 `protobuf:"varint,4,opt,name=file_offset,json=fileOffset,proto3" json:"file_offset,omitempty"` + FileOffset uint64 `protobuf:"varint,3,opt,name=file_offset,json=fileOffset,proto3" json:"file_offset,omitempty"` // The object this entry is loaded from. This can be a filename on // disk for the main binary and shared libraries, or virtual // abstractions like "[vdso]". - Filename int64 `protobuf:"varint,5,opt,name=filename,proto3" json:"filename,omitempty"` - // A string that uniquely identifies a particular program version - // with high probability. E.g., for binaries generated by GNU tools, - // it could be the contents of the .note.gnu.build-id field. - BuildId int64 `protobuf:"varint,6,opt,name=build_id,json=buildId,proto3" json:"build_id,omitempty"` - // Specifies the kind of build id. See BuildIdKind enum for more details [optional] - BuildIdKind BuildIdKind `protobuf:"varint,11,opt,name=build_id_kind,json=buildIdKind,proto3,enum=opentelemetry.proto.profiles.v1experimental.BuildIdKind" json:"build_id_kind,omitempty"` + FilenameStrindex int32 `protobuf:"varint,4,opt,name=filename_strindex,json=filenameStrindex,proto3" json:"filename_strindex,omitempty"` // References to attributes in Profile.attribute_table. [optional] - Attributes []uint64 `protobuf:"varint,12,rep,packed,name=attributes,proto3" json:"attributes,omitempty"` + AttributeIndices []int32 `protobuf:"varint,5,rep,packed,name=attribute_indices,json=attributeIndices,proto3" json:"attribute_indices,omitempty"` // The following fields indicate the resolution of symbolic info. - HasFunctions bool `protobuf:"varint,7,opt,name=has_functions,json=hasFunctions,proto3" json:"has_functions,omitempty"` - HasFilenames bool `protobuf:"varint,8,opt,name=has_filenames,json=hasFilenames,proto3" json:"has_filenames,omitempty"` - HasLineNumbers bool `protobuf:"varint,9,opt,name=has_line_numbers,json=hasLineNumbers,proto3" json:"has_line_numbers,omitempty"` - HasInlineFrames bool `protobuf:"varint,10,opt,name=has_inline_frames,json=hasInlineFrames,proto3" json:"has_inline_frames,omitempty"` + HasFunctions bool `protobuf:"varint,6,opt,name=has_functions,json=hasFunctions,proto3" json:"has_functions,omitempty"` + HasFilenames bool `protobuf:"varint,7,opt,name=has_filenames,json=hasFilenames,proto3" json:"has_filenames,omitempty"` + HasLineNumbers bool `protobuf:"varint,8,opt,name=has_line_numbers,json=hasLineNumbers,proto3" json:"has_line_numbers,omitempty"` + HasInlineFrames bool `protobuf:"varint,9,opt,name=has_inline_frames,json=hasInlineFrames,proto3" json:"has_inline_frames,omitempty"` } func (m *Mapping) Reset() { *m = Mapping{} } func (m *Mapping) String() string { return proto.CompactTextString(m) } func (*Mapping) ProtoMessage() {} func (*Mapping) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{6} + return fileDescriptor_ddd0cf081a2fe76f, []int{9} } func (m *Mapping) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -801,13 +998,6 @@ func (m *Mapping) XXX_DiscardUnknown() { var xxx_messageInfo_Mapping proto.InternalMessageInfo -func (m *Mapping) GetId() uint64 { - if m != nil { - return m.Id - } - return 0 -} - func (m *Mapping) GetMemoryStart() uint64 { if m != nil { return m.MemoryStart @@ -829,30 +1019,16 @@ func (m *Mapping) GetFileOffset() uint64 { return 0 } -func (m *Mapping) GetFilename() int64 { - if m != nil { - return m.Filename - } - return 0 -} - -func (m *Mapping) GetBuildId() int64 { +func (m *Mapping) GetFilenameStrindex() int32 { if m != nil { - return m.BuildId + return m.FilenameStrindex } return 0 } -func (m *Mapping) GetBuildIdKind() BuildIdKind { +func (m *Mapping) GetAttributeIndices() []int32 { if m != nil { - return m.BuildIdKind - } - return BuildIdKind_BUILD_ID_LINKER -} - -func (m *Mapping) GetAttributes() []uint64 { - if m != nil { - return m.Attributes + return m.AttributeIndices } return nil } @@ -887,19 +1063,19 @@ func (m *Mapping) GetHasInlineFrames() bool { // Describes function and line table debug information. type Location struct { - // Unique nonzero id for the location. A profile could use - // instruction addresses or any integer sequence as ids. [deprecated] - Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` - // The index of the corresponding profile.Mapping for this location. + // Reference to mapping in Profile.mapping_table. // It can be unset if the mapping is unknown or not applicable for // this profile type. - MappingIndex uint64 `protobuf:"varint,2,opt,name=mapping_index,json=mappingIndex,proto3" json:"mapping_index,omitempty"` + // + // Types that are valid to be assigned to MappingIndex_: + // *Location_MappingIndex + MappingIndex_ isLocation_MappingIndex_ `protobuf_oneof:"mapping_index_"` // The instruction address for this location, if available. It // should be within [Mapping.memory_start...Mapping.memory_limit] // for the corresponding mapping. A non-leaf address may be in the // middle of a call instruction. It is up to display tools to find // the beginning of the instruction if necessary. - Address uint64 `protobuf:"varint,3,opt,name=address,proto3" json:"address,omitempty"` + Address uint64 `protobuf:"varint,2,opt,name=address,proto3" json:"address,omitempty"` // Multiple line indicates this location has inlined functions, // where the last entry represents the caller into which the // preceding entries were inlined. @@ -907,24 +1083,22 @@ type Location struct { // E.g., if memcpy() is inlined into printf: // line[0].function_name == "memcpy" // line[1].function_name == "printf" - Line []*Line `protobuf:"bytes,4,rep,name=line,proto3" json:"line,omitempty"` + Line []*Line `protobuf:"bytes,3,rep,name=line,proto3" json:"line,omitempty"` // Provides an indication that multiple symbols map to this location's // address, for example due to identical code folding by the linker. In that // case the line information above represents one of the multiple // symbols. This field must be recomputed when the symbolization state of the // profile changes. - IsFolded bool `protobuf:"varint,5,opt,name=is_folded,json=isFolded,proto3" json:"is_folded,omitempty"` - // Type of frame (e.g. kernel, native, python, hotspot, php). Index into string table. - TypeIndex uint32 `protobuf:"varint,6,opt,name=type_index,json=typeIndex,proto3" json:"type_index,omitempty"` + IsFolded bool `protobuf:"varint,4,opt,name=is_folded,json=isFolded,proto3" json:"is_folded,omitempty"` // References to attributes in Profile.attribute_table. [optional] - Attributes []uint64 `protobuf:"varint,7,rep,packed,name=attributes,proto3" json:"attributes,omitempty"` + AttributeIndices []int32 `protobuf:"varint,5,rep,packed,name=attribute_indices,json=attributeIndices,proto3" json:"attribute_indices,omitempty"` } func (m *Location) Reset() { *m = Location{} } func (m *Location) String() string { return proto.CompactTextString(m) } func (*Location) ProtoMessage() {} func (*Location) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{7} + return fileDescriptor_ddd0cf081a2fe76f, []int{10} } func (m *Location) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -953,16 +1127,28 @@ func (m *Location) XXX_DiscardUnknown() { var xxx_messageInfo_Location proto.InternalMessageInfo -func (m *Location) GetId() uint64 { +type isLocation_MappingIndex_ interface { + isLocation_MappingIndex_() + MarshalTo([]byte) (int, error) + Size() int +} + +type Location_MappingIndex struct { + MappingIndex int32 `protobuf:"varint,1,opt,name=mapping_index,json=mappingIndex,proto3,oneof" json:"mapping_index,omitempty"` +} + +func (*Location_MappingIndex) isLocation_MappingIndex_() {} + +func (m *Location) GetMappingIndex_() isLocation_MappingIndex_ { if m != nil { - return m.Id + return m.MappingIndex_ } - return 0 + return nil } -func (m *Location) GetMappingIndex() uint64 { - if m != nil { - return m.MappingIndex +func (m *Location) GetMappingIndex() int32 { + if x, ok := m.GetMappingIndex_().(*Location_MappingIndex); ok { + return x.MappingIndex } return 0 } @@ -988,24 +1174,24 @@ func (m *Location) GetIsFolded() bool { return false } -func (m *Location) GetTypeIndex() uint32 { +func (m *Location) GetAttributeIndices() []int32 { if m != nil { - return m.TypeIndex + return m.AttributeIndices } - return 0 + return nil } -func (m *Location) GetAttributes() []uint64 { - if m != nil { - return m.Attributes +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Location) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Location_MappingIndex)(nil), } - return nil } // Details a specific line in a source code, linked to a function. type Line struct { - // The index of the corresponding profile.Function for this line. - FunctionIndex uint64 `protobuf:"varint,1,opt,name=function_index,json=functionIndex,proto3" json:"function_index,omitempty"` + // Reference to function in Profile.function_table. + FunctionIndex int32 `protobuf:"varint,1,opt,name=function_index,json=functionIndex,proto3" json:"function_index,omitempty"` // Line number in source code. Line int64 `protobuf:"varint,2,opt,name=line,proto3" json:"line,omitempty"` // Column number in source code. @@ -1016,7 +1202,7 @@ func (m *Line) Reset() { *m = Line{} } func (m *Line) String() string { return proto.CompactTextString(m) } func (*Line) ProtoMessage() {} func (*Line) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{8} + return fileDescriptor_ddd0cf081a2fe76f, []int{11} } func (m *Line) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1045,7 +1231,7 @@ func (m *Line) XXX_DiscardUnknown() { var xxx_messageInfo_Line proto.InternalMessageInfo -func (m *Line) GetFunctionIndex() uint64 { +func (m *Line) GetFunctionIndex() int32 { if m != nil { return m.FunctionIndex } @@ -1069,24 +1255,22 @@ func (m *Line) GetColumn() int64 { // Describes a function, including its human-readable name, system name, // source file, and starting line number in the source. type Function struct { - // Unique nonzero id for the function. [deprecated] - Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` // Name of the function, in human-readable form if available. - Name int64 `protobuf:"varint,2,opt,name=name,proto3" json:"name,omitempty"` + NameStrindex int32 `protobuf:"varint,1,opt,name=name_strindex,json=nameStrindex,proto3" json:"name_strindex,omitempty"` // Name of the function, as identified by the system. // For instance, it can be a C++ mangled name. - SystemName int64 `protobuf:"varint,3,opt,name=system_name,json=systemName,proto3" json:"system_name,omitempty"` + SystemNameStrindex int32 `protobuf:"varint,2,opt,name=system_name_strindex,json=systemNameStrindex,proto3" json:"system_name_strindex,omitempty"` // Source file containing the function. - Filename int64 `protobuf:"varint,4,opt,name=filename,proto3" json:"filename,omitempty"` + FilenameStrindex int32 `protobuf:"varint,3,opt,name=filename_strindex,json=filenameStrindex,proto3" json:"filename_strindex,omitempty"` // Line number in source file. - StartLine int64 `protobuf:"varint,5,opt,name=start_line,json=startLine,proto3" json:"start_line,omitempty"` + StartLine int64 `protobuf:"varint,4,opt,name=start_line,json=startLine,proto3" json:"start_line,omitempty"` } func (m *Function) Reset() { *m = Function{} } func (m *Function) String() string { return proto.CompactTextString(m) } func (*Function) ProtoMessage() {} func (*Function) Descriptor() ([]byte, []int) { - return fileDescriptor_05f9ce3fdbeb046f, []int{9} + return fileDescriptor_ddd0cf081a2fe76f, []int{12} } func (m *Function) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1115,30 +1299,23 @@ func (m *Function) XXX_DiscardUnknown() { var xxx_messageInfo_Function proto.InternalMessageInfo -func (m *Function) GetId() uint64 { - if m != nil { - return m.Id - } - return 0 -} - -func (m *Function) GetName() int64 { +func (m *Function) GetNameStrindex() int32 { if m != nil { - return m.Name + return m.NameStrindex } return 0 } -func (m *Function) GetSystemName() int64 { +func (m *Function) GetSystemNameStrindex() int32 { if m != nil { - return m.SystemName + return m.SystemNameStrindex } return 0 } -func (m *Function) GetFilename() int64 { +func (m *Function) GetFilenameStrindex() int32 { if m != nil { - return m.Filename + return m.FilenameStrindex } return 0 } @@ -1151,122 +1328,134 @@ func (m *Function) GetStartLine() int64 { } func init() { - proto.RegisterEnum("opentelemetry.proto.profiles.v1experimental.AggregationTemporality", AggregationTemporality_name, AggregationTemporality_value) - proto.RegisterEnum("opentelemetry.proto.profiles.v1experimental.BuildIdKind", BuildIdKind_name, BuildIdKind_value) - proto.RegisterType((*Profile)(nil), "opentelemetry.proto.profiles.v1experimental.Profile") - proto.RegisterType((*AttributeUnit)(nil), "opentelemetry.proto.profiles.v1experimental.AttributeUnit") - proto.RegisterType((*Link)(nil), "opentelemetry.proto.profiles.v1experimental.Link") - proto.RegisterType((*ValueType)(nil), "opentelemetry.proto.profiles.v1experimental.ValueType") - proto.RegisterType((*Sample)(nil), "opentelemetry.proto.profiles.v1experimental.Sample") - proto.RegisterType((*Label)(nil), "opentelemetry.proto.profiles.v1experimental.Label") - proto.RegisterType((*Mapping)(nil), "opentelemetry.proto.profiles.v1experimental.Mapping") - proto.RegisterType((*Location)(nil), "opentelemetry.proto.profiles.v1experimental.Location") - proto.RegisterType((*Line)(nil), "opentelemetry.proto.profiles.v1experimental.Line") - proto.RegisterType((*Function)(nil), "opentelemetry.proto.profiles.v1experimental.Function") + proto.RegisterEnum("opentelemetry.proto.profiles.v1development.AggregationTemporality", AggregationTemporality_name, AggregationTemporality_value) + proto.RegisterType((*ProfilesData)(nil), "opentelemetry.proto.profiles.v1development.ProfilesData") + proto.RegisterType((*ResourceProfiles)(nil), "opentelemetry.proto.profiles.v1development.ResourceProfiles") + proto.RegisterType((*ScopeProfiles)(nil), "opentelemetry.proto.profiles.v1development.ScopeProfiles") + proto.RegisterType((*Profile)(nil), "opentelemetry.proto.profiles.v1development.Profile") + proto.RegisterType((*AttributeUnit)(nil), "opentelemetry.proto.profiles.v1development.AttributeUnit") + proto.RegisterType((*Link)(nil), "opentelemetry.proto.profiles.v1development.Link") + proto.RegisterType((*ValueType)(nil), "opentelemetry.proto.profiles.v1development.ValueType") + proto.RegisterType((*Sample)(nil), "opentelemetry.proto.profiles.v1development.Sample") + proto.RegisterType((*Label)(nil), "opentelemetry.proto.profiles.v1development.Label") + proto.RegisterType((*Mapping)(nil), "opentelemetry.proto.profiles.v1development.Mapping") + proto.RegisterType((*Location)(nil), "opentelemetry.proto.profiles.v1development.Location") + proto.RegisterType((*Line)(nil), "opentelemetry.proto.profiles.v1development.Line") + proto.RegisterType((*Function)(nil), "opentelemetry.proto.profiles.v1development.Function") } func init() { - proto.RegisterFile("opentelemetry/proto/profiles/v1experimental/pprofextended.proto", fileDescriptor_05f9ce3fdbeb046f) -} - -var fileDescriptor_05f9ce3fdbeb046f = []byte{ - // 1480 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x57, 0x4f, 0x4f, 0x23, 0x47, - 0x16, 0xa7, 0xb1, 0xf1, 0x9f, 0x67, 0x0c, 0xa6, 0x86, 0x65, 0x7b, 0x67, 0x35, 0xc0, 0x18, 0xed, - 0x0e, 0xcb, 0x48, 0x66, 0x61, 0x76, 0xa3, 0x49, 0x14, 0x29, 0x31, 0x83, 0x19, 0x5a, 0x78, 0x0c, - 0x29, 0x0c, 0x09, 0xd1, 0x44, 0xad, 0xc6, 0x5d, 0x98, 0x16, 0xdd, 0xd5, 0xad, 0xee, 0x32, 0xc2, - 0x52, 0x8e, 0x39, 0x45, 0x39, 0xe4, 0x9c, 0x8f, 0x90, 0x5b, 0x3e, 0x41, 0xae, 0x23, 0xe5, 0x32, - 0x97, 0x48, 0x51, 0x0e, 0xa3, 0x68, 0xe6, 0x6b, 0xe4, 0x10, 0xd5, 0xab, 0x6a, 0xdb, 0x30, 0xcc, - 0xc1, 0x73, 0x41, 0xf5, 0x7e, 0xf5, 0xea, 0x57, 0xaf, 0xfa, 0xbd, 0xdf, 0x7b, 0x06, 0x3e, 0x09, - 0x23, 0xc6, 0x05, 0xf3, 0x59, 0xc0, 0x44, 0xdc, 0x5f, 0x8f, 0xe2, 0x50, 0x84, 0xf2, 0xef, 0x99, - 0xe7, 0xb3, 0x64, 0xfd, 0x72, 0x83, 0x5d, 0x45, 0x2c, 0xf6, 0x02, 0xc6, 0x85, 0xe3, 0xaf, 0x47, - 0x72, 0x83, 0x5d, 0x09, 0xc6, 0x5d, 0xe6, 0xd6, 0xd0, 0x97, 0x3c, 0xbc, 0x46, 0xa0, 0xc0, 0x5a, - 0x4a, 0x50, 0xbb, 0x4e, 0x70, 0x77, 0xbe, 0x1b, 0x76, 0x43, 0x75, 0x87, 0x5c, 0x29, 0xef, 0xbb, - 0x6b, 0xb7, 0xc5, 0xd0, 0x09, 0x83, 0x20, 0xe4, 0xeb, 0x97, 0x1b, 0x7a, 0xa5, 0x7c, 0xab, 0xbf, - 0x14, 0x20, 0x7f, 0xa0, 0xd8, 0xc9, 0xe7, 0x50, 0x4a, 0x9c, 0x20, 0xf2, 0x99, 0x2d, 0xfa, 0x11, - 0x33, 0x8d, 0xe5, 0xcc, 0x6a, 0x69, 0xf3, 0x83, 0xda, 0x18, 0x01, 0xd5, 0x8e, 0x1d, 0xbf, 0xc7, - 0xda, 0xfd, 0x88, 0x51, 0x50, 0x54, 0x72, 0x4d, 0xf6, 0x20, 0xa7, 0x2c, 0x73, 0x12, 0x39, 0x1f, - 0x8d, 0xc5, 0x79, 0x88, 0x47, 0xa9, 0xa6, 0x20, 0x2d, 0xc8, 0x07, 0x4e, 0x14, 0x79, 0xbc, 0x6b, - 0x66, 0x90, 0xed, 0x7f, 0x63, 0xb1, 0x3d, 0x53, 0x67, 0x69, 0x4a, 0x42, 0x3e, 0x83, 0x82, 0x1f, - 0x76, 0x1c, 0xe1, 0x85, 0xdc, 0xcc, 0x22, 0xe1, 0xff, 0xc7, 0x22, 0x6c, 0xea, 0xc3, 0x74, 0x40, - 0x43, 0xfe, 0x03, 0x95, 0x74, 0x6d, 0x7b, 0xdc, 0xf5, 0x3a, 0x2c, 0x31, 0x67, 0x97, 0x33, 0xab, - 0x19, 0x3a, 0x9b, 0xe2, 0x96, 0x82, 0xe5, 0xed, 0x67, 0x3d, 0xde, 0xc1, 0xdb, 0xa7, 0xde, 0xe3, - 0xf6, 0x1d, 0x7d, 0x98, 0x0e, 0x68, 0xc8, 0x31, 0xcc, 0x3a, 0x42, 0xc4, 0xde, 0x69, 0x4f, 0x30, - 0x5b, 0x38, 0xa7, 0x3e, 0x33, 0x2b, 0xc8, 0xfc, 0xe0, 0x56, 0x66, 0x5d, 0x0e, 0x97, 0x1b, 0xb5, - 0x3d, 0xd6, 0xc7, 0xfc, 0x6d, 0x65, 0x5f, 0xbc, 0x5a, 0x9a, 0xa0, 0x33, 0x03, 0x96, 0xb6, 0x24, - 0x21, 0x9d, 0x51, 0xde, 0x1e, 0xf7, 0x44, 0x62, 0xce, 0x21, 0xef, 0x47, 0x63, 0x45, 0x5c, 0x4f, - 0x39, 0x8e, 0xb8, 0x27, 0x46, 0x2e, 0x91, 0x66, 0x42, 0x0e, 0x00, 0x7c, 0x8f, 0x5f, 0xe8, 0xb8, - 0x09, 0xf2, 0x6f, 0x8c, 0x97, 0x0f, 0x8f, 0x5f, 0xd0, 0xa2, 0x24, 0x51, 0x61, 0xdf, 0x87, 0xe9, - 0x44, 0xc4, 0x1e, 0xef, 0x6a, 0xce, 0xdc, 0x72, 0x66, 0xb5, 0x48, 0x4b, 0x0a, 0x53, 0x2e, 0x4b, - 0x50, 0x72, 0xe3, 0x30, 0xb2, 0xcf, 0x62, 0x27, 0x60, 0x89, 0x99, 0x5f, 0x36, 0x56, 0x33, 0x14, - 0x24, 0xb4, 0x83, 0x88, 0x74, 0xb8, 0x60, 0x6c, 0xe0, 0x50, 0x50, 0x0e, 0x12, 0xd2, 0x0e, 0xf7, - 0x00, 0x84, 0x17, 0x30, 0x9b, 0x3b, 0x3c, 0x4c, 0xcc, 0x22, 0xee, 0x17, 0x25, 0xd2, 0x92, 0x00, - 0xf9, 0x17, 0xcc, 0xb8, 0xbd, 0x58, 0x15, 0x84, 0x72, 0x01, 0x74, 0x29, 0xa7, 0xa8, 0x72, 0xfb, - 0x0a, 0x4a, 0xf2, 0x21, 0xa1, 0xab, 0x04, 0x58, 0x5a, 0x36, 0xde, 0x5f, 0x80, 0x3a, 0x89, 0xa0, - 0x08, 0x51, 0x86, 0x0b, 0x90, 0x53, 0x96, 0x39, 0x8d, 0xb7, 0x6b, 0x8b, 0x98, 0x90, 0x97, 0x45, - 0xc0, 0xb8, 0x30, 0xcb, 0x58, 0xa5, 0xa9, 0x49, 0x6a, 0x70, 0xc7, 0x65, 0x67, 0x4e, 0xcf, 0x17, - 0xf6, 0x68, 0x67, 0x98, 0xc1, 0xe3, 0x73, 0x7a, 0xeb, 0x70, 0x20, 0xf4, 0xea, 0x2e, 0x94, 0xaf, - 0xa5, 0x97, 0xac, 0x40, 0x79, 0x58, 0x33, 0x17, 0xac, 0x6f, 0x1a, 0x78, 0x74, 0x7a, 0x00, 0xee, - 0xb1, 0x3e, 0x21, 0x90, 0x95, 0xe5, 0x64, 0x4e, 0xe2, 0x1e, 0xae, 0xab, 0x3f, 0x1b, 0x90, 0x95, - 0x99, 0x24, 0xcf, 0xa1, 0x20, 0x62, 0xa7, 0xc3, 0x6c, 0xcf, 0xc5, 0xc3, 0xd3, 0x5b, 0x75, 0xf9, - 0xb0, 0xdf, 0x5f, 0x2d, 0x7d, 0xd8, 0x0d, 0x6f, 0x7c, 0x1a, 0x4f, 0xb6, 0x39, 0xdf, 0x67, 0x1d, - 0x11, 0xc6, 0xeb, 0x91, 0xeb, 0x08, 0x67, 0xdd, 0xe3, 0x82, 0xc5, 0xdc, 0xf1, 0xd7, 0xa5, 0x55, - 0x6b, 0x4b, 0x26, 0x6b, 0x9b, 0xe6, 0x91, 0xd2, 0x72, 0xc9, 0x09, 0xe4, 0x93, 0xc8, 0xe1, 0x92, - 0x7c, 0x12, 0xc9, 0x3f, 0xd5, 0xe4, 0x8f, 0xc7, 0x27, 0x3f, 0x8c, 0x1c, 0x6e, 0x6d, 0xd3, 0x9c, - 0x24, 0xb4, 0xdc, 0xea, 0x4f, 0x06, 0x14, 0x07, 0xd9, 0x90, 0x6f, 0xd4, 0x4d, 0x15, 0xdf, 0x28, - 0x34, 0x76, 0xf3, 0xdd, 0xe4, 0x6b, 0xf8, 0xbb, 0xd3, 0xed, 0xc6, 0xac, 0xab, 0x8a, 0x45, 0xb0, - 0x20, 0x0a, 0x63, 0xc7, 0xf7, 0x44, 0xdf, 0xcc, 0x2c, 0x1b, 0xab, 0x33, 0x9b, 0x4f, 0xc6, 0x13, - 0xdb, 0x90, 0xab, 0x3d, 0xa4, 0xa2, 0x0b, 0xce, 0xad, 0x78, 0xf5, 0x9b, 0x0c, 0xe4, 0x54, 0x3a, - 0x65, 0xc9, 0x8e, 0xf6, 0x30, 0x76, 0x85, 0xf3, 0x20, 0x4b, 0xcb, 0x23, 0x1d, 0x8c, 0x5d, 0x91, - 0x4d, 0xf8, 0x5b, 0x0a, 0x24, 0x76, 0x22, 0x9c, 0x58, 0x68, 0x6f, 0x29, 0xa2, 0x2c, 0xbd, 0x33, - 0xd8, 0x3c, 0x94, 0x7b, 0xea, 0xcc, 0x48, 0x7b, 0x4c, 0x6c, 0x9f, 0xf1, 0xae, 0x38, 0x47, 0x49, - 0x65, 0x87, 0xed, 0x31, 0x69, 0x22, 0x2c, 0x0b, 0x30, 0x11, 0x4e, 0xe7, 0x22, 0x2d, 0x01, 0x4d, - 0x2e, 0x05, 0x56, 0xa6, 0x73, 0xc3, 0x2d, 0xcb, 0x55, 0xd4, 0xf3, 0x30, 0x75, 0x29, 0xbf, 0x39, - 0x0e, 0x9a, 0x0c, 0x55, 0x06, 0xd9, 0x85, 0x29, 0xdf, 0x39, 0x65, 0xbe, 0x1e, 0x18, 0x9b, 0xe3, - 0xf5, 0x13, 0x79, 0x92, 0x2a, 0x02, 0xb2, 0x08, 0x30, 0x28, 0x5d, 0x29, 0x62, 0xf9, 0x45, 0x46, - 0x10, 0x99, 0x52, 0xd9, 0x79, 0x50, 0x60, 0x59, 0x8a, 0x6b, 0xf2, 0x5f, 0x98, 0x97, 0x9d, 0x20, - 0x11, 0x4e, 0x10, 0x25, 0xb2, 0x71, 0x5e, 0x61, 0x0f, 0x40, 0xad, 0x65, 0x29, 0x19, 0xee, 0x1d, - 0x71, 0xef, 0x4a, 0x36, 0x82, 0xea, 0x17, 0x30, 0x85, 0xb7, 0x92, 0x0a, 0x64, 0x86, 0xa2, 0x91, - 0x4b, 0x89, 0x24, 0x22, 0xd6, 0x25, 0x23, 0x97, 0x12, 0xe1, 0xbd, 0x00, 0xab, 0x23, 0x43, 0xe5, - 0x92, 0xfc, 0x03, 0x0a, 0xbc, 0x17, 0x60, 0x8b, 0x36, 0xb3, 0x08, 0xe7, 0x79, 0x2f, 0x90, 0x7a, - 0xac, 0xfe, 0x9a, 0x81, 0xbc, 0x9e, 0x80, 0x64, 0x06, 0x26, 0xb5, 0xa6, 0xb2, 0x74, 0xd2, 0x73, - 0x65, 0xa3, 0x0c, 0x58, 0x10, 0xc6, 0x7d, 0x95, 0x47, 0xbc, 0x23, 0x4b, 0x4b, 0x0a, 0xc3, 0xf4, - 0x8d, 0xb8, 0xf8, 0x5e, 0xe0, 0x09, 0xbc, 0x74, 0xe0, 0xd2, 0x94, 0x90, 0x6c, 0x95, 0xf2, 0x33, - 0xda, 0xe1, 0xd9, 0x59, 0xc2, 0xd4, 0xfd, 0x59, 0x0a, 0x12, 0xda, 0x47, 0x84, 0xdc, 0x85, 0x82, - 0xb4, 0xb8, 0x13, 0x30, 0x73, 0x0a, 0xa3, 0x1b, 0xd8, 0x32, 0xf2, 0xd3, 0x9e, 0xe7, 0xbb, 0x52, - 0x8f, 0x39, 0x15, 0x39, 0xda, 0x96, 0x4b, 0x9e, 0x43, 0x39, 0xdd, 0xb2, 0x2f, 0x3c, 0xee, 0x62, - 0x77, 0x9c, 0xd9, 0x7c, 0x3c, 0x56, 0x2e, 0xb7, 0x14, 0xd9, 0x9e, 0xc7, 0x5d, 0x5a, 0x3a, 0x1d, - 0x1a, 0x37, 0xf2, 0x3a, 0xfd, 0x56, 0x5e, 0x57, 0xa0, 0x7c, 0xee, 0x24, 0x76, 0x3a, 0x63, 0xd5, - 0x8c, 0x28, 0xd0, 0xe9, 0x73, 0x27, 0x49, 0x27, 0xf0, 0xd0, 0x49, 0xbf, 0x46, 0xcd, 0x09, 0xed, - 0x94, 0x62, 0x64, 0x15, 0x2a, 0xd2, 0xc9, 0xf7, 0x38, 0xb3, 0x79, 0x2f, 0x38, 0x65, 0xb1, 0x9a, - 0x17, 0x05, 0x3a, 0x73, 0xee, 0x24, 0x4d, 0x8f, 0xb3, 0x96, 0x42, 0xc9, 0x1a, 0xcc, 0x49, 0x4f, - 0x8f, 0xa3, 0xaf, 0x1e, 0x3d, 0x80, 0xae, 0xb3, 0xe7, 0x4e, 0x62, 0x21, 0xae, 0xe6, 0x4f, 0xf5, - 0x4f, 0x03, 0x0a, 0xe9, 0x0f, 0x91, 0xb7, 0x12, 0xbb, 0x02, 0x65, 0xfd, 0x63, 0x47, 0xcb, 0x47, - 0x65, 0x76, 0x5a, 0x83, 0x4a, 0x39, 0x26, 0xe4, 0x1d, 0xd7, 0x8d, 0x59, 0x92, 0xe8, 0xac, 0xa6, - 0x26, 0x69, 0x60, 0x4d, 0x33, 0xfd, 0xe3, 0x68, 0xec, 0x61, 0xcc, 0x50, 0x06, 0x8c, 0xfc, 0x13, - 0x8a, 0x5e, 0x62, 0x9f, 0x85, 0xbe, 0xcb, 0x5c, 0x4c, 0x7c, 0x81, 0x16, 0xbc, 0x64, 0x07, 0x6d, - 0x9c, 0x9f, 0xfd, 0x88, 0xe9, 0xf8, 0x72, 0x28, 0xef, 0xa2, 0x44, 0x54, 0x70, 0xd7, 0xd3, 0x93, - 0xbf, 0x99, 0x9e, 0xea, 0x09, 0x0e, 0x0b, 0x6c, 0x5a, 0x69, 0x8a, 0x06, 0x4d, 0x4b, 0xbe, 0xa5, - 0x9c, 0xa2, 0x8a, 0x8e, 0xe8, 0x17, 0xe9, 0xc6, 0x8b, 0xe1, 0x2d, 0x40, 0xae, 0x13, 0xfa, 0xbd, - 0x80, 0x6b, 0x25, 0x69, 0xab, 0xfa, 0xad, 0x01, 0x85, 0x34, 0xc5, 0x6f, 0x7d, 0x59, 0x02, 0x59, - 0xac, 0x63, 0x4d, 0x84, 0x35, 0xbc, 0x04, 0xa5, 0xa4, 0x9f, 0x08, 0x16, 0xd8, 0xb8, 0xa5, 0xd8, - 0x40, 0x41, 0x2d, 0xe9, 0x30, 0x2a, 0x80, 0xec, 0x0d, 0x01, 0xdc, 0x03, 0x50, 0x4d, 0x14, 0xe3, - 0x53, 0xf2, 0x28, 0x22, 0x22, 0xdf, 0xb7, 0xf6, 0x9d, 0x01, 0x0b, 0xb7, 0xb7, 0x74, 0xf2, 0x00, - 0x56, 0xea, 0x4f, 0x9f, 0xd2, 0xc6, 0xd3, 0x7a, 0xdb, 0xda, 0x6f, 0xd9, 0xed, 0xc6, 0xb3, 0x83, - 0x7d, 0x5a, 0x6f, 0x5a, 0xed, 0x13, 0xfb, 0xa8, 0x75, 0x78, 0xd0, 0x78, 0x62, 0xed, 0x58, 0x8d, - 0xed, 0xca, 0x04, 0xb9, 0x0f, 0xf7, 0xde, 0xe5, 0xb8, 0xdd, 0x68, 0xb6, 0xeb, 0x15, 0x83, 0xfc, - 0x1b, 0xaa, 0xef, 0x72, 0x79, 0x72, 0xf4, 0xec, 0xa8, 0x59, 0x6f, 0x5b, 0xc7, 0x8d, 0xca, 0xe4, - 0xda, 0xc7, 0x50, 0x1a, 0x51, 0x14, 0xb9, 0x03, 0xb3, 0x5b, 0x47, 0x56, 0x73, 0xdb, 0xb6, 0xb6, - 0xed, 0xa6, 0xd5, 0xda, 0x6b, 0xd0, 0xca, 0x04, 0x31, 0x61, 0x7e, 0x00, 0x6e, 0x59, 0xad, 0x3a, - 0x3d, 0xb1, 0x77, 0xeb, 0x87, 0xbb, 0x15, 0x63, 0xeb, 0x07, 0xe3, 0xc5, 0xeb, 0x45, 0xe3, 0xe5, - 0xeb, 0x45, 0xe3, 0x8f, 0xd7, 0x8b, 0xc6, 0xf7, 0x6f, 0x16, 0x27, 0x5e, 0xbe, 0x59, 0x9c, 0xf8, - 0xed, 0xcd, 0xe2, 0xc4, 0x97, 0x74, 0xec, 0xe9, 0xab, 0xfe, 0xcb, 0xe9, 0x32, 0xfe, 0xae, 0x7f, - 0xb6, 0x7e, 0x9c, 0x7c, 0xb8, 0x1f, 0x31, 0xde, 0x1e, 0x30, 0x1e, 0x60, 0xe1, 0x1e, 0xa4, 0x85, - 0x7b, 0xbc, 0xd1, 0x18, 0xf1, 0x3e, 0xcd, 0x21, 0xdf, 0xa3, 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, - 0x92, 0x79, 0x01, 0x14, 0xd0, 0x0d, 0x00, 0x00, -} - -func (m *Profile) Marshal() (dAtA []byte, err error) { + proto.RegisterFile("opentelemetry/proto/profiles/v1development/profiles.proto", fileDescriptor_ddd0cf081a2fe76f) +} + +var fileDescriptor_ddd0cf081a2fe76f = []byte{ + // 1647 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0x5b, 0x6f, 0x1b, 0xc7, + 0x15, 0xd6, 0xf2, 0xce, 0xc3, 0x8b, 0xa8, 0x89, 0xac, 0x6c, 0xd3, 0x5a, 0x66, 0x68, 0xa4, 0x61, + 0x54, 0x44, 0xb2, 0xe4, 0xb4, 0x88, 0xd1, 0x02, 0xad, 0x64, 0x4a, 0x0e, 0x63, 0xea, 0xd2, 0x15, + 0x25, 0xd4, 0x6d, 0x80, 0xed, 0x88, 0x3b, 0xa4, 0xb6, 0xda, 0x9d, 0x5d, 0xec, 0x0c, 0x05, 0x13, + 0xfd, 0x09, 0xcd, 0x43, 0x1f, 0xfb, 0x1b, 0x0a, 0xf4, 0x37, 0xf4, 0x35, 0x8f, 0x46, 0x9f, 0x82, + 0x3e, 0x04, 0x85, 0xfd, 0x92, 0x16, 0xe8, 0x7f, 0x28, 0xe6, 0xb2, 0xcb, 0x4b, 0x29, 0x24, 0xeb, + 0x17, 0x61, 0xe7, 0x9c, 0x6f, 0xbe, 0x39, 0x67, 0xce, 0x65, 0x8e, 0x08, 0x4f, 0x82, 0x90, 0x50, + 0x4e, 0x3c, 0xe2, 0x13, 0x1e, 0x4d, 0x76, 0xc2, 0x28, 0xe0, 0x81, 0xf8, 0x3b, 0x74, 0x3d, 0xc2, + 0x76, 0x6e, 0x77, 0x1d, 0x72, 0x4b, 0xbc, 0x20, 0xf4, 0x09, 0xe5, 0x89, 0x78, 0x5b, 0xa2, 0xd0, + 0xd6, 0xdc, 0x56, 0x25, 0xdc, 0x4e, 0x30, 0x73, 0x5b, 0xdf, 0x5b, 0x1f, 0x05, 0xa3, 0x40, 0x91, + 0x8b, 0x2f, 0x05, 0x7e, 0x6f, 0x6b, 0xd9, 0xe1, 0x83, 0xc0, 0xf7, 0x03, 0xba, 0x73, 0xbb, 0xab, + 0xbf, 0x34, 0x76, 0x7b, 0x19, 0x36, 0x22, 0x2c, 0x18, 0x47, 0x03, 0x22, 0xd0, 0xf1, 0xb7, 0xc2, + 0xb7, 0x26, 0x50, 0x3d, 0xd3, 0xb6, 0x74, 0x30, 0xc7, 0xc8, 0x85, 0xb5, 0x18, 0x61, 0xc7, 0x46, + 0x9a, 0x46, 0x33, 0xdb, 0xae, 0xec, 0xfd, 0x62, 0xfb, 0xfb, 0x7b, 0xb2, 0x6d, 0x69, 0x92, 0x98, + 0xdc, 0x6a, 0x44, 0x0b, 0x92, 0xd6, 0xb7, 0x06, 0x34, 0x16, 0x61, 0xe8, 0x39, 0x94, 0x62, 0xa0, + 0x69, 0x34, 0x8d, 0x76, 0x65, 0xef, 0xa3, 0xa5, 0xc7, 0x26, 0x6e, 0xdc, 0xee, 0x26, 0x67, 0x1d, + 0xe4, 0xbe, 0xfa, 0xe6, 0xc1, 0x8a, 0x95, 0x10, 0xa0, 0xdf, 0x43, 0x9d, 0x0d, 0x82, 0x70, 0xc6, + 0x93, 0x8c, 0xf4, 0xe4, 0x49, 0x1a, 0x4f, 0xce, 0x05, 0x43, 0xe2, 0x46, 0x8d, 0xcd, 0x2e, 0xd1, + 0x7d, 0x00, 0x36, 0xb8, 0x26, 0x3e, 0xb6, 0xc7, 0x91, 0x67, 0x66, 0x9b, 0x46, 0xbb, 0x6c, 0x95, + 0x95, 0xe4, 0x22, 0xf2, 0x3e, 0x2f, 0x94, 0xbe, 0x2d, 0x36, 0xfe, 0x5d, 0x6c, 0xbd, 0x32, 0xa0, + 0x36, 0xc7, 0x83, 0x4e, 0x21, 0x2f, 0x99, 0xb4, 0x93, 0x8f, 0x97, 0x5a, 0xa4, 0x23, 0x7b, 0xbb, + 0xbb, 0xdd, 0xa5, 0x8c, 0x47, 0x63, 0x61, 0x0f, 0xe6, 0x6e, 0x40, 0x25, 0x97, 0x76, 0x57, 0xf1, + 0xa0, 0x53, 0x28, 0x2d, 0x78, 0xf9, 0x38, 0x8d, 0x97, 0xda, 0x30, 0x2b, 0x21, 0xf9, 0x0e, 0xd7, + 0x5a, 0x7f, 0xa9, 0x40, 0x51, 0x6f, 0x42, 0x97, 0x50, 0x61, 0xd8, 0x0f, 0x3d, 0x62, 0xf3, 0x89, + 0x74, 0x49, 0x1c, 0xff, 0xd3, 0x34, 0xc7, 0x5f, 0x62, 0x6f, 0x4c, 0xfa, 0x93, 0x90, 0x58, 0xa0, + 0x98, 0xc4, 0x37, 0xfa, 0x1c, 0x0a, 0x6a, 0xa5, 0x3d, 0xda, 0x4b, 0x15, 0x37, 0xb9, 0xd3, 0xd2, + 0x0c, 0xe8, 0x37, 0x50, 0xf3, 0x71, 0x18, 0xba, 0x74, 0x64, 0x73, 0x7c, 0xe5, 0x11, 0x33, 0x9b, + 0xfe, 0x92, 0x8e, 0x15, 0x81, 0x55, 0xd5, 0x4c, 0x7d, 0x41, 0x84, 0x7e, 0x07, 0x75, 0x2f, 0x18, + 0xc8, 0xb8, 0x68, 0xea, 0x9c, 0xa4, 0xfe, 0x24, 0x0d, 0x75, 0x4f, 0x33, 0x58, 0xb5, 0x98, 0x4b, + 0x91, 0x7f, 0x04, 0x8d, 0x84, 0xdc, 0xa5, 0x8e, 0x3b, 0x20, 0xcc, 0xcc, 0x37, 0xb3, 0xed, 0xbc, + 0xb5, 0x1a, 0xcb, 0xbb, 0x4a, 0x2c, 0xec, 0x18, 0x8e, 0xe9, 0x60, 0xc6, 0x8e, 0x42, 0x7a, 0x3b, + 0x8e, 0x34, 0x83, 0x55, 0x8b, 0xb9, 0x94, 0x1d, 0x97, 0xb0, 0x8a, 0x39, 0x8f, 0xdc, 0xab, 0x31, + 0x27, 0x9a, 0xbd, 0x28, 0xd9, 0x3f, 0xfc, 0x8e, 0xcc, 0x7d, 0x4e, 0x26, 0x32, 0xb8, 0x3a, 0x5b, + 0xeb, 0x09, 0x8b, 0xe2, 0xbd, 0x9a, 0xe5, 0x1d, 0x53, 0x97, 0x33, 0xb3, 0x94, 0xbe, 0x46, 0xf7, + 0x63, 0x8a, 0x0b, 0xea, 0xf2, 0x99, 0x33, 0xc4, 0x52, 0xd4, 0x1a, 0x78, 0x2e, 0xbd, 0xd1, 0x66, + 0x97, 0x25, 0xfd, 0xa3, 0x54, 0xc1, 0x71, 0xe9, 0x8d, 0x55, 0x16, 0x1c, 0xca, 0xe8, 0xf7, 0xa1, + 0xca, 0x78, 0x34, 0x4d, 0x25, 0x68, 0x66, 0xdb, 0x65, 0xab, 0xa2, 0x64, 0x0a, 0x72, 0x1f, 0x80, + 0xbb, 0x3e, 0xb1, 0x29, 0xa6, 0x01, 0x33, 0x2b, 0x4d, 0xa3, 0x9d, 0xb5, 0xca, 0x42, 0x72, 0x22, + 0x04, 0xe8, 0x03, 0xa8, 0x3b, 0xe3, 0x48, 0x85, 0x55, 0x41, 0xaa, 0x12, 0x52, 0x8b, 0xa5, 0x0a, + 0xf6, 0x05, 0x54, 0x42, 0x12, 0xb9, 0x81, 0xa3, 0x0a, 0xab, 0x26, 0x7b, 0xc5, 0xdb, 0x15, 0x96, + 0xbe, 0x7f, 0x50, 0x7c, 0xb2, 0xbc, 0x36, 0xa0, 0xa0, 0x56, 0x66, 0x5d, 0x1e, 0xae, 0x57, 0xe8, + 0x63, 0x40, 0x22, 0x7e, 0x84, 0x72, 0x5b, 0xba, 0xa4, 0xb2, 0x6e, 0x55, 0x66, 0xdd, 0x9a, 0xd6, + 0x9c, 0x27, 0x0a, 0xf4, 0x4b, 0xf8, 0x91, 0x43, 0x86, 0x78, 0xec, 0x71, 0x7b, 0xa6, 0x0b, 0xe8, + 0xad, 0xe4, 0xa5, 0xd9, 0x68, 0x1a, 0xed, 0xbc, 0xf5, 0x03, 0x8d, 0x39, 0x4f, 0xca, 0xfb, 0x5c, + 0x03, 0xd0, 0x15, 0x80, 0xb6, 0xde, 0x76, 0x1d, 0x73, 0xad, 0x69, 0xb4, 0xab, 0x07, 0x4f, 0x85, + 0xb5, 0xff, 0xfc, 0xe6, 0xc1, 0xcf, 0x47, 0xc1, 0x82, 0xbb, 0xae, 0x78, 0xfb, 0x3c, 0x8f, 0x0c, + 0x78, 0x10, 0xed, 0x84, 0x0e, 0xe6, 0x78, 0xc7, 0xa5, 0x9c, 0x44, 0x14, 0x7b, 0x3b, 0x62, 0x15, + 0xb7, 0xb2, 0x6e, 0xc7, 0x2a, 0x6b, 0xda, 0xae, 0x83, 0x8e, 0x01, 0x92, 0xac, 0x60, 0x26, 0x7a, + 0x9b, 0xd4, 0x9d, 0x21, 0x40, 0x9f, 0x82, 0xe9, 0x44, 0x41, 0x18, 0x12, 0xc7, 0x9e, 0x4a, 0xed, + 0x41, 0x30, 0xa6, 0xdc, 0x7c, 0xa7, 0x69, 0xb4, 0x6b, 0xd6, 0x86, 0xd6, 0x27, 0xa9, 0xc9, 0x9e, + 0x0a, 0x2d, 0xfa, 0x19, 0xbc, 0x1b, 0x44, 0xee, 0xc8, 0xa5, 0xd8, 0xb3, 0x43, 0x3c, 0xf1, 0x02, + 0xec, 0xd8, 0xc3, 0x20, 0xf2, 0x31, 0x37, 0xd7, 0x65, 0x8f, 0xbd, 0x17, 0xab, 0xcf, 0x94, 0xf6, + 0x48, 0x2a, 0x45, 0x23, 0x58, 0xdc, 0x67, 0xde, 0x13, 0x57, 0x65, 0xad, 0x2e, 0x6c, 0x68, 0xfd, + 0x01, 0x6a, 0x73, 0x05, 0x81, 0x3e, 0x81, 0x8d, 0x69, 0x91, 0xdd, 0x90, 0xc9, 0x34, 0x36, 0x86, + 0x8c, 0xcd, 0x7a, 0xa2, 0x7d, 0x4e, 0x26, 0x49, 0x58, 0x1e, 0x42, 0x4d, 0x14, 0xe4, 0x14, 0x9c, + 0x91, 0xe0, 0xaa, 0x10, 0xc6, 0xa0, 0xd6, 0xdf, 0x0d, 0xc8, 0x89, 0xf2, 0x40, 0x5f, 0x40, 0x89, + 0x47, 0x78, 0x20, 0x43, 0x68, 0xc8, 0x10, 0xee, 0xeb, 0x10, 0x3e, 0x49, 0x1f, 0xc2, 0xbe, 0x60, + 0xea, 0x76, 0xac, 0xa2, 0xa4, 0xec, 0x3a, 0xe8, 0x05, 0x14, 0x59, 0x88, 0xa9, 0x20, 0xcf, 0x48, + 0xf2, 0x5f, 0x69, 0xf2, 0x4f, 0xd3, 0x93, 0x9f, 0x87, 0x98, 0x76, 0x3b, 0x56, 0x41, 0x10, 0x76, + 0x9d, 0xd6, 0x3f, 0x0c, 0x28, 0x27, 0x55, 0x22, 0x9c, 0x9e, 0xcf, 0x5e, 0x75, 0x43, 0x55, 0x3e, + 0x9b, 0xb0, 0xdf, 0xe7, 0x66, 0xd0, 0x1f, 0xe1, 0x5d, 0x3c, 0x1a, 0x45, 0x64, 0xa4, 0x5f, 0x06, + 0xe2, 0x87, 0x41, 0x84, 0x3d, 0x97, 0x4f, 0xe4, 0x63, 0x5a, 0xdf, 0x3b, 0x48, 0xd5, 0xe1, 0xa6, + 0x54, 0xfd, 0x29, 0x93, 0xb5, 0x81, 0x97, 0xca, 0x5b, 0x5f, 0x66, 0xa0, 0xa0, 0x2a, 0x0d, 0xed, + 0xc1, 0xbd, 0xf8, 0xa5, 0x60, 0x36, 0xe3, 0x38, 0xe2, 0xf6, 0xac, 0x67, 0xef, 0x24, 0xca, 0x73, + 0xa1, 0xeb, 0x4a, 0xdb, 0x67, 0x5e, 0x1d, 0x66, 0x7b, 0x84, 0x8e, 0xf8, 0xb5, 0xf6, 0x31, 0x79, + 0x75, 0x58, 0x4f, 0x8a, 0xd1, 0x3a, 0xe4, 0x6f, 0xc5, 0xed, 0xc9, 0xf7, 0x34, 0x6b, 0xa9, 0x05, + 0xfa, 0x09, 0xac, 0x4d, 0x33, 0x2e, 0xee, 0x20, 0x39, 0xd9, 0x41, 0x1a, 0x89, 0x22, 0x7e, 0xb8, + 0x1e, 0xe8, 0xfe, 0xac, 0xcc, 0xca, 0x8b, 0x73, 0x3e, 0x5b, 0x51, 0xfd, 0x56, 0x99, 0xf3, 0x08, + 0xd6, 0x45, 0xeb, 0x64, 0x1c, 0xfb, 0x21, 0x13, 0xaf, 0xc4, 0x4b, 0xd9, 0x34, 0xe5, 0xfb, 0x96, + 0xb3, 0xd0, 0x54, 0x77, 0x41, 0xdd, 0x97, 0xa2, 0x73, 0x1e, 0xd4, 0xa0, 0x32, 0xa5, 0xb4, 0x5b, + 0x7f, 0x32, 0x20, 0xdf, 0xc3, 0x57, 0xc4, 0x13, 0xad, 0x7b, 0x49, 0x01, 0x54, 0x6e, 0x66, 0xf2, + 0x5e, 0x75, 0xf7, 0xc5, 0xe0, 0x8a, 0xee, 0x9e, 0x40, 0x1a, 0x90, 0xa5, 0x63, 0x5f, 0xc6, 0x31, + 0x6b, 0x89, 0x4f, 0xb4, 0x05, 0x6b, 0x74, 0xec, 0xdb, 0xf3, 0x69, 0x91, 0x53, 0x57, 0x46, 0xc7, + 0xfe, 0xc5, 0x6c, 0xcd, 0xfc, 0x27, 0x03, 0x45, 0x3d, 0x4a, 0x88, 0xc3, 0x7c, 0xe2, 0x07, 0xd1, + 0x44, 0x85, 0x46, 0xda, 0x93, 0xb3, 0x2a, 0x4a, 0x26, 0x23, 0x32, 0x03, 0xf1, 0x5c, 0xdf, 0xe5, + 0xd2, 0x9e, 0x04, 0xd2, 0x13, 0x22, 0xf4, 0x00, 0x2a, 0xb2, 0x7d, 0x06, 0xc3, 0x21, 0x23, 0x5c, + 0xda, 0x95, 0xb3, 0x40, 0x88, 0x4e, 0xa5, 0x44, 0xc4, 0x43, 0xac, 0x28, 0xf6, 0xc9, 0xa2, 0x79, + 0x8d, 0x58, 0x91, 0x78, 0xb7, 0x34, 0x78, 0xf9, 0x3b, 0x82, 0xf7, 0x10, 0x6a, 0xd7, 0x98, 0xd9, + 0xf1, 0xb4, 0xc0, 0xcc, 0x42, 0xd3, 0x68, 0x97, 0xac, 0xea, 0x35, 0x66, 0xf1, 0x2c, 0x31, 0x05, + 0xe9, 0x93, 0x98, 0x59, 0x9c, 0x82, 0x62, 0x19, 0x6a, 0x43, 0x43, 0x80, 0x3c, 0x97, 0x12, 0x9b, + 0x8e, 0xfd, 0x2b, 0x12, 0x89, 0x59, 0x40, 0xe0, 0xea, 0xd7, 0x98, 0xf5, 0x5c, 0x4a, 0x4e, 0x94, + 0x54, 0x5c, 0xb6, 0x40, 0xba, 0x54, 0x62, 0x87, 0x91, 0xa4, 0x2c, 0x4b, 0xe8, 0xea, 0x35, 0x66, + 0x5d, 0x29, 0x3f, 0x92, 0xe2, 0xd6, 0x7f, 0x0d, 0x28, 0xc5, 0xc3, 0x15, 0xfa, 0x60, 0x3a, 0x04, + 0xce, 0x84, 0xff, 0xb3, 0x95, 0x64, 0xa2, 0x53, 0xf9, 0x66, 0x42, 0x11, 0x3b, 0x4e, 0x44, 0x18, + 0xd3, 0x97, 0x1d, 0x2f, 0x51, 0x07, 0x72, 0x82, 0x5b, 0x0f, 0x8f, 0x69, 0x87, 0x08, 0x62, 0xc9, + 0xdd, 0xe8, 0x87, 0x50, 0x76, 0x99, 0x3d, 0x0c, 0x3c, 0x87, 0x38, 0x32, 0x0a, 0x25, 0xab, 0xe4, + 0xb2, 0x23, 0xb9, 0x4e, 0x75, 0xfb, 0x07, 0x0d, 0xa8, 0xcf, 0x39, 0x64, 0xb7, 0x5e, 0xc8, 0x7e, + 0x4c, 0xc4, 0x84, 0x91, 0x4c, 0x83, 0xb3, 0xa9, 0x9e, 0xcc, 0x75, 0xca, 0x55, 0xa4, 0x1d, 0xca, + 0xc8, 0x54, 0x56, 0xe6, 0x6d, 0x40, 0x61, 0x10, 0x78, 0x63, 0x9f, 0xea, 0x04, 0xd7, 0xab, 0xd6, + 0xdf, 0x0c, 0x28, 0xc5, 0x31, 0x15, 0x21, 0x9d, 0xcf, 0x26, 0xdd, 0x28, 0xe7, 0x32, 0xe9, 0x11, + 0xac, 0xb3, 0x09, 0xe3, 0xc4, 0xb7, 0xe7, 0xb1, 0xaa, 0xa4, 0x90, 0xd2, 0x9d, 0x2c, 0xe4, 0xde, + 0xff, 0x27, 0x6a, 0xf6, 0x8e, 0x44, 0x15, 0xff, 0xa2, 0xc8, 0x86, 0x26, 0x5d, 0xc8, 0xa9, 0x21, + 0x4b, 0x4a, 0xc4, 0x15, 0x6c, 0x7d, 0x69, 0xc0, 0xc6, 0xf2, 0xbe, 0x89, 0x3e, 0x84, 0x87, 0xfb, + 0xcf, 0x9e, 0x59, 0x87, 0xcf, 0xf6, 0xfb, 0xdd, 0xd3, 0x13, 0xbb, 0x7f, 0x78, 0x7c, 0x76, 0x6a, + 0xed, 0xf7, 0xba, 0xfd, 0x17, 0xf6, 0xc5, 0xc9, 0xf9, 0xd9, 0xe1, 0xd3, 0xee, 0x51, 0xf7, 0xb0, + 0xd3, 0x58, 0x41, 0xef, 0xc3, 0xfd, 0xbb, 0x80, 0x9d, 0xc3, 0x5e, 0x7f, 0xbf, 0x61, 0xa0, 0x1f, + 0x43, 0xeb, 0x2e, 0xc8, 0xd3, 0x8b, 0xe3, 0x8b, 0xde, 0x7e, 0xbf, 0x7b, 0x79, 0xd8, 0xc8, 0x1c, + 0x7c, 0x6d, 0x7c, 0xf5, 0x7a, 0xd3, 0x78, 0xf5, 0x7a, 0xd3, 0xf8, 0xd7, 0xeb, 0x4d, 0xe3, 0xcf, + 0x6f, 0x36, 0x57, 0x5e, 0xbd, 0xd9, 0x5c, 0xf9, 0xfa, 0xcd, 0xe6, 0x0a, 0x7c, 0xec, 0x06, 0x29, + 0x52, 0xe9, 0xa0, 0x16, 0xff, 0x1b, 0x79, 0x26, 0x50, 0x67, 0xc6, 0x6f, 0x7f, 0x9d, 0xfa, 0x15, + 0x54, 0xbf, 0x0c, 0x8c, 0x08, 0xbd, 0xe3, 0x57, 0x8c, 0xbf, 0x66, 0xb6, 0x4e, 0x43, 0x42, 0xfb, + 0x09, 0xa1, 0x3c, 0x2a, 0x9e, 0xad, 0xd8, 0xf6, 0xe5, 0x6e, 0x67, 0x0a, 0xbe, 0x2a, 0x48, 0xb6, + 0xc7, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xbc, 0x31, 0xf3, 0xe8, 0x27, 0x11, 0x00, 0x00, +} + +func (m *ProfilesData) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1276,92 +1465,225 @@ func (m *Profile) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *Profile) MarshalTo(dAtA []byte) (int, error) { +func (m *ProfilesData) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *Profile) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *ProfilesData) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if len(m.LinkTable) > 0 { - for iNdEx := len(m.LinkTable) - 1; iNdEx >= 0; iNdEx-- { + if len(m.ResourceProfiles) > 0 { + for iNdEx := len(m.ResourceProfiles) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.LinkTable[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.ResourceProfiles[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x1 - i-- - dAtA[i] = 0x92 + dAtA[i] = 0xa } } - if len(m.AttributeUnits) > 0 { - for iNdEx := len(m.AttributeUnits) - 1; iNdEx >= 0; iNdEx-- { + return len(dAtA) - i, nil +} + +func (m *ResourceProfiles) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ResourceProfiles) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ResourceProfiles) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.SchemaUrl) > 0 { + i -= len(m.SchemaUrl) + copy(dAtA[i:], m.SchemaUrl) + i = encodeVarintProfiles(dAtA, i, uint64(len(m.SchemaUrl))) + i-- + dAtA[i] = 0x1a + } + if len(m.ScopeProfiles) > 0 { + for iNdEx := len(m.ScopeProfiles) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.AttributeUnits[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.ScopeProfiles[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x1 - i-- - dAtA[i] = 0x8a + dAtA[i] = 0x12 } } - if len(m.AttributeTable) > 0 { - for iNdEx := len(m.AttributeTable) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Resource.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintProfiles(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *ScopeProfiles) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ScopeProfiles) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ScopeProfiles) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.SchemaUrl) > 0 { + i -= len(m.SchemaUrl) + copy(dAtA[i:], m.SchemaUrl) + i = encodeVarintProfiles(dAtA, i, uint64(len(m.SchemaUrl))) + i-- + dAtA[i] = 0x1a + } + if len(m.Profiles) > 0 { + for iNdEx := len(m.Profiles) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.AttributeTable[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Profiles[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x1 - i-- - dAtA[i] = 0x82 + dAtA[i] = 0x12 } } - if len(m.LocationIndices) > 0 { - dAtA2 := make([]byte, len(m.LocationIndices)*10) - var j1 int - for _, num1 := range m.LocationIndices { - num := uint64(num1) - for num >= 1<<7 { - dAtA2[j1] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j1++ - } - dAtA2[j1] = uint8(num) - j1++ + { + size, err := m.Scope.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err } - i -= j1 - copy(dAtA[i:], dAtA2[:j1]) - i = encodeVarintPprofextended(dAtA, i, uint64(j1)) - i-- - dAtA[i] = 0x7a + i -= size + i = encodeVarintProfiles(dAtA, i, uint64(size)) } - if m.DefaultSampleType != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.DefaultSampleType)) + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *Profile) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Profile) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Profile) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.OriginalPayload) > 0 { + i -= len(m.OriginalPayload) + copy(dAtA[i:], m.OriginalPayload) + i = encodeVarintProfiles(dAtA, i, uint64(len(m.OriginalPayload))) i-- - dAtA[i] = 0x70 + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xaa + } + if len(m.OriginalPayloadFormat) > 0 { + i -= len(m.OriginalPayloadFormat) + copy(dAtA[i:], m.OriginalPayloadFormat) + i = encodeVarintProfiles(dAtA, i, uint64(len(m.OriginalPayloadFormat))) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xa2 + } + if m.DroppedAttributesCount != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.DroppedAttributesCount)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x98 + } + if len(m.Attributes) > 0 { + for iNdEx := len(m.Attributes) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Attributes[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintProfiles(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x92 + } + } + { + size := m.ProfileId.Size() + i -= size + if _, err := m.ProfileId.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintProfiles(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + if m.DefaultSampleTypeStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.DefaultSampleTypeStrindex)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x80 } - if len(m.Comment) > 0 { - dAtA4 := make([]byte, len(m.Comment)*10) + if len(m.CommentStrindices) > 0 { + dAtA4 := make([]byte, len(m.CommentStrindices)*10) var j3 int - for _, num1 := range m.Comment { + for _, num1 := range m.CommentStrindices { num := uint64(num1) for num >= 1<<7 { dAtA4[j3] = uint8(uint64(num)&0x7f | 0x80) @@ -1373,14 +1695,14 @@ func (m *Profile) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i -= j3 copy(dAtA[i:], dAtA4[:j3]) - i = encodeVarintPprofextended(dAtA, i, uint64(j3)) + i = encodeVarintProfiles(dAtA, i, uint64(j3)) i-- - dAtA[i] = 0x6a + dAtA[i] = 0x7a } if m.Period != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Period)) + i = encodeVarintProfiles(dAtA, i, uint64(m.Period)) i-- - dAtA[i] = 0x60 + dAtA[i] = 0x70 } { size, err := m.PeriodType.MarshalToSizedBuffer(dAtA[:i]) @@ -1388,76 +1710,127 @@ func (m *Profile) MarshalToSizedBuffer(dAtA []byte) (int, error) { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x5a + dAtA[i] = 0x6a if m.DurationNanos != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.DurationNanos)) + i = encodeVarintProfiles(dAtA, i, uint64(m.DurationNanos)) i-- - dAtA[i] = 0x50 + dAtA[i] = 0x60 } if m.TimeNanos != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.TimeNanos)) - i-- - dAtA[i] = 0x48 - } - if m.KeepFrames != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.KeepFrames)) - i-- - dAtA[i] = 0x40 - } - if m.DropFrames != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.DropFrames)) + i = encodeVarintProfiles(dAtA, i, uint64(m.TimeNanos)) i-- - dAtA[i] = 0x38 + dAtA[i] = 0x58 } if len(m.StringTable) > 0 { for iNdEx := len(m.StringTable) - 1; iNdEx >= 0; iNdEx-- { i -= len(m.StringTable[iNdEx]) copy(dAtA[i:], m.StringTable[iNdEx]) - i = encodeVarintPprofextended(dAtA, i, uint64(len(m.StringTable[iNdEx]))) + i = encodeVarintProfiles(dAtA, i, uint64(len(m.StringTable[iNdEx]))) i-- - dAtA[i] = 0x32 + dAtA[i] = 0x52 + } + } + if len(m.LinkTable) > 0 { + for iNdEx := len(m.LinkTable) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.LinkTable[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintProfiles(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + } + if len(m.AttributeUnits) > 0 { + for iNdEx := len(m.AttributeUnits) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.AttributeUnits[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintProfiles(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + } + if len(m.AttributeTable) > 0 { + for iNdEx := len(m.AttributeTable) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.AttributeTable[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintProfiles(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a } } - if len(m.Function) > 0 { - for iNdEx := len(m.Function) - 1; iNdEx >= 0; iNdEx-- { + if len(m.FunctionTable) > 0 { + for iNdEx := len(m.FunctionTable) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Function[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.FunctionTable[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x2a + dAtA[i] = 0x32 + } + } + if len(m.LocationIndices) > 0 { + dAtA7 := make([]byte, len(m.LocationIndices)*10) + var j6 int + for _, num1 := range m.LocationIndices { + num := uint64(num1) + for num >= 1<<7 { + dAtA7[j6] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j6++ + } + dAtA7[j6] = uint8(num) + j6++ } + i -= j6 + copy(dAtA[i:], dAtA7[:j6]) + i = encodeVarintProfiles(dAtA, i, uint64(j6)) + i-- + dAtA[i] = 0x2a } - if len(m.Location) > 0 { - for iNdEx := len(m.Location) - 1; iNdEx >= 0; iNdEx-- { + if len(m.LocationTable) > 0 { + for iNdEx := len(m.LocationTable) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Location[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.LocationTable[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- dAtA[i] = 0x22 } } - if len(m.Mapping) > 0 { - for iNdEx := len(m.Mapping) - 1; iNdEx >= 0; iNdEx-- { + if len(m.MappingTable) > 0 { + for iNdEx := len(m.MappingTable) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Mapping[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.MappingTable[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- dAtA[i] = 0x1a @@ -1471,7 +1844,7 @@ func (m *Profile) MarshalToSizedBuffer(dAtA []byte) (int, error) { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- dAtA[i] = 0x12 @@ -1485,7 +1858,7 @@ func (m *Profile) MarshalToSizedBuffer(dAtA []byte) (int, error) { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- dAtA[i] = 0xa @@ -1514,13 +1887,13 @@ func (m *AttributeUnit) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Unit != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Unit)) + if m.UnitStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.UnitStrindex)) i-- dAtA[i] = 0x10 } - if m.AttributeKey != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.AttributeKey)) + if m.AttributeKeyStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.AttributeKeyStrindex)) i-- dAtA[i] = 0x8 } @@ -1553,7 +1926,7 @@ func (m *Link) MarshalToSizedBuffer(dAtA []byte) (int, error) { if _, err := m.SpanId.MarshalTo(dAtA[i:]); err != nil { return 0, err } - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- dAtA[i] = 0x12 @@ -1563,7 +1936,7 @@ func (m *Link) MarshalToSizedBuffer(dAtA []byte) (int, error) { if _, err := m.TraceId.MarshalTo(dAtA[i:]); err != nil { return 0, err } - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- dAtA[i] = 0xa @@ -1591,17 +1964,17 @@ func (m *ValueType) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if m.AggregationTemporality != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.AggregationTemporality)) + i = encodeVarintProfiles(dAtA, i, uint64(m.AggregationTemporality)) i-- dAtA[i] = 0x18 } - if m.Unit != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Unit)) + if m.UnitStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.UnitStrindex)) i-- dAtA[i] = 0x10 } - if m.Type != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Type)) + if m.TypeStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.TypeStrindex)) i-- dAtA[i] = 0x8 } @@ -1629,32 +2002,9 @@ func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if len(m.TimestampsUnixNano) > 0 { - dAtA7 := make([]byte, len(m.TimestampsUnixNano)*10) - var j6 int - for _, num := range m.TimestampsUnixNano { - for num >= 1<<7 { - dAtA7[j6] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j6++ - } - dAtA7[j6] = uint8(num) - j6++ - } - i -= j6 - copy(dAtA[i:], dAtA7[:j6]) - i = encodeVarintPprofextended(dAtA, i, uint64(j6)) - i-- - dAtA[i] = 0x6a - } - if m.Link != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Link)) - i-- - dAtA[i] = 0x60 - } - if len(m.Attributes) > 0 { - dAtA9 := make([]byte, len(m.Attributes)*10) + dAtA9 := make([]byte, len(m.TimestampsUnixNano)*10) var j8 int - for _, num := range m.Attributes { + for _, num := range m.TimestampsUnixNano { for num >= 1<<7 { dAtA9[j8] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 @@ -1665,43 +2015,23 @@ func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i -= j8 copy(dAtA[i:], dAtA9[:j8]) - i = encodeVarintPprofextended(dAtA, i, uint64(j8)) - i-- - dAtA[i] = 0x52 - } - if m.StacktraceIdIndex != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.StacktraceIdIndex)) - i-- - dAtA[i] = 0x48 - } - if m.LocationsLength != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.LocationsLength)) - i-- - dAtA[i] = 0x40 - } - if m.LocationsStartIndex != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.LocationsStartIndex)) + i = encodeVarintProfiles(dAtA, i, uint64(j8)) i-- - dAtA[i] = 0x38 + dAtA[i] = 0x32 } - if len(m.Label) > 0 { - for iNdEx := len(m.Label) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Label[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + if m.LinkIndex_ != nil { + { + size := m.LinkIndex_.Size() + i -= size + if _, err := m.LinkIndex_.MarshalTo(dAtA[i:]); err != nil { + return 0, err } - i-- - dAtA[i] = 0x1a } } - if len(m.Value) > 0 { - dAtA11 := make([]byte, len(m.Value)*10) + if len(m.AttributeIndices) > 0 { + dAtA11 := make([]byte, len(m.AttributeIndices)*10) var j10 int - for _, num1 := range m.Value { + for _, num1 := range m.AttributeIndices { num := uint64(num1) for num >= 1<<7 { dAtA11[j10] = uint8(uint64(num)&0x7f | 0x80) @@ -1713,14 +2043,15 @@ func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i -= j10 copy(dAtA[i:], dAtA11[:j10]) - i = encodeVarintPprofextended(dAtA, i, uint64(j10)) + i = encodeVarintProfiles(dAtA, i, uint64(j10)) i-- - dAtA[i] = 0x12 + dAtA[i] = 0x22 } - if len(m.LocationIndex) > 0 { - dAtA13 := make([]byte, len(m.LocationIndex)*10) + if len(m.Value) > 0 { + dAtA13 := make([]byte, len(m.Value)*10) var j12 int - for _, num := range m.LocationIndex { + for _, num1 := range m.Value { + num := uint64(num1) for num >= 1<<7 { dAtA13[j12] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 @@ -1731,13 +2062,35 @@ func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i -= j12 copy(dAtA[i:], dAtA13[:j12]) - i = encodeVarintPprofextended(dAtA, i, uint64(j12)) + i = encodeVarintProfiles(dAtA, i, uint64(j12)) + i-- + dAtA[i] = 0x1a + } + if m.LocationsLength != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.LocationsLength)) + i-- + dAtA[i] = 0x10 + } + if m.LocationsStartIndex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.LocationsStartIndex)) i-- - dAtA[i] = 0xa + dAtA[i] = 0x8 } return len(dAtA) - i, nil } +func (m *Sample_LinkIndex) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Sample_LinkIndex) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintProfiles(dAtA, i, uint64(m.LinkIndex)) + i-- + dAtA[i] = 0x28 + return len(dAtA) - i, nil +} func (m *Label) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1758,23 +2111,23 @@ func (m *Label) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.NumUnit != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.NumUnit)) + if m.NumUnitStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.NumUnitStrindex)) i-- dAtA[i] = 0x20 } if m.Num != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Num)) + i = encodeVarintProfiles(dAtA, i, uint64(m.Num)) i-- dAtA[i] = 0x18 } - if m.Str != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Str)) + if m.StrStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.StrStrindex)) i-- dAtA[i] = 0x10 } - if m.Key != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Key)) + if m.KeyStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.KeyStrindex)) i-- dAtA[i] = 0x8 } @@ -1801,29 +2154,6 @@ func (m *Mapping) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.Attributes) > 0 { - dAtA15 := make([]byte, len(m.Attributes)*10) - var j14 int - for _, num := range m.Attributes { - for num >= 1<<7 { - dAtA15[j14] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j14++ - } - dAtA15[j14] = uint8(num) - j14++ - } - i -= j14 - copy(dAtA[i:], dAtA15[:j14]) - i = encodeVarintPprofextended(dAtA, i, uint64(j14)) - i-- - dAtA[i] = 0x62 - } - if m.BuildIdKind != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.BuildIdKind)) - i-- - dAtA[i] = 0x58 - } if m.HasInlineFrames { i-- if m.HasInlineFrames { @@ -1832,7 +2162,7 @@ func (m *Mapping) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0 } i-- - dAtA[i] = 0x50 + dAtA[i] = 0x48 } if m.HasLineNumbers { i-- @@ -1842,7 +2172,7 @@ func (m *Mapping) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0 } i-- - dAtA[i] = 0x48 + dAtA[i] = 0x40 } if m.HasFilenames { i-- @@ -1852,7 +2182,7 @@ func (m *Mapping) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0 } i-- - dAtA[i] = 0x40 + dAtA[i] = 0x38 } if m.HasFunctions { i-- @@ -1862,35 +2192,44 @@ func (m *Mapping) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0 } i-- - dAtA[i] = 0x38 - } - if m.BuildId != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.BuildId)) - i-- dAtA[i] = 0x30 } - if m.Filename != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Filename)) + if len(m.AttributeIndices) > 0 { + dAtA15 := make([]byte, len(m.AttributeIndices)*10) + var j14 int + for _, num1 := range m.AttributeIndices { + num := uint64(num1) + for num >= 1<<7 { + dAtA15[j14] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j14++ + } + dAtA15[j14] = uint8(num) + j14++ + } + i -= j14 + copy(dAtA[i:], dAtA15[:j14]) + i = encodeVarintProfiles(dAtA, i, uint64(j14)) i-- - dAtA[i] = 0x28 + dAtA[i] = 0x2a } - if m.FileOffset != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.FileOffset)) + if m.FilenameStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.FilenameStrindex)) i-- dAtA[i] = 0x20 } - if m.MemoryLimit != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.MemoryLimit)) + if m.FileOffset != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.FileOffset)) i-- dAtA[i] = 0x18 } - if m.MemoryStart != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.MemoryStart)) + if m.MemoryLimit != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.MemoryLimit)) i-- dAtA[i] = 0x10 } - if m.Id != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Id)) + if m.MemoryStart != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.MemoryStart)) i-- dAtA[i] = 0x8 } @@ -1917,10 +2256,11 @@ func (m *Location) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.Attributes) > 0 { - dAtA17 := make([]byte, len(m.Attributes)*10) + if len(m.AttributeIndices) > 0 { + dAtA17 := make([]byte, len(m.AttributeIndices)*10) var j16 int - for _, num := range m.Attributes { + for _, num1 := range m.AttributeIndices { + num := uint64(num1) for num >= 1<<7 { dAtA17[j16] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 @@ -1931,14 +2271,9 @@ func (m *Location) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i -= j16 copy(dAtA[i:], dAtA17[:j16]) - i = encodeVarintPprofextended(dAtA, i, uint64(j16)) - i-- - dAtA[i] = 0x3a - } - if m.TypeIndex != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.TypeIndex)) + i = encodeVarintProfiles(dAtA, i, uint64(j16)) i-- - dAtA[i] = 0x30 + dAtA[i] = 0x2a } if m.IsFolded { i-- @@ -1948,7 +2283,7 @@ func (m *Location) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0 } i-- - dAtA[i] = 0x28 + dAtA[i] = 0x20 } if len(m.Line) > 0 { for iNdEx := len(m.Line) - 1; iNdEx >= 0; iNdEx-- { @@ -1958,30 +2293,41 @@ func (m *Location) MarshalToSizedBuffer(dAtA []byte) (int, error) { return 0, err } i -= size - i = encodeVarintPprofextended(dAtA, i, uint64(size)) + i = encodeVarintProfiles(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x22 + dAtA[i] = 0x1a } } if m.Address != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Address)) - i-- - dAtA[i] = 0x18 - } - if m.MappingIndex != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.MappingIndex)) + i = encodeVarintProfiles(dAtA, i, uint64(m.Address)) i-- dAtA[i] = 0x10 } - if m.Id != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Id)) - i-- - dAtA[i] = 0x8 + if m.MappingIndex_ != nil { + { + size := m.MappingIndex_.Size() + i -= size + if _, err := m.MappingIndex_.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } } return len(dAtA) - i, nil } +func (m *Location_MappingIndex) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Location_MappingIndex) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintProfiles(dAtA, i, uint64(m.MappingIndex)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} func (m *Line) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2003,17 +2349,17 @@ func (m *Line) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if m.Column != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Column)) + i = encodeVarintProfiles(dAtA, i, uint64(m.Column)) i-- dAtA[i] = 0x18 } if m.Line != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Line)) + i = encodeVarintProfiles(dAtA, i, uint64(m.Line)) i-- dAtA[i] = 0x10 } if m.FunctionIndex != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.FunctionIndex)) + i = encodeVarintProfiles(dAtA, i, uint64(m.FunctionIndex)) i-- dAtA[i] = 0x8 } @@ -2041,35 +2387,30 @@ func (m *Function) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if m.StartLine != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.StartLine)) - i-- - dAtA[i] = 0x28 - } - if m.Filename != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Filename)) + i = encodeVarintProfiles(dAtA, i, uint64(m.StartLine)) i-- dAtA[i] = 0x20 } - if m.SystemName != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.SystemName)) + if m.FilenameStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.FilenameStrindex)) i-- dAtA[i] = 0x18 } - if m.Name != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Name)) + if m.SystemNameStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.SystemNameStrindex)) i-- dAtA[i] = 0x10 } - if m.Id != 0 { - i = encodeVarintPprofextended(dAtA, i, uint64(m.Id)) + if m.NameStrindex != 0 { + i = encodeVarintProfiles(dAtA, i, uint64(m.NameStrindex)) i-- dAtA[i] = 0x8 } return len(dAtA) - i, nil } -func encodeVarintPprofextended(dAtA []byte, offset int, v uint64) int { - offset -= sovPprofextended(v) +func encodeVarintProfiles(dAtA []byte, offset int, v uint64) int { + offset -= sovProfiles(v) base := offset for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -2079,100 +2420,170 @@ func encodeVarintPprofextended(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } -func (m *Profile) Size() (n int) { +func (m *ProfilesData) Size() (n int) { if m == nil { return 0 } var l int _ = l - if len(m.SampleType) > 0 { - for _, e := range m.SampleType { - l = e.Size() - n += 1 + l + sovPprofextended(uint64(l)) - } - } - if len(m.Sample) > 0 { - for _, e := range m.Sample { + if len(m.ResourceProfiles) > 0 { + for _, e := range m.ResourceProfiles { l = e.Size() - n += 1 + l + sovPprofextended(uint64(l)) + n += 1 + l + sovProfiles(uint64(l)) } } - if len(m.Mapping) > 0 { - for _, e := range m.Mapping { - l = e.Size() - n += 1 + l + sovPprofextended(uint64(l)) - } + return n +} + +func (m *ResourceProfiles) Size() (n int) { + if m == nil { + return 0 } - if len(m.Location) > 0 { - for _, e := range m.Location { + var l int + _ = l + l = m.Resource.Size() + n += 1 + l + sovProfiles(uint64(l)) + if len(m.ScopeProfiles) > 0 { + for _, e := range m.ScopeProfiles { l = e.Size() - n += 1 + l + sovPprofextended(uint64(l)) + n += 1 + l + sovProfiles(uint64(l)) } } - if len(m.Function) > 0 { - for _, e := range m.Function { - l = e.Size() - n += 1 + l + sovPprofextended(uint64(l)) - } + l = len(m.SchemaUrl) + if l > 0 { + n += 1 + l + sovProfiles(uint64(l)) } - if len(m.StringTable) > 0 { - for _, s := range m.StringTable { - l = len(s) - n += 1 + l + sovPprofextended(uint64(l)) - } + return n +} + +func (m *ScopeProfiles) Size() (n int) { + if m == nil { + return 0 } - if m.DropFrames != 0 { - n += 1 + sovPprofextended(uint64(m.DropFrames)) + var l int + _ = l + l = m.Scope.Size() + n += 1 + l + sovProfiles(uint64(l)) + if len(m.Profiles) > 0 { + for _, e := range m.Profiles { + l = e.Size() + n += 1 + l + sovProfiles(uint64(l)) + } } - if m.KeepFrames != 0 { - n += 1 + sovPprofextended(uint64(m.KeepFrames)) + l = len(m.SchemaUrl) + if l > 0 { + n += 1 + l + sovProfiles(uint64(l)) } - if m.TimeNanos != 0 { - n += 1 + sovPprofextended(uint64(m.TimeNanos)) + return n +} + +func (m *Profile) Size() (n int) { + if m == nil { + return 0 } - if m.DurationNanos != 0 { - n += 1 + sovPprofextended(uint64(m.DurationNanos)) + var l int + _ = l + if len(m.SampleType) > 0 { + for _, e := range m.SampleType { + l = e.Size() + n += 1 + l + sovProfiles(uint64(l)) + } } - l = m.PeriodType.Size() - n += 1 + l + sovPprofextended(uint64(l)) - if m.Period != 0 { - n += 1 + sovPprofextended(uint64(m.Period)) + if len(m.Sample) > 0 { + for _, e := range m.Sample { + l = e.Size() + n += 1 + l + sovProfiles(uint64(l)) + } } - if len(m.Comment) > 0 { - l = 0 - for _, e := range m.Comment { - l += sovPprofextended(uint64(e)) + if len(m.MappingTable) > 0 { + for _, e := range m.MappingTable { + l = e.Size() + n += 1 + l + sovProfiles(uint64(l)) } - n += 1 + sovPprofextended(uint64(l)) + l } - if m.DefaultSampleType != 0 { - n += 1 + sovPprofextended(uint64(m.DefaultSampleType)) + if len(m.LocationTable) > 0 { + for _, e := range m.LocationTable { + l = e.Size() + n += 1 + l + sovProfiles(uint64(l)) + } } if len(m.LocationIndices) > 0 { l = 0 for _, e := range m.LocationIndices { - l += sovPprofextended(uint64(e)) + l += sovProfiles(uint64(e)) + } + n += 1 + sovProfiles(uint64(l)) + l + } + if len(m.FunctionTable) > 0 { + for _, e := range m.FunctionTable { + l = e.Size() + n += 1 + l + sovProfiles(uint64(l)) } - n += 1 + sovPprofextended(uint64(l)) + l } if len(m.AttributeTable) > 0 { for _, e := range m.AttributeTable { l = e.Size() - n += 2 + l + sovPprofextended(uint64(l)) + n += 1 + l + sovProfiles(uint64(l)) } } if len(m.AttributeUnits) > 0 { for _, e := range m.AttributeUnits { l = e.Size() - n += 2 + l + sovPprofextended(uint64(l)) + n += 1 + l + sovProfiles(uint64(l)) } } if len(m.LinkTable) > 0 { for _, e := range m.LinkTable { l = e.Size() - n += 2 + l + sovPprofextended(uint64(l)) + n += 1 + l + sovProfiles(uint64(l)) + } + } + if len(m.StringTable) > 0 { + for _, s := range m.StringTable { + l = len(s) + n += 1 + l + sovProfiles(uint64(l)) + } + } + if m.TimeNanos != 0 { + n += 1 + sovProfiles(uint64(m.TimeNanos)) + } + if m.DurationNanos != 0 { + n += 1 + sovProfiles(uint64(m.DurationNanos)) + } + l = m.PeriodType.Size() + n += 1 + l + sovProfiles(uint64(l)) + if m.Period != 0 { + n += 1 + sovProfiles(uint64(m.Period)) + } + if len(m.CommentStrindices) > 0 { + l = 0 + for _, e := range m.CommentStrindices { + l += sovProfiles(uint64(e)) + } + n += 1 + sovProfiles(uint64(l)) + l + } + if m.DefaultSampleTypeStrindex != 0 { + n += 2 + sovProfiles(uint64(m.DefaultSampleTypeStrindex)) + } + l = m.ProfileId.Size() + n += 2 + l + sovProfiles(uint64(l)) + if len(m.Attributes) > 0 { + for _, e := range m.Attributes { + l = e.Size() + n += 2 + l + sovProfiles(uint64(l)) } } + if m.DroppedAttributesCount != 0 { + n += 2 + sovProfiles(uint64(m.DroppedAttributesCount)) + } + l = len(m.OriginalPayloadFormat) + if l > 0 { + n += 2 + l + sovProfiles(uint64(l)) + } + l = len(m.OriginalPayload) + if l > 0 { + n += 2 + l + sovProfiles(uint64(l)) + } return n } @@ -2182,11 +2593,11 @@ func (m *AttributeUnit) Size() (n int) { } var l int _ = l - if m.AttributeKey != 0 { - n += 1 + sovPprofextended(uint64(m.AttributeKey)) + if m.AttributeKeyStrindex != 0 { + n += 1 + sovProfiles(uint64(m.AttributeKeyStrindex)) } - if m.Unit != 0 { - n += 1 + sovPprofextended(uint64(m.Unit)) + if m.UnitStrindex != 0 { + n += 1 + sovProfiles(uint64(m.UnitStrindex)) } return n } @@ -2198,9 +2609,9 @@ func (m *Link) Size() (n int) { var l int _ = l l = m.TraceId.Size() - n += 1 + l + sovPprofextended(uint64(l)) + n += 1 + l + sovProfiles(uint64(l)) l = m.SpanId.Size() - n += 1 + l + sovPprofextended(uint64(l)) + n += 1 + l + sovProfiles(uint64(l)) return n } @@ -2210,14 +2621,14 @@ func (m *ValueType) Size() (n int) { } var l int _ = l - if m.Type != 0 { - n += 1 + sovPprofextended(uint64(m.Type)) + if m.TypeStrindex != 0 { + n += 1 + sovProfiles(uint64(m.TypeStrindex)) } - if m.Unit != 0 { - n += 1 + sovPprofextended(uint64(m.Unit)) + if m.UnitStrindex != 0 { + n += 1 + sovProfiles(uint64(m.UnitStrindex)) } if m.AggregationTemporality != 0 { - n += 1 + sovPprofextended(uint64(m.AggregationTemporality)) + n += 1 + sovProfiles(uint64(m.AggregationTemporality)) } return n } @@ -2228,72 +2639,65 @@ func (m *Sample) Size() (n int) { } var l int _ = l - if len(m.LocationIndex) > 0 { - l = 0 - for _, e := range m.LocationIndex { - l += sovPprofextended(uint64(e)) - } - n += 1 + sovPprofextended(uint64(l)) + l + if m.LocationsStartIndex != 0 { + n += 1 + sovProfiles(uint64(m.LocationsStartIndex)) + } + if m.LocationsLength != 0 { + n += 1 + sovProfiles(uint64(m.LocationsLength)) } if len(m.Value) > 0 { l = 0 for _, e := range m.Value { - l += sovPprofextended(uint64(e)) + l += sovProfiles(uint64(e)) } - n += 1 + sovPprofextended(uint64(l)) + l + n += 1 + sovProfiles(uint64(l)) + l } - if len(m.Label) > 0 { - for _, e := range m.Label { - l = e.Size() - n += 1 + l + sovPprofextended(uint64(l)) - } - } - if m.LocationsStartIndex != 0 { - n += 1 + sovPprofextended(uint64(m.LocationsStartIndex)) - } - if m.LocationsLength != 0 { - n += 1 + sovPprofextended(uint64(m.LocationsLength)) - } - if m.StacktraceIdIndex != 0 { - n += 1 + sovPprofextended(uint64(m.StacktraceIdIndex)) - } - if len(m.Attributes) > 0 { + if len(m.AttributeIndices) > 0 { l = 0 - for _, e := range m.Attributes { - l += sovPprofextended(uint64(e)) + for _, e := range m.AttributeIndices { + l += sovProfiles(uint64(e)) } - n += 1 + sovPprofextended(uint64(l)) + l + n += 1 + sovProfiles(uint64(l)) + l } - if m.Link != 0 { - n += 1 + sovPprofextended(uint64(m.Link)) + if m.LinkIndex_ != nil { + n += m.LinkIndex_.Size() } if len(m.TimestampsUnixNano) > 0 { l = 0 for _, e := range m.TimestampsUnixNano { - l += sovPprofextended(uint64(e)) + l += sovProfiles(uint64(e)) } - n += 1 + sovPprofextended(uint64(l)) + l + n += 1 + sovProfiles(uint64(l)) + l } return n } +func (m *Sample_LinkIndex) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovProfiles(uint64(m.LinkIndex)) + return n +} func (m *Label) Size() (n int) { if m == nil { return 0 } var l int _ = l - if m.Key != 0 { - n += 1 + sovPprofextended(uint64(m.Key)) + if m.KeyStrindex != 0 { + n += 1 + sovProfiles(uint64(m.KeyStrindex)) } - if m.Str != 0 { - n += 1 + sovPprofextended(uint64(m.Str)) + if m.StrStrindex != 0 { + n += 1 + sovProfiles(uint64(m.StrStrindex)) } if m.Num != 0 { - n += 1 + sovPprofextended(uint64(m.Num)) + n += 1 + sovProfiles(uint64(m.Num)) } - if m.NumUnit != 0 { - n += 1 + sovPprofextended(uint64(m.NumUnit)) + if m.NumUnitStrindex != 0 { + n += 1 + sovProfiles(uint64(m.NumUnitStrindex)) } return n } @@ -2304,23 +2708,24 @@ func (m *Mapping) Size() (n int) { } var l int _ = l - if m.Id != 0 { - n += 1 + sovPprofextended(uint64(m.Id)) - } if m.MemoryStart != 0 { - n += 1 + sovPprofextended(uint64(m.MemoryStart)) + n += 1 + sovProfiles(uint64(m.MemoryStart)) } if m.MemoryLimit != 0 { - n += 1 + sovPprofextended(uint64(m.MemoryLimit)) + n += 1 + sovProfiles(uint64(m.MemoryLimit)) } if m.FileOffset != 0 { - n += 1 + sovPprofextended(uint64(m.FileOffset)) + n += 1 + sovProfiles(uint64(m.FileOffset)) } - if m.Filename != 0 { - n += 1 + sovPprofextended(uint64(m.Filename)) + if m.FilenameStrindex != 0 { + n += 1 + sovProfiles(uint64(m.FilenameStrindex)) } - if m.BuildId != 0 { - n += 1 + sovPprofextended(uint64(m.BuildId)) + if len(m.AttributeIndices) > 0 { + l = 0 + for _, e := range m.AttributeIndices { + l += sovProfiles(uint64(e)) + } + n += 1 + sovProfiles(uint64(l)) + l } if m.HasFunctions { n += 2 @@ -2334,16 +2739,6 @@ func (m *Mapping) Size() (n int) { if m.HasInlineFrames { n += 2 } - if m.BuildIdKind != 0 { - n += 1 + sovPprofextended(uint64(m.BuildIdKind)) - } - if len(m.Attributes) > 0 { - l = 0 - for _, e := range m.Attributes { - l += sovPprofextended(uint64(e)) - } - n += 1 + sovPprofextended(uint64(l)) + l - } return n } @@ -2353,37 +2748,40 @@ func (m *Location) Size() (n int) { } var l int _ = l - if m.Id != 0 { - n += 1 + sovPprofextended(uint64(m.Id)) - } - if m.MappingIndex != 0 { - n += 1 + sovPprofextended(uint64(m.MappingIndex)) + if m.MappingIndex_ != nil { + n += m.MappingIndex_.Size() } if m.Address != 0 { - n += 1 + sovPprofextended(uint64(m.Address)) + n += 1 + sovProfiles(uint64(m.Address)) } if len(m.Line) > 0 { for _, e := range m.Line { l = e.Size() - n += 1 + l + sovPprofextended(uint64(l)) + n += 1 + l + sovProfiles(uint64(l)) } } if m.IsFolded { n += 2 } - if m.TypeIndex != 0 { - n += 1 + sovPprofextended(uint64(m.TypeIndex)) - } - if len(m.Attributes) > 0 { + if len(m.AttributeIndices) > 0 { l = 0 - for _, e := range m.Attributes { - l += sovPprofextended(uint64(e)) + for _, e := range m.AttributeIndices { + l += sovProfiles(uint64(e)) } - n += 1 + sovPprofextended(uint64(l)) + l + n += 1 + sovProfiles(uint64(l)) + l } return n } +func (m *Location_MappingIndex) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovProfiles(uint64(m.MappingIndex)) + return n +} func (m *Line) Size() (n int) { if m == nil { return 0 @@ -2391,13 +2789,13 @@ func (m *Line) Size() (n int) { var l int _ = l if m.FunctionIndex != 0 { - n += 1 + sovPprofextended(uint64(m.FunctionIndex)) + n += 1 + sovProfiles(uint64(m.FunctionIndex)) } if m.Line != 0 { - n += 1 + sovPprofextended(uint64(m.Line)) + n += 1 + sovProfiles(uint64(m.Line)) } if m.Column != 0 { - n += 1 + sovPprofextended(uint64(m.Column)) + n += 1 + sovProfiles(uint64(m.Column)) } return n } @@ -2408,31 +2806,28 @@ func (m *Function) Size() (n int) { } var l int _ = l - if m.Id != 0 { - n += 1 + sovPprofextended(uint64(m.Id)) - } - if m.Name != 0 { - n += 1 + sovPprofextended(uint64(m.Name)) + if m.NameStrindex != 0 { + n += 1 + sovProfiles(uint64(m.NameStrindex)) } - if m.SystemName != 0 { - n += 1 + sovPprofextended(uint64(m.SystemName)) + if m.SystemNameStrindex != 0 { + n += 1 + sovProfiles(uint64(m.SystemNameStrindex)) } - if m.Filename != 0 { - n += 1 + sovPprofextended(uint64(m.Filename)) + if m.FilenameStrindex != 0 { + n += 1 + sovProfiles(uint64(m.FilenameStrindex)) } if m.StartLine != 0 { - n += 1 + sovPprofextended(uint64(m.StartLine)) + n += 1 + sovProfiles(uint64(m.StartLine)) } return n } -func sovPprofextended(x uint64) (n int) { +func sovProfiles(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } -func sozPprofextended(x uint64) (n int) { - return sovPprofextended(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +func sozProfiles(x uint64) (n int) { + return sovProfiles(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } -func (m *Profile) Unmarshal(dAtA []byte) error { +func (m *ProfilesData) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2440,7 +2835,7 @@ func (m *Profile) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -2455,20 +2850,20 @@ func (m *Profile) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Profile: wiretype end group for non-group") + return fmt.Errorf("proto: ProfilesData: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Profile: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ProfilesData: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SampleType", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ResourceProfiles", wireType) } var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -2481,28 +2876,78 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.SampleType = append(m.SampleType, &ValueType{}) - if err := m.SampleType[len(m.SampleType)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.ResourceProfiles = append(m.ResourceProfiles, &ResourceProfiles{}) + if err := m.ResourceProfiles[len(m.ResourceProfiles)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + default: + iNdEx = preIndex + skippy, err := skipProfiles(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthProfiles + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ResourceProfiles) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ResourceProfiles: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ResourceProfiles: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sample", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Resource", wireType) } var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -2515,28 +2960,27 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.Sample = append(m.Sample, &Sample{}) - if err := m.Sample[len(m.Sample)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Resource.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 3: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Mapping", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ScopeProfiles", wireType) } var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -2549,62 +2993,110 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.Mapping = append(m.Mapping, &Mapping{}) - if err := m.Mapping[len(m.Mapping)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.ScopeProfiles = append(m.ScopeProfiles, &ScopeProfiles{}) + if err := m.ScopeProfiles[len(m.ScopeProfiles)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 4: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Location", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SchemaUrl", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthPprofextended + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthProfiles } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.Location = append(m.Location, &Location{}) - if err := m.Location[len(m.Location)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.SchemaUrl = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipProfiles(dAtA[iNdEx:]) + if err != nil { return err } - iNdEx = postIndex - case 5: + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthProfiles + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ScopeProfiles) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ScopeProfiles: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ScopeProfiles: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Function", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Scope", wireType) } var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -2617,136 +3109,211 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.Function = append(m.Function, &Function{}) - if err := m.Function[len(m.Function)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Scope.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 6: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field StringTable", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Profiles", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthPprofextended + if msglen < 0 { + return ErrInvalidLengthProfiles } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.StringTable = append(m.StringTable, string(dAtA[iNdEx:postIndex])) + m.Profiles = append(m.Profiles, &Profile{}) + if err := m.Profiles[len(m.Profiles)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DropFrames", wireType) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SchemaUrl", wireType) } - m.DropFrames = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.DropFrames |= int64(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - case 8: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field KeepFrames", wireType) - } - m.KeepFrames = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.KeepFrames |= int64(b&0x7F) << shift - if b < 0x80 { - break - } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthProfiles } - case 9: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TimeNanos", wireType) + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthProfiles } - m.TimeNanos = 0 + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SchemaUrl = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipProfiles(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthProfiles + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Profile) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Profile: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Profile: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SampleType", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.TimeNanos |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 10: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DurationNanos", wireType) + if msglen < 0 { + return ErrInvalidLengthProfiles } - m.DurationNanos = 0 + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SampleType = append(m.SampleType, &ValueType{}) + if err := m.SampleType[len(m.SampleType)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Sample", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.DurationNanos |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 11: + if msglen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Sample = append(m.Sample, &Sample{}) + if err := m.Sample[len(m.Sample)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PeriodType", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MappingTable", wireType) } var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -2759,61 +3326,77 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.PeriodType.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.MappingTable = append(m.MappingTable, &Mapping{}) + if err := m.MappingTable[len(m.MappingTable)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 12: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Period", wireType) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LocationTable", wireType) } - m.Period = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Period |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 13: + if msglen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LocationTable = append(m.LocationTable, &Location{}) + if err := m.LocationTable[len(m.LocationTable)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: if wireType == 0 { - var v int64 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - v |= int64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.Comment = append(m.Comment, v) + m.LocationIndices = append(m.LocationIndices, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -2826,11 +3409,11 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } if packedLen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + packedLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -2843,72 +3426,311 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.Comment) == 0 { - m.Comment = make([]int64, 0, elementCount) + if elementCount != 0 && len(m.LocationIndices) == 0 { + m.LocationIndices = make([]int32, 0, elementCount) } for iNdEx < postIndex { - var v int64 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - v |= int64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.Comment = append(m.Comment, v) + m.LocationIndices = append(m.LocationIndices, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field Comment", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LocationIndices", wireType) + } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FunctionTable", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProfiles } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FunctionTable = append(m.FunctionTable, &Function{}) + if err := m.FunctionTable[len(m.FunctionTable)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttributeTable", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttributeTable = append(m.AttributeTable, v11.KeyValue{}) + if err := m.AttributeTable[len(m.AttributeTable)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AttributeUnits", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AttributeUnits = append(m.AttributeUnits, &AttributeUnit{}) + if err := m.AttributeUnits[len(m.AttributeUnits)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LinkTable", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LinkTable = append(m.LinkTable, &Link{}) + if err := m.LinkTable[len(m.LinkTable)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StringTable", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StringTable = append(m.StringTable, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TimeNanos", wireType) + } + m.TimeNanos = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TimeNanos |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DurationNanos", wireType) + } + m.DurationNanos = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DurationNanos |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PeriodType", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.PeriodType.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 14: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DefaultSampleType", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Period", wireType) } - m.DefaultSampleType = 0 + m.Period = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.DefaultSampleType |= int64(b&0x7F) << shift + m.Period |= int64(b&0x7F) << shift if b < 0x80 { break } } case 15: if wireType == 0 { - var v int64 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - v |= int64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.LocationIndices = append(m.LocationIndices, v) + m.CommentStrindices = append(m.CommentStrindices, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -2921,11 +3743,11 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } if packedLen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + packedLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -2938,72 +3760,90 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.LocationIndices) == 0 { - m.LocationIndices = make([]int64, 0, elementCount) + if elementCount != 0 && len(m.CommentStrindices) == 0 { + m.CommentStrindices = make([]int32, 0, elementCount) } for iNdEx < postIndex { - var v int64 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - v |= int64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.LocationIndices = append(m.LocationIndices, v) + m.CommentStrindices = append(m.CommentStrindices, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field LocationIndices", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CommentStrindices", wireType) } case 16: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DefaultSampleTypeStrindex", wireType) + } + m.DefaultSampleTypeStrindex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DefaultSampleTypeStrindex |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 17: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field AttributeTable", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ProfileId", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthPprofextended + if byteLen < 0 { + return ErrInvalidLengthProfiles } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.AttributeTable = append(m.AttributeTable, v1.KeyValue{}) - if err := m.AttributeTable[len(m.AttributeTable)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ProfileId.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 17: + case 18: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field AttributeUnits", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Attributes", wireType) } var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3016,62 +3856,113 @@ func (m *Profile) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.AttributeUnits = append(m.AttributeUnits, &AttributeUnit{}) - if err := m.AttributeUnits[len(m.AttributeUnits)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Attributes = append(m.Attributes, v11.KeyValue{}) + if err := m.Attributes[len(m.Attributes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 18: + case 19: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DroppedAttributesCount", wireType) + } + m.DroppedAttributesCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DroppedAttributesCount |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 20: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field LinkTable", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OriginalPayloadFormat", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthPprofextended + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthProfiles } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF } - m.LinkTable = append(m.LinkTable, &Link{}) - if err := m.LinkTable[len(m.LinkTable)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.OriginalPayloadFormat = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 21: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OriginalPayload", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OriginalPayload = append(m.OriginalPayload[:0], dAtA[iNdEx:postIndex]...) + if m.OriginalPayload == nil { + m.OriginalPayload = []byte{} } iNdEx = postIndex default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -3093,7 +3984,7 @@ func (m *AttributeUnit) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3116,50 +4007,50 @@ func (m *AttributeUnit) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AttributeKey", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field AttributeKeyStrindex", wireType) } - m.AttributeKey = 0 + m.AttributeKeyStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.AttributeKey |= int64(b&0x7F) << shift + m.AttributeKeyStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Unit", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field UnitStrindex", wireType) } - m.Unit = 0 + m.UnitStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Unit |= int64(b&0x7F) << shift + m.UnitStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -3181,7 +4072,7 @@ func (m *Link) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3209,7 +4100,7 @@ func (m *Link) Unmarshal(dAtA []byte) error { var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3222,11 +4113,11 @@ func (m *Link) Unmarshal(dAtA []byte) error { } } if byteLen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + byteLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -3242,7 +4133,7 @@ func (m *Link) Unmarshal(dAtA []byte) error { var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3255,11 +4146,11 @@ func (m *Link) Unmarshal(dAtA []byte) error { } } if byteLen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + byteLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -3270,12 +4161,12 @@ func (m *Link) Unmarshal(dAtA []byte) error { iNdEx = postIndex default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -3297,7 +4188,7 @@ func (m *ValueType) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3320,38 +4211,38 @@ func (m *ValueType) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TypeStrindex", wireType) } - m.Type = 0 + m.TypeStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Type |= int64(b&0x7F) << shift + m.TypeStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Unit", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field UnitStrindex", wireType) } - m.Unit = 0 + m.UnitStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Unit |= int64(b&0x7F) << shift + m.UnitStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -3363,7 +4254,7 @@ func (m *ValueType) Unmarshal(dAtA []byte) error { m.AggregationTemporality = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3377,12 +4268,12 @@ func (m *ValueType) Unmarshal(dAtA []byte) error { } default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -3404,7 +4295,7 @@ func (m *Sample) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3426,87 +4317,49 @@ func (m *Sample) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType == 0 { - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LocationsStartIndex", wireType) + } + m.LocationsStartIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles } - m.LocationIndex = append(m.LocationIndex, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } + if iNdEx >= l { + return io.ErrUnexpectedEOF } - if packedLen < 0 { - return ErrInvalidLengthPprofextended + b := dAtA[iNdEx] + iNdEx++ + m.LocationsStartIndex |= int32(b&0x7F) << shift + if b < 0x80 { + break } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthPprofextended + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LocationsLength", wireType) + } + m.LocationsLength = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles } - if postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.LocationIndex) == 0 { - m.LocationIndex = make([]uint64, 0, elementCount) - } - for iNdEx < postIndex { - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.LocationIndex = append(m.LocationIndex, v) + b := dAtA[iNdEx] + iNdEx++ + m.LocationsLength |= int32(b&0x7F) << shift + if b < 0x80 { + break } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field LocationIndex", wireType) } - case 2: + case 3: if wireType == 0 { var v int64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3523,7 +4376,7 @@ func (m *Sample) Unmarshal(dAtA []byte) error { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3536,11 +4389,11 @@ func (m *Sample) Unmarshal(dAtA []byte) error { } } if packedLen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + packedLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -3560,137 +4413,46 @@ func (m *Sample) Unmarshal(dAtA []byte) error { var v int64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] - iNdEx++ - v |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Value = append(m.Value, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) - } - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Label", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthPprofextended - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthPprofextended - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Label = append(m.Label, &Label{}) - if err := m.Label[len(m.Label)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LocationsStartIndex", wireType) - } - m.LocationsStartIndex = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.LocationsStartIndex |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 8: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LocationsLength", wireType) - } - m.LocationsLength = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.LocationsLength |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 9: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StacktraceIdIndex", wireType) - } - m.StacktraceIdIndex = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.StacktraceIdIndex |= uint32(b&0x7F) << shift - if b < 0x80 { - break + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Value = append(m.Value, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) } - case 10: + case 4: if wireType == 0 { - var v uint64 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - v |= uint64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.Attributes = append(m.Attributes, v) + m.AttributeIndices = append(m.AttributeIndices, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3703,11 +4465,11 @@ func (m *Sample) Unmarshal(dAtA []byte) error { } } if packedLen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + packedLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -3720,55 +4482,56 @@ func (m *Sample) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.Attributes) == 0 { - m.Attributes = make([]uint64, 0, elementCount) + if elementCount != 0 && len(m.AttributeIndices) == 0 { + m.AttributeIndices = make([]int32, 0, elementCount) } for iNdEx < postIndex { - var v uint64 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - v |= uint64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.Attributes = append(m.Attributes, v) + m.AttributeIndices = append(m.AttributeIndices, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field Attributes", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field AttributeIndices", wireType) } - case 12: + case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Link", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LinkIndex", wireType) } - m.Link = 0 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Link |= uint64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - case 13: + m.LinkIndex_ = &Sample_LinkIndex{v} + case 6: if wireType == 0 { var v uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3785,7 +4548,7 @@ func (m *Sample) Unmarshal(dAtA []byte) error { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3798,11 +4561,11 @@ func (m *Sample) Unmarshal(dAtA []byte) error { } } if packedLen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + packedLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -3822,7 +4585,7 @@ func (m *Sample) Unmarshal(dAtA []byte) error { var v uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3841,12 +4604,12 @@ func (m *Sample) Unmarshal(dAtA []byte) error { } default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -3868,7 +4631,7 @@ func (m *Label) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3891,38 +4654,38 @@ func (m *Label) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field KeyStrindex", wireType) } - m.Key = 0 + m.KeyStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Key |= int64(b&0x7F) << shift + m.KeyStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Str", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StrStrindex", wireType) } - m.Str = 0 + m.StrStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Str |= int64(b&0x7F) << shift + m.StrStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -3934,7 +4697,7 @@ func (m *Label) Unmarshal(dAtA []byte) error { m.Num = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -3948,31 +4711,31 @@ func (m *Label) Unmarshal(dAtA []byte) error { } case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NumUnit", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field NumUnitStrindex", wireType) } - m.NumUnit = 0 + m.NumUnitStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.NumUnit |= int64(b&0x7F) << shift + m.NumUnitStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -3994,7 +4757,7 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4016,32 +4779,13 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) - } - m.Id = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Id |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field MemoryStart", wireType) } m.MemoryStart = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4053,14 +4797,14 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { break } } - case 3: + case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field MemoryLimit", wireType) } m.MemoryLimit = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4072,14 +4816,14 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { break } } - case 4: + case 3: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field FileOffset", wireType) } m.FileOffset = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4091,52 +4835,109 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { break } } - case 5: + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Filename", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field FilenameStrindex", wireType) } - m.Filename = 0 + m.FilenameStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Filename |= int64(b&0x7F) << shift + m.FilenameStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field BuildId", wireType) - } - m.BuildId = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended + case 5: + if wireType == 0 { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + m.AttributeIndices = append(m.AttributeIndices, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthProfiles + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthProfiles + } + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - m.BuildId |= int64(b&0x7F) << shift - if b < 0x80 { - break + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.AttributeIndices) == 0 { + m.AttributeIndices = make([]int32, 0, elementCount) + } + for iNdEx < postIndex { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProfiles + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AttributeIndices = append(m.AttributeIndices, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field AttributeIndices", wireType) } - case 7: + case 6: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field HasFunctions", wireType) } var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4149,14 +4950,14 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { } } m.HasFunctions = bool(v != 0) - case 8: + case 7: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field HasFilenames", wireType) } var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4169,14 +4970,14 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { } } m.HasFilenames = bool(v != 0) - case 9: + case 8: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field HasLineNumbers", wireType) } var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4189,14 +4990,14 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { } } m.HasLineNumbers = bool(v != 0) - case 10: + case 9: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field HasInlineFrames", wireType) } var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4209,109 +5010,14 @@ func (m *Mapping) Unmarshal(dAtA []byte) error { } } m.HasInlineFrames = bool(v != 0) - case 11: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field BuildIdKind", wireType) - } - m.BuildIdKind = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.BuildIdKind |= BuildIdKind(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 12: - if wireType == 0 { - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Attributes = append(m.Attributes, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthPprofextended - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthPprofextended - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.Attributes) == 0 { - m.Attributes = make([]uint64, 0, elementCount) - } - for iNdEx < postIndex { - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Attributes = append(m.Attributes, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field Attributes", wireType) - } default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -4333,7 +5039,7 @@ func (m *Location) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4355,51 +5061,33 @@ func (m *Location) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) - } - m.Id = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Id |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field MappingIndex", wireType) } - m.MappingIndex = 0 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.MappingIndex |= uint64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - case 3: + m.MappingIndex_ = &Location_MappingIndex{v} + case 2: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Address", wireType) } m.Address = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4411,14 +5099,14 @@ func (m *Location) Unmarshal(dAtA []byte) error { break } } - case 4: + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Line", wireType) } var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4431,11 +5119,11 @@ func (m *Location) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -4445,14 +5133,14 @@ func (m *Location) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 5: + case 4: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field IsFolded", wireType) } var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4465,48 +5153,29 @@ func (m *Location) Unmarshal(dAtA []byte) error { } } m.IsFolded = bool(v != 0) - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TypeIndex", wireType) - } - m.TypeIndex = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TypeIndex |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 7: + case 5: if wireType == 0 { - var v uint64 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - v |= uint64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.Attributes = append(m.Attributes, v) + m.AttributeIndices = append(m.AttributeIndices, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4519,11 +5188,11 @@ func (m *Location) Unmarshal(dAtA []byte) error { } } if packedLen < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } postIndex := iNdEx + packedLen if postIndex < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if postIndex > l { return io.ErrUnexpectedEOF @@ -4536,38 +5205,38 @@ func (m *Location) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.Attributes) == 0 { - m.Attributes = make([]uint64, 0, elementCount) + if elementCount != 0 && len(m.AttributeIndices) == 0 { + m.AttributeIndices = make([]int32, 0, elementCount) } for iNdEx < postIndex { - var v uint64 + var v int32 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - v |= uint64(b&0x7F) << shift + v |= int32(b&0x7F) << shift if b < 0x80 { break } } - m.Attributes = append(m.Attributes, v) + m.AttributeIndices = append(m.AttributeIndices, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field Attributes", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field AttributeIndices", wireType) } default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -4589,7 +5258,7 @@ func (m *Line) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4617,14 +5286,14 @@ func (m *Line) Unmarshal(dAtA []byte) error { m.FunctionIndex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.FunctionIndex |= uint64(b&0x7F) << shift + m.FunctionIndex |= int32(b&0x7F) << shift if b < 0x80 { break } @@ -4636,7 +5305,7 @@ func (m *Line) Unmarshal(dAtA []byte) error { m.Line = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4655,7 +5324,7 @@ func (m *Line) Unmarshal(dAtA []byte) error { m.Column = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4669,12 +5338,12 @@ func (m *Line) Unmarshal(dAtA []byte) error { } default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -4696,7 +5365,7 @@ func (m *Function) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4719,88 +5388,69 @@ func (m *Function) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field NameStrindex", wireType) } - m.Id = 0 + m.NameStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Id |= uint64(b&0x7F) << shift + m.NameStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SystemNameStrindex", wireType) } - m.Name = 0 + m.SystemNameStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.Name |= int64(b&0x7F) << shift + m.SystemNameStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SystemName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field FilenameStrindex", wireType) } - m.SystemName = 0 + m.FilenameStrindex = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ - m.SystemName |= int64(b&0x7F) << shift + m.FilenameStrindex |= int32(b&0x7F) << shift if b < 0x80 { break } } case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Filename", wireType) - } - m.Filename = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPprofextended - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Filename |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 5: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field StartLine", wireType) } m.StartLine = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowPprofextended + return ErrIntOverflowProfiles } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -4814,12 +5464,12 @@ func (m *Function) Unmarshal(dAtA []byte) error { } default: iNdEx = preIndex - skippy, err := skipPprofextended(dAtA[iNdEx:]) + skippy, err := skipProfiles(dAtA[iNdEx:]) if err != nil { return err } if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPprofextended + return ErrInvalidLengthProfiles } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -4833,7 +5483,7 @@ func (m *Function) Unmarshal(dAtA []byte) error { } return nil } -func skipPprofextended(dAtA []byte) (n int, err error) { +func skipProfiles(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 depth := 0 @@ -4841,7 +5491,7 @@ func skipPprofextended(dAtA []byte) (n int, err error) { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowPprofextended + return 0, ErrIntOverflowProfiles } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -4858,7 +5508,7 @@ func skipPprofextended(dAtA []byte) (n int, err error) { case 0: for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowPprofextended + return 0, ErrIntOverflowProfiles } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -4874,7 +5524,7 @@ func skipPprofextended(dAtA []byte) (n int, err error) { var length int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowPprofextended + return 0, ErrIntOverflowProfiles } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -4887,14 +5537,14 @@ func skipPprofextended(dAtA []byte) (n int, err error) { } } if length < 0 { - return 0, ErrInvalidLengthPprofextended + return 0, ErrInvalidLengthProfiles } iNdEx += length case 3: depth++ case 4: if depth == 0 { - return 0, ErrUnexpectedEndOfGroupPprofextended + return 0, ErrUnexpectedEndOfGroupProfiles } depth-- case 5: @@ -4903,7 +5553,7 @@ func skipPprofextended(dAtA []byte) (n int, err error) { return 0, fmt.Errorf("proto: illegal wireType %d", wireType) } if iNdEx < 0 { - return 0, ErrInvalidLengthPprofextended + return 0, ErrInvalidLengthProfiles } if depth == 0 { return iNdEx, nil @@ -4913,7 +5563,7 @@ func skipPprofextended(dAtA []byte) (n int, err error) { } var ( - ErrInvalidLengthPprofextended = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowPprofextended = fmt.Errorf("proto: integer overflow") - ErrUnexpectedEndOfGroupPprofextended = fmt.Errorf("proto: unexpected end of group") + ErrInvalidLengthProfiles = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowProfiles = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupProfiles = fmt.Errorf("proto: unexpected end of group") ) diff --git a/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental/profiles.pb.go b/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental/profiles.pb.go deleted file mode 100644 index cd1c215adb441..0000000000000 --- a/vendor/go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental/profiles.pb.go +++ /dev/null @@ -1,1477 +0,0 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: opentelemetry/proto/profiles/v1experimental/profiles.proto - -package v1experimental - -import ( - encoding_binary "encoding/binary" - fmt "fmt" - io "io" - math "math" - math_bits "math/bits" - - _ "github.com/gogo/protobuf/gogoproto" - proto "github.com/gogo/protobuf/proto" - - go_opentelemetry_io_collector_pdata_internal_data "go.opentelemetry.io/collector/pdata/internal/data" - v11 "go.opentelemetry.io/collector/pdata/internal/data/protogen/common/v1" - v1 "go.opentelemetry.io/collector/pdata/internal/data/protogen/resource/v1" -) - -// Reference imports to suppress errors if they are not otherwise used. -var _ = proto.Marshal -var _ = fmt.Errorf -var _ = math.Inf - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the proto package it is being compiled against. -// A compilation error at this line likely means your copy of the -// proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package - -// ProfilesData represents the profiles data that can be stored in persistent storage, -// OR can be embedded by other protocols that transfer OTLP profiles data but do not -// implement the OTLP protocol. -// -// The main difference between this message and collector protocol is that -// in this message there will not be any "control" or "metadata" specific to -// OTLP protocol. -// -// When new fields are added into this message, the OTLP request MUST be updated -// as well. -type ProfilesData struct { - // An array of ResourceProfiles. - // For data coming from a single resource this array will typically contain - // one element. Intermediary nodes that receive data from multiple origins - // typically batch the data before forwarding further and in that case this - // array will contain multiple elements. - ResourceProfiles []*ResourceProfiles `protobuf:"bytes,1,rep,name=resource_profiles,json=resourceProfiles,proto3" json:"resource_profiles,omitempty"` -} - -func (m *ProfilesData) Reset() { *m = ProfilesData{} } -func (m *ProfilesData) String() string { return proto.CompactTextString(m) } -func (*ProfilesData) ProtoMessage() {} -func (*ProfilesData) Descriptor() ([]byte, []int) { - return fileDescriptor_394731f2296acea3, []int{0} -} -func (m *ProfilesData) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *ProfilesData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_ProfilesData.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (m *ProfilesData) XXX_Merge(src proto.Message) { - xxx_messageInfo_ProfilesData.Merge(m, src) -} -func (m *ProfilesData) XXX_Size() int { - return m.Size() -} -func (m *ProfilesData) XXX_DiscardUnknown() { - xxx_messageInfo_ProfilesData.DiscardUnknown(m) -} - -var xxx_messageInfo_ProfilesData proto.InternalMessageInfo - -func (m *ProfilesData) GetResourceProfiles() []*ResourceProfiles { - if m != nil { - return m.ResourceProfiles - } - return nil -} - -// A collection of ScopeProfiles from a Resource. -type ResourceProfiles struct { - // The resource for the profiles in this message. - // If this field is not set then no resource info is known. - Resource v1.Resource `protobuf:"bytes,1,opt,name=resource,proto3" json:"resource"` - // A list of ScopeProfiles that originate from a resource. - ScopeProfiles []*ScopeProfiles `protobuf:"bytes,2,rep,name=scope_profiles,json=scopeProfiles,proto3" json:"scope_profiles,omitempty"` - // The Schema URL, if known. This is the identifier of the Schema that the resource data - // is recorded in. To learn more about Schema URL see - // https://opentelemetry.io/docs/specs/otel/schemas/#schema-url - // This schema_url applies to the data in the "resource" field. It does not apply - // to the data in the "scope_profiles" field which have their own schema_url field. - SchemaUrl string `protobuf:"bytes,3,opt,name=schema_url,json=schemaUrl,proto3" json:"schema_url,omitempty"` -} - -func (m *ResourceProfiles) Reset() { *m = ResourceProfiles{} } -func (m *ResourceProfiles) String() string { return proto.CompactTextString(m) } -func (*ResourceProfiles) ProtoMessage() {} -func (*ResourceProfiles) Descriptor() ([]byte, []int) { - return fileDescriptor_394731f2296acea3, []int{1} -} -func (m *ResourceProfiles) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *ResourceProfiles) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_ResourceProfiles.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (m *ResourceProfiles) XXX_Merge(src proto.Message) { - xxx_messageInfo_ResourceProfiles.Merge(m, src) -} -func (m *ResourceProfiles) XXX_Size() int { - return m.Size() -} -func (m *ResourceProfiles) XXX_DiscardUnknown() { - xxx_messageInfo_ResourceProfiles.DiscardUnknown(m) -} - -var xxx_messageInfo_ResourceProfiles proto.InternalMessageInfo - -func (m *ResourceProfiles) GetResource() v1.Resource { - if m != nil { - return m.Resource - } - return v1.Resource{} -} - -func (m *ResourceProfiles) GetScopeProfiles() []*ScopeProfiles { - if m != nil { - return m.ScopeProfiles - } - return nil -} - -func (m *ResourceProfiles) GetSchemaUrl() string { - if m != nil { - return m.SchemaUrl - } - return "" -} - -// A collection of ProfileContainers produced by an InstrumentationScope. -type ScopeProfiles struct { - // The instrumentation scope information for the profiles in this message. - // Semantically when InstrumentationScope isn't set, it is equivalent with - // an empty instrumentation scope name (unknown). - Scope v11.InstrumentationScope `protobuf:"bytes,1,opt,name=scope,proto3" json:"scope"` - // A list of ProfileContainers that originate from an instrumentation scope. - Profiles []*ProfileContainer `protobuf:"bytes,2,rep,name=profiles,proto3" json:"profiles,omitempty"` - // The Schema URL, if known. This is the identifier of the Schema that the metric data - // is recorded in. To learn more about Schema URL see - // https://opentelemetry.io/docs/specs/otel/schemas/#schema-url - // This schema_url applies to all profiles in the "profiles" field. - SchemaUrl string `protobuf:"bytes,3,opt,name=schema_url,json=schemaUrl,proto3" json:"schema_url,omitempty"` -} - -func (m *ScopeProfiles) Reset() { *m = ScopeProfiles{} } -func (m *ScopeProfiles) String() string { return proto.CompactTextString(m) } -func (*ScopeProfiles) ProtoMessage() {} -func (*ScopeProfiles) Descriptor() ([]byte, []int) { - return fileDescriptor_394731f2296acea3, []int{2} -} -func (m *ScopeProfiles) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *ScopeProfiles) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_ScopeProfiles.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (m *ScopeProfiles) XXX_Merge(src proto.Message) { - xxx_messageInfo_ScopeProfiles.Merge(m, src) -} -func (m *ScopeProfiles) XXX_Size() int { - return m.Size() -} -func (m *ScopeProfiles) XXX_DiscardUnknown() { - xxx_messageInfo_ScopeProfiles.DiscardUnknown(m) -} - -var xxx_messageInfo_ScopeProfiles proto.InternalMessageInfo - -func (m *ScopeProfiles) GetScope() v11.InstrumentationScope { - if m != nil { - return m.Scope - } - return v11.InstrumentationScope{} -} - -func (m *ScopeProfiles) GetProfiles() []*ProfileContainer { - if m != nil { - return m.Profiles - } - return nil -} - -func (m *ScopeProfiles) GetSchemaUrl() string { - if m != nil { - return m.SchemaUrl - } - return "" -} - -// A ProfileContainer represents a single profile. It wraps pprof profile with OpenTelemetry specific metadata. -type ProfileContainer struct { - // A globally unique identifier for a profile. The ID is a 16-byte array. An ID with - // all zeroes is considered invalid. - // - // This field is required. - ProfileId go_opentelemetry_io_collector_pdata_internal_data.ProfileID `protobuf:"bytes,1,opt,name=profile_id,json=profileId,proto3,customtype=go.opentelemetry.io/collector/pdata/internal/data.ProfileID" json:"profile_id"` - // start_time_unix_nano is the start time of the profile. - // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January 1970. - // - // This field is semantically required and it is expected that end_time >= start_time. - StartTimeUnixNano uint64 `protobuf:"fixed64,2,opt,name=start_time_unix_nano,json=startTimeUnixNano,proto3" json:"start_time_unix_nano,omitempty"` - // end_time_unix_nano is the end time of the profile. - // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January 1970. - // - // This field is semantically required and it is expected that end_time >= start_time. - EndTimeUnixNano uint64 `protobuf:"fixed64,3,opt,name=end_time_unix_nano,json=endTimeUnixNano,proto3" json:"end_time_unix_nano,omitempty"` - // attributes is a collection of key/value pairs. Note, global attributes - // like server name can be set using the resource API. Examples of attributes: - // - // "/http/user_agent": "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_14_2) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/71.0.3578.98 Safari/537.36" - // "/http/server_latency": 300 - // "abc.com/myattribute": true - // "abc.com/score": 10.239 - // - // The OpenTelemetry API specification further restricts the allowed value types: - // https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/common/README.md#attribute - // Attribute keys MUST be unique (it is not allowed to have more than one - // attribute with the same key). - Attributes []v11.KeyValue `protobuf:"bytes,4,rep,name=attributes,proto3" json:"attributes"` - // dropped_attributes_count is the number of attributes that were discarded. Attributes - // can be discarded because their keys are too long or because there are too many - // attributes. If this value is 0, then no attributes were dropped. - DroppedAttributesCount uint32 `protobuf:"varint,5,opt,name=dropped_attributes_count,json=droppedAttributesCount,proto3" json:"dropped_attributes_count,omitempty"` - // Specifies format of the original payload. Common values are defined in semantic conventions. [required if original_payload is present] - OriginalPayloadFormat string `protobuf:"bytes,6,opt,name=original_payload_format,json=originalPayloadFormat,proto3" json:"original_payload_format,omitempty"` - // Original payload can be stored in this field. This can be useful for users who want to get the original payload. - // Formats such as JFR are highly extensible and can contain more information than what is defined in this spec. - // Inclusion of original payload should be configurable by the user. Default behavior should be to not include the original payload. - // If the original payload is in pprof format, it SHOULD not be included in this field. - // The field is optional, however if it is present `profile` MUST be present and contain the same profiling information. - OriginalPayload []byte `protobuf:"bytes,7,opt,name=original_payload,json=originalPayload,proto3" json:"original_payload,omitempty"` - // This is a reference to a pprof profile. Required, even when original_payload is present. - Profile Profile `protobuf:"bytes,8,opt,name=profile,proto3" json:"profile"` -} - -func (m *ProfileContainer) Reset() { *m = ProfileContainer{} } -func (m *ProfileContainer) String() string { return proto.CompactTextString(m) } -func (*ProfileContainer) ProtoMessage() {} -func (*ProfileContainer) Descriptor() ([]byte, []int) { - return fileDescriptor_394731f2296acea3, []int{3} -} -func (m *ProfileContainer) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *ProfileContainer) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_ProfileContainer.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (m *ProfileContainer) XXX_Merge(src proto.Message) { - xxx_messageInfo_ProfileContainer.Merge(m, src) -} -func (m *ProfileContainer) XXX_Size() int { - return m.Size() -} -func (m *ProfileContainer) XXX_DiscardUnknown() { - xxx_messageInfo_ProfileContainer.DiscardUnknown(m) -} - -var xxx_messageInfo_ProfileContainer proto.InternalMessageInfo - -func (m *ProfileContainer) GetStartTimeUnixNano() uint64 { - if m != nil { - return m.StartTimeUnixNano - } - return 0 -} - -func (m *ProfileContainer) GetEndTimeUnixNano() uint64 { - if m != nil { - return m.EndTimeUnixNano - } - return 0 -} - -func (m *ProfileContainer) GetAttributes() []v11.KeyValue { - if m != nil { - return m.Attributes - } - return nil -} - -func (m *ProfileContainer) GetDroppedAttributesCount() uint32 { - if m != nil { - return m.DroppedAttributesCount - } - return 0 -} - -func (m *ProfileContainer) GetOriginalPayloadFormat() string { - if m != nil { - return m.OriginalPayloadFormat - } - return "" -} - -func (m *ProfileContainer) GetOriginalPayload() []byte { - if m != nil { - return m.OriginalPayload - } - return nil -} - -func (m *ProfileContainer) GetProfile() Profile { - if m != nil { - return m.Profile - } - return Profile{} -} - -func init() { - proto.RegisterType((*ProfilesData)(nil), "opentelemetry.proto.profiles.v1experimental.ProfilesData") - proto.RegisterType((*ResourceProfiles)(nil), "opentelemetry.proto.profiles.v1experimental.ResourceProfiles") - proto.RegisterType((*ScopeProfiles)(nil), "opentelemetry.proto.profiles.v1experimental.ScopeProfiles") - proto.RegisterType((*ProfileContainer)(nil), "opentelemetry.proto.profiles.v1experimental.ProfileContainer") -} - -func init() { - proto.RegisterFile("opentelemetry/proto/profiles/v1experimental/profiles.proto", fileDescriptor_394731f2296acea3) -} - -var fileDescriptor_394731f2296acea3 = []byte{ - // 671 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0xdf, 0x6e, 0xd3, 0x30, - 0x14, 0xc6, 0xeb, 0xfd, 0x69, 0x3b, 0x6f, 0x65, 0x9d, 0x35, 0x20, 0x9a, 0x44, 0x57, 0xf5, 0x86, - 0x8e, 0x49, 0x89, 0xba, 0x21, 0x84, 0x86, 0x10, 0xa2, 0x1b, 0x48, 0x63, 0x82, 0x55, 0x61, 0x9b, - 0x04, 0x37, 0x91, 0xd7, 0x78, 0xc5, 0x28, 0xb1, 0x23, 0xc7, 0xa9, 0x3a, 0x9e, 0x82, 0x2b, 0x1e, - 0x82, 0x27, 0xd9, 0xe5, 0x2e, 0xd1, 0x90, 0x26, 0xb4, 0xdd, 0xb0, 0xb7, 0x40, 0x71, 0x9c, 0xd0, - 0x46, 0x45, 0x53, 0xb9, 0x89, 0x1c, 0x9f, 0xef, 0xfc, 0xce, 0xf9, 0x7c, 0x9c, 0xc0, 0x2d, 0x1e, - 0x10, 0x26, 0x89, 0x47, 0x7c, 0x22, 0xc5, 0xa9, 0x15, 0x08, 0x2e, 0x79, 0xfc, 0x3c, 0xa1, 0x1e, - 0x09, 0xad, 0x7e, 0x8b, 0x0c, 0x02, 0x22, 0xa8, 0x4f, 0x98, 0xc4, 0x5e, 0xb6, 0x6f, 0x2a, 0x19, - 0x5a, 0x1f, 0xc9, 0x4d, 0x36, 0xcd, 0x4c, 0x33, 0x9a, 0xbb, 0xb2, 0xdc, 0xe3, 0x3d, 0x9e, 0xe0, - 0xe3, 0x55, 0xa2, 0x5e, 0x79, 0x34, 0xae, 0x7c, 0x97, 0xfb, 0x3e, 0x67, 0x56, 0xbf, 0xa5, 0x57, - 0x5a, 0x6b, 0x8e, 0xd3, 0x0a, 0x12, 0xf2, 0x48, 0x74, 0x49, 0xac, 0x4e, 0xd7, 0x5a, 0xff, 0x62, - 0x22, 0x6b, 0x71, 0x80, 0x0c, 0x24, 0x61, 0x2e, 0x71, 0x13, 0x40, 0xe3, 0x0b, 0x5c, 0xe8, 0x68, - 0xf9, 0x0e, 0x96, 0x18, 0x7d, 0x86, 0x4b, 0x69, 0x09, 0x27, 0xe5, 0x18, 0xa0, 0x3e, 0xdd, 0x9c, - 0xdf, 0x78, 0x6e, 0x4e, 0x70, 0x16, 0xa6, 0xad, 0x29, 0x29, 0xdd, 0xae, 0x8a, 0xdc, 0x4e, 0xe3, - 0x06, 0xc0, 0x6a, 0x5e, 0x86, 0xf6, 0x60, 0x39, 0x15, 0x1a, 0xa0, 0x0e, 0x9a, 0xf3, 0x1b, 0x6b, - 0x63, 0xeb, 0x66, 0x07, 0xd1, 0x6f, 0x65, 0xb5, 0xda, 0x33, 0x67, 0x97, 0xab, 0x05, 0x3b, 0x03, - 0x20, 0x0c, 0xef, 0x84, 0x5d, 0x1e, 0x0c, 0x59, 0x99, 0x52, 0x56, 0xb6, 0x26, 0xb2, 0xf2, 0x3e, - 0x46, 0x64, 0x3e, 0x2a, 0xe1, 0xf0, 0x2b, 0x7a, 0x00, 0x61, 0xd8, 0xfd, 0x44, 0x7c, 0xec, 0x44, - 0xc2, 0x33, 0xa6, 0xeb, 0xa0, 0x39, 0x67, 0xcf, 0x25, 0x3b, 0x87, 0xc2, 0x7b, 0x53, 0x2c, 0xff, - 0x2e, 0x55, 0x6f, 0x4a, 0x8d, 0x0b, 0x00, 0x2b, 0x23, 0x1c, 0xb4, 0x0f, 0x67, 0x15, 0x49, 0xbb, - 0xdc, 0x1c, 0xdb, 0x92, 0xbe, 0x1c, 0xfd, 0x96, 0xb9, 0xcb, 0x42, 0x29, 0x22, 0xd5, 0x91, 0xa4, - 0x9c, 0x29, 0x96, 0xf6, 0x9b, 0x70, 0xd0, 0x07, 0x58, 0xce, 0xd9, 0x9c, 0x6c, 0x62, 0xba, 0xb3, - 0x6d, 0xce, 0x24, 0xa6, 0x8c, 0x08, 0x3b, 0xc3, 0xdd, 0x62, 0xb2, 0xf1, 0x6d, 0x06, 0x56, 0xf3, - 0xd9, 0xe8, 0x18, 0x42, 0x9d, 0xef, 0x50, 0x57, 0x99, 0x5c, 0x68, 0x6f, 0xc7, 0xfd, 0x5e, 0x5c, - 0xae, 0x3e, 0xeb, 0xf1, 0x5c, 0x6b, 0x34, 0xfe, 0x24, 0x3c, 0x8f, 0x74, 0x25, 0x17, 0x56, 0xe0, - 0x62, 0x89, 0x2d, 0xca, 0x24, 0x11, 0x0c, 0x7b, 0x56, 0xfc, 0x96, 0x76, 0xb7, 0xbb, 0x63, 0xcf, - 0x69, 0xec, 0xae, 0x8b, 0x2c, 0xb8, 0x1c, 0x4a, 0x2c, 0xa4, 0x23, 0xa9, 0x4f, 0x9c, 0x88, 0xd1, - 0x81, 0xc3, 0x30, 0xe3, 0xc6, 0x54, 0x1d, 0x34, 0x8b, 0xf6, 0x92, 0x8a, 0x1d, 0x50, 0x9f, 0x1c, - 0x32, 0x3a, 0x78, 0x87, 0x19, 0x47, 0xeb, 0x10, 0x11, 0xe6, 0xe6, 0xe5, 0xd3, 0x4a, 0xbe, 0x48, - 0x98, 0x3b, 0x22, 0x7e, 0x0b, 0x21, 0x96, 0x52, 0xd0, 0xe3, 0x48, 0x92, 0xd0, 0x98, 0x51, 0x47, - 0xfa, 0xf0, 0x96, 0x31, 0xed, 0x91, 0xd3, 0x23, 0xec, 0x45, 0xe9, 0x68, 0x86, 0x00, 0xe8, 0x29, - 0x34, 0x5c, 0xc1, 0x83, 0x80, 0xb8, 0xce, 0xdf, 0x5d, 0xa7, 0xcb, 0x23, 0x26, 0x8d, 0xd9, 0x3a, - 0x68, 0x56, 0xec, 0x7b, 0x3a, 0xfe, 0x32, 0x0b, 0x6f, 0xc7, 0x51, 0xf4, 0x04, 0xde, 0xe7, 0x82, - 0xf6, 0x28, 0xc3, 0x9e, 0x13, 0xe0, 0x53, 0x8f, 0x63, 0xd7, 0x39, 0xe1, 0xc2, 0xc7, 0xd2, 0x28, - 0xaa, 0x59, 0xdc, 0x4d, 0xc3, 0x9d, 0x24, 0xfa, 0x5a, 0x05, 0xd1, 0x1a, 0xac, 0xe6, 0xf3, 0x8c, - 0x52, 0x3c, 0x08, 0x7b, 0x31, 0x97, 0x80, 0x0e, 0x60, 0x49, 0x1f, 0xab, 0x51, 0x56, 0xf7, 0xf1, - 0xf1, 0xff, 0xdc, 0x1d, 0xed, 0x3a, 0x45, 0xb5, 0x7f, 0x82, 0xb3, 0xab, 0x1a, 0x38, 0xbf, 0xaa, - 0x81, 0x5f, 0x57, 0x35, 0xf0, 0xf5, 0xba, 0x56, 0x38, 0xbf, 0xae, 0x15, 0x7e, 0x5c, 0xd7, 0x0a, - 0xd0, 0xa4, 0x7c, 0x92, 0x0a, 0xed, 0x4a, 0xfa, 0xe1, 0x74, 0x62, 0x59, 0x07, 0x7c, 0xb4, 0x27, - 0xbe, 0x43, 0xc9, 0xef, 0xb1, 0x47, 0xd8, 0xbf, 0xfe, 0x90, 0xdf, 0xa7, 0xd6, 0xf7, 0x03, 0xc2, - 0x0e, 0x32, 0xa2, 0xaa, 0x95, 0x9a, 0x0b, 0xcd, 0xa3, 0xd6, 0xab, 0x21, 0xf5, 0x71, 0x51, 0xf1, - 0x36, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x73, 0x8f, 0x2e, 0xdd, 0x60, 0x06, 0x00, 0x00, -} - -func (m *ProfilesData) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *ProfilesData) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *ProfilesData) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.ResourceProfiles) > 0 { - for iNdEx := len(m.ResourceProfiles) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.ResourceProfiles[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintProfiles(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - } - return len(dAtA) - i, nil -} - -func (m *ResourceProfiles) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *ResourceProfiles) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *ResourceProfiles) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.SchemaUrl) > 0 { - i -= len(m.SchemaUrl) - copy(dAtA[i:], m.SchemaUrl) - i = encodeVarintProfiles(dAtA, i, uint64(len(m.SchemaUrl))) - i-- - dAtA[i] = 0x1a - } - if len(m.ScopeProfiles) > 0 { - for iNdEx := len(m.ScopeProfiles) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.ScopeProfiles[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintProfiles(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - } - { - size, err := m.Resource.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintProfiles(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - return len(dAtA) - i, nil -} - -func (m *ScopeProfiles) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *ScopeProfiles) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *ScopeProfiles) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.SchemaUrl) > 0 { - i -= len(m.SchemaUrl) - copy(dAtA[i:], m.SchemaUrl) - i = encodeVarintProfiles(dAtA, i, uint64(len(m.SchemaUrl))) - i-- - dAtA[i] = 0x1a - } - if len(m.Profiles) > 0 { - for iNdEx := len(m.Profiles) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Profiles[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintProfiles(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - } - { - size, err := m.Scope.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintProfiles(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - return len(dAtA) - i, nil -} - -func (m *ProfileContainer) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *ProfileContainer) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *ProfileContainer) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - { - size, err := m.Profile.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintProfiles(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x42 - if len(m.OriginalPayload) > 0 { - i -= len(m.OriginalPayload) - copy(dAtA[i:], m.OriginalPayload) - i = encodeVarintProfiles(dAtA, i, uint64(len(m.OriginalPayload))) - i-- - dAtA[i] = 0x3a - } - if len(m.OriginalPayloadFormat) > 0 { - i -= len(m.OriginalPayloadFormat) - copy(dAtA[i:], m.OriginalPayloadFormat) - i = encodeVarintProfiles(dAtA, i, uint64(len(m.OriginalPayloadFormat))) - i-- - dAtA[i] = 0x32 - } - if m.DroppedAttributesCount != 0 { - i = encodeVarintProfiles(dAtA, i, uint64(m.DroppedAttributesCount)) - i-- - dAtA[i] = 0x28 - } - if len(m.Attributes) > 0 { - for iNdEx := len(m.Attributes) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Attributes[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintProfiles(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x22 - } - } - if m.EndTimeUnixNano != 0 { - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.EndTimeUnixNano)) - i-- - dAtA[i] = 0x19 - } - if m.StartTimeUnixNano != 0 { - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.StartTimeUnixNano)) - i-- - dAtA[i] = 0x11 - } - { - size := m.ProfileId.Size() - i -= size - if _, err := m.ProfileId.MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - i = encodeVarintProfiles(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - return len(dAtA) - i, nil -} - -func encodeVarintProfiles(dAtA []byte, offset int, v uint64) int { - offset -= sovProfiles(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ - } - dAtA[offset] = uint8(v) - return base -} -func (m *ProfilesData) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if len(m.ResourceProfiles) > 0 { - for _, e := range m.ResourceProfiles { - l = e.Size() - n += 1 + l + sovProfiles(uint64(l)) - } - } - return n -} - -func (m *ResourceProfiles) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = m.Resource.Size() - n += 1 + l + sovProfiles(uint64(l)) - if len(m.ScopeProfiles) > 0 { - for _, e := range m.ScopeProfiles { - l = e.Size() - n += 1 + l + sovProfiles(uint64(l)) - } - } - l = len(m.SchemaUrl) - if l > 0 { - n += 1 + l + sovProfiles(uint64(l)) - } - return n -} - -func (m *ScopeProfiles) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = m.Scope.Size() - n += 1 + l + sovProfiles(uint64(l)) - if len(m.Profiles) > 0 { - for _, e := range m.Profiles { - l = e.Size() - n += 1 + l + sovProfiles(uint64(l)) - } - } - l = len(m.SchemaUrl) - if l > 0 { - n += 1 + l + sovProfiles(uint64(l)) - } - return n -} - -func (m *ProfileContainer) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = m.ProfileId.Size() - n += 1 + l + sovProfiles(uint64(l)) - if m.StartTimeUnixNano != 0 { - n += 9 - } - if m.EndTimeUnixNano != 0 { - n += 9 - } - if len(m.Attributes) > 0 { - for _, e := range m.Attributes { - l = e.Size() - n += 1 + l + sovProfiles(uint64(l)) - } - } - if m.DroppedAttributesCount != 0 { - n += 1 + sovProfiles(uint64(m.DroppedAttributesCount)) - } - l = len(m.OriginalPayloadFormat) - if l > 0 { - n += 1 + l + sovProfiles(uint64(l)) - } - l = len(m.OriginalPayload) - if l > 0 { - n += 1 + l + sovProfiles(uint64(l)) - } - l = m.Profile.Size() - n += 1 + l + sovProfiles(uint64(l)) - return n -} - -func sovProfiles(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozProfiles(x uint64) (n int) { - return sovProfiles(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} -func (m *ProfilesData) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ProfilesData: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ProfilesData: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResourceProfiles", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ResourceProfiles = append(m.ResourceProfiles, &ResourceProfiles{}) - if err := m.ResourceProfiles[len(m.ResourceProfiles)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipProfiles(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthProfiles - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *ResourceProfiles) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ResourceProfiles: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ResourceProfiles: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Resource", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.Resource.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ScopeProfiles", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ScopeProfiles = append(m.ScopeProfiles, &ScopeProfiles{}) - if err := m.ScopeProfiles[len(m.ScopeProfiles)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SchemaUrl", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.SchemaUrl = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipProfiles(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthProfiles - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *ScopeProfiles) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ScopeProfiles: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ScopeProfiles: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Scope", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.Scope.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Profiles", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Profiles = append(m.Profiles, &ProfileContainer{}) - if err := m.Profiles[len(m.Profiles)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SchemaUrl", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.SchemaUrl = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipProfiles(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthProfiles - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *ProfileContainer) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: ProfileContainer: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ProfileContainer: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ProfileId", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + byteLen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.ProfileId.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field StartTimeUnixNano", wireType) - } - m.StartTimeUnixNano = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.StartTimeUnixNano = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 3: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field EndTimeUnixNano", wireType) - } - m.EndTimeUnixNano = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.EndTimeUnixNano = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Attributes", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Attributes = append(m.Attributes, v11.KeyValue{}) - if err := m.Attributes[len(m.Attributes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DroppedAttributesCount", wireType) - } - m.DroppedAttributesCount = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.DroppedAttributesCount |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field OriginalPayloadFormat", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.OriginalPayloadFormat = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 7: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field OriginalPayload", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + byteLen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.OriginalPayload = append(m.OriginalPayload[:0], dAtA[iNdEx:postIndex]...) - if m.OriginalPayload == nil { - m.OriginalPayload = []byte{} - } - iNdEx = postIndex - case 8: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Profile", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProfiles - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthProfiles - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthProfiles - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.Profile.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipProfiles(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthProfiles - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func skipProfiles(dAtA []byte) (n int, err error) { - l := len(dAtA) - iNdEx := 0 - depth := 0 - for iNdEx < l { - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowProfiles - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - wireType := int(wire & 0x7) - switch wireType { - case 0: - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowProfiles - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - iNdEx++ - if dAtA[iNdEx-1] < 0x80 { - break - } - } - case 1: - iNdEx += 8 - case 2: - var length int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowProfiles - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - length |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if length < 0 { - return 0, ErrInvalidLengthProfiles - } - iNdEx += length - case 3: - depth++ - case 4: - if depth == 0 { - return 0, ErrUnexpectedEndOfGroupProfiles - } - depth-- - case 5: - iNdEx += 4 - default: - return 0, fmt.Errorf("proto: illegal wireType %d", wireType) - } - if iNdEx < 0 { - return 0, ErrInvalidLengthProfiles - } - if depth == 0 { - return iNdEx, nil - } - } - return 0, io.ErrUnexpectedEOF -} - -var ( - ErrInvalidLengthProfiles = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowProfiles = fmt.Errorf("proto: integer overflow") - ErrUnexpectedEndOfGroupProfiles = fmt.Errorf("proto: unexpected end of group") -) diff --git a/vendor/go.opentelemetry.io/collector/pdata/internal/generated_wrapper_int32slice.go b/vendor/go.opentelemetry.io/collector/pdata/internal/generated_wrapper_int32slice.go new file mode 100644 index 0000000000000..31f642d75bd58 --- /dev/null +++ b/vendor/go.opentelemetry.io/collector/pdata/internal/generated_wrapper_int32slice.go @@ -0,0 +1,34 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +// Code generated by "pdata/internal/cmd/pdatagen/main.go". DO NOT EDIT. +// To regenerate this file run "make genpdata". + +package internal + +type Int32Slice struct { + orig *[]int32 + state *State +} + +func GetOrigInt32Slice(ms Int32Slice) *[]int32 { + return ms.orig +} + +func GetInt32SliceState(ms Int32Slice) *State { + return ms.state +} + +func NewInt32Slice(orig *[]int32, state *State) Int32Slice { + return Int32Slice{orig: orig, state: state} +} + +func FillTestInt32Slice(tv Int32Slice) { +} + +func GenerateTestInt32Slice() Int32Slice { + state := StateMutable + var orig []int32 = nil + + return Int32Slice{&orig, &state} +} diff --git a/vendor/go.opentelemetry.io/collector/pdata/internal/generated_wrapper_intslice.go b/vendor/go.opentelemetry.io/collector/pdata/internal/generated_wrapper_intslice.go new file mode 100644 index 0000000000000..5f3fe569ba5f4 --- /dev/null +++ b/vendor/go.opentelemetry.io/collector/pdata/internal/generated_wrapper_intslice.go @@ -0,0 +1,34 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +// Code generated by "pdata/internal/cmd/pdatagen/main.go". DO NOT EDIT. +// To regenerate this file run "make genpdata". + +package internal + +type IntSlice struct { + orig *[]int + state *State +} + +func GetOrigIntSlice(ms IntSlice) *[]int { + return ms.orig +} + +func GetIntSliceState(ms IntSlice) *State { + return ms.state +} + +func NewIntSlice(orig *[]int, state *State) IntSlice { + return IntSlice{orig: orig, state: state} +} + +func FillTestIntSlice(tv IntSlice) { +} + +func GenerateTestIntSlice() IntSlice { + state := StateMutable + var orig []int = nil + + return IntSlice{&orig, &state} +} diff --git a/vendor/go.opentelemetry.io/collector/pdata/internal/otlp/profiles.go b/vendor/go.opentelemetry.io/collector/pdata/internal/otlp/profiles.go index d134ccf9c0630..59c23cc672bbe 100644 --- a/vendor/go.opentelemetry.io/collector/pdata/internal/otlp/profiles.go +++ b/vendor/go.opentelemetry.io/collector/pdata/internal/otlp/profiles.go @@ -4,7 +4,7 @@ package otlp // import "go.opentelemetry.io/collector/pdata/internal/otlp" import ( - otlpprofiles "go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental" + otlpprofiles "go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1development" ) // MigrateProfiles implements any translation needed due to deprecation in OTLP profiles protocol. diff --git a/vendor/go.opentelemetry.io/collector/pdata/internal/wrapper_profiles.go b/vendor/go.opentelemetry.io/collector/pdata/internal/wrapper_profiles.go index 564c894586291..2230b079c36d6 100644 --- a/vendor/go.opentelemetry.io/collector/pdata/internal/wrapper_profiles.go +++ b/vendor/go.opentelemetry.io/collector/pdata/internal/wrapper_profiles.go @@ -4,8 +4,8 @@ package internal // import "go.opentelemetry.io/collector/pdata/internal" import ( - otlpcollectorprofile "go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1experimental" - otlpprofile "go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental" + otlpcollectorprofile "go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1development" + otlpprofile "go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1development" ) type Profiles struct { diff --git a/vendor/go.opentelemetry.io/collector/pdata/pcommon/generated_int32slice.go b/vendor/go.opentelemetry.io/collector/pdata/pcommon/generated_int32slice.go new file mode 100644 index 0000000000000..35a40bd079cf1 --- /dev/null +++ b/vendor/go.opentelemetry.io/collector/pdata/pcommon/generated_int32slice.go @@ -0,0 +1,108 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +// Code generated by "pdata/internal/cmd/pdatagen/main.go". DO NOT EDIT. +// To regenerate this file run "make genpdata". + +package pcommon + +import ( + "go.opentelemetry.io/collector/pdata/internal" +) + +// Int32Slice represents a []int32 slice. +// The instance of Int32Slice can be assigned to multiple objects since it's immutable. +// +// Must use NewInt32Slice function to create new instances. +// Important: zero-initialized instance is not valid for use. +type Int32Slice internal.Int32Slice + +func (ms Int32Slice) getOrig() *[]int32 { + return internal.GetOrigInt32Slice(internal.Int32Slice(ms)) +} + +func (ms Int32Slice) getState() *internal.State { + return internal.GetInt32SliceState(internal.Int32Slice(ms)) +} + +// NewInt32Slice creates a new empty Int32Slice. +func NewInt32Slice() Int32Slice { + orig := []int32(nil) + state := internal.StateMutable + return Int32Slice(internal.NewInt32Slice(&orig, &state)) +} + +// AsRaw returns a copy of the []int32 slice. +func (ms Int32Slice) AsRaw() []int32 { + return copyInt32Slice(nil, *ms.getOrig()) +} + +// FromRaw copies raw []int32 into the slice Int32Slice. +func (ms Int32Slice) FromRaw(val []int32) { + ms.getState().AssertMutable() + *ms.getOrig() = copyInt32Slice(*ms.getOrig(), val) +} + +// Len returns length of the []int32 slice value. +// Equivalent of len(int32Slice). +func (ms Int32Slice) Len() int { + return len(*ms.getOrig()) +} + +// At returns an item from particular index. +// Equivalent of int32Slice[i]. +func (ms Int32Slice) At(i int) int32 { + return (*ms.getOrig())[i] +} + +// SetAt sets int32 item at particular index. +// Equivalent of int32Slice[i] = val +func (ms Int32Slice) SetAt(i int, val int32) { + ms.getState().AssertMutable() + (*ms.getOrig())[i] = val +} + +// EnsureCapacity ensures Int32Slice has at least the specified capacity. +// 1. If the newCap <= cap, then is no change in capacity. +// 2. If the newCap > cap, then the slice capacity will be expanded to the provided value which will be equivalent of: +// buf := make([]int32, len(int32Slice), newCap) +// copy(buf, int32Slice) +// int32Slice = buf +func (ms Int32Slice) EnsureCapacity(newCap int) { + ms.getState().AssertMutable() + oldCap := cap(*ms.getOrig()) + if newCap <= oldCap { + return + } + + newOrig := make([]int32, len(*ms.getOrig()), newCap) + copy(newOrig, *ms.getOrig()) + *ms.getOrig() = newOrig +} + +// Append appends extra elements to Int32Slice. +// Equivalent of int32Slice = append(int32Slice, elms...) +func (ms Int32Slice) Append(elms ...int32) { + ms.getState().AssertMutable() + *ms.getOrig() = append(*ms.getOrig(), elms...) +} + +// MoveTo moves all elements from the current slice overriding the destination and +// resetting the current instance to its zero value. +func (ms Int32Slice) MoveTo(dest Int32Slice) { + ms.getState().AssertMutable() + dest.getState().AssertMutable() + *dest.getOrig() = *ms.getOrig() + *ms.getOrig() = nil +} + +// CopyTo copies all elements from the current slice overriding the destination. +func (ms Int32Slice) CopyTo(dest Int32Slice) { + dest.getState().AssertMutable() + *dest.getOrig() = copyInt32Slice(*dest.getOrig(), *ms.getOrig()) +} + +func copyInt32Slice(dst, src []int32) []int32 { + dst = dst[:0] + return append(dst, src...) +} diff --git a/vendor/go.opentelemetry.io/collector/pdata/pcommon/generated_intslice.go b/vendor/go.opentelemetry.io/collector/pdata/pcommon/generated_intslice.go new file mode 100644 index 0000000000000..1a72889d55409 --- /dev/null +++ b/vendor/go.opentelemetry.io/collector/pdata/pcommon/generated_intslice.go @@ -0,0 +1,108 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +// Code generated by "pdata/internal/cmd/pdatagen/main.go". DO NOT EDIT. +// To regenerate this file run "make genpdata". + +package pcommon + +import ( + "go.opentelemetry.io/collector/pdata/internal" +) + +// IntSlice represents a []int slice. +// The instance of IntSlice can be assigned to multiple objects since it's immutable. +// +// Must use NewIntSlice function to create new instances. +// Important: zero-initialized instance is not valid for use. +type IntSlice internal.IntSlice + +func (ms IntSlice) getOrig() *[]int { + return internal.GetOrigIntSlice(internal.IntSlice(ms)) +} + +func (ms IntSlice) getState() *internal.State { + return internal.GetIntSliceState(internal.IntSlice(ms)) +} + +// NewIntSlice creates a new empty IntSlice. +func NewIntSlice() IntSlice { + orig := []int(nil) + state := internal.StateMutable + return IntSlice(internal.NewIntSlice(&orig, &state)) +} + +// AsRaw returns a copy of the []int slice. +func (ms IntSlice) AsRaw() []int { + return copyIntSlice(nil, *ms.getOrig()) +} + +// FromRaw copies raw []int into the slice IntSlice. +func (ms IntSlice) FromRaw(val []int) { + ms.getState().AssertMutable() + *ms.getOrig() = copyIntSlice(*ms.getOrig(), val) +} + +// Len returns length of the []int slice value. +// Equivalent of len(intSlice). +func (ms IntSlice) Len() int { + return len(*ms.getOrig()) +} + +// At returns an item from particular index. +// Equivalent of intSlice[i]. +func (ms IntSlice) At(i int) int { + return (*ms.getOrig())[i] +} + +// SetAt sets int item at particular index. +// Equivalent of intSlice[i] = val +func (ms IntSlice) SetAt(i int, val int) { + ms.getState().AssertMutable() + (*ms.getOrig())[i] = val +} + +// EnsureCapacity ensures IntSlice has at least the specified capacity. +// 1. If the newCap <= cap, then is no change in capacity. +// 2. If the newCap > cap, then the slice capacity will be expanded to the provided value which will be equivalent of: +// buf := make([]int, len(intSlice), newCap) +// copy(buf, intSlice) +// intSlice = buf +func (ms IntSlice) EnsureCapacity(newCap int) { + ms.getState().AssertMutable() + oldCap := cap(*ms.getOrig()) + if newCap <= oldCap { + return + } + + newOrig := make([]int, len(*ms.getOrig()), newCap) + copy(newOrig, *ms.getOrig()) + *ms.getOrig() = newOrig +} + +// Append appends extra elements to IntSlice. +// Equivalent of intSlice = append(intSlice, elms...) +func (ms IntSlice) Append(elms ...int) { + ms.getState().AssertMutable() + *ms.getOrig() = append(*ms.getOrig(), elms...) +} + +// MoveTo moves all elements from the current slice overriding the destination and +// resetting the current instance to its zero value. +func (ms IntSlice) MoveTo(dest IntSlice) { + ms.getState().AssertMutable() + dest.getState().AssertMutable() + *dest.getOrig() = *ms.getOrig() + *ms.getOrig() = nil +} + +// CopyTo copies all elements from the current slice overriding the destination. +func (ms IntSlice) CopyTo(dest IntSlice) { + dest.getState().AssertMutable() + *dest.getOrig() = copyIntSlice(*dest.getOrig(), *ms.getOrig()) +} + +func copyIntSlice(dst, src []int) []int { + dst = dst[:0] + return append(dst, src...) +} diff --git a/vendor/modules.txt b/vendor/modules.txt index ad64ef0e776b9..e83bfca8383dd 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1710,18 +1710,18 @@ go.opencensus.io/trace go.opencensus.io/trace/internal go.opencensus.io/trace/propagation go.opencensus.io/trace/tracestate -# go.opentelemetry.io/collector/pdata v1.20.0 +# go.opentelemetry.io/collector/pdata v1.21.0 ## explicit; go 1.22.0 go.opentelemetry.io/collector/pdata/internal go.opentelemetry.io/collector/pdata/internal/data go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/logs/v1 go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/metrics/v1 -go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1experimental +go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/profiles/v1development go.opentelemetry.io/collector/pdata/internal/data/protogen/collector/trace/v1 go.opentelemetry.io/collector/pdata/internal/data/protogen/common/v1 go.opentelemetry.io/collector/pdata/internal/data/protogen/logs/v1 go.opentelemetry.io/collector/pdata/internal/data/protogen/metrics/v1 -go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1experimental +go.opentelemetry.io/collector/pdata/internal/data/protogen/profiles/v1development go.opentelemetry.io/collector/pdata/internal/data/protogen/resource/v1 go.opentelemetry.io/collector/pdata/internal/data/protogen/trace/v1 go.opentelemetry.io/collector/pdata/internal/json From d16ae25f5aacac3100663d9527f00da6170242d9 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Wed, 4 Dec 2024 09:21:49 -0500 Subject: [PATCH 28/44] chore(deps): update terraform aws to ~> 5.79.0 (#15234) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: Paul Rogers <129207811+paul1r@users.noreply.github.com> --- production/terraform/modules/s3/versions.tf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/production/terraform/modules/s3/versions.tf b/production/terraform/modules/s3/versions.tf index 60decd4d7f8c0..b33267726f3ec 100644 --- a/production/terraform/modules/s3/versions.tf +++ b/production/terraform/modules/s3/versions.tf @@ -2,7 +2,7 @@ terraform { required_providers { aws = { source = "hashicorp/aws" - version = "~> 5.78.0" + version = "~> 5.79.0" } random = { From 5689e2e90e57cca8e51581aeeefbad633676ad08 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Wed, 4 Dec 2024 09:51:24 -0500 Subject: [PATCH 29/44] fix(deps): update module github.com/ibm/ibm-cos-sdk-go to v1.12.0 (#15228) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: Paul Rogers <129207811+paul1r@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 +- .../ibm-cos-sdk-go/aws/endpoints/defaults.go | 8 +- .../IBM/ibm-cos-sdk-go/aws/version.go | 2 +- .../IBM/ibm-cos-sdk-go/service/s3/api.go | 237 ++++++++++++++++++ vendor/modules.txt | 4 +- 6 files changed, 247 insertions(+), 10 deletions(-) diff --git a/go.mod b/go.mod index d242de7e31902..0cf6fe30c32cc 100644 --- a/go.mod +++ b/go.mod @@ -115,7 +115,7 @@ require ( github.com/DataDog/sketches-go v1.4.6 github.com/DmitriyVTitov/size v1.5.0 github.com/IBM/go-sdk-core/v5 v5.18.1 - github.com/IBM/ibm-cos-sdk-go v1.11.1 + github.com/IBM/ibm-cos-sdk-go v1.12.0 github.com/axiomhq/hyperloglog v0.2.0 github.com/buger/jsonparser v1.1.1 github.com/d4l3k/messagediff v1.2.1 diff --git a/go.sum b/go.sum index d27c746629d85..2369755051bfc 100644 --- a/go.sum +++ b/go.sum @@ -863,8 +863,8 @@ github.com/HdrHistogram/hdrhistogram-go v1.1.2 h1:5IcZpTvzydCQeHzK4Ef/D5rrSqwxob github.com/HdrHistogram/hdrhistogram-go v1.1.2/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= github.com/IBM/go-sdk-core/v5 v5.18.1 h1:wdftQO8xejECTWTKF3FGXyW0McKxxDAopH7MKwA187c= github.com/IBM/go-sdk-core/v5 v5.18.1/go.mod h1:3ywpylZ41WhWPusqtpJZWopYlt2brebcphV7mA2JncU= -github.com/IBM/ibm-cos-sdk-go v1.11.1 h1:Pye61hmWA4ZVCfOfFLTJBjPka4HIGrLqmpZ2d2KlrCE= -github.com/IBM/ibm-cos-sdk-go v1.11.1/go.mod h1:d8vET3w8wgmGwCsCVs+0y4V8+1hRNT6+pbpGaEHvSCI= +github.com/IBM/ibm-cos-sdk-go v1.12.0 h1:Wrk3ve4JS3euhl7XjNFd3RlvPT56199G2/rKaPWpRKU= +github.com/IBM/ibm-cos-sdk-go v1.12.0/go.mod h1:v/VBvFuysZMIX9HcaIrz6a+FLVw9px8fq6XabFwD+E4= github.com/IBM/sarama v1.43.3 h1:Yj6L2IaNvb2mRBop39N7mmJAHBVY3dTPncr3qGVkxPA= github.com/IBM/sarama v1.43.3/go.mod h1:FVIRaLrhK3Cla/9FfRF5X9Zua2KpS3SYIXxhac1H+FQ= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= diff --git a/vendor/github.com/IBM/ibm-cos-sdk-go/aws/endpoints/defaults.go b/vendor/github.com/IBM/ibm-cos-sdk-go/aws/endpoints/defaults.go index bf1d4fd0cd673..3bb705161a027 100644 --- a/vendor/github.com/IBM/ibm-cos-sdk-go/aws/endpoints/defaults.go +++ b/vendor/github.com/IBM/ibm-cos-sdk-go/aws/endpoints/defaults.go @@ -73,10 +73,10 @@ func DefaultResolver() Resolver { // DefaultPartitions returns a list of the partitions the SDK is bundled // with. The available partitions are: AWS Standard, AWS China, AWS GovCloud (US), AWS ISO (US), and AWS ISOB (US). // -// partitions := endpoints.DefaultPartitions -// for _, p := range partitions { -// // ... inspect partitions -// } +// partitions := endpoints.DefaultPartitions +// for _, p := range partitions { +// // ... inspect partitions +// } func DefaultPartitions() []Partition { return defaultPartitions.Partitions() } diff --git a/vendor/github.com/IBM/ibm-cos-sdk-go/aws/version.go b/vendor/github.com/IBM/ibm-cos-sdk-go/aws/version.go index 982d490db1aee..0246ba2503fc7 100644 --- a/vendor/github.com/IBM/ibm-cos-sdk-go/aws/version.go +++ b/vendor/github.com/IBM/ibm-cos-sdk-go/aws/version.go @@ -7,6 +7,6 @@ package aws const SDKName = "ibm-cos-sdk-go" // SDKVersion is the version of this SDK -const SDKVersion = "1.11.1" +const SDKVersion = "1.12.0" // IBM COS SDK Code -- END diff --git a/vendor/github.com/IBM/ibm-cos-sdk-go/service/s3/api.go b/vendor/github.com/IBM/ibm-cos-sdk-go/service/s3/api.go index db47502f0a3af..b1968d8c30040 100644 --- a/vendor/github.com/IBM/ibm-cos-sdk-go/service/s3/api.go +++ b/vendor/github.com/IBM/ibm-cos-sdk-go/service/s3/api.go @@ -15613,6 +15613,23 @@ type LifecycleRule struct { // period in the object's lifetime. NoncurrentVersionExpiration *NoncurrentVersionExpiration `type:"structure"` + // Specifies the transition rule for the lifecycle rule that describes when + // noncurrent objects transition to a specific storage class. If your bucket + // is versioning-enabled (or versioning is suspended), you can set this action + // to request that Amazon S3 transition noncurrent object versions to a specific + // storage class at a set period in the object's lifetime. + NoncurrentVersionTransitions []*NoncurrentVersionTransition `locationName:"NoncurrentVersionTransition" type:"list" flattened:"true"` + + // Prefix identifying one or more objects to which the rule applies. This is + // no longer used; use Filter instead. + // + // Replacement must be made for object keys containing special characters (such + // as carriage returns) when using XML requests. For more information, see XML + // related object key constraints (https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-keys.html#object-key-xml-related-constraints). + // + // Deprecated: Prefix has been deprecated + Prefix *string `deprecated:"true" type:"string"` + // If 'Enabled', the rule is currently being applied. If 'Disabled', the rule // is not currently being applied. // @@ -15650,6 +15667,11 @@ func (s *LifecycleRule) Validate() error { if s.Status == nil { invalidParams.Add(request.NewErrParamRequired("Status")) } + if s.Filter != nil { + if err := s.Filter.Validate(); err != nil { + invalidParams.AddNested("Filter", err.(request.ErrInvalidParams)) + } + } if invalidParams.Len() > 0 { return invalidParams @@ -15687,6 +15709,18 @@ func (s *LifecycleRule) SetNoncurrentVersionExpiration(v *NoncurrentVersionExpir return s } +// SetNoncurrentVersionTransitions sets the NoncurrentVersionTransitions field's value. +func (s *LifecycleRule) SetNoncurrentVersionTransitions(v []*NoncurrentVersionTransition) *LifecycleRule { + s.NoncurrentVersionTransitions = v + return s +} + +// SetPrefix sets the Prefix field's value. +func (s *LifecycleRule) SetPrefix(v string) *LifecycleRule { + s.Prefix = &v + return s +} + // SetStatus sets the Status field's value. func (s *LifecycleRule) SetStatus(v string) *LifecycleRule { s.Status = &v @@ -15699,17 +15733,109 @@ func (s *LifecycleRule) SetTransitions(v []*Transition) *LifecycleRule { return s } +// This is used in a Lifecycle Rule Filter to apply a logical AND to two or +// more predicates. The Lifecycle Rule will apply to any object matching all +// of the predicates configured inside the And operator. +type LifecycleRuleAndOperator struct { + _ struct{} `type:"structure"` + + ObjectSizeGreaterThan *int64 `type:"long"` + + ObjectSizeLessThan *int64 `type:"long"` + + // Prefix identifying one or more objects to which the rule applies. + Prefix *string `type:"string"` + + // All of these tags must exist in the object's tag set in order for the rule + // to apply. + Tags []*Tag `locationName:"Tag" locationNameList:"Tag" type:"list" flattened:"true"` +} + +// String returns the string representation. +// +// API parameter values that are decorated as "sensitive" in the API will not +// be included in the string output. The member name will be present, but the +// value will be replaced with "sensitive". +func (s LifecycleRuleAndOperator) String() string { + return awsutil.Prettify(s) +} + +// GoString returns the string representation. +// +// API parameter values that are decorated as "sensitive" in the API will not +// be included in the string output. The member name will be present, but the +// value will be replaced with "sensitive". +func (s LifecycleRuleAndOperator) GoString() string { + return s.String() +} + +// Validate inspects the fields of the type to determine if they are valid. +func (s *LifecycleRuleAndOperator) Validate() error { + invalidParams := request.ErrInvalidParams{Context: "LifecycleRuleAndOperator"} + if s.Tags != nil { + for i, v := range s.Tags { + if v == nil { + continue + } + if err := v.Validate(); err != nil { + invalidParams.AddNested(fmt.Sprintf("%s[%v]", "Tags", i), err.(request.ErrInvalidParams)) + } + } + } + + if invalidParams.Len() > 0 { + return invalidParams + } + return nil +} + +// SetObjectSizeGreaterThan sets the ObjectSizeGreaterThan field's value. +func (s *LifecycleRuleAndOperator) SetObjectSizeGreaterThan(v int64) *LifecycleRuleAndOperator { + s.ObjectSizeGreaterThan = &v + return s +} + +// SetObjectSizeLessThan sets the ObjectSizeLessThan field's value. +func (s *LifecycleRuleAndOperator) SetObjectSizeLessThan(v int64) *LifecycleRuleAndOperator { + s.ObjectSizeLessThan = &v + return s +} + +// SetPrefix sets the Prefix field's value. +func (s *LifecycleRuleAndOperator) SetPrefix(v string) *LifecycleRuleAndOperator { + s.Prefix = &v + return s +} + +// SetTags sets the Tags field's value. +func (s *LifecycleRuleAndOperator) SetTags(v []*Tag) *LifecycleRuleAndOperator { + s.Tags = v + return s +} + // The Filter is used to identify objects that a Lifecycle Rule applies to. // A Filter must have exactly one of Prefix, Tag, or And specified. type LifecycleRuleFilter struct { _ struct{} `type:"structure"` + // This is used in a Lifecycle Rule Filter to apply a logical AND to two or + // more predicates. The Lifecycle Rule will apply to any object matching all + // of the predicates configured inside the And operator. + And *LifecycleRuleAndOperator `type:"structure"` + + ObjectSizeGreaterThan *int64 `type:"long"` + + ObjectSizeLessThan *int64 `type:"long"` + // Prefix identifying one or more objects to which the rule applies. // // Replacement must be made for object keys containing special characters (such // as carriage returns) when using XML requests. For more information, see XML // related object key constraints (https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-keys.html#object-key-xml-related-constraints). Prefix *string `type:"string"` + + // This tag must exist in the object's tag set in order for the rule to apply. + Tag *Tag `type:"structure"` } // String returns the string representation. @@ -15730,12 +15856,56 @@ func (s LifecycleRuleFilter) GoString() string { return s.String() } +// Validate inspects the fields of the type to determine if they are valid. +func (s *LifecycleRuleFilter) Validate() error { + invalidParams := request.ErrInvalidParams{Context: "LifecycleRuleFilter"} + if s.And != nil { + if err := s.And.Validate(); err != nil { + invalidParams.AddNested("And", err.(request.ErrInvalidParams)) + } + } + if s.Tag != nil { + if err := s.Tag.Validate(); err != nil { + invalidParams.AddNested("Tag", err.(request.ErrInvalidParams)) + } + } + + if invalidParams.Len() > 0 { + return invalidParams + } + return nil +} + +// SetAnd sets the And field's value. +func (s *LifecycleRuleFilter) SetAnd(v *LifecycleRuleAndOperator) *LifecycleRuleFilter { + s.And = v + return s +} + +// SetObjectSizeGreaterThan sets the ObjectSizeGreaterThan field's value. +func (s *LifecycleRuleFilter) SetObjectSizeGreaterThan(v int64) *LifecycleRuleFilter { + s.ObjectSizeGreaterThan = &v + return s +} + +// SetObjectSizeLessThan sets the ObjectSizeLessThan field's value. +func (s *LifecycleRuleFilter) SetObjectSizeLessThan(v int64) *LifecycleRuleFilter { + s.ObjectSizeLessThan = &v + return s +} + // SetPrefix sets the Prefix field's value. func (s *LifecycleRuleFilter) SetPrefix(v string) *LifecycleRuleFilter { s.Prefix = &v return s } +// SetTag sets the Tag field's value. +func (s *LifecycleRuleFilter) SetTag(v *Tag) *LifecycleRuleFilter { + s.Tag = v + return s +} + type ListBucketsExtendedInput struct { _ struct{} `locationName:"ListBucketsExtendedRequest" type:"structure"` @@ -17861,6 +18031,8 @@ func (s *MultipartUpload) SetUploadId(v string) *MultipartUpload { type NoncurrentVersionExpiration struct { _ struct{} `type:"structure"` + NewerNoncurrentVersions *int64 `type:"integer"` + // Specifies the number of days an object is noncurrent before Amazon S3 can // perform the associated action. For information about the noncurrent days // calculations, see How Amazon S3 Calculates When an Object Became Noncurrent @@ -17887,12 +18059,77 @@ func (s NoncurrentVersionExpiration) GoString() string { return s.String() } +// SetNewerNoncurrentVersions sets the NewerNoncurrentVersions field's value. +func (s *NoncurrentVersionExpiration) SetNewerNoncurrentVersions(v int64) *NoncurrentVersionExpiration { + s.NewerNoncurrentVersions = &v + return s +} + // SetNoncurrentDays sets the NoncurrentDays field's value. func (s *NoncurrentVersionExpiration) SetNoncurrentDays(v int64) *NoncurrentVersionExpiration { s.NoncurrentDays = &v return s } +// Container for the transition rule that describes when noncurrent objects +// transition to the STANDARD_IA, ONEZONE_IA, INTELLIGENT_TIERING, GLACIER, +// or DEEP_ARCHIVE storage class. If your bucket is versioning-enabled (or versioning +// is suspended), you can set this action to request that Amazon S3 transition +// noncurrent object versions to the STANDARD_IA, ONEZONE_IA, INTELLIGENT_TIERING, +// GLACIER, or DEEP_ARCHIVE storage class at a specific period in the object's +// lifetime. +type NoncurrentVersionTransition struct { + _ struct{} `type:"structure"` + + NewerNoncurrentVersions *int64 `type:"integer"` + + // Specifies the number of days an object is noncurrent before Amazon S3 can + // perform the associated action. For information about the noncurrent days + // calculations, see How Amazon S3 Calculates How Long an Object Has Been Noncurrent + // (https://docs.aws.amazon.com/AmazonS3/latest/dev/intro-lifecycle-rules.html#non-current-days-calculations) + // in the Amazon S3 User Guide. + NoncurrentDays *int64 `type:"integer"` + + // The class of storage used to store the object. + StorageClass *string `type:"string" enum:"TransitionStorageClass"` +} + +// String returns the string representation. +// +// API parameter values that are decorated as "sensitive" in the API will not +// be included in the string output. The member name will be present, but the +// value will be replaced with "sensitive". +func (s NoncurrentVersionTransition) String() string { + return awsutil.Prettify(s) +} + +// GoString returns the string representation. +// +// API parameter values that are decorated as "sensitive" in the API will not +// be included in the string output. The member name will be present, but the +// value will be replaced with "sensitive". +func (s NoncurrentVersionTransition) GoString() string { + return s.String() +} + +// SetNewerNoncurrentVersions sets the NewerNoncurrentVersions field's value. +func (s *NoncurrentVersionTransition) SetNewerNoncurrentVersions(v int64) *NoncurrentVersionTransition { + s.NewerNoncurrentVersions = &v + return s +} + +// SetNoncurrentDays sets the NoncurrentDays field's value. +func (s *NoncurrentVersionTransition) SetNoncurrentDays(v int64) *NoncurrentVersionTransition { + s.NoncurrentDays = &v + return s +} + +// SetStorageClass sets the StorageClass field's value. +func (s *NoncurrentVersionTransition) SetStorageClass(v string) *NoncurrentVersionTransition { + s.StorageClass = &v + return s +} + // An object consists of data and its descriptive metadata. type Object struct { _ struct{} `type:"structure"` diff --git a/vendor/modules.txt b/vendor/modules.txt index e83bfca8383dd..95e862740ba90 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -223,8 +223,8 @@ github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapp # github.com/IBM/go-sdk-core/v5 v5.18.1 ## explicit; go 1.21 github.com/IBM/go-sdk-core/v5/core -# github.com/IBM/ibm-cos-sdk-go v1.11.1 -## explicit; go 1.19 +# github.com/IBM/ibm-cos-sdk-go v1.12.0 +## explicit; go 1.21 github.com/IBM/ibm-cos-sdk-go/aws github.com/IBM/ibm-cos-sdk-go/aws/arn github.com/IBM/ibm-cos-sdk-go/aws/awserr From 3150f073db00a7bc6000fd84035e9f3a68497bdc Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Wed, 4 Dec 2024 09:51:52 -0500 Subject: [PATCH 30/44] fix(deps): update module golang.org/x/sync to v0.10.0 (#15253) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- vendor/modules.txt | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index 0cf6fe30c32cc..a6eb23369c6e4 100644 --- a/go.mod +++ b/go.mod @@ -100,7 +100,7 @@ require ( go.uber.org/goleak v1.3.0 golang.org/x/crypto v0.29.0 golang.org/x/net v0.31.0 - golang.org/x/sync v0.9.0 + golang.org/x/sync v0.10.0 golang.org/x/sys v0.27.0 golang.org/x/time v0.8.0 google.golang.org/api v0.209.0 diff --git a/go.sum b/go.sum index 2369755051bfc..cd965701b4da8 100644 --- a/go.sum +++ b/go.sum @@ -3098,8 +3098,8 @@ golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -golang.org/x/sync v0.9.0 h1:fEo0HyrW1GIgZdpbhCRO0PkJajUS5H9IFUztCgEo2jQ= -golang.org/x/sync v0.9.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20170830134202-bb24a47a89ea/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= diff --git a/vendor/modules.txt b/vendor/modules.txt index 95e862740ba90..bab919649f522 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1870,7 +1870,7 @@ golang.org/x/oauth2/google/internal/stsexchange golang.org/x/oauth2/internal golang.org/x/oauth2/jws golang.org/x/oauth2/jwt -# golang.org/x/sync v0.9.0 +# golang.org/x/sync v0.10.0 ## explicit; go 1.18 golang.org/x/sync/errgroup golang.org/x/sync/semaphore From 1b03a08bad4c1cc5294cd1f667204c2a86fbf95f Mon Sep 17 00:00:00 2001 From: J Stickler Date: Wed, 4 Dec 2024 10:26:35 -0500 Subject: [PATCH 31/44] docs: fixing some Vale linter errors (#15212) --- docs/sources/configure/bp-configure.md | 10 +- docs/sources/configure/storage.md | 6 +- docs/sources/setup/upgrade/_index.md | 377 +++++++++++++------------ 3 files changed, 197 insertions(+), 196 deletions(-) diff --git a/docs/sources/configure/bp-configure.md b/docs/sources/configure/bp-configure.md index 28feb68b1ba4c..3d00867201815 100644 --- a/docs/sources/configure/bp-configure.md +++ b/docs/sources/configure/bp-configure.md @@ -6,7 +6,7 @@ weight: 100 --- # Configuration best practices -Grafana Loki is under active development, and we are constantly working to improve performance. But here are some of the most current best practices for configuration that will give you the best experience with Loki. +Grafana Loki is under active development, and the Loki team is constantly working to improve performance. But here are some of the most current best practices for configuration that will give you the best experience with Loki. ## Configure caching @@ -36,7 +36,7 @@ If Loki received these two lines which are for the same stream, everything would {job="syslog"} 00:00:01 i'm a syslog! <- Rejected out of order! ``` -What can we do about this? What if this was because the sources of these logs were different systems? We can solve this with an additional label which is unique per system: +What can you do about this? What if this was because the sources of these logs were different systems? You can solve this with an additional label which is unique per system: ``` {job="syslog", instance="host1"} 00:00:00 i'm a syslog! @@ -56,9 +56,9 @@ Using `chunk_target_size` instructs Loki to try to fill all chunks to a target _ Other configuration variables affect how full a chunk can get. Loki has a default `max_chunk_age` of 2h and `chunk_idle_period` of 30m to limit the amount of memory used as well as the exposure of lost logs if the process crashes. -Depending on the compression used (we have been using snappy which has less compressibility but faster performance), you need 5-10x or 7.5-10MB of raw log data to fill a 1.5MB chunk. Remembering that a chunk is per stream, the more streams you break up your log files into, the more chunks that sit in memory, and the higher likelihood they get flushed by hitting one of those timeouts mentioned above before they are filled. +Depending on the compression used (Loki has been using snappy which has less compressibility but faster performance), you need 5-10x or 7.5-10MB of raw log data to fill a 1.5MB chunk. Remembering that a chunk is per stream, the more streams you break up your log files into, the more chunks that sit in memory, and the higher likelihood they get flushed by hitting one of those timeouts mentioned above before they are filled. -Lots of small, unfilled chunks negatively affect Loki. We are always working to improve this and may consider a compactor to improve this in some situations. But, in general, the guidance should stay about the same: try your best to fill chunks. +Lots of small, unfilled chunks negatively affect Loki. The team is always working to improve this and may consider a compactor to improve this in some situations. But, in general, the guidance should stay about the same: try your best to fill chunks. If you have an application that can log fast enough to fill these chunks quickly (much less than `max_chunk_age`), then it becomes more reasonable to use dynamic labels to break that up into separate streams. @@ -68,4 +68,4 @@ Loki and Promtail have flags which will dump the entire config object to stderr `-print-config-stderr` works well when invoking Loki from the command line, as you can get a quick output of the entire Loki configuration. -`-log-config-reverse-order` is the flag we run Loki with in all our environments. The configuration entries are reversed, so that the order of the configuration reads correctly top to bottom when viewed in Grafana's Explore. +`-log-config-reverse-order` is the flag Grafana runs Loki with in all our environments. The configuration entries are reversed, so that the order of the configuration reads correctly top to bottom when viewed in Grafana's Explore. diff --git a/docs/sources/configure/storage.md b/docs/sources/configure/storage.md index c0178b67bdcbb..276834a8f8445 100644 --- a/docs/sources/configure/storage.md +++ b/docs/sources/configure/storage.md @@ -193,7 +193,7 @@ When a new schema is released and you want to gain the advantages it provides, y First, you'll want to create a new [period_config](https://grafana.com/docs/loki//configure/#period_config) entry in your [schema_config](https://grafana.com/docs/loki//configure/#schema_config). The important thing to remember here is to set this at some point in the _future_ and then roll out the config file changes to Loki. This allows the table manager to create the required table in advance of writes and ensures that existing data isn't queried as if it adheres to the new schema. -As an example, let's say it's 2023-07-14 and we want to start using the `v13` schema on the 20th: +As an example, let's say it's 2023-07-14 and you want to start using the `v13` schema on the 20th: ```yaml schema_config: @@ -214,7 +214,7 @@ schema_config: period: 24h ``` -It's that easy; we just created a new entry starting on the 20th. +It's that easy; you just created a new entry starting on the 20th. ## Retention @@ -485,7 +485,7 @@ schema_config: ### On premise deployment (MinIO Single Store) -We configure MinIO by using the AWS config because MinIO implements the S3 API: +You configure MinIO by using the AWS config because MinIO implements the S3 API: ```yaml storage_config: diff --git a/docs/sources/setup/upgrade/_index.md b/docs/sources/setup/upgrade/_index.md index e575f9688d1de..546d40f77d32e 100644 --- a/docs/sources/setup/upgrade/_index.md +++ b/docs/sources/setup/upgrade/_index.md @@ -22,7 +22,7 @@ If possible try to stay current and do sequential updates. If you want to skip v Using docker you can check changes between 2 versions of Loki with a command like this: -``` +```bash export OLD_LOKI=2.9.4 export NEW_LOKI=3.0.0 export CONFIG_FILE=local-config.yaml @@ -33,7 +33,6 @@ The `tr -d '\r'` is likely not necessary for most people, seems like WSL2 was sn The output is incredibly verbose as it shows the entire internal config struct used to run Loki, you can play around with the diff command if you prefer to only show changes or a different style output. - ## Main / Unreleased ### HTTP API @@ -76,10 +75,9 @@ Their YAML counterparts in the `limits_config` block are kept identical. All other CLI arguments (and their YAML counterparts) prefixed with `-bloom-compactor.` have been removed. - ## 3.3.0 -### Loki +### Loki 3.3.0 #### Experimental Bloom Filters @@ -104,16 +102,18 @@ Loki 3.0 is a major version increase and comes with several breaking changes. Here is the shortlist of things we think most people may encounter: - * Structured metadata is enabled by default and requires `tsdb` and `v13` schema or Loki won't start. Refer to [Structured Metadata, Open Telemetry, Schemas and Indexes](#structured-metadata-open-telemetry-schemas-and-indexes). - * The `shared_store` config is removed. Refer to [Removed `shared_store` and `shared_store_key_prefix` from shipper configuration](#removed-shared_store-and-shared_store_key_prefix-from-shipper-configuration). - * Loki now enforces a max line size of 256KB by default (you can disable this or increase this but this is how Grafana Labs runs Loki). Refer to [Changes to default configure values](#changes-to-default-configuration-values-in-30). - * Loki now enforces a max label limit of 15 labels per series, down from 30. Extra labels inflate the size of the index and reduce performance, you should almost never need more than 15 labels. Refer to [Changes to default configure values](#changes-to-default-configuration-values-in-30). - * Loki will automatically attempt to populate a `service_name` label on ingestion. Refer to [`service_name` label](#service_name-label). - * There are many metric name changes. Refer to [Distributor metric changes](#distributor-metric-changes), [Embedded cache metric changes](#embedded-cache-metric-changes), and [Metrics namespace](#metrics-namespace). +* Structured metadata is enabled by default and requires `tsdb` and `v13` schema or Loki won't start. Refer to [Structured Metadata, Open Telemetry, Schemas and Indexes](#structured-metadata-open-telemetry-schemas-and-indexes). +* The `shared_store` config is removed. Refer to [Removed `shared_store` and `shared_store_key_prefix` from shipper configuration](#removed-shared_store-and-shared_store_key_prefix-from-shipper-configuration). +* Loki now enforces a max line size of 256KB by default (you can disable this or increase this but this is how Grafana Labs runs Loki). Refer to [Changes to default configure values](#changes-to-default-configuration-values-in-30). +* Loki now enforces a max label limit of 15 labels per series, down from 30. Extra labels inflate the size of the index and reduce performance, you should almost never need more than 15 labels. Refer to [Changes to default configure values](#changes-to-default-configuration-values-in-30). +* Loki will automatically attempt to populate a `service_name` label on ingestion. Refer to [`service_name` label](#service_name-label). +* There are many metric name changes. Refer to [Distributor metric changes](#distributor-metric-changes), [Embedded cache metric changes](#embedded-cache-metric-changes), and [Metrics namespace](#metrics-namespace). If you would like to see if your existing configuration will work with Loki 3.0: + 1. In an empty directory on your computer, copy you configuration into a file named `loki-config.yaml`. -1. Run this command from that directory: +1. Run this command from that directory: + ```bash docker run --rm -t -v "${PWD}":/config grafana/loki:3.0.0 -config.file=/config/loki-config.yaml -verify-config=true ``` @@ -126,16 +126,16 @@ If you introduce a new schema_config entry it may cause additional validation er If you configure `path_prefix` in the `common` config section this can help save a lot of configuration. Refer to the [Common Config Docs](https://grafana.com/docs/loki//configure/#common). {{< /admonition >}} - The **Helm chart** has gone through some significant changes and has a separate upgrade guide: [Upgrading to Helm 6.x](https://grafana.com/docs/loki//setup/upgrade/upgrade-to-6x/). -### Loki +### Loki 3.0.0 #### Structured Metadata, Open Telemetry, Schemas and Indexes A flagship feature of Loki 3.0 is native support for the Open Telemetry Protocol (OTLP). This is made possible by a new feature in Loki called [Structured Metadata](https://grafana.com/docs/loki//get-started/labels/structured-metadata/), a place for metadata which doesn't belong in labels or log lines. OTel resources and attributes are often a great example of data which doesn't belong in the index nor in the log line. Structured Metadata is enabled by default in Loki 3.0, however, it requires your active schema be using both the `tsdb` index type AND the `v13` storage schema. If you are not using both of these you have two options: + * Upgrade your index version and schema version before updating to 3.0, see [schema config upgrade](https://grafana.com/docs/loki//operations/storage/schema/). * Disable Structured Metadata (and therefore OTLP support) and upgrade to 3.0 and perform the schema migration after. This can be done by setting `allow_structured_metadata: false` in the `limits_config` section or set the command line argument `-validation.allow-structured-metadata=false`. @@ -145,17 +145,17 @@ Loki 3.0 will automatically assign a `service_name` label to all ingested logs b Loki will attempt to create the `service_name` label by looking for the following labels in this order: - - service_name - - service - - app - - application - - name - - app_kubernetes_io_name - - container - - container_name - - component - - workload - - job +- service_name +- service +- app +- application +- name +- app_kubernetes_io_name +- container +- container_name +- component +- workload +- job If no label is found matching the list, a value of `unknown_service` is applied. @@ -170,6 +170,7 @@ If you are already using a `service_label`, Loki will not make a new assignment. #### Removed `shared_store` and `shared_store_key_prefix` from shipper configuration The following CLI flags and the corresponding YAML settings to configure shared store for TSDB and BoltDB shippers are now removed: + - `-boltdb.shipper.shared-store` - `-tsdb.shipper.shared-store` @@ -180,8 +181,7 @@ We are removing the shared store setting in an effort to simplify storage config {{< admonition type="warning" >}} With this change Loki no longer allows storing chunks and indexes for a given period in different storage buckets. -This is a breaking change for setups that store chunks and indexes in different storage buckets by setting `-boltdb.shipper.shared-store` or `-tsdb.shipper.shared-store` to a value -different from `object_store` in `period_config`. +This is a breaking change for setups that store chunks and indexes in different storage buckets by setting `-boltdb.shipper.shared-store` or `-tsdb.shipper.shared-store` to a value different from `object_store` in `period_config`. {{< /admonition >}} - If you have not configured `-boltdb.shipper.shared-store`,`-tsdb.shipper.shared-store` or their corresponding YAML setting before, no changes are required as part of the upgrade. @@ -195,12 +195,14 @@ different from `object_store` in `period_config`. - To make these indexes queryable, index tables need to moved or copied to the store configured in `object_store`. The following CLI flags and the corresponding YAML settings to configure a path prefix for TSDB and BoltDB shippers are now removed: + - `-boltdb.shipper.shared-store.key-prefix` - `-tsdb.shipper.shared-store.key-prefix` Path prefix for storing the index can now be configured by setting `path_prefix` under `index` key in [period_config](/docs/loki//configure/#period_config). This enables users to change the path prefix by adding a new period config. -``` + +```yaml period_config: index: path_prefix: "index/" @@ -220,6 +222,7 @@ period_config: #### Removed `shared_store` and `shared_store_key_prefix` from compactor configuration The following CLI flags and the corresponding YAML settings to configure the shared store and path prefix for compactor are now removed: + - `-boltdb.shipper.compactor.shared-store` - `-boltdb.shipper.compactor.shared-store.key-prefix` @@ -243,16 +246,17 @@ additionally the `Background` configuration also lest you set `writeback_size_li The already deprecated handler `/ingester/flush_shutdown` is removed in favor of `/ingester/shutdown?flush=true`. -#### Ingester configuration `max_transfer_retries` is removed. +#### Ingester configuration `max_transfer_retries` is removed The setting `max_transfer_retries` (`-ingester.max-transfer-retries`) is removed in favor of the Write Ahead log (WAL). It was used to allow transferring chunks to new ingesters when the old ingester was shutting down during a rolling restart. Alternatives to this setting are: + - **A. (Preferred)** Enable the WAL and rely on the new ingester to replay the WAL. - Optionally, you can enable `flush_on_shutdown` (`-ingester.flush-on-shutdown`) to flush to long-term storage on shutdowns. - **B.** Manually flush during shutdowns via [the ingester `/shutdown?flush=true` endpoint](https://grafana.com/docs/loki//reference/api/#flush-in-memory-chunks-and-shut-down). -#### Removed the `default` section of the runtime overrides config file. +#### Removed the `default` section of the runtime overrides config file This was introduced in 2.9 and likely not widely used. This only affects you if you run Loki with a runtime config file AND you had populated the new `default` block added in 2.9. @@ -283,19 +287,6 @@ The previous default value `false` is applied. 1. `boltdb.shipper.compactor.deletion-mode` CLI flag and the corresponding YAML setting are removed. You can instead configure the `compactor.deletion-mode` CLI flag or `deletion_mode` YAML setting in [Limits Config](/docs/loki//configuration/#limits_config). 1. Compactor CLI flags that use the prefix `boltdb.shipper.compactor.` are removed. You can instead use CLI flags with the `compactor.` prefix. -#### Legacy ingester shutdown handler is removed - -The already deprecated handler `/ingester/flush_shutdown` is removed in favor of `/ingester/shutdown?flush=true`. - -#### Ingester configuration `max_transfer_retries` is removed. - -The setting `max_transfer_retries` (`-ingester.max-transfer-retries`) is removed in favor of the Write Ahead log (WAL). -It was used to allow transferring chunks to new ingesters when the old ingester was shutting down during a rolling restart. -Alternatives to this setting are: -- **A. (Preferred)** Enable the WAL and rely on the new ingester to replay the WAL. - - Optionally, you can enable `flush_on_shutdown` (`-ingester.flush-on-shutdown`) to flush to long-term storage on shutdowns. -- **B.** Manually flush during shutdowns via [the ingester `/shutdown?flush=true` endpoint](https://grafana.com/docs/loki//reference/loki-http-api#flush-in-memory-chunks-and-shut-down). - #### Distributor metric changes The `loki_distributor_ingester_append_failures_total` metric has been removed in favour of `loki_distributor_ingester_append_timeouts_total`. @@ -304,8 +295,9 @@ This new metric will provide a more clear signal that there is an issue with ing #### Changes to default configuration values in 3.0 {{< responsive-table >}} + | configuration | new default | old default | notes | -| ------------------------------------------------------ | ----------- | ----------- | -------- +| ------------------------------------------------------ | ----------- | ----------- | --------| | `compactor.delete-max-interval` | 24h | 0 | splits the delete requests into intervals no longer than `delete_max_interval` | | `distributor.max-line-size` | 256KB | 0 | - | | `ingester.sync-period` | 1h | 0 | ensures that the chunk cuts for a given stream are synchronized across the ingesters in the replication set. Helps with deduplicating chunks. | @@ -323,6 +315,7 @@ This new metric will provide a more clear signal that there is an issue with ing | `query-scheduler.max-outstanding-requests-per-tenant` | 32000 | 100 | - | | `validation.max-label-names-per-series` | 15 | 30 | - | | `legacy-read-mode` | false | true | Deprecated. It will be removed in the next minor release. | + {{< /responsive-table >}} #### Automatic stream sharding is enabled by default @@ -338,6 +331,7 @@ The TSDB index type has support for caching results for 'stats' and 'volume' que All of these are cached to the `results_cache` which is configured in the `query_range` config section. By default, an in memory cache is used. #### Write dedupe cache is deprecated + Write dedupe cache is deprecated because it not required by the newer single store indexes ([TSDB](https://grafana.com/docs/loki//operations/storage/tsdb/) and [boltdb-shipper](https://grafana.com/docs/loki//operations/storage/boltdb-shipper/)). If you using a [legacy index type](https://grafana.com/docs/loki//configure/storage/#index-storage), consider migrating to TSDB (recommended). @@ -360,74 +354,73 @@ If you using a [legacy index type](https://grafana.com/docs/loki// Some Loki metrics started with the prefix `cortex_`. In this release they will be changed so they start with `loki_`. To keep them at `cortex_` change the `metrics_namespace` from the default `loki` to `cortex`. These metrics will be changed: - - `cortex_distributor_ingester_clients` - - `cortex_dns_failures_total` - - `cortex_dns_lookups_total` - - `cortex_dns_provider_results` - - `cortex_frontend_query_range_duration_seconds_bucket` - - `cortex_frontend_query_range_duration_seconds_count` - - `cortex_frontend_query_range_duration_seconds_sum` - - `cortex_ingester_flush_queue_length` - - `cortex_kv_request_duration_seconds_bucket` - - `cortex_kv_request_duration_seconds_count` - - `cortex_kv_request_duration_seconds_sum` - - `cortex_member_consul_heartbeats_total` - - `cortex_prometheus_last_evaluation_samples` - - `cortex_prometheus_notifications_alertmanagers_discovered` - - `cortex_prometheus_notifications_dropped_total` - - `cortex_prometheus_notifications_errors_total` - - `cortex_prometheus_notifications_latency_seconds` - - `cortex_prometheus_notifications_latency_seconds_count` - - `cortex_prometheus_notifications_latency_seconds_sum` - - `cortex_prometheus_notifications_queue_capacity` - - `cortex_prometheus_notifications_queue_length` - - `cortex_prometheus_notifications_sent_total` - - `cortex_prometheus_rule_evaluation_duration_seconds` - - `cortex_prometheus_rule_evaluation_duration_seconds_count` - - `cortex_prometheus_rule_evaluation_duration_seconds_sum` - - `cortex_prometheus_rule_evaluation_failures_total` - - `cortex_prometheus_rule_evaluations_total` - - `cortex_prometheus_rule_group_duration_seconds` - - `cortex_prometheus_rule_group_duration_seconds_count` - - `cortex_prometheus_rule_group_duration_seconds_sum` - - `cortex_prometheus_rule_group_interval_seconds` - - `cortex_prometheus_rule_group_iterations_missed_total` - - `cortex_prometheus_rule_group_iterations_total` - - `cortex_prometheus_rule_group_last_duration_seconds` - - `cortex_prometheus_rule_group_last_evaluation_timestamp_seconds` - - `cortex_prometheus_rule_group_rules` - - `cortex_query_frontend_connected_schedulers` - - `cortex_query_frontend_queries_in_progress` - - `cortex_query_frontend_retries_bucket` - - `cortex_query_frontend_retries_count` - - `cortex_query_frontend_retries_sum` - - `cortex_query_scheduler_connected_frontend_clients` - - `cortex_query_scheduler_connected_querier_clients` - - `cortex_query_scheduler_inflight_requests` - - `cortex_query_scheduler_inflight_requests_count` - - `cortex_query_scheduler_inflight_requests_sum` - - `cortex_query_scheduler_queue_duration_seconds_bucket` - - `cortex_query_scheduler_queue_duration_seconds_count` - - `cortex_query_scheduler_queue_duration_seconds_sum` - - `cortex_query_scheduler_queue_length` - - `cortex_query_scheduler_running` - - `cortex_ring_member_heartbeats_total` - - `cortex_ring_member_tokens_owned` - - `cortex_ring_member_tokens_to_own` - - `cortex_ring_members` - - `cortex_ring_oldest_member_timestamp` - - `cortex_ring_tokens_total` - - `cortex_ruler_client_request_duration_seconds_bucket` - - `cortex_ruler_client_request_duration_seconds_count` - - `cortex_ruler_client_request_duration_seconds_sum` - - `cortex_ruler_clients` - - `cortex_ruler_config_last_reload_successful` - - `cortex_ruler_config_last_reload_successful_seconds` - - `cortex_ruler_config_updates_total` - - `cortex_ruler_managers_total` - - `cortex_ruler_ring_check_errors_total` - - `cortex_ruler_sync_rules_total` - +- `cortex_distributor_ingester_clients` +- `cortex_dns_failures_total` +- `cortex_dns_lookups_total` +- `cortex_dns_provider_results` +- `cortex_frontend_query_range_duration_seconds_bucket` +- `cortex_frontend_query_range_duration_seconds_count` +- `cortex_frontend_query_range_duration_seconds_sum` +- `cortex_ingester_flush_queue_length` +- `cortex_kv_request_duration_seconds_bucket` +- `cortex_kv_request_duration_seconds_count` +- `cortex_kv_request_duration_seconds_sum` +- `cortex_member_consul_heartbeats_total` +- `cortex_prometheus_last_evaluation_samples` +- `cortex_prometheus_notifications_alertmanagers_discovered` +- `cortex_prometheus_notifications_dropped_total` +- `cortex_prometheus_notifications_errors_total` +- `cortex_prometheus_notifications_latency_seconds` +- `cortex_prometheus_notifications_latency_seconds_count` +- `cortex_prometheus_notifications_latency_seconds_sum` +- `cortex_prometheus_notifications_queue_capacity` +- `cortex_prometheus_notifications_queue_length` +- `cortex_prometheus_notifications_sent_total` +- `cortex_prometheus_rule_evaluation_duration_seconds` +- `cortex_prometheus_rule_evaluation_duration_seconds_count` +- `cortex_prometheus_rule_evaluation_duration_seconds_sum` +- `cortex_prometheus_rule_evaluation_failures_total` +- `cortex_prometheus_rule_evaluations_total` +- `cortex_prometheus_rule_group_duration_seconds` +- `cortex_prometheus_rule_group_duration_seconds_count` +- `cortex_prometheus_rule_group_duration_seconds_sum` +- `cortex_prometheus_rule_group_interval_seconds` +- `cortex_prometheus_rule_group_iterations_missed_total` +- `cortex_prometheus_rule_group_iterations_total` +- `cortex_prometheus_rule_group_last_duration_seconds` +- `cortex_prometheus_rule_group_last_evaluation_timestamp_seconds` +- `cortex_prometheus_rule_group_rules` +- `cortex_query_frontend_connected_schedulers` +- `cortex_query_frontend_queries_in_progress` +- `cortex_query_frontend_retries_bucket` +- `cortex_query_frontend_retries_count` +- `cortex_query_frontend_retries_sum` +- `cortex_query_scheduler_connected_frontend_clients` +- `cortex_query_scheduler_connected_querier_clients` +- `cortex_query_scheduler_inflight_requests` +- `cortex_query_scheduler_inflight_requests_count` +- `cortex_query_scheduler_inflight_requests_sum` +- `cortex_query_scheduler_queue_duration_seconds_bucket` +- `cortex_query_scheduler_queue_duration_seconds_count` +- `cortex_query_scheduler_queue_duration_seconds_sum` +- `cortex_query_scheduler_queue_length` +- `cortex_query_scheduler_running` +- `cortex_ring_member_heartbeats_total` +- `cortex_ring_member_tokens_owned` +- `cortex_ring_member_tokens_to_own` +- `cortex_ring_members` +- `cortex_ring_oldest_member_timestamp` +- `cortex_ring_tokens_total` +- `cortex_ruler_client_request_duration_seconds_bucket` +- `cortex_ruler_client_request_duration_seconds_count` +- `cortex_ruler_client_request_duration_seconds_sum` +- `cortex_ruler_clients` +- `cortex_ruler_config_last_reload_successful` +- `cortex_ruler_config_last_reload_successful_seconds` +- `cortex_ruler_config_updates_total` +- `cortex_ruler_managers_total` +- `cortex_ruler_ring_check_errors_total` +- `cortex_ruler_sync_rules_total` The `metrics_namespace` setting is deprecated already. It will be removed in the next minor release. The default prefix will be `loki` then. @@ -440,7 +433,7 @@ A new CLI flag `-schema-store` is introduced as a replacement to configure the s ## 2.9.0 -### Loki +### Loki 2.9.0 #### Index gateway shuffle sharding @@ -475,6 +468,7 @@ in the logs on startup and the `/ingester/prepare_shutdown` endpoint will return #### Compactor multi-store support In previous releases, setting `-boltdb.shipper.compactor.shared-store` configured the following: + - store used for managing delete requests. - store on which index compaction should be performed. @@ -483,11 +477,13 @@ If `-boltdb.shipper.compactor.shared-store` was not set, it used to default to t Compactor now supports index compaction on multiple buckets/object stores. And going forward loki will not set any defaults on `-boltdb.shipper.compactor.shared-store`, this has a couple of side effects detailed as follows: -##### store on which index compaction should be performed: +##### store on which index compaction should be performed + If `-boltdb.shipper.compactor.shared-store` is configured by the user, loki would run index compaction only on the store specified by the config. If not set, compaction would be performed on all the object stores that contain either a boltdb-shipper or tsdb index. -##### store used for managing delete requests: +##### store used for managing delete requests + A new config option `-boltdb.shipper.compactor.delete-request-store` decides where delete requests should be stored. This new option takes precedence over `-boltdb.shipper.compactor.shared-store`. In the case where neither of these options are set, the `object_store` configured in the latest `period_config` that uses either a tsdb or boltdb-shipper index is used for storing delete requests to ensure pending requests are processed. @@ -501,9 +497,9 @@ If you have a use-case that relies on strict parsing where you expect the parser logfmt parser doesn't include standalone keys(keys without a value) in the resulting label set anymore. You can use `--keep-empty` flag to retain them. -### Jsonnet +### Jsonnet 2..9.0 -##### Deprecated PodDisruptionBudget definition has been removed +#### Deprecated PodDisruptionBudget definition has been removed The `policy/v1beta1` API version of PodDisruptionBudget is no longer served as of Kubernetes v1.25. To support the latest versions of the Kubernetes, it was necessary to replace `policy/v1beta1` with the new definition `policy/v1` that is available since v1.21. @@ -512,10 +508,9 @@ No impact is expected if you use Kubernetes v1.21 or newer. Please refer to [official migration guide](https://kubernetes.io/docs/reference/using-api/deprecation-guide/#poddisruptionbudget-v125) for more details. - ## 2.8.0 -### Loki +### Loki 2.8.0 #### Change in LogQL behavior @@ -525,6 +520,7 @@ was kept. #### Default retention_period has changed This change will affect you if you have: + ```yaml compactor: retention_enabled: true @@ -537,6 +533,7 @@ In this release the default has been changed to `0s`. A value of `0s` is the same as "retain forever" or "disable retention". If, **and only if**, you wish to retain the previous default of 744h, apply this config. + ```yaml limits_config: retention_period: 744h @@ -563,7 +560,7 @@ Instead, now you can use `splits` to see how many split by time intervals were c Currently not every query can be sharded and a shards value of zero is a good indicator the query was not able to be sharded. {{< /admonition >}} -### Promtail +### Promtail 2.8.0 #### The go build tag `promtail_journal_enabled` was introduced @@ -573,6 +570,7 @@ If you need Journal support you will need to run go build with tag `promtail_jou ```shell go build --tags=promtail_journal_enabled ./clients/cmd/promtail ``` + Introducing this tag aims to relieve Linux/CentOS users with CGO enabled from installing libsystemd-dev/systemd-devel libraries if they don't need Journal support. ### Ruler @@ -597,6 +595,7 @@ This is relevant only if you are using [jsonnet for deploying Loki in Kubernetes {{< /admonition >}} The `query-frontend` Kubernetes service was previously headless and was used for two purposes: + * Distributing the Loki query requests amongst all the available Query Frontend pods. * Discover IPs of Query Frontend pods from Queriers to connect as workers. @@ -604,14 +603,16 @@ The problem here is that a headless service does not support load balancing and Additionally, a load-balanced service does not let us discover the IPs of the underlying pods. To meet both these requirements, we have made the following changes: + * Changed the existing `query-frontend` Kubernetes service from headless to load-balanced to have a fair load distribution on all the Query Frontend instances. * Added `query-frontend-headless` to discover QF pod IPs from queriers to connect as workers. If you are deploying Loki with Query Scheduler by setting [query_scheduler_enabled](https://github.com/grafana/loki/blob/cc4ab7487ab3cd3b07c63601b074101b0324083b/production/ksonnet/loki/config.libsonnet#L18) config to `true`, then there is nothing to do here for this change. If you are not using Query Scheduler, then to avoid any issues on the Read path until the rollout finishes, it would be good to follow below steps: -* Create just the `query-frontend-headless` service without applying any changes to the `query-frontend` service. -* Rollout changes to `queriers`. -* Roll out the rest of the changes. + +- Create just the `query-frontend-headless` service without applying any changes to the `query-frontend` service. +- Rollout changes to `queriers`. +- Roll out the rest of the changes. ### General @@ -621,7 +622,7 @@ Statistics are now logged in `metrics.go` lines about how long it takes to downl Example (note the `*_download_time` fields): -``` +```bash level=info ts=2022-12-20T15:27:54.858554127Z caller=metrics.go:147 component=frontend org_id=docker latency=fast query="sum(count_over_time({job=\"generated-logs\"}[1h]))" query_type=metric range_type=range length=6h17m48.865587821s start_delta=6h17m54.858533178s end_delta=5.99294552s step=1m30s duration=5.990829396s status=200 limit=30 returned_lines=0 throughput=123MB total_bytes=738MB total_entries=1 store_chunks_download_time=2.319297059s queue_time=2m21.476090991s subqueries=8 cache_chunk_req=81143 cache_chunk_hit=32390 cache_chunk_bytes_stored=1874098 cache_chunk_bytes_fetched=94289610 cache_chunk_download_time=56.96914ms cache_index_req=994 cache_index_hit=710 cache_index_download_time=1.587842ms cache_result_req=7 cache_result_hit=0 cache_result_download_time=380.555”s ``` @@ -629,7 +630,7 @@ These statistics are also displayed when using `--stats` with LogCLI. ## 2.7.0 -### Loki +### Loki 2.7.0 ### Loki Canary Permission @@ -639,12 +640,12 @@ So if you run Loki behind some proxy with different authorization policies to re ### `engine.timeout` and `querier.query_timeout` are deprecated Previously, we had two configurations to define a query timeout: `engine.timeout` and `querier.query-timeout`. -As they were conflicting and `engine.timeout` isn't as expressive as `querier.query-tiomeout`, +As they were conflicting and `engine.timeout` isn't as expressive as `querier.query-timeout`, we're deprecating it and moving it to [Limits Config](/docs/loki//configuration/#limits_config) `limits_config.query_timeout` with same default values. #### `fifocache` has been renamed -The in-memory `fifocache` has been renamed to `embedded-cache`. This allows us to replace the implementation (currently a simple FIFO datastructure) with something else in the future without causing confusion +The in-memory `fifocache` has been renamed to `embedded-cache`. This allows us to replace the implementation (currently a simple FIFO data structure) with something else in the future without causing confusion #### Evenly spread Memcached pods for chunks across kubernetes nodes @@ -682,6 +683,7 @@ This value now defaults to 3100, so the Loki process doesn't require special pri The docker-compose [setup](https://github.com/grafana/loki/blob/main/production/docker) has been updated to **v2.6.0** and includes many improvements. Notable changes include: + - authentication (multi-tenancy) is **enabled** by default; you can disable it in `production/docker/config/loki.yaml` by setting `auth_enabled: false` - storage is now using Minio instead of local filesystem - move your current storage into `.data/minio` and it should work transparently @@ -715,7 +717,7 @@ ConfigDB was disallowed as a Ruler storage option back in 2.0. The config struct Can no longer specify a remote write client for the ruler. -### Promtail +### Promtail 2.7.0 #### `gcp_push_target_parsing_errors_total` has a new `reason` label @@ -723,15 +725,15 @@ The `gcp_push_target_parsing_errors_total` GCP Push Target metrics has been adde #### Windows event logs: now correctly includes `user_data` -The contents of the `user_data` field was erroneously set to the same value as `event_data` in previous versions. This was fixed in [#7461](https://github.com/grafana/loki/pull/7461) and log queries relying on this broken behaviour may be impacted. +The contents of the `user_data` field was erroneously set to the same value as `event_data` in previous versions. This was fixed in [#7461](https://github.com/grafana/loki/pull/7461) and log queries relying on this broken behavior may be impacted. ## 2.6.0 -### Loki +### Loki 2.6.0 #### Implementation of unwrapped `rate` aggregation changed -The implementation of the `rate()` aggregation function changed back to the previous implemention prior to [#5013](https://github.com/grafana/loki/pulls/5013). +The implementation of the `rate()` aggregation function changed back to the previous implementation prior to [#5013](https://github.com/grafana/loki/pulls/5013). This means that the rate per second is calculated based on the sum of the extracted values, instead of the average increase over time. If you want the extracted values to be treated as [Counter](https://prometheus.io/docs/concepts/metric_types/#counter) metric, you should use the new `rate_counter()` aggregation function, which calculates the per-second average rate of increase of the vector. @@ -742,9 +744,9 @@ This value now defaults to `loki`, it was previously set to `cortex`. If you are ## 2.5.0 -### Loki +### Loki 2.5.0 -#### `split_queries_by_interval` yaml configuration has moved. +#### `split_queries_by_interval` yaml configuration has moved It was previously possible to define this value in two places @@ -755,7 +757,7 @@ query_range: and/or -``` +```yaml limits_config: split_queries_by_interval: 10m ``` @@ -776,6 +778,7 @@ In case you're still using the legacy format, take a look at on how to write alerting rules in the new format. For reference, the newer format follows a structure similar to the one below: + ```yaml groups: - name: example @@ -790,6 +793,7 @@ For reference, the newer format follows a structure similar to the one below: ``` Meanwhile, the legacy format is a string in the following format: + ``` ALERT IF @@ -800,7 +804,7 @@ Meanwhile, the legacy format is a string in the following format: #### Changes to default configuration values -* `parallelise_shardable_queries` under the `query_range` config now defaults to `true`. +* `parallelize_shardable_queries` under the `query_range` config now defaults to `true`. * `split_queries_by_interval` under the `limits_config` config now defaults to `30m`, it was `0s`. * `max_chunk_age` in the `ingester` config now defaults to `2h` previously it was `1h`. * `query_ingesters_within` under the `querier` config now defaults to `3h`, previously it was `0s`. Any query (or subquery) that has an end time more than `3h` ago will not be sent to the ingesters, this saves work on the ingesters for data they normally don't contain. If you regularly write old data to Loki you may need to return this value to `0s` to always query ingesters. @@ -808,19 +812,18 @@ Meanwhile, the legacy format is a string in the following format: * `match_max_concurrent` under the `frontend_worker` config now defaults to true, this supersedes the `parallelism` setting which can now be removed from your config. Controlling query parallelism of a single process can now be done with the `querier` `max_concurrent` setting. * `flush_op_timeout` under the `ingester` configuration block now defaults to `10m`, increased from `10s`. This can help when replaying a large WAL on Loki startup, and avoid `msg="failed to flush" ... context deadline exceeded` errors. -### Promtail +### Promtail 2.5.0 #### `gcplog` labels have changed - - Resource labels have been moved from `__` to `__gcp_resource_labels_` - e.g. if you previously used `__project_id` then you'll need to update your relabel config to use `__gcp_resource_labels_project_id`. - - `resource_type` has been moved to `__gcp_resource_type` +- Resource labels have been moved from `__` to `__gcp_resource_labels_` for example, if you previously used `__project_id` then you'll need to update your relabel config to use `__gcp_resource_labels_project_id`. +- `resource_type` has been moved to `__gcp_resource_type` -#### `promtail_log_entries_bytes_bucket` histogram has been removed. +#### `promtail_log_entries_bytes_bucket` histogram has been removed This histogram reports the distribution of log line sizes by file. It has 8 buckets for every file being tailed. -This creates a lot of series and we don't think this metric has enough value to offset the amount of series genereated so we are removing it. +This creates a lot of series and we don't think this metric has enough value to offset the amount of series generated so we are removing it. While this isn't a direct replacement, two metrics we find more useful are size and line counters configured via pipeline stages, an example of how to configure these metrics can be found in the [metrics pipeline stage docs](https://grafana.com/docs/loki//send-data/promtail/stages/metrics/#counter). @@ -829,7 +832,7 @@ While this isn't a direct replacement, two metrics we find more useful are size If you have dashboards that depended on the log level, change them to search for the `msg="added Docker target"` property. -### Jsonnet +### Jsonnet 2.5.0 #### Compactor config defined as command line args moved to yaml config @@ -850,7 +853,7 @@ Following 2 compactor configs that were defined as command line arguments in jso The following are important changes which should be reviewed and understood prior to upgrading Loki. -### Loki +### Loki 2.4.0 The following changes pertain to upgrading Loki. @@ -869,7 +872,7 @@ Anyone in situation #2, you have two options, the first (and not recommended) is The second and recommended solution, is to use deletes via the compactor: -``` +```yaml compactor: retention_enabled: true limits_config: @@ -878,7 +881,7 @@ limits_config: See the [retention docs](https://grafana.com/docs/loki//operations/storage/retention/) for more info. -#### Log messages on startup: proto: duplicate proto type registered: +#### Log messages on startup: proto: duplicate proto type registered PR [#3842](https://github.com/grafana/loki/pull/3842) **cyriltovena**: Fork cortex chunk storage into Loki. @@ -886,7 +889,7 @@ Since Cortex doesn't plan to use the `chunk` package anymore, we decided to fork be able to evolve and modify it easily. However, as a side-effect, we still vendor Cortex which includes this forked code and protobuf files resulting in log messages like these at startup: -``` +```bash 2021-11-04 15:30:02.437911 I | proto: duplicate proto type registered: purgeplan.DeletePlan 2021-11-04 15:30:02.437936 I | proto: duplicate proto type registered: purgeplan.ChunksGroup 2021-11-04 15:30:02.437939 I | proto: duplicate proto type registered: purgeplan.ChunkDetails @@ -936,7 +939,7 @@ If you would like to disable these caches or change this memory limit: Disable: -``` +```yaml chunk_store_config: chunk_cache_config: enable_fifocache: false @@ -948,7 +951,7 @@ query_range: Resize: -``` +```yaml chunk_store_config: chunk_cache_config: enable_fifocache: true @@ -980,6 +983,7 @@ This changes a few default values, resulting in the ingester WAL now being on by and chunk transfer retries are disabled by default. Note, this now means Loki will depend on local disk by default for its WAL (write ahead log) directory. This defaults to `wal` but can be overridden via the `--ingester.wal-dir` or via `path_prefix` in the common configuration section. Below are config snippets with the previous defaults, and another with the new values. Previous defaults: + ```yaml ingester: max_transfer_retries: 10 @@ -988,6 +992,7 @@ ingester: ``` New defaults: + ```yaml ingester: max_transfer_retries: 0 @@ -998,6 +1003,7 @@ ingester: Using the write ahead log (WAL) is recommended and is now the default. However using the WAL is incompatible with chunk transfers, if you have explicitly configured `ingester.max-transfer-retries` to a non-zero value, you must set it to 0 to disable transfers. #### Memberlist config now automatically applies to all non-configured rings + * [4400](https://github.com/grafana/loki/pull/4400) **trevorwhitney**: Config: automatically apply memberlist config too all rings when provided This change affects the behavior of the ingester, distributor, and ruler rings. Previously, if you wanted to use memberlist for all of these rings, you @@ -1040,19 +1046,22 @@ ruler: ``` #### Changed defaults for some GRPC server settings + * [4435](https://github.com/grafana/loki/pull/4435) **trevorwhitney**: Change default values for two GRPC settings so querier can connect to frontend/scheduler This changes two default values, `grpc_server_min_time_between_pings` and `grpc_server_ping_without_stream_allowed` used by the GRPC server. *Previous Values*: -``` + +```yaml server: grpc_server_min_time_between_pings: '5m' grpc_server_ping_without_stream_allowed: false ``` *New Values*: -``` + +```yaml server: grpc_server_min_time_between_pings: '10s' grpc_server_ping_without_stream_allowed: true @@ -1064,7 +1073,7 @@ server: * [#3842](https://github.com/grafana/loki/pull/3842)/[#4253](https://github.com/grafana/loki/pull/4253) **jordanrushing**: Metrics related to chunk storage and runtime config have changed their prefixes from `cortex_` to `loki_`. -``` +```yaml cortex_runtime_config* -> loki_runtime_config* cortex_chunks_store* -> loki_chunks_store* ``` @@ -1080,17 +1089,17 @@ per-tenant WAL can be found [here](/docs/loki//operations/recordin The `ruler` now requires persistent storage - see the [Operations](/docs/loki//operations/recording-rules/#deployment) page for more details about deployment. -### Promtail +### Promtail 2.4.0 The following changes pertain to upgrading Promtail. #### Promtail no longer insert `promtail_instance` label when scraping `gcplog` target -* [4556](https://github.com/grafana/loki/pull/4556) **james-callahan**: Remove `promtail_instance` label that was being added by promtail when scraping `gcplog` target. +* [4556](https://github.com/grafana/loki/pull/4556) **james-callahan**: Remove `promtail_instance` label that was being added by promtail when scraping `gcplog` target. ## 2.3.0 -### Loki +### Loki 2.3.0 #### Query restriction introduced for queries which do not have at least one equality matcher @@ -1114,10 +1123,9 @@ This difference may seem subtle but if we break it down `.` matches any characte The reasoning for this change has to do with how index lookups work in Loki, if you don't have at least one equality matcher Loki has to perform a complete index table scan which is an expensive and slow operation. - ## 2.2.0 -### Loki +### Loki 2.2.0 **Be sure to upgrade to 2.0 or 2.1 BEFORE upgrading to 2.2** @@ -1128,7 +1136,7 @@ In Loki 2.2 we changed the internal version of our chunk format from v2 to v3, t This makes it important to first upgrade to 2.0, 2.0.1, or 2.1 **before** upgrading to 2.2 so that if you need to rollback for any reason you can do so easily. {{< admonition type="note" >}} -2.0 and 2.0.1 are identical in every aspect except 2.0.1 contains the code necessary to read the v3 chunk format. Therefor if you are on 2.0 and ugrade to 2.2, if you want to rollback, you must rollback to 2.0.1. +2.0 and 2.0.1 are identical in every aspect except 2.0.1 contains the code necessary to read the v3 chunk format. Therefor if you are on 2.0 and upgrade to 2.2, if you want to rollback, you must rollback to 2.0.1. {{< /admonition >}} ### Loki Config @@ -1147,9 +1155,7 @@ You could consider multiplying your current `max_query_parallelism` setting by 1 **Also be aware to make sure `max_outstanding_per_tenant` is always greater than `max_query_parallelism` or large queries will automatically fail with a 429 back to the user.** - - -### Promtail +### Promtail 2.2.0 For 2.0 we eliminated the long deprecated `entry_parser` configuration in Promtail configs, however in doing so we introduced a very confusing and erroneous default behavior: @@ -1157,7 +1163,7 @@ If you did not specify a `pipeline_stages` entry you would be provided with a de In [3404](https://github.com/grafana/loki/pull/3404), we corrected this behavior -**If you are using docker, and any of your `scrape_configs` are missing a `pipeline_stages` definition**, you should add the following to obtain the correct behaviour: +**If you are using docker, and any of your `scrape_configs` are missing a `pipeline_stages` definition**, you should add the following to obtain the correct behavior: ```yaml pipeline_stages: @@ -1168,15 +1174,15 @@ pipeline_stages: The upgrade from 2.0.0 to 2.1.0 should be fairly smooth, be aware of these two things: -### Helm charts have moved! +### Helm charts have moved -Helm charts are now located at: https://github.com/grafana/helm-charts/ +Helm charts are now located in the [Grafana Helm charts repo](https://github.com/grafana/helm-charts/). -The helm repo URL is now: https://grafana.github.io/helm-charts +The helm repo URL is now: [https://grafana.github.io/helm-charts](https://grafana.github.io/helm-charts). ### Fluent Bit plugin renamed -Fluent bit officially supports Loki as an output plugin now! WoooHOOO! +Fluent bit officially supports Loki as an output plugin now. However this created a naming conflict with our existing output plugin (the new native output uses the name `loki`) so we have renamed our plugin. @@ -1184,14 +1190,14 @@ In time our plan is to deprecate and eliminate our output plugin in favor of the Old: -``` +```shell [Output] Name loki ``` New: -``` +```shell [Output] Name grafana-loki ``` @@ -1215,11 +1221,12 @@ For the most part, there are very few impactful changes and for most this will b The default config file in the docker image, as well as the default helm values.yaml and jsonnet for Tanka all specify a schema definition to make things easier to get started. ->**If you have not specified your own config file with your own schema definition (or you do not have a custom schema definition in your values.yaml), upgrading to 2.0 will break things!** +{{< admonition type="caution >}} +If you have not specified your own config file with your own schema definition (or you do not have a custom schema definition in your values.yaml), upgrading to 2.0 will break things! +{{< /admonition >}} In 2.0 the defaults are now v11 schema and the `boltdb-shipper` index type. - If you are using an index type of `aws`, `bigtable`, or `cassandra` this means you have already defined a custom schema and there is _nothing_ further you need to do regarding the schema. You can consider however adding a new schema entry to use the new `boltdb-shipper` type if you want to move away from these separate index stores and instead use just one object store. @@ -1277,14 +1284,13 @@ This likely only affects a small portion of tanka users because the default sche If you had set `index_period_hours` to a value other than 168h (the previous default) you must update this in the above config `period:` to match what you chose. {{< /admonition >}} - {{< admonition type="note" >}} We have changed the default index store to `boltdb-shipper` it's important to add `using_boltdb_shipper: false,` until you are ready to change (if you want to change) {{< /admonition >}} Changing the jsonnet config to use the `boltdb-shipper` type is the same as [below](#upgrading-schema-to-use-boltdb-shipper-andor-v11-schema) where you need to add a new schema section. -**HOWEVER** Be aware when you change `using_boltdb_shipper: true` the deployment type for the ingesters and queriers will change to statefulsets! Statefulsets are required for the ingester and querier using boltdb-shipper. +**HOWEVER** Be aware when you change `using_boltdb_shipper: true` the deployment type for the ingesters and queriers will change to StatefulSets! StatefulSets are required for the ingester and querier using boltdb-shipper. ##### Docker (e.g. docker-compose) @@ -1300,8 +1306,7 @@ docker run -d --name=loki --mount type=bind,source="path to loki-config.yaml",ta The Loki docker image is expecting to find the config file at `/etc/loki/local-config.yaml` - -### IMPORTANT: boltdb-shipper upgrade considerations. +### IMPORTANT: boltdb-shipper upgrade considerations Significant changes have taken place between 1.6.0 and 2.0.0 for boltdb-shipper index type, if you are already running this index and are upgrading some extra caution is warranted. @@ -1313,6 +1318,7 @@ The chunks directory should not need any special backups. If you have an environment to test this in, do so before upgrading against critical data. There are 2 significant changes warranting the backup of this data because they will make rolling back impossible: + * A compactor is included which will take existing index files and compact them to one per day and remove non compacted files * All index files are now gzipped before uploading @@ -1336,7 +1342,7 @@ Ingesters now expose a new RPC method that queriers use when the index type is ` Queriers generally roll out faster than ingesters, so if new queriers query older ingesters using the new RPC, the queries would fail. To avoid any query downtime during the upgrade, rollout ingesters before queriers. -#### If running the compactor, ensure it has delete permissions for the object storage. +#### If running the compactor, ensure it has delete permissions for the object storage The compactor is an optional but suggested component that combines and deduplicates the boltdb-shipper index files. When compacting index files, the compactor writes a new file and deletes unoptimized files. Ensure that the compactor has appropriate permissions for deleting files, for example, s3:DeleteObject permission for AWS S3. @@ -1375,6 +1381,7 @@ schema_config: prefix: index_ period: 24h â‘€ ``` + ① Make sure all of these match your current schema config ② Make sure this matches your previous schema version, Helm for example is likely v9 ⑱ Make sure this is a date in the **FUTURE** keep in mind Loki only knows UTC so make sure it's a future UTC date @@ -1383,7 +1390,6 @@ schema_config: There are more examples on the [Storage description page](https://grafana.com/docs/loki//configure/storage/#examples) including the information you need to setup the `storage` section for boltdb-shipper. - ## 1.6.0 ### Important: Ksonnet port changed and removed NET_BIND_SERVICE capability from Docker image @@ -1398,11 +1404,10 @@ the capability was removed. It is now no longer possible for the Loki to be started with a port less than 1024 with the published docker image. -The default for Helm has always been port 3100, and Helm users should be unaffect unless they changed the default. +The default for Helm has always been port 3100, and Helm users should be unaffected unless they changed the default. **Ksonnet users however should closely check their configuration, in PR 2294 the loki port was changed from 80 to 3100** - ### IMPORTANT: If you run Loki in microservices mode, special rollout instructions A new ingester GRPC API has been added allowing to speed up metric queries, to ensure a rollout without query errors **make sure you upgrade all ingesters first.** @@ -1414,10 +1419,9 @@ This will only affect reads(queries) and not writes and only for the duration of ### IMPORTANT: Scrape config changes to both Helm and Ksonnet will affect labels created by Promtail -PR [2091](https://github.com/grafana/loki/pull/2091) Makes several changes to the Promtail scrape config: +Loki PR [2091](https://github.com/grafana/loki/pull/2091) makes several changes to the Promtail scrape config: -```` -This is triggered by https://github.com/grafana/jsonnet-libs/pull/261 +This is triggered by jsonnet-libs PR [261](https://github.com/grafana/jsonnet-libs/pull/261): The above PR changes the instance label to be actually unique within a scrape config. It also adds a pod and a container target label @@ -1429,7 +1433,6 @@ the container_name label. It is the same as the container label and was already added to Loki previously. However, the container_name label is deprecated and has disappeared in Kubernetes 1.16, so that it will soon become useless for direct joining. -```` TL;DR @@ -1438,7 +1441,6 @@ The following label have been changed in both the Helm and Ksonnet Promtail scra `instance` -> `pod` `container_name` -> `container` - ### Experimental boltdb-shipper changes PR [2166](https://github.com/grafana/loki/pull/2166) now forces the index to have a period of exactly `24h`: @@ -1465,6 +1467,7 @@ schema_config: prefix: index_ period: 24h <--- This must be 24h ``` + If you are not on `schema: v11` this would be a good opportunity to make that change _in the new schema config_ also. {{< admonition type="note" >}} @@ -1532,7 +1535,7 @@ Defaulting to `gcs,bigtable` was confusing for anyone using ksonnet with other s The required upgrade path outlined for version 1.4.0 below is still true for moving to 1.5.0 from any release older than 1.4.0 (e.g. 1.3.0 -> 1.5.0 needs to also look at the 1.4.0 upgrade requirements). {{< /admonition >}} -### Breaking config changes! +### Breaking config changes Loki 1.5.0 vendors Cortex v1.0.0 (congratulations!), which has a [massive list of changes](https://cortexmetrics.io/docs/changelog/#1-0-0-2020-04-02). @@ -1563,8 +1566,7 @@ Referencing the [list of diffs](https://cortexmetrics.io/docs/changelog/#config- + dynamodb: ``` -Also several other AWS related configs changed and would need to udpate those as well. - +Also several other AWS related configs changed and would need to update those as well. ### Loki Docker Image User and File Location Changes @@ -1590,14 +1592,14 @@ The location Loki is looking for files with the provided config in the docker im In 1.4.0 and earlier the included config file in the docker container was using directories: -``` +```shell /tmp/loki/index /tmp/loki/chunks ``` In 1.5.0 this has changed: -``` +```shell /loki/index /loki/chunks ``` @@ -1614,7 +1616,7 @@ This would mount a docker volume named `loki-data` to the `/tmp/loki` folder whi To move to 1.5.0 I can do the following (note that your container names and paths and volumes etc may be different): -``` +```bash docker stop loki docker rm loki docker run --rm --name="loki-perm" -it --mount source=loki-data,target=/mnt ubuntu /bin/bash @@ -1629,7 +1631,6 @@ Notice the change in the `target=/loki` for 1.5.0 to the new data directory loca The intermediate step of using an ubuntu image to change the ownership of the Loki files to the new user might not be necessary if you can easily access these files to run the `chown` command directly. That is if you have access to `/var/lib/docker/volumes` or if you mounted to a different local filesystem directory, you can change the ownership directly without using a container. - ### Loki Duration Configs If you get an error like: From 03fa28e3f8a7b09e885b2e150afdb228fec71d2e Mon Sep 17 00:00:00 2001 From: J Stickler Date: Wed, 4 Dec 2024 11:53:43 -0500 Subject: [PATCH 32/44] docs: fixing broken admonition (#15261) --- docs/sources/setup/upgrade/_index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/setup/upgrade/_index.md b/docs/sources/setup/upgrade/_index.md index 546d40f77d32e..3ecf9107f91ac 100644 --- a/docs/sources/setup/upgrade/_index.md +++ b/docs/sources/setup/upgrade/_index.md @@ -1221,7 +1221,7 @@ For the most part, there are very few impactful changes and for most this will b The default config file in the docker image, as well as the default helm values.yaml and jsonnet for Tanka all specify a schema definition to make things easier to get started. -{{< admonition type="caution >}} +{{< admonition type="caution" >}} If you have not specified your own config file with your own schema definition (or you do not have a custom schema definition in your values.yaml), upgrading to 2.0 will break things! {{< /admonition >}} From acd4d112b0721a2c24fbfad6c5c317f33b07c3b4 Mon Sep 17 00:00:00 2001 From: mericks Date: Wed, 4 Dec 2024 12:54:38 -0800 Subject: [PATCH 33/44] fix(helm): yaml templates incorrectly annotated (#15207) --- .../helm/loki/templates/compactor/statefulset-compactor.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/production/helm/loki/templates/compactor/statefulset-compactor.yaml b/production/helm/loki/templates/compactor/statefulset-compactor.yaml index 62944882a79ee..ded01be3c62dc 100644 --- a/production/helm/loki/templates/compactor/statefulset-compactor.yaml +++ b/production/helm/loki/templates/compactor/statefulset-compactor.yaml @@ -51,7 +51,7 @@ spec: {{- if semverCompare ">=1.19-0" .Capabilities.KubeVersion.Version }} {{- with .Values.compactor.topologySpreadConstraints }} topologySpreadConstraints: - {{- tpl . $ | nindent 8 }} + {{- toYaml . | nindent 8 }} {{- end }} {{- end }} serviceAccountName: {{ include "loki.serviceAccountName" . }} From 818446c4234384cec8feef93690574d4f68b7c77 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Wed, 4 Dec 2024 16:13:58 -0500 Subject: [PATCH 34/44] fix(deps): update module golang.org/x/text to v0.21.0 (#15266) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- vendor/modules.txt | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index a6eb23369c6e4..4129e73d8df14 100644 --- a/go.mod +++ b/go.mod @@ -145,7 +145,7 @@ require ( go.opentelemetry.io/collector/pdata v1.21.0 go4.org/netipx v0.0.0-20230125063823-8449b0a6169f golang.org/x/oauth2 v0.24.0 - golang.org/x/text v0.20.0 + golang.org/x/text v0.21.0 google.golang.org/protobuf v1.35.2 gotest.tools v2.2.0+incompatible k8s.io/apimachinery v0.31.3 diff --git a/go.sum b/go.sum index cd965701b4da8..962a8a24570fb 100644 --- a/go.sum +++ b/go.sum @@ -3293,8 +3293,8 @@ golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= -golang.org/x/text v0.20.0 h1:gK/Kv2otX8gz+wn7Rmb3vT96ZwuoxnQlY+HlJVj7Qug= -golang.org/x/text v0.20.0/go.mod h1:D4IsuqiFMhST5bX19pQ9ikHC2GsaKyk/oF+pn3ducp4= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= diff --git a/vendor/modules.txt b/vendor/modules.txt index bab919649f522..cec029a516fc8 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1886,7 +1886,7 @@ golang.org/x/sys/windows/svc/eventlog # golang.org/x/term v0.26.0 ## explicit; go 1.18 golang.org/x/term -# golang.org/x/text v0.20.0 +# golang.org/x/text v0.21.0 ## explicit; go 1.18 golang.org/x/text/cases golang.org/x/text/encoding From f57ea87ae64714cab6a1802cb8d8106c235d6d3c Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Wed, 4 Dec 2024 16:14:47 -0500 Subject: [PATCH 35/44] fix(deps): update module golang.org/x/sys to v0.28.0 (#15254) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 +- vendor/golang.org/x/sys/unix/zerrors_linux.go | 9 +++ .../x/sys/unix/zerrors_linux_386.go | 6 ++ .../x/sys/unix/zerrors_linux_amd64.go | 6 ++ .../x/sys/unix/zerrors_linux_arm.go | 6 ++ .../x/sys/unix/zerrors_linux_arm64.go | 7 +++ .../x/sys/unix/zerrors_linux_loong64.go | 6 ++ .../x/sys/unix/zerrors_linux_mips.go | 6 ++ .../x/sys/unix/zerrors_linux_mips64.go | 6 ++ .../x/sys/unix/zerrors_linux_mips64le.go | 6 ++ .../x/sys/unix/zerrors_linux_mipsle.go | 6 ++ .../x/sys/unix/zerrors_linux_ppc.go | 6 ++ .../x/sys/unix/zerrors_linux_ppc64.go | 6 ++ .../x/sys/unix/zerrors_linux_ppc64le.go | 6 ++ .../x/sys/unix/zerrors_linux_riscv64.go | 6 ++ .../x/sys/unix/zerrors_linux_s390x.go | 6 ++ .../x/sys/unix/zerrors_linux_sparc64.go | 6 ++ .../x/sys/unix/ztypes_darwin_amd64.go | 60 +++++++++++++++++++ .../x/sys/unix/ztypes_darwin_arm64.go | 60 +++++++++++++++++++ vendor/golang.org/x/sys/unix/ztypes_linux.go | 20 ++++--- .../x/sys/windows/syscall_windows.go | 2 + .../golang.org/x/sys/windows/types_windows.go | 1 + .../x/sys/windows/zsyscall_windows.go | 28 +++++++-- vendor/modules.txt | 2 +- 25 files changed, 261 insertions(+), 18 deletions(-) diff --git a/go.mod b/go.mod index 4129e73d8df14..f4729e824d1bc 100644 --- a/go.mod +++ b/go.mod @@ -101,7 +101,7 @@ require ( golang.org/x/crypto v0.29.0 golang.org/x/net v0.31.0 golang.org/x/sync v0.10.0 - golang.org/x/sys v0.27.0 + golang.org/x/sys v0.28.0 golang.org/x/time v0.8.0 google.golang.org/api v0.209.0 google.golang.org/grpc v1.68.0 diff --git a/go.sum b/go.sum index 962a8a24570fb..caecf10edf400 100644 --- a/go.sum +++ b/go.sum @@ -3248,8 +3248,8 @@ golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.19.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.27.0 h1:wBqf8DvsY9Y/2P8gAfPDEYNuS30J4lPHJxXSb/nJZ+s= -golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux.go b/vendor/golang.org/x/sys/unix/zerrors_linux.go index ccba391c9fb09..6ebc48b3fecd7 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux.go @@ -321,6 +321,9 @@ const ( AUDIT_INTEGRITY_STATUS = 0x70a AUDIT_IPC = 0x517 AUDIT_IPC_SET_PERM = 0x51f + AUDIT_IPE_ACCESS = 0x58c + AUDIT_IPE_CONFIG_CHANGE = 0x58d + AUDIT_IPE_POLICY_LOAD = 0x58e AUDIT_KERNEL = 0x7d0 AUDIT_KERNEL_OTHER = 0x524 AUDIT_KERN_MODULE = 0x532 @@ -489,6 +492,7 @@ const ( BPF_F_ID = 0x20 BPF_F_NETFILTER_IP_DEFRAG = 0x1 BPF_F_QUERY_EFFECTIVE = 0x1 + BPF_F_REDIRECT_FLAGS = 0x19 BPF_F_REPLACE = 0x4 BPF_F_SLEEPABLE = 0x10 BPF_F_STRICT_ALIGNMENT = 0x1 @@ -1166,6 +1170,7 @@ const ( EXTA = 0xe EXTB = 0xf F2FS_SUPER_MAGIC = 0xf2f52010 + FALLOC_FL_ALLOCATE_RANGE = 0x0 FALLOC_FL_COLLAPSE_RANGE = 0x8 FALLOC_FL_INSERT_RANGE = 0x20 FALLOC_FL_KEEP_SIZE = 0x1 @@ -1799,6 +1804,8 @@ const ( LANDLOCK_ACCESS_NET_BIND_TCP = 0x1 LANDLOCK_ACCESS_NET_CONNECT_TCP = 0x2 LANDLOCK_CREATE_RULESET_VERSION = 0x1 + LANDLOCK_SCOPE_ABSTRACT_UNIX_SOCKET = 0x1 + LANDLOCK_SCOPE_SIGNAL = 0x2 LINUX_REBOOT_CMD_CAD_OFF = 0x0 LINUX_REBOOT_CMD_CAD_ON = 0x89abcdef LINUX_REBOOT_CMD_HALT = 0xcdef0123 @@ -1924,6 +1931,7 @@ const ( MNT_FORCE = 0x1 MNT_ID_REQ_SIZE_VER0 = 0x18 MNT_ID_REQ_SIZE_VER1 = 0x20 + MNT_NS_INFO_SIZE_VER0 = 0x10 MODULE_INIT_COMPRESSED_FILE = 0x4 MODULE_INIT_IGNORE_MODVERSIONS = 0x1 MODULE_INIT_IGNORE_VERMAGIC = 0x2 @@ -2970,6 +2978,7 @@ const ( RWF_WRITE_LIFE_NOT_SET = 0x0 SCHED_BATCH = 0x3 SCHED_DEADLINE = 0x6 + SCHED_EXT = 0x7 SCHED_FIFO = 0x1 SCHED_FLAG_ALL = 0x7f SCHED_FLAG_DL_OVERRUN = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_386.go b/vendor/golang.org/x/sys/unix/zerrors_linux_386.go index 0c00cb3f3af83..c0d45e320505f 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_386.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_386.go @@ -109,6 +109,7 @@ const ( HIDIOCGRAWINFO = 0x80084803 HIDIOCGRDESC = 0x90044802 HIDIOCGRDESCSIZE = 0x80044801 + HIDIOCREVOKE = 0x4004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x8000 @@ -297,6 +298,8 @@ const ( RTC_WIE_ON = 0x700f RTC_WKALM_RD = 0x80287010 RTC_WKALM_SET = 0x4028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -335,6 +338,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_amd64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_amd64.go index dfb364554dd58..c731d24f02529 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_amd64.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_amd64.go @@ -109,6 +109,7 @@ const ( HIDIOCGRAWINFO = 0x80084803 HIDIOCGRDESC = 0x90044802 HIDIOCGRDESCSIZE = 0x80044801 + HIDIOCREVOKE = 0x4004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x8000 @@ -298,6 +299,8 @@ const ( RTC_WIE_ON = 0x700f RTC_WKALM_RD = 0x80287010 RTC_WKALM_SET = 0x4028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -336,6 +339,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_arm.go b/vendor/golang.org/x/sys/unix/zerrors_linux_arm.go index d46dcf78abc9a..680018a4a7c9f 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_arm.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_arm.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x80084803 HIDIOCGRDESC = 0x90044802 HIDIOCGRDESCSIZE = 0x80044801 + HIDIOCREVOKE = 0x4004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x8000 @@ -303,6 +304,8 @@ const ( RTC_WIE_ON = 0x700f RTC_WKALM_RD = 0x80287010 RTC_WKALM_SET = 0x4028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -341,6 +344,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_arm64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_arm64.go index 3af3248a7f2e1..a63909f308d6d 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_arm64.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_arm64.go @@ -112,6 +112,7 @@ const ( HIDIOCGRAWINFO = 0x80084803 HIDIOCGRDESC = 0x90044802 HIDIOCGRDESCSIZE = 0x80044801 + HIDIOCREVOKE = 0x4004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x8000 @@ -205,6 +206,7 @@ const ( PERF_EVENT_IOC_SET_BPF = 0x40042408 PERF_EVENT_IOC_SET_FILTER = 0x40082406 PERF_EVENT_IOC_SET_OUTPUT = 0x2405 + POE_MAGIC = 0x504f4530 PPPIOCATTACH = 0x4004743d PPPIOCATTCHAN = 0x40047438 PPPIOCBRIDGECHAN = 0x40047435 @@ -294,6 +296,8 @@ const ( RTC_WIE_ON = 0x700f RTC_WKALM_RD = 0x80287010 RTC_WKALM_SET = 0x4028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -332,6 +336,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_loong64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_loong64.go index 292bcf0283d17..9b0a2573fe3fb 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_loong64.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_loong64.go @@ -109,6 +109,7 @@ const ( HIDIOCGRAWINFO = 0x80084803 HIDIOCGRDESC = 0x90044802 HIDIOCGRDESCSIZE = 0x80044801 + HIDIOCREVOKE = 0x4004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x8000 @@ -290,6 +291,8 @@ const ( RTC_WIE_ON = 0x700f RTC_WKALM_RD = 0x80287010 RTC_WKALM_SET = 0x4028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -328,6 +331,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_mips.go b/vendor/golang.org/x/sys/unix/zerrors_linux_mips.go index 782b7110fa19b..958e6e0645acd 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_mips.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_mips.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x40084803 HIDIOCGRDESC = 0x50044802 HIDIOCGRDESCSIZE = 0x40044801 + HIDIOCREVOKE = 0x8004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x100 @@ -296,6 +297,8 @@ const ( RTC_WIE_ON = 0x2000700f RTC_WKALM_RD = 0x40287010 RTC_WKALM_SET = 0x8028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -334,6 +337,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x1029 SO_DONTROUTE = 0x10 SO_ERROR = 0x1007 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_mips64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_mips64.go index 84973fd9271fb..50c7f25bd16c6 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_mips64.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_mips64.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x40084803 HIDIOCGRDESC = 0x50044802 HIDIOCGRDESCSIZE = 0x40044801 + HIDIOCREVOKE = 0x8004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x100 @@ -296,6 +297,8 @@ const ( RTC_WIE_ON = 0x2000700f RTC_WKALM_RD = 0x40287010 RTC_WKALM_SET = 0x8028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -334,6 +337,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x1029 SO_DONTROUTE = 0x10 SO_ERROR = 0x1007 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_mips64le.go b/vendor/golang.org/x/sys/unix/zerrors_linux_mips64le.go index 6d9cbc3b274bf..ced21d66d955a 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_mips64le.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_mips64le.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x40084803 HIDIOCGRDESC = 0x50044802 HIDIOCGRDESCSIZE = 0x40044801 + HIDIOCREVOKE = 0x8004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x100 @@ -296,6 +297,8 @@ const ( RTC_WIE_ON = 0x2000700f RTC_WKALM_RD = 0x40287010 RTC_WKALM_SET = 0x8028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -334,6 +337,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x1029 SO_DONTROUTE = 0x10 SO_ERROR = 0x1007 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_mipsle.go b/vendor/golang.org/x/sys/unix/zerrors_linux_mipsle.go index 5f9fedbce028d..226c044190235 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_mipsle.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_mipsle.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x40084803 HIDIOCGRDESC = 0x50044802 HIDIOCGRDESCSIZE = 0x40044801 + HIDIOCREVOKE = 0x8004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x100 @@ -296,6 +297,8 @@ const ( RTC_WIE_ON = 0x2000700f RTC_WKALM_RD = 0x40287010 RTC_WKALM_SET = 0x8028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -334,6 +337,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x1029 SO_DONTROUTE = 0x10 SO_ERROR = 0x1007 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc.go b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc.go index bb0026ee0c467..3122737cd464f 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x40084803 HIDIOCGRDESC = 0x50044802 HIDIOCGRDESCSIZE = 0x40044801 + HIDIOCREVOKE = 0x8004480d HUPCL = 0x4000 ICANON = 0x100 IEXTEN = 0x400 @@ -351,6 +352,8 @@ const ( RTC_WIE_ON = 0x2000700f RTC_WKALM_RD = 0x40287010 RTC_WKALM_SET = 0x8028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -389,6 +392,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64.go index 46120db5c9a19..eb5d3467edf0c 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x40084803 HIDIOCGRDESC = 0x50044802 HIDIOCGRDESCSIZE = 0x40044801 + HIDIOCREVOKE = 0x8004480d HUPCL = 0x4000 ICANON = 0x100 IEXTEN = 0x400 @@ -355,6 +356,8 @@ const ( RTC_WIE_ON = 0x2000700f RTC_WKALM_RD = 0x40287010 RTC_WKALM_SET = 0x8028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -393,6 +396,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64le.go b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64le.go index 5c951634fbed3..e921ebc60b714 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64le.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64le.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x40084803 HIDIOCGRDESC = 0x50044802 HIDIOCGRDESCSIZE = 0x40044801 + HIDIOCREVOKE = 0x8004480d HUPCL = 0x4000 ICANON = 0x100 IEXTEN = 0x400 @@ -355,6 +356,8 @@ const ( RTC_WIE_ON = 0x2000700f RTC_WKALM_RD = 0x40287010 RTC_WKALM_SET = 0x8028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -393,6 +396,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_riscv64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_riscv64.go index 11a84d5af2083..38ba81c55c1fd 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_riscv64.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_riscv64.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x80084803 HIDIOCGRDESC = 0x90044802 HIDIOCGRDESCSIZE = 0x80044801 + HIDIOCREVOKE = 0x4004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x8000 @@ -287,6 +288,8 @@ const ( RTC_WIE_ON = 0x700f RTC_WKALM_RD = 0x80287010 RTC_WKALM_SET = 0x4028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -325,6 +328,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_s390x.go b/vendor/golang.org/x/sys/unix/zerrors_linux_s390x.go index f78c4617cac1f..71f0400977b36 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_s390x.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_s390x.go @@ -108,6 +108,7 @@ const ( HIDIOCGRAWINFO = 0x80084803 HIDIOCGRDESC = 0x90044802 HIDIOCGRDESCSIZE = 0x80044801 + HIDIOCREVOKE = 0x4004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x8000 @@ -359,6 +360,8 @@ const ( RTC_WIE_ON = 0x700f RTC_WKALM_RD = 0x80287010 RTC_WKALM_SET = 0x4028700f + SCM_DEVMEM_DMABUF = 0x4f + SCM_DEVMEM_LINEAR = 0x4e SCM_TIMESTAMPING = 0x25 SCM_TIMESTAMPING_OPT_STATS = 0x36 SCM_TIMESTAMPING_PKTINFO = 0x3a @@ -397,6 +400,9 @@ const ( SO_CNX_ADVICE = 0x35 SO_COOKIE = 0x39 SO_DETACH_REUSEPORT_BPF = 0x44 + SO_DEVMEM_DMABUF = 0x4f + SO_DEVMEM_DONTNEED = 0x50 + SO_DEVMEM_LINEAR = 0x4e SO_DOMAIN = 0x27 SO_DONTROUTE = 0x5 SO_ERROR = 0x4 diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_sparc64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_sparc64.go index aeb777c34427d..c44a313322c54 100644 --- a/vendor/golang.org/x/sys/unix/zerrors_linux_sparc64.go +++ b/vendor/golang.org/x/sys/unix/zerrors_linux_sparc64.go @@ -112,6 +112,7 @@ const ( HIDIOCGRAWINFO = 0x40084803 HIDIOCGRDESC = 0x50044802 HIDIOCGRDESCSIZE = 0x40044801 + HIDIOCREVOKE = 0x8004480d HUPCL = 0x400 ICANON = 0x2 IEXTEN = 0x8000 @@ -350,6 +351,8 @@ const ( RTC_WIE_ON = 0x2000700f RTC_WKALM_RD = 0x40287010 RTC_WKALM_SET = 0x8028700f + SCM_DEVMEM_DMABUF = 0x58 + SCM_DEVMEM_LINEAR = 0x57 SCM_TIMESTAMPING = 0x23 SCM_TIMESTAMPING_OPT_STATS = 0x38 SCM_TIMESTAMPING_PKTINFO = 0x3c @@ -436,6 +439,9 @@ const ( SO_CNX_ADVICE = 0x37 SO_COOKIE = 0x3b SO_DETACH_REUSEPORT_BPF = 0x47 + SO_DEVMEM_DMABUF = 0x58 + SO_DEVMEM_DONTNEED = 0x59 + SO_DEVMEM_LINEAR = 0x57 SO_DOMAIN = 0x1029 SO_DONTROUTE = 0x10 SO_ERROR = 0x1007 diff --git a/vendor/golang.org/x/sys/unix/ztypes_darwin_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_darwin_amd64.go index d003c3d43780c..17c53bd9b3315 100644 --- a/vendor/golang.org/x/sys/unix/ztypes_darwin_amd64.go +++ b/vendor/golang.org/x/sys/unix/ztypes_darwin_amd64.go @@ -462,11 +462,14 @@ type FdSet struct { const ( SizeofIfMsghdr = 0x70 + SizeofIfMsghdr2 = 0xa0 SizeofIfData = 0x60 + SizeofIfData64 = 0x80 SizeofIfaMsghdr = 0x14 SizeofIfmaMsghdr = 0x10 SizeofIfmaMsghdr2 = 0x14 SizeofRtMsghdr = 0x5c + SizeofRtMsghdr2 = 0x5c SizeofRtMetrics = 0x38 ) @@ -480,6 +483,20 @@ type IfMsghdr struct { Data IfData } +type IfMsghdr2 struct { + Msglen uint16 + Version uint8 + Type uint8 + Addrs int32 + Flags int32 + Index uint16 + Snd_len int32 + Snd_maxlen int32 + Snd_drops int32 + Timer int32 + Data IfData64 +} + type IfData struct { Type uint8 Typelen uint8 @@ -512,6 +529,34 @@ type IfData struct { Reserved2 uint32 } +type IfData64 struct { + Type uint8 + Typelen uint8 + Physical uint8 + Addrlen uint8 + Hdrlen uint8 + Recvquota uint8 + Xmitquota uint8 + Unused1 uint8 + Mtu uint32 + Metric uint32 + Baudrate uint64 + Ipackets uint64 + Ierrors uint64 + Opackets uint64 + Oerrors uint64 + Collisions uint64 + Ibytes uint64 + Obytes uint64 + Imcasts uint64 + Omcasts uint64 + Iqdrops uint64 + Noproto uint64 + Recvtiming uint32 + Xmittiming uint32 + Lastchange Timeval32 +} + type IfaMsghdr struct { Msglen uint16 Version uint8 @@ -557,6 +602,21 @@ type RtMsghdr struct { Rmx RtMetrics } +type RtMsghdr2 struct { + Msglen uint16 + Version uint8 + Type uint8 + Index uint16 + Flags int32 + Addrs int32 + Refcnt int32 + Parentflags int32 + Reserved int32 + Use int32 + Inits uint32 + Rmx RtMetrics +} + type RtMetrics struct { Locks uint32 Mtu uint32 diff --git a/vendor/golang.org/x/sys/unix/ztypes_darwin_arm64.go b/vendor/golang.org/x/sys/unix/ztypes_darwin_arm64.go index 0d45a941aaecc..2392226a743eb 100644 --- a/vendor/golang.org/x/sys/unix/ztypes_darwin_arm64.go +++ b/vendor/golang.org/x/sys/unix/ztypes_darwin_arm64.go @@ -462,11 +462,14 @@ type FdSet struct { const ( SizeofIfMsghdr = 0x70 + SizeofIfMsghdr2 = 0xa0 SizeofIfData = 0x60 + SizeofIfData64 = 0x80 SizeofIfaMsghdr = 0x14 SizeofIfmaMsghdr = 0x10 SizeofIfmaMsghdr2 = 0x14 SizeofRtMsghdr = 0x5c + SizeofRtMsghdr2 = 0x5c SizeofRtMetrics = 0x38 ) @@ -480,6 +483,20 @@ type IfMsghdr struct { Data IfData } +type IfMsghdr2 struct { + Msglen uint16 + Version uint8 + Type uint8 + Addrs int32 + Flags int32 + Index uint16 + Snd_len int32 + Snd_maxlen int32 + Snd_drops int32 + Timer int32 + Data IfData64 +} + type IfData struct { Type uint8 Typelen uint8 @@ -512,6 +529,34 @@ type IfData struct { Reserved2 uint32 } +type IfData64 struct { + Type uint8 + Typelen uint8 + Physical uint8 + Addrlen uint8 + Hdrlen uint8 + Recvquota uint8 + Xmitquota uint8 + Unused1 uint8 + Mtu uint32 + Metric uint32 + Baudrate uint64 + Ipackets uint64 + Ierrors uint64 + Opackets uint64 + Oerrors uint64 + Collisions uint64 + Ibytes uint64 + Obytes uint64 + Imcasts uint64 + Omcasts uint64 + Iqdrops uint64 + Noproto uint64 + Recvtiming uint32 + Xmittiming uint32 + Lastchange Timeval32 +} + type IfaMsghdr struct { Msglen uint16 Version uint8 @@ -557,6 +602,21 @@ type RtMsghdr struct { Rmx RtMetrics } +type RtMsghdr2 struct { + Msglen uint16 + Version uint8 + Type uint8 + Index uint16 + Flags int32 + Addrs int32 + Refcnt int32 + Parentflags int32 + Reserved int32 + Use int32 + Inits uint32 + Rmx RtMetrics +} + type RtMetrics struct { Locks uint32 Mtu uint32 diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux.go b/vendor/golang.org/x/sys/unix/ztypes_linux.go index 8daaf3faf4c78..5537148dcbb3d 100644 --- a/vendor/golang.org/x/sys/unix/ztypes_linux.go +++ b/vendor/golang.org/x/sys/unix/ztypes_linux.go @@ -2594,8 +2594,8 @@ const ( SOF_TIMESTAMPING_BIND_PHC = 0x8000 SOF_TIMESTAMPING_OPT_ID_TCP = 0x10000 - SOF_TIMESTAMPING_LAST = 0x10000 - SOF_TIMESTAMPING_MASK = 0x1ffff + SOF_TIMESTAMPING_LAST = 0x20000 + SOF_TIMESTAMPING_MASK = 0x3ffff SCM_TSTAMP_SND = 0x0 SCM_TSTAMP_SCHED = 0x1 @@ -3541,7 +3541,7 @@ type Nhmsg struct { type NexthopGrp struct { Id uint32 Weight uint8 - Resvd1 uint8 + High uint8 Resvd2 uint16 } @@ -3802,7 +3802,7 @@ const ( ETHTOOL_MSG_PSE_GET = 0x24 ETHTOOL_MSG_PSE_SET = 0x25 ETHTOOL_MSG_RSS_GET = 0x26 - ETHTOOL_MSG_USER_MAX = 0x2c + ETHTOOL_MSG_USER_MAX = 0x2d ETHTOOL_MSG_KERNEL_NONE = 0x0 ETHTOOL_MSG_STRSET_GET_REPLY = 0x1 ETHTOOL_MSG_LINKINFO_GET_REPLY = 0x2 @@ -3842,7 +3842,7 @@ const ( ETHTOOL_MSG_MODULE_NTF = 0x24 ETHTOOL_MSG_PSE_GET_REPLY = 0x25 ETHTOOL_MSG_RSS_GET_REPLY = 0x26 - ETHTOOL_MSG_KERNEL_MAX = 0x2c + ETHTOOL_MSG_KERNEL_MAX = 0x2e ETHTOOL_FLAG_COMPACT_BITSETS = 0x1 ETHTOOL_FLAG_OMIT_REPLY = 0x2 ETHTOOL_FLAG_STATS = 0x4 @@ -3850,7 +3850,7 @@ const ( ETHTOOL_A_HEADER_DEV_INDEX = 0x1 ETHTOOL_A_HEADER_DEV_NAME = 0x2 ETHTOOL_A_HEADER_FLAGS = 0x3 - ETHTOOL_A_HEADER_MAX = 0x3 + ETHTOOL_A_HEADER_MAX = 0x4 ETHTOOL_A_BITSET_BIT_UNSPEC = 0x0 ETHTOOL_A_BITSET_BIT_INDEX = 0x1 ETHTOOL_A_BITSET_BIT_NAME = 0x2 @@ -4031,11 +4031,11 @@ const ( ETHTOOL_A_CABLE_RESULT_UNSPEC = 0x0 ETHTOOL_A_CABLE_RESULT_PAIR = 0x1 ETHTOOL_A_CABLE_RESULT_CODE = 0x2 - ETHTOOL_A_CABLE_RESULT_MAX = 0x2 + ETHTOOL_A_CABLE_RESULT_MAX = 0x3 ETHTOOL_A_CABLE_FAULT_LENGTH_UNSPEC = 0x0 ETHTOOL_A_CABLE_FAULT_LENGTH_PAIR = 0x1 ETHTOOL_A_CABLE_FAULT_LENGTH_CM = 0x2 - ETHTOOL_A_CABLE_FAULT_LENGTH_MAX = 0x2 + ETHTOOL_A_CABLE_FAULT_LENGTH_MAX = 0x3 ETHTOOL_A_CABLE_TEST_NTF_STATUS_UNSPEC = 0x0 ETHTOOL_A_CABLE_TEST_NTF_STATUS_STARTED = 0x1 ETHTOOL_A_CABLE_TEST_NTF_STATUS_COMPLETED = 0x2 @@ -4200,7 +4200,8 @@ type ( } PtpSysOffsetExtended struct { Samples uint32 - Rsv [3]uint32 + Clockid int32 + Rsv [2]uint32 Ts [25][3]PtpClockTime } PtpSysOffsetPrecise struct { @@ -4399,6 +4400,7 @@ const ( type LandlockRulesetAttr struct { Access_fs uint64 Access_net uint64 + Scoped uint64 } type LandlockPathBeneathAttr struct { diff --git a/vendor/golang.org/x/sys/windows/syscall_windows.go b/vendor/golang.org/x/sys/windows/syscall_windows.go index 4510bfc3f5c66..4a32543868500 100644 --- a/vendor/golang.org/x/sys/windows/syscall_windows.go +++ b/vendor/golang.org/x/sys/windows/syscall_windows.go @@ -168,6 +168,8 @@ func NewCallbackCDecl(fn interface{}) uintptr { //sys CreateNamedPipe(name *uint16, flags uint32, pipeMode uint32, maxInstances uint32, outSize uint32, inSize uint32, defaultTimeout uint32, sa *SecurityAttributes) (handle Handle, err error) [failretval==InvalidHandle] = CreateNamedPipeW //sys ConnectNamedPipe(pipe Handle, overlapped *Overlapped) (err error) //sys DisconnectNamedPipe(pipe Handle) (err error) +//sys GetNamedPipeClientProcessId(pipe Handle, clientProcessID *uint32) (err error) +//sys GetNamedPipeServerProcessId(pipe Handle, serverProcessID *uint32) (err error) //sys GetNamedPipeInfo(pipe Handle, flags *uint32, outSize *uint32, inSize *uint32, maxInstances *uint32) (err error) //sys GetNamedPipeHandleState(pipe Handle, state *uint32, curInstances *uint32, maxCollectionCount *uint32, collectDataTimeout *uint32, userName *uint16, maxUserNameSize uint32) (err error) = GetNamedPipeHandleStateW //sys SetNamedPipeHandleState(pipe Handle, state *uint32, maxCollectionCount *uint32, collectDataTimeout *uint32) (err error) = SetNamedPipeHandleState diff --git a/vendor/golang.org/x/sys/windows/types_windows.go b/vendor/golang.org/x/sys/windows/types_windows.go index 51311e205ff00..9d138de5fed63 100644 --- a/vendor/golang.org/x/sys/windows/types_windows.go +++ b/vendor/golang.org/x/sys/windows/types_windows.go @@ -176,6 +176,7 @@ const ( WAIT_FAILED = 0xFFFFFFFF // Access rights for process. + PROCESS_ALL_ACCESS = 0xFFFF PROCESS_CREATE_PROCESS = 0x0080 PROCESS_CREATE_THREAD = 0x0002 PROCESS_DUP_HANDLE = 0x0040 diff --git a/vendor/golang.org/x/sys/windows/zsyscall_windows.go b/vendor/golang.org/x/sys/windows/zsyscall_windows.go index 6f5252880ceed..01c0716c2c4e8 100644 --- a/vendor/golang.org/x/sys/windows/zsyscall_windows.go +++ b/vendor/golang.org/x/sys/windows/zsyscall_windows.go @@ -280,8 +280,10 @@ var ( procGetMaximumProcessorCount = modkernel32.NewProc("GetMaximumProcessorCount") procGetModuleFileNameW = modkernel32.NewProc("GetModuleFileNameW") procGetModuleHandleExW = modkernel32.NewProc("GetModuleHandleExW") + procGetNamedPipeClientProcessId = modkernel32.NewProc("GetNamedPipeClientProcessId") procGetNamedPipeHandleStateW = modkernel32.NewProc("GetNamedPipeHandleStateW") procGetNamedPipeInfo = modkernel32.NewProc("GetNamedPipeInfo") + procGetNamedPipeServerProcessId = modkernel32.NewProc("GetNamedPipeServerProcessId") procGetOverlappedResult = modkernel32.NewProc("GetOverlappedResult") procGetPriorityClass = modkernel32.NewProc("GetPriorityClass") procGetProcAddress = modkernel32.NewProc("GetProcAddress") @@ -1612,7 +1614,7 @@ func DwmSetWindowAttribute(hwnd HWND, attribute uint32, value unsafe.Pointer, si } func CancelMibChangeNotify2(notificationHandle Handle) (errcode error) { - r0, _, _ := syscall.SyscallN(procCancelMibChangeNotify2.Addr(), uintptr(notificationHandle)) + r0, _, _ := syscall.Syscall(procCancelMibChangeNotify2.Addr(), 1, uintptr(notificationHandle), 0, 0) if r0 != 0 { errcode = syscall.Errno(r0) } @@ -1652,7 +1654,7 @@ func GetIfEntry(pIfRow *MibIfRow) (errcode error) { } func GetIfEntry2Ex(level uint32, row *MibIfRow2) (errcode error) { - r0, _, _ := syscall.SyscallN(procGetIfEntry2Ex.Addr(), uintptr(level), uintptr(unsafe.Pointer(row))) + r0, _, _ := syscall.Syscall(procGetIfEntry2Ex.Addr(), 2, uintptr(level), uintptr(unsafe.Pointer(row)), 0) if r0 != 0 { errcode = syscall.Errno(r0) } @@ -1660,7 +1662,7 @@ func GetIfEntry2Ex(level uint32, row *MibIfRow2) (errcode error) { } func GetUnicastIpAddressEntry(row *MibUnicastIpAddressRow) (errcode error) { - r0, _, _ := syscall.SyscallN(procGetUnicastIpAddressEntry.Addr(), uintptr(unsafe.Pointer(row))) + r0, _, _ := syscall.Syscall(procGetUnicastIpAddressEntry.Addr(), 1, uintptr(unsafe.Pointer(row)), 0, 0) if r0 != 0 { errcode = syscall.Errno(r0) } @@ -1672,7 +1674,7 @@ func NotifyIpInterfaceChange(family uint16, callback uintptr, callerContext unsa if initialNotification { _p0 = 1 } - r0, _, _ := syscall.SyscallN(procNotifyIpInterfaceChange.Addr(), uintptr(family), uintptr(callback), uintptr(callerContext), uintptr(_p0), uintptr(unsafe.Pointer(notificationHandle))) + r0, _, _ := syscall.Syscall6(procNotifyIpInterfaceChange.Addr(), 5, uintptr(family), uintptr(callback), uintptr(callerContext), uintptr(_p0), uintptr(unsafe.Pointer(notificationHandle)), 0) if r0 != 0 { errcode = syscall.Errno(r0) } @@ -1684,7 +1686,7 @@ func NotifyUnicastIpAddressChange(family uint16, callback uintptr, callerContext if initialNotification { _p0 = 1 } - r0, _, _ := syscall.SyscallN(procNotifyUnicastIpAddressChange.Addr(), uintptr(family), uintptr(callback), uintptr(callerContext), uintptr(_p0), uintptr(unsafe.Pointer(notificationHandle))) + r0, _, _ := syscall.Syscall6(procNotifyUnicastIpAddressChange.Addr(), 5, uintptr(family), uintptr(callback), uintptr(callerContext), uintptr(_p0), uintptr(unsafe.Pointer(notificationHandle)), 0) if r0 != 0 { errcode = syscall.Errno(r0) } @@ -2446,6 +2448,14 @@ func GetModuleHandleEx(flags uint32, moduleName *uint16, module *Handle) (err er return } +func GetNamedPipeClientProcessId(pipe Handle, clientProcessID *uint32) (err error) { + r1, _, e1 := syscall.Syscall(procGetNamedPipeClientProcessId.Addr(), 2, uintptr(pipe), uintptr(unsafe.Pointer(clientProcessID)), 0) + if r1 == 0 { + err = errnoErr(e1) + } + return +} + func GetNamedPipeHandleState(pipe Handle, state *uint32, curInstances *uint32, maxCollectionCount *uint32, collectDataTimeout *uint32, userName *uint16, maxUserNameSize uint32) (err error) { r1, _, e1 := syscall.Syscall9(procGetNamedPipeHandleStateW.Addr(), 7, uintptr(pipe), uintptr(unsafe.Pointer(state)), uintptr(unsafe.Pointer(curInstances)), uintptr(unsafe.Pointer(maxCollectionCount)), uintptr(unsafe.Pointer(collectDataTimeout)), uintptr(unsafe.Pointer(userName)), uintptr(maxUserNameSize), 0, 0) if r1 == 0 { @@ -2462,6 +2472,14 @@ func GetNamedPipeInfo(pipe Handle, flags *uint32, outSize *uint32, inSize *uint3 return } +func GetNamedPipeServerProcessId(pipe Handle, serverProcessID *uint32) (err error) { + r1, _, e1 := syscall.Syscall(procGetNamedPipeServerProcessId.Addr(), 2, uintptr(pipe), uintptr(unsafe.Pointer(serverProcessID)), 0) + if r1 == 0 { + err = errnoErr(e1) + } + return +} + func GetOverlappedResult(handle Handle, overlapped *Overlapped, done *uint32, wait bool) (err error) { var _p0 uint32 if wait { diff --git a/vendor/modules.txt b/vendor/modules.txt index cec029a516fc8..6868137a4b0e8 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1875,7 +1875,7 @@ golang.org/x/oauth2/jwt golang.org/x/sync/errgroup golang.org/x/sync/semaphore golang.org/x/sync/singleflight -# golang.org/x/sys v0.27.0 +# golang.org/x/sys v0.28.0 ## explicit; go 1.18 golang.org/x/sys/cpu golang.org/x/sys/plan9 From 85795659dd98538e4a76b1a631b72b45d0f0cfd1 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Wed, 4 Dec 2024 16:16:20 -0500 Subject: [PATCH 36/44] fix(deps): update module cloud.google.com/go/pubsub to v1.45.2 (#15237) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 +- vendor/cloud.google.com/go/pubsub/CHANGES.md | 14 ++++++ .../go/pubsub/apiv1/auxiliary.go | 10 ++--- .../cloud.google.com/go/pubsub/apiv1/doc.go | 43 +----------------- .../go/pubsub/apiv1/helpers.go | 45 +++++++++++++++++++ .../go/pubsub/internal/version.go | 2 +- vendor/cloud.google.com/go/pubsub/iterator.go | 6 +-- .../go/pubsub/subscription.go | 7 ++- vendor/cloud.google.com/go/pubsub/topic.go | 3 +- vendor/cloud.google.com/go/pubsub/trace.go | 19 ++++---- vendor/modules.txt | 2 +- 12 files changed, 86 insertions(+), 71 deletions(-) create mode 100644 vendor/cloud.google.com/go/pubsub/apiv1/helpers.go diff --git a/go.mod b/go.mod index f4729e824d1bc..c3bb6eb8fa034 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,7 @@ toolchain go1.23.1 require ( cloud.google.com/go/bigtable v1.33.0 - cloud.google.com/go/pubsub v1.45.1 + cloud.google.com/go/pubsub v1.45.2 cloud.google.com/go/storage v1.47.0 dario.cat/mergo v1.0.1 github.com/Azure/azure-pipeline-go v0.2.3 diff --git a/go.sum b/go.sum index caecf10edf400..f804a1ce363d1 100644 --- a/go.sum +++ b/go.sum @@ -536,8 +536,8 @@ cloud.google.com/go/pubsub v1.27.1/go.mod h1:hQN39ymbV9geqBnfQq6Xf63yNhUAhv9CZhz cloud.google.com/go/pubsub v1.28.0/go.mod h1:vuXFpwaVoIPQMGXqRyUQigu/AX1S3IWugR9xznmcXX8= cloud.google.com/go/pubsub v1.30.0/go.mod h1:qWi1OPS0B+b5L+Sg6Gmc9zD1Y+HaM0MdUr7LsupY1P4= cloud.google.com/go/pubsub v1.32.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= -cloud.google.com/go/pubsub v1.45.1 h1:ZC/UzYcrmK12THWn1P72z+Pnp2vu/zCZRXyhAfP1hJY= -cloud.google.com/go/pubsub v1.45.1/go.mod h1:3bn7fTmzZFwaUjllitv1WlsNMkqBgGUb3UdMhI54eCc= +cloud.google.com/go/pubsub v1.45.2 h1:TLsGaQDFg92yLCm993tS0md22eku0weBWlLhInxizgs= +cloud.google.com/go/pubsub v1.45.2/go.mod h1:3bn7fTmzZFwaUjllitv1WlsNMkqBgGUb3UdMhI54eCc= cloud.google.com/go/pubsublite v1.5.0/go.mod h1:xapqNQ1CuLfGi23Yda/9l4bBCKz/wC3KIJ5gKcxveZg= cloud.google.com/go/pubsublite v1.6.0/go.mod h1:1eFCS0U11xlOuMFV/0iBqw3zP12kddMeCbj/F3FSj9k= cloud.google.com/go/pubsublite v1.7.0/go.mod h1:8hVMwRXfDfvGm3fahVbtDbiLePT3gpoiJYJY+vxWxVM= diff --git a/vendor/cloud.google.com/go/pubsub/CHANGES.md b/vendor/cloud.google.com/go/pubsub/CHANGES.md index 11038ccad2f58..d4c79a08cc28a 100644 --- a/vendor/cloud.google.com/go/pubsub/CHANGES.md +++ b/vendor/cloud.google.com/go/pubsub/CHANGES.md @@ -1,5 +1,19 @@ # Changes +## [1.45.2](https://github.com/googleapis/google-cloud-go/compare/pubsub/v1.45.1...pubsub/v1.45.2) (2024-12-03) + + +### Bug Fixes + +* **pubsub/pstest:** Make invalid filter return error instead of panic ([#11087](https://github.com/googleapis/google-cloud-go/issues/11087)) ([45e1ce7](https://github.com/googleapis/google-cloud-go/commit/45e1ce70e2757b78b868768b93e05da8858bab85)) +* **pubsub:** Only init batch span if trace enabled ([#11193](https://github.com/googleapis/google-cloud-go/issues/11193)) ([f843d50](https://github.com/googleapis/google-cloud-go/commit/f843d50f849c5014eba33d923085f0add41365a6)) +* **pubsub:** Use official semconv variable whenever possible ([#10904](https://github.com/googleapis/google-cloud-go/issues/10904)) ([1ce4b6d](https://github.com/googleapis/google-cloud-go/commit/1ce4b6dc31653ca6f28c50d5149d74b827caaeaa)) + + +### Documentation + +* **pubsub:** MinExtensionPeriod defaults to 60 seconds ([#10791](https://github.com/googleapis/google-cloud-go/issues/10791)) ([cc88fe1](https://github.com/googleapis/google-cloud-go/commit/cc88fe1c8d834903e2b5f8d3a7cc9bbd65c70b29)) + ## [1.45.1](https://github.com/googleapis/google-cloud-go/compare/pubsub/v1.45.0...pubsub/v1.45.1) (2024-10-24) diff --git a/vendor/cloud.google.com/go/pubsub/apiv1/auxiliary.go b/vendor/cloud.google.com/go/pubsub/apiv1/auxiliary.go index 3a9bcffd52543..46380a0e05dac 100644 --- a/vendor/cloud.google.com/go/pubsub/apiv1/auxiliary.go +++ b/vendor/cloud.google.com/go/pubsub/apiv1/auxiliary.go @@ -41,7 +41,7 @@ type SchemaIterator struct { InternalFetch func(pageSize int, pageToken string) (results []*pubsubpb.Schema, nextPageToken string, err error) } -// PageInfo supports pagination. See the google.golang.org/api/iterator package for details. +// PageInfo supports pagination. See the [google.golang.org/api/iterator] package for details. func (it *SchemaIterator) PageInfo() *iterator.PageInfo { return it.pageInfo } @@ -88,7 +88,7 @@ type SnapshotIterator struct { InternalFetch func(pageSize int, pageToken string) (results []*pubsubpb.Snapshot, nextPageToken string, err error) } -// PageInfo supports pagination. See the google.golang.org/api/iterator package for details. +// PageInfo supports pagination. See the [google.golang.org/api/iterator] package for details. func (it *SnapshotIterator) PageInfo() *iterator.PageInfo { return it.pageInfo } @@ -135,7 +135,7 @@ type StringIterator struct { InternalFetch func(pageSize int, pageToken string) (results []string, nextPageToken string, err error) } -// PageInfo supports pagination. See the google.golang.org/api/iterator package for details. +// PageInfo supports pagination. See the [google.golang.org/api/iterator] package for details. func (it *StringIterator) PageInfo() *iterator.PageInfo { return it.pageInfo } @@ -182,7 +182,7 @@ type SubscriptionIterator struct { InternalFetch func(pageSize int, pageToken string) (results []*pubsubpb.Subscription, nextPageToken string, err error) } -// PageInfo supports pagination. See the google.golang.org/api/iterator package for details. +// PageInfo supports pagination. See the [google.golang.org/api/iterator] package for details. func (it *SubscriptionIterator) PageInfo() *iterator.PageInfo { return it.pageInfo } @@ -229,7 +229,7 @@ type TopicIterator struct { InternalFetch func(pageSize int, pageToken string) (results []*pubsubpb.Topic, nextPageToken string, err error) } -// PageInfo supports pagination. See the google.golang.org/api/iterator package for details. +// PageInfo supports pagination. See the [google.golang.org/api/iterator] package for details. func (it *TopicIterator) PageInfo() *iterator.PageInfo { return it.pageInfo } diff --git a/vendor/cloud.google.com/go/pubsub/apiv1/doc.go b/vendor/cloud.google.com/go/pubsub/apiv1/doc.go index 650d23343fb9f..03aeef34e3910 100644 --- a/vendor/cloud.google.com/go/pubsub/apiv1/doc.go +++ b/vendor/cloud.google.com/go/pubsub/apiv1/doc.go @@ -36,6 +36,7 @@ // // To get started with this package, create a client. // +// // go get cloud.google.com/go/pubsub/apiv1@latest // ctx := context.Background() // // This snippet has been automatically generated and should be regarded as a code template only. // // It will require modifications to work: @@ -54,19 +55,7 @@ // // # Using the Client // -// The following is an example of making an API call with the newly created client. -// -// ctx := context.Background() -// // This snippet has been automatically generated and should be regarded as a code template only. -// // It will require modifications to work: -// // - It may require correct/in-range values for request initialization. -// // - It may require specifying regional endpoints when creating the service client as shown in: -// // https://pkg.go.dev/cloud.google.com/go#hdr-Client_Options -// c, err := pubsub.NewSchemaClient(ctx) -// if err != nil { -// // TODO: Handle error. -// } -// defer c.Close() +// The following is an example of making an API call with the newly created client, mentioned above. // // req := &pubsubpb.CommitSchemaRequest{ // // TODO: Fill request struct fields. @@ -93,31 +82,3 @@ // [Debugging Client Libraries]: https://pkg.go.dev/cloud.google.com/go#hdr-Debugging // [Inspecting errors]: https://pkg.go.dev/cloud.google.com/go#hdr-Inspecting_errors package pubsub // import "cloud.google.com/go/pubsub/apiv1" - -import ( - "context" - - "google.golang.org/api/option" -) - -// For more information on implementing a client constructor hook, see -// https://github.com/googleapis/google-cloud-go/wiki/Customizing-constructors. -type clientHookParams struct{} -type clientHook func(context.Context, clientHookParams) ([]option.ClientOption, error) - -var versionClient string - -func getVersionClient() string { - if versionClient == "" { - return "UNKNOWN" - } - return versionClient -} - -// DefaultAuthScopes reports the default set of authentication scopes to use with this package. -func DefaultAuthScopes() []string { - return []string{ - "https://www.googleapis.com/auth/cloud-platform", - "https://www.googleapis.com/auth/pubsub", - } -} diff --git a/vendor/cloud.google.com/go/pubsub/apiv1/helpers.go b/vendor/cloud.google.com/go/pubsub/apiv1/helpers.go new file mode 100644 index 0000000000000..512907422f30a --- /dev/null +++ b/vendor/cloud.google.com/go/pubsub/apiv1/helpers.go @@ -0,0 +1,45 @@ +// Copyright 2024 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Code generated by protoc-gen-go_gapic. DO NOT EDIT. + +package pubsub + +import ( + "context" + + "google.golang.org/api/option" +) + +// For more information on implementing a client constructor hook, see +// https://github.com/googleapis/google-cloud-go/wiki/Customizing-constructors. +type clientHookParams struct{} +type clientHook func(context.Context, clientHookParams) ([]option.ClientOption, error) + +var versionClient string + +func getVersionClient() string { + if versionClient == "" { + return "UNKNOWN" + } + return versionClient +} + +// DefaultAuthScopes reports the default set of authentication scopes to use with this package. +func DefaultAuthScopes() []string { + return []string{ + "https://www.googleapis.com/auth/cloud-platform", + "https://www.googleapis.com/auth/pubsub", + } +} diff --git a/vendor/cloud.google.com/go/pubsub/internal/version.go b/vendor/cloud.google.com/go/pubsub/internal/version.go index e7729577cf3b9..79d07af8e898d 100644 --- a/vendor/cloud.google.com/go/pubsub/internal/version.go +++ b/vendor/cloud.google.com/go/pubsub/internal/version.go @@ -15,4 +15,4 @@ package internal // Version is the current tagged release of the library. -const Version = "1.45.1" +const Version = "1.45.2" diff --git a/vendor/cloud.google.com/go/pubsub/iterator.go b/vendor/cloud.google.com/go/pubsub/iterator.go index 4f791fa6c06ae..c533f29877d14 100644 --- a/vendor/cloud.google.com/go/pubsub/iterator.go +++ b/vendor/cloud.google.com/go/pubsub/iterator.go @@ -279,7 +279,7 @@ func (it *messageIterator) receive(maxToPull int32) ([]*Message, error) { // If the cancellation comes from the underlying grpc client getting closed, // do propagate the cancellation error. // See https://github.com/googleapis/google-cloud-go/pull/10153#discussion_r1600814775 - if err != nil && it.ps.ctx.Err() == context.Canceled { + if err != nil && errors.Is(it.ps.ctx.Err(), context.Canceled) { err = io.EOF } } @@ -340,7 +340,7 @@ func (it *messageIterator) receive(maxToPull int32) ([]*Message, error) { opts, trace.WithAttributes( attribute.Bool(eosAttribute, it.enableExactlyOnceDelivery), - attribute.String(ackIDAttribute, ackID), + semconv.MessagingGCPPubsubMessageAckID(ackID), semconv.MessagingBatchMessageCount(len(msgs)), semconv.CodeFunction("receive"), ), @@ -406,7 +406,7 @@ func (it *messageIterator) pullMessages(maxToPull int32) ([]*pb.ReceivedMessage, MaxMessages: maxToPull, }, gax.WithGRPCOptions(grpc.MaxCallRecvMsgSize(maxSendRecvBytes))) switch { - case err == context.Canceled: + case errors.Is(err, context.Canceled): return nil, nil case status.Code(err) == codes.Canceled: return nil, nil diff --git a/vendor/cloud.google.com/go/pubsub/subscription.go b/vendor/cloud.google.com/go/pubsub/subscription.go index c6311d6e933bf..ebd522bc013bb 100644 --- a/vendor/cloud.google.com/go/pubsub/subscription.go +++ b/vendor/cloud.google.com/go/pubsub/subscription.go @@ -546,7 +546,7 @@ type SubscriptionConfig struct { // When calling Subscription.Receive(), the client will check this // value with a call to Subscription.Config(), which requires the // roles/viewer or roles/pubsub.viewer role on your service account. - // If that call fails, mesages with ordering keys will be delivered in order. + // If that call fails, messages with ordering keys will be delivered in order. EnableMessageOrdering bool // DeadLetterPolicy specifies the conditions for dead lettering messages in @@ -899,8 +899,7 @@ type ReceiveSettings struct { // // MinExtensionPeriod must be between 10s and 600s (inclusive). This configuration // can be disabled by specifying a duration less than (or equal to) 0. - // Defaults to off but set to 60 seconds if the subscription has exactly-once delivery enabled, - // which will be added in a future release. + // Disabled by default but set to 60 seconds if the subscription has exactly-once delivery enabled. MinExtensionPeriod time.Duration // MaxOutstandingMessages is the maximum number of unprocessed messages @@ -1379,7 +1378,7 @@ func (s *Subscription) Receive(ctx context.Context, f func(context.Context, *Mes } msgs, err := iter.receive(maxToPull) - if err == io.EOF { + if errors.Is(err, io.EOF) { return nil } if err != nil { diff --git a/vendor/cloud.google.com/go/pubsub/topic.go b/vendor/cloud.google.com/go/pubsub/topic.go index bb916f5290565..72d9a34590358 100644 --- a/vendor/cloud.google.com/go/pubsub/topic.go +++ b/vendor/cloud.google.com/go/pubsub/topic.go @@ -993,8 +993,6 @@ func (t *Topic) Publish(ctx context.Context, msg *Message) *PublishResult { fcSpan.End() } - _, batcherSpan = startSpan(ctx, batcherSpanName, "") - bmsg := &bundledMessage{ msg: msg, res: r, @@ -1003,6 +1001,7 @@ func (t *Topic) Publish(ctx context.Context, msg *Message) *PublishResult { } if t.enableTracing { + _, batcherSpan = startSpan(ctx, batcherSpanName, "") bmsg.batcherSpan = batcherSpan // Inject the context from the first publish span rather than from flow control / batching. diff --git a/vendor/cloud.google.com/go/pubsub/trace.go b/vendor/cloud.google.com/go/pubsub/trace.go index 51112bb50f5f3..6ff88a8479343 100644 --- a/vendor/cloud.google.com/go/pubsub/trace.go +++ b/vendor/cloud.google.com/go/pubsub/trace.go @@ -360,14 +360,11 @@ const ( resultExpired = "expired" // custom pubsub specific attributes - gcpProjectIDAttribute = "gcp.project_id" - pubsubPrefix = "messaging.gcp_pubsub." - orderingAttribute = pubsubPrefix + "message.ordering_key" - deliveryAttemptAttribute = pubsubPrefix + "message.delivery_attempt" - eosAttribute = pubsubPrefix + "exactly_once_delivery" - ackIDAttribute = pubsubPrefix + "message.ack_id" - resultAttribute = pubsubPrefix + "result" - receiptModackAttribute = pubsubPrefix + "is_receipt_modack" + gcpProjectIDAttribute = "gcp.project_id" + pubsubPrefix = "messaging.gcp_pubsub." + eosAttribute = pubsubPrefix + "exactly_once_delivery" + resultAttribute = pubsubPrefix + "result" + receiptModackAttribute = pubsubPrefix + "is_receipt_modack" ) func startSpan(ctx context.Context, spanType, resourceID string, opts ...trace.SpanStartOption) (context.Context, trace.Span) { @@ -383,7 +380,7 @@ func getPublishSpanAttributes(project, dst string, msg *Message, attrs ...attrib trace.WithAttributes( semconv.MessagingMessageID(msg.ID), semconv.MessagingMessageBodySize(len(msg.Data)), - attribute.String(orderingAttribute, msg.OrderingKey), + semconv.MessagingGCPPubsubMessageOrderingKey(msg.OrderingKey), ), trace.WithAttributes(attrs...), trace.WithSpanKind(trace.SpanKindProducer), @@ -397,13 +394,13 @@ func getSubscriberOpts(project, dst string, msg *Message, attrs ...attribute.Key trace.WithAttributes( semconv.MessagingMessageID(msg.ID), semconv.MessagingMessageBodySize(len(msg.Data)), - attribute.String(orderingAttribute, msg.OrderingKey), + semconv.MessagingGCPPubsubMessageOrderingKey(msg.OrderingKey), ), trace.WithAttributes(attrs...), trace.WithSpanKind(trace.SpanKindConsumer), } if msg.DeliveryAttempt != nil { - opts = append(opts, trace.WithAttributes(attribute.Int(deliveryAttemptAttribute, *msg.DeliveryAttempt))) + opts = append(opts, trace.WithAttributes(semconv.MessagingGCPPubsubMessageDeliveryAttempt(*msg.DeliveryAttempt))) } opts = append(opts, getCommonOptions(project, dst)...) return opts diff --git a/vendor/modules.txt b/vendor/modules.txt index 6868137a4b0e8..a244470a5972c 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -55,7 +55,7 @@ cloud.google.com/go/longrunning/autogen/longrunningpb cloud.google.com/go/monitoring/apiv3/v2 cloud.google.com/go/monitoring/apiv3/v2/monitoringpb cloud.google.com/go/monitoring/internal -# cloud.google.com/go/pubsub v1.45.1 +# cloud.google.com/go/pubsub v1.45.2 ## explicit; go 1.21 cloud.google.com/go/pubsub cloud.google.com/go/pubsub/apiv1 From 4cce207c6ab767f13cf55347b9cfe0a3a595b8d9 Mon Sep 17 00:00:00 2001 From: jxie <95172247+xiejian06033218@users.noreply.github.com> Date: Thu, 5 Dec 2024 20:39:06 +1100 Subject: [PATCH 37/44] fix(blooms): Enable metas cache on backend when running in ssd mode (#15275) --- pkg/loki/modules.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index c4c4a6b0412c5..6ad0adde14cb6 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -784,7 +784,7 @@ func (t *Loki) initBloomStore() (services.Service, error) { bsCfg := t.Cfg.StorageConfig.BloomShipperConfig var metasCache cache.Cache - if t.Cfg.isTarget(IndexGateway) && cache.IsCacheConfigured(bsCfg.MetasCache) { + if (t.Cfg.isTarget(IndexGateway) || t.Cfg.isTarget(Backend)) && cache.IsCacheConfigured(bsCfg.MetasCache) { metasCache, err = cache.New(bsCfg.MetasCache, reg, logger, stats.BloomMetasCache, constants.Loki) // always enable LRU cache From af22f25b02ea253a3adfb6d65812f1c52f87ca02 Mon Sep 17 00:00:00 2001 From: J Stickler Date: Thu, 5 Dec 2024 12:37:18 -0500 Subject: [PATCH 38/44] docs: Release notes for 3.2.2 (#15277) --- docs/sources/release-notes/v3-2.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/sources/release-notes/v3-2.md b/docs/sources/release-notes/v3-2.md index 89b0b48d2674d..c061ab2eae07c 100644 --- a/docs/sources/release-notes/v3-2.md +++ b/docs/sources/release-notes/v3-2.md @@ -63,6 +63,7 @@ For important upgrade guidance, refer to the [Upgrade Guide](https://grafana.com - **BREAKING CHANGE - API:** Fail log queries when executed on instant query endpoint ([#13421](https://github.com/grafana/loki/issues/13421)). - **BREAKING CHANGE - blooms:** Remove bloom compactor component ([#13969](https://github.com/grafana/loki/issues/13969)). +- **BREAKING CHANGE - docker:** Remove wget from Promtail docker image([#15101](https://github.com/grafana/loki/issues/15101)). - **BREAKING CHANGE - Helm:** Update Helm chart to support distributed mode and 3.0 ([#12067](https://github.com/grafana/loki/issues/12067)). - **BREAKING CHANGE - Helm:** Fix how we set imagePullSecrets for enterprise-gateway and admin-api. ([#13761](https://github.com/grafana/loki/issues/13761)) ([3be5a45](https://github.com/grafana/loki/commit/3be5a4576fd0f0dca321e017a637f7a3159c00e5)). - **BREAKING CHANGE - jsonnet:** Convert read statefulset into deployment for loki-simple-scalable ([#13977](https://github.com/grafana/loki/issues/13977)). @@ -73,6 +74,14 @@ Out of an abundance of caution, we advise that users with Loki or Grafana Enterp ## Bug fixes +### 3.2.2 (2024-12-04) + +- **BREAKING CHANGE - docker:** Remove wget from Promtail docker image([#15101](https://github.com/grafana/loki/issues/15101)). +- **docker:** Move from base-nossl to static. This PR removes the inclusion of glibc into most of the Docker images created by the Loki build system. ([#15203](https://github.com/grafana/loki/issues/15203)). +- **logql:** Updated JSONExpressionParser not to unescape extracted values if it is JSON object. ([#14499](https://github.com/grafana/loki/issues/14499)). +- **promtail:** Switch Promtail base image from Debian to Ubuntu to fix critical security issues ([#15195](https://github.com/grafana/loki/issues/15195)). +- **storage:** Have GetObject check for canceled context. S3ObjectClient.GetObject incorrectly returned nil, 0, nil when the provided context is already canceled ([#14420](https://github.com/grafana/loki/issues/14420)). + ### 3.2.1 (2024-10-17) - **config:** Copy Alibaba and IBM object storage configuration from common ([#14315](https://github.com/grafana/loki/issues/14315)) ([32a9bc0](https://github.com/grafana/loki/commit/32a9bc0ca852bdc692c2ccebbae448856e191953)). From d8d74493d851bcb4828e1ae04bb52470a2db8b89 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Thu, 5 Dec 2024 10:39:54 -0700 Subject: [PATCH 39/44] chore(deps): update terraform aws to ~> 5.80.0 (#15270) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- production/terraform/modules/s3/versions.tf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/production/terraform/modules/s3/versions.tf b/production/terraform/modules/s3/versions.tf index b33267726f3ec..0e8fff0ed099a 100644 --- a/production/terraform/modules/s3/versions.tf +++ b/production/terraform/modules/s3/versions.tf @@ -2,7 +2,7 @@ terraform { required_providers { aws = { source = "hashicorp/aws" - version = "~> 5.79.0" + version = "~> 5.80.0" } random = { From 7db62bfbc2e062d513e34afa63cbb2782e669624 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Thu, 5 Dec 2024 10:40:16 -0700 Subject: [PATCH 40/44] chore(deps): update grafana/grafana docker tag to v11.3.2 (#15268) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- production/docker/docker-compose.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/production/docker/docker-compose.yaml b/production/docker/docker-compose.yaml index eb0d5a7b0c1ca..b575d22b98e08 100644 --- a/production/docker/docker-compose.yaml +++ b/production/docker/docker-compose.yaml @@ -24,7 +24,7 @@ services: - loki grafana: - image: grafana/grafana:11.3.1 + image: grafana/grafana:11.3.2 ports: - "3000:3000" environment: From 3f5ee9f9cf886b8e4e2586cd2b4fd9c3f0bb68d2 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Thu, 5 Dec 2024 10:45:35 -0700 Subject: [PATCH 41/44] fix(deps): update module github.com/aws/aws-sdk-go-v2/service/s3 to v1.71.0 (#15238) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- tools/lambda-promtail/go.mod | 2 +- tools/lambda-promtail/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tools/lambda-promtail/go.mod b/tools/lambda-promtail/go.mod index 4662711dbf32c..1ef8003b97eb8 100644 --- a/tools/lambda-promtail/go.mod +++ b/tools/lambda-promtail/go.mod @@ -6,7 +6,7 @@ require ( github.com/aws/aws-lambda-go v1.47.0 github.com/aws/aws-sdk-go-v2 v1.32.6 github.com/aws/aws-sdk-go-v2/config v1.28.6 - github.com/aws/aws-sdk-go-v2/service/s3 v1.70.0 + github.com/aws/aws-sdk-go-v2/service/s3 v1.71.0 github.com/go-kit/log v0.2.1 github.com/gogo/protobuf v1.3.2 github.com/golang/snappy v0.0.4 diff --git a/tools/lambda-promtail/go.sum b/tools/lambda-promtail/go.sum index 382e523491456..ea2a92a1ad505 100644 --- a/tools/lambda-promtail/go.sum +++ b/tools/lambda-promtail/go.sum @@ -74,8 +74,8 @@ github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6 h1:50+XsN70R github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.6/go.mod h1:WqgLmwY7so32kG01zD8CPTJWVWM+TzJoOVHwTg4aPug= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.6 h1:BbGDtTi0T1DYlmjBiCr/le3wzhA37O8QTC5/Ab8+EXk= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.6/go.mod h1:hLMJt7Q8ePgViKupeymbqI0la+t9/iYFBjxQCFwuAwI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.70.0 h1:HrHFR8RoS4l4EvodRMFcJMYQ8o3UhmALn2nbInXaxZA= -github.com/aws/aws-sdk-go-v2/service/s3 v1.70.0/go.mod h1:sT/iQz8JK3u/5gZkT+Hmr7GzVZehUMkRZpOaAwYXeGY= +github.com/aws/aws-sdk-go-v2/service/s3 v1.71.0 h1:nyuzXooUNJexRT0Oy0UQY6AhOzxPxhtt4DcBIHyCnmw= +github.com/aws/aws-sdk-go-v2/service/s3 v1.71.0/go.mod h1:sT/iQz8JK3u/5gZkT+Hmr7GzVZehUMkRZpOaAwYXeGY= github.com/aws/aws-sdk-go-v2/service/sso v1.24.7 h1:rLnYAfXQ3YAccocshIH5mzNNwZBkBo+bP6EhIxak6Hw= github.com/aws/aws-sdk-go-v2/service/sso v1.24.7/go.mod h1:ZHtuQJ6t9A/+YDuxOLnbryAmITtr8UysSny3qcyvJTc= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.6 h1:JnhTZR3PiYDNKlXy50/pNeix9aGMo6lLpXwJ1mw8MD4= From a175ef73ff49aeb0cbb9f298d5077dee1f0d3091 Mon Sep 17 00:00:00 2001 From: J Stickler Date: Thu, 5 Dec 2024 12:48:08 -0500 Subject: [PATCH 42/44] docs: Release notes for 3.3.2 (#15278) --- docs/sources/release-notes/v3-3.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/sources/release-notes/v3-3.md b/docs/sources/release-notes/v3-3.md index d5c43e32355f3..b48042861512a 100644 --- a/docs/sources/release-notes/v3-3.md +++ b/docs/sources/release-notes/v3-3.md @@ -79,12 +79,19 @@ For important upgrade guidance, refer to the [Upgrade Guide](https://grafana.com - **BREAKING CHANGE - blooms:** Introduce a new block schema (V3) ([#14038](https://github.com/grafana/loki/issues/14038)). - **BREAKING CHANGE - blooms:** Index structured metadata into blooms ([#14061](https://github.com/grafana/loki/issues/14061)). +- **BREAKING CHANGE - docker:** Remove wget from Promtail docker image([#15101](https://github.com/grafana/loki/issues/15101)). - **BREAKING CHANGE - operator:** Migrate project layout to kubebuilder go/v4 ([#14447](https://github.com/grafana/loki/issues/14447)). - **BREAKING CHANGE - operator:** Rename Loki API go module ([#14568](https://github.com/grafana/loki/issues/14568)). - **BREAKING CHANGE - operator:** Provide default OTLP attribute configuration ([#14410](https://github.com/grafana/loki/issues/14410)). ## Bug fixes +### 3.3.1 (2024-12-04) + +- **BREAKING CHANGE - docker:** Remove wget from Promtail docker image([#15101](https://github.com/grafana/loki/issues/15101)). +- **docker:** Move from base-nossl to static. This PR removes the inclusion of glibc into most of the Docker images created by the Loki build system. ([#15203](https://github.com/grafana/loki/issues/15203)). + - **promtail:** Switch Promtail base image from Debian to Ubuntu to fix critical security issues ([#15195](https://github.com/grafana/loki/issues/15195)). + ### 3.3.0 (2024-11-19) - **blooms:** Add tenant limits as dependency to pattern ingester ([#14665](https://github.com/grafana/loki/issues/14665)) ([31eea90](https://github.com/grafana/loki/commit/31eea9042ada6650227eb281a36410ab521817a8)). From 1a6751fc17bb10ba0018057f56f5cc4395c547a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20=C5=81uczak?= Date: Thu, 5 Dec 2024 18:52:08 +0100 Subject: [PATCH 43/44] fix(helm): add default wal dir to ruler config (#14920) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Krzysztof Ɓuczak --- docs/sources/setup/install/helm/reference.md | 6 +++++- production/helm/loki/CHANGELOG.md | 1 + production/helm/loki/values.yaml | 4 +++- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/docs/sources/setup/install/helm/reference.md b/docs/sources/setup/install/helm/reference.md index 5dfd74ea279d2..4975387e96dc8 100644 --- a/docs/sources/setup/install/helm/reference.md +++ b/docs/sources/setup/install/helm/reference.md @@ -6247,7 +6247,11 @@ null object Check https://grafana.com/docs/loki/latest/configuration/#ruler for more info on configuring ruler
-{}
+{
+  "wal": {
+    "dir": "/var/loki/ruler-wal"
+  }
+}
 
diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index eac1c61615101..d93e82d275032 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -13,6 +13,7 @@ Entries should include a reference to the pull request that introduced the chang [//]: # ( : do not remove this line. This locator is used by the CI pipeline to automatically create a changelog entry for each new Loki release. Add other chart versions and respective changelog entries bellow this line.) +- [BUGFIX] Add default wal dir to ruler config ([#14920](https://github.com/grafana/loki/pull/14920)) ## 6.22.0 ## 6.20.0 diff --git a/production/helm/loki/values.yaml b/production/helm/loki/values.yaml index 78961a7243009..94afe8f5a829e 100644 --- a/production/helm/loki/values.yaml +++ b/production/helm/loki/values.yaml @@ -412,7 +412,9 @@ loki: prefix: index_ period: 24h # -- Check https://grafana.com/docs/loki/latest/configuration/#ruler for more info on configuring ruler - rulerConfig: {} + rulerConfig: + wal: + dir: /var/loki/ruler-wal # -- Structured loki configuration, takes precedence over `loki.config`, `loki.schemaConfig`, `loki.storageConfig` structuredConfig: {} # -- Additional query scheduler config From 7818e98d9fead719a7ed634f4acbf3f99bef92c7 Mon Sep 17 00:00:00 2001 From: Gerben Welter Date: Thu, 5 Dec 2024 18:52:34 +0100 Subject: [PATCH 44/44] chore: Update Minio helm chart dependency to 5.3.0 (#14834) Co-authored-by: Trevor Whitney --- production/helm/loki/CHANGELOG.md | 1 + production/helm/loki/Chart.lock | 6 +++--- production/helm/loki/Chart.yaml | 2 +- production/helm/loki/README.md | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index d93e82d275032..9f21593b2e403 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -12,6 +12,7 @@ Entries should be ordered as follows: Entries should include a reference to the pull request that introduced the change. [//]: # ( : do not remove this line. This locator is used by the CI pipeline to automatically create a changelog entry for each new Loki release. Add other chart versions and respective changelog entries bellow this line.) +- [CHANGE] Changed version of Minio helm chart to 5.3.0 (#14834) - [BUGFIX] Add default wal dir to ruler config ([#14920](https://github.com/grafana/loki/pull/14920)) ## 6.22.0 diff --git a/production/helm/loki/Chart.lock b/production/helm/loki/Chart.lock index 3e62c9b855cf1..e4ffe8c63da69 100644 --- a/production/helm/loki/Chart.lock +++ b/production/helm/loki/Chart.lock @@ -1,12 +1,12 @@ dependencies: - name: minio repository: https://charts.min.io/ - version: 4.1.0 + version: 5.3.0 - name: grafana-agent-operator repository: https://grafana.github.io/helm-charts version: 0.5.0 - name: rollout-operator repository: https://grafana.github.io/helm-charts version: 0.20.0 -digest: sha256:db3dad759bc313e566fb814371052d1d96e68325c88e0f9f086b0ff1288d5885 -generated: "2024-11-22T23:07:43.427037179Z" +digest: sha256:7d7078dd5a08a25d152fd78a6fe60a8795cff0f940507d5b4c7ad4df396a16d9 +generated: "2024-11-28T15:43:12.190731467+01:00" diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index ce11a1a5c2c4f..7fd96976ad7ad 100644 --- a/production/helm/loki/Chart.yaml +++ b/production/helm/loki/Chart.yaml @@ -13,7 +13,7 @@ icon: https://grafana.com/docs/loki/latest/logo_and_name.png dependencies: - name: minio alias: minio - version: 4.1.0 + version: 5.3.0 repository: https://charts.min.io/ condition: minio.enabled - name: grafana-agent-operator diff --git a/production/helm/loki/README.md b/production/helm/loki/README.md index 6651f00748afb..7396e9c3a61e6 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -14,7 +14,7 @@ Helm chart for Grafana Loki and Grafana Enterprise Logs supporting both simple, | Repository | Name | Version | |------------|------|---------| -| https://charts.min.io/ | minio(minio) | 4.1.0 | +| https://charts.min.io/ | minio(minio) | 5.3.0 | | https://grafana.github.io/helm-charts | grafana-agent-operator(grafana-agent-operator) | 0.5.0 | | https://grafana.github.io/helm-charts | rollout_operator(rollout-operator) | 0.20.0 |