Skip to content

Commit

Permalink
chore(dataobj): Create initial dataobj builder (#16011)
Browse files Browse the repository at this point in the history
Co-authored-by: Cyril Tovena <[email protected]>
Co-authored-by: Robert Fratto <[email protected]>
  • Loading branch information
3 people authored Feb 3, 2025
1 parent 4b44b59 commit ca4c025
Show file tree
Hide file tree
Showing 31 changed files with 1,352 additions and 46 deletions.
30 changes: 30 additions & 0 deletions docs/sources/shared/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -895,6 +895,36 @@ kafka_config:
# CLI flag: -kafka.max-consumer-lag-at-startup
[max_consumer_lag_at_startup: <duration> | default = 15s]

dataobj_consumer:
builderconfig:
# The size of the SHA prefix to use for the data object builder.
# CLI flag: -dataobj-consumer.sha-prefix-size
[sha_prefix_size: <int> | default = 2]

# The size of the target page to use for the data object builder.
# CLI flag: -dataobj-consumer.target-page-size
[target_page_size: <int> | default = 2MiB]

# The size of the target object to use for the data object builder.
# CLI flag: -dataobj-consumer.target-object-size
[target_object_size: <int> | default = 1GiB]

# Configures a maximum size for sections, for sections that support it.
# CLI flag: -dataobj-consumer.target-section-size
[target_section_size: <int> | default = 128MiB]

# The size of the buffer to use for sorting logs.
# CLI flag: -dataobj-consumer.buffer-size
[buffer_size: <int> | default = 16MiB]

# The tenant ID to use for the data object builder.
# CLI flag: -dataobj-consumer.tenant-id
[tenant_id: <string> | default = "fake"]

# The prefix to use for the storage bucket.
# CLI flag: -dataobj-consumer.storage-bucket-prefix
[storage_bucket_prefix: <string> | default = "dataobj/"]

dataobj_explorer:
# Prefix to use when exploring the bucket. If set, only objects under this
# prefix will be visible.
Expand Down
2 changes: 2 additions & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -408,3 +408,5 @@ replace github.com/grafana/loki/pkg/push => ./pkg/push

// leodido fork his project to continue support
replace github.com/influxdata/go-syslog/v3 => github.com/leodido/go-syslog/v4 v4.2.0

replace github.com/thanos-io/objstore => github.com/grafana/objstore v0.0.0-20250128154815-d7e99f81f866
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -628,6 +628,8 @@ github.com/grafana/jsonparser v0.0.0-20241004153430-023329977675 h1:U94jQ2TQr1m3
github.com/grafana/jsonparser v0.0.0-20241004153430-023329977675/go.mod h1:796sq+UcONnSlzA3RtlBZ+b/hrerkZXiEmO8oMjyRwY=
github.com/grafana/memberlist v0.3.1-0.20220714140823-09ffed8adbbe h1:yIXAAbLswn7VNWBIvM71O2QsgfgW9fRXZNR0DXe6pDU=
github.com/grafana/memberlist v0.3.1-0.20220714140823-09ffed8adbbe/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE=
github.com/grafana/objstore v0.0.0-20250128154815-d7e99f81f866 h1:/y3qC0I9kttHjLPxp4bGf+4jcJw60C6hrokTPckHYT8=
github.com/grafana/objstore v0.0.0-20250128154815-d7e99f81f866/go.mod h1:Quz9HUDjGidU0RQpoytzK4KqJ7kwzP+DMAm4K57/usM=
github.com/grafana/pyroscope-go/godeltaprof v0.1.8 h1:iwOtYXeeVSAeYefJNaxDytgjKtUuKQbJqgAIjlnicKg=
github.com/grafana/pyroscope-go/godeltaprof v0.1.8/go.mod h1:2+l7K7twW49Ct4wFluZD3tZ6e0SjanjcUUBPVD/UuGU=
github.com/grafana/regexp v0.0.0-20240518133315-a468a5bfb3bc h1:GN2Lv3MGO7AS6PrRoT6yV5+wkrOpcszoIsO4+4ds248=
Expand Down Expand Up @@ -1124,8 +1126,6 @@ github.com/stvp/tempredis v0.0.0-20181119212430-b82af8480203 h1:QVqDTf3h2WHt08Yu
github.com/stvp/tempredis v0.0.0-20181119212430-b82af8480203/go.mod h1:oqN97ltKNihBbwlX8dLpwxCl3+HnXKV/R0e+sRLd9C8=
github.com/tencentyun/cos-go-sdk-v5 v0.7.40 h1:W6vDGKCHe4wBACI1d2UgE6+50sJFhRWU4O8IB2ozzxM=
github.com/tencentyun/cos-go-sdk-v5 v0.7.40/go.mod h1:4dCEtLHGh8QPxHEkgq+nFaky7yZxQuYwgSJM87icDaw=
github.com/thanos-io/objstore v0.0.0-20250115091151-a54d0f04b42a h1:wFBHAmtq1tOLPFaiC4LozyG/BzkRa3ZTmVv1KujUNqk=
github.com/thanos-io/objstore v0.0.0-20250115091151-a54d0f04b42a/go.mod h1:Quz9HUDjGidU0RQpoytzK4KqJ7kwzP+DMAm4K57/usM=
github.com/tinylib/msgp v1.1.5/go.mod h1:eQsjooMTnV42mHu917E26IogZ2930nFyBQdofk10Udg=
github.com/tklauser/go-sysconf v0.3.13 h1:GBUpcahXSpR2xN01jhkNAbTLRk2Yzgggk8IM08lq3r4=
github.com/tklauser/go-sysconf v0.3.13/go.mod h1:zwleP4Q4OehZHGn4CYZDipCgg9usW5IJePewFCGVEa0=
Expand Down
46 changes: 34 additions & 12 deletions pkg/dataobj/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import (
"errors"
"flag"
"fmt"
"time"

"github.com/grafana/dskit/flagext"
lru "github.com/hashicorp/golang-lru/v2"
Expand Down Expand Up @@ -126,6 +127,12 @@ type Builder struct {

type builderState int

type FlushResult struct {
Path string
MinTimestamp time.Time
MaxTimestamp time.Time
}

const (
// builderStateReady indicates the builder is empty and ready to accept new data.
builderStateEmpty builderState = iota
Expand Down Expand Up @@ -285,15 +292,10 @@ func streamSizeEstimate(stream logproto.Stream) int {
// If Flush builds an object but fails to upload it to object storage, the
// built object is cached and can be retried. [Builder.Reset] can be called to
// discard any pending data and allow new data to be appended.
func (b *Builder) Flush(ctx context.Context) error {
switch b.state {
case builderStateEmpty:
return nil // Nothing to flush
case builderStateDirty:
if err := b.buildObject(); err != nil {
return fmt.Errorf("building object: %w", err)
}
b.state = builderStateFlush
func (b *Builder) Flush(ctx context.Context) (FlushResult, error) {
buf, err := b.FlushToBuffer()
if err != nil {
return FlushResult{}, fmt.Errorf("flushing buffer: %w", err)
}

timer := prometheus.NewTimer(b.metrics.flushTime)
Expand All @@ -303,12 +305,32 @@ func (b *Builder) Flush(ctx context.Context) error {
sumStr := hex.EncodeToString(sum[:])

objectPath := fmt.Sprintf("tenant-%s/objects/%s/%s", b.tenantID, sumStr[:b.cfg.SHAPrefixSize], sumStr[b.cfg.SHAPrefixSize:])
if err := b.bucket.Upload(ctx, objectPath, bytes.NewReader(b.flushBuffer.Bytes())); err != nil {
return err
if err := b.bucket.Upload(ctx, objectPath, bytes.NewReader(buf.Bytes())); err != nil {
return FlushResult{}, fmt.Errorf("uploading object: %w", err)
}

minTime, maxTime := b.streams.GetBounds()

b.Reset()
return nil
return FlushResult{
Path: objectPath,
MinTimestamp: minTime,
MaxTimestamp: maxTime,
}, nil
}

func (b *Builder) FlushToBuffer() (*bytes.Buffer, error) {
switch b.state {
case builderStateEmpty:
return nil, nil // Nothing to flush
case builderStateDirty:
if err := b.buildObject(); err != nil {
return nil, fmt.Errorf("building object: %w", err)
}
b.state = builderStateFlush
}

return b.flushBuffer, nil
}

func (b *Builder) buildObject() error {
Expand Down
3 changes: 2 additions & 1 deletion pkg/dataobj/builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,8 @@ func TestBuilder(t *testing.T) {
for _, entry := range streams {
require.NoError(t, builder.Append(entry))
}
require.NoError(t, builder.Flush(context.Background()))
_, err = builder.Flush(context.Background())
require.NoError(t, err)
})

t.Run("Read", func(t *testing.T) {
Expand Down
32 changes: 32 additions & 0 deletions pkg/dataobj/consumer/config.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
package consumer

import (
"errors"
"flag"

"github.com/grafana/loki/v3/pkg/dataobj"
)

type Config struct {
dataobj.BuilderConfig
TenantID string `yaml:"tenant_id"`
// StorageBucketPrefix is the prefix to use for the storage bucket.
StorageBucketPrefix string `yaml:"storage_bucket_prefix"`
}

func (cfg *Config) Validate() error {
if cfg.TenantID == "" {
return errors.New("tenantID is required")
}
return cfg.BuilderConfig.Validate()
}

func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.RegisterFlagsWithPrefix("dataobj-consumer.", f)
}

func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
cfg.BuilderConfig.RegisterFlagsWithPrefix(prefix, f)
f.StringVar(&cfg.TenantID, prefix+"tenant-id", "fake", "The tenant ID to use for the data object builder.")
f.StringVar(&cfg.StorageBucketPrefix, prefix+"storage-bucket-prefix", "dataobj/", "The prefix to use for the storage bucket.")
}
117 changes: 117 additions & 0 deletions pkg/dataobj/consumer/metrics.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
package consumer

import (
"time"

"go.uber.org/atomic"

"github.com/prometheus/client_golang/prometheus"
)

type partitionOffsetMetrics struct {
currentOffset prometheus.GaugeFunc
lastOffset atomic.Int64

// Error counters
flushFailures prometheus.Counter
commitFailures prometheus.Counter
appendFailures prometheus.Counter

// Processing delay histogram
processingDelay prometheus.Histogram
}

func newPartitionOffsetMetrics() *partitionOffsetMetrics {
p := &partitionOffsetMetrics{
flushFailures: prometheus.NewCounter(prometheus.CounterOpts{
Name: "loki_dataobj_consumer_flush_failures_total",
Help: "Total number of flush failures",
}),
commitFailures: prometheus.NewCounter(prometheus.CounterOpts{
Name: "loki_dataobj_consumer_commit_failures_total",
Help: "Total number of commit failures",
}),
appendFailures: prometheus.NewCounter(prometheus.CounterOpts{
Name: "loki_dataobj_consumer_append_failures_total",
Help: "Total number of append failures",
}),
processingDelay: prometheus.NewHistogram(prometheus.HistogramOpts{
Name: "loki_dataobj_consumer_processing_delay_seconds",
Help: "Time difference between record timestamp and processing time in seconds",
Buckets: prometheus.DefBuckets,
NativeHistogramBucketFactor: 1.1,
NativeHistogramMaxBucketNumber: 100,
NativeHistogramMinResetDuration: 0,
}),
}

p.currentOffset = prometheus.NewGaugeFunc(
prometheus.GaugeOpts{
Name: "loki_dataobj_consumer_current_offset",
Help: "The last consumed offset for this partition",
},
p.getCurrentOffset,
)

return p
}

func (p *partitionOffsetMetrics) getCurrentOffset() float64 {
return float64(p.lastOffset.Load())
}

func (p *partitionOffsetMetrics) register(reg prometheus.Registerer) error {
collectors := []prometheus.Collector{
p.currentOffset,
p.flushFailures,
p.commitFailures,
p.appendFailures,
p.processingDelay,
}

for _, collector := range collectors {
if err := reg.Register(collector); err != nil {
if _, ok := err.(prometheus.AlreadyRegisteredError); !ok {
return err
}
}
}
return nil
}

func (p *partitionOffsetMetrics) unregister(reg prometheus.Registerer) {
collectors := []prometheus.Collector{
p.currentOffset,
p.flushFailures,
p.commitFailures,
p.appendFailures,
p.processingDelay,
}

for _, collector := range collectors {
reg.Unregister(collector)
}
}

func (p *partitionOffsetMetrics) updateOffset(offset int64) {
p.lastOffset.Store(offset)
}

func (p *partitionOffsetMetrics) incFlushFailures() {
p.flushFailures.Inc()
}

func (p *partitionOffsetMetrics) incCommitFailures() {
p.commitFailures.Inc()
}

func (p *partitionOffsetMetrics) incAppendFailures() {
p.appendFailures.Inc()
}

func (p *partitionOffsetMetrics) observeProcessingDelay(recordTimestamp time.Time) {
// Convert milliseconds to seconds and calculate delay
if !recordTimestamp.IsZero() { // Only observe if timestamp is valid
p.processingDelay.Observe(time.Since(recordTimestamp).Seconds())
}
}
Loading

0 comments on commit ca4c025

Please sign in to comment.