fix: use latest timetick to expire cache (#45699)

issue: #45697
pr: #45717

- also remove the  too frequent metric collection of tokenizer.

---------

Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
Zhen Ye 2025-11-20 18:32:52 +08:00 committed by GitHub
parent 151ade0215
commit 92f4b48a3c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
19 changed files with 66 additions and 74 deletions

View File

@ -10,35 +10,26 @@
// or implied. See the License for the specific language governing permissions and limitations under the License // or implied. See the License for the specific language governing permissions and limitations under the License
#include "segcore/token_stream_c.h" #include "segcore/token_stream_c.h"
#include "token-stream.h" #include "token-stream.h"
#include "monitor/scope_metric.h"
void void
free_token_stream(CTokenStream token_stream) { free_token_stream(CTokenStream token_stream) {
SCOPE_CGO_CALL_METRIC();
delete static_cast<milvus::tantivy::TokenStream*>(token_stream); delete static_cast<milvus::tantivy::TokenStream*>(token_stream);
} }
bool bool
token_stream_advance(CTokenStream token_stream) { token_stream_advance(CTokenStream token_stream) {
SCOPE_CGO_CALL_METRIC();
return static_cast<milvus::tantivy::TokenStream*>(token_stream)->advance(); return static_cast<milvus::tantivy::TokenStream*>(token_stream)->advance();
} }
// Note: returned token must be freed by the caller using `free_token`. // Note: returned token must be freed by the caller using `free_token`.
const char* const char*
token_stream_get_token(CTokenStream token_stream) { token_stream_get_token(CTokenStream token_stream) {
SCOPE_CGO_CALL_METRIC();
return static_cast<milvus::tantivy::TokenStream*>(token_stream) return static_cast<milvus::tantivy::TokenStream*>(token_stream)
->get_token_no_copy(); ->get_token_no_copy();
} }
CToken CToken
token_stream_get_detailed_token(CTokenStream token_stream) { token_stream_get_detailed_token(CTokenStream token_stream) {
SCOPE_CGO_CALL_METRIC();
auto token = static_cast<milvus::tantivy::TokenStream*>(token_stream) auto token = static_cast<milvus::tantivy::TokenStream*>(token_stream)
->get_detailed_token(); ->get_detailed_token();
return CToken{token.token, return CToken{token.token,
@ -50,7 +41,5 @@ token_stream_get_detailed_token(CTokenStream token_stream) {
void void
free_token(void* token) { free_token(void* token) {
SCOPE_CGO_CALL_METRIC();
free_rust_string(static_cast<const char*>(token)); free_rust_string(static_cast<const char*>(token));
} }

View File

@ -13,7 +13,6 @@
#include <memory> #include <memory>
#include "common/FieldMeta.h" #include "common/FieldMeta.h"
#include "common/protobuf_utils.h" #include "common/protobuf_utils.h"
#include "monitor/scope_metric.h"
#include "pb/schema.pb.h" #include "pb/schema.pb.h"
#include "common/EasyAssert.h" #include "common/EasyAssert.h"
#include "tokenizer.h" #include "tokenizer.h"
@ -22,8 +21,6 @@ using Map = std::map<std::string, std::string>;
CStatus CStatus
create_tokenizer(const char* params, CTokenizer* tokenizer) { create_tokenizer(const char* params, CTokenizer* tokenizer) {
SCOPE_CGO_CALL_METRIC();
try { try {
auto impl = std::make_unique<milvus::tantivy::Tokenizer>(params); auto impl = std::make_unique<milvus::tantivy::Tokenizer>(params);
*tokenizer = impl.release(); *tokenizer = impl.release();
@ -35,8 +32,6 @@ create_tokenizer(const char* params, CTokenizer* tokenizer) {
CStatus CStatus
clone_tokenizer(CTokenizer* tokenizer, CTokenizer* rst) { clone_tokenizer(CTokenizer* tokenizer, CTokenizer* rst) {
SCOPE_CGO_CALL_METRIC();
try { try {
auto impl = reinterpret_cast<milvus::tantivy::Tokenizer*>(*tokenizer); auto impl = reinterpret_cast<milvus::tantivy::Tokenizer*>(*tokenizer);
*rst = impl->Clone().release(); *rst = impl->Clone().release();
@ -48,24 +43,18 @@ clone_tokenizer(CTokenizer* tokenizer, CTokenizer* rst) {
void void
free_tokenizer(CTokenizer tokenizer) { free_tokenizer(CTokenizer tokenizer) {
SCOPE_CGO_CALL_METRIC();
auto impl = reinterpret_cast<milvus::tantivy::Tokenizer*>(tokenizer); auto impl = reinterpret_cast<milvus::tantivy::Tokenizer*>(tokenizer);
delete impl; delete impl;
} }
CTokenStream CTokenStream
create_token_stream(CTokenizer tokenizer, const char* text, uint32_t text_len) { create_token_stream(CTokenizer tokenizer, const char* text, uint32_t text_len) {
SCOPE_CGO_CALL_METRIC();
auto impl = reinterpret_cast<milvus::tantivy::Tokenizer*>(tokenizer); auto impl = reinterpret_cast<milvus::tantivy::Tokenizer*>(tokenizer);
return impl->CreateTokenStream(std::string(text, text_len)).release(); return impl->CreateTokenStream(std::string(text, text_len)).release();
} }
CStatus CStatus
validate_tokenizer(const char* params) { validate_tokenizer(const char* params) {
SCOPE_CGO_CALL_METRIC();
try { try {
auto impl = std::make_unique<milvus::tantivy::Tokenizer>(params); auto impl = std::make_unique<milvus::tantivy::Tokenizer>(params);
return milvus::SuccessCStatus(); return milvus::SuccessCStatus();
@ -76,8 +65,6 @@ validate_tokenizer(const char* params) {
CStatus CStatus
validate_text_schema(const uint8_t* field_schema, uint64_t length) { validate_text_schema(const uint8_t* field_schema, uint64_t length) {
SCOPE_CGO_CALL_METRIC();
try { try {
auto schema = std::make_unique<milvus::proto::schema::FieldSchema>(); auto schema = std::make_unique<milvus::proto::schema::FieldSchema>();
AssertInfo(schema->ParseFromArray(field_schema, length), AssertInfo(schema->ParseFromArray(field_schema, length),

View File

@ -134,16 +134,17 @@ func (node *Proxy) InvalidateCollectionMetaCache(ctx context.Context, request *p
if globalMetaCache != nil { if globalMetaCache != nil {
switch msgType { switch msgType {
case commonpb.MsgType_DropCollection, commonpb.MsgType_RenameCollection, commonpb.MsgType_DropAlias, commonpb.MsgType_AlterAlias, commonpb.MsgType_CreateAlias: case commonpb.MsgType_DropCollection, commonpb.MsgType_RenameCollection, commonpb.MsgType_DropAlias, commonpb.MsgType_AlterAlias, commonpb.MsgType_CreateAlias:
// remove collection by name first, otherwise the drop collection remove version will be failed.
if collectionName != "" {
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName, request.GetBase().GetTimestamp()) // no need to return error, though collection may be not cached
node.shardMgr.DeprecateShardCache(request.GetDbName(), collectionName)
}
if request.CollectionID != UniqueID(0) { if request.CollectionID != UniqueID(0) {
aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, request.GetBase().GetTimestamp(), msgType == commonpb.MsgType_DropCollection) aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, request.GetBase().GetTimestamp(), msgType == commonpb.MsgType_DropCollection)
for _, name := range aliasName { for _, name := range aliasName {
node.shardMgr.DeprecateShardCache(request.GetDbName(), name) node.shardMgr.DeprecateShardCache(request.GetDbName(), name)
} }
} }
if collectionName != "" {
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName) // no need to return error, though collection may be not cached
node.shardMgr.DeprecateShardCache(request.GetDbName(), collectionName)
}
log.Info("complete to invalidate collection meta cache with collection name", zap.String("type", request.GetBase().GetMsgType().String())) log.Info("complete to invalidate collection meta cache with collection name", zap.String("type", request.GetBase().GetMsgType().String()))
case commonpb.MsgType_LoadCollection, commonpb.MsgType_ReleaseCollection: case commonpb.MsgType_LoadCollection, commonpb.MsgType_ReleaseCollection:
// All the request from query use collectionID // All the request from query use collectionID
@ -163,7 +164,7 @@ func (node *Proxy) InvalidateCollectionMetaCache(ctx context.Context, request *p
if request.CollectionID != UniqueID(0) { if request.CollectionID != UniqueID(0) {
aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, request.GetBase().GetTimestamp(), false) aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, request.GetBase().GetTimestamp(), false)
} }
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName) globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName, request.GetBase().GetTimestamp())
log.Info("complete to invalidate collection meta cache", zap.String("type", request.GetBase().GetMsgType().String())) log.Info("complete to invalidate collection meta cache", zap.String("type", request.GetBase().GetMsgType().String()))
case commonpb.MsgType_DropDatabase: case commonpb.MsgType_DropDatabase:
node.shardMgr.RemoveDatabase(request.GetDbName()) node.shardMgr.RemoveDatabase(request.GetDbName())
@ -178,7 +179,7 @@ func (node *Proxy) InvalidateCollectionMetaCache(ctx context.Context, request *p
} }
} }
if collectionName != "" { if collectionName != "" {
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName) globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName, request.GetBase().GetTimestamp())
} }
log.Info("complete to invalidate collection meta cache", zap.String("type", request.GetBase().GetMsgType().String())) log.Info("complete to invalidate collection meta cache", zap.String("type", request.GetBase().GetMsgType().String()))
default: default:
@ -191,7 +192,7 @@ func (node *Proxy) InvalidateCollectionMetaCache(ctx context.Context, request *p
} }
if collectionName != "" { if collectionName != "" {
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName) // no need to return error, though collection may be not cached globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName, request.GetBase().GetTimestamp()) // no need to return error, though collection may be not cached
node.shardMgr.DeprecateShardCache(request.GetDbName(), collectionName) node.shardMgr.DeprecateShardCache(request.GetDbName(), collectionName)
} }
} }

View File

@ -70,7 +70,7 @@ type Cache interface {
// DeprecateShardCache(database, collectionName string) // DeprecateShardCache(database, collectionName string)
// InvalidateShardLeaderCache(collections []int64) // InvalidateShardLeaderCache(collections []int64)
// ListShardLocation() map[int64]nodeInfo // ListShardLocation() map[int64]nodeInfo
RemoveCollection(ctx context.Context, database, collectionName string) RemoveCollection(ctx context.Context, database, collectionName string, version uint64)
RemoveCollectionsByID(ctx context.Context, collectionID UniqueID, version uint64, removeVersion bool) []string RemoveCollectionsByID(ctx context.Context, collectionID UniqueID, version uint64, removeVersion bool) []string
// GetCredentialInfo operate credential cache // GetCredentialInfo operate credential cache
@ -800,23 +800,33 @@ func parsePartitionsInfo(infos []*partitionInfo, hasPartitionKey bool) *partitio
return result return result
} }
func (m *MetaCache) RemoveCollection(ctx context.Context, database, collectionName string) { func (m *MetaCache) RemoveCollection(ctx context.Context, database, collectionName string, version uint64) {
m.mu.Lock() m.mu.Lock()
defer m.mu.Unlock() defer m.mu.Unlock()
_, dbOk := m.collInfo[database]
if dbOk { if db, dbOk := m.collInfo[database]; dbOk {
delete(m.collInfo[database], collectionName) if coll, ok := db[collectionName]; ok {
m.removeCollectionByID(ctx, coll.collID, version, false)
}
} }
if database == "" { if database == "" {
delete(m.collInfo[defaultDB], collectionName) if db, dbOk := m.collInfo[defaultDB]; dbOk {
if coll, ok := db[collectionName]; ok {
m.removeCollectionByID(ctx, coll.collID, version, false)
} }
log.Ctx(ctx).Debug("remove collection", zap.String("db", database), zap.String("collection", collectionName), zap.Bool("dbok", dbOk)) }
}
log.Ctx(ctx).Debug("remove collection", zap.String("db", database), zap.String("collection", collectionName))
} }
func (m *MetaCache) RemoveCollectionsByID(ctx context.Context, collectionID UniqueID, version uint64, removeVersion bool) []string { func (m *MetaCache) RemoveCollectionsByID(ctx context.Context, collectionID UniqueID, version uint64, removeVersion bool) []string {
m.mu.Lock() m.mu.Lock()
defer m.mu.Unlock() defer m.mu.Unlock()
return m.removeCollectionByID(ctx, collectionID, version, removeVersion)
}
func (m *MetaCache) removeCollectionByID(ctx context.Context, collectionID UniqueID, version uint64, removeVersion bool) []string {
curVersion := m.collectionCacheVersion[collectionID] curVersion := m.collectionCacheVersion[collectionID]
var collNames []string var collNames []string
for database, db := range m.collInfo { for database, db := range m.collInfo {

View File

@ -1304,7 +1304,7 @@ func TestMetaCache_ConcurrentTest1(t *testing.T) {
defer wg.Done() defer wg.Done()
for i := 0; i < cnt; i++ { for i := 0; i < cnt; i++ {
// periodically invalid collection cache // periodically invalid collection cache
globalMetaCache.RemoveCollection(ctx, dbName, "collection1") globalMetaCache.RemoveCollection(ctx, dbName, "collection1", 0)
time.Sleep(10 * time.Millisecond) time.Sleep(10 * time.Millisecond)
} }
} }
@ -1497,7 +1497,7 @@ func TestMetaCache_RemoveCollection(t *testing.T) {
// shouldn't access RootCoord again // shouldn't access RootCoord again
assert.Equal(t, rootCoord.GetAccessCount(), 1) assert.Equal(t, rootCoord.GetAccessCount(), 1)
globalMetaCache.RemoveCollection(ctx, dbName, "collection1") globalMetaCache.RemoveCollection(ctx, dbName, "collection1", 0)
// no collectionInfo of collection2, should access RootCoord // no collectionInfo of collection2, should access RootCoord
_, err = globalMetaCache.GetCollectionInfo(ctx, dbName, "collection1", 1) _, err = globalMetaCache.GetCollectionInfo(ctx, dbName, "collection1", 1)
assert.NoError(t, err) assert.NoError(t, err)

View File

@ -660,9 +660,9 @@ func (_c *MockCache_HasDatabase_Call) RunAndReturn(run func(context.Context, str
return _c return _c
} }
// RemoveCollection provides a mock function with given fields: ctx, database, collectionName // RemoveCollection provides a mock function with given fields: ctx, database, collectionName, version
func (_m *MockCache) RemoveCollection(ctx context.Context, database string, collectionName string) { func (_m *MockCache) RemoveCollection(ctx context.Context, database string, collectionName string, version uint64) {
_m.Called(ctx, database, collectionName) _m.Called(ctx, database, collectionName, version)
} }
// MockCache_RemoveCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveCollection' // MockCache_RemoveCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveCollection'
@ -674,13 +674,14 @@ type MockCache_RemoveCollection_Call struct {
// - ctx context.Context // - ctx context.Context
// - database string // - database string
// - collectionName string // - collectionName string
func (_e *MockCache_Expecter) RemoveCollection(ctx interface{}, database interface{}, collectionName interface{}) *MockCache_RemoveCollection_Call { // - version uint64
return &MockCache_RemoveCollection_Call{Call: _e.mock.On("RemoveCollection", ctx, database, collectionName)} func (_e *MockCache_Expecter) RemoveCollection(ctx interface{}, database interface{}, collectionName interface{}, version interface{}) *MockCache_RemoveCollection_Call {
return &MockCache_RemoveCollection_Call{Call: _e.mock.On("RemoveCollection", ctx, database, collectionName, version)}
} }
func (_c *MockCache_RemoveCollection_Call) Run(run func(ctx context.Context, database string, collectionName string)) *MockCache_RemoveCollection_Call { func (_c *MockCache_RemoveCollection_Call) Run(run func(ctx context.Context, database string, collectionName string, version uint64)) *MockCache_RemoveCollection_Call {
_c.Call.Run(func(args mock.Arguments) { _c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(string), args[2].(string)) run(args[0].(context.Context), args[1].(string), args[2].(string), args[3].(uint64))
}) })
return _c return _c
} }
@ -690,7 +691,7 @@ func (_c *MockCache_RemoveCollection_Call) Return() *MockCache_RemoveCollection_
return _c return _c
} }
func (_c *MockCache_RemoveCollection_Call) RunAndReturn(run func(context.Context, string, string)) *MockCache_RemoveCollection_Call { func (_c *MockCache_RemoveCollection_Call) RunAndReturn(run func(context.Context, string, string, uint64)) *MockCache_RemoveCollection_Call {
_c.Run(run) _c.Run(run)
return _c return _c
} }

View File

@ -1671,7 +1671,7 @@ func TestHasCollectionTask(t *testing.T) {
task.CollectionName = collectionName task.CollectionName = collectionName
// invalidate collection cache, trigger rootcoord rpc // invalidate collection cache, trigger rootcoord rpc
globalMetaCache.RemoveCollection(ctx, dbName, collectionName) globalMetaCache.RemoveCollection(ctx, dbName, collectionName, 0)
// rc return collection not found error // rc return collection not found error
mixc.describeCollectionFunc = func(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) { mixc.describeCollectionFunc = func(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {

View File

@ -97,7 +97,7 @@ type CacheExpirationsGetter interface {
} }
// ExpireCaches handles the cache // ExpireCaches handles the cache
func (c *DDLCallback) ExpireCaches(ctx context.Context, expirations any, timetick uint64) error { func (c *DDLCallback) ExpireCaches(ctx context.Context, expirations any) error {
var cacheExpirations *message.CacheExpirations var cacheExpirations *message.CacheExpirations
if g, ok := expirations.(CacheExpirationsGetter); ok { if g, ok := expirations.(CacheExpirationsGetter); ok {
cacheExpirations = g.GetCacheExpirations() cacheExpirations = g.GetCacheExpirations()
@ -109,18 +109,29 @@ func (c *DDLCallback) ExpireCaches(ctx context.Context, expirations any, timetic
panic(fmt.Sprintf("invalid getter type: %T", expirations)) panic(fmt.Sprintf("invalid getter type: %T", expirations))
} }
for _, cacheExpiration := range cacheExpirations.CacheExpirations { for _, cacheExpiration := range cacheExpirations.CacheExpirations {
if err := c.expireCache(ctx, cacheExpiration, timetick); err != nil { if err := c.expireCache(ctx, cacheExpiration); err != nil {
return err return err
} }
} }
return nil return nil
} }
func (c *DDLCallback) expireCache(ctx context.Context, cacheExpiration *message.CacheExpiration, timetick uint64) error { func (c *DDLCallback) expireCache(ctx context.Context, cacheExpiration *message.CacheExpiration) error {
ts, err := c.tsoAllocator.GenerateTSO(1)
if err != nil {
return errors.Wrap(err, "failed to generate timestamp")
}
switch cacheExpiration.Cache.(type) { switch cacheExpiration.Cache.(type) {
case *messagespb.CacheExpiration_LegacyProxyCollectionMetaCache: case *messagespb.CacheExpiration_LegacyProxyCollectionMetaCache:
legacyProxyCollectionMetaCache := cacheExpiration.GetLegacyProxyCollectionMetaCache() legacyProxyCollectionMetaCache := cacheExpiration.GetLegacyProxyCollectionMetaCache()
return c.Core.ExpireMetaCache(ctx, legacyProxyCollectionMetaCache.DbName, []string{legacyProxyCollectionMetaCache.CollectionName}, legacyProxyCollectionMetaCache.CollectionId, legacyProxyCollectionMetaCache.PartitionName, timetick, proxyutil.SetMsgType(legacyProxyCollectionMetaCache.MsgType)) return c.Core.ExpireMetaCache(
ctx,
legacyProxyCollectionMetaCache.DbName,
[]string{legacyProxyCollectionMetaCache.CollectionName},
legacyProxyCollectionMetaCache.CollectionId,
legacyProxyCollectionMetaCache.PartitionName,
ts,
proxyutil.SetMsgType(legacyProxyCollectionMetaCache.MsgType))
} }
return nil return nil
} }

View File

@ -112,6 +112,5 @@ func (c *DDLCallback) alterAliasV2AckCallback(ctx context.Context, result messag
ce.NewBuilder().WithLegacyProxyCollectionMetaCache( ce.NewBuilder().WithLegacyProxyCollectionMetaCache(
ce.OptLPCMDBName(result.Message.Header().DbName), ce.OptLPCMDBName(result.Message.Header().DbName),
ce.OptLPCMCollectionName(result.Message.Header().Alias), ce.OptLPCMCollectionName(result.Message.Header().Alias),
ce.OptLPCMMsgType(commonpb.MsgType_AlterAlias)), ce.OptLPCMMsgType(commonpb.MsgType_AlterAlias)))
result.GetControlChannelResult().TimeTick)
} }

View File

@ -296,5 +296,5 @@ func (c *DDLCallback) alterCollectionV2AckCallback(ctx context.Context, result m
if err := c.broker.BroadcastAlteredCollection(ctx, header.CollectionId); err != nil { if err := c.broker.BroadcastAlteredCollection(ctx, header.CollectionId); err != nil {
return errors.Wrap(err, "failed to broadcast altered collection") return errors.Wrap(err, "failed to broadcast altered collection")
} }
return c.ExpireCaches(ctx, header, result.GetControlChannelResult().TimeTick) return c.ExpireCaches(ctx, header)
} }

View File

@ -155,8 +155,7 @@ func (c *DDLCallback) alterDatabaseV1AckCallback(ctx context.Context, result mes
WithLegacyProxyCollectionMetaCache( WithLegacyProxyCollectionMetaCache(
ce.OptLPCMDBName(header.DbName), ce.OptLPCMDBName(header.DbName),
ce.OptLPCMMsgType(commonpb.MsgType_AlterDatabase), ce.OptLPCMMsgType(commonpb.MsgType_AlterDatabase),
), ))
result.GetControlChannelResult().TimeTick)
} }
func MergeProperties(oldProps, updatedProps []*commonpb.KeyValuePair) []*commonpb.KeyValuePair { func MergeProperties(oldProps, updatedProps []*commonpb.KeyValuePair) []*commonpb.KeyValuePair {

View File

@ -117,9 +117,7 @@ func (c *DDLCallback) createCollectionV1AckCallback(ctx context.Context, result
ce.OptLPCMDBName(body.DbName), ce.OptLPCMDBName(body.DbName),
ce.OptLPCMCollectionName(body.CollectionName), ce.OptLPCMCollectionName(body.CollectionName),
ce.OptLPCMCollectionID(header.CollectionId), ce.OptLPCMCollectionID(header.CollectionId),
ce.OptLPCMMsgType(commonpb.MsgType_DropCollection)), ce.OptLPCMMsgType(commonpb.MsgType_DropCollection)))
newCollInfo.UpdateTimestamp,
)
} }
func (c *DDLCallback) createCollectionShard(ctx context.Context, header *message.CreateCollectionMessageHeader, body *message.CreateCollectionRequest, vchannel string, appendResult *message.AppendResult) error { func (c *DDLCallback) createCollectionShard(ctx context.Context, header *message.CreateCollectionMessageHeader, body *message.CreateCollectionRequest, vchannel string, appendResult *message.AppendResult) error {

View File

@ -86,6 +86,5 @@ func (c *DDLCallback) createDatabaseV1AckCallback(ctx context.Context, result me
WithLegacyProxyCollectionMetaCache( WithLegacyProxyCollectionMetaCache(
ce.OptLPCMDBName(header.DbName), ce.OptLPCMDBName(header.DbName),
ce.OptLPCMMsgType(commonpb.MsgType_DropDatabase), ce.OptLPCMMsgType(commonpb.MsgType_DropDatabase),
), ))
result.GetControlChannelResult().TimeTick)
} }

View File

@ -109,6 +109,5 @@ func (c *DDLCallback) createPartitionV1AckCallback(ctx context.Context, result m
ce.OptLPCMCollectionID(header.CollectionId), ce.OptLPCMCollectionID(header.CollectionId),
ce.OptLPCMPartitionName(body.PartitionName), ce.OptLPCMPartitionName(body.PartitionName),
ce.OptLPCMMsgType(commonpb.MsgType_CreatePartition), ce.OptLPCMMsgType(commonpb.MsgType_CreatePartition),
), ))
result.GetControlChannelResult().TimeTick)
} }

View File

@ -68,7 +68,5 @@ func (c *DDLCallback) dropAliasV2AckCallback(ctx context.Context, result message
ce.OptLPCMDBName(result.Message.Header().DbName), ce.OptLPCMDBName(result.Message.Header().DbName),
ce.OptLPCMCollectionName(result.Message.Header().Alias), ce.OptLPCMCollectionName(result.Message.Header().Alias),
ce.OptLPCMMsgType(commonpb.MsgType_DropAlias), ce.OptLPCMMsgType(commonpb.MsgType_DropAlias),
), ))
result.GetControlChannelResult().TimeTick,
)
} }

