Skip to content

Commit

Permalink
Reduce the number of Kafka clients used by producers (#8088)
Browse files Browse the repository at this point in the history
* Reduce the number of Kafka clients used by producers

Signed-off-by: Marco Pracucci <marco@pracucci.com>

* Fix Writer.stopping()

Signed-off-by: Marco Pracucci <marco@pracucci.com>

* Improved TestWriter_WriteSync and fixed Writer.stopping()

Signed-off-by: Marco Pracucci <marco@pracucci.com>

* Fix unit test

Signed-off-by: Marco Pracucci <marco@pracucci.com>

* Use kgo.ManualPartitioner

Signed-off-by: Marco Pracucci <marco@pracucci.com>

* Fix TestDistributor_Push_ShouldSupportIngestStorage

Signed-off-by: Marco Pracucci <marco@pracucci.com>

---------

Signed-off-by: Marco Pracucci <marco@pracucci.com>
  • Loading branch information
pracucci authored May 8, 2024
1 parent ac3469a commit 3ef6b31
Show file tree
Hide file tree
Showing 5 changed files with 97 additions and 45 deletions.
5 changes: 5 additions & 0 deletions pkg/distributor/distributor_ingest_storage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,11 @@ func TestDistributor_Push_ShouldSupportIngestStorage(t *testing.T) {
ingestStorageEnabled: true,
ingestStoragePartitions: 3,
limits: limits,
configure: func(cfg *Config) {
// Run a number of clients equal to the number of partitions, so that each partition
// has its own client, as requested by some test cases.
cfg.IngestStorageConfig.KafkaConfig.WriteClients = 3
},
}

distributors, _, regs, kafkaCluster := prepare(t, testConfig)
Expand Down
6 changes: 6 additions & 0 deletions pkg/storage/ingest/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ const (
var (
ErrMissingKafkaAddress = errors.New("the Kafka address has not been configured")
ErrMissingKafkaTopic = errors.New("the Kafka topic has not been configured")
ErrInvalidWriteClients = errors.New("the configured number of write clients is invalid (must be greater than 0)")
ErrInvalidConsumePosition = errors.New("the configured consume position is invalid")

consumeFromPositionOptions = []string{consumeFromLastOffset, consumeFromStart, consumeFromEnd, consumeFromTimestamp}
Expand Down Expand Up @@ -61,6 +62,7 @@ type KafkaConfig struct {
ClientID string `yaml:"client_id"`
DialTimeout time.Duration `yaml:"dial_timeout"`
WriteTimeout time.Duration `yaml:"write_timeout"`
WriteClients int `yaml:"write_clients"`

ConsumerGroup string `yaml:"consumer_group"`

Expand All @@ -85,6 +87,7 @@ func (cfg *KafkaConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet)
f.StringVar(&cfg.ClientID, prefix+".client-id", "", "The Kafka client ID.")
f.DurationVar(&cfg.DialTimeout, prefix+".dial-timeout", 2*time.Second, "The maximum time allowed to open a connection to a Kafka broker.")
f.DurationVar(&cfg.WriteTimeout, prefix+".write-timeout", 10*time.Second, "How long to wait for an incoming write request to be successfully committed to the Kafka backend.")
f.IntVar(&cfg.WriteClients, prefix+".write-clients", 1, "The number of Kafka clients used by producers. When the configured number of clients is greater than 1, partitions are sharded among Kafka clients. An higher number of clients may provide higher write throughput at the cost of additional Metadata requests pressure to Kafka.")

f.StringVar(&cfg.ConsumerGroup, prefix+".consumer-group", "", "The consumer group used by the consumer to track the last consumed offset. The consumer group must be different for each ingester. If the configured consumer group contains the '<partition>' placeholder, it will be replaced with the actual partition ID owned by the ingester. When empty (recommended), Mimir will use the ingester instance ID to guarantee uniqueness.")

Expand All @@ -105,6 +108,9 @@ func (cfg *KafkaConfig) Validate() error {
if cfg.Topic == "" {
return ErrMissingKafkaTopic
}
if cfg.WriteClients < 1 {
return ErrInvalidWriteClients
}
if !slices.Contains(consumeFromPositionOptions, cfg.ConsumeFromPositionAtStartup) {
return ErrInvalidConsumePosition
}
Expand Down
9 changes: 9 additions & 0 deletions pkg/storage/ingest/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,15 @@ func TestConfig_Validate(t *testing.T) {
},
expectedErr: ErrInvalidConsumePosition,
},
"should fail if ingest storage is enabled and the configured number of Kafka write clients is 0": {
setup: func(cfg *Config) {
cfg.Enabled = true
cfg.KafkaConfig.Address = "localhost"
cfg.KafkaConfig.Topic = "test"
cfg.KafkaConfig.WriteClients = 0
},
expectedErr: ErrInvalidWriteClients,
},
}

for testName, testData := range tests {
Expand Down
68 changes: 24 additions & 44 deletions pkg/storage/ingest/writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,10 @@ type Writer struct {
logger log.Logger
registerer prometheus.Registerer

// We create 1 writer per partition to better parallelize the workload.
// We support multiple Kafka clients to better parallelize the workload. The number of
// clients is fixed during the Writer lifecycle, but they're initialised lazily.
writersMx sync.RWMutex
writers map[int32]*kgo.Client
writers []*kgo.Client

// Metrics.
writeLatency prometheus.Histogram
Expand All @@ -53,7 +54,7 @@ func NewWriter(kafkaCfg KafkaConfig, logger log.Logger, reg prometheus.Registere
kafkaCfg: kafkaCfg,
logger: logger,
registerer: reg,
writers: map[int32]*kgo.Client{},
writers: make([]*kgo.Client, kafkaCfg.WriteClients),
maxInflightProduceRequests: 20,

// Metrics.
Expand Down Expand Up @@ -87,9 +88,13 @@ func (w *Writer) stopping(_ error) error {
w.writersMx.Lock()
defer w.writersMx.Unlock()

for partitionID, client := range w.writers {
for idx, client := range w.writers {
if client == nil {
continue
}

client.Close()
delete(w.writers, partitionID)
w.writers[idx] = nil
}

return nil
Expand All @@ -112,8 +117,9 @@ func (w *Writer) WriteSync(ctx context.Context, partitionID int32, userID string

// Prepare the record to write.
record := &kgo.Record{
Key: []byte(userID), // We don't partition based on the key, so the value here doesn't make any difference.
Value: data,
Key: []byte(userID), // We don't partition based on the key, so the value here doesn't make any difference.
Value: data,
Partition: partitionID,
}

// Write to backend.
Expand Down Expand Up @@ -157,7 +163,8 @@ func (w *Writer) produceSync(ctx context.Context, client *kgo.Client, record *kg
func (w *Writer) getKafkaWriterForPartition(partitionID int32) (*kgo.Client, error) {
// Check if the writer has already been created.
w.writersMx.RLock()
writer := w.writers[partitionID]
clientID := int(partitionID) % len(w.writers)
writer := w.writers[clientID]
w.writersMx.RUnlock()

if writer != nil {
Expand All @@ -168,34 +175,34 @@ func (w *Writer) getKafkaWriterForPartition(partitionID int32) (*kgo.Client, err
defer w.writersMx.Unlock()

// Ensure a new writer wasn't created in the meanwhile. If so, use it.
writer = w.writers[partitionID]
writer = w.writers[clientID]
if writer != nil {
return writer, nil
}
newWriter, err := w.newKafkaWriter(partitionID)
newWriter, err := w.newKafkaWriter(clientID)
if err != nil {
return nil, err
}
w.writers[partitionID] = newWriter
w.writers[clientID] = newWriter
return newWriter, nil
}

// newKafkaWriter creates a new Kafka client used to write to a specific partition.
func (w *Writer) newKafkaWriter(partitionID int32) (*kgo.Client, error) {
logger := log.With(w.logger, "partition", partitionID)
// newKafkaWriter creates a new Kafka client.
func (w *Writer) newKafkaWriter(clientID int) (*kgo.Client, error) {
logger := log.With(w.logger, "client_id", clientID)

// Do not export the client ID, because we use it to specify options to the backend.
metrics := kprom.NewMetrics("cortex_ingest_storage_writer",
kprom.Registerer(prometheus.WrapRegistererWith(prometheus.Labels{"partition": strconv.Itoa(int(partitionID))}, w.registerer)),
kprom.Registerer(prometheus.WrapRegistererWith(prometheus.Labels{"client_id": strconv.Itoa(clientID)}, w.registerer)),
kprom.FetchAndProduceDetail(kprom.Batches, kprom.Records, kprom.CompressedBytes, kprom.UncompressedBytes))

opts := append(
commonKafkaClientOptions(w.kafkaCfg, metrics, logger),
kgo.RequiredAcks(kgo.AllISRAcks()),
kgo.DefaultProduceTopic(w.kafkaCfg.Topic),

// Use a static partitioner because we want to be in control of the partition.
kgo.RecordPartitioner(newKafkaStaticPartitioner(int(partitionID))),
// We set the partition field in each record.
kgo.RecordPartitioner(kgo.ManualPartitioner()),

// Set the upper bounds the size of a record batch.
kgo.ProducerBatchMaxBytes(16_000_000),
Expand Down Expand Up @@ -233,30 +240,3 @@ func (w *Writer) newKafkaWriter(partitionID int32) (*kgo.Client, error) {
)
return kgo.NewClient(opts...)
}

type kafkaStaticPartitioner struct {
partitionID int
}

func newKafkaStaticPartitioner(partitionID int) *kafkaStaticPartitioner {
return &kafkaStaticPartitioner{
partitionID: partitionID,
}
}

// ForTopic implements kgo.Partitioner.
func (p *kafkaStaticPartitioner) ForTopic(string) kgo.TopicPartitioner {
return p
}

// RequiresConsistency implements kgo.TopicPartitioner.
func (p *kafkaStaticPartitioner) RequiresConsistency(_ *kgo.Record) bool {
// Never let Kafka client to write the record to another partition
// if the partition is down.
return true
}

// Partition implements kgo.TopicPartitioner.
func (p *kafkaStaticPartitioner) Partition(_ *kgo.Record, _ int) int {
return p.partitionID
}
54 changes: 53 additions & 1 deletion pkg/storage/ingest/writer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ func TestMain(m *testing.M) {
func TestWriter_WriteSync(t *testing.T) {
const (
topicName = "test"
numPartitions = 1
numPartitions = 2
partitionID = 0
tenantID = "user-1"
)
Expand Down Expand Up @@ -99,6 +99,58 @@ func TestWriter_WriteSync(t *testing.T) {
`, len(fetches.Records()[0].Value))), "cortex_ingest_storage_writer_sent_bytes_total"))
})

t.Run("should write to the requested partition", func(t *testing.T) {
t.Parallel()

for _, writeClients := range []int{1, 2, 10} {
writeClients := writeClients

t.Run(fmt.Sprintf("Write clients = %d", writeClients), func(t *testing.T) {
t.Parallel()

seriesPerPartition := map[int32][]mimirpb.PreallocTimeseries{
0: series1,
1: series2,
}

_, clusterAddr := testkafka.CreateCluster(t, numPartitions, topicName)
config := createTestKafkaConfig(clusterAddr, topicName)
config.WriteClients = writeClients
writer, _ := createTestWriter(t, config)

// Write to partitions.
for partitionID, series := range seriesPerPartition {
err := writer.WriteSync(ctx, partitionID, tenantID, &mimirpb.WriteRequest{Timeseries: series, Metadata: nil, Source: mimirpb.API})
require.NoError(t, err)
}

// Read back from Kafka.
for partitionID, expectedSeries := range seriesPerPartition {
consumer, err := kgo.NewClient(kgo.SeedBrokers(clusterAddr), kgo.ConsumePartitions(map[string]map[int32]kgo.Offset{topicName: {partitionID: kgo.NewOffset().AtStart()}}))
require.NoError(t, err)
t.Cleanup(consumer.Close)

fetchCtx, cancel := context.WithTimeout(ctx, time.Second)
t.Cleanup(cancel)

fetches := consumer.PollFetches(fetchCtx)
require.NoError(t, fetches.Err())
require.Len(t, fetches.Records(), 1)
assert.Equal(t, []byte(tenantID), fetches.Records()[0].Key)

received := mimirpb.WriteRequest{}
require.NoError(t, received.Unmarshal(fetches.Records()[0].Value))
require.Len(t, received.Timeseries, len(expectedSeries))

for idx, expected := range expectedSeries {
assert.Equal(t, expected.Labels, received.Timeseries[idx].Labels)
assert.Equal(t, expected.Samples, received.Timeseries[idx].Samples)
}
}
})
}
})

t.Run("should interrupt the WriteSync() on context cancelled but other concurrent requests should not fail", func(t *testing.T) {
t.Parallel()

Expand Down

0 comments on commit 3ef6b31

Please sign in to comment.