Skip to content

Commit

Permalink
feat(kafka): Replay kafka from last commit before becoming ready
Browse files Browse the repository at this point in the history
  • Loading branch information
benclive committed Oct 1, 2024
1 parent 2d4792a commit b8ce205
Show file tree
Hide file tree
Showing 7 changed files with 379 additions and 20 deletions.
2 changes: 1 addition & 1 deletion pkg/ingester/ingester.go
Original file line number Diff line number Diff line change
Expand Up @@ -300,7 +300,7 @@ type Ingester struct {
}

// New makes a new Ingester.
func New(cfg Config, clientConfig client.Config, store Store, limits Limits, configs *runtime.TenantConfigs, registerer prometheus.Registerer, writeFailuresCfg writefailures.Cfg, metricsNamespace string, logger log.Logger, customStreamsTracker push.UsageTracker, readRing ring.ReadRing, partitionRingWatcher *ring.PartitionRingWatcher) (*Ingester, error) {
func New(cfg Config, clientConfig client.Config, store Store, limits Limits, configs *runtime.TenantConfigs, registerer prometheus.Registerer, writeFailuresCfg writefailures.Cfg, metricsNamespace string, logger log.Logger, customStreamsTracker push.UsageTracker, readRing ring.ReadRing, partitionRingWatcher ring.PartitionRingReader) (*Ingester, error) {
if cfg.ingesterClientFactory == nil {
cfg.ingesterClientFactory = client.New
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/ingester/kafka_consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ package ingester
import (
"context"
"errors"
math "math"
"math"
"sync"
"time"

Expand Down Expand Up @@ -137,8 +137,8 @@ func retryWithBackoff(ctx context.Context, fn func(attempts int) error) error {
return err
}
backoff := backoff.New(ctx, backoff.Config{
MinBackoff: 100 * time.Millisecond,
MaxBackoff: 5 * time.Second,
MinBackoff: 0 * time.Millisecond,
MaxBackoff: 1 * time.Second,
MaxRetries: 0, // Retry infinitely
})
backoff.Wait()
Expand Down
21 changes: 17 additions & 4 deletions pkg/kafka/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,16 +36,14 @@ const (
// in the worst case scenario, which is expected to be way above the actual one.
maxProducerRecordDataBytesLimit = producerBatchMaxBytes - 16384
minProducerRecordDataBytesLimit = 1024 * 1024

kafkaConfigFlagPrefix = "ingest-storage.kafka"
targetConsumerLagAtStartupFlag = kafkaConfigFlagPrefix + ".target-consumer-lag-at-startup"
maxConsumerLagAtStartupFlag = kafkaConfigFlagPrefix + ".max-consumer-lag-at-startup"
)

var (
ErrMissingKafkaAddress = errors.New("the Kafka address has not been configured")
ErrMissingKafkaTopic = errors.New("the Kafka topic has not been configured")
ErrInvalidProducerMaxRecordSizeBytes = fmt.Errorf("the configured producer max record size bytes must be a value between %d and %d", minProducerRecordDataBytesLimit, maxProducerRecordDataBytesLimit)
ErrInconsistentConsumerLagAtStartup = fmt.Errorf("the target and max consumer lag at startup must be either both set to 0 or to a value greater than 0")
ErrInvalidMaxConsumerLagAtStartup = fmt.Errorf("the configured max consumer lag at startup must greater or equal than the configured target consumer lag")

consumeFromPositionOptions = []string{consumeFromLastOffset, consumeFromStart, consumeFromEnd, consumeFromTimestamp}
)
Expand All @@ -68,6 +66,9 @@ type Config struct {

ProducerMaxRecordSizeBytes int `yaml:"producer_max_record_size_bytes"`
ProducerMaxBufferedBytes int64 `yaml:"producer_max_buffered_bytes"`

TargetConsumerLagAtStartup time.Duration `yaml:"target_consumer_lag_at_startup"`
MaxConsumerLagAtStartup time.Duration `yaml:"max_consumer_lag_at_startup"`
}

func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
Expand All @@ -91,6 +92,12 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {

f.IntVar(&cfg.ProducerMaxRecordSizeBytes, prefix+".producer-max-record-size-bytes", maxProducerRecordDataBytesLimit, "The maximum size of a Kafka record data that should be generated by the producer. An incoming write request larger than this size is split into multiple Kafka records. We strongly recommend to not change this setting unless for testing purposes.")
f.Int64Var(&cfg.ProducerMaxBufferedBytes, prefix+".producer-max-buffered-bytes", 1024*1024*1024, "The maximum size of (uncompressed) buffered and unacknowledged produced records sent to Kafka. The produce request fails once this limit is reached. This limit is per Kafka client. 0 to disable the limit.")

targetConsumerLagAtStartupFlag := prefix + ".target-consumer-lag-at-startup"
maxConsumerLagAtStartupFlag := prefix + ".max-consumer-lag-at-startup"
howToDisableConsumerLagAtStartup := fmt.Sprintf("Set both -%s and -%s to 0 to disable waiting for maximum consumer lag being honored at startup.", targetConsumerLagAtStartupFlag, maxConsumerLagAtStartupFlag)
f.DurationVar(&cfg.TargetConsumerLagAtStartup, targetConsumerLagAtStartupFlag, 2*time.Second, "The best-effort maximum lag a consumer tries to achieve at startup. "+howToDisableConsumerLagAtStartup)
f.DurationVar(&cfg.MaxConsumerLagAtStartup, maxConsumerLagAtStartupFlag, 15*time.Second, "The guaranteed maximum lag before a consumer is considered to have caught up reading from a partition at startup, becomes ACTIVE in the hash ring and passes the readiness check. "+howToDisableConsumerLagAtStartup)
}

func (cfg *Config) Validate() error {
Expand All @@ -103,6 +110,12 @@ func (cfg *Config) Validate() error {
if cfg.ProducerMaxRecordSizeBytes < minProducerRecordDataBytesLimit || cfg.ProducerMaxRecordSizeBytes > maxProducerRecordDataBytesLimit {
return ErrInvalidProducerMaxRecordSizeBytes
}
if (cfg.TargetConsumerLagAtStartup != 0) != (cfg.MaxConsumerLagAtStartup != 0) {
return ErrInconsistentConsumerLagAtStartup
}
if cfg.MaxConsumerLagAtStartup < cfg.TargetConsumerLagAtStartup {
return ErrInvalidMaxConsumerLagAtStartup
}

return nil
}
Expand Down
4 changes: 4 additions & 0 deletions pkg/kafka/partition/committer.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,10 @@ func (r *partitionCommitter) Commit(ctx context.Context, offset int64) (returnEr
return nil
}

func (r *partitionCommitter) Sync() {
r.admClient.ListCommittedOffsets(context.Background(), r.consumerGroup, r.kafkaCfg.Topic)

Check failure on line 155 in pkg/kafka/partition/committer.go

View workflow job for this annotation

GitHub Actions / check / golangciLint

Error return value of `r.admClient.ListCommittedOffsets` is not checked (errcheck)
}

func (r *partitionCommitter) Stop() {
if r.kafkaCfg.ConsumerGroupOffsetCommitInterval <= 0 {
return
Expand Down
Loading

0 comments on commit b8ce205

Please sign in to comment.