mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 01:28:27 +08:00
enhance: support load/release collection/partition with WAL-based DDL framework (#45154)
issue: #43897 - Load/Release collection/partition is implemented by WAL-based DDL framework now. - Support AlterLoadConfig/DropLoadConfig in wal now. - Load/Release operation can be synced by new CDC now. - Refactor some UT for load/release DDL. --------- Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
parent
e25ee08566
commit
00d8d2c33d
@ -16,13 +16,23 @@
|
||||
|
||||
package querycoordv2
|
||||
|
||||
import "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/broadcast"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
// RegisterDDLCallbacks registers the ddl callbacks.
|
||||
func RegisterDDLCallbacks(s *Server) {
|
||||
ddlCallback := &DDLCallbacks{
|
||||
Server: s,
|
||||
}
|
||||
ddlCallback.registerLoadConfigCallbacks()
|
||||
ddlCallback.registerResourceGroupCallbacks()
|
||||
}
|
||||
|
||||
@ -30,11 +40,29 @@ type DDLCallbacks struct {
|
||||
*Server
|
||||
}
|
||||
|
||||
// registerLoadConfigCallbacks registers the load config callbacks.
|
||||
func (c *DDLCallbacks) registerLoadConfigCallbacks() {
|
||||
registry.RegisterAlterLoadConfigV2AckCallback(c.alterLoadConfigV2AckCallback)
|
||||
registry.RegisterDropLoadConfigV2AckCallback(c.dropLoadConfigV2AckCallback)
|
||||
}
|
||||
|
||||
func (c *DDLCallbacks) registerResourceGroupCallbacks() {
|
||||
registry.RegisterAlterResourceGroupV2AckCallback(c.alterResourceGroupV2AckCallback)
|
||||
registry.RegisterDropResourceGroupV2AckCallback(c.dropResourceGroupV2AckCallback)
|
||||
}
|
||||
|
||||
func (c *DDLCallbacks) RegisterDDLCallbacks() {
|
||||
c.registerResourceGroupCallbacks()
|
||||
// startBroadcastWithCollectionIDLock starts a broadcast with collection id lock.
|
||||
func (c *Server) startBroadcastWithCollectionIDLock(ctx context.Context, collectionID int64) (broadcaster.BroadcastAPI, error) {
|
||||
coll, err := c.broker.DescribeCollection(ctx, collectionID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
broadcaster, err := broadcast.StartBroadcastWithResourceKeys(ctx,
|
||||
message.NewSharedDBNameResourceKey(coll.GetDbName()),
|
||||
message.NewExclusiveCollectionNameResourceKey(coll.GetDbName(), coll.GetCollectionName()),
|
||||
)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to start broadcast with collection lock")
|
||||
}
|
||||
return broadcaster, nil
|
||||
}
|
||||
|
||||
37
internal/querycoordv2/ddl_callbacks_alter_load_info.go
Normal file
37
internal/querycoordv2/ddl_callbacks_alter_load_info.go
Normal file
@ -0,0 +1,37 @@
|
||||
// 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 querycoordv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/job"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
// alterLoadConfigV2AckCallback is called when the put load config message is acknowledged
|
||||
func (s *Server) alterLoadConfigV2AckCallback(ctx context.Context, result message.BroadcastResultAlterLoadConfigMessageV2) error {
|
||||
// currently, we only sent the put load config message to the control channel
|
||||
// TODO: after we support query view in 3.0, we should broadcast the put load config message to all vchannels.
|
||||
job := job.NewLoadCollectionJob(ctx, result, s.dist, s.meta, s.broker, s.targetMgr, s.targetObserver, s.collectionObserver, s.nodeMgr)
|
||||
if err := job.Execute(); err != nil {
|
||||
return err
|
||||
}
|
||||
meta.GlobalFailedLoadCache.Remove(result.Message.Header().GetCollectionId())
|
||||
return nil
|
||||
}
|
||||
@ -0,0 +1,126 @@
|
||||
// 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 querycoordv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/job"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
)
|
||||
|
||||
// broadcastAlterLoadConfigCollectionV2ForLoadCollection is called when the load collection request is received.
|
||||
func (s *Server) broadcastAlterLoadConfigCollectionV2ForLoadCollection(ctx context.Context, req *querypb.LoadCollectionRequest) error {
|
||||
broadcaster, err := s.startBroadcastWithCollectionIDLock(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer broadcaster.Close()
|
||||
|
||||
// double check if the collection is already dropped
|
||||
coll, err := s.broker.DescribeCollection(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
partitionIDs, err := s.broker.GetPartitions(ctx, coll.CollectionID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// if user specified the replica number in load request, load config changes won't be apply to the collection automatically
|
||||
userSpecifiedReplicaMode := req.GetReplicaNumber() > 0
|
||||
replicaNumber, resourceGroups, err := s.getDefaultResourceGroupsAndReplicaNumber(ctx, req.GetReplicaNumber(), req.GetResourceGroups(), req.GetCollectionID())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
currentLoadConfig := s.getCurrentLoadConfig(ctx, req.GetCollectionID())
|
||||
// only check node number when the collection is not loaded
|
||||
expectedReplicasNumber, err := utils.AssignReplica(ctx, s.meta, resourceGroups, replicaNumber, currentLoadConfig.Collection == nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
alterLoadConfigReq := &job.AlterLoadConfigRequest{
|
||||
Meta: s.meta,
|
||||
CollectionInfo: coll,
|
||||
Current: currentLoadConfig,
|
||||
Expected: job.ExpectedLoadConfig{
|
||||
ExpectedPartitionIDs: partitionIDs,
|
||||
ExpectedReplicaNumber: expectedReplicasNumber,
|
||||
ExpectedFieldIndexID: req.GetFieldIndexID(),
|
||||
ExpectedLoadFields: req.GetLoadFields(),
|
||||
ExpectedPriority: req.GetPriority(),
|
||||
ExpectedUserSpecifiedReplicaMode: userSpecifiedReplicaMode,
|
||||
},
|
||||
}
|
||||
msg, err := job.GenerateAlterLoadConfigMessage(ctx, alterLoadConfigReq)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, err = broadcaster.Broadcast(ctx, msg)
|
||||
return err
|
||||
}
|
||||
|
||||
// getDefaultResourceGroupsAndReplicaNumber gets the default resource groups and replica number for the collection.
|
||||
func (s *Server) getDefaultResourceGroupsAndReplicaNumber(ctx context.Context, replicaNumber int32, resourceGroups []string, collectionID int64) (int32, []string, error) {
|
||||
// so only both replica and resource groups didn't set in request, it will turn to use the configured load info
|
||||
if replicaNumber <= 0 && len(resourceGroups) == 0 {
|
||||
// when replica number or resource groups is not set, use pre-defined load config
|
||||
rgs, replicas, err := s.broker.GetCollectionLoadInfo(ctx, collectionID)
|
||||
if err != nil {
|
||||
log.Warn("failed to get pre-defined load info", zap.Error(err))
|
||||
} else {
|
||||
replicaNumber = int32(replicas)
|
||||
resourceGroups = rgs
|
||||
}
|
||||
}
|
||||
// to be compatible with old sdk, which set replica=1 if replica is not specified
|
||||
if replicaNumber <= 0 {
|
||||
log.Info("request doesn't indicate the number of replicas, set it to 1")
|
||||
replicaNumber = 1
|
||||
}
|
||||
if len(resourceGroups) == 0 {
|
||||
log.Info(fmt.Sprintf("request doesn't indicate the resource groups, set it to %s", meta.DefaultResourceGroupName))
|
||||
resourceGroups = []string{meta.DefaultResourceGroupName}
|
||||
}
|
||||
return replicaNumber, resourceGroups, nil
|
||||
}
|
||||
|
||||
func (s *Server) getCurrentLoadConfig(ctx context.Context, collectionID int64) job.CurrentLoadConfig {
|
||||
partitionList := s.meta.CollectionManager.GetPartitionsByCollection(ctx, collectionID)
|
||||
loadedPartitions := make(map[int64]*meta.Partition)
|
||||
for _, partitioin := range partitionList {
|
||||
loadedPartitions[partitioin.PartitionID] = partitioin
|
||||
}
|
||||
|
||||
replicas := s.meta.ReplicaManager.GetByCollection(ctx, collectionID)
|
||||
loadedReplicas := make(map[int64]*meta.Replica)
|
||||
for _, replica := range replicas {
|
||||
loadedReplicas[replica.GetID()] = replica
|
||||
}
|
||||
return job.CurrentLoadConfig{
|
||||
Collection: s.meta.CollectionManager.GetCollection(ctx, collectionID),
|
||||
Partitions: loadedPartitions,
|
||||
Replicas: loadedReplicas,
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,80 @@
|
||||
// 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 querycoordv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/job"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
func (s *Server) broadcastAlterLoadConfigCollectionV2ForLoadPartitions(ctx context.Context, req *querypb.LoadPartitionsRequest) error {
|
||||
broadcaster, err := s.startBroadcastWithCollectionIDLock(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer broadcaster.Close()
|
||||
|
||||
// double check if the collection is already dropped
|
||||
coll, err := s.broker.DescribeCollection(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
userSpecifiedReplicaMode := req.GetReplicaNumber() > 0
|
||||
replicaNumber, resourceGroups, err := s.getDefaultResourceGroupsAndReplicaNumber(ctx, req.GetReplicaNumber(), req.GetResourceGroups(), req.GetCollectionID())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
expectedReplicasNumber, err := utils.AssignReplica(ctx, s.meta, resourceGroups, replicaNumber, true)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
currentLoadConfig := s.getCurrentLoadConfig(ctx, req.GetCollectionID())
|
||||
partitionIDsSet := typeutil.NewSet(currentLoadConfig.GetPartitionIDs()...)
|
||||
// add new incoming partitionIDs.
|
||||
for _, partition := range req.PartitionIDs {
|
||||
partitionIDsSet.Insert(partition)
|
||||
}
|
||||
alterLoadConfigReq := &job.AlterLoadConfigRequest{
|
||||
Meta: s.meta,
|
||||
CollectionInfo: coll,
|
||||
Current: s.getCurrentLoadConfig(ctx, req.GetCollectionID()),
|
||||
Expected: job.ExpectedLoadConfig{
|
||||
ExpectedPartitionIDs: partitionIDsSet.Collect(),
|
||||
ExpectedReplicaNumber: expectedReplicasNumber,
|
||||
ExpectedFieldIndexID: req.GetFieldIndexID(),
|
||||
ExpectedLoadFields: req.GetLoadFields(),
|
||||
ExpectedPriority: req.GetPriority(),
|
||||
ExpectedUserSpecifiedReplicaMode: userSpecifiedReplicaMode,
|
||||
},
|
||||
}
|
||||
if err := alterLoadConfigReq.CheckIfLoadPartitionsExecutable(); err != nil {
|
||||
return err
|
||||
}
|
||||
msg, err := job.GenerateAlterLoadConfigMessage(ctx, alterLoadConfigReq)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, err = broadcaster.Broadcast(ctx, msg)
|
||||
return err
|
||||
}
|
||||
@ -0,0 +1,95 @@
|
||||
// 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 querycoordv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/job"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
func (s *Server) broadcastAlterLoadConfigCollectionV2ForReleasePartitions(ctx context.Context, req *querypb.ReleasePartitionsRequest) (collectionReleased bool, err error) {
|
||||
broadcaster, err := s.startBroadcastWithCollectionIDLock(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
defer broadcaster.Close()
|
||||
|
||||
// double check if the collection is already dropped
|
||||
coll, err := s.broker.DescribeCollection(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
currentLoadConfig := s.getCurrentLoadConfig(ctx, req.GetCollectionID())
|
||||
if currentLoadConfig.Collection == nil {
|
||||
// collection is not loaded, return success directly.
|
||||
return true, nil
|
||||
}
|
||||
|
||||
// remove the partitions that should be released.
|
||||
partitionIDsSet := typeutil.NewSet(currentLoadConfig.GetPartitionIDs()...)
|
||||
previousLength := len(partitionIDsSet)
|
||||
for _, partitionID := range req.PartitionIDs {
|
||||
partitionIDsSet.Remove(partitionID)
|
||||
}
|
||||
|
||||
// no partition to be released, return success directly.
|
||||
if len(partitionIDsSet) == previousLength {
|
||||
return false, job.ErrIgnoredAlterLoadConfig
|
||||
}
|
||||
|
||||
var msg message.BroadcastMutableMessage
|
||||
if len(partitionIDsSet) == 0 {
|
||||
// all partitions are released, release the collection directly.
|
||||
msg = message.NewDropLoadConfigMessageBuilderV2().
|
||||
WithHeader(&message.DropLoadConfigMessageHeader{
|
||||
DbId: coll.DbId,
|
||||
CollectionId: coll.CollectionID,
|
||||
}).
|
||||
WithBody(&message.DropLoadConfigMessageBody{}).
|
||||
WithBroadcast([]string{streaming.WAL().ControlChannel()}). // TODO: after we support query view in 3.0, we should broadcast the drop load config message to all vchannels.
|
||||
MustBuildBroadcast()
|
||||
collectionReleased = true
|
||||
} else {
|
||||
// only some partitions are released, alter the load config.
|
||||
alterLoadConfigReq := &job.AlterLoadConfigRequest{
|
||||
Meta: s.meta,
|
||||
CollectionInfo: coll,
|
||||
Current: s.getCurrentLoadConfig(ctx, req.GetCollectionID()),
|
||||
Expected: job.ExpectedLoadConfig{
|
||||
ExpectedPartitionIDs: partitionIDsSet.Collect(),
|
||||
ExpectedReplicaNumber: currentLoadConfig.GetReplicaNumber(),
|
||||
ExpectedFieldIndexID: currentLoadConfig.GetFieldIndexID(),
|
||||
ExpectedLoadFields: currentLoadConfig.GetLoadFields(),
|
||||
ExpectedPriority: commonpb.LoadPriority_HIGH,
|
||||
ExpectedUserSpecifiedReplicaMode: currentLoadConfig.GetUserSpecifiedReplicaMode(),
|
||||
},
|
||||
}
|
||||
if msg, err = job.GenerateAlterLoadConfigMessage(ctx, alterLoadConfigReq); err != nil {
|
||||
return false, err
|
||||
}
|
||||
collectionReleased = false
|
||||
}
|
||||
_, err = broadcaster.Broadcast(ctx, msg)
|
||||
return collectionReleased, err
|
||||
}
|
||||
79
internal/querycoordv2/ddl_callbacks_drop_load_info.go
Normal file
79
internal/querycoordv2/ddl_callbacks_drop_load_info.go
Normal file
@ -0,0 +1,79 @@
|
||||
// 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 querycoordv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/job"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
var errReleaseCollectionNotLoaded = errors.New("release collection not loaded")
|
||||
|
||||
// broadcastDropLoadConfigCollectionV2ForReleaseCollection broadcasts the drop load config message for release collection.
|
||||
func (s *Server) broadcastDropLoadConfigCollectionV2ForReleaseCollection(ctx context.Context, req *querypb.ReleaseCollectionRequest) error {
|
||||
broadcaster, err := s.startBroadcastWithCollectionIDLock(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer broadcaster.Close()
|
||||
|
||||
// double check if the collection is already dropped.
|
||||
coll, err := s.broker.DescribeCollection(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if !s.meta.CollectionManager.Exist(ctx, req.GetCollectionID()) {
|
||||
return errReleaseCollectionNotLoaded
|
||||
}
|
||||
msg := message.NewDropLoadConfigMessageBuilderV2().
|
||||
WithHeader(&message.DropLoadConfigMessageHeader{
|
||||
DbId: coll.GetDbId(),
|
||||
CollectionId: coll.GetCollectionID(),
|
||||
}).
|
||||
WithBody(&message.DropLoadConfigMessageBody{}).
|
||||
WithBroadcast([]string{streaming.WAL().ControlChannel()}). // TODO: after we support query view in 3.0, we should broadcast the drop load config message to all vchannels.
|
||||
MustBuildBroadcast()
|
||||
|
||||
_, err = broadcaster.Broadcast(ctx, msg)
|
||||
return err
|
||||
}
|
||||
|
||||
func (s *Server) dropLoadConfigV2AckCallback(ctx context.Context, result message.BroadcastResultDropLoadConfigMessageV2) error {
|
||||
releaseJob := job.NewReleaseCollectionJob(ctx,
|
||||
result,
|
||||
s.dist,
|
||||
s.meta,
|
||||
s.broker,
|
||||
s.targetMgr,
|
||||
s.targetObserver,
|
||||
s.checkerController,
|
||||
s.proxyClientManager,
|
||||
)
|
||||
if err := releaseJob.Execute(); err != nil {
|
||||
return err
|
||||
}
|
||||
meta.GlobalFailedLoadCache.Remove(result.Message.Header().GetCollectionId())
|
||||
return nil
|
||||
}
|
||||
917
internal/querycoordv2/ddl_callbacks_load_info_test.go
Normal file
917
internal/querycoordv2/ddl_callbacks_load_info_test.go
Normal file
@ -0,0 +1,917 @@
|
||||
// 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 querycoordv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/rgpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/job"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
)
|
||||
|
||||
func (suite *ServiceSuite) TestDDLCallbacksLoadCollectionInfo() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
// Test load collection
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
// Load with 1 replica
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
// It will be set to 1
|
||||
// ReplicaNumber: 1,
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.NoError(err)
|
||||
suite.EqualValues(1, suite.meta.GetReplicaNumber(ctx, collection))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertCollectionLoaded(collection)
|
||||
}
|
||||
|
||||
// Test load again
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
|
||||
// Test load partition while collection exists
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
// Load with 1 replica
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
|
||||
// Test load existed collection with different replica number
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
ReplicaNumber: 3,
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
|
||||
cfg := &rgpb.ResourceGroupConfig{
|
||||
Requests: &rgpb.ResourceGroupLimit{
|
||||
NodeNum: 0,
|
||||
},
|
||||
Limits: &rgpb.ResourceGroupLimit{
|
||||
NodeNum: 0,
|
||||
},
|
||||
}
|
||||
|
||||
suite.meta.ResourceManager.AddResourceGroup(ctx, "rg1", cfg)
|
||||
suite.meta.ResourceManager.AddResourceGroup(ctx, "rg2", cfg)
|
||||
suite.meta.ResourceManager.AddResourceGroup(ctx, "rg3", cfg)
|
||||
|
||||
// Load with 3 replica on 1 rg
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: 1001,
|
||||
ReplicaNumber: 3,
|
||||
ResourceGroups: []string{"rg1"},
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().ErrorIs(merr.CheckRPCCall(resp, err), merr.ErrResourceGroupNodeNotEnough)
|
||||
|
||||
// Load with 3 replica on 3 rg
|
||||
req = &querypb.LoadCollectionRequest{
|
||||
CollectionID: 1001,
|
||||
ReplicaNumber: 3,
|
||||
ResourceGroups: []string{"rg1", "rg2", "rg3"},
|
||||
}
|
||||
|
||||
resp, err = suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().ErrorIs(merr.CheckRPCCall(resp, err), merr.ErrResourceGroupNodeNotEnough)
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestDDLCallbacksLoadCollectionWithReplicas() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
// Test load collection
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
// Load with 3 replica
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
ReplicaNumber: int32(len(suite.nodes) + 1),
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().ErrorIs(merr.CheckRPCCall(resp, err), merr.ErrResourceGroupNodeNotEnough)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestDDLCallbacksLoadCollectionWithLoadFields() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
suite.Run("init_load", func() {
|
||||
// Test load collection
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
// Load with 1 replica
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
LoadFields: []int64{100, 101, 102},
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.EqualValues(1, suite.meta.GetReplicaNumber(ctx, collection))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertCollectionLoaded(collection)
|
||||
}
|
||||
})
|
||||
|
||||
suite.Run("load_again_same_fields", func() {
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
LoadFields: []int64{102, 101, 100}, // field id order shall not matter
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
})
|
||||
|
||||
suite.Run("load_again_diff_fields", func() {
|
||||
// Test load existed collection with different load fields
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
LoadFields: []int64{100, 101},
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
})
|
||||
|
||||
suite.Run("load_from_legacy_proxy", func() {
|
||||
// Test load again with legacy proxy
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{FieldID: 100},
|
||||
{FieldID: 101},
|
||||
{FieldID: 102},
|
||||
},
|
||||
},
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestDDLCallbacksLoadPartition() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
// Test load partition
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
// Load with 1 replica
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.EqualValues(1, suite.meta.GetReplicaNumber(ctx, collection))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertCollectionLoaded(collection)
|
||||
}
|
||||
|
||||
// Test load partition again
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
// Load with 1 replica
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
// ReplicaNumber: 1,
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
|
||||
// Test load partition with different replica number
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: 3,
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().ErrorIs(merr.CheckRPCCall(resp, err), merr.ErrParameterInvalid)
|
||||
}
|
||||
|
||||
// Test load partition with more partition
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: append(suite.partitions[collection], 200),
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
|
||||
// Test load collection while partitions exists
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
|
||||
cfg := &rgpb.ResourceGroupConfig{
|
||||
Requests: &rgpb.ResourceGroupLimit{
|
||||
NodeNum: 1,
|
||||
},
|
||||
Limits: &rgpb.ResourceGroupLimit{
|
||||
NodeNum: 1,
|
||||
},
|
||||
}
|
||||
suite.meta.ResourceManager.AddResourceGroup(ctx, "rg1", cfg)
|
||||
suite.meta.ResourceManager.AddResourceGroup(ctx, "rg2", cfg)
|
||||
suite.meta.ResourceManager.AddResourceGroup(ctx, "rg3", cfg)
|
||||
|
||||
// test load 3 replica in 1 rg, should pass rg check
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: 999,
|
||||
PartitionIDs: []int64{888},
|
||||
ReplicaNumber: 3,
|
||||
ResourceGroups: []string{"rg1"},
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().ErrorIs(merr.CheckRPCCall(resp, err), merr.ErrResourceGroupNodeNotEnough)
|
||||
|
||||
// test load 3 replica in 3 rg, should pass rg check
|
||||
req = &querypb.LoadPartitionsRequest{
|
||||
CollectionID: 999,
|
||||
PartitionIDs: []int64{888},
|
||||
ReplicaNumber: 3,
|
||||
ResourceGroups: []string{"rg1", "rg2", "rg3"},
|
||||
}
|
||||
resp, err = suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().ErrorIs(merr.CheckRPCCall(resp, err), merr.ErrResourceGroupNodeNotEnough)
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestLoadPartitionWithLoadFields() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
suite.Run("init_load", func() {
|
||||
// Test load partition
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
// Load with 1 replica
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: 1,
|
||||
LoadFields: []int64{100, 101, 102},
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.EqualValues(1, suite.meta.GetReplicaNumber(ctx, collection))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertCollectionLoaded(collection)
|
||||
}
|
||||
})
|
||||
|
||||
suite.Run("load_with_same_load_fields", func() {
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
// Load with 1 replica
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: 1,
|
||||
LoadFields: []int64{102, 101, 100},
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
})
|
||||
|
||||
suite.Run("load_with_diff_load_fields", func() {
|
||||
// Test load partition with different load fields
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
LoadFields: []int64{100, 101},
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
})
|
||||
|
||||
suite.Run("load_legacy_proxy", func() {
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
// Load with 1 replica
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: 1,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{FieldID: 100},
|
||||
{FieldID: 101},
|
||||
{FieldID: 102},
|
||||
},
|
||||
},
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestDynamicLoad() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
collection := suite.collections[0]
|
||||
p0, p1, p2 := suite.partitions[collection][0], suite.partitions[collection][1], suite.partitions[collection][2]
|
||||
newLoadPartJob := func(partitions ...int64) *querypb.LoadPartitionsRequest {
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: partitions,
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
return req
|
||||
}
|
||||
newLoadColJob := func() *querypb.LoadCollectionRequest {
|
||||
return &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
}
|
||||
|
||||
// loaded: none
|
||||
// action: load p0, p1, p2
|
||||
// expect: p0, p1, p2 loaded
|
||||
req := newLoadPartJob(p0, p1, p2)
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertPartitionLoaded(ctx, collection, p0, p1, p2)
|
||||
|
||||
// loaded: p0, p1, p2
|
||||
// action: load p0, p1, p2
|
||||
// expect: do nothing, p0, p1, p2 loaded
|
||||
req = newLoadPartJob(p0, p1, p2)
|
||||
resp, err = suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertPartitionLoaded(ctx, collection)
|
||||
|
||||
// loaded: p0, p1
|
||||
// action: load p2
|
||||
// expect: p0, p1, p2 loaded
|
||||
suite.releaseAll()
|
||||
req = newLoadPartJob(p0, p1)
|
||||
resp, err = suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertPartitionLoaded(ctx, collection, p0, p1)
|
||||
req = newLoadPartJob(p2)
|
||||
resp, err = suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertPartitionLoaded(ctx, collection, p2)
|
||||
|
||||
// loaded: p0, p1
|
||||
// action: load p1, p2
|
||||
// expect: p0, p1, p2 loaded
|
||||
suite.releaseAll()
|
||||
req = newLoadPartJob(p0, p1)
|
||||
resp, err = suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertPartitionLoaded(ctx, collection, p0, p1)
|
||||
req = newLoadPartJob(p1, p2)
|
||||
resp, err = suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertPartitionLoaded(ctx, collection, p2)
|
||||
|
||||
// loaded: p0, p1
|
||||
// action: load col
|
||||
// expect: col loaded
|
||||
suite.releaseAll()
|
||||
req = newLoadPartJob(p0, p1)
|
||||
resp, err = suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertPartitionLoaded(ctx, collection, p0, p1)
|
||||
colJob := newLoadColJob()
|
||||
resp, err = suite.server.LoadCollection(ctx, colJob)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertPartitionLoaded(ctx, collection, p2)
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestLoadPartitionWithReplicas() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
// Test load partitions
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
// Load with 3 replica
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: 11,
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().ErrorIs(merr.CheckRPCCall(resp, err), merr.ErrResourceGroupNodeNotEnough)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestDDLCallbacksReleaseCollection() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
suite.loadAll()
|
||||
|
||||
// Test release collection and partition
|
||||
for _, collection := range suite.collections {
|
||||
req := &querypb.ReleaseCollectionRequest{
|
||||
CollectionID: collection,
|
||||
}
|
||||
resp, err := suite.server.ReleaseCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertCollectionReleased(collection)
|
||||
}
|
||||
|
||||
// Test release again
|
||||
for _, collection := range suite.collections {
|
||||
req := &querypb.ReleaseCollectionRequest{
|
||||
CollectionID: collection,
|
||||
}
|
||||
resp, err := suite.server.ReleaseCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertCollectionReleased(collection)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestDDLCallbacksReleasePartition() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
suite.loadAll()
|
||||
|
||||
// Test release partition
|
||||
for _, collection := range suite.collections {
|
||||
req := &querypb.ReleasePartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
}
|
||||
resp, err := suite.server.ReleasePartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertPartitionReleased(collection, suite.partitions[collection]...)
|
||||
}
|
||||
|
||||
// Test release again
|
||||
for _, collection := range suite.collections {
|
||||
req := &querypb.ReleasePartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
}
|
||||
resp, err := suite.server.ReleasePartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertPartitionReleased(collection, suite.partitions[collection]...)
|
||||
}
|
||||
|
||||
// Test release partial partitions
|
||||
suite.releaseAll()
|
||||
suite.loadAll()
|
||||
for _, collectionID := range suite.collections {
|
||||
// make collection able to get into loaded state
|
||||
suite.updateChannelDist(ctx, collectionID)
|
||||
suite.updateSegmentDist(collectionID, 3000, suite.partitions[collectionID]...)
|
||||
job.WaitCurrentTargetUpdated(ctx, suite.targetObserver, collectionID)
|
||||
}
|
||||
for _, collection := range suite.collections {
|
||||
req := &querypb.ReleasePartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection][1:],
|
||||
}
|
||||
ch := make(chan struct{})
|
||||
go func() {
|
||||
defer close(ch)
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
suite.updateChannelDist(ctx, collection)
|
||||
suite.updateSegmentDist(collection, 3000, suite.partitions[collection][:1]...)
|
||||
}()
|
||||
resp, err := suite.server.ReleasePartitions(ctx, req)
|
||||
<-ch
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
|
||||
suite.True(suite.meta.Exist(ctx, collection))
|
||||
partitions := suite.meta.GetPartitionsByCollection(ctx, collection)
|
||||
suite.Len(partitions, 1)
|
||||
suite.Equal(suite.partitions[collection][0], partitions[0].GetPartitionID())
|
||||
suite.assertPartitionReleased(collection, suite.partitions[collection][1:]...)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestDynamicRelease() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
col0, col1 := suite.collections[0], suite.collections[1]
|
||||
p0, p1, p2 := suite.partitions[col0][0], suite.partitions[col0][1], suite.partitions[col0][2]
|
||||
p3, p4, p5 := suite.partitions[col1][0], suite.partitions[col1][1], suite.partitions[col1][2]
|
||||
newReleasePartJob := func(col int64, partitions ...int64) *querypb.ReleasePartitionsRequest {
|
||||
return &querypb.ReleasePartitionsRequest{
|
||||
CollectionID: col,
|
||||
PartitionIDs: partitions,
|
||||
}
|
||||
}
|
||||
newReleaseColJob := func(col int64) *querypb.ReleaseCollectionRequest {
|
||||
return &querypb.ReleaseCollectionRequest{
|
||||
CollectionID: col,
|
||||
}
|
||||
}
|
||||
|
||||
// loaded: p0, p1, p2
|
||||
// action: release p0
|
||||
// expect: p0 released, p1, p2 loaded
|
||||
suite.loadAll()
|
||||
for _, collectionID := range suite.collections {
|
||||
// make collection able to get into loaded state
|
||||
suite.updateChannelDist(ctx, collectionID)
|
||||
suite.updateSegmentDist(collectionID, 3000, suite.partitions[collectionID]...)
|
||||
job.WaitCurrentTargetUpdated(ctx, suite.targetObserver, collectionID)
|
||||
}
|
||||
|
||||
req := newReleasePartJob(col0, p0)
|
||||
// update segments
|
||||
ch := make(chan struct{})
|
||||
go func() {
|
||||
defer close(ch)
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
suite.updateSegmentDist(col0, 3000, p1, p2)
|
||||
suite.updateChannelDist(ctx, col0)
|
||||
}()
|
||||
resp, err := suite.server.ReleasePartitions(ctx, req)
|
||||
<-ch
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertPartitionReleased(col0, p0)
|
||||
suite.assertPartitionLoaded(ctx, col0, p1, p2)
|
||||
|
||||
// loaded: p1, p2
|
||||
// action: release p0, p1
|
||||
// expect: p1 released, p2 loaded
|
||||
req = newReleasePartJob(col0, p0, p1)
|
||||
ch = make(chan struct{})
|
||||
go func() {
|
||||
defer close(ch)
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
suite.updateSegmentDist(col0, 3000, p2)
|
||||
suite.updateChannelDist(ctx, col0)
|
||||
}()
|
||||
resp, err = suite.server.ReleasePartitions(ctx, req)
|
||||
<-ch
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertPartitionReleased(col0, p0, p1)
|
||||
suite.assertPartitionLoaded(ctx, col0, p2)
|
||||
|
||||
// loaded: p2
|
||||
// action: release p2
|
||||
// expect: loadType=col: col loaded, p2 released, full collection should be released.
|
||||
req = newReleasePartJob(col0, p2)
|
||||
ch = make(chan struct{})
|
||||
go func() {
|
||||
defer close(ch)
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
suite.releaseSegmentDist(3000)
|
||||
suite.releaseAllChannelDist()
|
||||
}()
|
||||
resp, err = suite.server.ReleasePartitions(ctx, req)
|
||||
<-ch
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertPartitionReleased(col0, p0, p1, p2)
|
||||
suite.False(suite.meta.Exist(ctx, col0))
|
||||
|
||||
// loaded: p0, p1, p2
|
||||
// action: release col
|
||||
// expect: col released
|
||||
suite.releaseAll()
|
||||
suite.loadAll()
|
||||
|
||||
req2 := newReleaseColJob(col0)
|
||||
resp, err = suite.server.ReleaseCollection(ctx, req2)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertCollectionReleased(col0)
|
||||
suite.assertPartitionReleased(col0, p0, p1, p2)
|
||||
|
||||
// loaded: p3, p4, p5
|
||||
// action: release p3, p4, p5
|
||||
// expect: loadType=partition: col released
|
||||
suite.releaseAll()
|
||||
suite.loadAll()
|
||||
|
||||
req = newReleasePartJob(col1, p3, p4, p5)
|
||||
resp, err = suite.server.ReleasePartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertCollectionReleased(col1)
|
||||
suite.assertPartitionReleased(col1, p3, p4, p5)
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) releaseAll() {
|
||||
ctx := context.Background()
|
||||
for _, collection := range suite.collections {
|
||||
resp, err := suite.server.ReleaseCollection(ctx, &querypb.ReleaseCollectionRequest{
|
||||
CollectionID: collection,
|
||||
})
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.assertCollectionReleased(collection)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) assertCollectionReleased(collection int64) {
|
||||
ctx := context.Background()
|
||||
suite.False(suite.meta.Exist(ctx, collection))
|
||||
suite.Equal(0, len(suite.meta.ReplicaManager.GetByCollection(ctx, collection)))
|
||||
for _, channel := range suite.channels[collection] {
|
||||
suite.Nil(suite.targetMgr.GetDmChannel(ctx, collection, channel, meta.CurrentTarget))
|
||||
}
|
||||
for _, partitions := range suite.segments[collection] {
|
||||
for _, segment := range partitions {
|
||||
suite.Nil(suite.targetMgr.GetSealedSegment(ctx, collection, segment, meta.CurrentTarget))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) assertPartitionReleased(collection int64, partitionIDs ...int64) {
|
||||
ctx := context.Background()
|
||||
for _, partition := range partitionIDs {
|
||||
suite.Nil(suite.meta.GetPartition(ctx, partition))
|
||||
segments := suite.segments[collection][partition]
|
||||
for _, segment := range segments {
|
||||
suite.Nil(suite.targetMgr.GetSealedSegment(ctx, collection, segment, meta.CurrentTarget))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestDDLCallbacksLoadCollectionWithUserSpecifiedReplicaMode() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
// Test load collection with userSpecifiedReplicaMode = true
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
|
||||
continue
|
||||
}
|
||||
|
||||
req := &querypb.LoadCollectionRequest{
|
||||
CollectionID: collection,
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
|
||||
// Verify UserSpecifiedReplicaMode is set correctly
|
||||
loadedCollection := suite.meta.GetCollection(ctx, collection)
|
||||
suite.NotNil(loadedCollection)
|
||||
suite.True(loadedCollection.GetUserSpecifiedReplicaMode())
|
||||
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertCollectionLoaded(collection)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestLoadPartitionWithUserSpecifiedReplicaMode() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
// Test load partition with userSpecifiedReplicaMode = true
|
||||
for _, collection := range suite.collections {
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
continue
|
||||
}
|
||||
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
|
||||
// Verify UserSpecifiedReplicaMode is set correctly
|
||||
loadedCollection := suite.meta.GetCollection(ctx, collection)
|
||||
suite.NotNil(loadedCollection)
|
||||
suite.True(loadedCollection.GetUserSpecifiedReplicaMode())
|
||||
|
||||
suite.targetMgr.UpdateCollectionCurrentTarget(ctx, collection)
|
||||
suite.assertCollectionLoaded(collection)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestLoadPartitionUpdateUserSpecifiedReplicaMode() {
|
||||
ctx := context.Background()
|
||||
suite.expectGetRecoverInfoForAllCollections()
|
||||
|
||||
// First load partition with userSpecifiedReplicaMode = false
|
||||
collection := suite.collections[1] // Use partition load type collection
|
||||
if suite.loadTypes[collection] != querypb.LoadType_LoadPartition {
|
||||
return
|
||||
}
|
||||
|
||||
req := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection][:1], // Load first partition
|
||||
}
|
||||
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
|
||||
// Verify UserSpecifiedReplicaMode is false
|
||||
loadedCollection := suite.meta.GetCollection(ctx, collection)
|
||||
suite.NotNil(loadedCollection)
|
||||
suite.False(loadedCollection.GetUserSpecifiedReplicaMode())
|
||||
|
||||
// Load another partition with userSpecifiedReplicaMode = true
|
||||
req2 := &querypb.LoadPartitionsRequest{
|
||||
CollectionID: collection,
|
||||
PartitionIDs: suite.partitions[collection][1:2], // Load second partition
|
||||
ReplicaNumber: 1,
|
||||
}
|
||||
resp, err = suite.server.LoadPartitions(ctx, req2)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
|
||||
// Verify UserSpecifiedReplicaMode is updated to true
|
||||
updatedCollection := suite.meta.GetCollection(ctx, collection)
|
||||
suite.NotNil(updatedCollection)
|
||||
suite.True(updatedCollection.GetUserSpecifiedReplicaMode())
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) TestSyncNewCreatedPartition() {
|
||||
newPartition := int64(999)
|
||||
ctx := context.Background()
|
||||
|
||||
// test sync new created partition
|
||||
suite.loadAll()
|
||||
collectionID := suite.collections[0]
|
||||
// make collection able to get into loaded state
|
||||
suite.updateChannelDist(ctx, collectionID)
|
||||
suite.updateSegmentDist(collectionID, 3000, suite.partitions[collectionID]...)
|
||||
|
||||
req := &querypb.SyncNewCreatedPartitionRequest{
|
||||
CollectionID: collectionID,
|
||||
PartitionID: newPartition,
|
||||
}
|
||||
syncJob := job.NewSyncNewCreatedPartitionJob(
|
||||
ctx,
|
||||
req,
|
||||
suite.meta,
|
||||
suite.broker,
|
||||
suite.targetObserver,
|
||||
suite.targetMgr,
|
||||
)
|
||||
suite.jobScheduler.Add(syncJob)
|
||||
err := syncJob.Wait()
|
||||
suite.NoError(err)
|
||||
partition := suite.meta.CollectionManager.GetPartition(ctx, newPartition)
|
||||
suite.NotNil(partition)
|
||||
suite.Equal(querypb.LoadStatus_Loaded, partition.GetStatus())
|
||||
|
||||
// test collection not loaded
|
||||
req = &querypb.SyncNewCreatedPartitionRequest{
|
||||
CollectionID: int64(888),
|
||||
PartitionID: newPartition,
|
||||
}
|
||||
syncJob = job.NewSyncNewCreatedPartitionJob(
|
||||
ctx,
|
||||
req,
|
||||
suite.meta,
|
||||
suite.broker,
|
||||
suite.targetObserver,
|
||||
suite.targetMgr,
|
||||
)
|
||||
suite.jobScheduler.Add(syncJob)
|
||||
err = syncJob.Wait()
|
||||
suite.NoError(err)
|
||||
|
||||
// test collection loaded, but its loadType is loadPartition
|
||||
req = &querypb.SyncNewCreatedPartitionRequest{
|
||||
CollectionID: suite.collections[1],
|
||||
PartitionID: newPartition,
|
||||
}
|
||||
syncJob = job.NewSyncNewCreatedPartitionJob(
|
||||
ctx,
|
||||
req,
|
||||
suite.meta,
|
||||
suite.broker,
|
||||
suite.targetObserver,
|
||||
suite.targetMgr,
|
||||
)
|
||||
suite.jobScheduler.Add(syncJob)
|
||||
err = syncJob.Wait()
|
||||
suite.NoError(err)
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) assertCollectionLoaded(collection int64) {
|
||||
ctx := context.Background()
|
||||
suite.True(suite.meta.Exist(ctx, collection))
|
||||
suite.NotEqual(0, len(suite.meta.ReplicaManager.GetByCollection(ctx, collection)))
|
||||
for _, channel := range suite.channels[collection] {
|
||||
suite.NotNil(suite.targetMgr.GetDmChannel(ctx, collection, channel, meta.CurrentTarget))
|
||||
}
|
||||
for _, segments := range suite.segments[collection] {
|
||||
for _, segment := range segments {
|
||||
suite.NotNil(suite.targetMgr.GetSealedSegment(ctx, collection, segment, meta.CurrentTarget))
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -27,7 +27,6 @@ import (
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/observers"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/session"
|
||||
@ -36,15 +35,15 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type LoadCollectionJob struct {
|
||||
*BaseJob
|
||||
req *querypb.LoadCollectionRequest
|
||||
undo *UndoList
|
||||
|
||||
result message.BroadcastResultAlterLoadConfigMessageV2
|
||||
undo *UndoList
|
||||
dist *meta.DistributionManager
|
||||
meta *meta.Meta
|
||||
broker meta.Broker
|
||||
@ -52,13 +51,11 @@ type LoadCollectionJob struct {
|
||||
targetObserver *observers.TargetObserver
|
||||
collectionObserver *observers.CollectionObserver
|
||||
nodeMgr *session.NodeManager
|
||||
collInfo *milvuspb.DescribeCollectionResponse
|
||||
userSpecifiedReplicaMode bool
|
||||
}
|
||||
|
||||
func NewLoadCollectionJob(
|
||||
ctx context.Context,
|
||||
req *querypb.LoadCollectionRequest,
|
||||
result message.BroadcastResultAlterLoadConfigMessageV2,
|
||||
dist *meta.DistributionManager,
|
||||
meta *meta.Meta,
|
||||
broker meta.Broker,
|
||||
@ -66,11 +63,10 @@ func NewLoadCollectionJob(
|
||||
targetObserver *observers.TargetObserver,
|
||||
collectionObserver *observers.CollectionObserver,
|
||||
nodeMgr *session.NodeManager,
|
||||
userSpecifiedReplicaMode bool,
|
||||
) *LoadCollectionJob {
|
||||
return &LoadCollectionJob{
|
||||
BaseJob: NewBaseJob(ctx, req.Base.GetMsgID(), req.GetCollectionID()),
|
||||
req: req,
|
||||
BaseJob: NewBaseJob(ctx, 0, result.Message.Header().GetCollectionId()),
|
||||
result: result,
|
||||
undo: NewUndoList(ctx, meta, targetMgr, targetObserver),
|
||||
dist: dist,
|
||||
meta: meta,
|
||||
@ -79,117 +75,48 @@ func NewLoadCollectionJob(
|
||||
targetObserver: targetObserver,
|
||||
collectionObserver: collectionObserver,
|
||||
nodeMgr: nodeMgr,
|
||||
userSpecifiedReplicaMode: userSpecifiedReplicaMode,
|
||||
}
|
||||
}
|
||||
|
||||
func (job *LoadCollectionJob) PreExecute() error {
|
||||
req := job.req
|
||||
log := log.Ctx(job.ctx).With(zap.Int64("collectionID", req.GetCollectionID()))
|
||||
|
||||
if req.GetReplicaNumber() <= 0 {
|
||||
log.Info("request doesn't indicate the number of replicas, set it to 1",
|
||||
zap.Int32("replicaNumber", req.GetReplicaNumber()))
|
||||
req.ReplicaNumber = 1
|
||||
}
|
||||
|
||||
if len(req.GetResourceGroups()) == 0 {
|
||||
req.ResourceGroups = []string{meta.DefaultResourceGroupName}
|
||||
}
|
||||
|
||||
var err error
|
||||
job.collInfo, err = job.broker.DescribeCollection(job.ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("failed to describe collection from RootCoord", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
collection := job.meta.GetCollection(job.ctx, req.GetCollectionID())
|
||||
if collection == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if collection.GetReplicaNumber() != req.GetReplicaNumber() {
|
||||
msg := fmt.Sprintf("collection with different replica number %d existed, release this collection first before changing its replica number",
|
||||
job.meta.GetReplicaNumber(job.ctx, req.GetCollectionID()),
|
||||
)
|
||||
log.Warn(msg)
|
||||
return merr.WrapErrParameterInvalid(collection.GetReplicaNumber(), req.GetReplicaNumber(), "can't change the replica number for loaded collection")
|
||||
}
|
||||
|
||||
collectionUsedRG := job.meta.ReplicaManager.GetResourceGroupByCollection(job.ctx, collection.GetCollectionID()).Collect()
|
||||
left, right := lo.Difference(collectionUsedRG, req.GetResourceGroups())
|
||||
if len(left) > 0 || len(right) > 0 {
|
||||
msg := fmt.Sprintf("collection with different resource groups %v existed, release this collection first before changing its resource groups",
|
||||
collectionUsedRG)
|
||||
log.Warn(msg)
|
||||
return merr.WrapErrParameterInvalid(collectionUsedRG, req.GetResourceGroups(), "can't change the resource groups for loaded partitions")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (job *LoadCollectionJob) Execute() error {
|
||||
req := job.req
|
||||
log := log.Ctx(job.ctx).With(zap.Int64("collectionID", req.GetCollectionID()))
|
||||
meta.GlobalFailedLoadCache.Remove(req.GetCollectionID())
|
||||
req := job.result.Message.Header()
|
||||
vchannels := job.result.GetVChannelsWithoutControlChannel()
|
||||
|
||||
// 1. Fetch target partitions
|
||||
partitionIDs, err := job.broker.GetPartitions(job.ctx, req.GetCollectionID())
|
||||
log := log.Ctx(job.ctx).With(zap.Int64("collectionID", req.GetCollectionId()))
|
||||
meta.GlobalFailedLoadCache.Remove(req.GetCollectionId())
|
||||
|
||||
// 1. create replica if not exist
|
||||
if _, err := utils.SpawnReplicasWithReplicaConfig(job.ctx, job.meta, meta.SpawnWithReplicaConfigParams{
|
||||
CollectionID: req.GetCollectionId(),
|
||||
Channels: vchannels,
|
||||
Configs: req.GetReplicas(),
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
collInfo, err := job.broker.DescribeCollection(job.ctx, req.GetCollectionId())
|
||||
if err != nil {
|
||||
msg := "failed to get partitions from RootCoord"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
loadedPartitionIDs := lo.Map(job.meta.CollectionManager.GetPartitionsByCollection(job.ctx, req.GetCollectionID()),
|
||||
func(partition *meta.Partition, _ int) int64 {
|
||||
return partition.GetPartitionID()
|
||||
})
|
||||
lackPartitionIDs := lo.FilterMap(partitionIDs, func(partID int64, _ int) (int64, bool) {
|
||||
return partID, !lo.Contains(loadedPartitionIDs, partID)
|
||||
})
|
||||
if len(lackPartitionIDs) == 0 {
|
||||
return nil
|
||||
}
|
||||
job.undo.CollectionID = req.GetCollectionID()
|
||||
job.undo.LackPartitions = lackPartitionIDs
|
||||
log.Info("find partitions to load", zap.Int64s("partitions", lackPartitionIDs))
|
||||
|
||||
colExisted := job.meta.CollectionManager.Exist(job.ctx, req.GetCollectionID())
|
||||
if !colExisted {
|
||||
// Clear stale replicas, https://github.com/milvus-io/milvus/issues/20444
|
||||
err = job.meta.ReplicaManager.RemoveCollection(job.ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
msg := "failed to clear stale replicas"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// 2. create replica if not exist
|
||||
replicas := job.meta.ReplicaManager.GetByCollection(job.ctx, req.GetCollectionID())
|
||||
if len(replicas) == 0 {
|
||||
// API of LoadCollection is wired, we should use map[resourceGroupNames]replicaNumber as input, to keep consistency with `TransferReplica` API.
|
||||
// Then we can implement dynamic replica changed in different resource group independently.
|
||||
_, err = utils.SpawnReplicasWithRG(job.ctx, job.meta, req.GetCollectionID(), req.GetResourceGroups(),
|
||||
req.GetReplicaNumber(), job.collInfo.GetVirtualChannelNames(), req.GetPriority())
|
||||
if err != nil {
|
||||
msg := "failed to spawn replica for collection"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
// 2. put load info meta
|
||||
fieldIndexIDs := make(map[int64]int64, len(req.GetLoadFields()))
|
||||
fieldIDs := make([]int64, len(req.GetLoadFields()))
|
||||
for _, loadField := range req.GetLoadFields() {
|
||||
if loadField.GetIndexId() != 0 {
|
||||
fieldIndexIDs[loadField.GetFieldId()] = loadField.GetIndexId()
|
||||
}
|
||||
job.undo.IsReplicaCreated = true
|
||||
fieldIDs = append(fieldIDs, loadField.GetFieldId())
|
||||
}
|
||||
|
||||
// 4. put collection/partitions meta
|
||||
partitions := lo.Map(lackPartitionIDs, func(partID int64, _ int) *meta.Partition {
|
||||
replicaNumber := int32(len(req.GetReplicas()))
|
||||
partitions := lo.Map(req.GetPartitionIds(), func(partID int64, _ int) *meta.Partition {
|
||||
return &meta.Partition{
|
||||
PartitionLoadInfo: &querypb.PartitionLoadInfo{
|
||||
CollectionID: req.GetCollectionID(),
|
||||
CollectionID: req.GetCollectionId(),
|
||||
PartitionID: partID,
|
||||
ReplicaNumber: req.GetReplicaNumber(),
|
||||
ReplicaNumber: replicaNumber,
|
||||
Status: querypb.LoadStatus_Loading,
|
||||
FieldIndexID: req.GetFieldIndexID(),
|
||||
FieldIndexID: fieldIndexIDs,
|
||||
},
|
||||
CreatedAt: time.Now(),
|
||||
}
|
||||
@ -198,22 +125,35 @@ func (job *LoadCollectionJob) Execute() error {
|
||||
ctx, sp := otel.Tracer(typeutil.QueryCoordRole).Start(job.ctx, "LoadCollection", trace.WithNewRoot())
|
||||
collection := &meta.Collection{
|
||||
CollectionLoadInfo: &querypb.CollectionLoadInfo{
|
||||
CollectionID: req.GetCollectionID(),
|
||||
ReplicaNumber: req.GetReplicaNumber(),
|
||||
CollectionID: req.GetCollectionId(),
|
||||
ReplicaNumber: replicaNumber,
|
||||
Status: querypb.LoadStatus_Loading,
|
||||
FieldIndexID: req.GetFieldIndexID(),
|
||||
FieldIndexID: fieldIndexIDs,
|
||||
LoadType: querypb.LoadType_LoadCollection,
|
||||
LoadFields: req.GetLoadFields(),
|
||||
DbID: job.collInfo.GetDbId(),
|
||||
UserSpecifiedReplicaMode: job.userSpecifiedReplicaMode,
|
||||
LoadFields: fieldIDs,
|
||||
DbID: req.GetDbId(),
|
||||
UserSpecifiedReplicaMode: req.GetUserSpecifiedReplicaMode(),
|
||||
},
|
||||
CreatedAt: time.Now(),
|
||||
LoadSpan: sp,
|
||||
Schema: job.collInfo.GetSchema(),
|
||||
Schema: collInfo.GetSchema(),
|
||||
}
|
||||
job.undo.IsNewCollection = true
|
||||
err = job.meta.CollectionManager.PutCollection(job.ctx, collection, partitions...)
|
||||
if err != nil {
|
||||
incomingPartitions := typeutil.NewSet(req.GetPartitionIds()...)
|
||||
currentPartitions := job.meta.CollectionManager.GetPartitionsByCollection(job.ctx, req.GetCollectionId())
|
||||
toReleasePartitions := make([]int64, 0)
|
||||
for _, partition := range currentPartitions {
|
||||
if !incomingPartitions.Contain(partition.GetPartitionID()) {
|
||||
toReleasePartitions = append(toReleasePartitions, partition.GetPartitionID())
|
||||
}
|
||||
}
|
||||
if len(toReleasePartitions) > 0 {
|
||||
job.targetObserver.ReleasePartition(req.GetCollectionId(), toReleasePartitions...)
|
||||
if err := job.meta.CollectionManager.RemovePartition(job.ctx, req.GetCollectionId(), toReleasePartitions...); err != nil {
|
||||
return errors.Wrap(err, "failed to remove partitions")
|
||||
}
|
||||
}
|
||||
|
||||
if err = job.meta.CollectionManager.PutCollection(job.ctx, collection, partitions...); err != nil {
|
||||
msg := "failed to store collection and partitions"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
@ -222,233 +162,11 @@ func (job *LoadCollectionJob) Execute() error {
|
||||
metrics.QueryCoordNumPartitions.WithLabelValues().Add(float64(len(partitions)))
|
||||
|
||||
// 5. update next target, no need to rollback if pull target failed, target observer will pull target in periodically
|
||||
_, err = job.targetObserver.UpdateNextTarget(req.GetCollectionID())
|
||||
if err != nil {
|
||||
msg := "failed to update next target"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
}
|
||||
job.undo.IsTargetUpdated = true
|
||||
|
||||
// 6. register load task into collection observer
|
||||
job.collectionObserver.LoadCollection(ctx, req.GetCollectionID())
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (job *LoadCollectionJob) PostExecute() {
|
||||
if job.Error() != nil {
|
||||
job.undo.RollBack()
|
||||
}
|
||||
}
|
||||
|
||||
type LoadPartitionJob struct {
|
||||
*BaseJob
|
||||
req *querypb.LoadPartitionsRequest
|
||||
undo *UndoList
|
||||
|
||||
dist *meta.DistributionManager
|
||||
meta *meta.Meta
|
||||
broker meta.Broker
|
||||
targetMgr meta.TargetManagerInterface
|
||||
targetObserver *observers.TargetObserver
|
||||
collectionObserver *observers.CollectionObserver
|
||||
nodeMgr *session.NodeManager
|
||||
collInfo *milvuspb.DescribeCollectionResponse
|
||||
userSpecifiedReplicaMode bool
|
||||
}
|
||||
|
||||
func NewLoadPartitionJob(
|
||||
ctx context.Context,
|
||||
req *querypb.LoadPartitionsRequest,
|
||||
dist *meta.DistributionManager,
|
||||
meta *meta.Meta,
|
||||
broker meta.Broker,
|
||||
targetMgr meta.TargetManagerInterface,
|
||||
targetObserver *observers.TargetObserver,
|
||||
collectionObserver *observers.CollectionObserver,
|
||||
nodeMgr *session.NodeManager,
|
||||
userSpecifiedReplicaMode bool,
|
||||
) *LoadPartitionJob {
|
||||
return &LoadPartitionJob{
|
||||
BaseJob: NewBaseJob(ctx, req.Base.GetMsgID(), req.GetCollectionID()),
|
||||
req: req,
|
||||
undo: NewUndoList(ctx, meta, targetMgr, targetObserver),
|
||||
dist: dist,
|
||||
meta: meta,
|
||||
broker: broker,
|
||||
targetMgr: targetMgr,
|
||||
targetObserver: targetObserver,
|
||||
collectionObserver: collectionObserver,
|
||||
nodeMgr: nodeMgr,
|
||||
userSpecifiedReplicaMode: userSpecifiedReplicaMode,
|
||||
}
|
||||
}
|
||||
|
||||
func (job *LoadPartitionJob) PreExecute() error {
|
||||
req := job.req
|
||||
log := log.Ctx(job.ctx).With(zap.Int64("collectionID", req.GetCollectionID()))
|
||||
|
||||
if req.GetReplicaNumber() <= 0 {
|
||||
log.Info("request doesn't indicate the number of replicas, set it to 1",
|
||||
zap.Int32("replicaNumber", req.GetReplicaNumber()))
|
||||
req.ReplicaNumber = 1
|
||||
}
|
||||
|
||||
if len(req.GetResourceGroups()) == 0 {
|
||||
req.ResourceGroups = []string{meta.DefaultResourceGroupName}
|
||||
}
|
||||
|
||||
var err error
|
||||
job.collInfo, err = job.broker.DescribeCollection(job.ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("failed to describe collection from RootCoord", zap.Error(err))
|
||||
if _, err = job.targetObserver.UpdateNextTarget(req.GetCollectionId()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
collection := job.meta.GetCollection(job.ctx, req.GetCollectionID())
|
||||
if collection == nil {
|
||||
// 6. register load task into collection observer
|
||||
job.collectionObserver.LoadPartitions(ctx, req.GetCollectionId(), incomingPartitions.Collect())
|
||||
return nil
|
||||
}
|
||||
|
||||
if collection.GetReplicaNumber() != req.GetReplicaNumber() {
|
||||
msg := "collection with different replica number existed, release this collection first before changing its replica number"
|
||||
log.Warn(msg)
|
||||
return merr.WrapErrParameterInvalid(collection.GetReplicaNumber(), req.GetReplicaNumber(), "can't change the replica number for loaded partitions")
|
||||
}
|
||||
|
||||
collectionUsedRG := job.meta.ReplicaManager.GetResourceGroupByCollection(job.ctx, collection.GetCollectionID()).Collect()
|
||||
left, right := lo.Difference(collectionUsedRG, req.GetResourceGroups())
|
||||
if len(left) > 0 || len(right) > 0 {
|
||||
msg := fmt.Sprintf("collection with different resource groups %v existed, release this collection first before changing its resource groups",
|
||||
collectionUsedRG)
|
||||
log.Warn(msg)
|
||||
return merr.WrapErrParameterInvalid(collectionUsedRG, req.GetResourceGroups(), "can't change the resource groups for loaded partitions")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (job *LoadPartitionJob) Execute() error {
|
||||
req := job.req
|
||||
log := log.Ctx(job.ctx).With(
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Int64s("partitionIDs", req.GetPartitionIDs()),
|
||||
)
|
||||
meta.GlobalFailedLoadCache.Remove(req.GetCollectionID())
|
||||
|
||||
// 1. Fetch target partitions
|
||||
loadedPartitionIDs := lo.Map(job.meta.CollectionManager.GetPartitionsByCollection(job.ctx, req.GetCollectionID()),
|
||||
func(partition *meta.Partition, _ int) int64 {
|
||||
return partition.GetPartitionID()
|
||||
})
|
||||
lackPartitionIDs := lo.FilterMap(req.GetPartitionIDs(), func(partID int64, _ int) (int64, bool) {
|
||||
return partID, !lo.Contains(loadedPartitionIDs, partID)
|
||||
})
|
||||
if len(lackPartitionIDs) == 0 {
|
||||
return nil
|
||||
}
|
||||
job.undo.CollectionID = req.GetCollectionID()
|
||||
job.undo.LackPartitions = lackPartitionIDs
|
||||
log.Info("find partitions to load", zap.Int64s("partitions", lackPartitionIDs))
|
||||
|
||||
var err error
|
||||
if !job.meta.CollectionManager.Exist(job.ctx, req.GetCollectionID()) {
|
||||
// Clear stale replicas, https://github.com/milvus-io/milvus/issues/20444
|
||||
err = job.meta.ReplicaManager.RemoveCollection(job.ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
msg := "failed to clear stale replicas"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
}
|
||||
|
||||
// 2. create replica if not exist
|
||||
replicas := job.meta.ReplicaManager.GetByCollection(context.TODO(), req.GetCollectionID())
|
||||
if len(replicas) == 0 {
|
||||
_, err = utils.SpawnReplicasWithRG(job.ctx, job.meta, req.GetCollectionID(), req.GetResourceGroups(), req.GetReplicaNumber(),
|
||||
job.collInfo.GetVirtualChannelNames(), req.GetPriority())
|
||||
if err != nil {
|
||||
msg := "failed to spawn replica for collection"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
job.undo.IsReplicaCreated = true
|
||||
}
|
||||
|
||||
// 4. put collection/partitions meta
|
||||
partitions := lo.Map(lackPartitionIDs, func(partID int64, _ int) *meta.Partition {
|
||||
return &meta.Partition{
|
||||
PartitionLoadInfo: &querypb.PartitionLoadInfo{
|
||||
CollectionID: req.GetCollectionID(),
|
||||
PartitionID: partID,
|
||||
ReplicaNumber: req.GetReplicaNumber(),
|
||||
Status: querypb.LoadStatus_Loading,
|
||||
FieldIndexID: req.GetFieldIndexID(),
|
||||
},
|
||||
CreatedAt: time.Now(),
|
||||
}
|
||||
})
|
||||
ctx, sp := otel.Tracer(typeutil.QueryCoordRole).Start(job.ctx, "LoadPartition", trace.WithNewRoot())
|
||||
if !job.meta.CollectionManager.Exist(job.ctx, req.GetCollectionID()) {
|
||||
job.undo.IsNewCollection = true
|
||||
|
||||
collection := &meta.Collection{
|
||||
CollectionLoadInfo: &querypb.CollectionLoadInfo{
|
||||
CollectionID: req.GetCollectionID(),
|
||||
ReplicaNumber: req.GetReplicaNumber(),
|
||||
Status: querypb.LoadStatus_Loading,
|
||||
FieldIndexID: req.GetFieldIndexID(),
|
||||
LoadType: querypb.LoadType_LoadPartition,
|
||||
LoadFields: req.GetLoadFields(),
|
||||
DbID: job.collInfo.GetDbId(),
|
||||
UserSpecifiedReplicaMode: job.userSpecifiedReplicaMode,
|
||||
},
|
||||
CreatedAt: time.Now(),
|
||||
LoadSpan: sp,
|
||||
Schema: job.collInfo.GetSchema(),
|
||||
}
|
||||
err = job.meta.CollectionManager.PutCollection(job.ctx, collection, partitions...)
|
||||
if err != nil {
|
||||
msg := "failed to store collection and partitions"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
} else { // collection exists, put partitions only
|
||||
coll := job.meta.GetCollection(job.ctx, req.GetCollectionID())
|
||||
if job.userSpecifiedReplicaMode && !coll.CollectionLoadInfo.UserSpecifiedReplicaMode {
|
||||
coll.CollectionLoadInfo.UserSpecifiedReplicaMode = job.userSpecifiedReplicaMode
|
||||
err = job.meta.CollectionManager.PutCollection(job.ctx, coll)
|
||||
if err != nil {
|
||||
msg := "failed to store collection"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
}
|
||||
|
||||
err = job.meta.CollectionManager.PutPartition(job.ctx, partitions...)
|
||||
if err != nil {
|
||||
msg := "failed to store partitions"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
}
|
||||
metrics.QueryCoordNumPartitions.WithLabelValues().Add(float64(len(partitions)))
|
||||
|
||||
// 5. update next target, no need to rollback if pull target failed, target observer will pull target in periodically
|
||||
_, err = job.targetObserver.UpdateNextTarget(req.GetCollectionID())
|
||||
if err != nil {
|
||||
msg := "failed to update next target"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
}
|
||||
job.undo.IsTargetUpdated = true
|
||||
|
||||
job.collectionObserver.LoadPartitions(ctx, req.GetCollectionID(), lackPartitionIDs)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (job *LoadPartitionJob) PostExecute() {
|
||||
if job.Error() != nil {
|
||||
job.undo.RollBack()
|
||||
}
|
||||
}
|
||||
|
||||
@ -20,7 +20,6 @@ import (
|
||||
"context"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
@ -30,14 +29,13 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/session"
|
||||
"github.com/milvus-io/milvus/internal/util/proxyutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/proxypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
type ReleaseCollectionJob struct {
|
||||
*BaseJob
|
||||
req *querypb.ReleaseCollectionRequest
|
||||
result message.BroadcastResultDropLoadConfigMessageV2
|
||||
dist *meta.DistributionManager
|
||||
meta *meta.Meta
|
||||
broker meta.Broker
|
||||
@ -49,7 +47,7 @@ type ReleaseCollectionJob struct {
|
||||
}
|
||||
|
||||
func NewReleaseCollectionJob(ctx context.Context,
|
||||
req *querypb.ReleaseCollectionRequest,
|
||||
result message.BroadcastResultDropLoadConfigMessageV2,
|
||||
dist *meta.DistributionManager,
|
||||
meta *meta.Meta,
|
||||
broker meta.Broker,
|
||||
@ -59,8 +57,8 @@ func NewReleaseCollectionJob(ctx context.Context,
|
||||
proxyManager proxyutil.ProxyClientManagerInterface,
|
||||
) *ReleaseCollectionJob {
|
||||
return &ReleaseCollectionJob{
|
||||
BaseJob: NewBaseJob(ctx, req.Base.GetMsgID(), req.GetCollectionID()),
|
||||
req: req,
|
||||
BaseJob: NewBaseJob(ctx, 0, result.Message.Header().GetCollectionId()),
|
||||
result: result,
|
||||
dist: dist,
|
||||
meta: meta,
|
||||
broker: broker,
|
||||
@ -72,146 +70,40 @@ func NewReleaseCollectionJob(ctx context.Context,
|
||||
}
|
||||
|
||||
func (job *ReleaseCollectionJob) Execute() error {
|
||||
req := job.req
|
||||
log := log.Ctx(job.ctx).With(zap.Int64("collectionID", req.GetCollectionID()))
|
||||
collectionID := job.result.Message.Header().GetCollectionId()
|
||||
log := log.Ctx(job.ctx).With(zap.Int64("collectionID", collectionID))
|
||||
|
||||
if !job.meta.CollectionManager.Exist(job.ctx, req.GetCollectionID()) {
|
||||
if !job.meta.CollectionManager.Exist(job.ctx, collectionID) {
|
||||
log.Info("release collection end, the collection has not been loaded into QueryNode")
|
||||
return nil
|
||||
}
|
||||
|
||||
err := job.meta.CollectionManager.RemoveCollection(job.ctx, req.GetCollectionID())
|
||||
err := job.meta.CollectionManager.RemoveCollection(job.ctx, collectionID)
|
||||
if err != nil {
|
||||
msg := "failed to remove collection"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
|
||||
err = job.meta.ReplicaManager.RemoveCollection(job.ctx, req.GetCollectionID())
|
||||
err = job.meta.ReplicaManager.RemoveCollection(job.ctx, collectionID)
|
||||
if err != nil {
|
||||
msg := "failed to remove replicas"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
}
|
||||
|
||||
job.targetObserver.ReleaseCollection(req.GetCollectionID())
|
||||
job.targetObserver.ReleaseCollection(collectionID)
|
||||
|
||||
// try best discard cache
|
||||
// shall not affect releasing if failed
|
||||
job.proxyManager.InvalidateCollectionMetaCache(job.ctx,
|
||||
&proxypb.InvalidateCollMetaCacheRequest{
|
||||
CollectionID: req.GetCollectionID(),
|
||||
CollectionID: collectionID,
|
||||
},
|
||||
proxyutil.SetMsgType(commonpb.MsgType_ReleaseCollection))
|
||||
|
||||
// try best clean shard leader cache
|
||||
job.proxyManager.InvalidateShardLeaderCache(job.ctx, &proxypb.InvalidateShardLeaderCacheRequest{
|
||||
CollectionIDs: []int64{req.GetCollectionID()},
|
||||
CollectionIDs: []int64{collectionID},
|
||||
})
|
||||
|
||||
waitCollectionReleased(job.dist, job.checkerController, req.GetCollectionID())
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.TotalLabel).Inc()
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return nil
|
||||
}
|
||||
|
||||
type ReleasePartitionJob struct {
|
||||
*BaseJob
|
||||
releasePartitionsOnly bool
|
||||
|
||||
req *querypb.ReleasePartitionsRequest
|
||||
dist *meta.DistributionManager
|
||||
meta *meta.Meta
|
||||
broker meta.Broker
|
||||
cluster session.Cluster
|
||||
targetMgr meta.TargetManagerInterface
|
||||
targetObserver *observers.TargetObserver
|
||||
checkerController *checkers.CheckerController
|
||||
proxyManager proxyutil.ProxyClientManagerInterface
|
||||
}
|
||||
|
||||
func NewReleasePartitionJob(ctx context.Context,
|
||||
req *querypb.ReleasePartitionsRequest,
|
||||
dist *meta.DistributionManager,
|
||||
meta *meta.Meta,
|
||||
broker meta.Broker,
|
||||
targetMgr meta.TargetManagerInterface,
|
||||
targetObserver *observers.TargetObserver,
|
||||
checkerController *checkers.CheckerController,
|
||||
proxyManager proxyutil.ProxyClientManagerInterface,
|
||||
) *ReleasePartitionJob {
|
||||
return &ReleasePartitionJob{
|
||||
BaseJob: NewBaseJob(ctx, req.Base.GetMsgID(), req.GetCollectionID()),
|
||||
req: req,
|
||||
dist: dist,
|
||||
meta: meta,
|
||||
broker: broker,
|
||||
targetMgr: targetMgr,
|
||||
targetObserver: targetObserver,
|
||||
checkerController: checkerController,
|
||||
proxyManager: proxyManager,
|
||||
}
|
||||
}
|
||||
|
||||
func (job *ReleasePartitionJob) Execute() error {
|
||||
req := job.req
|
||||
log := log.Ctx(job.ctx).With(
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Int64s("partitionIDs", req.GetPartitionIDs()),
|
||||
)
|
||||
|
||||
if !job.meta.CollectionManager.Exist(job.ctx, req.GetCollectionID()) {
|
||||
log.Info("release collection end, the collection has not been loaded into QueryNode")
|
||||
return nil
|
||||
}
|
||||
|
||||
loadedPartitions := job.meta.CollectionManager.GetPartitionsByCollection(job.ctx, req.GetCollectionID())
|
||||
toRelease := lo.FilterMap(loadedPartitions, func(partition *meta.Partition, _ int) (int64, bool) {
|
||||
return partition.GetPartitionID(), lo.Contains(req.GetPartitionIDs(), partition.GetPartitionID())
|
||||
})
|
||||
|
||||
if len(toRelease) == 0 {
|
||||
log.Warn("releasing partition(s) not loaded")
|
||||
return nil
|
||||
}
|
||||
|
||||
// If all partitions are released, clear all
|
||||
if len(toRelease) == len(loadedPartitions) {
|
||||
log.Info("release partitions covers all partitions, will remove the whole collection")
|
||||
err := job.meta.CollectionManager.RemoveCollection(job.ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
msg := "failed to release partitions from store"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
err = job.meta.ReplicaManager.RemoveCollection(job.ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("failed to remove replicas", zap.Error(err))
|
||||
}
|
||||
job.targetObserver.ReleaseCollection(req.GetCollectionID())
|
||||
// try best discard cache
|
||||
// shall not affect releasing if failed
|
||||
job.proxyManager.InvalidateCollectionMetaCache(job.ctx,
|
||||
&proxypb.InvalidateCollMetaCacheRequest{
|
||||
CollectionID: req.GetCollectionID(),
|
||||
},
|
||||
proxyutil.SetMsgType(commonpb.MsgType_ReleaseCollection))
|
||||
// try best clean shard leader cache
|
||||
job.proxyManager.InvalidateShardLeaderCache(job.ctx, &proxypb.InvalidateShardLeaderCacheRequest{
|
||||
CollectionIDs: []int64{req.GetCollectionID()},
|
||||
})
|
||||
|
||||
waitCollectionReleased(job.dist, job.checkerController, req.GetCollectionID())
|
||||
} else {
|
||||
err := job.meta.CollectionManager.RemovePartition(job.ctx, req.GetCollectionID(), toRelease...)
|
||||
if err != nil {
|
||||
msg := "failed to release partitions from store"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
job.targetObserver.ReleasePartition(req.GetCollectionID(), toRelease...)
|
||||
// wait current target updated, so following querys will act as expected
|
||||
waitCurrentTargetUpdated(job.ctx, job.targetObserver, job.req.GetCollectionID())
|
||||
waitCollectionReleased(job.dist, job.checkerController, req.GetCollectionID(), toRelease...)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -96,5 +96,5 @@ func (job *SyncNewCreatedPartitionJob) Execute() error {
|
||||
return errors.Wrap(err, msg)
|
||||
}
|
||||
|
||||
return waitCurrentTargetUpdated(job.ctx, job.targetObserver, job.req.GetCollectionID())
|
||||
return WaitCurrentTargetUpdated(job.ctx, job.targetObserver, job.req.GetCollectionID())
|
||||
}
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
248
internal/querycoordv2/job/load_config.go
Normal file
248
internal/querycoordv2/job/load_config.go
Normal file
@ -0,0 +1,248 @@
|
||||
// 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 job
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sort"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/samber/lo"
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
)
|
||||
|
||||
var ErrIgnoredAlterLoadConfig = errors.New("ignored alter load config")
|
||||
|
||||
type AlterLoadConfigRequest struct {
|
||||
Meta *meta.Meta
|
||||
CollectionInfo *milvuspb.DescribeCollectionResponse
|
||||
Expected ExpectedLoadConfig
|
||||
Current CurrentLoadConfig
|
||||
}
|
||||
|
||||
// CheckIfLoadPartitionsExecutable checks if the load partitions is executable.
|
||||
func (req *AlterLoadConfigRequest) CheckIfLoadPartitionsExecutable() error {
|
||||
if req.Current.Collection == nil {
|
||||
return nil
|
||||
}
|
||||
expectedReplicaNumber := 0
|
||||
for _, num := range req.Expected.ExpectedReplicaNumber {
|
||||
expectedReplicaNumber += num
|
||||
}
|
||||
if len(req.Current.Replicas) != expectedReplicaNumber {
|
||||
return merr.WrapErrParameterInvalid(len(req.Current.Replicas), expectedReplicaNumber, "can't change the replica number for loaded partitions")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type ExpectedLoadConfig struct {
|
||||
ExpectedPartitionIDs []int64
|
||||
ExpectedReplicaNumber map[string]int // map resource group name to replica number in resource group
|
||||
ExpectedFieldIndexID map[int64]int64
|
||||
ExpectedLoadFields []int64
|
||||
ExpectedPriority commonpb.LoadPriority
|
||||
ExpectedUserSpecifiedReplicaMode bool
|
||||
}
|
||||
|
||||
type CurrentLoadConfig struct {
|
||||
Collection *meta.Collection
|
||||
Partitions map[int64]*meta.Partition
|
||||
Replicas map[int64]*meta.Replica
|
||||
}
|
||||
|
||||
func (c *CurrentLoadConfig) GetLoadPriority() commonpb.LoadPriority {
|
||||
for _, replica := range c.Replicas {
|
||||
return replica.LoadPriority()
|
||||
}
|
||||
return commonpb.LoadPriority_HIGH
|
||||
}
|
||||
|
||||
func (c *CurrentLoadConfig) GetFieldIndexID() map[int64]int64 {
|
||||
return c.Collection.FieldIndexID
|
||||
}
|
||||
|
||||
func (c *CurrentLoadConfig) GetLoadFields() []int64 {
|
||||
return c.Collection.LoadFields
|
||||
}
|
||||
|
||||
func (c *CurrentLoadConfig) GetUserSpecifiedReplicaMode() bool {
|
||||
return c.Collection.UserSpecifiedReplicaMode
|
||||
}
|
||||
|
||||
func (c *CurrentLoadConfig) GetReplicaNumber() map[string]int {
|
||||
replicaNumber := make(map[string]int)
|
||||
for _, replica := range c.Replicas {
|
||||
replicaNumber[replica.GetResourceGroup()]++
|
||||
}
|
||||
return replicaNumber
|
||||
}
|
||||
|
||||
func (c *CurrentLoadConfig) GetPartitionIDs() []int64 {
|
||||
partitionIDs := make([]int64, 0, len(c.Partitions))
|
||||
for _, partition := range c.Partitions {
|
||||
partitionIDs = append(partitionIDs, partition.GetPartitionID())
|
||||
}
|
||||
return partitionIDs
|
||||
}
|
||||
|
||||
// IntoLoadConfigMessageHeader converts the current load config into a load config message header.
|
||||
func (c *CurrentLoadConfig) IntoLoadConfigMessageHeader() *messagespb.AlterLoadConfigMessageHeader {
|
||||
if c.Collection == nil {
|
||||
return nil
|
||||
}
|
||||
partitionIDs := make([]int64, 0, len(c.Partitions))
|
||||
partitionIDs = append(partitionIDs, c.GetPartitionIDs()...)
|
||||
sort.Slice(partitionIDs, func(i, j int) bool {
|
||||
return partitionIDs[i] < partitionIDs[j]
|
||||
})
|
||||
|
||||
loadFields := generateLoadFields(c.GetLoadFields(), c.GetFieldIndexID())
|
||||
|
||||
replicas := make([]*messagespb.LoadReplicaConfig, 0, len(c.Replicas))
|
||||
for _, replica := range c.Replicas {
|
||||
replicas = append(replicas, &messagespb.LoadReplicaConfig{
|
||||
ReplicaId: replica.GetID(),
|
||||
ResourceGroupName: replica.GetResourceGroup(),
|
||||
Priority: replica.LoadPriority(),
|
||||
})
|
||||
}
|
||||
sort.Slice(replicas, func(i, j int) bool {
|
||||
return replicas[i].GetReplicaId() < replicas[j].GetReplicaId()
|
||||
})
|
||||
return &messagespb.AlterLoadConfigMessageHeader{
|
||||
DbId: c.Collection.DbID,
|
||||
CollectionId: c.Collection.CollectionID,
|
||||
PartitionIds: partitionIDs,
|
||||
LoadFields: loadFields,
|
||||
Replicas: replicas,
|
||||
UserSpecifiedReplicaMode: c.GetUserSpecifiedReplicaMode(),
|
||||
}
|
||||
}
|
||||
|
||||
// GenerateAlterLoadConfigMessage generates the alter load config message for the collection.
|
||||
func GenerateAlterLoadConfigMessage(ctx context.Context, req *AlterLoadConfigRequest) (message.BroadcastMutableMessage, error) {
|
||||
loadFields := generateLoadFields(req.Expected.ExpectedLoadFields, req.Expected.ExpectedFieldIndexID)
|
||||
loadReplicaConfigs, err := req.generateReplicas(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
partitionIDs := make([]int64, 0, len(req.Expected.ExpectedPartitionIDs))
|
||||
partitionIDs = append(partitionIDs, req.Expected.ExpectedPartitionIDs...)
|
||||
sort.Slice(partitionIDs, func(i, j int) bool {
|
||||
return partitionIDs[i] < partitionIDs[j]
|
||||
})
|
||||
header := &messagespb.AlterLoadConfigMessageHeader{
|
||||
DbId: req.CollectionInfo.DbId,
|
||||
CollectionId: req.CollectionInfo.CollectionID,
|
||||
PartitionIds: partitionIDs,
|
||||
LoadFields: loadFields,
|
||||
Replicas: loadReplicaConfigs,
|
||||
UserSpecifiedReplicaMode: req.Expected.ExpectedUserSpecifiedReplicaMode,
|
||||
}
|
||||
// check if the load configuration is changed
|
||||
if previousHeader := req.Current.IntoLoadConfigMessageHeader(); proto.Equal(previousHeader, header) {
|
||||
return nil, ErrIgnoredAlterLoadConfig
|
||||
}
|
||||
return message.NewAlterLoadConfigMessageBuilderV2().
|
||||
WithHeader(header).
|
||||
WithBody(&messagespb.AlterLoadConfigMessageBody{}).
|
||||
WithBroadcast([]string{streaming.WAL().ControlChannel()}).
|
||||
MustBuildBroadcast(), nil
|
||||
}
|
||||
|
||||
// generateLoadFields generates the load fields for the collection.
|
||||
func generateLoadFields(loadedFields []int64, fieldIndexID map[int64]int64) []*messagespb.LoadFieldConfig {
|
||||
loadFields := lo.Map(loadedFields, func(fieldID int64, _ int) *messagespb.LoadFieldConfig {
|
||||
if indexID, ok := fieldIndexID[fieldID]; ok {
|
||||
return &messagespb.LoadFieldConfig{
|
||||
FieldId: fieldID,
|
||||
IndexId: indexID,
|
||||
}
|
||||
}
|
||||
return &messagespb.LoadFieldConfig{
|
||||
FieldId: fieldID,
|
||||
IndexId: 0,
|
||||
}
|
||||
})
|
||||
sort.Slice(loadFields, func(i, j int) bool {
|
||||
return loadFields[i].GetFieldId() < loadFields[j].GetFieldId()
|
||||
})
|
||||
return loadFields
|
||||
}
|
||||
|
||||
// generateReplicas generates the replicas for the collection.
|
||||
func (req *AlterLoadConfigRequest) generateReplicas(ctx context.Context) ([]*messagespb.LoadReplicaConfig, error) {
|
||||
// fill up the existsReplicaNum found the redundant replicas and the replicas that should be kept
|
||||
existsReplicaNum := make(map[string]int)
|
||||
keptReplicas := make(map[int64]struct{}) // replica that should be kept
|
||||
redundantReplicas := make([]int64, 0) // replica that should be removed
|
||||
loadReplicaConfigs := make([]*messagespb.LoadReplicaConfig, 0)
|
||||
for _, replica := range req.Current.Replicas {
|
||||
if existsReplicaNum[replica.GetResourceGroup()] >= req.Expected.ExpectedReplicaNumber[replica.GetResourceGroup()] {
|
||||
redundantReplicas = append(redundantReplicas, replica.GetID())
|
||||
continue
|
||||
}
|
||||
keptReplicas[replica.GetID()] = struct{}{}
|
||||
loadReplicaConfigs = append(loadReplicaConfigs, &messagespb.LoadReplicaConfig{
|
||||
ReplicaId: replica.GetID(),
|
||||
ResourceGroupName: replica.GetResourceGroup(),
|
||||
Priority: replica.LoadPriority(),
|
||||
})
|
||||
existsReplicaNum[replica.GetResourceGroup()]++
|
||||
}
|
||||
|
||||
// check if there should generate new incoming replicas.
|
||||
for rg, num := range req.Expected.ExpectedReplicaNumber {
|
||||
for i := existsReplicaNum[rg]; i < num; i++ {
|
||||
if len(redundantReplicas) > 0 {
|
||||
// reuse the replica from redundant replicas.
|
||||
// make a transfer operation from a resource group to another resource group.
|
||||
replicaID := redundantReplicas[0]
|
||||
redundantReplicas = redundantReplicas[1:]
|
||||
loadReplicaConfigs = append(loadReplicaConfigs, &messagespb.LoadReplicaConfig{
|
||||
ReplicaId: replicaID,
|
||||
ResourceGroupName: rg,
|
||||
Priority: req.Expected.ExpectedPriority,
|
||||
})
|
||||
} else {
|
||||
// allocate a new replica.
|
||||
newID, err := req.Meta.ReplicaManager.AllocateReplicaID(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
loadReplicaConfigs = append(loadReplicaConfigs, &messagespb.LoadReplicaConfig{
|
||||
ReplicaId: newID,
|
||||
ResourceGroupName: rg,
|
||||
Priority: req.Expected.ExpectedPriority,
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
sort.Slice(loadReplicaConfigs, func(i, j int) bool {
|
||||
return loadReplicaConfigs[i].GetReplicaId() < loadReplicaConfigs[j].GetReplicaId()
|
||||
})
|
||||
return loadReplicaConfigs, nil
|
||||
}
|
||||
@ -30,10 +30,10 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
// waitCollectionReleased blocks until
|
||||
// WaitCollectionReleased blocks until
|
||||
// all channels and segments of given collection(partitions) are released,
|
||||
// empty partition list means wait for collection released
|
||||
func waitCollectionReleased(dist *meta.DistributionManager, checkerController *checkers.CheckerController, collection int64, partitions ...int64) {
|
||||
func WaitCollectionReleased(dist *meta.DistributionManager, checkerController *checkers.CheckerController, collection int64, partitions ...int64) {
|
||||
partitionSet := typeutil.NewUniqueSet(partitions...)
|
||||
for {
|
||||
var (
|
||||
@ -64,7 +64,7 @@ func waitCollectionReleased(dist *meta.DistributionManager, checkerController *c
|
||||
}
|
||||
}
|
||||
|
||||
func waitCurrentTargetUpdated(ctx context.Context, targetObserver *observers.TargetObserver, collection int64) error {
|
||||
func WaitCurrentTargetUpdated(ctx context.Context, targetObserver *observers.TargetObserver, collection int64) error {
|
||||
// manual trigger update next target
|
||||
ready, err := targetObserver.UpdateNextTarget(collection)
|
||||
if err != nil {
|
||||
|
||||
@ -114,7 +114,7 @@ func NewReplicaWithPriority(replica *querypb.Replica, priority commonpb.LoadPrio
|
||||
}
|
||||
|
||||
func (replica *Replica) LoadPriority() commonpb.LoadPriority {
|
||||
return replica.loadPriority
|
||||
return replica.loadPriority // TODO: the load priority doesn't persisted into the replica recovery info.
|
||||
}
|
||||
|
||||
// GetID returns the id of the replica.
|
||||
|
||||
@ -30,6 +30,7 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/metastore"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
@ -162,6 +163,70 @@ func (m *ReplicaManager) Get(ctx context.Context, id typeutil.UniqueID) *Replica
|
||||
return m.replicas[id]
|
||||
}
|
||||
|
||||
type SpawnWithReplicaConfigParams struct {
|
||||
CollectionID int64
|
||||
Channels []string
|
||||
Configs []*messagespb.LoadReplicaConfig
|
||||
}
|
||||
|
||||
// SpawnWithReplicaConfig spawns replicas with replica config.
|
||||
func (m *ReplicaManager) SpawnWithReplicaConfig(ctx context.Context, params SpawnWithReplicaConfigParams) ([]*Replica, error) {
|
||||
m.rwmutex.Lock()
|
||||
defer m.rwmutex.Unlock()
|
||||
|
||||
balancePolicy := paramtable.Get().QueryCoordCfg.Balancer.GetValue()
|
||||
enableChannelExclusiveMode := balancePolicy == ChannelLevelScoreBalancerName
|
||||
replicas := make([]*Replica, 0)
|
||||
for _, config := range params.Configs {
|
||||
replica := NewReplicaWithPriority(&querypb.Replica{
|
||||
ID: config.GetReplicaId(),
|
||||
CollectionID: params.CollectionID,
|
||||
ResourceGroup: config.ResourceGroupName,
|
||||
}, config.GetPriority())
|
||||
if enableChannelExclusiveMode {
|
||||
mutableReplica := replica.CopyForWrite()
|
||||
mutableReplica.TryEnableChannelExclusiveMode(params.Channels...)
|
||||
replica = mutableReplica.IntoReplica()
|
||||
}
|
||||
replicas = append(replicas, replica)
|
||||
}
|
||||
if err := m.put(ctx, replicas...); err != nil {
|
||||
return nil, errors.Wrap(err, "failed to put replicas")
|
||||
}
|
||||
if err := m.removeRedundantReplicas(ctx, params); err != nil {
|
||||
return nil, errors.Wrap(err, "failed to remove redundant replicas")
|
||||
}
|
||||
return replicas, nil
|
||||
}
|
||||
|
||||
// removeRedundantReplicas removes redundant replicas that is not in the new replica config.
|
||||
func (m *ReplicaManager) removeRedundantReplicas(ctx context.Context, params SpawnWithReplicaConfigParams) error {
|
||||
existedReplicas, ok := m.coll2Replicas[params.CollectionID]
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
toRemoveReplicas := make([]int64, 0)
|
||||
for _, replica := range existedReplicas.replicas {
|
||||
found := false
|
||||
replicaID := replica.GetID()
|
||||
for _, channel := range params.Configs {
|
||||
if channel.GetReplicaId() == replicaID {
|
||||
found = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !found {
|
||||
toRemoveReplicas = append(toRemoveReplicas, replicaID)
|
||||
}
|
||||
}
|
||||
return m.removeReplicas(ctx, params.CollectionID, toRemoveReplicas...)
|
||||
}
|
||||
|
||||
// AllocateReplicaID allocates a replica ID.
|
||||
func (m *ReplicaManager) AllocateReplicaID(ctx context.Context) (int64, error) {
|
||||
return m.idAllocator()
|
||||
}
|
||||
|
||||
// Spawn spawns N replicas at resource group for given collection in ReplicaManager.
|
||||
func (m *ReplicaManager) Spawn(ctx context.Context, collection int64, replicaNumInRG map[string]int,
|
||||
channels []string, loadPriority commonpb.LoadPriority,
|
||||
|
||||
@ -35,6 +35,7 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/metastore/mocks"
|
||||
. "github.com/milvus-io/milvus/internal/querycoordv2/params"
|
||||
"github.com/milvus-io/milvus/pkg/v2/kv"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
||||
@ -116,6 +117,42 @@ func (suite *ReplicaManagerSuite) TearDownTest() {
|
||||
suite.kv.Close()
|
||||
}
|
||||
|
||||
func (suite *ReplicaManagerSuite) TestSpawnWithReplicaConfig() {
|
||||
mgr := suite.mgr
|
||||
ctx := suite.ctx
|
||||
|
||||
replicas, err := mgr.SpawnWithReplicaConfig(ctx, SpawnWithReplicaConfigParams{
|
||||
CollectionID: 100,
|
||||
Channels: []string{"channel1", "channel2"},
|
||||
Configs: []*messagespb.LoadReplicaConfig{
|
||||
{ReplicaId: 1000, ResourceGroupName: "RG1", Priority: commonpb.LoadPriority_HIGH},
|
||||
},
|
||||
})
|
||||
suite.NoError(err)
|
||||
suite.Len(replicas, 1)
|
||||
|
||||
replicas, err = mgr.SpawnWithReplicaConfig(ctx, SpawnWithReplicaConfigParams{
|
||||
CollectionID: 100,
|
||||
Channels: []string{"channel1", "channel2"},
|
||||
Configs: []*messagespb.LoadReplicaConfig{
|
||||
{ReplicaId: 1000, ResourceGroupName: "RG1", Priority: commonpb.LoadPriority_HIGH},
|
||||
{ReplicaId: 1001, ResourceGroupName: "RG1", Priority: commonpb.LoadPriority_HIGH},
|
||||
},
|
||||
})
|
||||
suite.NoError(err)
|
||||
suite.Len(replicas, 2)
|
||||
|
||||
replicas, err = mgr.SpawnWithReplicaConfig(ctx, SpawnWithReplicaConfigParams{
|
||||
CollectionID: 100,
|
||||
Channels: []string{"channel1", "channel2"},
|
||||
Configs: []*messagespb.LoadReplicaConfig{
|
||||
{ReplicaId: 1000, ResourceGroupName: "RG1", Priority: commonpb.LoadPriority_HIGH},
|
||||
},
|
||||
})
|
||||
suite.NoError(err)
|
||||
suite.Len(replicas, 1)
|
||||
}
|
||||
|
||||
func (suite *ReplicaManagerSuite) TestSpawn() {
|
||||
mgr := suite.mgr
|
||||
ctx := suite.ctx
|
||||
|
||||
@ -21,6 +21,7 @@ import (
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"os"
|
||||
"strconv"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
@ -45,14 +46,15 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/session"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/task"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
|
||||
kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/commonpbutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
@ -86,8 +88,6 @@ type ServerSuite struct {
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
var testMeta string
|
||||
|
||||
func (suite *ServerSuite) SetupSuite() {
|
||||
paramtable.Init()
|
||||
params.GenerateEtcdConfig()
|
||||
@ -124,8 +124,9 @@ func (suite *ServerSuite) SetupSuite() {
|
||||
}
|
||||
|
||||
func (suite *ServerSuite) SetupTest() {
|
||||
initStreamingSystem()
|
||||
|
||||
var err error
|
||||
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, testMeta)
|
||||
suite.tikvCli = tikv.SetupLocalTxn()
|
||||
suite.server, err = suite.newQueryCoord()
|
||||
|
||||
@ -627,11 +628,17 @@ func (suite *ServerSuite) hackServer() {
|
||||
suite.server.proxyClientManager,
|
||||
)
|
||||
|
||||
suite.broker.EXPECT().DescribeCollection(mock.Anything, mock.Anything).Return(&milvuspb.DescribeCollectionResponse{Schema: &schemapb.CollectionSchema{}}, nil).Maybe()
|
||||
suite.broker.EXPECT().DescribeDatabase(mock.Anything, mock.Anything).Return(&rootcoordpb.DescribeDatabaseResponse{}, nil).Maybe()
|
||||
suite.broker.EXPECT().ListIndexes(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
|
||||
suite.broker.EXPECT().DescribeDatabase(mock.Anything, mock.Anything).Return(&rootcoordpb.DescribeDatabaseResponse{}, nil).Maybe()
|
||||
for _, collection := range suite.collections {
|
||||
suite.broker.EXPECT().GetPartitions(mock.Anything, collection).Return(suite.partitions[collection], nil).Maybe()
|
||||
suite.broker.EXPECT().DescribeCollection(mock.Anything, collection).Return(&milvuspb.DescribeCollectionResponse{
|
||||
DbName: util.DefaultDBName,
|
||||
DbId: 1,
|
||||
CollectionID: collection,
|
||||
CollectionName: "collection_" + strconv.FormatInt(collection, 10),
|
||||
Schema: &schemapb.CollectionSchema{},
|
||||
}, nil).Maybe()
|
||||
suite.expectGetRecoverInfo(collection)
|
||||
}
|
||||
log.Debug("server hacked")
|
||||
@ -670,18 +677,7 @@ func (suite *ServerSuite) newQueryCoord() (*Server, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
etcdCli, err := etcd.GetEtcdClient(
|
||||
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
|
||||
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
|
||||
Params.EtcdCfg.Endpoints.GetAsStrings(),
|
||||
Params.EtcdCfg.EtcdTLSCert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSKey.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
etcdCli, _ := kvfactory.GetEtcdAndPath()
|
||||
server.SetEtcdClient(etcdCli)
|
||||
server.SetTiKVClient(suite.tikvCli)
|
||||
|
||||
@ -931,9 +927,5 @@ func createTestSession(nodeID int64, address string, stopping bool) *sessionutil
|
||||
}
|
||||
|
||||
func TestServer(t *testing.T) {
|
||||
parameters := []string{"tikv", "etcd"}
|
||||
for _, v := range parameters {
|
||||
testMeta = v
|
||||
suite.Run(t, new(ServerSuite))
|
||||
}
|
||||
}
|
||||
|
||||
@ -195,284 +195,173 @@ func (s *Server) ShowLoadPartitions(ctx context.Context, req *querypb.ShowPartit
|
||||
}
|
||||
|
||||
func (s *Server) LoadCollection(ctx context.Context, req *querypb.LoadCollectionRequest) (*commonpb.Status, error) {
|
||||
log := log.Ctx(ctx).With(
|
||||
logger := log.Ctx(ctx).With(
|
||||
zap.Int64("dbID", req.GetDbID()),
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Int32("replicaNumber", req.GetReplicaNumber()),
|
||||
zap.Strings("resourceGroups", req.GetResourceGroups()),
|
||||
zap.Bool("refreshMode", req.GetRefresh()),
|
||||
)
|
||||
|
||||
log.Info("load collection request received",
|
||||
logger.Info("load collection request received",
|
||||
zap.Any("schema", req.Schema),
|
||||
zap.Int64s("fieldIndexes", lo.Values(req.GetFieldIndexID())),
|
||||
)
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.TotalLabel).Inc()
|
||||
|
||||
if err := merr.CheckHealthy(s.State()); err != nil {
|
||||
msg := "failed to load collection"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
logger.Warn("failed to load collection", zap.Error(err))
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(errors.Wrap(err, msg)), nil
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
// If refresh mode is ON.
|
||||
if req.GetRefresh() {
|
||||
err := s.refreshCollection(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("failed to refresh collection", zap.Error(err))
|
||||
logger.Warn("failed to refresh collection", zap.Error(err))
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
logger.Info("refresh collection done")
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
if err := s.broadcastAlterLoadConfigCollectionV2ForLoadCollection(ctx, req); err != nil {
|
||||
if errors.Is(err, job.ErrIgnoredAlterLoadConfig) {
|
||||
logger.Info("load collection ignored, collection is already loaded")
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return merr.Success(), nil
|
||||
}
|
||||
logger.Warn("failed to load collection", zap.Error(err))
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
// if user specified the replica number in load request, load config changes won't be apply to the collection automatically
|
||||
userSpecifiedReplicaMode := req.GetReplicaNumber() > 0
|
||||
// to be compatible with old sdk, which set replica=1 if replica is not specified
|
||||
// so only both replica and resource groups didn't set in request, it will turn to use the configured load info
|
||||
if req.GetReplicaNumber() <= 0 && len(req.GetResourceGroups()) == 0 {
|
||||
// when replica number or resource groups is not set, use pre-defined load config
|
||||
rgs, replicas, err := s.broker.GetCollectionLoadInfo(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("failed to get pre-defined load info", zap.Error(err))
|
||||
} else {
|
||||
if req.GetReplicaNumber() <= 0 && replicas > 0 {
|
||||
req.ReplicaNumber = int32(replicas)
|
||||
}
|
||||
|
||||
if len(req.GetResourceGroups()) == 0 && len(rgs) > 0 {
|
||||
req.ResourceGroups = rgs
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if req.GetReplicaNumber() <= 0 {
|
||||
log.Info("request doesn't indicate the number of replicas, set it to 1")
|
||||
req.ReplicaNumber = 1
|
||||
}
|
||||
|
||||
if len(req.GetResourceGroups()) == 0 {
|
||||
log.Info(fmt.Sprintf("request doesn't indicate the resource groups, set it to %s", meta.DefaultResourceGroupName))
|
||||
req.ResourceGroups = []string{meta.DefaultResourceGroupName}
|
||||
}
|
||||
|
||||
var loadJob job.Job
|
||||
collection := s.meta.GetCollection(ctx, req.GetCollectionID())
|
||||
if collection != nil {
|
||||
// if collection is loaded, check if collection is loaded with the same replica number and resource groups
|
||||
// if replica number or resource group changes, switch to update load config
|
||||
collectionUsedRG := s.meta.ReplicaManager.GetResourceGroupByCollection(ctx, collection.GetCollectionID()).Collect()
|
||||
left, right := lo.Difference(collectionUsedRG, req.GetResourceGroups())
|
||||
rgChanged := len(left) > 0 || len(right) > 0
|
||||
replicaChanged := collection.GetReplicaNumber() != req.GetReplicaNumber()
|
||||
if replicaChanged || rgChanged {
|
||||
log.Warn("collection is loaded with different replica number or resource group, switch to update load config",
|
||||
zap.Int32("oldReplicaNumber", collection.GetReplicaNumber()),
|
||||
zap.Strings("oldResourceGroups", collectionUsedRG))
|
||||
updateReq := &querypb.UpdateLoadConfigRequest{
|
||||
CollectionIDs: []int64{req.GetCollectionID()},
|
||||
ReplicaNumber: req.GetReplicaNumber(),
|
||||
ResourceGroups: req.GetResourceGroups(),
|
||||
}
|
||||
loadJob = job.NewUpdateLoadConfigJob(
|
||||
ctx,
|
||||
updateReq,
|
||||
s.meta,
|
||||
s.targetMgr,
|
||||
s.targetObserver,
|
||||
s.collectionObserver,
|
||||
userSpecifiedReplicaMode,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
if loadJob == nil {
|
||||
loadJob = job.NewLoadCollectionJob(ctx,
|
||||
req,
|
||||
s.dist,
|
||||
s.meta,
|
||||
s.broker,
|
||||
s.targetMgr,
|
||||
s.targetObserver,
|
||||
s.collectionObserver,
|
||||
s.nodeMgr,
|
||||
userSpecifiedReplicaMode,
|
||||
)
|
||||
}
|
||||
|
||||
s.jobScheduler.Add(loadJob)
|
||||
err := loadJob.Wait()
|
||||
if err != nil {
|
||||
msg := "failed to load collection"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(errors.Wrap(err, msg)), nil
|
||||
}
|
||||
|
||||
logger.Info("load collection done")
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func (s *Server) ReleaseCollection(ctx context.Context, req *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) {
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
)
|
||||
logger := log.Ctx(ctx).With(zap.Int64("collectionID", req.GetCollectionID()))
|
||||
|
||||
log.Info("release collection request received")
|
||||
logger.Info("release collection request received")
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.TotalLabel).Inc()
|
||||
tr := timerecord.NewTimeRecorder("release-collection")
|
||||
|
||||
if err := merr.CheckHealthy(s.State()); err != nil {
|
||||
msg := "failed to release collection"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
logger.Warn("failed to release collection", zap.Error(err))
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(errors.Wrap(err, msg)), nil
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
releaseJob := job.NewReleaseCollectionJob(ctx,
|
||||
req,
|
||||
s.dist,
|
||||
s.meta,
|
||||
s.broker,
|
||||
s.targetMgr,
|
||||
s.targetObserver,
|
||||
s.checkerController,
|
||||
s.proxyClientManager,
|
||||
)
|
||||
s.jobScheduler.Add(releaseJob)
|
||||
err := releaseJob.Wait()
|
||||
if err != nil {
|
||||
msg := "failed to release collection"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(errors.Wrap(err, msg)), nil
|
||||
if err := s.broadcastDropLoadConfigCollectionV2ForReleaseCollection(ctx, req); err != nil {
|
||||
if errors.Is(err, errReleaseCollectionNotLoaded) {
|
||||
logger.Info("release collection ignored, collection is not loaded")
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
log.Info("collection released")
|
||||
logger.Warn("failed to release collection", zap.Error(err))
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
job.WaitCollectionReleased(s.dist, s.checkerController, req.GetCollectionID())
|
||||
logger.Info("release collection done")
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
metrics.QueryCoordReleaseLatency.WithLabelValues().Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||
meta.GlobalFailedLoadCache.Remove(req.GetCollectionID())
|
||||
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func (s *Server) LoadPartitions(ctx context.Context, req *querypb.LoadPartitionsRequest) (*commonpb.Status, error) {
|
||||
log := log.Ctx(ctx).With(
|
||||
logger := log.Ctx(ctx).With(
|
||||
zap.Int64("dbID", req.GetDbID()),
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Int32("replicaNumber", req.GetReplicaNumber()),
|
||||
zap.Int64s("partitions", req.GetPartitionIDs()),
|
||||
zap.Strings("resourceGroups", req.GetResourceGroups()),
|
||||
zap.Bool("refreshMode", req.GetRefresh()),
|
||||
)
|
||||
|
||||
log.Info("received load partitions request",
|
||||
zap.Any("schema", req.Schema),
|
||||
zap.Int64s("partitions", req.GetPartitionIDs()))
|
||||
logger.Info("received load partitions request",
|
||||
zap.Any("schema", req.Schema))
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.TotalLabel).Inc()
|
||||
|
||||
if err := merr.CheckHealthy(s.State()); err != nil {
|
||||
msg := "failed to load partitions"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
logger.Warn("failed to load partitions", zap.Error(err))
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(errors.Wrap(err, msg)), nil
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
// If refresh mode is ON.
|
||||
if req.GetRefresh() {
|
||||
err := s.refreshCollection(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("failed to refresh partitions", zap.Error(err))
|
||||
}
|
||||
logger.Warn("failed to refresh partitions", zap.Error(err))
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
// if user specified the replica number in load request, load config changes won't be apply to the collection automatically
|
||||
userSpecifiedReplicaMode := req.GetReplicaNumber() > 0
|
||||
|
||||
// to be compatible with old sdk, which set replica=1 if replica is not specified
|
||||
// so only both replica and resource groups didn't set in request, it will turn to use the configured load info
|
||||
if req.GetReplicaNumber() <= 0 && len(req.GetResourceGroups()) == 0 {
|
||||
// when replica number or resource groups is not set, use database level config
|
||||
rgs, replicas, err := s.broker.GetCollectionLoadInfo(ctx, req.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("failed to get data base level load info", zap.Error(err))
|
||||
logger.Info("refresh partitions done")
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
if req.GetReplicaNumber() <= 0 {
|
||||
log.Info("load collection use database level replica number", zap.Int64("databaseLevelReplicaNum", replicas))
|
||||
req.ReplicaNumber = int32(replicas)
|
||||
if err := s.broadcastAlterLoadConfigCollectionV2ForLoadPartitions(ctx, req); err != nil {
|
||||
if errors.Is(err, job.ErrIgnoredAlterLoadConfig) {
|
||||
logger.Info("load partitions ignored, partitions are already loaded")
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
if len(req.GetResourceGroups()) == 0 {
|
||||
log.Info("load collection use database level resource groups", zap.Strings("databaseLevelResourceGroups", rgs))
|
||||
req.ResourceGroups = rgs
|
||||
}
|
||||
}
|
||||
|
||||
loadJob := job.NewLoadPartitionJob(ctx,
|
||||
req,
|
||||
s.dist,
|
||||
s.meta,
|
||||
s.broker,
|
||||
s.targetMgr,
|
||||
s.targetObserver,
|
||||
s.collectionObserver,
|
||||
s.nodeMgr,
|
||||
userSpecifiedReplicaMode,
|
||||
)
|
||||
s.jobScheduler.Add(loadJob)
|
||||
err := loadJob.Wait()
|
||||
if err != nil {
|
||||
msg := "failed to load partitions"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
logger.Warn("failed to load partitions", zap.Error(err))
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(errors.Wrap(err, msg)), nil
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
logger.Info("load partitions done")
|
||||
metrics.QueryCoordLoadCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func (s *Server) ReleasePartitions(ctx context.Context, req *querypb.ReleasePartitionsRequest) (*commonpb.Status, error) {
|
||||
log := log.Ctx(ctx).With(
|
||||
logger := log.Ctx(ctx).With(
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Int64s("partitionIDs", req.GetPartitionIDs()),
|
||||
)
|
||||
|
||||
log.Info("release partitions", zap.Int64s("partitions", req.GetPartitionIDs()))
|
||||
logger.Info("release partitions")
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.TotalLabel).Inc()
|
||||
|
||||
if err := merr.CheckHealthy(s.State()); err != nil {
|
||||
msg := "failed to release partitions"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(errors.Wrap(err, msg)), nil
|
||||
}
|
||||
|
||||
if len(req.GetPartitionIDs()) == 0 {
|
||||
err := merr.WrapErrParameterInvalid("any partition", "empty partition list")
|
||||
log.Warn("no partition to release", zap.Error(err))
|
||||
logger.Warn("failed to release partitions", zap.Error(err))
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
tr := timerecord.NewTimeRecorder("release-partitions")
|
||||
releaseJob := job.NewReleasePartitionJob(ctx,
|
||||
req,
|
||||
s.dist,
|
||||
s.meta,
|
||||
s.broker,
|
||||
s.targetMgr,
|
||||
s.targetObserver,
|
||||
s.checkerController,
|
||||
s.proxyClientManager,
|
||||
)
|
||||
s.jobScheduler.Add(releaseJob)
|
||||
err := releaseJob.Wait()
|
||||
if err != nil {
|
||||
msg := "failed to release partitions"
|
||||
log.Warn(msg, zap.Error(err))
|
||||
if len(req.GetPartitionIDs()) == 0 {
|
||||
err := merr.WrapErrParameterInvalid("any partition", "empty partition list")
|
||||
logger.Warn("no partition to release", zap.Error(err))
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(errors.Wrap(err, msg)), nil
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
collectionReleased, err := s.broadcastAlterLoadConfigCollectionV2ForReleasePartitions(ctx, req)
|
||||
if err != nil {
|
||||
if errors.Is(err, job.ErrIgnoredAlterLoadConfig) {
|
||||
logger.Info("release partitions ignored, partitions are already released")
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
return merr.Success(), nil
|
||||
}
|
||||
logger.Warn("failed to release partitions", zap.Error(err))
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
if collectionReleased {
|
||||
job.WaitCollectionReleased(s.dist, s.checkerController, req.GetCollectionID())
|
||||
} else {
|
||||
job.WaitCurrentTargetUpdated(ctx, s.targetObserver, req.GetCollectionID())
|
||||
job.WaitCollectionReleased(s.dist, s.checkerController, req.GetCollectionID(), req.GetPartitionIDs()...)
|
||||
}
|
||||
logger.Info("release partitions done", zap.Bool("collectionReleased", collectionReleased))
|
||||
metrics.QueryCoordReleaseCount.WithLabelValues(metrics.SuccessLabel).Inc()
|
||||
metrics.QueryCoordReleaseLatency.WithLabelValues().Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||
|
||||
meta.GlobalFailedLoadCache.Remove(req.GetCollectionID())
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
@ -18,6 +18,7 @@ package querycoordv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sort"
|
||||
"testing"
|
||||
"time"
|
||||
@ -32,6 +33,7 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/rgpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||
"github.com/milvus-io/milvus/internal/json"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
@ -61,11 +63,13 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/walimplstest"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/retry"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
@ -122,7 +126,9 @@ func initStreamingSystem() {
|
||||
LastConfirmedMessageID: walimplstest.NewTestMessageID(1),
|
||||
}
|
||||
}
|
||||
registry.CallMessageAckCallback(context.Background(), msg, results)
|
||||
retry.Do(context.Background(), func() error {
|
||||
return registry.CallMessageAckCallback(context.Background(), msg, results)
|
||||
}, retry.AttemptAlways())
|
||||
return &types.BroadcastAppendResult{}, nil
|
||||
})
|
||||
bapi.EXPECT().Close().Return()
|
||||
@ -143,8 +149,8 @@ func (suite *ServiceSuite) SetupSuite() {
|
||||
|
||||
suite.collections = []int64{1000, 1001}
|
||||
suite.partitions = map[int64][]int64{
|
||||
1000: {100, 101},
|
||||
1001: {102, 103},
|
||||
1000: {100, 101, 102},
|
||||
1001: {103, 104, 105},
|
||||
}
|
||||
suite.channels = map[int64][]string{
|
||||
1000: {"1000-dmc0", "1000-dmc1"},
|
||||
@ -154,16 +160,19 @@ func (suite *ServiceSuite) SetupSuite() {
|
||||
1000: {
|
||||
100: {1, 2},
|
||||
101: {3, 4},
|
||||
102: {5, 6},
|
||||
},
|
||||
1001: {
|
||||
102: {5, 6},
|
||||
103: {7, 8},
|
||||
104: {9, 10},
|
||||
105: {11, 12},
|
||||
},
|
||||
}
|
||||
suite.loadTypes = map[int64]querypb.LoadType{
|
||||
1000: querypb.LoadType_LoadCollection,
|
||||
1001: querypb.LoadType_LoadPartition,
|
||||
}
|
||||
|
||||
suite.replicaNumber = map[int64]int32{
|
||||
1000: 1,
|
||||
1001: 3,
|
||||
@ -279,7 +288,35 @@ func (suite *ServiceSuite) SetupTest() {
|
||||
suite.server.UpdateStateCode(commonpb.StateCode_Healthy)
|
||||
|
||||
suite.broker.EXPECT().GetCollectionLoadInfo(mock.Anything, mock.Anything).Return([]string{meta.DefaultResourceGroupName}, 1, nil).Maybe()
|
||||
|
||||
suite.broker.EXPECT().DescribeCollection(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, collectionID int64) (*milvuspb.DescribeCollectionResponse, error) {
|
||||
for _, collection := range suite.collections {
|
||||
if collection == collectionID {
|
||||
return &milvuspb.DescribeCollectionResponse{
|
||||
DbName: util.DefaultDBName,
|
||||
DbId: 1,
|
||||
CollectionID: collectionID,
|
||||
CollectionName: fmt.Sprintf("collection_%d", collectionID),
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{FieldID: 100},
|
||||
{FieldID: 101},
|
||||
{FieldID: 102},
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
}
|
||||
return &milvuspb.DescribeCollectionResponse{
|
||||
Status: merr.Status(merr.ErrCollectionNotFound),
|
||||
}, nil
|
||||
}).Maybe()
|
||||
suite.broker.EXPECT().GetPartitions(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, collectionID int64) ([]int64, error) {
|
||||
partitionIDs, ok := suite.partitions[collectionID]
|
||||
if !ok {
|
||||
return nil, merr.WrapErrCollectionNotFound(collectionID)
|
||||
}
|
||||
return partitionIDs, nil
|
||||
}).Maybe()
|
||||
registry.ResetRegistration()
|
||||
RegisterDDLCallbacks(suite.server)
|
||||
}
|
||||
@ -447,7 +484,6 @@ func (suite *ServiceSuite) TestLoadCollectionWithUserSpecifiedReplicaMode() {
|
||||
collectionID := suite.collections[0]
|
||||
|
||||
// Mock broker methods using mockey
|
||||
mockey.Mock(mockey.GetMethod(suite.broker, "DescribeCollection")).Return(nil, nil).Build()
|
||||
suite.expectGetRecoverInfo(collectionID)
|
||||
|
||||
// Test when user specifies replica number - should set IsUserSpecifiedReplicaMode to true
|
||||
@ -473,7 +509,6 @@ func (suite *ServiceSuite) TestLoadCollectionWithoutUserSpecifiedReplicaMode() {
|
||||
collectionID := suite.collections[0]
|
||||
|
||||
// Mock broker methods using mockey
|
||||
mockey.Mock(mockey.GetMethod(suite.broker, "DescribeCollection")).Return(nil, nil).Build()
|
||||
suite.expectGetRecoverInfo(collectionID)
|
||||
|
||||
// Test when user doesn't specify replica number - should not set IsUserSpecifiedReplicaMode
|
||||
@ -1076,7 +1111,6 @@ func (suite *ServiceSuite) TestLoadPartitionsWithUserSpecifiedReplicaMode() {
|
||||
partitionIDs := suite.partitions[collectionID]
|
||||
|
||||
// Mock broker methods using mockey
|
||||
mockey.Mock(mockey.GetMethod(suite.broker, "DescribeCollection")).Return(nil, nil).Build()
|
||||
suite.expectGetRecoverInfo(collectionID)
|
||||
|
||||
// Test when user specifies replica number - should set IsUserSpecifiedReplicaMode to true
|
||||
@ -1104,7 +1138,6 @@ func (suite *ServiceSuite) TestLoadPartitionsWithoutUserSpecifiedReplicaMode() {
|
||||
partitionIDs := suite.partitions[collectionID]
|
||||
|
||||
// Mock broker methods using mockey
|
||||
mockey.Mock(mockey.GetMethod(suite.broker, "DescribeCollection")).Return(nil, nil).Build()
|
||||
suite.expectGetRecoverInfo(collectionID)
|
||||
|
||||
// Test when user doesn't specify replica number - should not set IsUserSpecifiedReplicaMode
|
||||
@ -1954,21 +1987,8 @@ func (suite *ServiceSuite) loadAll() {
|
||||
CollectionID: collection,
|
||||
ReplicaNumber: suite.replicaNumber[collection],
|
||||
}
|
||||
job := job.NewLoadCollectionJob(
|
||||
ctx,
|
||||
req,
|
||||
suite.dist,
|
||||
suite.meta,
|
||||
suite.broker,
|
||||
suite.targetMgr,
|
||||
suite.targetObserver,
|
||||
suite.collectionObserver,
|
||||
suite.nodeMgr,
|
||||
false,
|
||||
)
|
||||
suite.jobScheduler.Add(job)
|
||||
err := job.Wait()
|
||||
suite.NoError(err)
|
||||
resp, err := suite.server.LoadCollection(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.EqualValues(suite.replicaNumber[collection], suite.meta.GetReplicaNumber(ctx, collection))
|
||||
suite.True(suite.meta.Exist(ctx, collection))
|
||||
suite.NotNil(suite.meta.GetCollection(ctx, collection))
|
||||
@ -1979,21 +1999,8 @@ func (suite *ServiceSuite) loadAll() {
|
||||
PartitionIDs: suite.partitions[collection],
|
||||
ReplicaNumber: suite.replicaNumber[collection],
|
||||
}
|
||||
job := job.NewLoadPartitionJob(
|
||||
ctx,
|
||||
req,
|
||||
suite.dist,
|
||||
suite.meta,
|
||||
suite.broker,
|
||||
suite.targetMgr,
|
||||
suite.targetObserver,
|
||||
suite.collectionObserver,
|
||||
suite.nodeMgr,
|
||||
false,
|
||||
)
|
||||
suite.jobScheduler.Add(job)
|
||||
err := job.Wait()
|
||||
suite.NoError(err)
|
||||
resp, err := suite.server.LoadPartitions(ctx, req)
|
||||
suite.Require().NoError(merr.CheckRPCCall(resp, err))
|
||||
suite.EqualValues(suite.replicaNumber[collection], suite.meta.GetReplicaNumber(ctx, collection))
|
||||
suite.True(suite.meta.Exist(ctx, collection))
|
||||
suite.NotNil(suite.meta.GetPartitionsByCollection(ctx, collection))
|
||||
@ -2058,6 +2065,12 @@ func (suite *ServiceSuite) assertSegments(collection int64, segments []*querypb.
|
||||
return true
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) expectGetRecoverInfoForAllCollections() {
|
||||
for _, collection := range suite.collections {
|
||||
suite.expectGetRecoverInfo(collection)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) expectGetRecoverInfo(collection int64) {
|
||||
suite.broker.EXPECT().GetPartitions(mock.Anything, collection).Return(suite.partitions[collection], nil).Maybe()
|
||||
vChannels := []*datapb.VchannelInfo{}
|
||||
@ -2119,6 +2132,7 @@ func (suite *ServiceSuite) updateChannelDist(ctx context.Context, collection int
|
||||
segments := lo.Flatten(lo.Values(suite.segments[collection]))
|
||||
|
||||
replicas := suite.meta.ReplicaManager.GetByCollection(ctx, collection)
|
||||
targetVersion := suite.targetMgr.GetCollectionTargetVersion(ctx, collection, meta.CurrentTargetFirst)
|
||||
for _, replica := range replicas {
|
||||
i := 0
|
||||
for _, node := range suite.sortInt64(replica.GetNodes()) {
|
||||
@ -2138,6 +2152,7 @@ func (suite *ServiceSuite) updateChannelDist(ctx context.Context, collection int
|
||||
Version: time.Now().Unix(),
|
||||
}
|
||||
}),
|
||||
TargetVersion: targetVersion,
|
||||
Status: &querypb.LeaderViewStatus{
|
||||
Serviceable: true,
|
||||
},
|
||||
@ -2151,6 +2166,16 @@ func (suite *ServiceSuite) updateChannelDist(ctx context.Context, collection int
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) releaseSegmentDist(nodeID int64) {
|
||||
suite.dist.SegmentDistManager.Update(nodeID)
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) releaseAllChannelDist() {
|
||||
for _, node := range suite.nodes {
|
||||
suite.dist.ChannelDistManager.Update(node)
|
||||
}
|
||||
}
|
||||
|
||||
func (suite *ServiceSuite) sortInt64(ints []int64) []int64 {
|
||||
sort.Slice(ints, func(i int, j int) bool {
|
||||
return ints[i] < ints[j]
|
||||
@ -2218,6 +2243,8 @@ func (suite *ServiceSuite) fetchHeartbeats(time time.Time) {
|
||||
|
||||
func (suite *ServiceSuite) TearDownTest() {
|
||||
suite.targetObserver.Stop()
|
||||
suite.collectionObserver.Stop()
|
||||
suite.jobScheduler.Stop()
|
||||
}
|
||||
|
||||
func TestService(t *testing.T) {
|
||||
|
||||
@ -21,7 +21,6 @@ import (
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
|
||||
@ -109,8 +108,7 @@ func RecoverAllCollection(m *meta.Meta) {
|
||||
|
||||
func AssignReplica(ctx context.Context, m *meta.Meta, resourceGroups []string, replicaNumber int32, checkNodeNum bool) (map[string]int, error) {
|
||||
if len(resourceGroups) != 0 && len(resourceGroups) != 1 && len(resourceGroups) != int(replicaNumber) {
|
||||
return nil, errors.Errorf(
|
||||
"replica=[%d] resource group=[%s], resource group num can only be 0, 1 or same as replica number", replicaNumber, strings.Join(resourceGroups, ","))
|
||||
return nil, merr.WrapErrParameterInvalidMsg("replica=[%d] resource group=[%s], resource group num can only be 0, 1 or same as replica number", replicaNumber, strings.Join(resourceGroups, ","))
|
||||
}
|
||||
|
||||
if streamingutil.IsStreamingServiceEnabled() {
|
||||
@ -158,6 +156,19 @@ func AssignReplica(ctx context.Context, m *meta.Meta, resourceGroups []string, r
|
||||
return replicaNumInRG, nil
|
||||
}
|
||||
|
||||
// SpawnReplicasWithReplicaConfig spawns replicas with replica config.
|
||||
func SpawnReplicasWithReplicaConfig(ctx context.Context, m *meta.Meta, params meta.SpawnWithReplicaConfigParams) ([]*meta.Replica, error) {
|
||||
replicas, err := m.ReplicaManager.SpawnWithReplicaConfig(ctx, params)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
RecoverReplicaOfCollection(ctx, m, params.CollectionID)
|
||||
if streamingutil.IsStreamingServiceEnabled() {
|
||||
m.RecoverSQNodesInCollection(ctx, params.CollectionID, snmanager.StaticStreamingNodeManager.GetStreamingQueryNodeIDs())
|
||||
}
|
||||
return replicas, nil
|
||||
}
|
||||
|
||||
// SpawnReplicasWithRG spawns replicas in rgs one by one for given collection.
|
||||
func SpawnReplicasWithRG(ctx context.Context, m *meta.Meta, collection int64, resourceGroups []string,
|
||||
replicaNumber int32, channels []string, loadPriority commonpb.LoadPriority,
|
||||
|
||||
@ -28,7 +28,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message/ce"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/commonpbutil"
|
||||
@ -77,32 +76,21 @@ func (c *DDLCallback) dropCollectionV1AckCallback(ctx context.Context, result me
|
||||
// when the control channel is acknowledged, we should do the following steps:
|
||||
|
||||
// 1. release the collection from querycoord first.
|
||||
resp, err := c.mixCoord.ReleaseCollection(ctx, &querypb.ReleaseCollectionRequest{
|
||||
Base: commonpbutil.NewMsgBase(
|
||||
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
||||
),
|
||||
CollectionID: collectionID,
|
||||
})
|
||||
if err := merr.CheckRPCCall(resp, err); err != nil {
|
||||
dropLoadConfigMsg := message.NewDropLoadConfigMessageBuilderV2().
|
||||
WithHeader(&message.DropLoadConfigMessageHeader{
|
||||
DbId: msg.Header().DbId,
|
||||
CollectionId: collectionID,
|
||||
}).
|
||||
WithBody(&message.DropLoadConfigMessageBody{}).
|
||||
WithBroadcast([]string{streaming.WAL().ControlChannel()}).
|
||||
MustBuildBroadcast().
|
||||
WithBroadcastID(msg.BroadcastHeader().BroadcastID)
|
||||
if err := registry.CallMessageAckCallback(ctx, dropLoadConfigMsg, map[string]*message.AppendResult{
|
||||
streaming.WAL().ControlChannel(): result,
|
||||
}); err != nil {
|
||||
return errors.Wrap(err, "failed to release collection")
|
||||
}
|
||||
|
||||
// TODO: after DropLoadConfig is supported, we should uncomment the following code.
|
||||
// dropLoadConfigMsg := message.NewDropLoadConfigMessageBuilderV2().
|
||||
// WithHeader(&message.DropLoadConfigMessageHeader{
|
||||
// DbId: msg.Header().DbId,
|
||||
// CollectionId: collectionID,
|
||||
// }).
|
||||
// WithBody(&message.DropLoadConfigMessageBody{}).
|
||||
// WithBroadcast([]string{streaming.WAL().ControlChannel()}).
|
||||
// MustBuildBroadcast().
|
||||
// WithBroadcastID(msg.BroadcastHeader().BroadcastID)
|
||||
// if err := registry.CallMessageAckCallback(ctx, dropLoadConfigMsg, map[string]*message.AppendResult{
|
||||
// streaming.WAL().ControlChannel(): result,
|
||||
// }); err != nil {
|
||||
// return errors.Wrap(err, "failed to release collection")
|
||||
// }
|
||||
|
||||
// 2. drop the collection index.
|
||||
dropIndexMsg := message.NewDropIndexMessageBuilderV2().
|
||||
WithHeader(&message.DropIndexMessageHeader{
|
||||
|
||||
@ -61,6 +61,7 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/retry"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
@ -101,6 +102,9 @@ func initStreamingSystemAndCore(t *testing.T) *Core {
|
||||
registry.RegisterDropIndexV2AckCallback(func(ctx context.Context, result message.BroadcastResultDropIndexMessageV2) error {
|
||||
return nil
|
||||
})
|
||||
registry.RegisterDropLoadConfigV2AckCallback(func(ctx context.Context, result message.BroadcastResultDropLoadConfigMessageV2) error {
|
||||
return nil
|
||||
})
|
||||
|
||||
wal := mock_streaming.NewMockWALAccesser(t)
|
||||
wal.EXPECT().ControlChannel().Return(funcutil.GetControlChannel("by-dev-rootcoord-dml_0")).Maybe()
|
||||
@ -116,7 +120,9 @@ func initStreamingSystemAndCore(t *testing.T) *Core {
|
||||
LastConfirmedMessageID: rmq.NewRmqID(1),
|
||||
}
|
||||
}
|
||||
registry.CallMessageAckCallback(context.Background(), msg, results)
|
||||
retry.Do(context.Background(), func() error {
|
||||
return registry.CallMessageAckCallback(context.Background(), msg, results)
|
||||
}, retry.AttemptAlways())
|
||||
return &types.BroadcastAppendResult{}, nil
|
||||
}).Maybe()
|
||||
bapi.EXPECT().Close().Return().Maybe()
|
||||
|
||||
@ -151,9 +151,9 @@ func (p *producerImpl) Append(ctx context.Context, msg message.MutableMessage) (
|
||||
return nil, ctx.Err()
|
||||
case resp := <-respCh:
|
||||
if resp.err != nil {
|
||||
if s := status.AsStreamingError(resp.err); s.IsFenced() {
|
||||
if s := status.AsStreamingError(resp.err); s.IsFenced() || s.IsOnShutdown() {
|
||||
if p.isFenced.CompareAndSwap(false, true) {
|
||||
p.logger.Warn("producer client is fenced", zap.Error(resp.err))
|
||||
p.logger.Warn("producer client is fenced or on shutdown", zap.Error(resp.err))
|
||||
p.available.Close()
|
||||
}
|
||||
}
|
||||
|
||||
@ -81,6 +81,11 @@ func (e *StreamingError) IsResourceAcquired() bool {
|
||||
return e.Code == streamingpb.StreamingCode_STREAMING_CODE_RESOURCE_ACQUIRED
|
||||
}
|
||||
|
||||
// IsOnShutdown returns true if the error is caused by on shutdown.
|
||||
func (e *StreamingError) IsOnShutdown() bool {
|
||||
return e.Code == streamingpb.StreamingCode_STREAMING_CODE_ON_SHUTDOWN
|
||||
}
|
||||
|
||||
// NewOnShutdownError creates a new StreamingError with code STREAMING_CODE_ON_SHUTDOWN.
|
||||
func NewOnShutdownError(format string, args ...interface{}) *StreamingError {
|
||||
return New(streamingpb.StreamingCode_STREAMING_CODE_ON_SHUTDOWN, format, args...)
|
||||
|
||||
@ -133,17 +133,22 @@ func marshalSpecializedHeader(t MessageType, v Version, h string, enc zapcore.Ob
|
||||
enc.AddInt64("segmentID", header.GetSegmentId())
|
||||
case *ManualFlushMessageHeader:
|
||||
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||
encodeIDs(header.GetSegmentIds(), enc)
|
||||
encodeIDs("flushedSegmentIDs", header.GetSegmentIds(), enc)
|
||||
case *SchemaChangeMessageHeader:
|
||||
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||
encodeIDs(header.GetFlushedSegmentIds(), enc)
|
||||
encodeIDs("flushedSegmentIDs", header.GetFlushedSegmentIds(), enc)
|
||||
case *AlterCollectionMessageHeader:
|
||||
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||
enc.AddString("udpateMasks", strings.Join(header.UpdateMask.GetPaths(), "|"))
|
||||
encodeIDs(header.GetFlushedSegmentIds(), enc)
|
||||
encodeIDs("flushedSegmentIDs", header.GetFlushedSegmentIds(), enc)
|
||||
case *AlterLoadConfigMessageHeader:
|
||||
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||
enc.AddInt64("replicaNumber", int64(len(header.GetReplicas())))
|
||||
encodeIDs("partitionIDs", header.GetPartitionIds(), enc)
|
||||
replicaIDs := make([]int64, 0, len(header.GetReplicas()))
|
||||
for _, replica := range header.GetReplicas() {
|
||||
replicaIDs = append(replicaIDs, replica.GetReplicaId())
|
||||
}
|
||||
encodeIDs("replicaIDs", replicaIDs, enc)
|
||||
case *DropLoadConfigMessageHeader:
|
||||
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||
case *CreateDatabaseMessageHeader:
|
||||
@ -190,10 +195,10 @@ func marshalSpecializedHeader(t MessageType, v Version, h string, enc zapcore.Ob
|
||||
enc.AddString("indexName", header.GetIndexName())
|
||||
case *AlterIndexMessageHeader:
|
||||
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||
encodeIDs(header.GetIndexIds(), enc)
|
||||
encodeIDs("indexIDs", header.GetIndexIds(), enc)
|
||||
case *DropIndexMessageHeader:
|
||||
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||
encodeIDs(header.GetIndexIds(), enc)
|
||||
encodeIDs("indexIDs", header.GetIndexIds(), enc)
|
||||
case *ImportMessageHeader:
|
||||
case *AlterResourceGroupMessageHeader:
|
||||
encodeResourceGroupConfigs(header.GetResourceGroupConfigs(), enc)
|
||||
@ -206,12 +211,12 @@ func marshalSpecializedHeader(t MessageType, v Version, h string, enc zapcore.Ob
|
||||
}
|
||||
}
|
||||
|
||||
func encodeIDs(targetIDs []int64, enc zapcore.ObjectEncoder) {
|
||||
func encodeIDs(name string, targetIDs []int64, enc zapcore.ObjectEncoder) {
|
||||
ids := make([]string, 0, len(targetIDs))
|
||||
for _, id := range targetIDs {
|
||||
ids = append(ids, strconv.FormatInt(id, 10))
|
||||
}
|
||||
enc.AddString("segmentIDs", strings.Join(ids, "|"))
|
||||
enc.AddString(name, strings.Join(ids, "|"))
|
||||
}
|
||||
|
||||
func encodeResourceGroupConfigs(configs map[string]*rgpb.ResourceGroupConfig, enc zapcore.ObjectEncoder) {
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user