View File

@ -131,8 +131,7 @@ func (c *DDLCallback) dropCollectionV1AckCallback(ctx context.Context, result me
ce.OptLPCMDBName(body.DbName), ce.OptLPCMDBName(body.DbName),
ce.OptLPCMCollectionName(body.CollectionName), ce.OptLPCMCollectionName(body.CollectionName),
ce.OptLPCMCollectionID(header.CollectionId), ce.OptLPCMCollectionID(header.CollectionId),
ce.OptLPCMMsgType(commonpb.MsgType_DropCollection)).Build(), ce.OptLPCMMsgType(commonpb.MsgType_DropCollection)).Build())
result.GetControlChannelResult().TimeTick)
} }
// newCollectionTombstone creates a new collection tombstone. // newCollectionTombstone creates a new collection tombstone.

View File

@ -68,6 +68,5 @@ func (c *DDLCallback) dropDatabaseV1AckCallback(ctx context.Context, result mess
WithLegacyProxyCollectionMetaCache( WithLegacyProxyCollectionMetaCache(
ce.OptLPCMDBName(header.DbName), ce.OptLPCMDBName(header.DbName),
ce.OptLPCMMsgType(commonpb.MsgType_DropDatabase), ce.OptLPCMMsgType(commonpb.MsgType_DropDatabase),
), ))
result.GetControlChannelResult().TimeTick)
} }

