Zhen Ye 4c6e33f326
fix: lost tenant/namespace support for pulsar since 2.6 (#46752)
issue: #46748

Signed-off-by: chyezh <chyezh@outlook.com>
2026-01-06 14:33:24 +08:00

76 lines
2.2 KiB
Go

package pulsar
import (
"context"
"fmt"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls"
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/helper"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
)
const (
truncateCursorSubscriptionName = "truncate-cursor"
defaultBacklogSize = 100 * 1024 * 1024 // default 100MB
)
var _ walimpls.OpenerImpls = (*openerImpl)(nil)
// tenant is the tenant of pulsar.
type tenant struct {
tenant string
namespace string
}
// MustGetFullTopicName gets the full topic name of pulsar.
// If the tenant or namespace or topic is empty, it will panic.
func (t tenant) MustGetFullTopicName(topic string) string {
if len(t.tenant) == 0 || len(t.namespace) == 0 || len(topic) == 0 {
panic("tenant or namespace or topic is empty")
}
return fmt.Sprintf("%s/%s/%s", t.tenant, t.namespace, topic)
}
// openerImpl is the opener for pulsar wal.
type openerImpl struct {
tenant tenant
c pulsar.Client
}
// Open opens a wal instance.
func (o *openerImpl) Open(ctx context.Context, opt *walimpls.OpenOption) (walimpls.WALImpls, error) {
if err := opt.Validate(); err != nil {
return nil, err
}
var backlogClearHelper *backlogClearHelper
if opt.Channel.AccessMode == types.AccessModeRW {
backlogAutoClearBytes := paramtable.Get().PulsarCfg.BacklogAutoClearBytes.GetAsSize()
if backlogAutoClearBytes <= 0 {
backlogAutoClearBytes = defaultBacklogSize
}
backlogClearHelper = newBacklogClearHelper(o.c, opt.Channel, backlogAutoClearBytes, o.tenant)
}
w := &walImpl{
WALHelper: helper.NewWALHelper(opt),
c: o.c,
p: syncutil.NewFuture[pulsar.Producer](),
notifier: syncutil.NewAsyncTaskNotifier[struct{}](),
backlogClearHelper: backlogClearHelper,
tenant: o.tenant,
}
// because the producer of pulsar cannot be created if the topic is backlog exceeded,
// so we need to set the producer at background with backoff retry.
w.initProducerAtBackground()
return w, nil
}
// Close closes the opener resources.
func (o *openerImpl) Close() {
o.c.Close()
}