enhance: refine access log to support format access log by yaml and print name info. (#28319)

relate: https://github.com/milvus-io/milvus/issues/28086

---------

Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
This commit is contained in:
aoiasd 2023-11-28 15:32:31 +08:00 committed by GitHub
parent 1b7a503f89
commit 89d8ce2f73
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
30 changed files with 1019 additions and 407 deletions

View File

@ -200,10 +200,32 @@ proxy:
ginLogging: true
maxTaskNum: 1024 # max task number of proxy task queue
accessLog:
enable: false
filename: "" # Log filename, leave empty to use stdout.
enable: true
# Log filename, set as "" to use stdout.
filename: ""
# define formatters for access log by names:{format: XXX, method:[XXX]}
formatters:
# "base" formatter could not set method
# all method will use "base" formatter default
base:
# will not print access log if set as ""
format: "[$time_now] [ACCESS] <$user_name: $user_addr> $method_name-$method_status-$error_code [traceID: $trace_id] [timeCost: $time_cost]"
query:
format: "[$time_now] [ACCESS] <$user_name: $user_addr> $method_status-$method_name [traceID: $trace_id] [timeCost: $time_cost] [database: $database_name] [collection: $collection_name] [partitions: $partition_name] [expr: $method_expr]"
# set formatter owners by method name(method was all milvus external interface)
# all method will use base formatter default
# one method only could use one formatter
# if set a method formatter mutiple times, will use random fomatter.
methods: ["Query", "Search"]
# localPath: /tmp/milvus_accesslog // log file rootpath
# maxSize: 64 # max log file size of singal log file to trigger rotate.
# maxSize: 64 # max log file size(MB) of singal log file, mean close when time <= 0.
# rotatedTime: 0 # max time range of singal log file, mean close when time <= 0;
# maxBackups: 8 # num of reserved backups. will rotate and crate a new backup when access log file trigger maxSize or rotatedTime.
# cacheSize: 10240 # write cache of accesslog in Byte
# minioEnable: false # update backups to milvus minio when minioEnable is true.
# remotePath: "access_log/" # file path when update backups to minio
# remoteMaxTime: 0 # max time range(in Hour) of backups in minio, 0 means close time retention.
http:
enabled: true # Whether to enable the http server
debug_mode: false # Whether to enable http server debug mode

View File

@ -241,6 +241,7 @@ func (s *Server) startExternalGrpc(grpcPort int, errChan chan error) {
var unaryServerOption grpc.ServerOption
if enableCustomInterceptor {
unaryServerOption = grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
accesslog.UnaryAccessLogInterceptor,
otelgrpc.UnaryServerInterceptor(opts...),
grpc_auth.UnaryServerInterceptor(proxy.AuthenticationInterceptor),
proxy.DatabaseInterceptor(),
@ -248,7 +249,7 @@ func (s *Server) startExternalGrpc(grpcPort int, errChan chan error) {
proxy.UnaryServerInterceptor(proxy.PrivilegeInterceptor),
logutil.UnaryTraceLoggerInterceptor,
proxy.RateLimitInterceptor(limiter),
accesslog.UnaryAccessLoggerInterceptor,
accesslog.UnaryUpdateAccessInfoInterceptor,
proxy.TraceLogInterceptor,
proxy.KeepActiveInterceptor,
))

View File

@ -1,164 +0,0 @@
// 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 accesslog
import (
"context"
"fmt"
"path"
"sync"
"sync/atomic"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
const (
clientRequestIDKey = "client_request_id"
)
var (
_globalL, _globalW atomic.Value
once sync.Once
)
func A() *zap.Logger {
return _globalL.Load().(*zap.Logger)
}
func W() *RotateLogger {
return _globalW.Load().(*RotateLogger)
}
func SetupAccseeLog(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) {
once.Do(func() {
_, err := InitAccessLogger(logCfg, minioCfg)
if err != nil {
log.Fatal("initialize access logger error", zap.Error(err))
}
})
}
// InitAccessLogger initializes a zap access logger for proxy
func InitAccessLogger(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) (*RotateLogger, error) {
var lg *RotateLogger
var err error
if !logCfg.Enable.GetAsBool() {
return nil, nil
}
var writeSyncer zapcore.WriteSyncer
if len(logCfg.Filename.GetValue()) > 0 {
lg, err = NewRotateLogger(logCfg, minioCfg)
if err != nil {
return nil, err
}
if logCfg.CacheSize.GetAsInt() > 0 {
blg := NewCacheLogger(lg, logCfg.CacheSize.GetAsInt())
writeSyncer = zapcore.AddSync(blg)
} else {
writeSyncer = zapcore.AddSync(lg)
}
} else {
stdout, _, err := zap.Open([]string{"stdout"}...)
if err != nil {
return nil, err
}
writeSyncer = stdout
}
encoder := NewAccessEncoder()
logger := zap.New(zapcore.NewCore(encoder, writeSyncer, zapcore.DebugLevel))
logger.Info("Access log start successful")
_globalL.Store(logger)
_globalW.Store(lg)
return lg, nil
}
func NewAccessEncoder() zapcore.Encoder {
encoderConfig := zapcore.EncoderConfig{
TimeKey: "ts",
NameKey: "logger",
FunctionKey: zapcore.OmitKey,
MessageKey: "msg",
StacktraceKey: "stacktrace",
LineEnding: zapcore.DefaultLineEnding,
EncodeTime: log.DefaultTimeEncoder,
EncodeDuration: zapcore.SecondsDurationEncoder,
}
return log.NewTextEncoder(&encoderConfig, false, false)
}
func PrintAccessInfo(ctx context.Context, resp interface{}, err error, rpcInfo *grpc.UnaryServerInfo, timeCost int64) bool {
if _globalL.Load() == nil {
return false
}
fields := []zap.Field{
// format time cost of task
zap.String("timeCost", fmt.Sprintf("%d ms", timeCost)),
}
// get trace ID of task
traceID, ok := getTraceID(ctx)
if !ok {
log.Warn("access log print failed: could not get trace ID")
return false
}
fields = append(fields, zap.String("traceId", traceID))
// get response size of task
responseSize, ok := getResponseSize(resp)
if !ok {
log.Warn("access log print failed: could not get response size")
return false
}
fields = append(fields, zap.Int("responseSize", responseSize))
// get err code of task
errCode, ok := getErrCode(resp)
if !ok {
// unknown error code
errCode = -1
}
fields = append(fields, zap.Int("errorCode", errCode))
// get status of grpc
Status := getGrpcStatus(err)
if Status == "OK" && errCode > 0 {
Status = "TaskFailed"
}
// get method name of grpc
_, methodName := path.Split(rpcInfo.FullMethod)
A().Info(fmt.Sprintf("%v: %s-%s", Status, getAccessAddr(ctx), methodName), fields...)
return true
}
func Rotate() error {
err := W().Rotate()
return err
}

View File

@ -0,0 +1,192 @@
// 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 accesslog
import (
"context"
"net"
"strings"
"testing"
"github.com/stretchr/testify/suite"
"go.opentelemetry.io/otel"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/peer"
"google.golang.org/grpc/status"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util"
"github.com/milvus-io/milvus/pkg/util/crypto"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type LogFormatterSuite struct {
suite.Suite
serverinfo *grpc.UnaryServerInfo
reqs []interface{}
resps []interface{}
errs []error
username string
traceID string
ctx context.Context
}
func (s *LogFormatterSuite) SetupSuite() {
s.username = "test-user"
s.ctx = peer.NewContext(
context.Background(),
&peer.Peer{
Addr: &net.IPAddr{
IP: net.IPv4(0, 0, 0, 0),
Zone: "test",
},
})
md := metadata.Pairs(util.HeaderAuthorize, crypto.Base64Encode("mockUser:mockPass"))
s.ctx = metadata.NewIncomingContext(s.ctx, md)
s.traceID = "test-trace"
s.serverinfo = &grpc.UnaryServerInfo{
FullMethod: "test",
}
s.reqs = []interface{}{
&milvuspb.QueryRequest{
DbName: "test-db",
CollectionName: "test-collection",
PartitionNames: []string{"test-partition-1", "test-partition-2"},
},
&milvuspb.SearchRequest{
DbName: "test-db",
CollectionName: "test-collection",
PartitionNames: []string{"test-partition-1", "test-partition-2"},
},
&milvuspb.InsertRequest{
DbName: "test-db",
CollectionName: "test-collection",
PartitionName: "test-partition-1",
},
}
s.resps = []interface{}{
&milvuspb.QueryResults{
Status: merr.Status(nil),
},
&milvuspb.MutationResult{
Status: merr.Status(merr.ErrCollectionNotFound),
},
merr.Status(nil),
}
s.errs = []error{nil, nil, status.Errorf(codes.Unavailable, "")}
}
func (s *LogFormatterSuite) TestFormatNames() {
fmt := "{$database_name}: $collection_name: $partition_name"
formatter := NewFormatter(fmt)
for _, req := range s.reqs {
info := NewGrpcAccessInfo(s.ctx, s.serverinfo, req)
fs := formatter.Format(info)
s.False(strings.Contains(fs, unknownString))
}
info := NewGrpcAccessInfo(s.ctx, s.serverinfo, nil)
fs := formatter.Format(info)
s.True(strings.Contains(fs, unknownString))
}
func (s *LogFormatterSuite) TestFormatTime() {
fmt := "$time_now: $time_start: $time_end: $time_cost: $time_now"
formatter := NewFormatter(fmt)
for id, req := range s.reqs {
info := NewGrpcAccessInfo(s.ctx, s.serverinfo, req)
fs := formatter.Format(info)
s.True(strings.Contains(fs, unknownString))
info.UpdateCtx(s.ctx)
info.SetResult(s.resps[id], s.errs[id])
fs = formatter.Format(info)
s.False(strings.Contains(fs, unknownString))
}
}
func (s *LogFormatterSuite) TestFormatUserInfo() {
fmt := "$user_name: $user_addr"
formatter := NewFormatter(fmt)
for _, req := range s.reqs {
info := NewGrpcAccessInfo(s.ctx, s.serverinfo, req)
fs := formatter.Format(info)
s.False(strings.Contains(fs, unknownString))
}
// test unknown
info := NewGrpcAccessInfo(context.Background(), &grpc.UnaryServerInfo{}, nil)
fs := formatter.Format(info)
s.True(strings.Contains(fs, unknownString))
}
func (s *LogFormatterSuite) TestFormatMethodInfo() {
fmt := "$method_name: $method_status $trace_id"
formatter := NewFormatter(fmt)
metaContext := metadata.AppendToOutgoingContext(s.ctx, clientRequestIDKey, s.traceID)
for _, req := range s.reqs {
info := NewGrpcAccessInfo(metaContext, s.serverinfo, req)
fs := formatter.Format(info)
log.Info(fs)
s.True(strings.Contains(fs, s.traceID))
}
traceContext, traceSpan := otel.Tracer(typeutil.ProxyRole).Start(s.ctx, "test")
trueTraceID := traceSpan.SpanContext().TraceID().String()
for _, req := range s.reqs {
info := NewGrpcAccessInfo(traceContext, s.serverinfo, req)
fs := formatter.Format(info)
log.Info(fs)
s.True(strings.Contains(fs, trueTraceID))
}
}
func (s *LogFormatterSuite) TestFormatMethodResult() {
fmt := "$method_name: $method_status $response_size $error_code $error_msg"
formatter := NewFormatter(fmt)
for id, req := range s.reqs {
info := NewGrpcAccessInfo(s.ctx, s.serverinfo, req)
fs := formatter.Format(info)
s.True(strings.Contains(fs, unknownString))
info.SetResult(s.resps[id], s.errs[id])
fs = formatter.Format(info)
s.False(strings.Contains(fs, unknownString))
}
}
func (s *LogFormatterSuite) TestParseConfigKeyFailed() {
configKey := ".testf.invalidSub"
_, _, err := parseConfigKey(configKey)
s.Error(err)
}
func TestLogFormatter(t *testing.T) {
suite.Run(t, new(LogFormatterSuite))
}

View File

@ -0,0 +1,154 @@
// 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 accesslog
import (
"strings"
"github.com/milvus-io/milvus/pkg/util/merr"
)
const (
unknownString = "Unknown"
fomaterkey = "format"
methodKey = "methods"
)
type getMetricFunc func(i *GrpcAccessInfo) string
// supported metrics
var metricFuncMap = map[string]getMetricFunc{
"$method_name": getMethodName,
"$method_status": getMethodStatus,
"$trace_id": getTraceID,
"$user_addr": getAddr,
"$user_name": getUserName,
"$response_size": getResponseSize,
"$error_code": getErrorCode,
"$error_msg": getErrorMsg,
"$database_name": getDbName,
"$collection_name": getCollectionName,
"$partition_name": getPartitionName,
"$time_cost": getTimeCost,
"$time_now": getTimeNow,
"$time_start": getTimeStart,
"$time_end": getTimeEnd,
"$method_expr": getExpr,
}
var BaseFormatterKey = "base"
// Formaater manager not concurrent safe
// make sure init with Add and SetMethod before use Get
type FormatterManger struct {
formatters map[string]*Formatter
methodMap map[string]string
}
func NewFormatterManger() *FormatterManger {
return &FormatterManger{
formatters: make(map[string]*Formatter),
methodMap: make(map[string]string),
}
}
func (m *FormatterManger) Add(name, fmt string) {
m.formatters[name] = NewFormatter(fmt)
}
func (m *FormatterManger) SetMethod(name string, methods ...string) {
for _, method := range methods {
m.methodMap[method] = name
}
}
func (m *FormatterManger) GetByMethod(method string) (*Formatter, bool) {
formatterName, ok := m.methodMap[method]
if !ok {
formatterName = BaseFormatterKey
}
formatter, ok := m.formatters[formatterName]
if !ok {
return nil, false
}
return formatter, true
}
type Formatter struct {
fmt string
fields []string
prefixs []string
}
func NewFormatter(base string) *Formatter {
formatter := &Formatter{
fmt: base,
}
formatter.build()
return formatter
}
func (f *Formatter) buildMetric(metric string) ([]string, []string) {
newFields := []string{}
newPrefixs := []string{}
for id, prefix := range f.prefixs {
prefixs := strings.Split(prefix, metric)
newPrefixs = append(newPrefixs, prefixs...)
for i := 1; i < len(prefixs); i++ {
newFields = append(newFields, metric)
}
if id < len(f.fields) {
newFields = append(newFields, f.fields[id])
}
}
return newFields, newPrefixs
}
func (f *Formatter) build() {
f.prefixs = []string{f.fmt}
f.fields = []string{}
for mertric := range metricFuncMap {
if strings.Contains(f.fmt, mertric) {
f.fields, f.prefixs = f.buildMetric(mertric)
}
}
}
func (f *Formatter) Format(info AccessInfo) string {
fieldValues := info.Get(f.fields...)
result := ""
for id, prefix := range f.prefixs {
result += prefix
if id < len(fieldValues) {
result += fieldValues[id]
}
}
result += "\n"
return result
}
func parseConfigKey(k string) (string, string, error) {
fields := strings.Split(k, ".")
if len(fields) != 2 || (fields[1] != fomaterkey && fields[1] != methodKey) {
return "", "", merr.WrapErrParameterInvalid("<FormatterName>.(format|methods)", k, "parse accsslog formatter config key failed")
}
return fields[0], fields[1], nil
}

View File

@ -0,0 +1,122 @@
// 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 accesslog
import (
"io"
"sync"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
const (
clientRequestIDKey = "client_request_id"
)
var (
_globalW io.Writer
_globalR *RotateLogger
_globalF *FormatterManger
once sync.Once
)
func InitAccessLog(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) {
once.Do(func() {
err := initAccessLogger(logCfg, minioCfg)
if err != nil {
log.Fatal("initialize access logger error", zap.Error(err))
}
log.Info("Init access log success")
})
}
func initFormatter(logCfg *paramtable.AccessLogConfig) error {
formatterManger := NewFormatterManger()
formatMap := make(map[string]string) // fommatter name -> formatter format
methodMap := make(map[string][]string) // fommatter name -> formatter owner method
for key, value := range logCfg.Formatter.GetValue() {
formatterName, option, err := parseConfigKey(key)
if err != nil {
return err
}
if option == fomaterkey {
formatMap[formatterName] = value
} else if option == methodKey {
methodMap[formatterName] = paramtable.ParseAsStings(value)
}
}
for name, format := range formatMap {
formatterManger.Add(name, format)
if methods, ok := methodMap[name]; ok {
formatterManger.SetMethod(name, methods...)
}
}
_globalF = formatterManger
return nil
}
// initAccessLogger initializes a zap access logger for proxy
func initAccessLogger(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) error {
var lg *RotateLogger
var err error
if !logCfg.Enable.GetAsBool() {
return nil
}
err = initFormatter(logCfg)
if err != nil {
return err
}
if len(logCfg.Filename.GetValue()) > 0 {
lg, err = NewRotateLogger(logCfg, minioCfg)
if err != nil {
return err
}
if logCfg.CacheSize.GetAsInt() > 0 {
blg := NewCacheLogger(lg, logCfg.CacheSize.GetAsInt())
_globalW = zapcore.AddSync(blg)
} else {
_globalW = zapcore.AddSync(lg)
}
} else {
stdout, _, err := zap.Open([]string{"stdout"}...)
if err != nil {
return err
}
_globalW = stdout
}
_globalR = lg
return nil
}
func Rotate() error {
if _globalR == nil {
return nil
}
err := _globalR.Rotate()
return err
}

View File

@ -39,31 +39,26 @@ func TestAccessLogger_NotEnable(t *testing.T) {
Params.Init(paramtable.NewBaseTable(paramtable.SkipRemote(true)))
Params.Save(Params.ProxyCfg.AccessLog.Enable.Key, "false")
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
ctx := peer.NewContext(
context.Background(),
&peer.Peer{
Addr: &net.IPAddr{
IP: net.IPv4(0, 0, 0, 0),
Zone: "test",
},
})
ctx = metadata.AppendToOutgoingContext(ctx, clientRequestIDKey, "test")
resp := &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
},
Value: false,
}
err := initAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
assert.NoError(t, err)
rpcInfo := &grpc.UnaryServerInfo{Server: nil, FullMethod: "testMethod"}
ok := PrintAccessInfo(ctx, resp, nil, rpcInfo, 0)
accessInfo := NewGrpcAccessInfo(context.Background(), rpcInfo, nil)
ok := accessInfo.Write()
assert.False(t, ok)
}
func TestAccessLogger_InitFailed(t *testing.T) {
var Params paramtable.ComponentParam
Params.Init(paramtable.NewBaseTable(paramtable.SkipRemote(true)))
Params.Save(Params.ProxyCfg.AccessLog.Enable.Key, "true")
Params.SaveGroup(map[string]string{Params.ProxyCfg.AccessLog.Formatter.KeyPrefix + "testf.invaild": "invalidConfig"})
err := initAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
assert.Error(t, err)
}
func TestAccessLogger_Basic(t *testing.T) {
var Params paramtable.ComponentParam
@ -73,7 +68,7 @@ func TestAccessLogger_Basic(t *testing.T) {
Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
defer os.RemoveAll(testPath)
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
initAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
ctx := peer.NewContext(
context.Background(),
@ -85,6 +80,12 @@ func TestAccessLogger_Basic(t *testing.T) {
})
ctx = metadata.AppendToOutgoingContext(ctx, clientRequestIDKey, "test")
req := &milvuspb.QueryRequest{
DbName: "test-db",
CollectionName: "test-collection",
PartitionNames: []string{"test-partition-1", "test-partition-2"},
}
resp := &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -94,7 +95,11 @@ func TestAccessLogger_Basic(t *testing.T) {
}
rpcInfo := &grpc.UnaryServerInfo{Server: nil, FullMethod: "testMethod"}
ok := PrintAccessInfo(ctx, resp, nil, rpcInfo, 0)
accessInfo := NewGrpcAccessInfo(ctx, rpcInfo, req)
accessInfo.SetResult(resp, nil)
ok := accessInfo.Write()
assert.True(t, ok)
}
@ -105,7 +110,7 @@ func TestAccessLogger_Stdout(t *testing.T) {
Params.Save(Params.ProxyCfg.AccessLog.Enable.Key, "true")
Params.Save(Params.ProxyCfg.AccessLog.Filename.Key, "")
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
initAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
ctx := peer.NewContext(
context.Background(),
@ -117,6 +122,12 @@ func TestAccessLogger_Stdout(t *testing.T) {
})
ctx = metadata.AppendToOutgoingContext(ctx, clientRequestIDKey, "test")
req := &milvuspb.QueryRequest{
DbName: "test-db",
CollectionName: "test-collection",
PartitionNames: []string{"test-partition-1", "test-partition-2"},
}
resp := &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -126,7 +137,10 @@ func TestAccessLogger_Stdout(t *testing.T) {
}
rpcInfo := &grpc.UnaryServerInfo{Server: nil, FullMethod: "testMethod"}
ok := PrintAccessInfo(ctx, resp, nil, rpcInfo, 0)
accessInfo := NewGrpcAccessInfo(ctx, rpcInfo, req)
accessInfo.SetResult(resp, nil)
ok := accessInfo.Write()
assert.True(t, ok)
}
@ -139,11 +153,16 @@ func TestAccessLogger_WithMinio(t *testing.T) {
Params.Save(Params.ProxyCfg.AccessLog.Filename.Key, "test_access")
Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
Params.Save(Params.ProxyCfg.AccessLog.MinioEnable.Key, "true")
Params.Save(Params.ProxyCfg.AccessLog.CacheSize.Key, "0")
Params.Save(Params.ProxyCfg.AccessLog.RemotePath.Key, "access_log/")
Params.Save(Params.ProxyCfg.AccessLog.MaxSize.Key, "1")
defer os.RemoveAll(testPath)
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
// test rotate before init
err := Rotate()
assert.NoError(t, err)
initAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
ctx := peer.NewContext(
context.Background(),
@ -155,6 +174,12 @@ func TestAccessLogger_WithMinio(t *testing.T) {
})
ctx = metadata.AppendToOutgoingContext(ctx, clientRequestIDKey, "test")
req := &milvuspb.QueryRequest{
DbName: "test-db",
CollectionName: "test-collection",
PartitionNames: []string{"test-partition-1", "test-partition-2"},
}
resp := &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -164,44 +189,17 @@ func TestAccessLogger_WithMinio(t *testing.T) {
}
rpcInfo := &grpc.UnaryServerInfo{Server: nil, FullMethod: "testMethod"}
ok := PrintAccessInfo(ctx, resp, nil, rpcInfo, 0)
accessInfo := NewGrpcAccessInfo(ctx, rpcInfo, req)
accessInfo.SetResult(resp, nil)
ok := accessInfo.Write()
assert.True(t, ok)
W().Rotate()
defer W().handler.Clean()
Rotate()
defer _globalR.handler.Clean()
time.Sleep(time.Duration(1) * time.Second)
logfiles, err := W().handler.listAll()
logfiles, err := _globalR.handler.listAll()
assert.NoError(t, err)
assert.Equal(t, 1, len(logfiles))
}
func TestAccessLogger_Error(t *testing.T) {
var Params paramtable.ComponentParam
Params.Init(paramtable.NewBaseTable(paramtable.SkipRemote(true)))
testPath := "/tmp/accesstest"
Params.Save(Params.ProxyCfg.AccessLog.Enable.Key, "true")
Params.Save(Params.ProxyCfg.AccessLog.Filename.Key, "test_access")
Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
defer os.RemoveAll(testPath)
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
ctx := peer.NewContext(
context.Background(),
&peer.Peer{
Addr: &net.IPAddr{
IP: net.IPv4(0, 0, 0, 0),
Zone: "test",
},
})
rpcInfo := &grpc.UnaryServerInfo{Server: nil, FullMethod: "testMethod"}
ok := PrintAccessInfo(ctx, nil, nil, rpcInfo, 0)
assert.False(t, ok)
ctx = metadata.AppendToOutgoingContext(ctx, clientRequestIDKey, "test")
ok = PrintAccessInfo(ctx, nil, nil, rpcInfo, 0)
assert.False(t, ok)
}

View File

@ -0,0 +1,250 @@
// 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 accesslog
import (
"context"
"fmt"
"path"
"time"
"go.opentelemetry.io/otel/trace"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/peer"
"google.golang.org/grpc/status"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/requestutil"
)
type AccessInfo interface {
Get(keys ...string) []string
}
type GrpcAccessInfo struct {
ctx context.Context
info *grpc.UnaryServerInfo
req interface{}
resp interface{}
err error
start time.Time
end time.Time
}
func NewGrpcAccessInfo(ctx context.Context, info *grpc.UnaryServerInfo, req interface{}) *GrpcAccessInfo {
accessInfo := &GrpcAccessInfo{
ctx: ctx,
info: info,
req: req,
start: time.Now(),
}
return accessInfo
}
// update context for more info
func (i *GrpcAccessInfo) UpdateCtx(ctx context.Context) {
i.ctx = ctx
}
func (i *GrpcAccessInfo) SetResult(resp interface{}, err error) {
i.resp = resp
i.err = err
i.end = time.Now()
}
func (i *GrpcAccessInfo) Get(keys ...string) []string {
result := []string{}
metricMap := map[string]string{}
for _, key := range keys {
if value, ok := metricMap[key]; ok {
result = append(result, value)
} else if getFunc, ok := metricFuncMap[key]; ok {
result = append(result, getFunc(i))
}
}
return result
}
func (i *GrpcAccessInfo) Write() bool {
if _globalW == nil {
return false
}
formatter, ok := _globalF.GetByMethod(getMethodName(i))
if !ok {
return false
}
_, err := _globalW.Write([]byte(formatter.Format(i)))
return err == nil
}
func getTimeCost(i *GrpcAccessInfo) string {
if i.end.IsZero() {
return unknownString
}
return fmt.Sprint(i.end.Sub(i.start))
}
func getTimeNow(i *GrpcAccessInfo) string {
return time.Now().Format(timePrintFormat)
}
func getTimeStart(i *GrpcAccessInfo) string {
if i.start.IsZero() {
return unknownString
}
return i.start.Format(timePrintFormat)
}
func getTimeEnd(i *GrpcAccessInfo) string {
if i.end.IsZero() {
return unknownString
}
return i.end.Format(timePrintFormat)
}
func getMethodName(i *GrpcAccessInfo) string {
_, methodName := path.Split(i.info.FullMethod)
return methodName
}
func getAddr(i *GrpcAccessInfo) string {
ip, ok := peer.FromContext(i.ctx)
if !ok {
return "Unknown"
}
return fmt.Sprintf("%s-%s", ip.Addr.Network(), ip.Addr.String())
}
func getTraceID(i *GrpcAccessInfo) string {
meta, ok := metadata.FromOutgoingContext(i.ctx)
if ok {
return meta.Get(clientRequestIDKey)[0]
}
traceID := trace.SpanFromContext(i.ctx).SpanContext().TraceID()
return traceID.String()
}
func getMethodStatus(i *GrpcAccessInfo) string {
code := status.Code(i.err)
if code != codes.OK && code != codes.Unknown {
return fmt.Sprintf("Grpc%s", code.String())
}
return code.String()
}
func getUserName(i *GrpcAccessInfo) string {
username, err := getCurUserFromContext(i.ctx)
if err != nil {
return unknownString
}
return username
}
type SizeResponse interface {
XXX_Size() int
}
func getResponseSize(i *GrpcAccessInfo) string {
message, ok := i.resp.(SizeResponse)
if !ok {
return unknownString
}
return fmt.Sprint(message.XXX_Size())
}
type BaseResponse interface {
GetStatus() *commonpb.Status
}
func getErrorCode(i *GrpcAccessInfo) string {
baseResp, ok := i.resp.(BaseResponse)
if ok {
status := baseResp.GetStatus()
return fmt.Sprint(int(status.GetErrorCode()))
}
status, ok := i.resp.(*commonpb.Status)
if ok {
return fmt.Sprint(int(status.GetErrorCode()))
}
return fmt.Sprint(merr.Code(i.err))
}
func getErrorMsg(i *GrpcAccessInfo) string {
if i.err != nil {
return i.err.Error()
}
baseResp, ok := i.resp.(BaseResponse)
if ok {
status := baseResp.GetStatus()
return status.GetReason()
}
status, ok := i.resp.(*commonpb.Status)
if ok {
return status.GetReason()
}
return unknownString
}
func getDbName(i *GrpcAccessInfo) string {
name, ok := requestutil.GetDbNameFromRequest(i.req)
if !ok {
return unknownString
}
return name.(string)
}
func getCollectionName(i *GrpcAccessInfo) string {
name, ok := requestutil.GetCollectionNameFromRequest(i.req)
if !ok {
return unknownString
}
return name.(string)
}
func getPartitionName(i *GrpcAccessInfo) string {
name, ok := requestutil.GetPartitionNameFromRequest(i.req)
if ok {
return name.(string)
}
names, ok := requestutil.GetPartitionNamesFromRequest(i.req)
if ok {
return fmt.Sprint(names.([]string))
}
return unknownString
}
func getExpr(i *GrpcAccessInfo) string {
expr, ok := requestutil.GetExprFromRequest(i.req)
if !ok {
return unknownString
}
return expr.(string)
}

View File

@ -0,0 +1,113 @@
// 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 accesslog
import (
"context"
"fmt"
"net"
"testing"
"github.com/stretchr/testify/suite"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/peer"
"google.golang.org/grpc/status"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/pkg/util"
"github.com/milvus-io/milvus/pkg/util/crypto"
"github.com/milvus-io/milvus/pkg/util/merr"
)
type GrpcAccessInfoSuite struct {
suite.Suite
username string
traceID string
info *GrpcAccessInfo
}
func (s *GrpcAccessInfoSuite) SetupSuite() {
s.username = "test-user"
s.traceID = "test-trace"
ctx := peer.NewContext(
context.Background(),
&peer.Peer{
Addr: &net.IPAddr{
IP: net.IPv4(0, 0, 0, 0),
Zone: "test",
},
})
md := metadata.Pairs(util.HeaderAuthorize, crypto.Base64Encode("mockUser:mockPass"))
ctx = metadata.NewIncomingContext(ctx, md)
serverinfo := &grpc.UnaryServerInfo{
FullMethod: "test",
}
s.info = &GrpcAccessInfo{
ctx: ctx,
info: serverinfo,
}
}
func (s *GrpcAccessInfoSuite) TestErrorCode() {
s.info.resp = &milvuspb.QueryResults{
Status: merr.Status(nil),
}
result := s.info.Get("$error_code")
s.Equal(fmt.Sprint(0), result[0])
s.info.resp = merr.Status(nil)
result = s.info.Get("$error_code")
s.Equal(fmt.Sprint(0), result[0])
}
func (s *GrpcAccessInfoSuite) TestErrorMsg() {
s.info.resp = &milvuspb.QueryResults{
Status: merr.Status(merr.ErrChannelLack),
}
result := s.info.Get("$error_msg")
s.Equal(merr.ErrChannelLack.Error(), result[0])
s.info.resp = merr.Status(merr.ErrChannelLack)
result = s.info.Get("$error_msg")
s.Equal(merr.ErrChannelLack.Error(), result[0])
s.info.err = status.Errorf(codes.Unavailable, "mock")
result = s.info.Get("$error_msg")
s.Equal("rpc error: code = Unavailable desc = mock", result[0])
}
func (s *GrpcAccessInfoSuite) TestDbName() {
s.info.req = nil
result := s.info.Get("$database_name")
s.Equal(unknownString, result[0])
s.info.req = &milvuspb.QueryRequest{
DbName: "test",
}
result = s.info.Get("$database_name")
s.Equal("test", result[0])
}
func TestGrpcAccssInfo(t *testing.T) {
suite.Run(t, new(GrpcAccessInfoSuite))
}

View File

@ -181,7 +181,7 @@ func (c *minioHandler) Update(objectName string, filePath string) {
// update log file to minio
func (c *minioHandler) update(objectName string, filePath string) error {
path := Join(c.rootPath, filePath)
path := join(c.rootPath, filePath)
_, err := c.client.FPutObject(context.Background(), c.bucketName, path, objectName, minio.PutObjectOptions{})
return err
}
@ -266,7 +266,7 @@ func getTimeRetentionFunc(retentionTime int, prefix, ext string) RetentionFunc {
return false
}
nowWallTime, _ := time.Parse(timeFormat, time.Now().Format(timeFormat))
nowWallTime, _ := time.Parse(timeNameFormat, time.Now().Format(timeNameFormat))
intervalTime := nowWallTime.Sub(fileTime)
return intervalTime > (time.Duration(retentionTime) * time.Hour)
}

View File

@ -43,11 +43,6 @@ func TestMinioHandler_ConnectError(t *testing.T) {
assert.Error(t, err)
}
func TestMinioHandler_Join(t *testing.T) {
assert.Equal(t, "a/b", Join("a", "b"))
assert.Equal(t, "a/b", Join("a/", "b"))
}
func TestMinHandler_Basic(t *testing.T) {
var Params paramtable.ComponentParam
Params.Init(paramtable.NewBaseTable(paramtable.SkipRemote(true)))
@ -147,7 +142,7 @@ func TestMinioHandler_WithTimeRetention(t *testing.T) {
}
func createAndUpdateFile(handler *minioHandler, t time.Time, rootPath, prefix, ext string) error {
oldFileName := prefix + t.Format(timeFormat) + ext
oldFileName := prefix + t.Format(timeNameFormat) + ext
oldFilePath := path.Join(rootPath, oldFileName)
oldFileMode := os.FileMode(0o644)
_, err := os.OpenFile(oldFilePath, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, oldFileMode)

View File

@ -23,29 +23,31 @@ import (
"time"
"github.com/cockroachdb/errors"
"github.com/golang/protobuf/proto"
"go.opentelemetry.io/otel/trace"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/peer"
"google.golang.org/grpc/status"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/pkg/util"
"github.com/milvus-io/milvus/pkg/util/crypto"
)
type BaseResponse interface {
GetStatus() *commonpb.Status
}
type AccessKey struct{}
func UnaryAccessLoggerInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
starttime := time.Now()
resp, err := handler(ctx, req)
PrintAccessInfo(ctx, resp, err, info, time.Since(starttime).Milliseconds())
func UnaryAccessLogInterceptor(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
accessInfo := NewGrpcAccessInfo(ctx, info, req)
newCtx := context.WithValue(ctx, AccessKey{}, accessInfo)
resp, err := handler(newCtx, req)
accessInfo.SetResult(resp, err)
accessInfo.Write()
return resp, err
}
func Join(path1, path2 string) string {
func UnaryUpdateAccessInfoInterceptor(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
accessInfo := ctx.Value(AccessKey{}).(*GrpcAccessInfo)
accessInfo.UpdateCtx(ctx)
return handler(ctx, req)
}
func join(path1, path2 string) string {
if strings.HasSuffix(path1, "/") {
return path1 + path2
}
@ -60,50 +62,27 @@ func timeFromName(filename, prefix, ext string) (time.Time, error) {
return time.Time{}, errors.New("mismatched extension")
}
ts := filename[len(prefix) : len(filename)-len(ext)]
return time.Parse(timeFormat, ts)
return time.Parse(timeNameFormat, ts)
}
func getAccessAddr(ctx context.Context) string {
ip, ok := peer.FromContext(ctx)
func getCurUserFromContext(ctx context.Context) (string, error) {
md, ok := metadata.FromIncomingContext(ctx)
if !ok {
return "Unknown"
return "", fmt.Errorf("fail to get md from the context")
}
return fmt.Sprintf("%s-%s", ip.Addr.Network(), ip.Addr.String())
}
func getTraceID(ctx context.Context) (id string, ok bool) {
meta, ok := metadata.FromOutgoingContext(ctx)
if ok {
return meta.Get(clientRequestIDKey)[0], true
authorization, ok := md[strings.ToLower(util.HeaderAuthorize)]
if !ok || len(authorization) < 1 {
return "", fmt.Errorf("fail to get authorization from the md, authorize:[%s]", util.HeaderAuthorize)
}
traceID := trace.SpanFromContext(ctx).SpanContext().TraceID()
return traceID.String(), traceID.IsValid()
}
func getResponseSize(resq interface{}) (int, bool) {
message, ok := resq.(proto.Message)
if !ok {
return 0, false
token := authorization[0]
rawToken, err := crypto.Base64Decode(token)
if err != nil {
return "", fmt.Errorf("fail to decode the token, token: %s", token)
}
return proto.Size(message), true
}
func getErrCode(resp interface{}) (int, bool) {
baseResp, ok := resp.(BaseResponse)
if !ok {
return 0, false
secrets := strings.SplitN(rawToken, util.CredentialSeperator, 2)
if len(secrets) < 2 {
return "", fmt.Errorf("fail to get user info from the raw token, raw token: %s", rawToken)
}
status := baseResp.GetStatus()
return int(status.GetErrorCode()), true
}
func getGrpcStatus(err error) string {
code := status.Code(err)
if code != codes.OK {
return fmt.Sprintf("Grpc%s", code.String())
}
return code.String()
username := secrets[0]
return username, nil
}

View File

@ -17,92 +17,12 @@
package accesslog
import (
"context"
"net"
"testing"
"github.com/stretchr/testify/assert"
"go.opentelemetry.io/otel"
"go.uber.org/zap"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/peer"
"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/pkg/log"
"github.com/milvus-io/milvus/pkg/tracer"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
func TestGetAccessAddr(t *testing.T) {
ctx := context.Background()
addr := getAccessAddr(ctx)
assert.Equal(t, "Unknown", addr)
newctx := peer.NewContext(
ctx,
&peer.Peer{
Addr: &net.IPAddr{
IP: net.IPv4(0, 0, 0, 0),
Zone: "test",
},
})
addr = getAccessAddr(newctx)
assert.Equal(t, "ip-0.0.0.0%test", addr)
}
func TestGetTraceID(t *testing.T) {
paramtable.Init()
tracer.Init()
ctx := context.Background()
traceID, ok := getTraceID(ctx)
log.Debug("traceID", zap.String("id", traceID))
assert.False(t, ok)
traceContext, traceSpan := otel.Tracer("proxy").Start(ctx, "demo")
trueTraceID := traceSpan.SpanContext().TraceID().String()
log.Debug("traceID", zap.String("trueTraceID", trueTraceID))
ID, ok := getTraceID(traceContext)
assert.True(t, ok)
assert.Equal(t, trueTraceID, ID)
ctx = metadata.AppendToOutgoingContext(ctx, clientRequestIDKey, "test")
ID, ok = getTraceID(ctx)
assert.True(t, ok)
assert.Equal(t, "test", ID)
}
func TestGetResponseSize(t *testing.T) {
resp := &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
},
Value: false,
}
_, ok := getResponseSize(nil)
assert.False(t, ok)
_, ok = getResponseSize(resp)
assert.True(t, ok)
}
func TestGetErrCode(t *testing.T) {
resp := &milvuspb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
},
Value: false,
}
_, ok := getErrCode(nil)
assert.False(t, ok)
code, ok := getErrCode(resp)
assert.True(t, ok)
assert.Equal(t, int(commonpb.ErrorCode_UnexpectedError), code)
func TestJoin(t *testing.T) {
assert.Equal(t, "a/b", join("a", "b"))
assert.Equal(t, "a/b", join("a/", "b"))
}

View File

@ -36,7 +36,8 @@ const megabyte = 1024 * 1024
var (
CheckBucketRetryAttempts uint = 20
timeFormat = ".2006-01-02T15-04-05.000"
timeNameFormat = ".2006-01-02T15-04-05.000"
timePrintFormat = "2006/01/02 15:04:05.000 -07:00"
)
type CacheLogger struct {
@ -106,7 +107,10 @@ func NewRotateLogger(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.Mi
return nil, err
}
prefix, ext := logger.prefixAndExt()
handler.retentionPolicy = getTimeRetentionFunc(logCfg.RemoteMaxTime.GetAsInt(), prefix, ext)
if logCfg.RemoteMaxTime.GetAsInt() > 0 {
handler.retentionPolicy = getTimeRetentionFunc(logCfg.RemoteMaxTime.GetAsInt(), prefix, ext)
}
logger.handler = handler
}
@ -247,7 +251,7 @@ func (l *RotateLogger) millRunOnce() error {
return err
}
if l.maxBackups > 0 && l.maxBackups < len(files) {
if l.maxBackups >= 0 && l.maxBackups < len(files) {
for _, f := range files[l.maxBackups:] {
errRemove := os.Remove(path.Join(l.dir(), f.fileName))
if err == nil && errRemove != nil {
@ -336,7 +340,7 @@ func (l *RotateLogger) prefixAndExt() (string, string) {
func (l *RotateLogger) newBackupName() string {
t := time.Now()
timestamp := t.Format(timeFormat)
timestamp := t.Format(timeNameFormat)
prefix, ext := l.prefixAndExt()
return path.Join(l.dir(), prefix+timestamp+ext)
}

View File

@ -23,9 +23,7 @@ import (
"time"
"github.com/stretchr/testify/assert"
"go.uber.org/zap"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
@ -119,7 +117,6 @@ func TestRotateLogger_SizeRotate(t *testing.T) {
num := 1024 * 1024
text := getText(num + 1)
_, err = logger.Write(text)
log.Error("write failed", zap.Error(err))
assert.Error(t, err)
for i := 1; i <= 2; i++ {

View File

@ -10,7 +10,7 @@ import (
// DatabaseInterceptor fill dbname into request based on kv pair <"dbname": "xx"> in header
func DatabaseInterceptor() grpc.UnaryServerInterceptor {
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
filledCtx, filledReq := fillDatabase(ctx, req)
return handler(filledCtx, filledReq)
}

View File

@ -89,7 +89,7 @@ func UnaryServerHookInterceptor() grpc.UnaryServerInterceptor {
logger.Error("hook error", zap.String("path", Params.ProxyCfg.SoPath.GetValue()), zap.Error(hookError))
hoo = defaultHook{}
}
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
var (
fullMethod = info.FullMethod
newCtx context.Context

View File

@ -28,7 +28,7 @@ func getIdentifierFromContext(ctx context.Context) (int64, error) {
return identifier, nil
}
func KeepActiveInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
func KeepActiveInterceptor(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
// We shouldn't block the normal rpc. though this may be not very accurate enough.
// On the other hand, too many goroutines will also influence the rpc.
// Not sure which way is better, since actually we already make the `keepActive` asynchronous.

View File

@ -53,7 +53,7 @@ func getPolicyModel(modelString string) model.Model {
// UnaryServerInterceptor returns a new unary server interceptors that performs per-request privilege access.
func UnaryServerInterceptor(privilegeFunc PrivilegeFunc) grpc.UnaryServerInterceptor {
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
newCtx, err := privilegeFunc(ctx, req)
if err != nil {
return nil, err

View File

@ -217,7 +217,7 @@ func (node *Proxy) Init() error {
node.factory.Init(Params)
accesslog.SetupAccseeLog(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
accesslog.InitAccessLog(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
log.Debug("init access log for Proxy done")
err := node.initRateCollector()

View File

@ -33,7 +33,7 @@ import (
// RateLimitInterceptor returns a new unary server interceptors that performs request rate limiting.
func RateLimitInterceptor(limiter types.Limiter) grpc.UnaryServerInterceptor {
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
collectionID, rt, n, err := getRequestInfo(req)
if err != nil {
return handler(ctx, req)

View File

@ -263,7 +263,7 @@ func (dt *deleteTask) PostExecute(ctx context.Context) error {
func (dt *deleteTask) getStreamingQueryAndDelteFunc(stream msgstream.MsgStream, plan *planpb.PlanNode) executeFunc {
return func(ctx context.Context, nodeID int64, qn types.QueryNodeClient, channelIDs ...string) error {
var partationIDs []int64
var partitionIDs []int64
// optimize query when partitionKey on
if dt.partitionKeyMode {
@ -276,17 +276,17 @@ func (dt *deleteTask) getStreamingQueryAndDelteFunc(stream msgstream.MsgStream,
if err != nil {
return err
}
partationIDs, err = getPartitionIDs(ctx, dt.req.GetDbName(), dt.req.GetCollectionName(), hashedPartitionNames)
partitionIDs, err = getPartitionIDs(ctx, dt.req.GetDbName(), dt.req.GetCollectionName(), hashedPartitionNames)
if err != nil {
return err
}
} else if dt.partitionID != common.InvalidFieldID {
partationIDs = []int64{dt.partitionID}
partitionIDs = []int64{dt.partitionID}
}
log := log.Ctx(ctx).With(
zap.Int64("collectionID", dt.collectionID),
zap.Int64s("partationIDs", partationIDs),
zap.Int64s("partitionIDs", partitionIDs),
zap.Strings("channels", channelIDs),
zap.Int64("nodeID", nodeID))
// set plan
@ -312,7 +312,7 @@ func (dt *deleteTask) getStreamingQueryAndDelteFunc(stream msgstream.MsgStream,
ReqID: paramtable.GetNodeID(),
DbID: 0, // TODO
CollectionID: dt.collectionID,
PartitionIDs: partationIDs,
PartitionIDs: partitionIDs,
SerializedExprPlan: serializedPlan,
OutputFieldsId: outputFieldIDs,
GuaranteeTimestamp: parseGuaranteeTsFromConsistency(dt.ts, dt.ts, commonpb.ConsistencyLevel_Bounded),
@ -378,7 +378,7 @@ func (dt *deleteTask) simpleDelete(ctx context.Context, termExp *planpb.Expr_Ter
log.Debug("get primary keys from expr",
zap.Int64("len of primary keys", numRow),
zap.Int64("collectionID", dt.collectionID),
zap.Int64("partationID", dt.partitionID))
zap.Int64("partitionID", dt.partitionID))
err = dt.produce(ctx, stream, primaryKeys)
if err != nil {
return err

View File

@ -30,7 +30,7 @@ import (
"github.com/milvus-io/milvus/pkg/util/requestutil"
)
func TraceLogInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
func TraceLogInterceptor(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
switch Params.CommonCfg.TraceLogMode.GetAsInt() {
case 0: // none
return handler(ctx, req)

View File

@ -203,6 +203,12 @@ func (m *Manager) SetConfig(key, value string) {
m.overlays[formatKey(key)] = value
}
func (m *Manager) SetMapConfig(key, value string) {
m.Lock()
defer m.Unlock()
m.overlays[strings.ToLower(key)] = value
}
// Delete config at runtime, which has the highest priority to override all other sources
func (m *Manager) DeleteConfig(key string) {
m.Lock()

View File

@ -32,7 +32,7 @@ const ClusterKey = "Cluster"
// rejects the request if the client's cluster differs from that of the server.
// It is chiefly employed to tackle the `Cross-Cluster Routing` issue.
func ClusterValidationUnaryServerInterceptor() grpc.UnaryServerInterceptor {
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
md, ok := metadata.FromIncomingContext(ctx)
if !ok {
return handler(ctx, req)

View File

@ -34,7 +34,7 @@ type GetServerIDFunc func() int64
// ServerIDValidationUnaryServerInterceptor returns a new unary server interceptor that
// verifies whether the target server ID of request matches with the server's ID and rejects it accordingly.
func ServerIDValidationUnaryServerInterceptor(fn GetServerIDFunc) grpc.UnaryServerInterceptor {
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
md, ok := metadata.FromIncomingContext(ctx)
if !ok {
return handler(ctx, req)

View File

@ -18,7 +18,7 @@ const (
)
// UnaryTraceLoggerInterceptor adds a traced logger in unary rpc call ctx
func UnaryTraceLoggerInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
func UnaryTraceLoggerInterceptor(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
newctx := withLevelAndTrace(ctx)
return handler(newctx, req)
}

View File

@ -261,6 +261,13 @@ func (bt *BaseTable) Save(key, value string) error {
return nil
}
func (bt *BaseTable) SaveGroup(group map[string]string) error {
for key, value := range group {
bt.mgr.SetMapConfig(key, value)
}
return nil
}
// Reset Config to default value
func (bt *BaseTable) Reset(key string) error {
bt.mgr.ResetConfig(key)

View File

@ -887,16 +887,17 @@ func (p *rootCoordConfig) init(base *BaseTable) {
// /////////////////////////////////////////////////////////////////////////////
// --- proxy ---
type AccessLogConfig struct {
Enable ParamItem `refreshable:"false"`
MinioEnable ParamItem `refreshable:"false"`
LocalPath ParamItem `refreshable:"false"`
Filename ParamItem `refreshable:"false"`
MaxSize ParamItem `refreshable:"false"`
CacheSize ParamItem `refreshable:"false"`
RotatedTime ParamItem `refreshable:"false"`
MaxBackups ParamItem `refreshable:"false"`
RemotePath ParamItem `refreshable:"false"`
RemoteMaxTime ParamItem `refreshable:"false"`
Enable ParamItem `refreshable:"false"`
MinioEnable ParamItem `refreshable:"false"`
LocalPath ParamItem `refreshable:"false"`
Filename ParamItem `refreshable:"false"`
MaxSize ParamItem `refreshable:"false"`
CacheSize ParamItem `refreshable:"false"`
RotatedTime ParamItem `refreshable:"false"`
MaxBackups ParamItem `refreshable:"false"`
RemotePath ParamItem `refreshable:"false"`
RemoteMaxTime ParamItem `refreshable:"false"`
Formatter ParamGroup `refreshable:"false"`
}
type proxyConfig struct {
@ -917,13 +918,14 @@ type proxyConfig struct {
MaxUserNum ParamItem `refreshable:"true"`
MaxRoleNum ParamItem `refreshable:"true"`
MaxTaskNum ParamItem `refreshable:"false"`
AccessLog AccessLogConfig
ShardLeaderCacheInterval ParamItem `refreshable:"false"`
ReplicaSelectionPolicy ParamItem `refreshable:"false"`
CheckQueryNodeHealthInterval ParamItem `refreshable:"false"`
CostMetricsExpireTime ParamItem `refreshable:"true"`
RetryTimesOnReplica ParamItem `refreshable:"true"`
RetryTimesOnHealthCheck ParamItem `refreshable:"true"`
AccessLog AccessLogConfig
}
func (p *proxyConfig) init(base *BaseTable) {
@ -1106,7 +1108,7 @@ please adjust in embedded Milvus: false`,
p.AccessLog.MaxSize.Init(base.mgr)
p.AccessLog.CacheSize = ParamItem{
Key: "proxy.accessLog.maxSize",
Key: "proxy.accessLog.cacheSize",
Version: "2.3.2",
DefaultValue: "10240",
Doc: "Size of log of memory cache, in B",
@ -1140,11 +1142,17 @@ please adjust in embedded Milvus: false`,
p.AccessLog.RemoteMaxTime = ParamItem{
Key: "proxy.accessLog.remoteMaxTime",
Version: "2.2.0",
DefaultValue: "168",
DefaultValue: "0",
Doc: "Max time for log file in minIO, in hours",
}
p.AccessLog.RemoteMaxTime.Init(base.mgr)
p.AccessLog.Formatter = ParamGroup{
KeyPrefix: "proxy.accessLog.formatters.",
Version: "2.3.4",
}
p.AccessLog.Formatter.Init(base.mgr)
p.ShardLeaderCacheInterval = ParamItem{
Key: "proxy.shardLeaderCacheInterval",
Version: "2.2.4",
@ -2803,6 +2811,10 @@ func (params *ComponentParam) Save(key string, value string) error {
return params.baseTable.Save(key, value)
}
func (params *ComponentParam) SaveGroup(group map[string]string) error {
return params.baseTable.SaveGroup(group)
}
func (params *ComponentParam) Remove(key string) error {
return params.baseTable.Remove(key)
}

View File

@ -177,6 +177,10 @@ func (pg *ParamGroup) GetValue() map[string]string {
return values
}
func ParseAsStings(v string) []string {
return getAsStrings(v)
}
func getAsStrings(v string) []string {
return getAndConvert(v, func(value string) ([]string, error) {
return strings.Split(value, ","), nil