milvus/pkg/mq/msgstream/mq_factory.go

202 lines
6.9 KiB
Go
Raw Normal View History

// 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 msgstream
import (
"context"
"strings"
"time"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/cockroachdb/errors"
"github.com/streamnative/pulsarctl/pkg/cli"
"github.com/streamnative/pulsarctl/pkg/pulsar/utils"
"go.uber.org/zap"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
kafkawrapper "github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper/kafka"
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper/nmq"
pulsarmqwrapper "github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper/pulsar"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry"
)
// PmsFactory is a pulsar msgstream factory that implemented Factory interface(msgstream.go)
type PmsFactory struct {
dispatcherFactory ProtoUDFactory
// the following members must be public, so that mapstructure.Decode() can access them
PulsarAddress string
PulsarWebAddress string
ReceiveBufSize int64
MQBufSize int64
PulsarAuthPlugin string
PulsarAuthParams string
PulsarTenant string
PulsarNameSpace string
RequestTimeout time.Duration
}
func NewPmsFactory(serviceParam *paramtable.ServiceParam) *PmsFactory {
config := &serviceParam.PulsarCfg
return &PmsFactory{
MQBufSize: serviceParam.MQCfg.MQBufSize.GetAsInt64(),
ReceiveBufSize: serviceParam.MQCfg.ReceiveBufSize.GetAsInt64(),
PulsarAddress: config.Address.GetValue(),
PulsarWebAddress: config.WebAddress.GetValue(),
PulsarAuthPlugin: config.AuthPlugin.GetValue(),
PulsarAuthParams: config.AuthParams.GetValue(),
PulsarTenant: config.Tenant.GetValue(),
PulsarNameSpace: config.Namespace.GetValue(),
RequestTimeout: config.RequestTimeout.GetAsDuration(time.Second),
}
}
// NewMsgStream is used to generate a new Msgstream object
func (f *PmsFactory) NewMsgStream(ctx context.Context) (MsgStream, error) {
auth, err := f.getAuthentication()
if err != nil {
return nil, err
}
clientOpts := pulsar.ClientOptions{
URL: f.PulsarAddress,
Authentication: auth,
OperationTimeout: f.RequestTimeout,
}
pulsarClient, err := pulsarmqwrapper.NewClient(f.PulsarTenant, f.PulsarNameSpace, clientOpts)
if err != nil {
return nil, err
}
return NewMqMsgStream(ctx, f.ReceiveBufSize, f.MQBufSize, pulsarClient, f.dispatcherFactory.NewUnmarshalDispatcher())
}
// NewTtMsgStream is used to generate a new TtMsgstream object
func (f *PmsFactory) NewTtMsgStream(ctx context.Context) (MsgStream, error) {
auth, err := f.getAuthentication()
if err != nil {
return nil, err
}
clientOpts := pulsar.ClientOptions{
URL: f.PulsarAddress,
Authentication: auth,
}
pulsarClient, err := pulsarmqwrapper.NewClient(f.PulsarTenant, f.PulsarNameSpace, clientOpts)
if err != nil {
return nil, err
}
return NewMqTtMsgStream(ctx, f.ReceiveBufSize, f.MQBufSize, pulsarClient, f.dispatcherFactory.NewUnmarshalDispatcher())
}
func (f *PmsFactory) getAuthentication() (pulsar.Authentication, error) {
auth, err := pulsar.NewAuthentication(f.PulsarAuthPlugin, f.PulsarAuthParams)
if err != nil {
log.Error("build authencation from config failed, please check it!",
zap.String("authPlugin", f.PulsarAuthPlugin),
zap.Error(err))
return nil, errors.New("build authencation from config failed")
}
return auth, nil
}
func (f *PmsFactory) NewMsgStreamDisposer(ctx context.Context) func([]string, string) error {
return func(channels []string, subname string) error {
// try to delete the old subscription
admin, err := pulsarmqwrapper.NewAdminClient(f.PulsarWebAddress, f.PulsarAuthPlugin, f.PulsarAuthParams)
if err != nil {
return err
}
for _, channel := range channels {
fullTopicName, err := pulsarmqwrapper.GetFullTopicName(f.PulsarTenant, f.PulsarNameSpace, channel)
if err != nil {
return err
}
topic, err := utils.GetTopicName(fullTopicName)
if err != nil {
log.Warn("failed to get topic name", zap.Error(err))
return retry.Unrecoverable(err)
}
err = admin.Subscriptions().Delete(*topic, subname, true)
if err != nil {
pulsarErr, ok := err.(cli.Error)
if ok {
// subscription not found, ignore error
if strings.Contains(pulsarErr.Reason, "Subscription not found") {
return nil
}
}
log.Warn("failed to clean up subscriptions", zap.String("pulsar web", f.PulsarWebAddress),
zap.String("topic", channel), zap.Any("subname", subname), zap.Error(err))
}
}
return nil
}
}
type KmsFactory struct {
dispatcherFactory ProtoUDFactory
config *paramtable.KafkaConfig
ReceiveBufSize int64
MQBufSize int64
}
func (f *KmsFactory) NewMsgStream(ctx context.Context) (MsgStream, error) {
kafkaClient := kafkawrapper.NewKafkaClientInstanceWithConfig(f.config)
return NewMqMsgStream(ctx, f.ReceiveBufSize, f.MQBufSize, kafkaClient, f.dispatcherFactory.NewUnmarshalDispatcher())
}
func (f *KmsFactory) NewTtMsgStream(ctx context.Context) (MsgStream, error) {
kafkaClient := kafkawrapper.NewKafkaClientInstanceWithConfig(f.config)
return NewMqTtMsgStream(ctx, f.ReceiveBufSize, f.MQBufSize, kafkaClient, f.dispatcherFactory.NewUnmarshalDispatcher())
}
func (f *KmsFactory) NewMsgStreamDisposer(ctx context.Context) func([]string, string) error {
return func(channels []string, subname string) error {
msgstream, err := f.NewMsgStream(ctx)
if err != nil {
return err
}
msgstream.AsConsumer(channels, subname, mqwrapper.SubscriptionPositionUnknown)
msgstream.Close()
return nil
}
}
func NewKmsFactory(config *paramtable.ServiceParam) Factory {
f := &KmsFactory{
dispatcherFactory: ProtoUDFactory{},
ReceiveBufSize: config.MQCfg.ReceiveBufSize.GetAsInt64(),
MQBufSize: config.MQCfg.MQBufSize.GetAsInt64(),
config: &config.KafkaCfg,
}
return f
}
// NewNatsmqFactory create a new nats-mq factory.
func NewNatsmqFactory() Factory {
paramtable.Init()
paramtable := paramtable.Get()
nmq.MustInitNatsMQ(nmq.ParseServerOption(paramtable))
return &CommonFactory{
Newer: nmq.NewClientWithDefaultOptions,
DispatcherFactory: ProtoUDFactory{},
ReceiveBufSize: paramtable.MQCfg.ReceiveBufSize.GetAsInt64(),
MQBufSize: paramtable.MQCfg.MQBufSize.GetAsInt64(),
}
}