Fix Chunk Manager Path mis use (#19141)

Signed-off-by: xiaofan-luan <xiaofan.luan@zilliz.com>

Signed-off-by: xiaofan-luan <xiaofan.luan@zilliz.com>
pull/19400/head
Xiaofan 2022-09-23 14:40:51 +08:00 committed by GitHub
parent c9195c56a3
commit 928a213e31
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 96 additions and 179 deletions

View File

@ -47,7 +47,7 @@ const (
)
type mck struct {
params paramtable.GrpcServerConfig
params *paramtable.ComponentParam
taskKeyMap map[int64]string
taskNameMap map[int64]string
allTaskInfo map[string]string
@ -232,27 +232,12 @@ func (c *mck) connectEctd() {
}
func (c *mck) connectMinio() {
useSSL := c.params.MinioCfg.UseSSL
if c.minioUseSSL == "true" || c.minioUseSSL == "false" {
minioUseSSL, err := strconv.ParseBool(c.minioUseSSL)
if err != nil {
log.Panic("fail to parse the 'minioUseSSL' string to the bool value", zap.String("minioUseSSL", c.minioUseSSL), zap.Error(err))
}
useSSL = minioUseSSL
}
chunkManagerFactory := storage.NewChunkManagerFactory("local", "minio",
storage.RootPath(c.params.LocalStorageCfg.Path),
storage.Address(getConfigValue(c.minioAddress, c.params.MinioCfg.Address, "minio_address")),
storage.AccessKeyID(getConfigValue(c.minioUsername, c.params.MinioCfg.AccessKeyID, "minio_username")),
storage.SecretAccessKeyID(getConfigValue(c.minioPassword, c.params.MinioCfg.SecretAccessKey, "minio_password")),
storage.UseSSL(useSSL),
storage.BucketName(getConfigValue(c.minioBucketName, c.params.MinioCfg.BucketName, "minio_bucket_name")),
storage.CreateBucket(true))
chunkManagerFactory := storage.NewChunkManagerFactoryWithParam(c.params)
var err error
c.minioChunkManager, err = chunkManagerFactory.NewVectorStorageChunkManager(context.Background())
c.minioChunkManager, err = chunkManagerFactory.NewPersistentStorageChunkManager(context.Background())
if err != nil {
log.Fatal("failed to connect to etcd", zap.Error(err))
log.Fatal("failed to connect to minio", zap.Error(err))
}
}

View File

@ -48,7 +48,6 @@ type GcOption struct {
checkInterval time.Duration // each interval
missingTolerance time.Duration // key missing in meta tolerance time
dropTolerance time.Duration // dropped segment related key tolerance time
rootPath string
}
// garbageCollector handles garbage files in object storage
@ -130,9 +129,9 @@ func (gc *garbageCollector) scan() {
// walk only data cluster related prefixes
prefixes := make([]string, 0, 3)
prefixes = append(prefixes, path.Join(gc.option.rootPath, insertLogPrefix))
prefixes = append(prefixes, path.Join(gc.option.rootPath, statsLogPrefix))
prefixes = append(prefixes, path.Join(gc.option.rootPath, deltaLogPrefix))
prefixes = append(prefixes, path.Join(gc.option.cli.RootPath(), insertLogPrefix))
prefixes = append(prefixes, path.Join(gc.option.cli.RootPath(), statsLogPrefix))
prefixes = append(prefixes, path.Join(gc.option.cli.RootPath(), deltaLogPrefix))
var removedKeys []string
for _, prefix := range prefixes {
@ -148,7 +147,7 @@ func (gc *garbageCollector) scan() {
continue
}
segmentID, err := storage.ParseSegmentIDByBinlog(gc.option.rootPath, infoKey)
segmentID, err := storage.ParseSegmentIDByBinlog(gc.option.cli.RootPath(), infoKey)
if err != nil && !common.IsIgnorableError(err) {
log.Error("parse segment id error", zap.String("infoKey", infoKey), zap.Error(err))
continue

View File

@ -65,7 +65,6 @@ func Test_garbageCollector_basic(t *testing.T) {
checkInterval: time.Millisecond * 10,
missingTolerance: time.Hour * 24,
dropTolerance: time.Hour * 24,
rootPath: rootPath,
})
gc.start()
@ -82,7 +81,6 @@ func Test_garbageCollector_basic(t *testing.T) {
checkInterval: time.Millisecond * 10,
missingTolerance: time.Hour * 24,
dropTolerance: time.Hour * 24,
rootPath: rootPath,
})
assert.NotPanics(t, func() {
gc.start()
@ -145,7 +143,6 @@ func Test_garbageCollector_scan(t *testing.T) {
checkInterval: time.Minute * 30,
missingTolerance: time.Hour * 24,
dropTolerance: time.Hour * 24,
rootPath: rootPath,
})
gc.segRefer = segReferManager
gc.scan()
@ -167,7 +164,6 @@ func Test_garbageCollector_scan(t *testing.T) {
checkInterval: time.Minute * 30,
missingTolerance: time.Hour * 24,
dropTolerance: time.Hour * 24,
rootPath: rootPath,
})
gc.scan()
@ -193,7 +189,6 @@ func Test_garbageCollector_scan(t *testing.T) {
checkInterval: time.Minute * 30,
missingTolerance: time.Hour * 24,
dropTolerance: time.Hour * 24,
rootPath: rootPath,
})
gc.start()
gc.scan()
@ -222,7 +217,6 @@ func Test_garbageCollector_scan(t *testing.T) {
checkInterval: time.Minute * 30,
missingTolerance: time.Hour * 24,
dropTolerance: 0,
rootPath: rootPath,
})
gc.clearEtcd()
validateMinioPrefixElements(t, cli.Client, bucketName, path.Join(rootPath, insertLogPrefix), inserts[1:])
@ -239,7 +233,6 @@ func Test_garbageCollector_scan(t *testing.T) {
checkInterval: time.Minute * 30,
missingTolerance: 0,
dropTolerance: 0,
rootPath: rootPath,
})
gc.start()
gc.scan()
@ -324,7 +317,7 @@ func initUtOSSEnv(bucket, root string, n int) (mcm *storage.MinioChunkManager, i
mcm = &storage.MinioChunkManager{
Client: cli,
}
mcm.SetVar(context.TODO(), bucket)
mcm.SetVar(context.TODO(), bucket, root)
return mcm, inserts, stats, delta, other, nil
}

