yihao.dai 8537f3daeb
enhance: Rename Compaction to CompactionV2 (#33858)
Due to the removal of injection and syncSegments from the compaction, we
need to ensure that no compaction is successfully executed during the
rolling upgrade. This PR renames Compaction to CompactionV2, with the
following effects:
- New datacoord + old datanode: Utilizes the CompactionV2 interface,
resulting in the datanode error "CompactionV2 not implemented," causing
compaction to fail;
- Old datacoord + new datanode: Utilizes the CompactionV1 interface,
resulting in the datanode error "CompactionV1 not implemented," causing
compaction to fail.

issue: https://github.com/milvus-io/milvus/issues/32809

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
2024-06-16 22:07:57 +08:00

413 lines
14 KiB
Go

// 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 grpcdatanode
import (
"context"
"fmt"
"net"
"strconv"
"sync"
"time"
grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
clientv3 "go.etcd.io/etcd/client/v3"
"go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc"
"go.uber.org/atomic"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/keepalive"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
dn "github.com/milvus-io/milvus/internal/datanode"
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
"github.com/milvus-io/milvus/internal/distributed/utils"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/componentutil"
"github.com/milvus-io/milvus/internal/util/dependency"
_ "github.com/milvus-io/milvus/internal/util/grpcclient"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/tracer"
"github.com/milvus-io/milvus/pkg/util/etcd"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/interceptor"
"github.com/milvus-io/milvus/pkg/util/logutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry"
)
type Server struct {
datanode types.DataNodeComponent
grpcWG sync.WaitGroup
grpcErrChan chan error
grpcServer *grpc.Server
ctx context.Context
cancel context.CancelFunc
etcdCli *clientv3.Client
factory dependency.Factory
serverID atomic.Int64
rootCoord types.RootCoord
dataCoord types.DataCoord
newRootCoordClient func() (types.RootCoordClient, error)
newDataCoordClient func() (types.DataCoordClient, error)
}
// NewServer new DataNode grpc server
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
ctx1, cancel := context.WithCancel(ctx)
s := &Server{
ctx: ctx1,
cancel: cancel,
factory: factory,
grpcErrChan: make(chan error),
newRootCoordClient: func() (types.RootCoordClient, error) {
return rcc.NewClient(ctx1)
},
newDataCoordClient: func() (types.DataCoordClient, error) {
return dcc.NewClient(ctx1)
},
}
s.serverID.Store(paramtable.GetNodeID())
s.datanode = dn.NewDataNode(s.ctx, s.factory)
return s, nil
}
func (s *Server) startGrpc() error {
Params := &paramtable.Get().DataNodeGrpcServerCfg
s.grpcWG.Add(1)
go s.startGrpcLoop(Params.Port.GetAsInt())
// wait for grpc server loop start
err := <-s.grpcErrChan
return err
}
// startGrpcLoop starts the grep loop of datanode component.
func (s *Server) startGrpcLoop(grpcPort int) {
defer s.grpcWG.Done()
Params := &paramtable.Get().DataNodeGrpcServerCfg
kaep := keepalive.EnforcementPolicy{
MinTime: 5 * time.Second, // If a client pings more than once every 5 seconds, terminate the connection
PermitWithoutStream: true, // Allow pings even when there are no active streams
}
kasp := keepalive.ServerParameters{
Time: 60 * time.Second, // Ping the client if it is idle for 60 seconds to ensure the connection is still active
Timeout: 10 * time.Second, // Wait 10 second for the ping ack before assuming the connection is dead
}
var lis net.Listener
err := retry.Do(s.ctx, func() error {
addr := ":" + strconv.Itoa(grpcPort)
var err error
lis, err = net.Listen("tcp", addr)
return err
}, retry.Attempts(10))
if err != nil {
log.Error("DataNode GrpcServer:failed to listen", zap.Error(err))
s.grpcErrChan <- err
return
}
opts := tracer.GetInterceptorOpts()
s.grpcServer = grpc.NewServer(
grpc.KeepaliveEnforcementPolicy(kaep),
grpc.KeepaliveParams(kasp),
grpc.MaxRecvMsgSize(Params.ServerMaxRecvSize.GetAsInt()),
grpc.MaxSendMsgSize(Params.ServerMaxSendSize.GetAsInt()),
grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
otelgrpc.UnaryServerInterceptor(opts...),
logutil.UnaryTraceLoggerInterceptor,
interceptor.ClusterValidationUnaryServerInterceptor(),
interceptor.ServerIDValidationUnaryServerInterceptor(func() int64 {
if s.serverID.Load() == 0 {
s.serverID.Store(paramtable.GetNodeID())
}
return s.serverID.Load()
}),
)),
grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(
otelgrpc.StreamServerInterceptor(opts...),
logutil.StreamTraceLoggerInterceptor,
interceptor.ClusterValidationStreamServerInterceptor(),
interceptor.ServerIDValidationStreamServerInterceptor(func() int64 {
if s.serverID.Load() == 0 {
s.serverID.Store(paramtable.GetNodeID())
}
return s.serverID.Load()
}),
)))
datapb.RegisterDataNodeServer(s.grpcServer, s)
ctx, cancel := context.WithCancel(s.ctx)
defer cancel()
go funcutil.CheckGrpcReady(ctx, s.grpcErrChan)
if err := s.grpcServer.Serve(lis); err != nil {
log.Warn("DataNode failed to start gRPC")
s.grpcErrChan <- err
}
}
func (s *Server) SetEtcdClient(client *clientv3.Client) {
s.datanode.SetEtcdClient(client)
}
func (s *Server) SetRootCoordInterface(ms types.RootCoordClient) error {
return s.datanode.SetRootCoordClient(ms)
}
func (s *Server) SetDataCoordInterface(ds types.DataCoordClient) error {
return s.datanode.SetDataCoordClient(ds)
}
// Run initializes and starts Datanode's grpc service.
func (s *Server) Run() error {
if err := s.init(); err != nil {
// errors are propagated upstream as panic.
return err
}
log.Info("DataNode gRPC services successfully initialized")
if err := s.start(); err != nil {
// errors are propagated upstream as panic.
return err
}
log.Info("DataNode gRPC services successfully started")
return nil
}
// Stop stops Datanode's grpc service.
func (s *Server) Stop() (err error) {
Params := &paramtable.Get().DataNodeGrpcServerCfg
logger := log.With(zap.String("address", Params.GetAddress()))
logger.Info("datanode stopping")
defer func() {
logger.Info("datanode stopped", zap.Error(err))
}()
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
if s.grpcServer != nil {
utils.GracefulStopGRPCServer(s.grpcServer)
}
s.grpcWG.Wait()
logger.Info("internal server[datanode] start to stop")
err = s.datanode.Stop()
if err != nil {
log.Error("failed to close datanode", zap.Error(err))
return err
}
s.cancel()
return nil
}
// init initializes Datanode's grpc service.
func (s *Server) init() error {
etcdConfig := &paramtable.Get().EtcdCfg
Params := &paramtable.Get().DataNodeGrpcServerCfg
ctx := context.Background()
if !funcutil.CheckPortAvailable(Params.Port.GetAsInt()) {
paramtable.Get().Save(Params.Port.Key, fmt.Sprintf("%d", funcutil.GetAvailablePort()))
log.Warn("DataNode found available port during init", zap.Int("port", Params.Port.GetAsInt()))
}
etcdCli, err := etcd.CreateEtcdClient(
etcdConfig.UseEmbedEtcd.GetAsBool(),
etcdConfig.EtcdEnableAuth.GetAsBool(),
etcdConfig.EtcdAuthUserName.GetValue(),
etcdConfig.EtcdAuthPassword.GetValue(),
etcdConfig.EtcdUseSSL.GetAsBool(),
etcdConfig.Endpoints.GetAsStrings(),
etcdConfig.EtcdTLSCert.GetValue(),
etcdConfig.EtcdTLSKey.GetValue(),
etcdConfig.EtcdTLSCACert.GetValue(),
etcdConfig.EtcdTLSMinVersion.GetValue())
if err != nil {
log.Error("failed to connect to etcd", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.SetEtcdClient(s.etcdCli)
s.datanode.SetAddress(Params.GetAddress())
log.Info("DataNode address", zap.String("address", Params.IP+":"+strconv.Itoa(Params.Port.GetAsInt())))
log.Info("DataNode serverID", zap.Int64("serverID", s.serverID.Load()))
err = s.startGrpc()
if err != nil {
return err
}
// --- RootCoord Client ---
if s.newRootCoordClient != nil {
log.Info("initializing RootCoord client for DataNode")
rootCoordClient, err := s.newRootCoordClient()
if err != nil {
log.Error("failed to create new RootCoord client", zap.Error(err))
panic(err)
}
if err = componentutil.WaitForComponentHealthy(ctx, rootCoordClient, "RootCoord", 1000000, time.Millisecond*200); err != nil {
log.Error("failed to wait for RootCoord client to be ready", zap.Error(err))
panic(err)
}
log.Info("RootCoord client is ready for DataNode")
if err = s.SetRootCoordInterface(rootCoordClient); err != nil {
panic(err)
}
}
// --- DataCoord Client ---
if s.newDataCoordClient != nil {
log.Debug("starting DataCoord client for DataNode")
dataCoordClient, err := s.newDataCoordClient()
if err != nil {
log.Error("failed to create new DataCoord client", zap.Error(err))
panic(err)
}
if err = componentutil.WaitForComponentInitOrHealthy(ctx, dataCoordClient, "DataCoord", 1000000, time.Millisecond*200); err != nil {
log.Error("failed to wait for DataCoord client to be ready", zap.Error(err))
panic(err)
}
log.Info("DataCoord client is ready for DataNode")
if err = s.SetDataCoordInterface(dataCoordClient); err != nil {
panic(err)
}
}
s.datanode.UpdateStateCode(commonpb.StateCode_Initializing)
if err := s.datanode.Init(); err != nil {
log.Error("failed to init DataNode server", zap.Error(err))
return err
}
log.Info("current DataNode state", zap.Any("state", s.datanode.GetStateCode()))
return nil
}
// start starts datanode's grpc service.
func (s *Server) start() error {
if err := s.datanode.Start(); err != nil {
return err
}
err := s.datanode.Register()
if err != nil {
log.Debug("failed to register to Etcd", zap.Error(err))
return err
}
return nil
}
// GetComponentStates gets the component states of Datanode
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.datanode.GetComponentStates(ctx, req)
}
// GetStatisticsChannel gets the statistics channel of Datanode.
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.datanode.GetStatisticsChannel(ctx, req)
}
// Deprecated
func (s *Server) WatchDmChannels(ctx context.Context, req *datapb.WatchDmChannelsRequest) (*commonpb.Status, error) {
return s.datanode.WatchDmChannels(ctx, req)
}
func (s *Server) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
if err := merr.CheckHealthy(s.datanode.GetStateCode()); err != nil {
return merr.Status(err), nil
}
return s.datanode.FlushSegments(ctx, req)
}
// ShowConfigurations gets specified configurations para of DataNode
func (s *Server) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
return s.datanode.ShowConfigurations(ctx, req)
}
// GetMetrics gets the metrics info of Datanode.
func (s *Server) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return s.datanode.GetMetrics(ctx, request)
}
func (s *Server) CompactionV2(ctx context.Context, request *datapb.CompactionPlan) (*commonpb.Status, error) {
return s.datanode.CompactionV2(ctx, request)
}
// GetCompactionState gets the Compaction tasks state of DataNode
func (s *Server) GetCompactionState(ctx context.Context, request *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
return s.datanode.GetCompactionState(ctx, request)
}
func (s *Server) ResendSegmentStats(ctx context.Context, request *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) {
return s.datanode.ResendSegmentStats(ctx, request)
}
func (s *Server) SyncSegments(ctx context.Context, request *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
return s.datanode.SyncSegments(ctx, request)
}
func (s *Server) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
return s.datanode.FlushChannels(ctx, req)
}
func (s *Server) NotifyChannelOperation(ctx context.Context, req *datapb.ChannelOperationsRequest) (*commonpb.Status, error) {
return s.datanode.NotifyChannelOperation(ctx, req)
}
func (s *Server) CheckChannelOperationProgress(ctx context.Context, req *datapb.ChannelWatchInfo) (*datapb.ChannelOperationProgressResponse, error) {
return s.datanode.CheckChannelOperationProgress(ctx, req)
}
func (s *Server) PreImport(ctx context.Context, req *datapb.PreImportRequest) (*commonpb.Status, error) {
return s.datanode.PreImport(ctx, req)
}
func (s *Server) ImportV2(ctx context.Context, req *datapb.ImportRequest) (*commonpb.Status, error) {
return s.datanode.ImportV2(ctx, req)
}
func (s *Server) QueryPreImport(ctx context.Context, req *datapb.QueryPreImportRequest) (*datapb.QueryPreImportResponse, error) {
return s.datanode.QueryPreImport(ctx, req)
}
func (s *Server) QueryImport(ctx context.Context, req *datapb.QueryImportRequest) (*datapb.QueryImportResponse, error) {
return s.datanode.QueryImport(ctx, req)
}
func (s *Server) DropImport(ctx context.Context, req *datapb.DropImportRequest) (*commonpb.Status, error) {
return s.datanode.DropImport(ctx, req)
}
func (s *Server) QuerySlot(ctx context.Context, req *datapb.QuerySlotRequest) (*datapb.QuerySlotResponse, error) {
return s.datanode.QuerySlot(ctx, req)
}
func (s *Server) DropCompactionPlan(ctx context.Context, req *datapb.DropCompactionPlanRequest) (*commonpb.Status, error) {
return s.datanode.DropCompactionPlan(ctx, req)
}