mirror of https://github.com/milvus-io/milvus.git
Add zap log to query node
Signed-off-by: bigsheeper <yihao.dai@zilliz.com>pull/4973/head^2
parent
801396e3ef
commit
447a15207e
|
@ -2,7 +2,6 @@ package components
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
|
||||
grpcquerynode "github.com/zilliztech/milvus-distributed/internal/distributed/querynode"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
|
@ -31,7 +30,6 @@ func (q *QueryNode) Run() error {
|
|||
if err := q.svr.Run(); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
log.Println("Query node successfully started ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -2,7 +2,6 @@ package components
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
|
||||
grpcqueryservice "github.com/zilliztech/milvus-distributed/internal/distributed/queryservice"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
|
@ -29,7 +28,6 @@ func (qs *QueryService) Run() error {
|
|||
if err := qs.svr.Run(); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
log.Println("QueryService successfully started ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -2,19 +2,31 @@ package main
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
distributed "github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/querynode"
|
||||
)
|
||||
|
||||
func main() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
querynode.Params.Init()
|
||||
logutil.SetupLogger(&querynode.Params.Log)
|
||||
defer func() {
|
||||
if err := log.Sync(); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}()
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
svr, err := distributed.NewQueryNode(ctx, msFactory)
|
||||
|
||||
|
@ -34,7 +46,7 @@ func main() {
|
|||
syscall.SIGQUIT)
|
||||
|
||||
sig := <-sc
|
||||
log.Print("Got signal to exit", sig.String())
|
||||
log.Debug("Get signal to exit", zap.String("signal", sig.String()))
|
||||
|
||||
if err := svr.Stop(); err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -19,7 +19,7 @@ NewCollection(const char* schema_proto_blob) {
|
|||
|
||||
auto collection = std::make_unique<milvus::segcore::Collection>(proto);
|
||||
|
||||
std::cout << "create collection " << collection->get_collection_name() << std::endl;
|
||||
// std::cout << "create collection " << collection->get_collection_name() << std::endl;
|
||||
|
||||
return (void*)collection.release();
|
||||
}
|
||||
|
@ -28,7 +28,7 @@ void
|
|||
DeleteCollection(CCollection collection) {
|
||||
auto col = (milvus::segcore::Collection*)collection;
|
||||
|
||||
std::cout << "delete collection " << col->get_collection_name() << std::endl;
|
||||
// std::cout << "delete collection " << col->get_collection_name() << std::endl;
|
||||
|
||||
delete col;
|
||||
}
|
||||
|
|
|
@ -86,12 +86,12 @@ void
|
|||
DeletePlan(CPlan cPlan) {
|
||||
auto plan = (milvus::query::Plan*)cPlan;
|
||||
delete plan;
|
||||
std::cout << "delete plan" << std::endl;
|
||||
// std::cout << "delete plan" << std::endl;
|
||||
}
|
||||
|
||||
void
|
||||
DeletePlaceholderGroup(CPlaceholderGroup cPlaceholder_group) {
|
||||
auto placeHolder_group = (milvus::query::PlaceholderGroup*)cPlaceholder_group;
|
||||
delete placeHolder_group;
|
||||
std::cout << "delete placeholder" << std::endl;
|
||||
// std::cout << "delete placeholder" << std::endl;
|
||||
}
|
||||
|
|
|
@ -43,7 +43,7 @@ NewSegment(CCollection collection, uint64_t segment_id, SegmentType seg_type) {
|
|||
std::cout << "invalid segment type" << std::endl;
|
||||
}
|
||||
|
||||
std::cout << "create segment " << segment_id << std::endl;
|
||||
// std::cout << "create segment " << segment_id << std::endl;
|
||||
return (void*)segment.release();
|
||||
}
|
||||
|
||||
|
@ -52,7 +52,7 @@ DeleteSegment(CSegmentInterface c_segment) {
|
|||
// TODO: use dynamic cast, and return c status
|
||||
auto s = (milvus::segcore::SegmentInterface*)c_segment;
|
||||
|
||||
std::cout << "delete segment " << std::endl;
|
||||
// std::cout << "delete segment " << std::endl;
|
||||
delete s;
|
||||
}
|
||||
|
||||
|
@ -219,8 +219,6 @@ LoadFieldData(CSegmentInterface c_segment, CLoadFieldDataInfo load_field_data_in
|
|||
auto load_info =
|
||||
LoadFieldDataInfo{load_field_data_info.field_id, load_field_data_info.blob, load_field_data_info.row_count};
|
||||
segment->LoadFieldData(load_info);
|
||||
std::cout << "load field done, field_id = " << load_info.field_id << ", row count = " << load_info.row_count
|
||||
<< std::endl;
|
||||
auto status = CStatus();
|
||||
status.error_code = Success;
|
||||
status.error_msg = "";
|
||||
|
|
|
@ -4,29 +4,29 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"log"
|
||||
"net"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
otgrpc "github.com/opentracing-contrib/go-grpc"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/uber/jaeger-client-go/config"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
dsc "github.com/zilliztech/milvus-distributed/internal/distributed/dataservice/client"
|
||||
isc "github.com/zilliztech/milvus-distributed/internal/distributed/indexservice/client"
|
||||
msc "github.com/zilliztech/milvus-distributed/internal/distributed/masterservice/client"
|
||||
qsc "github.com/zilliztech/milvus-distributed/internal/distributed/queryservice/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/funcutil"
|
||||
|
||||
otgrpc "github.com/opentracing-contrib/go-grpc"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/uber/jaeger-client-go/config"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
|
||||
qn "github.com/zilliztech/milvus-distributed/internal/querynode"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/funcutil"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
)
|
||||
|
||||
|
@ -85,7 +85,7 @@ func (s *Server) init() error {
|
|||
opentracing.SetGlobalTracer(tracer)
|
||||
s.closer = closer
|
||||
|
||||
log.Println("QueryNode, port:", Params.QueryNodePort)
|
||||
log.Debug("QueryNode", zap.Int("port", Params.QueryNodePort))
|
||||
s.wg.Add(1)
|
||||
go s.startGrpcLoop(Params.QueryNodePort)
|
||||
// wait for grpc server loop start
|
||||
|
@ -94,8 +94,8 @@ func (s *Server) init() error {
|
|||
return err
|
||||
}
|
||||
// --- QueryService ---
|
||||
log.Println("QueryService address:", Params.QueryServiceAddress)
|
||||
log.Println("Init Query service client ...")
|
||||
log.Debug("QueryService", zap.String("address", Params.QueryServiceAddress))
|
||||
log.Debug("Init Query service client ...")
|
||||
queryService, err := qsc.NewClient(Params.QueryServiceAddress, 20*time.Second)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
@ -121,8 +121,8 @@ func (s *Server) init() error {
|
|||
// --- Master Server Client ---
|
||||
//ms.Params.Init()
|
||||
addr := Params.MasterAddress
|
||||
log.Println("Master service address:", addr)
|
||||
log.Println("Init master service client ...")
|
||||
log.Debug("Master service", zap.String("address", addr))
|
||||
log.Debug("Init master service client ...")
|
||||
|
||||
masterService, err := msc.NewClient(addr, 20*time.Second)
|
||||
if err != nil {
|
||||
|
@ -147,7 +147,7 @@ func (s *Server) init() error {
|
|||
}
|
||||
|
||||
// --- IndexService ---
|
||||
log.Println("Index service address:", Params.IndexServiceAddress)
|
||||
log.Debug("Index service", zap.String("address", Params.IndexServiceAddress))
|
||||
indexService := isc.NewClient(Params.IndexServiceAddress)
|
||||
|
||||
if err := indexService.Init(); err != nil {
|
||||
|
@ -168,8 +168,8 @@ func (s *Server) init() error {
|
|||
}
|
||||
|
||||
// --- DataService ---
|
||||
log.Printf("Data service address: %s", Params.DataServiceAddress)
|
||||
log.Println("Querynode Init data service client ...")
|
||||
log.Debug("Data service", zap.String("address", Params.DataServiceAddress))
|
||||
log.Debug("QueryNode Init data service client ...")
|
||||
|
||||
dataService := dsc.NewClient(Params.DataServiceAddress)
|
||||
if err = dataService.Init(); err != nil {
|
||||
|
@ -195,7 +195,7 @@ func (s *Server) init() error {
|
|||
s.impl.UpdateStateCode(internalpb2.StateCode_INITIALIZING)
|
||||
|
||||
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
|
||||
|
@ -212,11 +212,11 @@ func (s *Server) startGrpcLoop(grpcPort int) {
|
|||
|
||||
lis, err := net.Listen("tcp", addr)
|
||||
if err != nil {
|
||||
log.Printf("QueryNode GrpcServer:failed to listen: %v", err)
|
||||
log.Error("QueryNode GrpcServer:failed to listen", zap.Error(err))
|
||||
s.grpcErrChan <- err
|
||||
return
|
||||
}
|
||||
log.Println("QueryNode:: addr:", addr)
|
||||
log.Debug("QueryNode", zap.String("address", addr))
|
||||
|
||||
tracer := opentracing.GlobalTracer()
|
||||
s.grpcServer = grpc.NewServer(grpc.UnaryInterceptor(
|
||||
|
@ -230,7 +230,7 @@ func (s *Server) startGrpcLoop(grpcPort int) {
|
|||
|
||||
go funcutil.CheckGrpcReady(ctx, s.grpcErrChan)
|
||||
if err := s.grpcServer.Serve(lis); err != nil {
|
||||
log.Println("QueryNode Start Grpc Failed!!!!")
|
||||
log.Debug("QueryNode Start Grpc Failed!!!!")
|
||||
s.grpcErrChan <- err
|
||||
}
|
||||
|
||||
|
@ -241,12 +241,12 @@ func (s *Server) Run() error {
|
|||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Println("querynode init done ...")
|
||||
log.Debug("QueryNode init done ...")
|
||||
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Println("querynode start done ...")
|
||||
log.Debug("QueryNode start done ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -13,7 +13,9 @@ package querynode
|
|||
import "C"
|
||||
import (
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type Collection struct {
|
||||
|
@ -61,6 +63,8 @@ func newCollection(collectionID UniqueID, schema *schemapb.CollectionSchema) *Co
|
|||
schema: schema,
|
||||
}
|
||||
|
||||
log.Debug("create collection", zap.Int64("collectionID", collectionID))
|
||||
|
||||
return newCollection
|
||||
}
|
||||
|
||||
|
@ -71,4 +75,10 @@ func deleteCollection(collection *Collection) {
|
|||
*/
|
||||
cPtr := collection.collectionPtr
|
||||
C.DeleteCollection(cPtr)
|
||||
|
||||
collection.collectionPtr = nil
|
||||
|
||||
log.Debug("delete collection", zap.Int64("collectionID", collection.ID()))
|
||||
|
||||
collection = nil
|
||||
}
|
||||
|
|
|
@ -12,11 +12,13 @@ package querynode
|
|||
*/
|
||||
import "C"
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"sync"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
)
|
||||
|
@ -392,7 +394,7 @@ func (colReplica *collectionReplicaImpl) removeSegment(segmentID UniqueID) error
|
|||
}
|
||||
|
||||
func (colReplica *collectionReplicaImpl) removeSegmentPrivate(segmentID UniqueID) error {
|
||||
fmt.Println("remove segment", segmentID)
|
||||
log.Debug("remove segment", zap.Int64("segmentID", segmentID))
|
||||
segment, err := colReplica.getSegmentByIDPrivate(segmentID)
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
|
@ -2,8 +2,10 @@ package querynode
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
|
||||
|
@ -66,7 +68,7 @@ func (dsService *dataSyncService) initNodes() {
|
|||
[]string{filterDmNode.Name()},
|
||||
)
|
||||
if err != nil {
|
||||
log.Fatal("set edges failed in node:", dmStreamNode.Name())
|
||||
log.Error("set edges failed in node:", zap.String("node name", dmStreamNode.Name()))
|
||||
}
|
||||
|
||||
// filterDmNode
|
||||
|
@ -75,7 +77,7 @@ func (dsService *dataSyncService) initNodes() {
|
|||
[]string{insertNode.Name()},
|
||||
)
|
||||
if err != nil {
|
||||
log.Fatal("set edges failed in node:", filterDmNode.Name())
|
||||
log.Error("set edges failed in node:", zap.String("node name", filterDmNode.Name()))
|
||||
}
|
||||
|
||||
// insertNode
|
||||
|
@ -84,7 +86,7 @@ func (dsService *dataSyncService) initNodes() {
|
|||
[]string{serviceTimeNode.Name()},
|
||||
)
|
||||
if err != nil {
|
||||
log.Fatal("set edges failed in node:", insertNode.Name())
|
||||
log.Error("set edges failed in node:", zap.String("node name", insertNode.Name()))
|
||||
}
|
||||
|
||||
// serviceTimeNode
|
||||
|
@ -93,7 +95,7 @@ func (dsService *dataSyncService) initNodes() {
|
|||
[]string{},
|
||||
)
|
||||
if err != nil {
|
||||
log.Fatal("set edges failed in node:", serviceTimeNode.Name())
|
||||
log.Error("set edges failed in node:", zap.String("node name", serviceTimeNode.Name()))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -2,10 +2,11 @@ package querynode
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
)
|
||||
|
@ -21,16 +22,16 @@ func (ddNode *ddNode) Name() string {
|
|||
}
|
||||
|
||||
func (ddNode *ddNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.Context) {
|
||||
//fmt.Println("Do filterDmNode operation")
|
||||
//log.Debug("Do filterDmNode operation")
|
||||
|
||||
if len(in) != 1 {
|
||||
log.Println("Invalid operate message input in ddNode, input length = ", len(in))
|
||||
log.Error("Invalid operate message input in ddNode", zap.Int("input length", len(in)))
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
msMsg, ok := in[0].(*MsgStreamMsg)
|
||||
if !ok {
|
||||
log.Println("type assertion failed for MsgStreamMsg")
|
||||
log.Error("type assertion failed for MsgStreamMsg")
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
|
@ -81,21 +82,21 @@ func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) {
|
|||
|
||||
hasCollection := ddNode.replica.hasCollection(collectionID)
|
||||
if hasCollection {
|
||||
log.Println("collection already exists, id = ", collectionID)
|
||||
log.Debug("collection already exists", zap.Int64("collectionID", collectionID))
|
||||
return
|
||||
}
|
||||
|
||||
var schema schemapb.CollectionSchema
|
||||
err := proto.Unmarshal(msg.Schema, &schema)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
// add collection
|
||||
err = ddNode.replica.addCollection(collectionID, &schema)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -103,7 +104,7 @@ func (ddNode *ddNode) createCollection(msg *msgstream.CreateCollectionMsg) {
|
|||
// TODO: allocate default partition id in master
|
||||
err = ddNode.replica.addPartition(collectionID, UniqueID(2021))
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -132,7 +133,7 @@ func (ddNode *ddNode) createPartition(msg *msgstream.CreatePartitionMsg) {
|
|||
|
||||
err := ddNode.replica.addPartition(collectionID, partitionID)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
|
|
|
@ -2,8 +2,10 @@ package querynode
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
)
|
||||
|
@ -18,16 +20,16 @@ func (fdmNode *filterDmNode) Name() string {
|
|||
}
|
||||
|
||||
func (fdmNode *filterDmNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.Context) {
|
||||
//fmt.Println("Do filterDmNode operation")
|
||||
//log.Debug("Do filterDmNode operation")
|
||||
|
||||
if len(in) != 1 {
|
||||
log.Println("Invalid operate message input in filterDmNode, input length = ", len(in))
|
||||
log.Error("Invalid operate message input in filterDmNode", zap.Int("input length", len(in)))
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
msgStreamMsg, ok := in[0].(*MsgStreamMsg)
|
||||
if !ok {
|
||||
log.Println("type assertion failed for MsgStreamMsg")
|
||||
log.Error("type assertion failed for MsgStreamMsg")
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
|
@ -48,7 +50,7 @@ func (fdmNode *filterDmNode) Operate(ctx context.Context, in []Msg) ([]Msg, cont
|
|||
// case commonpb.MsgType_kDelete:
|
||||
// dmMsg.deleteMessages = append(dmMsg.deleteMessages, (*msg).(*msgstream.DeleteTask))
|
||||
default:
|
||||
log.Println("Non supporting message type:", msg.Type())
|
||||
log.Warn("Non supporting", zap.Int32("message type", int32(msg.Type())))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -74,7 +76,7 @@ func (fdmNode *filterDmNode) filterInvalidInsertMessage(msg *msgstream.InsertMsg
|
|||
// Filter insert requests before last record.
|
||||
if len(msg.RowIDs) != len(msg.Timestamps) || len(msg.RowIDs) != len(msg.RowData) {
|
||||
// TODO: what if the messages are misaligned? Here, we ignore those messages and print error
|
||||
log.Println("Error, misaligned messages detected")
|
||||
log.Error("Error, misaligned messages detected")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -2,7 +2,10 @@ package querynode
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
)
|
||||
|
||||
type gcNode struct {
|
||||
|
@ -15,16 +18,16 @@ func (gcNode *gcNode) Name() string {
|
|||
}
|
||||
|
||||
func (gcNode *gcNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.Context) {
|
||||
//fmt.Println("Do gcNode operation")
|
||||
//log.Debug("Do gcNode operation")
|
||||
|
||||
if len(in) != 1 {
|
||||
log.Println("Invalid operate message input in gcNode, input length = ", len(in))
|
||||
log.Error("Invalid operate message input in gcNode", zap.Int("input length", len(in)))
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
_, ok := in[0].(*gcMsg)
|
||||
if !ok {
|
||||
log.Println("type assertion failed for gcMsg")
|
||||
log.Error("type assertion failed for gcMsg")
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
|
|
|
@ -2,10 +2,11 @@ package querynode
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"sync"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
)
|
||||
|
||||
|
@ -27,16 +28,16 @@ func (iNode *insertNode) Name() string {
|
|||
}
|
||||
|
||||
func (iNode *insertNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.Context) {
|
||||
// fmt.Println("Do insertNode operation")
|
||||
//log.Debug("Do insertNode operation")
|
||||
|
||||
if len(in) != 1 {
|
||||
log.Println("Invalid operate message input in insertNode, input length = ", len(in))
|
||||
log.Error("Invalid operate message input in insertNode", zap.Int("input length", len(in)))
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
iMsg, ok := in[0].(*insertMsg)
|
||||
if !ok {
|
||||
log.Println("type assertion failed for insertMsg")
|
||||
log.Error("type assertion failed for insertMsg")
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
|
@ -57,7 +58,7 @@ func (iNode *insertNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.
|
|||
if !iNode.replica.hasSegment(task.SegmentID) {
|
||||
err := iNode.replica.addSegment(task.SegmentID, task.PartitionID, task.CollectionID, segTypeGrowing)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
@ -67,7 +68,7 @@ func (iNode *insertNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.
|
|||
for segmentID := range insertData.insertRecords {
|
||||
var targetSegment, err = iNode.replica.getSegmentByID(segmentID)
|
||||
if err != nil {
|
||||
log.Println("preInsert failed")
|
||||
log.Error("preInsert failed")
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
|
@ -96,7 +97,7 @@ func (iNode *insertNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.
|
|||
func (iNode *insertNode) insert(insertData *InsertData, segmentID int64, wg *sync.WaitGroup) {
|
||||
var targetSegment, err = iNode.replica.getSegmentByID(segmentID)
|
||||
if err != nil {
|
||||
log.Println("cannot find segment:", segmentID)
|
||||
log.Error("cannot find segment:", zap.Int64("segmentID", segmentID))
|
||||
// TODO: add error handling
|
||||
wg.Done()
|
||||
return
|
||||
|
@ -109,13 +110,13 @@ func (iNode *insertNode) insert(insertData *InsertData, segmentID int64, wg *syn
|
|||
|
||||
err = targetSegment.segmentInsert(offsets, &ids, ×tamps, &records)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
// TODO: add error handling
|
||||
wg.Done()
|
||||
return
|
||||
}
|
||||
|
||||
fmt.Println("Do insert done, len = ", len(insertData.insertIDs[segmentID]))
|
||||
log.Debug("Do insert done", zap.Int("len", len(insertData.insertIDs[segmentID])))
|
||||
wg.Done()
|
||||
}
|
||||
|
||||
|
|
|
@ -2,8 +2,10 @@ package querynode
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
|
@ -20,25 +22,25 @@ func (stNode *serviceTimeNode) Name() string {
|
|||
}
|
||||
|
||||
func (stNode *serviceTimeNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.Context) {
|
||||
//fmt.Println("Do serviceTimeNode operation")
|
||||
//log.Debug("Do serviceTimeNode operation")
|
||||
|
||||
if len(in) != 1 {
|
||||
log.Println("Invalid operate message input in serviceTimeNode, input length = ", len(in))
|
||||
log.Error("Invalid operate message input in serviceTimeNode, input length = ", zap.Int("input node", len(in)))
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
serviceTimeMsg, ok := in[0].(*serviceTimeMsg)
|
||||
if !ok {
|
||||
log.Println("type assertion failed for serviceTimeMsg")
|
||||
log.Error("type assertion failed for serviceTimeMsg")
|
||||
// TODO: add error handling
|
||||
}
|
||||
|
||||
// update service time
|
||||
stNode.replica.getTSafe().set(serviceTimeMsg.timeRange.timestampMax)
|
||||
//fmt.Println("update tSafe to:", getPhysicalTime(serviceTimeMsg.timeRange.timestampMax))
|
||||
//log.Debug("update tSafe to:", getPhysicalTime(serviceTimeMsg.timeRange.timestampMax))
|
||||
|
||||
if err := stNode.sendTimeTick(serviceTimeMsg.timeRange.timestampMax); err != nil {
|
||||
log.Printf("Error: send time tick into pulsar channel failed, %s\n", err.Error())
|
||||
log.Error("Error: send time tick into pulsar channel failed", zap.Error(err))
|
||||
}
|
||||
|
||||
var res Msg = &gcMsg{
|
||||
|
|
|
@ -4,7 +4,6 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"log"
|
||||
"path"
|
||||
"sort"
|
||||
"strconv"
|
||||
|
@ -12,8 +11,11 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||
minioKV "github.com/zilliztech/milvus-distributed/internal/kv/minio"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/util"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
|
||||
|
@ -46,28 +48,28 @@ func (loader *indexLoader) doLoadIndex(wg *sync.WaitGroup) {
|
|||
wg.Done()
|
||||
return
|
||||
}
|
||||
fmt.Println("do load index for sealed segments:", segmentIDs)
|
||||
log.Debug("do load index for sealed segments:", zap.String("segmentIDs", fmt.Sprintln(segmentIDs)))
|
||||
for i := range collectionIDs {
|
||||
// we don't need index id yet
|
||||
_, buildID, err := loader.getIndexInfo(collectionIDs[i], segmentIDs[i])
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Warn(err.Error())
|
||||
continue
|
||||
}
|
||||
indexPaths, err := loader.getIndexPaths(buildID)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Warn(err.Error())
|
||||
continue
|
||||
}
|
||||
err = loader.loadIndexDelayed(collectionIDs[i], segmentIDs[i], indexPaths)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Warn(err.Error())
|
||||
}
|
||||
}
|
||||
// sendQueryNodeStats
|
||||
err := loader.sendQueryNodeStats()
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
wg.Done()
|
||||
return
|
||||
}
|
||||
|
@ -111,14 +113,14 @@ func (loader *indexLoader) execute(l *loadIndex) error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
fmt.Println("load index done")
|
||||
log.Debug("load index done")
|
||||
return nil
|
||||
}
|
||||
|
||||
func (loader *indexLoader) printIndexParams(index []*commonpb.KeyValuePair) {
|
||||
fmt.Println("=================================================")
|
||||
log.Debug("=================================================")
|
||||
for i := 0; i < len(index); i++ {
|
||||
fmt.Println(index[i])
|
||||
log.Debug(fmt.Sprintln(index[i]))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -216,7 +218,7 @@ func (loader *indexLoader) loadIndex(indexPath []string) ([][]byte, indexParam,
|
|||
var indexName string
|
||||
var indexID UniqueID
|
||||
for _, p := range indexPath {
|
||||
fmt.Println("load path = ", indexPath)
|
||||
log.Debug("", zap.String("load path", fmt.Sprintln(indexPath)))
|
||||
indexPiece, err := loader.kv.Load(p)
|
||||
if err != nil {
|
||||
return nil, nil, "", -1, err
|
||||
|
@ -288,7 +290,7 @@ func (loader *indexLoader) sendQueryNodeStats() error {
|
|||
}
|
||||
|
||||
loader.fieldStatsChan <- resultFieldsStats
|
||||
fmt.Println("sent field stats")
|
||||
log.Debug("sent field stats")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -10,10 +10,13 @@ package querynode
|
|||
import "C"
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
"unsafe"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
)
|
||||
|
||||
type LoadIndexInfo struct {
|
||||
|
@ -79,7 +82,7 @@ func (li *LoadIndexInfo) appendIndex(bytesIndex [][]byte, indexKeys []string) er
|
|||
indexPtr := unsafe.Pointer(&byteIndex[0])
|
||||
indexLen := C.long(len(byteIndex))
|
||||
binarySetKey := filepath.Base(indexKeys[i])
|
||||
fmt.Println("index key = ", binarySetKey)
|
||||
log.Debug("", zap.String("index key", binarySetKey))
|
||||
indexKey := C.CString(binarySetKey)
|
||||
status = C.AppendBinaryIndex(cBinarySet, indexPtr, indexLen, indexKey)
|
||||
errorCode = status.error_code
|
||||
|
|
|
@ -3,11 +3,13 @@ package querynode
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
)
|
||||
|
@ -47,22 +49,22 @@ func (s *loadService) loadSegmentActively(wg *sync.WaitGroup) {
|
|||
wg.Done()
|
||||
return
|
||||
}
|
||||
fmt.Println("do load segment for growing segments:", segmentIDs)
|
||||
log.Debug("do load segment for growing segments:", zap.String("segmentIDs", fmt.Sprintln(segmentIDs)))
|
||||
for i := range collectionIDs {
|
||||
fieldIDs, err := s.segLoader.replica.getFieldIDsByCollectionID(collectionIDs[i])
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
continue
|
||||
}
|
||||
err = s.loadSegmentInternal(collectionIDs[i], partitionIDs[i], segmentIDs[i], fieldIDs)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
}
|
||||
// sendQueryNodeStats
|
||||
err := s.segLoader.indexLoader.sendQueryNodeStats()
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
wg.Done()
|
||||
return
|
||||
}
|
||||
|
@ -83,12 +85,12 @@ func (s *loadService) loadSegment(collectionID UniqueID, partitionID UniqueID, s
|
|||
for _, segmentID := range segmentIDs {
|
||||
err := s.segLoader.replica.addSegment(segmentID, partitionID, collectionID, segTypeGrowing)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Warn(err.Error())
|
||||
continue
|
||||
}
|
||||
err = s.loadSegmentInternal(collectionID, partitionID, segmentID, fieldIDs)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Warn(err.Error())
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
@ -129,8 +131,8 @@ func (s *loadService) loadSegmentInternal(collectionID UniqueID, partitionID Uni
|
|||
return err
|
||||
}
|
||||
|
||||
//fmt.Println("srcFieldIDs in internal:", srcFieldIDs)
|
||||
//fmt.Println("dstFieldIDs in internal:", fieldIDs)
|
||||
//log.Debug("srcFieldIDs in internal:", srcFieldIDs)
|
||||
//log.Debug("dstFieldIDs in internal:", fieldIDs)
|
||||
targetFields, err := s.segLoader.checkTargetFields(paths, srcFieldIDs, fieldIDs)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -145,7 +147,7 @@ func (s *loadService) loadSegmentInternal(collectionID UniqueID, partitionID Uni
|
|||
return err
|
||||
}
|
||||
if errIndex == nil {
|
||||
fmt.Println("loading index...")
|
||||
log.Debug("loading index...")
|
||||
indexPaths, err := s.segLoader.indexLoader.getIndexPaths(buildID)
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
|
@ -3,18 +3,20 @@ package querynode
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"path"
|
||||
"reflect"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/retry"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -60,11 +62,11 @@ func (mService *metaService) start() {
|
|||
// init from meta
|
||||
err := mService.loadCollections()
|
||||
if err != nil {
|
||||
log.Fatal("metaService loadCollections failed")
|
||||
log.Error("metaService loadCollections failed")
|
||||
}
|
||||
err = mService.loadSegments()
|
||||
if err != nil {
|
||||
log.Fatal("metaService loadSegments failed")
|
||||
log.Error("metaService loadSegments failed")
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -111,7 +113,7 @@ func printCollectionStruct(obj *etcdpb.CollectionInfo) {
|
|||
if typeOfS.Field(i).Name == "GrpcMarshalString" {
|
||||
continue
|
||||
}
|
||||
fmt.Printf("Field: %s\tValue: %v\n", typeOfS.Field(i).Name, v.Field(i).Interface())
|
||||
log.Debug("Field", zap.String("field name", typeOfS.Field(i).Name), zap.String("field", fmt.Sprintln(v.Field(i).Interface())))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -121,24 +123,22 @@ func printSegmentStruct(obj *datapb.SegmentInfo) {
|
|||
typeOfS := v.Type()
|
||||
|
||||
for i := 0; i < v.NumField(); i++ {
|
||||
fmt.Printf("Field: %s\tValue: %v\n", typeOfS.Field(i).Name, v.Field(i).Interface())
|
||||
log.Debug("Field", zap.String("field name", typeOfS.Field(i).Name), zap.String("field", fmt.Sprintln(v.Field(i).Interface())))
|
||||
}
|
||||
}
|
||||
|
||||
func (mService *metaService) processCollectionCreate(id string, value string) {
|
||||
//println(fmt.Sprintf("Create Collection:$%s$", id))
|
||||
|
||||
col := mService.collectionUnmarshal(value)
|
||||
if col != nil {
|
||||
schema := col.Schema
|
||||
err := mService.replica.addCollection(col.ID, schema)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
for _, partitionID := range col.PartitionIDs {
|
||||
err = mService.replica.addPartition(col.ID, partitionID)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -154,7 +154,7 @@ func (mService *metaService) processSegmentCreate(id string, value string) {
|
|||
// TODO: get partition id from segment meta
|
||||
err := mService.replica.addSegment(seg.SegmentID, seg.PartitionID, seg.CollectionID, segTypeGrowing)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
return
|
||||
}
|
||||
}
|
||||
|
@ -206,7 +206,7 @@ func (mService *metaService) collectionUnmarshal(value string) *etcdpb.Collectio
|
|||
col := etcdpb.CollectionInfo{}
|
||||
err := proto.UnmarshalText(value, &col)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
return nil
|
||||
}
|
||||
return &col
|
||||
|
@ -215,7 +215,7 @@ func (mService *metaService) collectionUnmarshal(value string) *etcdpb.Collectio
|
|||
func (mService *metaService) collectionMarshal(col *etcdpb.CollectionInfo) string {
|
||||
value := proto.MarshalTextString(col)
|
||||
if value == "" {
|
||||
log.Println("marshal collection failed")
|
||||
log.Error("marshal collection failed")
|
||||
return ""
|
||||
}
|
||||
return value
|
||||
|
@ -225,7 +225,7 @@ func (mService *metaService) segmentUnmarshal(value string) *datapb.SegmentInfo
|
|||
seg := datapb.SegmentInfo{}
|
||||
err := proto.UnmarshalText(value, &seg)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
return nil
|
||||
}
|
||||
return &seg
|
||||
|
@ -234,7 +234,7 @@ func (mService *metaService) segmentUnmarshal(value string) *datapb.SegmentInfo
|
|||
func (mService *metaService) segmentMarshal(seg *etcdpb.SegmentMeta) string {
|
||||
value := proto.MarshalTextString(seg)
|
||||
if value == "" {
|
||||
log.Println("marshal segment failed")
|
||||
log.Error("marshal segment failed")
|
||||
return ""
|
||||
}
|
||||
return value
|
||||
|
|
|
@ -1,11 +1,13 @@
|
|||
package querynode
|
||||
|
||||
import (
|
||||
"log"
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
"strconv"
|
||||
"sync"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
|
||||
)
|
||||
|
||||
|
@ -60,6 +62,8 @@ type ParamTable struct {
|
|||
GracefulTime int64
|
||||
MsgChannelSubName string
|
||||
SliceIndex int
|
||||
|
||||
Log log.Config
|
||||
}
|
||||
|
||||
var Params ParamTable
|
||||
|
@ -129,6 +133,8 @@ func (p *ParamTable) Init() {
|
|||
p.initStatsPublishInterval()
|
||||
//p.initStatsChannelName()
|
||||
p.initStatsReceiveBufSize()
|
||||
|
||||
p.initLogCfg()
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -152,7 +158,7 @@ func (p *ParamTable) initQueryNodeNum() {
|
|||
func (p *ParamTable) initQueryTimeTickChannelName() {
|
||||
ch, err := p.Load("msgChannel.chanNamePrefix.queryTimeTick")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
p.QueryTimeTickChannelName = ch
|
||||
}
|
||||
|
@ -322,7 +328,7 @@ func (p *ParamTable) initInsertChannelNames() {
|
|||
|
||||
prefix, err := p.Load("msgChannel.chanNamePrefix.insert")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
prefix += "-"
|
||||
channelRange, err := p.Load("msgChannel.channelRange.insert")
|
||||
|
@ -347,7 +353,7 @@ func (p *ParamTable) initInsertChannelNames() {
|
|||
func (p *ParamTable) initSearchChannelNames() {
|
||||
prefix, err := p.Load("msgChannel.chanNamePrefix.search")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
prefix += "-"
|
||||
channelRange, err := p.Load("msgChannel.channelRange.search")
|
||||
|
@ -367,7 +373,7 @@ func (p *ParamTable) initSearchChannelNames() {
|
|||
func (p *ParamTable) initSearchResultChannelNames() {
|
||||
prefix, err := p.Load("msgChannel.chanNamePrefix.searchResult")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
prefix += "-"
|
||||
channelRange, err := p.Load("msgChannel.channelRange.searchResult")
|
||||
|
@ -388,7 +394,7 @@ func (p *ParamTable) initMsgChannelSubName() {
|
|||
// TODO: subName = namePrefix + "-" + queryNodeID, queryNodeID is assigned by master
|
||||
name, err := p.Load("msgChannel.subNamePrefix.queryNodeSubNamePrefix")
|
||||
if err != nil {
|
||||
log.Panic(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
queryNodeIDStr, err := p.Load("_QueryNodeID")
|
||||
if err != nil {
|
||||
|
@ -434,3 +440,34 @@ func (p *ParamTable) initSliceIndex() {
|
|||
}
|
||||
p.SliceIndex = -1
|
||||
}
|
||||
|
||||
func (p *ParamTable) initLogCfg() {
|
||||
p.Log = log.Config{}
|
||||
format, err := p.Load("log.format")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.Log.Format = format
|
||||
level, err := p.Load("log.level")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.Log.Level = level
|
||||
devStr, err := p.Load("log.dev")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
dev, err := strconv.ParseBool(devStr)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.Log.Development = dev
|
||||
p.Log.File.MaxSize = p.ParseInt("log.file.maxSize")
|
||||
p.Log.File.MaxBackups = p.ParseInt("log.file.maxBackups")
|
||||
p.Log.File.MaxDays = p.ParseInt("log.file.maxAge")
|
||||
rootPath, err := p.Load("log.file.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.Log.File.Filename = path.Join(rootPath, fmt.Sprintf("querynode-%d.log", p.QueryNodeID))
|
||||
}
|
||||
|
|
|
@ -11,7 +11,11 @@ package querynode
|
|||
|
||||
*/
|
||||
import "C"
|
||||
import "fmt"
|
||||
import (
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
)
|
||||
|
||||
type Partition struct {
|
||||
collectionID UniqueID
|
||||
|
@ -45,6 +49,6 @@ func newPartition(collectionID UniqueID, partitionID UniqueID) *Partition {
|
|||
enable: false,
|
||||
}
|
||||
|
||||
fmt.Println("create partition", partitionID)
|
||||
log.Debug("create partition", zap.Int64("partitionID", partitionID))
|
||||
return newPartition
|
||||
}
|
||||
|
|
|
@ -15,10 +15,12 @@ import "C"
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"sync/atomic"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"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"
|
||||
|
@ -146,18 +148,18 @@ func (node *QueryNode) Init() error {
|
|||
}
|
||||
}
|
||||
|
||||
fmt.Println("QueryNodeID is", Params.QueryNodeID)
|
||||
log.Debug("", zap.Int64("QueryNodeID", Params.QueryNodeID))
|
||||
|
||||
if node.masterClient == nil {
|
||||
log.Println("WARN: null master service detected")
|
||||
log.Error("null master service detected")
|
||||
}
|
||||
|
||||
if node.indexClient == nil {
|
||||
log.Println("WARN: null index service detected")
|
||||
log.Error("null index service detected")
|
||||
}
|
||||
|
||||
if node.dataClient == nil {
|
||||
log.Println("WARN: null data service detected")
|
||||
log.Error("null data service detected")
|
||||
}
|
||||
|
||||
return nil
|
||||
|
@ -403,7 +405,7 @@ func (node *QueryNode) LoadSegments(in *queryPb.LoadSegmentRequest) (*commonpb.S
|
|||
fieldIDs := in.FieldIDs
|
||||
schema := in.Schema
|
||||
|
||||
fmt.Println("query node load segment ,info = ", in)
|
||||
log.Debug("query node load segment", zap.String("loadSegmentRequest", fmt.Sprintln(in)))
|
||||
|
||||
status := &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
|
|
|
@ -4,13 +4,14 @@ import "C"
|
|||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"log"
|
||||
"regexp"
|
||||
"strconv"
|
||||
"sync"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
|
@ -137,14 +138,14 @@ func (ss *searchService) receiveSearchMsg() {
|
|||
for _, msg := range searchMsg {
|
||||
err := ss.search(msg)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
err2 := ss.publishFailedSearchResult(msg, err.Error())
|
||||
if err2 != nil {
|
||||
log.Println("publish FailedSearchResult failed, error message: ", err2)
|
||||
log.Error("publish FailedSearchResult failed", zap.Error(err2))
|
||||
}
|
||||
}
|
||||
}
|
||||
log.Println("ReceiveSearchMsg, do search done, num of searchMsg = ", len(searchMsg))
|
||||
log.Debug("ReceiveSearchMsg, do search done", zap.Int("num of searchMsg", len(searchMsg)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -189,14 +190,14 @@ func (ss *searchService) doUnsolvedMsgSearch() {
|
|||
for _, msg := range searchMsg {
|
||||
err := ss.search(msg)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
err2 := ss.publishFailedSearchResult(msg, err.Error())
|
||||
if err2 != nil {
|
||||
log.Println("publish FailedSearchResult failed, error message: ", err2)
|
||||
log.Error("publish FailedSearchResult failed", zap.Error(err2))
|
||||
}
|
||||
}
|
||||
}
|
||||
log.Println("doUnsolvedMsgSearch, do search done, num of searchMsg = ", len(searchMsg))
|
||||
log.Debug("doUnsolvedMsgSearch, do search done", zap.Int("num of searchMsg", len(searchMsg)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -237,7 +238,7 @@ func (ss *searchService) search(msg msgstream.TsMsg) error {
|
|||
searchResults := make([]*SearchResult, 0)
|
||||
matchedSegments := make([]*Segment, 0)
|
||||
|
||||
//fmt.Println("search msg's partitionID = ", partitionIDsInQuery)
|
||||
//log.Debug("search msg's partitionID = ", partitionIDsInQuery)
|
||||
partitionIDsInCol, err := ss.replica.getPartitionIDs(collectionID)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -263,7 +264,7 @@ func (ss *searchService) search(msg msgstream.TsMsg) error {
|
|||
return err
|
||||
}
|
||||
for _, segmentID := range segmentIDs {
|
||||
//fmt.Println("dsl = ", dsl)
|
||||
//log.Debug("dsl = ", dsl)
|
||||
segment, err := ss.replica.getSegmentByID(segmentID)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -349,7 +350,7 @@ func (ss *searchService) search(msg msgstream.TsMsg) error {
|
|||
//if err != nil {
|
||||
// return err
|
||||
//}
|
||||
//fmt.Println("hits msg = ", unMarshaledHit)
|
||||
//log.Debug("hits msg = ", unMarshaledHit)
|
||||
offset += len
|
||||
}
|
||||
resultChannelInt, _ := strconv.ParseInt(searchMsg.ResultChannelID, 10, 64)
|
||||
|
@ -376,8 +377,8 @@ func (ss *searchService) search(msg msgstream.TsMsg) error {
|
|||
// if err != nil {
|
||||
// panic(err)
|
||||
// }
|
||||
// fmt.Println(testHits.IDs)
|
||||
// fmt.Println(testHits.Scores)
|
||||
// log.Debug(testHits.IDs)
|
||||
// log.Debug(testHits.Scores)
|
||||
//}
|
||||
err = ss.publishSearchResult(searchResultMsg)
|
||||
if err != nil {
|
||||
|
@ -406,7 +407,7 @@ func (ss *searchService) publishFailedSearchResult(msg msgstream.TsMsg, errMsg s
|
|||
// span, ctx := opentracing.StartSpanFromContext(msg.GetMsgContext(), "receive search msg")
|
||||
// defer span.Finish()
|
||||
// msg.SetMsgContext(ctx)
|
||||
//fmt.Println("Public fail SearchResult!")
|
||||
//log.Debug("Public fail SearchResult!")
|
||||
msgPack := msgstream.MsgPack{}
|
||||
searchMsg, ok := msg.(*msgstream.SearchMsg)
|
||||
if !ok {
|
||||
|
|
|
@ -12,15 +12,15 @@ package querynode
|
|||
*/
|
||||
import "C"
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"strconv"
|
||||
"sync"
|
||||
"unsafe"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
)
|
||||
|
||||
|
@ -117,17 +117,19 @@ func newSegment(collection *Collection, segmentID int64, partitionID UniqueID, c
|
|||
var segmentPtr C.CSegmentInterface
|
||||
switch segType {
|
||||
case segTypeInvalid:
|
||||
log.Println("illegal segment type when create segment")
|
||||
log.Error("illegal segment type when create segment")
|
||||
return nil
|
||||
case segTypeSealed:
|
||||
segmentPtr = C.NewSegment(collection.collectionPtr, C.ulong(segmentID), C.Sealed)
|
||||
case segTypeGrowing:
|
||||
segmentPtr = C.NewSegment(collection.collectionPtr, C.ulong(segmentID), C.Growing)
|
||||
default:
|
||||
log.Println("illegal segment type when create segment")
|
||||
log.Error("illegal segment type when create segment")
|
||||
return nil
|
||||
}
|
||||
|
||||
log.Debug("create segment", zap.Int64("segmentID", segmentID))
|
||||
|
||||
var newSegment = &Segment{
|
||||
segmentPtr: segmentPtr,
|
||||
segmentType: segType,
|
||||
|
@ -148,6 +150,9 @@ func deleteSegment(segment *Segment) {
|
|||
cPtr := segment.segmentPtr
|
||||
C.DeleteSegment(cPtr)
|
||||
segment.segmentPtr = nil
|
||||
|
||||
log.Debug("delete segment", zap.Int64("segmentID", segment.ID()))
|
||||
|
||||
segment = nil
|
||||
}
|
||||
|
||||
|
@ -213,7 +218,7 @@ func (s *Segment) segmentSearch(plan *Plan,
|
|||
var cPlaceHolder = (*C.CPlaceholderGroup)(&cPlaceholderGroups[0])
|
||||
var cNumGroups = C.int(len(placeHolderGroups))
|
||||
|
||||
fmt.Println("do search on segment, ", s.segmentID, ", segType = ", s.segmentType)
|
||||
log.Debug("do search on segment", zap.Int64("segmentID", s.segmentID), zap.Int("segType", s.segmentType))
|
||||
var status = C.Search(s.segmentPtr, plan.cPlan, cPlaceHolder, cTimestamp, cNumGroups, &searchResult.cQueryResult)
|
||||
errorCode := status.error_code
|
||||
|
||||
|
@ -500,5 +505,7 @@ func (s *Segment) segmentLoadFieldData(fieldID int64, rowCount int, data interfa
|
|||
return errors.New("LoadFieldData failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
|
||||
}
|
||||
|
||||
log.Debug("load field done", zap.Int64("fieldID", fieldID), zap.Int("row count", rowCount))
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -2,11 +2,9 @@ package querynode
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
|
@ -51,7 +49,6 @@ func (sService *statsService) start() {
|
|||
sService.statsStream.Start()
|
||||
|
||||
// start service
|
||||
fmt.Println("do segments statistic in ", strconv.Itoa(sleepTimeInterval), "ms")
|
||||
for {
|
||||
select {
|
||||
case <-sService.ctx.Done():
|
||||
|
@ -94,6 +91,6 @@ func (sService *statsService) publicStatistic(fieldStats []*internalpb2.FieldSta
|
|||
}
|
||||
err := sService.statsStream.Produce(context.TODO(), &msgPack)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
log.Error(err.Error())
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue