mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-01 19:39:21 +08:00
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:
parent
1b7a503f89
commit
89d8ce2f73
@ -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
|
||||
|
@ -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,
|
||||
))
|
||||
|
@ -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
|
||||
}
|
192
internal/proxy/accesslog/formater_test.go
Normal file
192
internal/proxy/accesslog/formater_test.go
Normal 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))
|
||||
}
|
154
internal/proxy/accesslog/formatter.go
Normal file
154
internal/proxy/accesslog/formatter.go
Normal 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
|
||||
}
|
122
internal/proxy/accesslog/global.go
Normal file
122
internal/proxy/accesslog/global.go
Normal 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
|
||||
}
|
@ -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)
|
||||
}
|
250
internal/proxy/accesslog/info.go
Normal file
250
internal/proxy/accesslog/info.go
Normal 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)
|
||||
}
|
113
internal/proxy/accesslog/info_test.go
Normal file
113
internal/proxy/accesslog/info_test.go
Normal 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))
|
||||
}
|
@ -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)
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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"))
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
@ -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++ {
|
@ -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)
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
|
@ -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()
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
@ -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()
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user