github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/servermaster/server.go (about)

     1  // Copyright 2022 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package servermaster
    15  
    16  import (
    17  	"context"
    18  	"database/sql"
    19  	"encoding/json"
    20  	"net"
    21  	"net/http"
    22  	"net/http/httputil"
    23  	"net/url"
    24  	"sort"
    25  	"strings"
    26  	"sync"
    27  	"time"
    28  
    29  	grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
    30  	"github.com/grpc-ecosystem/grpc-gateway/v2/runtime"
    31  	"github.com/pingcap/log"
    32  	pb "github.com/pingcap/tiflow/engine/enginepb"
    33  	"github.com/pingcap/tiflow/engine/framework/metadata"
    34  	frameModel "github.com/pingcap/tiflow/engine/framework/model"
    35  	"github.com/pingcap/tiflow/engine/model"
    36  	pkgClient "github.com/pingcap/tiflow/engine/pkg/client"
    37  	dcontext "github.com/pingcap/tiflow/engine/pkg/context"
    38  	"github.com/pingcap/tiflow/engine/pkg/deps"
    39  	"github.com/pingcap/tiflow/engine/pkg/externalresource/broker"
    40  	externRescManager "github.com/pingcap/tiflow/engine/pkg/externalresource/manager"
    41  	"github.com/pingcap/tiflow/engine/pkg/meta"
    42  	metaModel "github.com/pingcap/tiflow/engine/pkg/meta/model"
    43  	"github.com/pingcap/tiflow/engine/pkg/openapi"
    44  	pkgOrm "github.com/pingcap/tiflow/engine/pkg/orm"
    45  	"github.com/pingcap/tiflow/engine/pkg/p2p"
    46  	"github.com/pingcap/tiflow/engine/pkg/rpcutil"
    47  	"github.com/pingcap/tiflow/engine/pkg/tenant"
    48  	"github.com/pingcap/tiflow/engine/servermaster/scheduler"
    49  	schedModel "github.com/pingcap/tiflow/engine/servermaster/scheduler/model"
    50  	"github.com/pingcap/tiflow/engine/servermaster/serverutil"
    51  	"github.com/pingcap/tiflow/pkg/election"
    52  	"github.com/pingcap/tiflow/pkg/errors"
    53  	"github.com/pingcap/tiflow/pkg/label"
    54  	"github.com/pingcap/tiflow/pkg/security"
    55  	"github.com/pingcap/tiflow/pkg/tcpserver"
    56  	p2pProtocol "github.com/pingcap/tiflow/proto/p2p"
    57  	"github.com/prometheus/client_golang/prometheus"
    58  	"go.uber.org/zap"
    59  	"golang.org/x/sync/errgroup"
    60  	"golang.org/x/time/rate"
    61  	"google.golang.org/grpc"
    62  	"google.golang.org/grpc/codes"
    63  	"google.golang.org/grpc/credentials/insecure"
    64  	"google.golang.org/grpc/status"
    65  	"google.golang.org/grpc/test/bufconn"
    66  	"google.golang.org/protobuf/encoding/protojson"
    67  	"google.golang.org/protobuf/types/known/emptypb"
    68  )
    69  
    70  // TODO: make it configurable in the future.
    71  const (
    72  	leaderElectionTable  = "leader_election"
    73  	resignLeaderDuration = 10 * time.Second
    74  	grpcConnBufSize      = 32 * 1024
    75  )
    76  
    77  // use a slice instead of map because in small data size, slice search is faster
    78  // than map search.
    79  var masterRPCLimiterAllowList = []string{
    80  	"CreateJob",
    81  	"CancelJob",
    82  	"ScheduleTask",
    83  	"CreateResource",
    84  	"RemoveResource",
    85  }
    86  
    87  // Server handles PRC requests for df master.
    88  type Server struct {
    89  	id string // Server id, randomly generated when server is created.
    90  
    91  	cfg     *Config
    92  	metrics *serverMasterMetric
    93  
    94  	elector election.Elector
    95  
    96  	leaderServiceFn func(context.Context) error
    97  
    98  	// sched scheduler
    99  	executorManager        ExecutorManager
   100  	jobManager             JobManager
   101  	resourceManagerService *externRescManager.Service
   102  	scheduler              *scheduler.Scheduler
   103  
   104  	// file resource GC
   105  	gcRunner      externRescManager.GCRunner
   106  	gcCoordinator externRescManager.GCCoordinator
   107  
   108  	msgService   *p2p.MessageRPCService
   109  	p2pMsgRouter p2p.MessageRouter
   110  	rpcLogRL     *rate.Limiter
   111  
   112  	metaStoreManager MetaStoreManager
   113  
   114  	leaderDegrader *featureDegrader
   115  	forwardChecker *forwardChecker
   116  
   117  	// framework metastore client
   118  	frameMetaClient     pkgOrm.Client
   119  	frameworkClientConn metaModel.ClientConn
   120  	businessClientConn  metaModel.ClientConn
   121  }
   122  
   123  type serverMasterMetric struct {
   124  	metricJobNum      map[pb.Job_State]prometheus.Gauge
   125  	metricExecutorNum map[model.ExecutorStatus]prometheus.Gauge
   126  }
   127  
   128  func newServerMasterMetric() *serverMasterMetric {
   129  	// Following are leader only metrics
   130  	metricJobNum := make(map[pb.Job_State]prometheus.Gauge)
   131  	for state, statusName := range pb.Job_State_name {
   132  		metric := serverJobNumGauge.WithLabelValues(statusName)
   133  		metricJobNum[pb.Job_State(state)] = metric
   134  	}
   135  
   136  	metricExecutorNum := make(map[model.ExecutorStatus]prometheus.Gauge)
   137  	for state, name := range model.ExecutorStatusNameMapping {
   138  		metric := serverExecutorNumGauge.WithLabelValues(name)
   139  		metricExecutorNum[state] = metric
   140  	}
   141  
   142  	return &serverMasterMetric{
   143  		metricJobNum:      metricJobNum,
   144  		metricExecutorNum: metricExecutorNum,
   145  	}
   146  }
   147  
   148  // NewServer creates a new master-server.
   149  func NewServer(cfg *Config) (*Server, error) {
   150  	log.Info("creating server master", zap.Stringer("config", cfg))
   151  
   152  	id := generateNodeID(cfg.Name)
   153  	msgService := p2p.NewMessageRPCServiceWithRPCServer(id, nil, nil)
   154  	p2pMsgRouter := p2p.NewMessageRouter(id, cfg.AdvertiseAddr)
   155  
   156  	server := &Server{
   157  		id:               id,
   158  		cfg:              cfg,
   159  		leaderDegrader:   newFeatureDegrader(),
   160  		msgService:       msgService,
   161  		p2pMsgRouter:     p2pMsgRouter,
   162  		rpcLogRL:         rate.NewLimiter(rate.Every(time.Second*5), 3 /*burst*/),
   163  		metrics:          newServerMasterMetric(),
   164  		metaStoreManager: NewMetaStoreManager(),
   165  	}
   166  	server.leaderServiceFn = server.runLeaderService
   167  	return server, nil
   168  }
   169  
   170  // Heartbeat implements pb interface.
   171  func (s *Server) Heartbeat(ctx context.Context, req *pb.HeartbeatRequest) (*pb.HeartbeatResponse, error) {
   172  	return s.executorManager.HandleHeartbeat(req)
   173  }
   174  
   175  // CreateJob delegates request to leader's JobManager.CreateJob.
   176  func (s *Server) CreateJob(ctx context.Context, req *pb.CreateJobRequest) (*pb.Job, error) {
   177  	return s.jobManager.CreateJob(ctx, req)
   178  }
   179  
   180  // GetJob delegates request to leader's JobManager.GetJob.
   181  func (s *Server) GetJob(ctx context.Context, req *pb.GetJobRequest) (*pb.Job, error) {
   182  	return s.jobManager.GetJob(ctx, req)
   183  }
   184  
   185  // ListJobs delegates request to leader's JobManager.ListJobs.
   186  func (s *Server) ListJobs(ctx context.Context, req *pb.ListJobsRequest) (*pb.ListJobsResponse, error) {
   187  	return s.jobManager.ListJobs(ctx, req)
   188  }
   189  
   190  // CancelJob delegates request to leader's JobManager.CancelJob.
   191  func (s *Server) CancelJob(ctx context.Context, req *pb.CancelJobRequest) (*pb.Job, error) {
   192  	return s.jobManager.CancelJob(ctx, req)
   193  }
   194  
   195  // DeleteJob delegates request to leader's JobManager.DeleteJob.
   196  func (s *Server) DeleteJob(ctx context.Context, req *pb.DeleteJobRequest) (*emptypb.Empty, error) {
   197  	return s.jobManager.DeleteJob(ctx, req)
   198  }
   199  
   200  // RegisterExecutor implements grpc interface, and passes request onto executor manager.
   201  func (s *Server) RegisterExecutor(ctx context.Context, req *pb.RegisterExecutorRequest) (*pb.Executor, error) {
   202  	executorMeta, err := s.executorManager.AllocateNewExec(ctx, req)
   203  	if err != nil {
   204  		return nil, err
   205  	}
   206  
   207  	return &pb.Executor{
   208  		Id:      string(executorMeta.ID),
   209  		Name:    executorMeta.Name,
   210  		Address: executorMeta.Address,
   211  		Labels:  label.Set(executorMeta.Labels).ToMap(),
   212  	}, nil
   213  }
   214  
   215  // ListExecutors implements DiscoveryServer.ListExecutors.
   216  func (s *Server) ListExecutors(ctx context.Context, req *pb.ListExecutorsRequest) (*pb.ListExecutorsResponse, error) {
   217  	resp := &pb.ListExecutorsResponse{}
   218  	executors := s.executorManager.ListExecutors()
   219  	for _, executor := range executors {
   220  		resp.Executors = append(resp.Executors, &pb.Executor{
   221  			Id:      string(executor.ID),
   222  			Name:    executor.Name,
   223  			Address: executor.Address,
   224  			Labels:  executor.Labels.ToMap(),
   225  		})
   226  	}
   227  	sort.Slice(resp.Executors, func(i, j int) bool {
   228  		return resp.Executors[i].Id < resp.Executors[j].Id
   229  	})
   230  	return resp, nil
   231  }
   232  
   233  // ListMasters implements DiscoveryServer.ListMasters.
   234  func (s *Server) ListMasters(ctx context.Context, req *pb.ListMastersRequest) (*pb.ListMastersResponse, error) {
   235  	resp := &pb.ListMastersResponse{}
   236  	leaderAddr, ok := s.leaderAddr()
   237  	for _, m := range s.elector.GetMembers() {
   238  		isLeader := ok && m.Address == leaderAddr
   239  		resp.Masters = append(resp.Masters, &pb.Master{
   240  			Id:       m.ID,
   241  			Name:     m.Name,
   242  			Address:  m.Address,
   243  			IsLeader: isLeader,
   244  		})
   245  	}
   246  	sort.Slice(resp.Masters, func(i, j int) bool {
   247  		return resp.Masters[i].Id < resp.Masters[j].Id
   248  	})
   249  	return resp, nil
   250  }
   251  
   252  // ScheduleTask implements grpc interface. It works as follows
   253  // - receives request from job master
   254  // - queries resource manager to allocate resource and maps tasks to executors
   255  // - returns scheduler response to job master
   256  func (s *Server) ScheduleTask(ctx context.Context, req *pb.ScheduleTaskRequest) (*pb.ScheduleTaskResponse, error) {
   257  	schedulerReq, err := schedModel.NewSchedulerRequestFromPB(req)
   258  	if err != nil {
   259  		return nil, err
   260  	}
   261  
   262  	schedulerResp, err := s.scheduler.ScheduleTask(ctx, schedulerReq)
   263  	if err != nil {
   264  		return nil, err
   265  	}
   266  
   267  	addr, ok := s.executorManager.GetAddr(schedulerResp.ExecutorID)
   268  	if !ok {
   269  		log.Warn("Executor is gone, RPC call needs retry",
   270  			zap.Any("request", req),
   271  			zap.String("executor-id", string(schedulerResp.ExecutorID)))
   272  		return nil, errors.ErrUnknownExecutor.GenWithStackByArgs(string(schedulerResp.ExecutorID))
   273  	}
   274  
   275  	return &pb.ScheduleTaskResponse{
   276  		ExecutorId:   string(schedulerResp.ExecutorID),
   277  		ExecutorAddr: addr,
   278  	}, nil
   279  }
   280  
   281  // QueryMetaStore implements gRPC interface
   282  func (s *Server) QueryMetaStore(
   283  	ctx context.Context, req *pb.QueryMetaStoreRequest,
   284  ) (*pb.QueryMetaStoreResponse, error) {
   285  	getStore := func(storeID string) (*pb.QueryMetaStoreResponse, error) {
   286  		store := s.metaStoreManager.GetMetaStore(storeID)
   287  		if store == nil {
   288  			return nil, errors.ErrMetaStoreNotExists.GenWithStackByArgs(storeID)
   289  		}
   290  		config, err := json.Marshal(store)
   291  		if err != nil {
   292  			return nil, errors.Trace(err)
   293  		}
   294  
   295  		return &pb.QueryMetaStoreResponse{
   296  			Config: config,
   297  		}, nil
   298  	}
   299  
   300  	switch req.Tp {
   301  	case pb.StoreType_SystemMetaStore:
   302  		return getStore(FrameMetaID)
   303  	case pb.StoreType_AppMetaStore:
   304  		return getStore(DefaultBusinessMetaID)
   305  	default:
   306  		return nil, status.Errorf(codes.InvalidArgument, "unknown store type %v", req.Tp)
   307  	}
   308  }
   309  
   310  // QueryStorageConfig implements gRPC interface
   311  func (s *Server) QueryStorageConfig(
   312  	ctx context.Context, req *pb.QueryStorageConfigRequest,
   313  ) (*pb.QueryStorageConfigResponse, error) {
   314  	config, err := json.Marshal(s.cfg.Storage)
   315  	if err != nil {
   316  		return nil, errors.Trace(err)
   317  	}
   318  	return &pb.QueryStorageConfigResponse{
   319  		Config: config,
   320  	}, nil
   321  }
   322  
   323  // GetLeader implements DiscoveryServer.GetLeader.
   324  func (s *Server) GetLeader(_ context.Context, _ *pb.GetLeaderRequest) (*pb.GetLeaderResponse, error) {
   325  	leaderAddr, ok := s.leaderAddr()
   326  	if !ok {
   327  		return nil, status.Error(codes.NotFound, "no leader")
   328  	}
   329  	return &pb.GetLeaderResponse{
   330  		AdvertiseAddr: leaderAddr,
   331  	}, nil
   332  }
   333  
   334  // ResignLeader implements DiscoveryServer.ResignLeader.
   335  func (s *Server) ResignLeader(ctx context.Context, _ *pb.ResignLeaderRequest) (*emptypb.Empty, error) {
   336  	if err := s.elector.ResignLeader(ctx, resignLeaderDuration); err != nil {
   337  		return nil, err
   338  	}
   339  	return &emptypb.Empty{}, nil
   340  }
   341  
   342  // Run the server master.
   343  func (s *Server) Run(ctx context.Context) error {
   344  	err := s.registerMetaStore(ctx)
   345  	if err != nil {
   346  		return err
   347  	}
   348  
   349  	// Elector relies on meta store, so it should be initialized after meta store.
   350  	if err := s.initElector(); err != nil {
   351  		return errors.Trace(err)
   352  	}
   353  
   354  	// resourceManagerService relies on meta store
   355  	s.initResourceManagerService()
   356  
   357  	if err := broker.PreCheckConfig(&s.cfg.Storage); err != nil {
   358  		return err
   359  	}
   360  
   361  	defer func() {
   362  		if s.forwardChecker != nil {
   363  			if err := s.forwardChecker.Close(); err != nil {
   364  				log.Warn("failed to close forward checker", zap.Error(err))
   365  			}
   366  		}
   367  	}()
   368  
   369  	wg, ctx := errgroup.WithContext(ctx)
   370  
   371  	wg.Go(func() error {
   372  		return s.serve(ctx)
   373  	})
   374  
   375  	wg.Go(func() error {
   376  		return s.msgService.GetMessageServer().Run(ctx, nil)
   377  	})
   378  
   379  	wg.Go(func() error {
   380  		return s.elector.RunElection(ctx)
   381  	})
   382  
   383  	return wg.Wait()
   384  }
   385  
   386  func (s *Server) registerMetaStore(ctx context.Context) error {
   387  	// register metastore for framework
   388  	cfg := s.cfg
   389  	if err := s.metaStoreManager.Register(cfg.FrameworkMeta.StoreID, cfg.FrameworkMeta); err != nil {
   390  		return err
   391  	}
   392  	if cfg.FrameworkMeta.StoreType == metaModel.StoreTypeMySQL {
   393  		// Normally, a schema will be created in advance, and we may have no privilege
   394  		// to create schema for framework meta. Just for easy test here. Ignore any error.
   395  		ctx, cancel := context.WithTimeout(ctx, 3*time.Second)
   396  		defer cancel()
   397  		if err := meta.CreateSchemaIfNotExists(ctx, *(s.cfg.FrameworkMeta)); err != nil {
   398  			log.Error("create schema for framework metastore fail",
   399  				zap.String("schema", s.cfg.FrameworkMeta.Schema),
   400  				zap.Error(err))
   401  			return err
   402  		}
   403  	}
   404  	var err error
   405  	s.frameworkClientConn, err = meta.NewClientConn(cfg.FrameworkMeta)
   406  	if err != nil {
   407  		log.Error("connect to framework metastore fail", zap.Any("config", cfg.FrameworkMeta), zap.Error(err))
   408  		return err
   409  	}
   410  	// some components depend on this framework meta client
   411  	s.frameMetaClient, err = pkgOrm.NewClient(s.frameworkClientConn)
   412  	if err != nil {
   413  		log.Error("create framework meta client fail", zap.Error(err))
   414  		return err
   415  	}
   416  	log.Info("register framework metastore successfully", zap.Any("config", cfg.FrameworkMeta))
   417  
   418  	// register metastore for business
   419  	err = s.metaStoreManager.Register(cfg.BusinessMeta.StoreID, cfg.BusinessMeta)
   420  	if err != nil {
   421  		return err
   422  	}
   423  	if cfg.BusinessMeta.StoreType == metaModel.StoreTypeMySQL {
   424  		// Normally, a schema will be created in advance, and we may have no privilege
   425  		// to create schema for business meta. Just for easy test here. Ignore any error.
   426  		ctx, cancel := context.WithTimeout(ctx, 3*time.Second)
   427  		defer cancel()
   428  		if err := meta.CreateSchemaIfNotExists(ctx, *(s.cfg.BusinessMeta)); err != nil {
   429  			log.Error("create schema for business metastore fail",
   430  				zap.String("schema", s.cfg.BusinessMeta.Schema),
   431  				zap.Error(err))
   432  			return err
   433  		}
   434  	}
   435  	s.businessClientConn, err = meta.NewClientConn(cfg.BusinessMeta)
   436  	if err != nil {
   437  		log.Error("connect to business metastore fail", zap.Any("config", cfg.BusinessMeta), zap.Error(err))
   438  		return err
   439  	}
   440  	log.Info("register business metastore successfully", zap.Any("config", cfg.BusinessMeta))
   441  
   442  	return nil
   443  }
   444  
   445  func (s *Server) initResourceManagerService() {
   446  	s.resourceManagerService = externRescManager.NewService(s.frameMetaClient)
   447  }
   448  
   449  func (s *Server) initElector() error {
   450  	conn, err := s.frameworkClientConn.GetConn()
   451  	if err != nil {
   452  		return errors.Trace(err)
   453  	}
   454  	db, ok := conn.(*sql.DB)
   455  	if !ok {
   456  		return errors.Errorf("failed to convert conn to *sql.DB, got %T", conn)
   457  	}
   458  
   459  	sqlStorage, err := election.NewSQLStorage(db, leaderElectionTable)
   460  	if err != nil {
   461  		return err
   462  	}
   463  
   464  	elector, err := election.NewElector(election.Config{
   465  		ID:             s.id,
   466  		Name:           s.cfg.Name,
   467  		Address:        s.cfg.AdvertiseAddr,
   468  		Storage:        sqlStorage,
   469  		LeaderCallback: s.leaderServiceFn,
   470  	})
   471  	if err != nil {
   472  		return err
   473  	}
   474  
   475  	s.elector = elector
   476  	s.forwardChecker = newForwardChecker(elector)
   477  	return err
   478  }
   479  
   480  func (s *Server) serve(ctx context.Context) error {
   481  	var (
   482  		cleaupFuncs []func()
   483  		cleanupOnce sync.Once
   484  	)
   485  	cleanup := func() {
   486  		cleanupOnce.Do(func() {
   487  			for _, f := range cleaupFuncs {
   488  				f()
   489  			}
   490  		})
   491  	}
   492  	defer cleanup()
   493  
   494  	errGroup, ctx := errgroup.WithContext(ctx)
   495  
   496  	// TODO: Support TLS.
   497  	tcpServer, err := tcpserver.NewTCPServer(s.cfg.Addr, &security.Credential{})
   498  	if err != nil {
   499  		return err
   500  	}
   501  	cleaupFuncs = append(cleaupFuncs, func() {
   502  		if err := tcpServer.Close(); err != nil {
   503  			log.Warn("failed to close tcp server", zap.Error(err))
   504  		}
   505  	})
   506  	errGroup.Go(func() error {
   507  		return tcpServer.Run(ctx)
   508  	})
   509  
   510  	grpcServer := s.createGRPCServer()
   511  	cleaupFuncs = append(cleaupFuncs, grpcServer.Stop)
   512  	errGroup.Go(func() error {
   513  		return grpcServer.Serve(tcpServer.GrpcListener())
   514  	})
   515  
   516  	// gRPC-Gateway doesn't call gRPC interceptors when it directly forwards requests to the service handler.
   517  	// See https://github.com/grpc-ecosystem/grpc-gateway/issues/1043.
   518  	//
   519  	// To make the gRPC interceptors work, we have to create a client connection to the gRPC server and register it to
   520  	// the gRPC-Gateway ServerMux. bufconn is used to create an in-process gRPC server, so the client connection can
   521  	// bypass the network stack. See https://github.com/grpc/grpc-go/issues/906 for more details.
   522  	ln := bufconn.Listen(grpcConnBufSize)
   523  	cleaupFuncs = append(cleaupFuncs, func() {
   524  		if err := ln.Close(); err != nil {
   525  			log.Warn("failed to close bufconn", zap.Error(err))
   526  		}
   527  	})
   528  	inProcessGRPCServer := s.createGRPCServer()
   529  	cleaupFuncs = append(cleaupFuncs, inProcessGRPCServer.Stop)
   530  	errGroup.Go(func() error {
   531  		return inProcessGRPCServer.Serve(ln)
   532  	})
   533  
   534  	dial := func(ctx context.Context, target string) (net.Conn, error) {
   535  		return ln.DialContext(ctx)
   536  	}
   537  	conn, err := grpc.DialContext(
   538  		ctx,
   539  		"bufnet",
   540  		grpc.WithContextDialer(dial),
   541  		grpc.WithTransportCredentials(insecure.NewCredentials()),
   542  	)
   543  	if err != nil {
   544  		return errors.Trace(err)
   545  	}
   546  	cleaupFuncs = append(cleaupFuncs, func() {
   547  		if err := conn.Close(); err != nil {
   548  			log.Warn("failed to close grpc connection", zap.Error(err))
   549  		}
   550  	})
   551  
   552  	httpServer, err := s.createHTTPServer(conn)
   553  	if err != nil {
   554  		return err
   555  	}
   556  	cleaupFuncs = append(cleaupFuncs, func() {
   557  		if err := httpServer.Close(); err != nil {
   558  			log.Warn("failed to close http server", zap.Error(err))
   559  		}
   560  	})
   561  	errGroup.Go(func() error {
   562  		return httpServer.Serve(tcpServer.HTTP1Listener())
   563  	})
   564  
   565  	// Some background goroutines may still be running after context is canceled.
   566  	// We need to explicitly stop or close them and wait for them to exit.
   567  	errGroup.Go(func() error {
   568  		<-ctx.Done()
   569  		cleanup()
   570  		return errors.Trace(ctx.Err())
   571  	})
   572  
   573  	return errGroup.Wait()
   574  }
   575  
   576  func (s *Server) createGRPCServer() *grpc.Server {
   577  	logLimiter := rate.NewLimiter(rate.Every(time.Second*5), 3 /*burst*/)
   578  	grpcServer := grpc.NewServer(
   579  		grpc.StreamInterceptor(grpcprometheus.StreamServerInterceptor),
   580  		grpc.ChainUnaryInterceptor(
   581  			grpcprometheus.UnaryServerInterceptor,
   582  			rpcutil.ForwardToLeader[multiClient](s.forwardChecker),
   583  			rpcutil.CheckAvailable(s.leaderDegrader),
   584  			rpcutil.Logger(masterRPCLimiterAllowList, logLimiter),
   585  			rpcutil.NormalizeError(),
   586  		),
   587  	)
   588  	pb.RegisterDiscoveryServer(grpcServer, s)
   589  	pb.RegisterTaskSchedulerServer(grpcServer, s)
   590  	pb.RegisterJobManagerServer(grpcServer, s)
   591  	pb.RegisterResourceManagerServer(grpcServer, s.resourceManagerService)
   592  	p2pProtocol.RegisterCDCPeerToPeerServer(grpcServer, s.msgService.GetMessageServer())
   593  	return grpcServer
   594  }
   595  
   596  func (s *Server) createHTTPServer(conn *grpc.ClientConn) (*http.Server, error) {
   597  	grpcMux := runtime.NewServeMux(
   598  		runtime.WithMarshalerOption(runtime.MIMEWildcard, &runtime.JSONPb{
   599  			MarshalOptions:   protojson.MarshalOptions{UseProtoNames: true, EmitUnpopulated: true},
   600  			UnmarshalOptions: protojson.UnmarshalOptions{},
   601  		}),
   602  		runtime.WithErrorHandler(func(ctx context.Context, mux *runtime.ServeMux,
   603  			_ runtime.Marshaler, writer http.ResponseWriter, _ *http.Request, err error,
   604  		) {
   605  			// Since request may be forwarded to other servers through grpc, we should try
   606  			// to extract the error from gRPC error first, and then convert it to HTTP error.
   607  			openapi.WriteHTTPError(writer, rpcutil.FromGRPCError(err))
   608  		}),
   609  	)
   610  	if err := pb.RegisterJobManagerHandler(context.Background(), grpcMux, conn); err != nil {
   611  		return nil, errors.Trace(err)
   612  	}
   613  	if err := pb.RegisterDiscoveryHandler(context.Background(), grpcMux, conn); err != nil {
   614  		return nil, errors.Trace(err)
   615  	}
   616  
   617  	router := http.NewServeMux()
   618  	registerRoutes(router, grpcMux, s.forwardJobAPI)
   619  
   620  	return &http.Server{
   621  		Handler:           router,
   622  		ReadHeaderTimeout: time.Minute,
   623  	}, nil
   624  }
   625  
   626  func (s *Server) forwardJobAPI(w http.ResponseWriter, r *http.Request) {
   627  	if err := s.handleForwardJobAPI(w, r); err != nil {
   628  		openapi.WriteHTTPError(w, err)
   629  	}
   630  }
   631  
   632  func (s *Server) handleForwardJobAPI(w http.ResponseWriter, r *http.Request) error {
   633  	apiPath := strings.TrimPrefix(r.URL.Path, openapi.JobAPIPrefix)
   634  	fields := strings.SplitN(apiPath, "/", 2)
   635  	if len(fields) != 2 {
   636  		return errors.New("invalid job api path")
   637  	}
   638  	jobID := fields[0]
   639  
   640  	var targetAddr string
   641  	if s.elector.IsLeader() {
   642  		forwardAddr, err := s.jobManager.GetJobMasterForwardAddress(r.Context(), jobID)
   643  		if err != nil {
   644  			return err
   645  		}
   646  		targetAddr = forwardAddr
   647  	} else if leaderAddr, ok := s.leaderAddr(); ok {
   648  		targetAddr = leaderAddr
   649  	} else {
   650  		return errors.New("no leader found")
   651  	}
   652  
   653  	// TODO: Support TLS.
   654  	u, err := url.Parse("http://" + targetAddr)
   655  	if err != nil {
   656  		return errors.Errorf("invalid target address %s", targetAddr)
   657  	}
   658  	proxy := httputil.NewSingleHostReverseProxy(u)
   659  	proxy.ServeHTTP(w, r)
   660  	return nil
   661  }
   662  
   663  // name is a shortcut to etcd name
   664  func (s *Server) name() string {
   665  	return s.id
   666  }
   667  
   668  func (s *Server) initializedBackendMeta(ctx context.Context) error {
   669  	bctx, cancel := context.WithTimeout(ctx, 3*time.Second)
   670  	defer cancel()
   671  	if err := pkgOrm.InitAllFrameworkModels(bctx, s.frameworkClientConn); err != nil {
   672  		log.Error("framework metastore initializes all backend tables fail", zap.Error(err))
   673  		return err
   674  	}
   675  
   676  	// Since we have the sql-type business metastore,
   677  	// we need to initialize the logic_epoches table for all jobs
   678  	if s.cfg.BusinessMeta.StoreType == metaModel.StoreTypeMySQL {
   679  		if err := pkgOrm.InitEpochModel(ctx, s.businessClientConn); err != nil {
   680  			log.Error("business metastore initializes the logic epoch table fail", zap.Error(err))
   681  			return err
   682  		}
   683  	}
   684  
   685  	return nil
   686  }
   687  
   688  func (s *Server) runLeaderService(ctx context.Context) (err error) {
   689  	start := time.Now()
   690  
   691  	// leader master need Initialize all backend tables first
   692  	err = s.initializedBackendMeta(ctx)
   693  	if err != nil {
   694  		return
   695  	}
   696  
   697  	// TODO support TLS.
   698  	executorClients := pkgClient.NewExecutorGroup(&security.Credential{}, log.L())
   699  	masterClient, err := pkgClient.NewServerMasterClientWithFailOver(
   700  		pkgClient.MasterServerList{
   701  			s.cfg.Addr: true,
   702  		},
   703  		nil, // TODO support TLS
   704  	)
   705  	if err != nil {
   706  		return
   707  	}
   708  
   709  	dctx := dcontext.NewContext(ctx)
   710  	dctx.Environ.Addr = s.cfg.AdvertiseAddr
   711  	dctx.Environ.NodeID = s.name()
   712  	dctx.ProjectInfo = tenant.FrameProjectInfo
   713  
   714  	masterMeta := &frameModel.MasterMeta{
   715  		ProjectID: tenant.FrameProjectInfo.UniqueID(),
   716  		ID:        metadata.JobManagerUUID,
   717  		Type:      frameModel.JobManager,
   718  		// TODO: add other infos
   719  	}
   720  	masterMetaBytes, err := masterMeta.Marshal()
   721  	if err != nil {
   722  		return
   723  	}
   724  	dctx.Environ.MasterMetaBytes = masterMetaBytes
   725  
   726  	dp := deps.NewDeps()
   727  	if err := dp.Provide(func() pkgOrm.Client {
   728  		return s.frameMetaClient
   729  	}); err != nil {
   730  		return err
   731  	}
   732  
   733  	if err := dp.Provide(func() metaModel.ClientConn {
   734  		return s.businessClientConn
   735  	}); err != nil {
   736  		return err
   737  	}
   738  
   739  	if err := dp.Provide(func() pkgClient.ExecutorGroup {
   740  		return executorClients
   741  	}); err != nil {
   742  		return err
   743  	}
   744  
   745  	if err := dp.Provide(func() pkgClient.ServerMasterClient {
   746  		return masterClient
   747  	}); err != nil {
   748  		return err
   749  	}
   750  
   751  	if err := dp.Provide(func() p2p.MessageSender {
   752  		return p2p.NewMessageSender(s.p2pMsgRouter)
   753  	}); err != nil {
   754  		return err
   755  	}
   756  
   757  	if err := dp.Provide(func() p2p.MessageHandlerManager {
   758  		return s.msgService.MakeHandlerManager()
   759  	}); err != nil {
   760  		return err
   761  	}
   762  
   763  	defer func() {
   764  		s.leaderDegrader.reset()
   765  	}()
   766  
   767  	// The following member variables are used in leader only and released after
   768  	// the leader is resigned, so initialize these variables in this function,
   769  	// instead of initializing them in the NewServer or Server.Run
   770  
   771  	// executorMetaClient needs to be initialized after frameMetaClient is initialized.
   772  	s.executorManager = NewExecutorManagerImpl(s.frameMetaClient, s.cfg.KeepAliveTTL, s.cfg.KeepAliveInterval)
   773  
   774  	// ResourceManagerService should be initialized after registerMetaStore.
   775  	s.scheduler = scheduler.NewScheduler(
   776  		s.executorManager,
   777  		s.resourceManagerService)
   778  
   779  	// TODO refactor this method to make it more readable and maintainable.
   780  	errg, errgCtx := errgroup.WithContext(ctx)
   781  
   782  	errg.Go(func() error {
   783  		return s.executorManager.Run(errgCtx)
   784  	})
   785  
   786  	errg.Go(func() error {
   787  		updater := executorInfoUpdater{
   788  			msgRouter:     s.p2pMsgRouter,
   789  			executorGroup: executorClients,
   790  		}
   791  		return serverutil.WatchExecutors(errgCtx, s.executorManager, updater)
   792  	})
   793  
   794  	s.leaderDegrader.updateExecutorManager(true)
   795  
   796  	dctx = dctx.WithDeps(dp)
   797  	s.jobManager, err = NewJobManagerImpl(dctx, metadata.JobManagerUUID, s.cfg.JobBackoff)
   798  	if err != nil {
   799  		return
   800  	}
   801  	defer func() {
   802  		err := s.jobManager.Close(ctx)
   803  		if err != nil {
   804  			log.Warn("job manager close with error", zap.Error(err))
   805  		}
   806  		log.Info("job manager exited")
   807  	}()
   808  
   809  	s.gcRunner = externRescManager.NewGCRunner(s.frameMetaClient, executorClients, &s.cfg.Storage)
   810  	s.gcCoordinator = externRescManager.NewGCCoordinator(s.executorManager, s.jobManager, s.frameMetaClient, s.gcRunner)
   811  
   812  	errg.Go(func() error {
   813  		return s.gcRunner.Run(errgCtx)
   814  	})
   815  	errg.Go(func() error {
   816  		return s.gcCoordinator.Run(errgCtx)
   817  	})
   818  
   819  	errg.Go(func() error {
   820  		metricTicker := time.NewTicker(defaultMetricInterval)
   821  		defer metricTicker.Stop()
   822  		leaderTicker := time.NewTicker(time.Millisecond * 200)
   823  		defer leaderTicker.Stop()
   824  		for {
   825  			select {
   826  			case <-errgCtx.Done():
   827  				// errgCtx is a leaderCtx actually
   828  				return errors.Trace(errgCtx.Err())
   829  			case <-leaderTicker.C:
   830  				if err := s.jobManager.Poll(errgCtx); err != nil {
   831  					log.Warn("Polling JobManager failed", zap.Error(err))
   832  					return err
   833  				}
   834  			case <-leaderTicker.C:
   835  				s.collectLeaderMetric()
   836  			}
   837  		}
   838  	})
   839  
   840  	s.leaderDegrader.updateMasterWorkerManager(true)
   841  	log.Info("leader is initialized", zap.Duration("took", time.Since(start)))
   842  
   843  	return errg.Wait()
   844  }
   845  
   846  type executorInfoUpdater struct {
   847  	msgRouter     p2p.MessageRouter
   848  	executorGroup *pkgClient.DefaultExecutorGroup
   849  }
   850  
   851  func (e executorInfoUpdater) UpdateExecutorList(executors map[model.ExecutorID]string) error {
   852  	for id, addr := range executors {
   853  		e.msgRouter.AddPeer(string(id), addr)
   854  	}
   855  	return e.executorGroup.UpdateExecutorList(executors)
   856  }
   857  
   858  func (e executorInfoUpdater) AddExecutor(executorID model.ExecutorID, addr string) error {
   859  	e.msgRouter.AddPeer(string(executorID), addr)
   860  	return e.executorGroup.AddExecutor(executorID, addr)
   861  }
   862  
   863  func (e executorInfoUpdater) RemoveExecutor(executorID model.ExecutorID) error {
   864  	e.msgRouter.RemovePeer(string(executorID))
   865  	return e.executorGroup.RemoveExecutor(executorID)
   866  }
   867  
   868  func (s *Server) collectLeaderMetric() {
   869  	for statusName := range pb.Job_State_name {
   870  		pbStatus := pb.Job_State(statusName)
   871  		s.metrics.metricJobNum[pbStatus].Set(float64(s.jobManager.JobCount(pbStatus)))
   872  	}
   873  	for statusName := range model.ExecutorStatusNameMapping {
   874  		s.metrics.metricExecutorNum[statusName].Set(float64(s.executorManager.ExecutorCount(statusName)))
   875  	}
   876  }
   877  
   878  func (s *Server) leaderAddr() (string, bool) {
   879  	leader, ok := s.elector.GetLeader()
   880  	if ok {
   881  		return leader.Address, true
   882  	}
   883  	return "", false
   884  }