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

enhance: Use pool to limit kafka cgo thread number #37744

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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
84 changes: 52 additions & 32 deletions pkg/mq/msgstream/mqwrapper/kafka/kafka_consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,10 @@ func newKafkaConsumer(config *kafka.ConfigMap, bufSize int64, topic string, grou

func (kc *Consumer) createKafkaConsumer() error {
var err error
kc.c, err = kafka.NewConsumer(kc.config)
getPool().Submit(func() (any, error) {
kc.c, err = kafka.NewConsumer(kc.config)
return nil, err
}).Await()
if err != nil {
log.Error("create kafka consumer failed", zap.String("topic", kc.topic), zap.Error(err))
return err
Expand Down Expand Up @@ -133,7 +136,12 @@ func (kc *Consumer) Chan() <-chan common.Message {
return
default:
readTimeout := paramtable.Get().KafkaCfg.ReadTimeout.GetAsDuration(time.Second)
e, err := kc.c.ReadMessage(readTimeout)
var e *kafka.Message
var err error
getPool().Submit(func() (any, error) {
e, err = kc.c.ReadMessage(readTimeout)
return nil, err
}).Await()
if err != nil {
// if we failed to read message in 30 Seconds, print out a warn message since there should always be a tt
log.Warn("consume msg failed", zap.String("topic", kc.topic), zap.String("groupID", kc.groupID), zap.Error(err))
Expand Down Expand Up @@ -166,38 +174,40 @@ func (kc *Consumer) Seek(id common.MessageID, inclusive bool) error {
}

func (kc *Consumer) internalSeek(offset kafka.Offset, inclusive bool) error {
log.Info("kafka consumer seek start", zap.String("topic name", kc.topic),
zap.Any("Msg offset", offset), zap.Bool("inclusive", inclusive))
_, err := getPool().Submit(func() (any, error) {
log.Info("kafka consumer seek start", zap.String("topic name", kc.topic),
zap.Any("Msg offset", offset), zap.Bool("inclusive", inclusive))
start := time.Now()
err := kc.c.Assign([]kafka.TopicPartition{{Topic: &kc.topic, Partition: mqwrapper.DefaultPartitionIdx, Offset: offset}})
if err != nil {
log.Warn("kafka consumer assign failed ", zap.String("topic name", kc.topic), zap.Any("Msg offset", offset), zap.Error(err))
return nil, err
}

start := time.Now()
err := kc.c.Assign([]kafka.TopicPartition{{Topic: &kc.topic, Partition: mqwrapper.DefaultPartitionIdx, Offset: offset}})
if err != nil {
log.Warn("kafka consumer assign failed ", zap.String("topic name", kc.topic), zap.Any("Msg offset", offset), zap.Error(err))
return err
}
cost := time.Since(start).Milliseconds()
if cost > 200 {
log.Warn("kafka consumer assign take too long!", zap.String("topic name", kc.topic),
zap.Any("Msg offset", offset), zap.Bool("inclusive", inclusive), zap.Int64("time cost(ms)", cost))
}

cost := time.Since(start).Milliseconds()
if cost > 200 {
log.Warn("kafka consumer assign take too long!", zap.String("topic name", kc.topic),
// If seek timeout is not 0 the call twice will return error isStarted RD_KAFKA_RESP_ERR__STATE.
// if the timeout is 0 it will initiate the seek but return immediately without any error reporting
kc.skipMsg = !inclusive
if err := kc.c.Seek(kafka.TopicPartition{
Topic: &kc.topic,
Partition: mqwrapper.DefaultPartitionIdx,
Offset: offset,
}, timeout); err != nil {
return nil, err
}
cost = time.Since(start).Milliseconds()
log.Info("kafka consumer seek finished", zap.String("topic name", kc.topic),
zap.Any("Msg offset", offset), zap.Bool("inclusive", inclusive), zap.Int64("time cost(ms)", cost))
}

// If seek timeout is not 0 the call twice will return error isStarted RD_KAFKA_RESP_ERR__STATE.
// if the timeout is 0 it will initiate the seek but return immediately without any error reporting
kc.skipMsg = !inclusive
if err := kc.c.Seek(kafka.TopicPartition{
Topic: &kc.topic,
Partition: mqwrapper.DefaultPartitionIdx,
Offset: offset,
}, timeout); err != nil {
return err
}
cost = time.Since(start).Milliseconds()
log.Info("kafka consumer seek finished", zap.String("topic name", kc.topic),
zap.Any("Msg offset", offset), zap.Bool("inclusive", inclusive), zap.Int64("time cost(ms)", cost))

kc.hasAssign = true
return nil
kc.hasAssign = true
return nil, nil
}).Await()
return err
}

func (kc *Consumer) Ack(message common.Message) {
Expand All @@ -207,7 +217,13 @@ func (kc *Consumer) Ack(message common.Message) {
}

func (kc *Consumer) GetLatestMsgID() (common.MessageID, error) {
low, high, err := kc.c.QueryWatermarkOffsets(kc.topic, mqwrapper.DefaultPartitionIdx, timeout)
var low, high int64
var err error

getPool().Submit(func() (any, error) {
low, high, err = kc.c.QueryWatermarkOffsets(kc.topic, mqwrapper.DefaultPartitionIdx, timeout)
return nil, err
}).Await()
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -244,7 +260,11 @@ func (kc *Consumer) CheckTopicValid(topic string) error {
func (kc *Consumer) closeInternal() {
log.Info("close consumer ", zap.String("topic", kc.topic), zap.String("groupID", kc.groupID))
start := time.Now()
err := kc.c.Close()
var err error
getPool().Submit(func() (any, error) {
err = kc.c.Close()
return nil, err
}).Await()
if err != nil {
log.Warn("failed to close ", zap.String("topic", kc.topic), zap.Error(err))
}
Expand Down
14 changes: 9 additions & 5 deletions pkg/mq/msgstream/mqwrapper/kafka/kafka_producer.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,11 +44,15 @@ func (kp *kafkaProducer) Send(ctx context.Context, message *mqcommon.ProducerMes
header := kafka.Header{Key: key, Value: []byte(value)}
headers = append(headers, header)
}
err := kp.p.Produce(&kafka.Message{
TopicPartition: kafka.TopicPartition{Topic: &kp.topic, Partition: mqwrapper.DefaultPartitionIdx},
Value: message.Payload,
Headers: headers,
}, kp.deliveryChan)
var err error
getPool().Submit(func() (any, error) {
err = kp.p.Produce(&kafka.Message{
TopicPartition: kafka.TopicPartition{Topic: &kp.topic, Partition: mqwrapper.DefaultPartitionIdx},
Value: message.Payload,
Headers: headers,
}, kp.deliveryChan)
return nil, err
})
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
return nil, err
Expand Down
52 changes: 52 additions & 0 deletions pkg/mq/msgstream/mqwrapper/kafka/pool.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you 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
//
// http://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.

package kafka

import (
"runtime"
"sync"

"go.uber.org/atomic"
"go.uber.org/zap"

"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/conc"
"github.com/milvus-io/milvus/pkg/util/hardware"
)

var (
kafkaCPool atomic.Pointer[conc.Pool[any]]
initOnce sync.Once
)

func initPool() {
pool := conc.NewPool[any](
hardware.GetCPUNum(),
conc.WithPreAlloc(false),
conc.WithDisablePurge(false),
conc.WithPreHandler(runtime.LockOSThread), // lock os thread for cgo thread disposal
)

kafkaCPool.Store(pool)
log.Info("init dynamicPool done", zap.Int("size", hardware.GetCPUNum()))
}

// GetSQPool returns the singleton pool instance for search/query operations.
func getPool() *conc.Pool[any] {
initPool()
return kafkaCPool.Load()
}
Loading