View File

@ -114,8 +114,7 @@ func (c *DDLCallback) dropPartitionV1AckCallback(ctx context.Context, result mes
ce.OptLPCMCollectionID(header.CollectionId), ce.OptLPCMCollectionID(header.CollectionId),
ce.OptLPCMPartitionName(body.PartitionName), ce.OptLPCMPartitionName(body.PartitionName),
ce.OptLPCMMsgType(commonpb.MsgType_DropPartition), ce.OptLPCMMsgType(commonpb.MsgType_DropPartition),
), ))
result.GetControlChannelResult().TimeTick)
} }
// newPartitionTombstone creates a new partition tombstone. // newPartitionTombstone creates a new partition tombstone.

View File

@ -46,6 +46,7 @@ import (
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/channel" "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/channel"
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/broadcast" "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/internal/streamingcoord/server/broadcaster/registry"
mocktso "github.com/milvus-io/milvus/internal/tso/mocks"
kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/log"
@ -82,6 +83,8 @@ func initStreamingSystemAndCore(t *testing.T) *Core {
require.NoError(t, err) require.NoError(t, err)
testDB := newNameDb() testDB := newNameDb()
collID2Meta := make(map[typeutil.UniqueID]*model.Collection) collID2Meta := make(map[typeutil.UniqueID]*model.Collection)
tso := mocktso.NewAllocator(t)
tso.EXPECT().GenerateTSO(mock.Anything).Return(uint64(1), nil).Maybe()
core := newTestCore(withHealthyCode(), core := newTestCore(withHealthyCode(),
withMeta(&MetaTable{ withMeta(&MetaTable{
catalog: rootcoord.NewCatalog(catalogKV, ss), catalog: rootcoord.NewCatalog(catalogKV, ss),
@ -93,6 +96,7 @@ func initStreamingSystemAndCore(t *testing.T) *Core {
withValidMixCoord(), withValidMixCoord(),
withValidProxyManager(), withValidProxyManager(),
withValidIDAllocator(), withValidIDAllocator(),
withTsoAllocator(tso),
withBroker(newValidMockBroker()), withBroker(newValidMockBroker()),
) )
registry.ResetRegistration() registry.ResetRegistration()