diff --git a/cmd/datanode/main.go b/cmd/datanode/main.go
index 14f5468bf576361145d7e3ea543729f26add1c04..339da929e7064f249ace31f74b5b86bd69b53c3c 100644
--- a/cmd/datanode/main.go
+++ b/cmd/datanode/main.go
@@ -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 {
diff --git a/cmd/dataservice/main.go b/cmd/dataservice/main.go
index 2332cde84ad61ddb084073dd234bc2c48765898c..c4b9a6a84c088dc6966eab2a723889c45506a08e 100644
--- a/cmd/dataservice/main.go
+++ b/cmd/dataservice/main.go
@@ -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()
 
diff --git a/cmd/distributed/roles/roles.go b/cmd/distributed/roles/roles.go
index 75b6622340d5666eb3c19be925d0d9bb391fddb8..3369acb45ed27bde33b4398a5eecb7dfcc32e6aa 100644
--- a/cmd/distributed/roles/roles.go
+++ b/cmd/distributed/roles/roles.go
@@ -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)
diff --git a/cmd/masterservice/main.go b/cmd/masterservice/main.go
index 2d1d27d541324c2baec573d6ac876b2840b83fec..c869fe76d67c6994b5ebf518ffe44b2b3fda8353 100644
--- a/cmd/masterservice/main.go
+++ b/cmd/masterservice/main.go
@@ -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)
diff --git a/cmd/queryservice/queryservice.go b/cmd/queryservice/queryservice.go
index db1d212d2d5711424aecd39de2df75145af8b075..6edf7737ebf926bdb3483ff72e0d7ea8154abf66 100644
--- a/cmd/queryservice/queryservice.go
+++ b/cmd/queryservice/queryservice.go
@@ -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)
diff --git a/internal/dataservice/server.go b/internal/dataservice/server.go
index 61e195a38686de3e3845f53cff57c6a35e1c859e..8f0d1e4fd60fc33d35e5f416e9a2d18a58502e34 100644
--- a/internal/dataservice/server.go
+++ b/internal/dataservice/server.go
@@ -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)
diff --git a/internal/distributed/dataservice/service.go b/internal/distributed/dataservice/service.go
index e283210e8a62947e99dbbe4f67f9f1ec56ba13ce..32820dd841a6b76e8500cfc947d8c374c482bdda 100644
--- a/internal/distributed/dataservice/service.go
+++ b/internal/distributed/dataservice/service.go
@@ -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
diff --git a/internal/log/logutil.go b/internal/logutil/logutil.go
similarity index 79%
rename from internal/log/logutil.go
rename to internal/logutil/logutil.go
index c5cdf888706b3b3b894df2aa6e3baa848a05719b..347ebf055f14b026e213c4872750552a41677038 100644
--- a/internal/log/logutil.go
+++ b/internal/logutil/logutil.go
@@ -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}
diff --git a/internal/timesync/time_sync_producer.go b/internal/timesync/time_sync_producer.go
index 1fc4288ff2ebf18c73c5f7556927f24544dc628d..6b661f3fc906de7539e833e66cf8033b53762b2c 100644
--- a/internal/timesync/time_sync_producer.go
+++ b/internal/timesync/time_sync_producer.go
@@ -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)
 }
diff --git a/internal/timesync/timesync.go b/internal/timesync/timesync.go
index c1ceccfb8851c4ce979223afb236425ec9f8ca4e..b3290d0e3268d321b5553200f47dc547fe53bd47 100644
--- a/internal/timesync/timesync.go
+++ b/internal/timesync/timesync.go
@@ -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