mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
fix: remove redundant log (#46695)
issue: #45841 - CPP log make the multi log line in one debug, remove the "\n\t". - remove some log that make no sense. - slow down some log like ChannelDistManager. <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: logging is purely observational — this PR only reduces, consolidates, or reformats diagnostic output (removing per-item/noise logs, consolidating batched logs, and converting multi-line log strings) while preserving all control flow, return values, and state mutations across affected code paths. - Removed / simplified logic: deleted low-value per-operation debug/info logs (e.g., ListIndexes, GetRecoveryInfo, GcConfirm, push-to-reorder-buffer, several streaming/wal/debug traces), replaced per-item inline logs with single batched deferred logs in querynodev2/delegator (logExcludeInfo) and CleanInvalid, changed C++ PlanNode ToString() multi-line output to compact single-line bracketed format (removed "\n\t"), and added thresholded interceptor logging (InterceptorMetrics.ShouldBeLogged) and message-type-driven log levels to avoid verbose entries. - Why this does NOT cause data loss or behavioral regression: no function signatures, branching, state updates, persistence calls, or return values were changed — examples: ListIndexes still returns the same Status/IndexInfos; GcConfirm still constructs and returns resp.GetGcFinished(); Insert and CleanInvalid still perform the same insert/removal operations (only their per-item logging was aggregated); PlanNode ToString changes only affect emitted debug strings. All error handling and control flow paths remain intact. - Enhancement intent: reduce log volume and improve signal-to-noise for debugging by removing redundant, noisy logs and emitting concise, rate-/threshold-limited summaries while preserving necessary diagnostics and original program behavior. <!-- end of auto-generated comment: release notes by coderabbit.ai --> --------- Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
parent
c2677967ad
commit
ca8740c7c0
@ -167,7 +167,7 @@ class FilterBitsNode : public PlanNode {
|
||||
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format("FilterBitsNode:\n\t[filter_expr:{}]",
|
||||
return fmt::format("FilterBitsNode:[filter_expr:{}]",
|
||||
filter_->ToString());
|
||||
}
|
||||
|
||||
@ -228,7 +228,7 @@ class ElementFilterNode : public PlanNode {
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format(
|
||||
"ElementFilterNode:\n\t[struct_name:{}, element_filter:{}]",
|
||||
"ElementFilterNode:[struct_name:{}, element_filter:{}]",
|
||||
struct_name_,
|
||||
element_filter_->ToString());
|
||||
}
|
||||
@ -285,7 +285,7 @@ class ElementFilterBitsNode : public PlanNode {
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format(
|
||||
"ElementFilterBitsNode:\n\t[struct_name:{}, element_filter:{}]",
|
||||
"ElementFilterBitsNode:[struct_name:{}, element_filter:{}]",
|
||||
struct_name_,
|
||||
element_filter_->ToString());
|
||||
}
|
||||
@ -327,7 +327,7 @@ class MvccNode : public PlanNode {
|
||||
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format("MvccNode:\n\t[source node:{}]", SourceToString());
|
||||
return fmt::format("MvccNode:[source_node:{}]", SourceToString());
|
||||
}
|
||||
|
||||
private:
|
||||
@ -360,7 +360,7 @@ class RandomSampleNode : public PlanNode {
|
||||
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format("RandomSampleNode:\n\t[factor:{}]", factor_);
|
||||
return fmt::format("RandomSampleNode:[factor:{}]", factor_);
|
||||
}
|
||||
|
||||
float
|
||||
@ -398,7 +398,7 @@ class VectorSearchNode : public PlanNode {
|
||||
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format("VectorSearchNode:\n\t[source node:{}]",
|
||||
return fmt::format("VectorSearchNode:[source_node:{}]",
|
||||
SourceToString());
|
||||
}
|
||||
|
||||
@ -430,8 +430,7 @@ class GroupByNode : public PlanNode {
|
||||
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format("GroupByNode:\n\t[source node:{}]",
|
||||
SourceToString());
|
||||
return fmt::format("GroupByNode:[source_node:{}]", SourceToString());
|
||||
}
|
||||
|
||||
private:
|
||||
@ -463,7 +462,7 @@ class CountNode : public PlanNode {
|
||||
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format("CountNode:\n\t[source node:{}]", SourceToString());
|
||||
return fmt::format("CountNode:[source_node:{}]", SourceToString());
|
||||
}
|
||||
|
||||
private:
|
||||
@ -510,8 +509,7 @@ class RescoresNode : public PlanNode {
|
||||
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format("RescoresNode:\n\t[source node:{}]",
|
||||
SourceToString());
|
||||
return fmt::format("RescoresNode:[source_node:{}]", SourceToString());
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
@ -1118,7 +1118,6 @@ func (s *Server) ListIndexes(ctx context.Context, req *indexpb.ListIndexesReques
|
||||
UserIndexParams: index.UserIndexParams,
|
||||
}
|
||||
})
|
||||
log.Debug("List index success")
|
||||
return &indexpb.ListIndexesResponse{
|
||||
Status: merr.Success(),
|
||||
IndexInfos: indexInfos,
|
||||
|
||||
@ -453,7 +453,6 @@ func GetSegmentsChanPart(m *meta, collectionID int64, filters ...SegmentFilter)
|
||||
for _, entry := range mDimEntry {
|
||||
result = append(result, entry)
|
||||
}
|
||||
log.Ctx(context.TODO()).Debug("GetSegmentsChanPart", zap.Int("length", len(result)))
|
||||
return result
|
||||
}
|
||||
|
||||
|
||||
@ -836,7 +836,6 @@ func (s *Server) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInf
|
||||
zap.Int64("collectionID", collectionID),
|
||||
zap.Int64("partitionID", partitionID),
|
||||
)
|
||||
log.Info("get recovery info request received")
|
||||
resp := &datapb.GetRecoveryInfoResponse{
|
||||
Status: merr.Success(),
|
||||
}
|
||||
@ -973,7 +972,6 @@ func (s *Server) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryI
|
||||
zap.Int64("collectionID", collectionID),
|
||||
zap.Int64s("partitionIDs", partitionIDs),
|
||||
)
|
||||
log.Info("get recovery info request received")
|
||||
resp := &datapb.GetRecoveryInfoResponseV2{
|
||||
Status: merr.Success(),
|
||||
}
|
||||
|
||||
@ -21,9 +21,9 @@ import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/datacoord/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datacoord/session"
|
||||
globalTask "github.com/milvus-io/milvus/internal/datacoord/task"
|
||||
|
||||
@ -347,16 +347,11 @@ func (m *ChannelDistManager) updateCollectionIndex() {
|
||||
func (m *ChannelDistManager) GetShardLeader(channelName string, replica *Replica) *DmChannel {
|
||||
m.rwmutex.RLock()
|
||||
defer m.rwmutex.RUnlock()
|
||||
logger := log.With(zap.String("Scope", "ChannelDistManager"), zap.String("channelName", channelName),
|
||||
zap.Int64("replicaID", replica.GetID()))
|
||||
var setReason string
|
||||
channels := m.collectionIndex[replica.GetCollectionID()]
|
||||
|
||||
var candidates *DmChannel
|
||||
for chIdx, channel := range channels {
|
||||
logger := logger.With(zap.Int("channelIdx", chIdx))
|
||||
logger.Debug("process", zap.Int64("channelID", channel.Node), zap.Int64("channelVersion", channel.Version),
|
||||
zap.String("channel name", channel.GetChannelName()),
|
||||
zap.Bool("replicaContains", replica.Contains(channel.Node)))
|
||||
for _, channel := range channels {
|
||||
if channel.GetChannelName() == channelName && replica.Contains(channel.Node) {
|
||||
if candidates == nil {
|
||||
candidates = channel
|
||||
@ -370,26 +365,34 @@ func (m *ChannelDistManager) GetShardLeader(channelName string, replica *Replica
|
||||
case !candidatesServiceable && channelServiceable:
|
||||
// Current candidate is not serviceable but new channel is
|
||||
updateNeeded = true
|
||||
logger.Debug("set serviceable delegator to candidate shard leader", zap.Int64("node", channel.Node),
|
||||
zap.Int64("channel version", channel.Version))
|
||||
setReason = "serviceable"
|
||||
case candidatesServiceable == channelServiceable && channel.Version > candidates.Version:
|
||||
// Same service status but higher version
|
||||
updateNeeded = true
|
||||
logger.Debug("set serviceable delegator with larger version to candidate shard leader", zap.Int64("node", channel.Node),
|
||||
zap.Int64("channel version", channel.Version), zap.Int64("candidate version", candidates.Version))
|
||||
setReason = "version_updated"
|
||||
}
|
||||
if updateNeeded {
|
||||
candidates = channel
|
||||
} else {
|
||||
logger.Debug("not set any channel to candidates in this round")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if candidates != nil {
|
||||
logger.Debug("final", zap.Any("candidates", candidates),
|
||||
zap.Int64("candidates version", candidates.Version),
|
||||
zap.Int64("candidates node", candidates.Node))
|
||||
if log.Level().Enabled(zap.DebugLevel) {
|
||||
logger := log.With(
|
||||
zap.String("Scope", "ChannelDistManager"),
|
||||
zap.String("channelName", channelName),
|
||||
zap.Int64("replicaID", replica.GetID()),
|
||||
).WithRateGroup("ChannelDistManager", 1.0, 60.0)
|
||||
if candidates != nil {
|
||||
logger.RatedDebug(1.0, "final",
|
||||
zap.String("candidates", candidates.GetChannelName()),
|
||||
zap.Int64("candidates version", candidates.Version),
|
||||
zap.Int64("candidates node", candidates.Node),
|
||||
zap.String("reason", setReason),
|
||||
)
|
||||
} else {
|
||||
logger.RatedDebug(1.0, "no candidates found")
|
||||
}
|
||||
}
|
||||
return candidates
|
||||
}
|
||||
|
||||
@ -102,8 +102,6 @@ func NewTargetManager(broker Broker, meta *Meta) *TargetManager {
|
||||
func (mgr *TargetManager) UpdateCollectionCurrentTarget(ctx context.Context, collectionID int64) bool {
|
||||
log := log.With(zap.Int64("collectionID", collectionID))
|
||||
|
||||
log.Debug("start to update current target for collection")
|
||||
|
||||
newTarget := mgr.next.getCollectionTarget(collectionID)
|
||||
if newTarget == nil || newTarget.IsEmpty() {
|
||||
log.Info("next target does not exist, skip it")
|
||||
|
||||
@ -17,7 +17,6 @@
|
||||
package delegator
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -42,18 +41,34 @@ func NewExcludedSegments(cleanInterval time.Duration) *ExcludedSegments {
|
||||
}
|
||||
|
||||
func (s *ExcludedSegments) Insert(excludeInfo map[int64]uint64) {
|
||||
if log.Level().Enabled(zap.DebugLevel) {
|
||||
defer func() {
|
||||
s.logExcludeInfo(excludeInfo)
|
||||
}()
|
||||
}
|
||||
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
for segmentID, ts := range excludeInfo {
|
||||
log.Ctx(context.TODO()).Debug("add exclude info",
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.Uint64("ts", ts),
|
||||
)
|
||||
s.segments[segmentID] = ts
|
||||
}
|
||||
}
|
||||
|
||||
func (s *ExcludedSegments) logExcludeInfo(excludeInfo map[int64]uint64) {
|
||||
segmentIDs := make([]int64, 0, len(excludeInfo))
|
||||
timeTicks := make([]uint64, 0, len(excludeInfo))
|
||||
for segmentID, ts := range excludeInfo {
|
||||
segmentIDs = append(segmentIDs, segmentID)
|
||||
timeTicks = append(timeTicks, ts)
|
||||
}
|
||||
log.Debug("add exclude info",
|
||||
zap.Int("count", len(excludeInfo)),
|
||||
zap.Int64s("segmentIDs", segmentIDs),
|
||||
zap.Uint64s("timeTicks", timeTicks),
|
||||
)
|
||||
}
|
||||
|
||||
// return false if segment has been excluded
|
||||
func (s *ExcludedSegments) Verify(segmentID int64, ts uint64) bool {
|
||||
s.mu.RLock()
|
||||
@ -65,6 +80,17 @@ func (s *ExcludedSegments) Verify(segmentID int64, ts uint64) bool {
|
||||
}
|
||||
|
||||
func (s *ExcludedSegments) CleanInvalid(ts uint64) {
|
||||
removedSegmentIDs := make([]int64, 0, 32)
|
||||
if log.Level().Enabled(zap.DebugLevel) {
|
||||
defer func() {
|
||||
log.Debug("remove segment from exclude info",
|
||||
zap.Int("count", len(removedSegmentIDs)),
|
||||
zap.Uint64("ts", ts),
|
||||
zap.Int64s("segmentIDs", removedSegmentIDs),
|
||||
)
|
||||
}()
|
||||
}
|
||||
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
@ -77,7 +103,7 @@ func (s *ExcludedSegments) CleanInvalid(ts uint64) {
|
||||
|
||||
for _, segmentID := range invalidExcludedInfos {
|
||||
delete(s.segments, segmentID)
|
||||
log.Ctx(context.TODO()).Debug("remove segment from exclude info", zap.Int64("segmentID", segmentID))
|
||||
removedSegmentIDs = append(removedSegmentIDs, segmentID)
|
||||
}
|
||||
s.lastClean.Store(time.Now())
|
||||
}
|
||||
|
||||
@ -261,22 +261,12 @@ func (c *Collection) GetLoadType() querypb.LoadType {
|
||||
|
||||
func (c *Collection) Ref(count uint32) uint32 {
|
||||
refCount := c.refCount.Add(count)
|
||||
log.Debug("collection ref increment",
|
||||
zap.Int64("nodeID", paramtable.GetNodeID()),
|
||||
zap.Int64("collectionID", c.ID()),
|
||||
zap.Uint32("refCount", refCount),
|
||||
)
|
||||
putOrUpdateStorageContext(c.Schema().GetProperties(), c.ID())
|
||||
return refCount
|
||||
}
|
||||
|
||||
func (c *Collection) Unref(count uint32) uint32 {
|
||||
refCount := c.refCount.Sub(count)
|
||||
log.Debug("collection ref decrement",
|
||||
zap.Int64("nodeID", paramtable.GetNodeID()),
|
||||
zap.Int64("collectionID", c.ID()),
|
||||
zap.Uint32("refCount", refCount),
|
||||
)
|
||||
return refCount
|
||||
}
|
||||
|
||||
|
||||
@ -19,9 +19,6 @@ package segments
|
||||
import (
|
||||
"context"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
)
|
||||
|
||||
@ -31,8 +28,6 @@ func validate(ctx context.Context, manager *Manager, collectionID int64, partiti
|
||||
return nil, merr.WrapErrCollectionNotFound(collectionID)
|
||||
}
|
||||
|
||||
log.Ctx(ctx).Debug("read target partitions", zap.Int64("collectionID", collectionID), zap.Int64s("partitionIDs", partitionIDs))
|
||||
|
||||
// validate segment
|
||||
segments := make([]Segment, 0, len(segmentIDs))
|
||||
var err error
|
||||
|
||||
@ -285,8 +285,6 @@ func (b *ServerBroker) BroadcastAlteredCollection(ctx context.Context, collectio
|
||||
func (b *ServerBroker) GcConfirm(ctx context.Context, collectionID, partitionID UniqueID) bool {
|
||||
log := log.Ctx(ctx).With(zap.Int64("collection", collectionID), zap.Int64("partition", partitionID))
|
||||
|
||||
log.Info("confirming if gc is finished")
|
||||
|
||||
req := &datapb.GcConfirmRequest{CollectionId: collectionID, PartitionId: partitionID}
|
||||
resp, err := b.s.mixCoord.GcConfirm(ctx, req)
|
||||
if err != nil {
|
||||
@ -300,6 +298,5 @@ func (b *ServerBroker) GcConfirm(ctx context.Context, collectionID, partitionID
|
||||
return false
|
||||
}
|
||||
|
||||
log.Info("received gc_confirm response", zap.Bool("finished", resp.GetGcFinished()))
|
||||
return resp.GetGcFinished()
|
||||
}
|
||||
|
||||
@ -702,7 +702,6 @@ func (mt *MetaTable) getLatestCollectionByIDInternal(ctx context.Context, collec
|
||||
return coll.Clone(), nil
|
||||
}
|
||||
if !coll.Available() {
|
||||
log.Warn("collection not available", zap.Int64("collectionID", collectionID), zap.Any("state", coll.State))
|
||||
return nil, merr.WrapErrCollectionNotFound(collectionID)
|
||||
}
|
||||
return filterUnavailable(coll), nil
|
||||
|
||||
@ -1135,7 +1135,9 @@ func (c *Core) describeCollectionImpl(ctx context.Context, in *milvuspb.Describe
|
||||
}
|
||||
|
||||
if err := t.WaitToFinish(); err != nil {
|
||||
log.Warn("failed to describe collection", zap.Error(err))
|
||||
if !errors.Is(err, merr.ErrCollectionNotFound) {
|
||||
log.Warn("failed to describe collection", zap.Error(err))
|
||||
}
|
||||
metrics.RootCoordDDLReqCounter.WithLabelValues("DescribeCollection", metrics.FailLabel).Inc()
|
||||
return &milvuspb.DescribeCollectionResponse{
|
||||
Status: merr.Status(err),
|
||||
|
||||
@ -291,7 +291,11 @@ func (s *scannerAdaptorImpl) handleUpstream(msg message.ImmutableMessage) {
|
||||
// Push the confirmed messages into pending queue for consuming.
|
||||
if s.logger.Level().Enabled(zap.DebugLevel) {
|
||||
for _, m := range msgs {
|
||||
s.logger.Debug("push committed message into pending queue", zap.Uint64("committedTimeTick", msg.TimeTick()), log.FieldMessage(m))
|
||||
s.logger.Debug(
|
||||
"push message into pending queue",
|
||||
zap.Uint64("committedTimeTick", msg.TimeTick()),
|
||||
log.FieldMessage(m),
|
||||
)
|
||||
}
|
||||
}
|
||||
s.pendingQueue.Add(msgs)
|
||||
@ -328,10 +332,4 @@ func (s *scannerAdaptorImpl) handleUpstream(msg message.ImmutableMessage) {
|
||||
// Observe the filtered message.
|
||||
s.metrics.UpdateTimeTickBufSize(s.reorderBuffer.Bytes())
|
||||
s.metrics.ObservePassedMessage(isTailing, msg.MessageType(), msg.EstimateSize())
|
||||
if s.logger.Level().Enabled(zap.DebugLevel) {
|
||||
// Log the message if the log level is debug.
|
||||
s.logger.Debug("push message into reorder buffer",
|
||||
log.FieldMessage(msg),
|
||||
zap.Bool("tailing", isTailing))
|
||||
}
|
||||
}
|
||||
|
||||
@ -172,9 +172,6 @@ func (impl *shardInterceptor) handleInsertMessage(ctx context.Context, msg messa
|
||||
// 2. partition is fenced.
|
||||
// 3. segment is not ready.
|
||||
// we just redo it to refresh a new latest timetick.
|
||||
if impl.shardManager.Logger().Level().Enabled(zap.DebugLevel) {
|
||||
impl.shardManager.Logger().Debug("segment assign interceptor redo insert message", zap.Object("message", msg), zap.Error(err))
|
||||
}
|
||||
return nil, redo.ErrRedo
|
||||
}
|
||||
if errors.IsAny(err, shards.ErrTooLargeInsert, shards.ErrPartitionNotFound, shards.ErrCollectionNotFound) {
|
||||
|
||||
@ -11,7 +11,10 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
)
|
||||
|
||||
const maxRedoLogged = 3
|
||||
const (
|
||||
maxLogged = 3
|
||||
logThreshold = 10 * time.Millisecond
|
||||
)
|
||||
|
||||
type InterceptorMetrics struct {
|
||||
Before time.Duration
|
||||
@ -19,8 +22,12 @@ type InterceptorMetrics struct {
|
||||
After time.Duration
|
||||
}
|
||||
|
||||
func (im *InterceptorMetrics) ShouldBeLogged() bool {
|
||||
return im.Before > logThreshold || im.After > logThreshold || im.BeforeErr != nil
|
||||
}
|
||||
|
||||
func (im *InterceptorMetrics) String() string {
|
||||
return fmt.Sprintf("before: %s, after: %s, before_err: %v", im.Before, im.After, im.BeforeErr)
|
||||
return fmt.Sprintf("b:%s,a:%s,err:%s", im.Before, im.After, im.BeforeErr)
|
||||
}
|
||||
|
||||
// AppendMetrics is the metrics for append operation.
|
||||
@ -67,16 +74,10 @@ func (m *AppendMetrics) StartAppendGuard() *AppendMetricsGuard {
|
||||
func (m *AppendMetrics) IntoLogFields() []zap.Field {
|
||||
fields := []zap.Field{
|
||||
log.FieldMessage(m.msg),
|
||||
zap.Duration("appendDuration", m.appendDuration),
|
||||
zap.Duration("implAppendDuration", m.implAppendDuration),
|
||||
}
|
||||
for name, ims := range m.interceptors {
|
||||
for i, im := range ims {
|
||||
if i <= maxRedoLogged {
|
||||
fields = append(fields, zap.Any(fmt.Sprintf("%s_%d", name, i), im))
|
||||
}
|
||||
}
|
||||
zap.Duration("duration", m.appendDuration),
|
||||
zap.Duration("implDuration", m.implAppendDuration),
|
||||
}
|
||||
|
||||
if m.err != nil {
|
||||
fields = append(fields, zap.Error(m.err))
|
||||
} else {
|
||||
@ -87,6 +88,22 @@ func (m *AppendMetrics) IntoLogFields() []zap.Field {
|
||||
fields = append(fields, zap.Int64("txnID", int64(m.result.TxnCtx.TxnID)))
|
||||
}
|
||||
}
|
||||
loggedInterceptorCount := 0
|
||||
L:
|
||||
for name, ims := range m.interceptors {
|
||||
for i, im := range ims {
|
||||
if !im.ShouldBeLogged() {
|
||||
continue
|
||||
}
|
||||
if loggedInterceptorCount <= maxLogged {
|
||||
fields = append(fields, zap.Stringer(fmt.Sprintf("%s_%d", name, i), im))
|
||||
loggedInterceptorCount++
|
||||
}
|
||||
if loggedInterceptorCount >= maxLogged {
|
||||
break L
|
||||
}
|
||||
}
|
||||
}
|
||||
return fields
|
||||
}
|
||||
|
||||
|
||||
@ -5,7 +5,6 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"go.uber.org/zap/zapcore"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
@ -104,8 +103,9 @@ func (m *WriteMetrics) done(appendMetrics *AppendMetrics) {
|
||||
m.Logger().Warn("append message into wal too slow", appendMetrics.IntoLogFields()...)
|
||||
return
|
||||
}
|
||||
if m.Logger().Level().Enabled(zapcore.DebugLevel) {
|
||||
m.Logger().Debug("append message into wal", appendMetrics.IntoLogFields()...)
|
||||
logLV := appendMetrics.msg.MessageType().LogLevel()
|
||||
if m.Logger().Level().Enabled(logLV) {
|
||||
m.Logger().Log(logLV, "append message into wal", appendMetrics.IntoLogFields()...)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -351,9 +351,6 @@ func (r *recoveryStorageImpl) handleInsert(msg message.ImmutableInsertMessageV1)
|
||||
for _, partition := range msg.Header().GetPartitions() {
|
||||
if segment, ok := r.segments[partition.SegmentAssignment.SegmentId]; ok && segment.IsGrowing() {
|
||||
segment.ObserveInsert(msg.TimeTick(), partition)
|
||||
if r.Logger().Level().Enabled(zap.DebugLevel) {
|
||||
r.Logger().Debug("insert entity", log.FieldMessage(msg), zap.Uint64("segmentRows", segment.Rows()), zap.Uint64("segmentBinary", segment.BinarySize()))
|
||||
}
|
||||
} else {
|
||||
r.detectInconsistency(msg, "segment not found")
|
||||
}
|
||||
@ -362,10 +359,6 @@ func (r *recoveryStorageImpl) handleInsert(msg message.ImmutableInsertMessageV1)
|
||||
|
||||
// handleDelete handles the delete message.
|
||||
func (r *recoveryStorageImpl) handleDelete(msg message.ImmutableDeleteMessageV1) {
|
||||
// nothing, current delete operation is managed by flowgraph, not recovery storage.
|
||||
if r.Logger().Level().Enabled(zap.DebugLevel) {
|
||||
r.Logger().Debug("delete entity", log.FieldMessage(msg))
|
||||
}
|
||||
}
|
||||
|
||||
// handleCreateSegment handles the create segment message.
|
||||
|
||||
@ -79,6 +79,7 @@ func (m *immutableTxnMessageImpl) MarshalLogObject(enc zapcore.ObjectEncoder) er
|
||||
if m == nil {
|
||||
return nil
|
||||
}
|
||||
enc.AddInt("messageCount", m.Size())
|
||||
enc.AddArray("txn", zapcore.ArrayMarshalerFunc(func(enc zapcore.ArrayEncoder) error {
|
||||
txnMessage := AsImmutableTxnMessage(m)
|
||||
txnMessage.RangeOver(func(im ImmutableMessage) error {
|
||||
|
||||
@ -3,11 +3,118 @@ package message
|
||||
import (
|
||||
"strconv"
|
||||
|
||||
"go.uber.org/zap/zapcore"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
)
|
||||
|
||||
type MessageType messagespb.MessageType
|
||||
|
||||
type MessageTypeProperties struct {
|
||||
// The log level of the message, only used for logging, info by default.
|
||||
LogLevel zapcore.Level
|
||||
// A system message type is used to determine the commit behavior, used to achieve the consistency (timetick confirmed, txn) of the message.
|
||||
IsSystem bool
|
||||
// A self controlled message type is generated by the wal-system itself, not by the client.
|
||||
SelfControlled bool
|
||||
// An exclusive required message type should be appended exclusively at related vchannel(pchannel),
|
||||
ExclusiveRequired bool
|
||||
// a cipher enabled message type will be encrypted before appending to the wal if cipher is enabled.
|
||||
CipherEnabled bool
|
||||
}
|
||||
|
||||
var messageTypePropertiesMap = map[MessageType]MessageTypeProperties{
|
||||
MessageTypeTimeTick: {
|
||||
LogLevel: zapcore.DebugLevel,
|
||||
IsSystem: true,
|
||||
SelfControlled: true,
|
||||
},
|
||||
MessageTypeInsert: {
|
||||
LogLevel: zapcore.DebugLevel,
|
||||
CipherEnabled: true,
|
||||
},
|
||||
MessageTypeDelete: {
|
||||
LogLevel: zapcore.DebugLevel,
|
||||
CipherEnabled: true,
|
||||
},
|
||||
MessageTypeCreateCollection: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeDropCollection: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeTruncateCollection: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeCreatePartition: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeDropPartition: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeImport: {},
|
||||
MessageTypeCreateSegment: {
|
||||
SelfControlled: true,
|
||||
},
|
||||
MessageTypeFlush: {
|
||||
SelfControlled: true,
|
||||
},
|
||||
MessageTypeManualFlush: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeAlterReplicateConfig: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeBeginTxn: {
|
||||
LogLevel: zapcore.DebugLevel,
|
||||
IsSystem: true,
|
||||
},
|
||||
MessageTypeCommitTxn: {
|
||||
LogLevel: zapcore.DebugLevel,
|
||||
IsSystem: true,
|
||||
},
|
||||
MessageTypeRollbackTxn: {
|
||||
LogLevel: zapcore.DebugLevel,
|
||||
IsSystem: true,
|
||||
},
|
||||
MessageTypeTxn: {
|
||||
LogLevel: zapcore.DebugLevel,
|
||||
IsSystem: true,
|
||||
},
|
||||
MessageTypeSchemaChange: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeAlterCollection: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
MessageTypeAlterLoadConfig: {},
|
||||
MessageTypeDropLoadConfig: {},
|
||||
MessageTypeCreateDatabase: {},
|
||||
MessageTypeAlterDatabase: {},
|
||||
MessageTypeDropDatabase: {},
|
||||
MessageTypeAlterAlias: {},
|
||||
MessageTypeDropAlias: {},
|
||||
MessageTypeAlterUser: {},
|
||||
MessageTypeDropUser: {},
|
||||
MessageTypeAlterRole: {},
|
||||
MessageTypeDropRole: {},
|
||||
MessageTypeAlterUserRole: {},
|
||||
MessageTypeDropUserRole: {},
|
||||
MessageTypeAlterPrivilege: {},
|
||||
MessageTypeDropPrivilege: {},
|
||||
MessageTypeAlterPrivilegeGroup: {},
|
||||
MessageTypeDropPrivilegeGroup: {},
|
||||
MessageTypeRestoreRBAC: {},
|
||||
MessageTypeAlterResourceGroup: {},
|
||||
MessageTypeDropResourceGroup: {},
|
||||
MessageTypeCreateIndex: {},
|
||||
MessageTypeAlterIndex: {},
|
||||
MessageTypeDropIndex: {},
|
||||
MessageTypeFlushAll: {
|
||||
ExclusiveRequired: true,
|
||||
},
|
||||
}
|
||||
|
||||
// String implements fmt.Stringer interface.
|
||||
func (t MessageType) String() string {
|
||||
return messagespb.MessageType_name[int32(t)]
|
||||
@ -29,26 +136,27 @@ func (t MessageType) Valid() bool {
|
||||
// An exclusive required message type is that the message's timetick should keep same order with message id.
|
||||
// And when the message is appending, other messages with the same vchannel cannot append concurrently.
|
||||
func (t MessageType) IsExclusiveRequired() bool {
|
||||
_, ok := exclusiveRequiredMessageType[t]
|
||||
return ok
|
||||
return messageTypePropertiesMap[t].ExclusiveRequired
|
||||
}
|
||||
|
||||
// CanEnableCipher checks if the MessageType can enable cipher.
|
||||
func (t MessageType) CanEnableCipher() bool {
|
||||
_, ok := cipherMessageType[t]
|
||||
return ok
|
||||
return messageTypePropertiesMap[t].CipherEnabled
|
||||
}
|
||||
|
||||
// IsSysmtem checks if the MessageType is a system type.
|
||||
func (t MessageType) IsSystem() bool {
|
||||
_, ok := systemMessageType[t]
|
||||
return ok
|
||||
return messageTypePropertiesMap[t].IsSystem
|
||||
}
|
||||
|
||||
// IsSelfControlled checks if the MessageType is self controlled.
|
||||
func (t MessageType) IsSelfControlled() bool {
|
||||
_, ok := selfControlledMessageType[t]
|
||||
return ok
|
||||
return messageTypePropertiesMap[t].SelfControlled
|
||||
}
|
||||
|
||||
// LogLevel returns the log level of the MessageType.
|
||||
func (t MessageType) LogLevel() zapcore.Level {
|
||||
return messageTypePropertiesMap[t].LogLevel
|
||||
}
|
||||
|
||||
// unmarshalMessageType unmarshal MessageType from string.
|
||||
|
||||
@ -1921,6 +1921,7 @@ type (
|
||||
ImmutableFlushAllMessageV2 = SpecializedImmutableMessage[*FlushAllMessageHeader, *FlushAllMessageBody]
|
||||
BroadcastFlushAllMessageV2 = SpecializedBroadcastMessage[*FlushAllMessageHeader, *FlushAllMessageBody]
|
||||
BroadcastResultFlushAllMessageV2 = BroadcastResult[*FlushAllMessageHeader, *FlushAllMessageBody]
|
||||
AckResultFlushAllMessageV2 = AckResult[*FlushAllMessageHeader, *FlushAllMessageBody]
|
||||
)
|
||||
|
||||
// MessageTypeWithVersion for FlushAllMessageV2
|
||||
|
||||
@ -8,39 +8,6 @@ import (
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
// A system preserved message, should not allowed to provide outside of the streaming system.
|
||||
var systemMessageType = map[MessageType]struct{}{
|
||||
MessageTypeTimeTick: {},
|
||||
MessageTypeBeginTxn: {},
|
||||
MessageTypeCommitTxn: {},
|
||||
MessageTypeRollbackTxn: {},
|
||||
MessageTypeTxn: {},
|
||||
}
|
||||
|
||||
var selfControlledMessageType = map[MessageType]struct{}{
|
||||
MessageTypeTimeTick: {},
|
||||
MessageTypeCreateSegment: {},
|
||||
MessageTypeFlush: {},
|
||||
}
|
||||
|
||||
var cipherMessageType = map[MessageType]struct{}{
|
||||
MessageTypeInsert: {},
|
||||
MessageTypeDelete: {},
|
||||
}
|
||||
|
||||
var exclusiveRequiredMessageType = map[MessageType]struct{}{
|
||||
MessageTypeCreateCollection: {},
|
||||
MessageTypeDropCollection: {},
|
||||
MessageTypeCreatePartition: {},
|
||||
MessageTypeDropPartition: {},
|
||||
MessageTypeManualFlush: {},
|
||||
MessageTypeFlushAll: {},
|
||||
MessageTypeSchemaChange: {},
|
||||
MessageTypeAlterReplicateConfig: {},
|
||||
MessageTypeAlterCollection: {},
|
||||
MessageTypeTruncateCollection: {},
|
||||
}
|
||||
|
||||
// mustAsSpecializedMutableMessage converts a MutableMessage to a specialized MutableMessage.
|
||||
// It will panic if the message is not the target specialized message or failed to decode the specialized header.
|
||||
func mustAsSpecializedMutableMessage[H proto.Message, B proto.Message](msg BasicMessage) specializedMutableMessage[H, B] {
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user