未验证 提交 470e8ef4 编写于 作者: Y yah01 提交者: GitHub

Store index ID to filter out invalid handoff event (#19703)

Signed-off-by: Nyah01 <yang.cen@zilliz.com>
Signed-off-by: Nyah01 <yang.cen@zilliz.com>
上级 02a8a073
......@@ -868,6 +868,7 @@ func (i *IndexCoord) DescribeIndex(ctx context.Context, req *indexpb.DescribeInd
IndexParams: index.IndexParams,
IsAutoIndex: index.IsAutoIndex,
UserIndexParams: index.UserIndexParams,
IndexID: index.IndexID,
}
if err := i.completeIndexInfo(ctx, indexInfo); err != nil {
log.Error("IndexCoord describe index fail", zap.Int64("collectionID", req.CollectionID),
......
......@@ -466,6 +466,7 @@ message CollectionLoadInfo {
repeated int64 released_partitions = 2;
int32 replica_number = 3;
LoadStatus status = 4;
map<int64, int64> field_indexID = 5;
}
message PartitionLoadInfo {
......@@ -473,6 +474,7 @@ message PartitionLoadInfo {
int64 partitionID = 2;
int32 replica_number = 3;
LoadStatus status = 4;
map<int64, int64> field_indexID = 5;
}
message Replica {
......
......@@ -152,13 +152,17 @@ func (job *LoadCollectionJob) PreExecute() error {
msg := "load the partition after load collection is not supported"
log.Warn(msg)
return utils.WrapError(msg, ErrLoadParameterMismatched)
}
if old.GetReplicaNumber() != req.GetReplicaNumber() {
} else if old.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(req.GetCollectionID()),
)
log.Warn(msg)
return utils.WrapError(msg, ErrLoadParameterMismatched)
} else if !typeutil.MapEqual(old.GetFieldIndexID(), req.GetFieldIndexID()) {
msg := fmt.Sprintf("collection with different index %v existed, release this collection first before changing its index",
old.GetFieldIndexID())
log.Warn(msg)
return utils.WrapError(msg, ErrLoadParameterMismatched)
}
return ErrCollectionLoaded
}
......@@ -221,6 +225,7 @@ func (job *LoadCollectionJob) Execute() error {
CollectionID: req.GetCollectionID(),
ReplicaNumber: req.GetReplicaNumber(),
Status: querypb.LoadStatus_Loading,
FieldIndexID: req.GetFieldIndexID(),
},
CreatedAt: time.Now(),
UpdatedAt: time.Now(),
......@@ -350,11 +355,15 @@ func (job *LoadPartitionJob) PreExecute() error {
msg := "load the partition after load collection is not supported"
log.Warn(msg)
return utils.WrapError(msg, ErrLoadParameterMismatched)
}
if job.meta.GetReplicaNumber(req.GetCollectionID()) != req.GetReplicaNumber() {
} else if job.meta.GetReplicaNumber(req.GetCollectionID()) != req.GetReplicaNumber() {
msg := "collection with different replica number existed, release this collection first before changing its replica number"
log.Warn(msg)
return utils.WrapError(msg, ErrLoadParameterMismatched)
} else if !typeutil.MapEqual(job.meta.GetFieldIndex(req.GetCollectionID()), req.GetFieldIndexID()) {
msg := fmt.Sprintf("collection with different index %v existed, release this collection first before changing its index",
job.meta.GetFieldIndex(req.GetCollectionID()))
log.Warn(msg)
return utils.WrapError(msg, ErrLoadParameterMismatched)
}
// Check whether one of the given partitions not loaded
......@@ -423,6 +432,7 @@ func (job *LoadPartitionJob) Execute() error {
PartitionID: partition,
ReplicaNumber: req.GetReplicaNumber(),
Status: querypb.LoadStatus_Loading,
FieldIndexID: req.GetFieldIndexID(),
},
CreatedAt: time.Now(),
}
......
......@@ -34,6 +34,11 @@ import (
"github.com/stretchr/testify/suite"
)
const (
defaultVecFieldID = 1
defaultIndexID = 1
)
type JobSuite struct {
suite.Suite
......@@ -286,6 +291,65 @@ func (suite *JobSuite) TestLoadCollectionWithReplicas() {
}
}
func (suite *JobSuite) TestLoadCollectionWithDiffIndex() {
ctx := context.Background()
// 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,
FieldIndexID: map[int64]int64{
defaultVecFieldID: defaultIndexID,
},
}
job := NewLoadCollectionJob(
ctx,
req,
suite.dist,
suite.meta,
suite.targetMgr,
suite.broker,
suite.nodeMgr,
suite.handoffObserver,
)
suite.scheduler.Add(job)
err := job.Wait()
suite.NoError(err)
suite.EqualValues(1, suite.meta.GetReplicaNumber(collection))
suite.assertLoaded(collection)
}
// Test load with different index
for _, collection := range suite.collections {
if suite.loadTypes[collection] != querypb.LoadType_LoadCollection {
continue
}
req := &querypb.LoadCollectionRequest{
CollectionID: collection,
FieldIndexID: map[int64]int64{
defaultVecFieldID: -defaultIndexID,
},
}
job := NewLoadCollectionJob(
ctx,
req,
suite.dist,
suite.meta,
suite.targetMgr,
suite.broker,
suite.nodeMgr,
suite.handoffObserver,
)
suite.scheduler.Add(job)
err := job.Wait()
suite.ErrorIs(err, ErrLoadParameterMismatched)
}
}
func (suite *JobSuite) TestLoadPartition() {
ctx := context.Background()
......@@ -451,6 +515,68 @@ func (suite *JobSuite) TestLoadPartitionWithReplicas() {
}
}
func (suite *JobSuite) TestLoadPartitionWithDiffIndex() {
ctx := context.Background()
// 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],
FieldIndexID: map[int64]int64{
defaultVecFieldID: defaultIndexID,
},
}
job := NewLoadPartitionJob(
ctx,
req,
suite.dist,
suite.meta,
suite.targetMgr,
suite.broker,
suite.nodeMgr,
suite.handoffObserver,
)
suite.scheduler.Add(job)
err := job.Wait()
suite.NoError(err)
suite.EqualValues(1, suite.meta.GetReplicaNumber(collection))
suite.assertLoaded(collection)
}
// Test load partition with different index
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],
FieldIndexID: map[int64]int64{
defaultVecFieldID: -defaultIndexID,
},
}
job := NewLoadPartitionJob(
ctx,
req,
suite.dist,
suite.meta,
suite.targetMgr,
suite.broker,
suite.nodeMgr,
suite.handoffObserver,
)
suite.scheduler.Add(job)
err := job.Wait()
suite.ErrorIs(err, ErrLoadParameterMismatched)
}
}
func (suite *JobSuite) TestReleaseCollection() {
ctx := context.Background()
......
......@@ -193,6 +193,52 @@ func (m *CollectionManager) GetStatus(id UniqueID) querypb.LoadStatus {
return querypb.LoadStatus_Loaded
}
func (m *CollectionManager) GetFieldIndex(collectionID UniqueID) map[int64]int64 {
m.rwmutex.RLock()
defer m.rwmutex.RUnlock()
collection, ok := m.collections[collectionID]
if ok {
return collection.GetFieldIndexID()
}
partitions := m.getPartitionsByCollection(collectionID)
if len(partitions) == 0 {
return nil
}
return partitions[0].GetFieldIndexID()
}
// ContainAnyIndex returns true if the loaded collection contains one of the given indexes,
// returns false otherwise.
func (m *CollectionManager) ContainAnyIndex(collectionID int64, indexIDs ...int64) bool {
m.rwmutex.RLock()
defer m.rwmutex.RUnlock()
for _, indexID := range indexIDs {
if m.containIndex(collectionID, indexID) {
return true
}
}
return false
}
func (m *CollectionManager) containIndex(collectionID, indexID int64) bool {
collection, ok := m.collections[collectionID]
if ok {
return lo.Contains(lo.Values(collection.GetFieldIndexID()), indexID)
}
partitions := m.getPartitionsByCollection(collectionID)
if len(partitions) == 0 {
return false
}
for _, partition := range partitions {
if lo.Contains(lo.Values(partition.GetFieldIndexID()), indexID) {
return true
}
}
return false
}
func (m *CollectionManager) Exist(id UniqueID) bool {
m.rwmutex.RLock()
defer m.rwmutex.RUnlock()
......
......@@ -22,6 +22,7 @@ import (
"time"
"github.com/golang/protobuf/proto"
"github.com/samber/lo"
"go.etcd.io/etcd/api/v3/mvccpb"
"go.uber.org/zap"
......@@ -219,29 +220,36 @@ func (ob *HandoffObserver) schedule(ctx context.Context) {
func (ob *HandoffObserver) tryHandoff(ctx context.Context, segment *querypb.SegmentInfo) {
ob.handoffEventLock.Lock()
defer ob.handoffEventLock.Unlock()
log := log.With(zap.Int64("collectionID", segment.CollectionID),
zap.Int64("partitionID", segment.PartitionID),
zap.Int64("segmentID", segment.SegmentID))
partitionStatus, collectionRegistered := ob.collectionStatus[segment.CollectionID]
if Params.QueryCoordCfg.AutoHandoff && collectionRegistered {
if partitionStatus == CollectionHandoffStatusRegistered {
ob.handoffEvents[segment.SegmentID] = &HandoffEvent{
indexIDs := lo.Map(segment.GetIndexInfos(), func(indexInfo *querypb.FieldIndexInfo, _ int) int64 { return indexInfo.GetIndexID() })
log := log.With(zap.Int64("collectionID", segment.GetCollectionID()),
zap.Int64("partitionID", segment.GetPartitionID()),
zap.Int64("segmentID", segment.GetSegmentID()),
zap.Int64s("indexIDs", indexIDs),
)
log.Info("try handoff segment...")
status, ok := ob.collectionStatus[segment.GetCollectionID()]
if Params.QueryCoordCfg.AutoHandoff &&
ok &&
ob.meta.CollectionManager.ContainAnyIndex(segment.GetCollectionID(), indexIDs...) {
if status == CollectionHandoffStatusRegistered {
ob.handoffEvents[segment.GetSegmentID()] = &HandoffEvent{
Segment: segment,
Status: HandoffEventStatusReceived,
}
} else {
ob.handoffEvents[segment.SegmentID] = &HandoffEvent{
ob.handoffEvents[segment.GetSegmentID()] = &HandoffEvent{
Segment: segment,
Status: HandoffEventStatusTriggered,
}
ob.handoff(segment)
}
_, ok := ob.handoffSubmitOrders[segment.PartitionID]
_, ok := ob.handoffSubmitOrders[segment.GetPartitionID()]
if !ok {
ob.handoffSubmitOrders[segment.PartitionID] = make([]int64, 0)
ob.handoffSubmitOrders[segment.GetPartitionID()] = make([]int64, 0)
}
ob.handoffSubmitOrders[segment.PartitionID] = append(ob.handoffSubmitOrders[segment.PartitionID], segment.SegmentID)
ob.handoffSubmitOrders[segment.GetPartitionID()] = append(ob.handoffSubmitOrders[segment.GetPartitionID()], segment.GetSegmentID())
} else {
// ignore handoff task
log.Debug("handoff event trigger failed due to collection/partition is not loaded!")
......@@ -258,9 +266,9 @@ func (ob *HandoffObserver) handoff(segment *querypb.SegmentInfo) {
uniqueSet.Insert(segment.GetCompactionFrom()...)
segmentInfo := &datapb.SegmentInfo{
ID: segment.SegmentID,
CollectionID: segment.CollectionID,
PartitionID: segment.PartitionID,
ID: segment.GetSegmentID(),
CollectionID: segment.GetCollectionID(),
PartitionID: segment.GetPartitionID(),
NumOfRows: segment.NumRows,
InsertChannel: segment.GetDmChannel(),
State: segment.GetSegmentState(),
......@@ -282,9 +290,9 @@ func (ob *HandoffObserver) isGrowingSegmentReleased(id int64) bool {
func (ob *HandoffObserver) isSealedSegmentLoaded(segment *querypb.SegmentInfo) bool {
// must be sealed Segment loaded in all replica, in case of handoff between growing and sealed
nodes := ob.dist.LeaderViewManager.GetSealedSegmentDist(segment.SegmentID)
replicas := utils.GroupNodesByReplica(ob.meta.ReplicaManager, segment.CollectionID, nodes)
return len(replicas) == len(ob.meta.ReplicaManager.GetByCollection(segment.CollectionID))
nodes := ob.dist.LeaderViewManager.GetSealedSegmentDist(segment.GetSegmentID())
replicas := utils.GroupNodesByReplica(ob.meta.ReplicaManager, segment.GetCollectionID(), nodes)
return len(replicas) == len(ob.meta.ReplicaManager.GetByCollection(segment.GetCollectionID()))
}
func (ob *HandoffObserver) getOverrideSegmentInfo(handOffSegments []*datapb.SegmentInfo, segmentIDs ...int64) []int64 {
......@@ -313,9 +321,9 @@ func (ob *HandoffObserver) tryRelease(ctx context.Context, event *HandoffEvent)
return
}
log.Info("remove compactFrom segments",
zap.Int64("collectionID", segment.CollectionID),
zap.Int64("partitionID", segment.PartitionID),
zap.Int64("segmentID", segment.SegmentID),
zap.Int64("collectionID", segment.GetCollectionID()),
zap.Int64("partitionID", segment.GetPartitionID()),
zap.Int64("segmentID", segment.GetSegmentID()),
zap.Int64s("sourceSegments", compactSource),
)
for _, toRelease := range compactSource {
......@@ -342,13 +350,13 @@ func (ob *HandoffObserver) tryClean(ctx context.Context) {
segment := event.Segment
if ob.isAllCompactFromReleased(segment) {
log.Info("HandoffObserver: clean handoff event after handoff finished!",
zap.Int64("collectionID", segment.CollectionID),
zap.Int64("partitionID", segment.PartitionID),
zap.Int64("segmentID", segment.SegmentID),
zap.Int64("collectionID", segment.GetCollectionID()),
zap.Int64("partitionID", segment.GetPartitionID()),
zap.Int64("segmentID", segment.GetSegmentID()),
)
err := ob.cleanEvent(ctx, segment)
if err == nil {
delete(ob.handoffEvents, segment.SegmentID)
delete(ob.handoffEvents, segment.GetSegmentID())
}
pos++
} else {
......
......@@ -39,6 +39,11 @@ import (
"github.com/milvus-io/milvus/internal/util/typeutil"
)
const (
defaultVecFieldID = 1
defaultIndexID = 1
)
type HandoffObserverTestSuit struct {
suite.Suite
// Data
......@@ -144,6 +149,7 @@ func (suite *HandoffObserverTestSuit) TestFlushingHandoff() {
CollectionID: suite.collection,
PartitionID: suite.partition,
SegmentState: commonpb.SegmentState_Sealed,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
suite.produceHandOffEvent(flushingSegment)
......@@ -199,6 +205,7 @@ func (suite *HandoffObserverTestSuit) TestCompactHandoff() {
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{1},
CreatedByCompaction: true,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
suite.produceHandOffEvent(compactSegment)
......@@ -252,6 +259,7 @@ func (suite *HandoffObserverTestSuit) TestRecursiveHandoff() {
CollectionID: suite.collection,
PartitionID: suite.partition,
SegmentState: commonpb.SegmentState_Sealed,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
compactSegment1 := &querypb.SegmentInfo{
......@@ -261,6 +269,7 @@ func (suite *HandoffObserverTestSuit) TestRecursiveHandoff() {
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{3},
CreatedByCompaction: true,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
compactSegment2 := &querypb.SegmentInfo{
......@@ -270,6 +279,7 @@ func (suite *HandoffObserverTestSuit) TestRecursiveHandoff() {
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{4},
CreatedByCompaction: true,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
suite.produceHandOffEvent(flushingSegment)
......@@ -336,6 +346,7 @@ func (suite *HandoffObserverTestSuit) TestReloadHandoffEventOrder() {
CollectionID: suite.collection,
PartitionID: suite.partition,
SegmentState: commonpb.SegmentState_Sealed,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
compactSegment1 := &querypb.SegmentInfo{
SegmentID: 9,
......@@ -344,6 +355,7 @@ func (suite *HandoffObserverTestSuit) TestReloadHandoffEventOrder() {
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{3},
CreatedByCompaction: true,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
compactSegment2 := &querypb.SegmentInfo{
SegmentID: 10,
......@@ -352,6 +364,7 @@ func (suite *HandoffObserverTestSuit) TestReloadHandoffEventOrder() {
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{4},
CreatedByCompaction: true,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
suite.produceHandOffEvent(flushingSegment)
......@@ -381,6 +394,7 @@ func (suite *HandoffObserverTestSuit) TestLoadHandoffEventFromStore() {
CollectionID: suite.collection,
PartitionID: suite.partition,
SegmentState: commonpb.SegmentState_Sealed,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
compactSegment1 := &querypb.SegmentInfo{
SegmentID: 4,
......@@ -389,6 +403,7 @@ func (suite *HandoffObserverTestSuit) TestLoadHandoffEventFromStore() {
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{3},
CreatedByCompaction: true,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
compactSegment2 := &querypb.SegmentInfo{
SegmentID: 5,
......@@ -397,6 +412,7 @@ func (suite *HandoffObserverTestSuit) TestLoadHandoffEventFromStore() {
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{4},
CreatedByCompaction: true,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
suite.produceHandOffEvent(flushingSegment)
......@@ -459,6 +475,12 @@ func (suite *HandoffObserverTestSuit) produceHandOffEvent(segmentInfo *querypb.S
suite.NoError(err)
}
func (suite *HandoffObserverTestSuit) existHandOffEvent(segmentInfo *querypb.SegmentInfo) bool {
key := fmt.Sprintf("%s/%d/%d/%d", util.HandoffSegmentPrefix, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.SegmentID)
_, err := suite.kv.Load(key)
return err == nil
}
func (suite *HandoffObserverTestSuit) load() {
// Mock meta data
replicas, err := suite.meta.ReplicaManager.Spawn(suite.collection, suite.replicaNumber)
......@@ -474,6 +496,7 @@ func (suite *HandoffObserverTestSuit) load() {
CollectionID: suite.collection,
ReplicaNumber: suite.replicaNumber,
Status: querypb.LoadStatus_Loaded,
FieldIndexID: map[int64]int64{defaultVecFieldID: defaultIndexID},
},
LoadPercentage: 0,
CreatedAt: time.Now(),
......@@ -519,6 +542,7 @@ func (suite *HandoffObserverTestSuit) TestHandoffOnUnLoadedPartition() {
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{2},
CreatedByCompaction: true,
IndexInfos: []*querypb.FieldIndexInfo{{IndexID: defaultIndexID}},
}
suite.produceHandOffEvent(compactSegment)
......@@ -537,6 +561,37 @@ func (suite *HandoffObserverTestSuit) TestHandoffOnUnLoadedPartition() {
}, 3*time.Second, 1*time.Second)
}
func (suite *HandoffObserverTestSuit) TestFilterOutEventByIndexID() {
// init leader view
suite.dist.LeaderViewManager.Update(2, &meta.LeaderView{
ID: 1,
CollectionID: suite.collection,
Channel: suite.channel.ChannelName,
Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 1, Version: 0}, 2: {NodeID: 2, Version: 0}},
})
Params.QueryCoordCfg.CheckHandoffInterval = 1 * time.Second
err := suite.observer.Start(context.Background())
suite.NoError(err)
compactSegment := &querypb.SegmentInfo{
SegmentID: 3,
CollectionID: suite.collection,
PartitionID: suite.partition,
SegmentState: commonpb.SegmentState_Sealed,
CompactionFrom: []int64{1},
CreatedByCompaction: true,
}
suite.produceHandOffEvent(compactSegment)
suite.Eventually(func() bool {
suite.observer.handoffEventLock.RLock()
defer suite.observer.handoffEventLock.RUnlock()
_, ok := suite.observer.handoffEvents[compactSegment.GetSegmentID()]
return !ok && !suite.target.ContainSegment(3) && !suite.existHandOffEvent(compactSegment)
}, 3*time.Second, 1*time.Second)
}
func TestHandoffObserverSuit(t *testing.T) {
suite.Run(t, new(HandoffObserverTestSuit))
}
......@@ -188,7 +188,9 @@ func (s *Server) LoadCollection(ctx context.Context, req *querypb.LoadCollection
log.Info("load collection request received",
zap.Any("schema", req.Schema),
zap.Int32("replicaNumber", req.ReplicaNumber))
zap.Int32("replicaNumber", req.ReplicaNumber),
zap.Int64s("fieldIndexes", lo.Values(req.GetFieldIndexID())),
)
metrics.QueryCoordLoadCount.WithLabelValues(metrics.TotalLabel).Inc()
if s.status.Load() != commonpb.StateCode_Healthy {
......
......@@ -2,6 +2,20 @@ package typeutil
import "sync"
// MapEqual returns true if the two map contain the same keys and values
func MapEqual(left, right map[int64]int64) bool {
if len(left) != len(right) {
return false
}
for k, v := range left {
if v2, ok := right[k]; !ok || v != v2 {
return false
}
}
return true
}
// MergeMap merge one map to another
func MergeMap(src map[string]string, dst map[string]string) map[string]string {
for k, v := range src {
......
......@@ -3,28 +3,57 @@ package typeutil
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/suite"
)
func TestMergeMap(t *testing.T) {
type MapUtilSuite struct {
suite.Suite
}
func (suite *MapUtilSuite) TestMapEqual() {
left := map[int64]int64{
1: 10,
2: 20,
}
right := map[int64]int64{
1: 10,
2: 200,
}
suite.False(MapEqual(left, right))
suite.False(MapEqual(right, left))
right[2] = 20
suite.True(MapEqual(left, right))
suite.True(MapEqual(right, left))
left[3] = 30
suite.False(MapEqual(left, right))
suite.False(MapEqual(right, left))
}
func (suite *MapUtilSuite) TestMergeMap() {
src := make(map[string]string)
src["Alice"] = "female"
src["Bob"] = "male"
dst := make(map[string]string)
dst = MergeMap(src, dst)
assert.EqualValues(t, dst, src)
suite.EqualValues(dst, src)
src = nil
dst = nil
dst = MergeMap(src, dst)
assert.Nil(t, dst)
suite.Nil(dst)
}
func TestGetMapKeys(t *testing.T) {
func (suite *MapUtilSuite) TestGetMapKeys() {
src := make(map[string]string)
src["Alice"] = "female"
src["Bob"] = "male"
keys := GetMapKeys(src)
assert.Equal(t, 2, len(keys))
assert.Contains(t, keys, "Alice", "Bob")
suite.Equal(2, len(keys))
suite.Contains(keys, "Alice", "Bob")
}
func TestMapUtil(t *testing.T) {
suite.Run(t, new(MapUtilSuite))
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册