github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logtail/service/server.go (about)

     1  // Copyright 2021 Matrix Origin
     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  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package service
    16  
    17  import (
    18  	"context"
    19  	"sync/atomic"
    20  	"time"
    21  
    22  	"github.com/fagongzi/goetty/v2"
    23  	"github.com/google/uuid"
    24  	"github.com/matrixorigin/matrixone/pkg/common/log"
    25  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    26  	"github.com/matrixorigin/matrixone/pkg/common/moprobe"
    27  	"github.com/matrixorigin/matrixone/pkg/common/morpc"
    28  	"github.com/matrixorigin/matrixone/pkg/common/runtime"
    29  	"github.com/matrixorigin/matrixone/pkg/common/stopper"
    30  	"github.com/matrixorigin/matrixone/pkg/pb/logtail"
    31  	"github.com/matrixorigin/matrixone/pkg/pb/timestamp"
    32  	v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2"
    33  	"github.com/matrixorigin/matrixone/pkg/util/trace"
    34  	taelogtail "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail"
    35  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/options"
    36  	"go.uber.org/zap"
    37  )
    38  
    39  const (
    40  	LogtailServiceRPCName = "logtail-server"
    41  
    42  	// updateEventMaxInterval is the max interval between update events.
    43  	// If 3s has passed since last update event, we should try to send an
    44  	// update event, rather than a suscription, to avoid there are no
    45  	// update events and cause logtail consumer waits for too long.
    46  	updateEventMaxInterval = time.Second * 3
    47  )
    48  
    49  type ServerOption func(*LogtailServer)
    50  
    51  // WithServerMaxMessageSize sets max rpc message size
    52  func WithServerMaxMessageSize(maxMessageSize int64) ServerOption {
    53  	return func(s *LogtailServer) {
    54  		s.cfg.RpcMaxMessageSize = maxMessageSize
    55  	}
    56  }
    57  
    58  // WithServerEnableChecksum enables checksum
    59  func WithServerEnableChecksum(enable bool) ServerOption {
    60  	return func(s *LogtailServer) {
    61  		s.cfg.RpcEnableChecksum = enable
    62  	}
    63  }
    64  
    65  // WithServerCollectInterval sets logtail collection interval.
    66  func WithServerCollectInterval(interval time.Duration) ServerOption {
    67  	return func(s *LogtailServer) {
    68  		s.cfg.LogtailCollectInterval = interval
    69  	}
    70  }
    71  
    72  // WithServerSendTimeout sets timeout for response sending.
    73  func WithServerSendTimeout(timeout time.Duration) ServerOption {
    74  	return func(s *LogtailServer) {
    75  		s.cfg.ResponseSendTimeout = timeout
    76  	}
    77  }
    78  
    79  // tableLogtail describes a table's logtail.
    80  type wrapLogtail struct {
    81  	id   TableID
    82  	tail logtail.TableLogtail
    83  }
    84  
    85  // sessionError describes error when writing via morpc client session.
    86  type sessionError struct {
    87  	session *Session
    88  	err     error
    89  }
    90  
    91  // subscription describes new subscription.
    92  type subscription struct {
    93  	timeout time.Duration
    94  	tableID TableID
    95  	req     *logtail.SubscribeRequest
    96  	session *Session
    97  }
    98  
    99  // LogtailServer handles logtail push logic.
   100  type LogtailServer struct {
   101  	pool struct {
   102  		requests  LogtailRequestPool
   103  		responses LogtailResponsePool
   104  		segments  LogtailServerSegmentPool
   105  	}
   106  	maxChunkSize int
   107  
   108  	rt     runtime.Runtime
   109  	logger *log.MOLogger
   110  
   111  	// FIXME: change s.cfg.LogtailCollectInterval as hearbeat interval
   112  	cfg *options.LogtailServerCfg
   113  
   114  	ssmgr     *SessionManager
   115  	waterline *Waterliner
   116  
   117  	errChan chan sessionError // errChan has no buffer in order to improve sensitivity.
   118  	subChan chan subscription
   119  	event   *Notifier
   120  
   121  	logtail taelogtail.Logtailer
   122  
   123  	rpc morpc.RPCServer
   124  
   125  	rootCtx    context.Context
   126  	cancelFunc context.CancelFunc
   127  	stopper    *stopper.Stopper
   128  }
   129  
   130  // NewLogtailServer initializes a server for logtail push model.
   131  func NewLogtailServer(
   132  	address string, cfg *options.LogtailServerCfg, logtail taelogtail.Logtailer, rt runtime.Runtime, opts ...ServerOption,
   133  ) (*LogtailServer, error) {
   134  	s := &LogtailServer{
   135  		rt:        rt,
   136  		logger:    rt.Logger(),
   137  		cfg:       cfg,
   138  		ssmgr:     NewSessionManager(),
   139  		waterline: NewWaterliner(),
   140  		errChan:   make(chan sessionError, 1),
   141  		subChan:   make(chan subscription, 100),
   142  		logtail:   logtail,
   143  	}
   144  
   145  	for _, opt := range opts {
   146  		opt(s)
   147  	}
   148  
   149  	uid, _ := uuid.NewV7()
   150  	s.logger = s.logger.Named(LogtailServiceRPCName).
   151  		With(zap.String("server-id", uid.String()))
   152  
   153  	s.pool.requests = NewLogtailRequestPool()
   154  	s.pool.responses = NewLogtailResponsePool()
   155  	s.pool.segments = NewLogtailServerSegmentPool(int(s.cfg.RpcMaxMessageSize))
   156  	s.maxChunkSize = s.pool.segments.LeastEffectiveCapacity()
   157  	if s.maxChunkSize <= 0 {
   158  		panic("rpc max message size isn't enough")
   159  	}
   160  
   161  	s.logger.Debug("max data chunk size for segment", zap.Int("value", s.maxChunkSize))
   162  
   163  	codecOpts := []morpc.CodecOption{
   164  		morpc.WithCodecMaxBodySize(int(s.cfg.RpcMaxMessageSize)),
   165  	}
   166  	if s.cfg.RpcEnableChecksum {
   167  		codecOpts = append(codecOpts, morpc.WithCodecEnableChecksum())
   168  	}
   169  	codec := morpc.NewMessageCodec(func() morpc.Message {
   170  		return s.pool.requests.Acquire()
   171  	}, codecOpts...)
   172  
   173  	rpc, err := morpc.NewRPCServer(LogtailServiceRPCName, address, codec,
   174  		morpc.WithServerLogger(s.logger.RawLogger()),
   175  		morpc.WithServerGoettyOptions(
   176  			goetty.WithSessionReleaseMsgFunc(func(v interface{}) {
   177  				msg := v.(morpc.RPCMessage)
   178  				if !msg.InternalMessage() {
   179  					s.pool.segments.Release(msg.Message.(*LogtailResponseSegment))
   180  				}
   181  			}),
   182  		),
   183  	)
   184  	if err != nil {
   185  		return nil, err
   186  	}
   187  
   188  	rpc.RegisterRequestHandler(s.onMessage)
   189  	s.rpc = rpc
   190  
   191  	// control background goroutines
   192  	ctx, cancel := context.WithCancel(context.Background())
   193  	s.rootCtx = ctx
   194  	s.cancelFunc = cancel
   195  	s.stopper = stopper.NewStopper(
   196  		LogtailServiceRPCName, stopper.WithLogger(s.logger.RawLogger()),
   197  	)
   198  
   199  	// receive logtail on event
   200  	s.event = NewNotifier(s.rootCtx, eventBufferSize)
   201  	logtail.RegisterCallback(s.event.NotifyLogtail)
   202  
   203  	return s, nil
   204  }
   205  
   206  // onMessage is the handler for morpc client session.
   207  func (s *LogtailServer) onMessage(
   208  	ctx context.Context,
   209  	value morpc.RPCMessage,
   210  	seq uint64,
   211  	cs morpc.ClientSession,
   212  ) error {
   213  	ctx, span := trace.Debug(ctx, "LogtailServer.onMessage")
   214  	defer span.End()
   215  
   216  	logger := s.logger
   217  	request := value.Message
   218  	msg, ok := request.(*LogtailRequest)
   219  	if !ok {
   220  		logger.Fatal("receive invalid message", zap.Any("message", request))
   221  	}
   222  	defer s.pool.requests.Release(msg)
   223  
   224  	select {
   225  	case <-ctx.Done():
   226  		return nil
   227  	default:
   228  	}
   229  
   230  	stream := morpcStream{
   231  		streamID: msg.RequestId,
   232  		remote:   cs.RemoteAddress(),
   233  		limit:    s.maxChunkSize,
   234  		logger:   s.logger,
   235  		cs:       cs,
   236  		segments: s.pool.segments,
   237  	}
   238  
   239  	if req := msg.GetSubscribeTable(); req != nil {
   240  		logger.Debug("on subscription", zap.Any("request", req))
   241  		return s.onSubscription(ctx, stream, req)
   242  	}
   243  
   244  	if req := msg.GetUnsubscribeTable(); req != nil {
   245  		logger.Debug("on unsubscription", zap.Any("request", req))
   246  		return s.onUnsubscription(ctx, stream, req)
   247  	}
   248  
   249  	return moerr.NewInvalidArg(ctx, "request", msg)
   250  }
   251  
   252  // onSubscription handls subscription.
   253  func (s *LogtailServer) onSubscription(
   254  	sendCtx context.Context, stream morpcStream, req *logtail.SubscribeRequest,
   255  ) error {
   256  	logger := s.logger
   257  
   258  	tableID := MarshalTableID(req.Table)
   259  	session := s.ssmgr.GetSession(
   260  		// FIXME: using s.cfg
   261  		s.rootCtx, logger, s.pool.responses, s, stream,
   262  		s.cfg.ResponseSendTimeout,
   263  		s.cfg.RPCStreamPoisonTime,
   264  		s.cfg.LogtailCollectInterval,
   265  	)
   266  
   267  	repeated := session.Register(tableID, *req.Table)
   268  	if repeated {
   269  		logger.Info("repeated sub request", zap.String("table ID", string(tableID)))
   270  		return nil
   271  	}
   272  
   273  	sub := subscription{
   274  		timeout: ContextTimeout(sendCtx, s.cfg.ResponseSendTimeout),
   275  		tableID: tableID,
   276  		req:     req,
   277  		session: session,
   278  	}
   279  
   280  	for {
   281  		select {
   282  		case <-s.rootCtx.Done():
   283  			logger.Error("logtail server context done", zap.Error(s.rootCtx.Err()))
   284  			return s.rootCtx.Err()
   285  		case <-sendCtx.Done():
   286  			logger.Error("request context done", zap.Error(sendCtx.Err()))
   287  			return sendCtx.Err()
   288  		case <-time.After(time.Second):
   289  			logger.Error("cannot send subscription request, retry",
   290  				zap.Int("chan cap", cap(s.subChan)),
   291  				zap.Int("chan len", len(s.subChan)),
   292  			)
   293  		case s.subChan <- sub:
   294  			return nil
   295  		}
   296  	}
   297  }
   298  
   299  // onUnsubscription sends response for unsubscription.
   300  func (s *LogtailServer) onUnsubscription(
   301  	sendCtx context.Context, stream morpcStream, req *logtail.UnsubscribeRequest,
   302  ) error {
   303  	tableID := MarshalTableID(req.Table)
   304  	session := s.ssmgr.GetSession(
   305  		// FIXME: using s.cfg
   306  		s.rootCtx, s.logger, s.pool.responses, s, stream,
   307  		s.cfg.ResponseSendTimeout,
   308  		s.cfg.RPCStreamPoisonTime,
   309  		s.cfg.LogtailCollectInterval,
   310  	)
   311  
   312  	state := session.Unregister(tableID)
   313  	if state == TableNotFound {
   314  		return nil
   315  	}
   316  
   317  	return session.SendUnsubscriptionResponse(sendCtx, *req.Table)
   318  }
   319  
   320  // NotifySessionError notifies session manager with session error.
   321  func (s *LogtailServer) NotifySessionError(
   322  	session *Session, err error,
   323  ) {
   324  	select {
   325  	case <-s.rootCtx.Done():
   326  		s.logger.Error("fail to notify session error", zap.Error(s.rootCtx.Err()))
   327  	case s.errChan <- sessionError{session: session, err: err}:
   328  	}
   329  }
   330  
   331  // sessionErrorHandler handles morpc client session writing error.
   332  func (s *LogtailServer) sessionErrorHandler(ctx context.Context) {
   333  	logger := s.logger
   334  
   335  	for {
   336  		select {
   337  		case <-ctx.Done():
   338  			logger.Error("stop session error handler", zap.Error(ctx.Err()))
   339  			return
   340  
   341  		case e, ok := <-s.errChan:
   342  			if !ok {
   343  				logger.Info("session error channel closed")
   344  				return
   345  			}
   346  
   347  			// drop session directly
   348  			if e.err != nil && s.ssmgr.HasSession(e.session.stream) {
   349  				e.session.PostClean()
   350  				s.ssmgr.DeleteSession(e.session.stream)
   351  			}
   352  		}
   353  	}
   354  }
   355  
   356  // logtailSender sends total or incremental logtail.
   357  func (s *LogtailServer) logtailSender(ctx context.Context) {
   358  	e, ok := <-s.event.C
   359  	if !ok {
   360  		s.logger.Info("publishemtn channel closed")
   361  		return
   362  	}
   363  	s.waterline.Advance(e.to)
   364  	s.logger.Info("init waterline", zap.String("to", e.to.String()))
   365  
   366  	// lastUpdate is used to record the time of update event.
   367  	lastUpdate := time.Now()
   368  
   369  	for {
   370  		select {
   371  		case <-ctx.Done():
   372  			s.logger.Error("stop subscription handler", zap.Error(ctx.Err()))
   373  			return
   374  
   375  		case sub, ok := <-s.subChan:
   376  			if !ok {
   377  				s.logger.Info("subscription channel closed")
   378  				return
   379  			}
   380  
   381  			v2.LogTailSubscriptionCounter.Inc()
   382  			interval := time.Since(lastUpdate)
   383  			if interval > updateEventMaxInterval {
   384  				s.logger.Info("long time passed since last update event", zap.Duration("interval", interval))
   385  
   386  				select {
   387  				case e, ok := <-s.event.C:
   388  					if !ok {
   389  						s.logger.Info("publishment channel closed")
   390  						return
   391  					}
   392  					s.logger.Info("send an update event first as long time passed since last one.")
   393  					s.publishEvent(ctx, e)
   394  					lastUpdate = time.Now()
   395  
   396  				default:
   397  					s.logger.Info("there is no update event, although we want to send it first")
   398  				}
   399  			}
   400  
   401  			s.logger.Info("handle subscription asynchronously", zap.Any("table", sub.req.Table))
   402  			s.sendSubscription(ctx, sub)
   403  
   404  		case e, ok := <-s.event.C:
   405  			if !ok {
   406  				s.logger.Info("publishment channel closed")
   407  				return
   408  			}
   409  			s.publishEvent(ctx, e)
   410  			lastUpdate = time.Now()
   411  		}
   412  	}
   413  }
   414  
   415  func (s *LogtailServer) sendSubscription(ctx context.Context, sub subscription) {
   416  	sendCtx, cancel := context.WithTimeout(ctx, sub.timeout)
   417  	defer cancel()
   418  
   419  	var subErr error
   420  	defer func() {
   421  		if subErr != nil {
   422  			sub.session.Unregister(sub.tableID)
   423  		}
   424  	}()
   425  
   426  	table := *sub.req.Table
   427  	from := timestamp.Timestamp{}
   428  	to := s.waterline.Waterline()
   429  
   430  	// fetch total logtail for table
   431  	var tail logtail.TableLogtail
   432  	var closeCB func()
   433  	moprobe.WithRegion(ctx, moprobe.SubscriptionPullLogTail, func() {
   434  		tail, closeCB, subErr = s.logtail.TableLogtail(sendCtx, table, from, to)
   435  	})
   436  
   437  	if subErr != nil {
   438  		if closeCB != nil {
   439  			closeCB()
   440  		}
   441  		s.logger.Error("fail to fetch table total logtail", zap.Error(subErr), zap.Any("table", table))
   442  		if err := sub.session.SendErrorResponse(
   443  			sendCtx, table, moerr.ErrInternal, "fail to fetch table total logtail",
   444  		); err != nil {
   445  			s.logger.Error("fail to send error response", zap.Error(err))
   446  		}
   447  		return
   448  	}
   449  
   450  	cb := func() {
   451  		if closeCB != nil {
   452  			closeCB()
   453  		}
   454  	}
   455  
   456  	// send subscription response
   457  	subErr = sub.session.SendSubscriptionResponse(sendCtx, tail, cb)
   458  	if subErr != nil {
   459  		s.logger.Error("fail to send subscription response", zap.Error(subErr))
   460  		return
   461  	}
   462  
   463  	// mark table as subscribed
   464  	sub.session.AdvanceState(sub.tableID)
   465  }
   466  
   467  func (s *LogtailServer) publishEvent(ctx context.Context, e event) {
   468  	// NOTE: there's gap between multiple (e.from, e.to], so we
   469  	// maintain waterline to make UpdateResponse monotonous.
   470  	from := s.waterline.Waterline()
   471  	to := e.to
   472  
   473  	wraps := make([]wrapLogtail, 0, len(e.logtails))
   474  	for _, tail := range e.logtails {
   475  		// skip empty logtail
   476  		if tail.CkpLocation == "" && len(tail.Commands) == 0 {
   477  			continue
   478  		}
   479  		wraps = append(wraps, wrapLogtail{
   480  			id:   MarshalTableID(tail.GetTable()),
   481  			tail: tail,
   482  		})
   483  	}
   484  
   485  	// publish incremental logtail for all subscribed tables
   486  	sessions := s.ssmgr.ListSession()
   487  
   488  	if len(sessions) == 0 {
   489  		if e.closeCB != nil {
   490  			e.closeCB()
   491  		}
   492  	} else {
   493  		var refcount atomic.Int32
   494  		closeCB := func() {
   495  			if refcount.Add(-1) == 0 {
   496  				if e.closeCB != nil {
   497  					e.closeCB()
   498  				}
   499  			}
   500  		}
   501  		refcount.Add(int32(len(sessions)))
   502  		for _, session := range sessions {
   503  			if err := session.Publish(ctx, from, to, closeCB, wraps...); err != nil {
   504  				s.logger.Error("fail to publish incremental logtail", zap.Error(err),
   505  					zap.Uint64("stream-id", session.stream.streamID), zap.String("remote", session.stream.remote),
   506  				)
   507  				continue
   508  			}
   509  		}
   510  	}
   511  
   512  	// update waterline for all subscribed tables
   513  	s.waterline.Advance(to)
   514  }
   515  
   516  func (s *LogtailServer) gcDeletedSessions(ctx context.Context) {
   517  	const gcTimeout = time.Hour * 24 * 7 // one week
   518  	ticker := time.NewTicker(time.Minute)
   519  	defer ticker.Stop()
   520  	for {
   521  		select {
   522  		case <-ctx.Done():
   523  			return
   524  		case <-ticker.C:
   525  			func() {
   526  				s.ssmgr.Lock()
   527  				defer s.ssmgr.Unlock()
   528  				var pos int
   529  				for i := range s.ssmgr.deletedClients {
   530  					if time.Since(s.ssmgr.deletedClients[i].deletedAt) > gcTimeout {
   531  						pos++
   532  					} else {
   533  						break
   534  					}
   535  				}
   536  				s.ssmgr.deletedClients = s.ssmgr.deletedClients[pos:]
   537  			}()
   538  		}
   539  	}
   540  }
   541  
   542  // Close closes api server.
   543  func (s *LogtailServer) Close() error {
   544  	s.logger.Info("close logtail service")
   545  
   546  	s.cancelFunc()
   547  	s.stopper.Stop()
   548  	return s.rpc.Close()
   549  }
   550  
   551  // Start starts logtail publishment service.
   552  func (s *LogtailServer) Start() error {
   553  	s.logger.Info("start logtail service")
   554  
   555  	if err := s.stopper.RunNamedTask("session error handler", s.sessionErrorHandler); err != nil {
   556  		s.logger.Error("fail to start session error handler", zap.Error(err))
   557  		return err
   558  	}
   559  
   560  	if err := s.stopper.RunNamedTask("logtail sender", s.logtailSender); err != nil {
   561  		s.logger.Error("fail to start logtail sender", zap.Error(err))
   562  		return err
   563  	}
   564  
   565  	if err := s.stopper.RunNamedTask("session cleaner", s.gcDeletedSessions); err != nil {
   566  		s.logger.Error("fail to start session cleaner", zap.Error(err))
   567  		return err
   568  	}
   569  
   570  	return s.rpc.Start()
   571  }
   572  
   573  // NotifyLogtail provides incremental logtail for server.
   574  func (s *LogtailServer) NotifyLogtail(
   575  	from, to timestamp.Timestamp, closeCB func(), tails ...logtail.TableLogtail,
   576  ) error {
   577  	return s.event.NotifyLogtail(from, to, closeCB, tails...)
   578  }
   579  
   580  func (s *LogtailServer) SessionMgr() *SessionManager {
   581  	return s.ssmgr
   582  }