Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

chore(dataobj): Use kgo Balancer for dataobj-consumer #16146

Merged
merged 4 commits into from
Feb 7, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
41 changes: 24 additions & 17 deletions pkg/dataobj/consumer/partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ type partitionProcessor struct {
builderOnce sync.Once
builderCfg dataobj.BuilderConfig
bucket objstore.Bucket
flushBuffer *bytes.Buffer
bufPool *sync.Pool

// Metrics
metrics *partitionOffsetMetrics
Expand All @@ -50,7 +50,7 @@ type partitionProcessor struct {
logger log.Logger
}

func newPartitionProcessor(ctx context.Context, client *kgo.Client, builderCfg dataobj.BuilderConfig, uploaderCfg uploader.Config, bucket objstore.Bucket, tenantID string, virtualShard int32, topic string, partition int32, logger log.Logger, reg prometheus.Registerer) *partitionProcessor {
func newPartitionProcessor(ctx context.Context, client *kgo.Client, builderCfg dataobj.BuilderConfig, uploaderCfg uploader.Config, bucket objstore.Bucket, tenantID string, virtualShard int32, topic string, partition int32, logger log.Logger, reg prometheus.Registerer, bufPool *sync.Pool) *partitionProcessor {
ctx, cancel := context.WithCancel(ctx)
decoder, err := kafka.NewDecoder()
if err != nil {
Expand Down Expand Up @@ -94,6 +94,7 @@ func newPartitionProcessor(ctx context.Context, client *kgo.Client, builderCfg d
metrics: metrics,
uploader: uploader,
metastoreManager: metastoreManager,
bufPool: bufPool,
}
}

Expand Down Expand Up @@ -158,7 +159,6 @@ func (p *partitionProcessor) initBuilder() error {
return
}
p.builder = builder
p.flushBuffer = bytes.NewBuffer(make([]byte, 0, p.builderCfg.TargetObjectSize))
})
return initErr
}
Expand Down Expand Up @@ -194,22 +194,29 @@ func (p *partitionProcessor) processRecord(record *kgo.Record) {
return
}

flushedDataobjStats, err := p.builder.Flush(p.flushBuffer)
if err != nil {
level.Error(p.logger).Log("msg", "failed to flush builder", "err", err)
return
}
func() {
flushBuffer := p.bufPool.Get().(*bytes.Buffer)
defer p.bufPool.Put(flushBuffer)

objectPath, err := p.uploader.Upload(p.ctx, p.flushBuffer)
if err != nil {
level.Error(p.logger).Log("msg", "failed to upload object", "err", err)
return
}
flushBuffer.Reset()

if err := p.metastoreManager.UpdateMetastore(p.ctx, objectPath, flushedDataobjStats); err != nil {
level.Error(p.logger).Log("msg", "failed to update metastore", "err", err)
return
}
flushedDataobjStats, err := p.builder.Flush(flushBuffer)
if err != nil {
level.Error(p.logger).Log("msg", "failed to flush builder", "err", err)
return
}

objectPath, err := p.uploader.Upload(p.ctx, flushBuffer)
if err != nil {
level.Error(p.logger).Log("msg", "failed to upload object", "err", err)
return
}

if err := p.metastoreManager.UpdateMetastore(p.ctx, objectPath, flushedDataobjStats); err != nil {
level.Error(p.logger).Log("msg", "failed to update metastore", "err", err)
return
}
}()

if err := p.commitRecords(record); err != nil {
level.Error(p.logger).Log("msg", "failed to commit records", "err", err)
Expand Down
10 changes: 9 additions & 1 deletion pkg/dataobj/consumer/service.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package consumer

import (
"bytes"
"context"
"errors"
"strconv"
Expand Down Expand Up @@ -39,6 +40,8 @@ type Service struct {
// Partition management
partitionMtx sync.RWMutex
partitionHandlers map[string]map[int32]*partitionProcessor

bufPool *sync.Pool
}

func New(kafkaCfg kafka.Config, cfg Config, topicPrefix string, bucket objstore.Bucket, instanceID string, partitionRing ring.PartitionRingReader, reg prometheus.Registerer, logger log.Logger) *Service {
Expand All @@ -52,6 +55,11 @@ func New(kafkaCfg kafka.Config, cfg Config, topicPrefix string, bucket objstore.
codec: distributor.TenantPrefixCodec(topicPrefix),
partitionHandlers: make(map[string]map[int32]*partitionProcessor),
reg: reg,
bufPool: &sync.Pool{
New: func() interface{} {
return bytes.NewBuffer(make([]byte, 0, cfg.BuilderConfig.TargetObjectSize))
},
},
}

client, err := consumer.NewGroupClient(
Expand Down Expand Up @@ -95,7 +103,7 @@ func (s *Service) handlePartitionsAssigned(ctx context.Context, client *kgo.Clie
}

for _, partition := range parts {
processor := newPartitionProcessor(ctx, client, s.cfg.BuilderConfig, s.cfg.UploaderConfig, s.bucket, tenant, virtualShard, topic, partition, s.logger, s.reg)
processor := newPartitionProcessor(ctx, client, s.cfg.BuilderConfig, s.cfg.UploaderConfig, s.bucket, tenant, virtualShard, topic, partition, s.logger, s.reg, s.bufPool)
s.partitionHandlers[topic][partition] = processor
processor.start()
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kafka/partitionring/consumer/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ func NewGroupClient(kafkaCfg kafka.Config, partitionRing ring.PartitionRingReade
kgo.ConsumerGroup(groupName),
kgo.ConsumeRegex(),
kgo.ConsumeTopics(kafkaCfg.Topic),
kgo.Balancers(NewCooperativeActiveStickyBalancer(partitionRing)),
kgo.Balancers(kgo.CooperativeStickyBalancer()),
kgo.ConsumeResetOffset(kgo.NewOffset().AtStart()),
kgo.DisableAutoCommit(),
kgo.RebalanceTimeout(5 * time.Minute),
Expand Down