View File

@ -361,39 +361,17 @@ func (s *Server) stopCompactionTrigger() {
func (s *Server) initGarbageCollection() error {
var cli storage.ChunkManager
var err error
if Params.CommonCfg.StorageType == "minio" {
chunkManagerFactory := storage.NewChunkManagerFactory("local", "minio",
storage.RootPath(Params.LocalStorageCfg.Path),
storage.Address(Params.MinioCfg.Address),
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
storage.UseSSL(Params.MinioCfg.UseSSL),
storage.BucketName(Params.MinioCfg.BucketName),
storage.UseIAM(Params.MinioCfg.UseIAM),
storage.IAMEndpoint(Params.MinioCfg.IAMEndpoint),
storage.CreateBucket(true))
cli, err = chunkManagerFactory.NewVectorStorageChunkManager(s.ctx)
if err != nil {
log.Error("minio chunk manager init failed", zap.String("error", err.Error()))
return err
}
log.Info("minio chunk manager init success", zap.String("bucketname", Params.MinioCfg.BucketName))
} else if Params.CommonCfg.StorageType == "local" {
chunkManagerFactory := storage.NewChunkManagerFactory("local", "local",
storage.RootPath(Params.LocalStorageCfg.Path))
cli, err = chunkManagerFactory.NewVectorStorageChunkManager(s.ctx)
if err != nil {
log.Error("local chunk manager init failed", zap.String("error", err.Error()))
return err
}
log.Info("local chunk manager init success")
chunkManagerFactory := storage.NewChunkManagerFactoryWithParam(&Params)
cli, err = chunkManagerFactory.NewPersistentStorageChunkManager(s.ctx)
if err != nil {
log.Error("chunk manager init failed", zap.Error(err))
return err
}
log.Info("Datacoord garbage collector chunk manager init success")
s.garbageCollector = newGarbageCollector(s.meta, s.segReferManager, s.indexCoord, GcOption{
cli: cli,
enabled: Params.DataCoordCfg.EnableGarbageCollection,
rootPath: Params.MinioCfg.RootPath,
cli: cli,
enabled: Params.DataCoordCfg.EnableGarbageCollection,
checkInterval: Params.DataCoordCfg.GCInterval,
missingTolerance: Params.DataCoordCfg.GCMissingTolerance,
dropTolerance: Params.DataCoordCfg.GCDropTolerance,

View File

@ -494,7 +494,7 @@ func (node *DataNode) Start() error {
return errors.New("DataNode fail to connect etcd")
}
chunkManager, err := node.factory.NewVectorStorageChunkManager(node.ctx)
chunkManager, err := node.factory.NewPersistentStorageChunkManager(node.ctx)
if err != nil {
return err

View File

@ -209,7 +209,7 @@ func (i *IndexCoord) Init() error {
// TODO silverxia add Rewatch logic
i.eventChan = i.session.WatchServices(typeutil.IndexNodeRole, revision+1, nil)
chunkManager, err := i.factory.NewVectorStorageChunkManager(i.loopCtx)
chunkManager, err := i.factory.NewPersistentStorageChunkManager(i.loopCtx)
if err != nil {
log.Error("IndexCoord new minio chunkManager failed", zap.Error(err))
initErr = err

View File

@ -5,6 +5,7 @@ import (
"fmt"
"sync"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/storage"
)
@ -23,24 +24,13 @@ func (m *chunkMgr) NewChunkManager(ctx context.Context, config *indexpb.StorageC
return v.(storage.ChunkManager), nil
}
opts := make([]storage.Option, 0)
if config.StorageType == "local" {
opts = append(opts, storage.RootPath(config.RootPath))
} else {
opts = append(opts, storage.Address(config.Address),
storage.AccessKeyID(config.AccessKeyID),
storage.SecretAccessKeyID(config.SecretAccessKey),
storage.UseSSL(config.UseSSL),
storage.BucketName(config.BucketName),
storage.UseIAM(config.UseIAM),
storage.IAMEndpoint(config.IAMEndpoint))
}
factory := storage.NewChunkManagerFactory("local", config.StorageType, opts...)
mgr, err := factory.NewVectorStorageChunkManager(ctx)
chunkManagerFactory := storage.NewChunkManagerFactoryWithParam(&Params)
mgr, err := chunkManagerFactory.NewPersistentStorageChunkManager(ctx)
if err != nil {
return nil, err
}
v, _ := m.cached.LoadOrStore(key, mgr)
log.Ctx(ctx).Info("index node successfully init chunk manager")
return v.(storage.ChunkManager), nil
}

View File

@ -227,7 +227,7 @@ func (f *mockFactory) NewCacheStorageChunkManager(context.Context) (storage.Chun
return nil, errNotImplErr
}
func (f *mockFactory) NewVectorStorageChunkManager(context.Context) (storage.ChunkManager, error) {
func (f *mockFactory) NewPersistentStorageChunkManager(context.Context) (storage.ChunkManager, error) {
if f.chunkMgr != nil {
return f.chunkMgr, nil
}

View File

@ -196,7 +196,7 @@ func (qc *QueryCoord) Init() error {
// we only try best to reload the leader addresses
reloadShardLeaderAddress(qc.meta, qc.cluster)
qc.chunkManager, initError = qc.factory.NewVectorStorageChunkManager(qc.loopCtx)
qc.chunkManager, initError = qc.factory.NewPersistentStorageChunkManager(qc.loopCtx)
if initError != nil {
log.Error("query coordinator init cluster failed", zap.Error(initError))

View File

@ -13,7 +13,6 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"go.uber.org/zap"
@ -37,20 +36,16 @@ type CoordinatorBroker struct {
dataCoord types.DataCoord
rootCoord types.RootCoord
indexCoord types.IndexCoord
cm storage.ChunkManager
}
func NewCoordinatorBroker(
dataCoord types.DataCoord,
rootCoord types.RootCoord,
indexCoord types.IndexCoord,
cm storage.ChunkManager) *CoordinatorBroker {
indexCoord types.IndexCoord) *CoordinatorBroker {
return &CoordinatorBroker{
dataCoord,
rootCoord,
indexCoord,
cm,
}
}

View File

@ -35,7 +35,6 @@ import (
"github.com/milvus-io/milvus/internal/querycoordv2/session"
"github.com/milvus-io/milvus/internal/querycoordv2/task"
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
@ -60,7 +59,6 @@ type Server struct {
idAllocator func() (int64, error)
factory dependency.Factory
metricsCacheManager *metricsinfo.MetricsCacheManager
chunkManager storage.ChunkManager
// Coordinators
dataCoord types.DataCoord
@ -159,13 +157,6 @@ func (s *Server) Init() error {
// Init metrics cache manager
s.metricsCacheManager = metricsinfo.NewMetricsCacheManager()
// Init chunk manager
s.chunkManager, err = s.factory.NewVectorStorageChunkManager(s.ctx)
if err != nil {
log.Error("failed to init chunk manager", zap.Error(err))
return err
}
// Init meta
err = s.initMeta()
if err != nil {
@ -252,7 +243,6 @@ func (s *Server) initMeta() error {
s.dataCoord,
s.rootCoord,
s.indexCoord,
s.chunkManager,
)
return nil
}

View File

@ -1741,9 +1741,12 @@ func genSimpleQueryNodeWithMQFactory(ctx context.Context, fac dependency.Factory
// init shard cluster service
node.ShardClusterService = newShardClusterService(node.etcdCli, node.session, node)
node.queryShardService = newQueryShardService(node.queryNodeLoopCtx,
node.queryShardService, err = newQueryShardService(node.queryNodeLoopCtx,
node.metaReplica, node.tSafeReplica,
node.ShardClusterService, node.factory, node.scheduler)
if err != nil {
return nil, err
}
node.UpdateStateCode(internalpb.StateCode_Healthy)
@ -1908,7 +1911,7 @@ func (mm *mockMsgStreamFactory) NewQueryMsgStream(ctx context.Context) (msgstrea
func (mm *mockMsgStreamFactory) NewCacheStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
return nil, nil
}
func (mm *mockMsgStreamFactory) NewVectorStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
func (mm *mockMsgStreamFactory) NewPersistentStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
return nil, nil
}

View File

@ -115,7 +115,6 @@ type QueryNode struct {
eventCh <-chan *sessionutil.SessionEvent
vectorStorage storage.ChunkManager
cacheStorage storage.ChunkManager
etcdKV *etcdkv.EtcdKV
// shard cluster service, handle shard leader functions
@ -245,20 +244,13 @@ func (node *QueryNode) Init() error {
}
log.Info("QueryNode init rateCollector done", zap.Int64("nodeID", Params.QueryNodeCfg.GetNodeID()))
node.vectorStorage, err = node.factory.NewVectorStorageChunkManager(node.queryNodeLoopCtx)
node.vectorStorage, err = node.factory.NewPersistentStorageChunkManager(node.queryNodeLoopCtx)
if err != nil {
log.Error("QueryNode init vector storage failed", zap.Error(err))
initError = err
return
}
node.cacheStorage, err = node.factory.NewCacheStorageChunkManager(node.queryNodeLoopCtx)
if err != nil {
log.Error("QueryNode init cache storage failed", zap.Error(err))
initError = err
return
}
node.etcdKV = etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath)
log.Info("queryNode try to connect etcd success", zap.Any("MetaRootPath", Params.EtcdCfg.MetaRootPath))
@ -320,8 +312,12 @@ func (node *QueryNode) Start() error {
// create shardClusterService for shardLeader functions.
node.ShardClusterService = newShardClusterService(node.etcdCli, node.session, node)
// create shard-level query service
node.queryShardService = newQueryShardService(node.queryNodeLoopCtx, node.metaReplica, node.tSafeReplica,
queryShardService, err := newQueryShardService(node.queryNodeLoopCtx, node.metaReplica, node.tSafeReplica,
node.ShardClusterService, node.factory, node.scheduler)
if err != nil {
return err
}
node.queryShardService = queryShardService
Params.QueryNodeCfg.CreatedTime = time.Now()
Params.QueryNodeCfg.UpdatedTime = time.Now()

View File

@ -103,11 +103,7 @@ func newQueryNodeMock() *QueryNode {
replica := newCollectionReplica(pool)
svr.metaReplica = replica
svr.dataSyncService = newDataSyncService(ctx, svr.metaReplica, tsReplica, factory)
svr.vectorStorage, err = factory.NewVectorStorageChunkManager(ctx)
if err != nil {
panic(err)
}
svr.cacheStorage, err = factory.NewCacheStorageChunkManager(ctx)
svr.vectorStorage, err = factory.NewPersistentStorageChunkManager(ctx)
if err != nil {
panic(err)
}

View File

@ -47,14 +47,15 @@ type queryShardService struct {
scheduler *taskScheduler
}
func newQueryShardService(ctx context.Context, metaReplica ReplicaInterface, tSafeReplica TSafeReplicaInterface, clusterService *ShardClusterService, factory dependency.Factory, scheduler *taskScheduler) *queryShardService {
func newQueryShardService(ctx context.Context, metaReplica ReplicaInterface, tSafeReplica TSafeReplicaInterface, clusterService *ShardClusterService, factory dependency.Factory, scheduler *taskScheduler) (*queryShardService, error) {
// TODO we don't need the local chunk manager any more
localChunkManager := storage.NewLocalChunkManager(storage.RootPath(Params.LocalStorageCfg.Path))
remoteChunkManager, err := factory.NewPersistentStorageChunkManager(ctx)
if err != nil {
log.Ctx(ctx).Warn("failed to init remote chunk manager", zap.Error(err))
return nil, err
}
queryShardServiceCtx, queryShardServiceCancel := context.WithCancel(ctx)
path := Params.LoadWithDefault("localStorage.Path", "/tmp/milvus/data")
localChunkManager := storage.NewLocalChunkManager(storage.RootPath(path))
remoteChunkManager, _ := factory.NewVectorStorageChunkManager(ctx)
qss := &queryShardService{
ctx: queryShardServiceCtx,
cancel: queryShardServiceCancel,
@ -68,7 +69,7 @@ func newQueryShardService(ctx context.Context, metaReplica ReplicaInterface, tSa
factory: factory,
scheduler: scheduler,
}
return qss
return qss, nil
}
func (q *queryShardService) addQueryShard(collectionID UniqueID, channel Channel, replicaID int64) error {

View File

@ -28,7 +28,8 @@ func TestQueryShardService(t *testing.T) {
qn, err := genSimpleQueryNode(context.Background())
require.NoError(t, err)
qss := newQueryShardService(context.Background(), qn.metaReplica, qn.tSafeReplica, qn.ShardClusterService, qn.factory, qn.scheduler)
qss, err := newQueryShardService(context.Background(), qn.metaReplica, qn.tSafeReplica, qn.ShardClusterService, qn.factory, qn.scheduler)
assert.NoError(t, err)
err = qss.addQueryShard(0, "vchan1", 0)
assert.NoError(t, err)
found1 := qss.hasQueryShard("vchan1")
@ -50,7 +51,8 @@ func TestQueryShardService_InvalidChunkManager(t *testing.T) {
qn, err := genSimpleQueryNode(context.Background())
require.NoError(t, err)
qss := newQueryShardService(context.Background(), qn.metaReplica, qn.tSafeReplica, qn.ShardClusterService, qn.factory, qn.scheduler)
qss, err := newQueryShardService(context.Background(), qn.metaReplica, qn.tSafeReplica, qn.ShardClusterService, qn.factory, qn.scheduler)
assert.NoError(t, err)
lcm := qss.localChunkManager
qss.localChunkManager = nil

View File

@ -3,23 +3,39 @@ package storage
import (
"context"
"errors"
"github.com/milvus-io/milvus/internal/util/paramtable"
)
type ChunkManagerFactory struct {
cacheStorage string
vectorStorage string
config *config
persistentStorage string
config *config
}
func NewChunkManagerFactory(cacheStorage, vectorStorage string, opts ...Option) *ChunkManagerFactory {
func NewChunkManagerFactoryWithParam(params *paramtable.ComponentParam) *ChunkManagerFactory {
if params.CommonCfg.StorageType == "local" {
return NewChunkManagerFactory("local", RootPath(params.LocalStorageCfg.Path))
}
return NewChunkManagerFactory("minio",
RootPath(params.MinioCfg.RootPath),
Address(params.MinioCfg.Address),
AccessKeyID(params.MinioCfg.AccessKeyID),
SecretAccessKeyID(params.MinioCfg.SecretAccessKey),
UseSSL(params.MinioCfg.UseSSL),
BucketName(params.MinioCfg.BucketName),
UseIAM(params.MinioCfg.UseIAM),
IAMEndpoint(params.MinioCfg.IAMEndpoint),
CreateBucket(true))
}
func NewChunkManagerFactory(persistentStorage string, opts ...Option) *ChunkManagerFactory {
c := newDefaultConfig()
for _, opt := range opts {
opt(c)
}
return &ChunkManagerFactory{
cacheStorage: cacheStorage,
vectorStorage: vectorStorage,
config: c,
persistentStorage: persistentStorage,
config: c,
}
}
@ -34,15 +50,10 @@ func (f *ChunkManagerFactory) newChunkManager(ctx context.Context, engine string
}
}
func (f *ChunkManagerFactory) NewCacheStorageChunkManager(ctx context.Context) (ChunkManager, error) {
return f.newChunkManager(ctx, f.cacheStorage)
}
func (f *ChunkManagerFactory) NewVectorStorageChunkManager(ctx context.Context) (ChunkManager, error) {
return f.newChunkManager(ctx, f.vectorStorage)
func (f *ChunkManagerFactory) NewPersistentStorageChunkManager(ctx context.Context) (ChunkManager, error) {
return f.newChunkManager(ctx, f.persistentStorage)
}
type Factory interface {
NewCacheStorageChunkManager(ctx context.Context) (ChunkManager, error)
NewVectorStorageChunkManager(ctx context.Context) (ChunkManager, error)
NewPersistentStorageChunkManager(ctx context.Context) (ChunkManager, error)
}

View File

@ -48,7 +48,7 @@ type MinioChunkManager struct {
var _ ChunkManager = (*MinioChunkManager)(nil)
// NewMinioChunkManager create a new local manager object.
// Deprecated: Do not call this directly! Use factory.NewVectorStorageChunkManager instead.
// Deprecated: Do not call this directly! Use factory.NewPersistentStorageChunkManager instead.
func NewMinioChunkManager(ctx context.Context, opts ...Option) (*MinioChunkManager, error) {
c := newDefaultConfig()
for _, opt := range opts {
@ -117,9 +117,10 @@ func (mcm *MinioChunkManager) normalizeRootPath(rootPath string) string {
}
// SetVar set the variable value of mcm
func (mcm *MinioChunkManager) SetVar(ctx context.Context, bucketName string) {
func (mcm *MinioChunkManager) SetVar(ctx context.Context, bucketName string, rootPath string) {
mcm.ctx = ctx
mcm.bucketName = bucketName
mcm.rootPath = rootPath
}
// RootPath returns minio root path.

View File

@ -14,11 +14,12 @@ type DefaultFactory struct {
msgStreamFactory msgstream.Factory
}
// Only for test
func NewDefaultFactory(standAlone bool) *DefaultFactory {
return &DefaultFactory{
standAlone: standAlone,
msgStreamFactory: msgstream.NewRmsFactory("/tmp/milvus/rocksmq/"),
chunkManagerFactory: storage.NewChunkManagerFactory("local", "local",
chunkManagerFactory: storage.NewChunkManagerFactory("local",
storage.RootPath("/tmp/milvus")),
}
}
@ -37,33 +38,14 @@ func (f *DefaultFactory) Init(params *paramtable.ComponentParam) {
return
}
// init storage
if params.CommonCfg.StorageType == "local" {
f.chunkManagerFactory = storage.NewChunkManagerFactory("local", "local",
storage.RootPath(params.LocalStorageCfg.Path))
} else {
f.chunkManagerFactory = storage.NewChunkManagerFactory("local", "minio",
storage.RootPath(params.MinioCfg.RootPath),
storage.Address(params.MinioCfg.Address),
storage.AccessKeyID(params.MinioCfg.AccessKeyID),
storage.SecretAccessKeyID(params.MinioCfg.SecretAccessKey),
storage.UseSSL(params.MinioCfg.UseSSL),
storage.BucketName(params.MinioCfg.BucketName),
storage.UseIAM(params.MinioCfg.UseIAM),
storage.IAMEndpoint(params.MinioCfg.IAMEndpoint),
storage.CreateBucket(true))
}
f.chunkManagerFactory = storage.NewChunkManagerFactoryWithParam(params)
// init mq storage
if f.standAlone {
f.msgStreamFactory = f.initMQLocalService(params)
if f.msgStreamFactory == nil {
f.msgStreamFactory = f.initMQRemoteService(params)
if f.msgStreamFactory == nil {
panic("no available mq configuration, must config rocksmq, Pulsar or Kafka at least one of these!")
}
if f.msgStreamFactory != nil {
return
}
return
}
f.msgStreamFactory = f.initMQRemoteService(params)
@ -112,17 +94,12 @@ func (f *DefaultFactory) NewMsgStreamDisposer(ctx context.Context) func([]string
return f.msgStreamFactory.NewMsgStreamDisposer(ctx)
}
func (f *DefaultFactory) NewCacheStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
return f.chunkManagerFactory.NewCacheStorageChunkManager(ctx)
}
func (f *DefaultFactory) NewVectorStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
return f.chunkManagerFactory.NewVectorStorageChunkManager(ctx)
func (f *DefaultFactory) NewPersistentStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
return f.chunkManagerFactory.NewPersistentStorageChunkManager(ctx)
}
type Factory interface {
msgstream.Factory
Init(p *paramtable.ComponentParam)
NewCacheStorageChunkManager(ctx context.Context) (storage.ChunkManager, error)
NewVectorStorageChunkManager(ctx context.Context) (storage.ChunkManager, error)
NewPersistentStorageChunkManager(ctx context.Context) (storage.ChunkManager, error)
}

View File

@ -98,7 +98,7 @@ func (mc *MockChunkManager) RemoveWithPrefix(prefix string) error {
func Test_NewImportWrapper(t *testing.T) {
f := dependency.NewDefaultFactory(true)
ctx := context.Background()
cm, err := f.NewVectorStorageChunkManager(ctx)
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, nil, 2, 1, nil, cm, nil, nil, nil)
@ -129,7 +129,7 @@ func Test_NewImportWrapper(t *testing.T) {
func Test_ImportRowBased(t *testing.T) {
f := dependency.NewDefaultFactory(true)
ctx := context.Background()
cm, err := f.NewVectorStorageChunkManager(ctx)
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(t, err)
idAllocator := newIDAllocator(ctx, t)
@ -216,7 +216,7 @@ func Test_ImportRowBased(t *testing.T) {
func Test_ImportColumnBased_json(t *testing.T) {
f := dependency.NewDefaultFactory(true)
ctx := context.Background()
cm, err := f.NewVectorStorageChunkManager(ctx)
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(t, err)
defer cm.RemoveWithPrefix("")
@ -316,7 +316,7 @@ func Test_ImportColumnBased_json(t *testing.T) {
func Test_ImportColumnBased_StringKey(t *testing.T) {
f := dependency.NewDefaultFactory(true)
ctx := context.Background()
cm, err := f.NewVectorStorageChunkManager(ctx)
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(t, err)
defer cm.RemoveWithPrefix("")
@ -383,7 +383,7 @@ func Test_ImportColumnBased_StringKey(t *testing.T) {
func Test_ImportColumnBased_numpy(t *testing.T) {
f := dependency.NewDefaultFactory(true)
ctx := context.Background()
cm, err := f.NewVectorStorageChunkManager(ctx)
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(t, err)
defer cm.RemoveWithPrefix("")
@ -520,7 +520,7 @@ func perfSchema(dim int) *schemapb.CollectionSchema {
func Test_ImportRowBased_perf(t *testing.T) {
f := dependency.NewDefaultFactory(true)
ctx := context.Background()
cm, err := f.NewVectorStorageChunkManager(ctx)
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(t, err)
defer cm.RemoveWithPrefix("")
@ -621,7 +621,7 @@ func Test_ImportRowBased_perf(t *testing.T) {
func Test_ImportColumnBased_perf(t *testing.T) {
f := dependency.NewDefaultFactory(true)
ctx := context.Background()
cm, err := f.NewVectorStorageChunkManager(ctx)
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(t, err)
defer cm.RemoveWithPrefix("")