mirror of https://github.com/milvus-io/milvus.git
Add log panic in log module
Signed-off-by: sunby <bingyi.sun@zilliz.com>pull/4973/head^2
parent
36bae90635
commit
dec0b40a66
|
@ -6,6 +6,8 @@ import (
|
|||
"os/signal"
|
||||
"syscall"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
dn "github.com/zilliztech/milvus-distributed/internal/datanode"
|
||||
|
@ -22,7 +24,7 @@ func main() {
|
|||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
dn.Params.Init()
|
||||
log.SetupLogger(&dn.Params.Log)
|
||||
logutil.SetupLogger(&dn.Params.Log)
|
||||
|
||||
dn, err := distributed.NewDataNode(ctx, msFactory)
|
||||
if err != nil {
|
||||
|
|
|
@ -6,6 +6,8 @@ import (
|
|||
"os/signal"
|
||||
"syscall"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/dataservice"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
|
@ -15,9 +17,10 @@ import (
|
|||
|
||||
func main() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer logutil.LogPanic()
|
||||
|
||||
dataservice.Params.Init()
|
||||
log.SetupLogger(&dataservice.Params.Log)
|
||||
logutil.SetupLogger(&dataservice.Params.Log)
|
||||
defer log.Sync()
|
||||
msFactory := pulsarms.NewFactory()
|
||||
|
||||
|
|
|
@ -8,9 +8,10 @@ import (
|
|||
"strings"
|
||||
"syscall"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
ds "github.com/zilliztech/milvus-distributed/internal/dataservice"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/rmqms"
|
||||
|
@ -135,7 +136,7 @@ func (mr *MilvusRoles) Run(localMsg bool) {
|
|||
var err error
|
||||
// Init data service params
|
||||
ds.Params.Init()
|
||||
log.SetupLogger(&ds.Params.Log)
|
||||
logutil.SetupLogger(&ds.Params.Log)
|
||||
dataService, err = components.NewDataService(ctx, factory)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -6,6 +6,8 @@ import (
|
|||
"os/signal"
|
||||
"syscall"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
distributed "github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/masterservice"
|
||||
|
@ -18,7 +20,7 @@ func main() {
|
|||
defer cancel()
|
||||
|
||||
masterservice.Params.Init()
|
||||
log.SetupLogger(&masterservice.Params.Log)
|
||||
logutil.SetupLogger(&masterservice.Params.Log)
|
||||
defer func() {
|
||||
if err := log.Sync(); err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -6,6 +6,8 @@ import (
|
|||
"os/signal"
|
||||
"syscall"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
distributed "github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
|
@ -19,7 +21,7 @@ func main() {
|
|||
defer cancel()
|
||||
|
||||
queryservice.Params.Init()
|
||||
log.SetupLogger(&queryservice.Params.Log)
|
||||
logutil.SetupLogger(&queryservice.Params.Log)
|
||||
defer func() {
|
||||
if err := log.Sync(); err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -9,6 +9,8 @@ import (
|
|||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
grpcdatanodeclient "github.com/zilliztech/milvus-distributed/internal/distributed/datanode/client"
|
||||
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
|
||||
|
@ -317,6 +319,7 @@ func (s *Server) startServerLoop() {
|
|||
}
|
||||
|
||||
func (s *Server) startStatsChannel(ctx context.Context) {
|
||||
defer logutil.LogPanic()
|
||||
defer s.serverLoopWg.Done()
|
||||
statsStream, _ := s.msFactory.NewMsgStream(ctx)
|
||||
statsStream.AsConsumer([]string{Params.StatisticsChannelName}, Params.DataServiceSubscriptionName)
|
||||
|
@ -345,6 +348,7 @@ func (s *Server) startStatsChannel(ctx context.Context) {
|
|||
}
|
||||
|
||||
func (s *Server) startSegmentFlushChannel(ctx context.Context) {
|
||||
defer logutil.LogPanic()
|
||||
defer s.serverLoopWg.Done()
|
||||
flushStream, _ := s.msFactory.NewMsgStream(ctx)
|
||||
flushStream.AsConsumer([]string{Params.SegmentInfoChannelName}, Params.DataServiceSubscriptionName)
|
||||
|
|
|
@ -4,16 +4,20 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"log"
|
||||
"net"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
|
||||
otgrpc "github.com/opentracing-contrib/go-grpc"
|
||||
msc "github.com/zilliztech/milvus-distributed/internal/distributed/masterservice/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/uber/jaeger-client-go/config"
|
||||
|
@ -83,12 +87,12 @@ func (s *Server) init() error {
|
|||
return err
|
||||
}
|
||||
|
||||
log.Println("DataService:: MasterServicAddr:", Params.MasterAddress)
|
||||
log.Debug("master address", zap.String("address", Params.MasterAddress))
|
||||
client, err := msc.NewClient(Params.MasterAddress, 10*time.Second)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
log.Println("master client create complete")
|
||||
log.Debug("master client create complete")
|
||||
if err = client.Init(); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -107,20 +111,20 @@ func (s *Server) init() error {
|
|||
|
||||
dataservice.Params.Init()
|
||||
if err := s.impl.Init(); err != nil {
|
||||
log.Println("impl init error: ", err)
|
||||
log.Error("impl init error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) startGrpcLoop(grpcPort int) {
|
||||
|
||||
defer logutil.LogPanic()
|
||||
defer s.wg.Done()
|
||||
|
||||
log.Println("network port: ", grpcPort)
|
||||
log.Debug("network port", zap.Int("port", grpcPort))
|
||||
lis, err := net.Listen("tcp", ":"+strconv.Itoa(grpcPort))
|
||||
if err != nil {
|
||||
log.Printf("GrpcServer:failed to listen: %v", err)
|
||||
log.Error("grpc server failed to listen error", zap.Error(err))
|
||||
s.grpcErrChan <- err
|
||||
return
|
||||
}
|
||||
|
@ -171,7 +175,7 @@ func (s *Server) Run() error {
|
|||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Println("dataservice init done ...")
|
||||
log.Debug("dataservice init done ...")
|
||||
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
|
|
|
@ -1,9 +1,11 @@
|
|||
package log
|
||||
package logutil
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
|
||||
"go.uber.org/zap/zapcore"
|
||||
|
||||
etcd "go.etcd.io/etcd/clientv3"
|
||||
|
@ -77,35 +79,43 @@ func (w *zapWrapper) V(l int) bool {
|
|||
return w.logger.Core().Enabled(zapcore.Level(zapLevel))
|
||||
}
|
||||
|
||||
// LogPanic logs the panic reason and stack, then exit the process.
|
||||
// Commonly used with a `defer`.
|
||||
func LogPanic() {
|
||||
if e := recover(); e != nil {
|
||||
log.Fatal("panic", zap.Reflect("recover", e))
|
||||
}
|
||||
}
|
||||
|
||||
var once sync.Once
|
||||
var _globalZapWrapper atomic.Value
|
||||
|
||||
const defaultLogLevel = "info"
|
||||
|
||||
func init() {
|
||||
conf := &Config{Level: defaultLogLevel, File: FileLogConfig{}}
|
||||
lg, _, _ := InitLogger(conf)
|
||||
conf := &log.Config{Level: defaultLogLevel, File: log.FileLogConfig{}}
|
||||
lg, _, _ := log.InitLogger(conf)
|
||||
_globalZapWrapper.Store(&zapWrapper{
|
||||
logger: lg,
|
||||
})
|
||||
}
|
||||
|
||||
func SetupLogger(cfg *Config) {
|
||||
func SetupLogger(cfg *log.Config) {
|
||||
once.Do(func() {
|
||||
// initialize logger
|
||||
logger, p, err := InitLogger(cfg, zap.AddStacktrace(zap.ErrorLevel))
|
||||
logger, p, err := log.InitLogger(cfg, zap.AddStacktrace(zap.ErrorLevel))
|
||||
if err == nil {
|
||||
ReplaceGlobals(logger, p)
|
||||
log.ReplaceGlobals(logger, p)
|
||||
} else {
|
||||
Fatal("initialize logger error", zap.Error(err))
|
||||
log.Fatal("initialize logger error", zap.Error(err))
|
||||
}
|
||||
|
||||
// initialize grpc and etcd logger
|
||||
c := *cfg
|
||||
c.Level = defaultLogLevel
|
||||
lg, _, err := InitLogger(&c)
|
||||
lg, _, err := log.InitLogger(&c)
|
||||
if err != nil {
|
||||
Fatal("initialize grpc/etcd logger error", zap.Error(err))
|
||||
log.Fatal("initialize grpc/etcd logger error", zap.Error(err))
|
||||
}
|
||||
|
||||
wrapper := &zapWrapper{lg}
|
|
@ -2,9 +2,11 @@ package timesync
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
"sync"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
|
@ -26,17 +28,18 @@ func NewTimeSyncMsgProducer(ttBarrier TimeTickBarrier, watchers ...TimeTickWatch
|
|||
}
|
||||
|
||||
func (producer *MsgProducer) broadcastMsg() {
|
||||
defer logutil.LogPanic()
|
||||
defer producer.wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-producer.ctx.Done():
|
||||
log.Printf("broadcast context done, exit")
|
||||
log.Debug("broadcast context done, exit")
|
||||
return
|
||||
default:
|
||||
}
|
||||
tt, err := producer.ttBarrier.GetTimeTick()
|
||||
if err != nil {
|
||||
log.Printf("broadcast get time tick error")
|
||||
log.Debug("broadcast get time tick error")
|
||||
}
|
||||
baseMsg := ms.BaseMsg{
|
||||
BeginTimestamp: tt,
|
||||
|
@ -71,6 +74,7 @@ func (producer *MsgProducer) Start(ctx context.Context) {
|
|||
}
|
||||
|
||||
func (producer *MsgProducer) startWatcher(watcher TimeTickWatcher) {
|
||||
defer logutil.LogPanic()
|
||||
defer producer.wg.Done()
|
||||
watcher.StartBackgroundLoop(producer.ctx)
|
||||
}
|
||||
|
|
|
@ -2,14 +2,18 @@ package timesync
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
"math"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
)
|
||||
|
||||
|
@ -45,7 +49,7 @@ type (
|
|||
|
||||
func NewSoftTimeTickBarrier(ctx context.Context, ttStream ms.MsgStream, peerIds []UniqueID, minTtInterval Timestamp) *softTimeTickBarrier {
|
||||
if len(peerIds) <= 0 {
|
||||
log.Printf("[newSoftTimeTickBarrier] Error: peerIds is empty!\n")
|
||||
log.Debug("[newSoftTimeTickBarrier] Error: peerIds is empty!")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -59,7 +63,7 @@ func NewSoftTimeTickBarrier(ctx context.Context, ttStream ms.MsgStream, peerIds
|
|||
sttbarrier.peer2LastTt[id] = Timestamp(0)
|
||||
}
|
||||
if len(peerIds) != len(sttbarrier.peer2LastTt) {
|
||||
log.Printf("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!\n")
|
||||
log.Debug("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!")
|
||||
}
|
||||
|
||||
return &sttbarrier
|
||||
|
@ -89,7 +93,7 @@ func (ttBarrier *softTimeTickBarrier) Start() {
|
|||
for {
|
||||
select {
|
||||
case <-ttBarrier.ctx.Done():
|
||||
log.Printf("[TtBarrierStart] %s\n", ttBarrier.ctx.Err())
|
||||
log.Debug("[TtBarrierStart] shut down", zap.Error(ttBarrier.ctx.Err()))
|
||||
return
|
||||
default:
|
||||
}
|
||||
|
@ -101,7 +105,7 @@ func (ttBarrier *softTimeTickBarrier) Start() {
|
|||
// log.Printf("[softTimeTickBarrier] peer(%d)=%d\n", ttmsg.PeerID, ttmsg.Timestamp)
|
||||
|
||||
if !ok {
|
||||
log.Printf("[softTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.Base.SourceID)
|
||||
log.Warn("[softTimeTickBarrier] peerID not exist", zap.Int64("peerID", ttmsg.Base.SourceID))
|
||||
continue
|
||||
}
|
||||
if ttmsg.Base.Timestamp > oldT {
|
||||
|
@ -148,17 +152,18 @@ func (ttBarrier *hardTimeTickBarrier) Start() {
|
|||
ttBarrier.loopCtx, ttBarrier.loopCancel = context.WithCancel(ttBarrier.ctx)
|
||||
state := Timestamp(0)
|
||||
go func(ctx context.Context) {
|
||||
defer logutil.LogPanic()
|
||||
defer ttBarrier.wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Printf("[TtBarrierStart] %s\n", ttBarrier.ctx.Err())
|
||||
log.Debug("[TtBarrierStart] shut down", zap.Error(ttBarrier.ctx.Err()))
|
||||
return
|
||||
default:
|
||||
}
|
||||
ttmsgs, _ := ttBarrier.ttStream.Consume()
|
||||
if len(ttmsgs.Msgs) > 0 {
|
||||
log.Printf("receive tt msg")
|
||||
log.Debug("receive tt msg")
|
||||
for _, timetickmsg := range ttmsgs.Msgs {
|
||||
// Suppose ttmsg.Timestamp from stream is always larger than the previous one,
|
||||
// that `ttmsg.Timestamp > oldT`
|
||||
|
@ -166,22 +171,20 @@ func (ttBarrier *hardTimeTickBarrier) Start() {
|
|||
|
||||
oldT, ok := ttBarrier.peer2Tt[ttmsg.Base.SourceID]
|
||||
if !ok {
|
||||
log.Printf("[hardTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.Base.SourceID)
|
||||
log.Warn("[hardTimeTickBarrier] peerID not exist", zap.Int64("peerID", ttmsg.Base.SourceID))
|
||||
continue
|
||||
}
|
||||
|
||||
if oldT > state {
|
||||
log.Printf("[hardTimeTickBarrier] Warning: peer(%d) timestamp(%d) ahead\n",
|
||||
ttmsg.Base.SourceID, ttmsg.Base.Timestamp)
|
||||
log.Warn("[hardTimeTickBarrier] peer's timestamp ahead",
|
||||
zap.Int64("peerID", ttmsg.Base.SourceID), zap.Uint64("timestamp", ttmsg.Base.Timestamp))
|
||||
}
|
||||
|
||||
ttBarrier.peer2Tt[ttmsg.Base.SourceID] = ttmsg.Base.Timestamp
|
||||
|
||||
newState := ttBarrier.minTimestamp()
|
||||
log.Printf("new state %d", newState)
|
||||
if newState > state {
|
||||
ttBarrier.outTt <- newState
|
||||
log.Printf("outtttt")
|
||||
state = newState
|
||||
}
|
||||
}
|
||||
|
@ -207,7 +210,7 @@ func (ttBarrier *hardTimeTickBarrier) minTimestamp() Timestamp {
|
|||
|
||||
func NewHardTimeTickBarrier(ctx context.Context, ttStream ms.MsgStream, peerIds []UniqueID) *hardTimeTickBarrier {
|
||||
if len(peerIds) <= 0 {
|
||||
log.Printf("[newSoftTimeTickBarrier] Error: peerIds is empty!")
|
||||
log.Error("[newSoftTimeTickBarrier] peerIds is empty!")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -221,7 +224,7 @@ func NewHardTimeTickBarrier(ctx context.Context, ttStream ms.MsgStream, peerIds
|
|||
sttbarrier.peer2Tt[id] = Timestamp(0)
|
||||
}
|
||||
if len(peerIds) != len(sttbarrier.peer2Tt) {
|
||||
log.Printf("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!")
|
||||
log.Warn("[newSoftTimeTickBarrier] there are duplicate peerIds!", zap.Int64s("peerIDs", peerIds))
|
||||
}
|
||||
|
||||
return &sttbarrier
|
||||
|
|
Loading…
Reference in New Issue