github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logtail/service/session.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"
    20  	"sync/atomic"
    21  	"time"
    22  	"unsafe"
    23  
    24  	"go.uber.org/zap"
    25  
    26  	"github.com/matrixorigin/matrixone/pkg/common/log"
    27  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    28  	"github.com/matrixorigin/matrixone/pkg/common/morpc"
    29  	"github.com/matrixorigin/matrixone/pkg/pb/api"
    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  )
    34  
    35  type TableState int
    36  
    37  const (
    38  	TableOnSubscription TableState = iota
    39  	TableSubscribed
    40  	TableNotFound
    41  )
    42  
    43  var (
    44  	// responseBufferSize is the buffer channel capacity for every morpc stream.
    45  	// We couldn't set this size to an unlimited value,
    46  	// because channel's memory is allocated according to the specified size.
    47  	// NOTE: we afford 1MiB heap memory for every morpc stream.
    48  	// but items within the channel would consume extra heap memory.
    49  	responseBufferSize = 1024 * 1024 / int(unsafe.Sizeof(message{}))
    50  )
    51  
    52  // SessionManager manages all client sessions.
    53  type SessionManager struct {
    54  	sync.RWMutex
    55  	clients        map[morpcStream]*Session
    56  	deletedClients []*Session
    57  }
    58  
    59  // NewSessionManager constructs a session manager.
    60  func NewSessionManager() *SessionManager {
    61  	return &SessionManager{
    62  		clients: make(map[morpcStream]*Session),
    63  	}
    64  }
    65  
    66  // GetSession constructs a session for new morpc.ClientSession.
    67  func (sm *SessionManager) GetSession(
    68  	rootCtx context.Context,
    69  	logger *log.MOLogger,
    70  	responses LogtailResponsePool,
    71  	notifier SessionErrorNotifier,
    72  	stream morpcStream,
    73  	sendTimeout time.Duration,
    74  	poisonTime time.Duration,
    75  	heartbeatInterval time.Duration,
    76  ) *Session {
    77  	sm.Lock()
    78  	defer sm.Unlock()
    79  
    80  	if _, ok := sm.clients[stream]; !ok {
    81  		sm.clients[stream] = NewSession(
    82  			rootCtx, logger, responses, notifier, stream,
    83  			sendTimeout, poisonTime, heartbeatInterval,
    84  		)
    85  	}
    86  	return sm.clients[stream]
    87  }
    88  
    89  // DeleteSession deletes session from manager.
    90  func (sm *SessionManager) DeleteSession(stream morpcStream) {
    91  	sm.Lock()
    92  	defer sm.Unlock()
    93  	ss, ok := sm.clients[stream]
    94  	if ok {
    95  		delete(sm.clients, stream)
    96  		ss.deletedAt = time.Now()
    97  		sm.deletedClients = append(sm.deletedClients, ss)
    98  	}
    99  }
   100  
   101  func (sm *SessionManager) HasSession(stream morpcStream) bool {
   102  	sm.RLock()
   103  	defer sm.RUnlock()
   104  	_, ok := sm.clients[stream]
   105  	return ok
   106  }
   107  
   108  // ListSession takes a snapshot of all sessions.
   109  func (sm *SessionManager) ListSession() []*Session {
   110  	sm.RLock()
   111  	defer sm.RUnlock()
   112  
   113  	sessions := make([]*Session, 0, len(sm.clients))
   114  	for _, ss := range sm.clients {
   115  		sessions = append(sessions, ss)
   116  	}
   117  	return sessions
   118  }
   119  
   120  // AddSession is only for test.
   121  func (sm *SessionManager) AddSession(id uint64) {
   122  	sm.Lock()
   123  	defer sm.Unlock()
   124  	stream := morpcStream{
   125  		streamID: id,
   126  	}
   127  	if _, ok := sm.clients[stream]; !ok {
   128  		sm.clients[stream] = &Session{
   129  			stream: stream,
   130  		}
   131  	}
   132  }
   133  
   134  // AddDeletedSession is only for test.
   135  func (sm *SessionManager) AddDeletedSession(id uint64) {
   136  	sm.Lock()
   137  	defer sm.Unlock()
   138  	stream := morpcStream{streamID: id}
   139  	sm.deletedClients = append(sm.deletedClients, &Session{
   140  		stream: stream,
   141  	})
   142  }
   143  
   144  func (sm *SessionManager) DeletedSessions() []*Session {
   145  	sm.RLock()
   146  	defer sm.RUnlock()
   147  	sessions := make([]*Session, 0, len(sm.deletedClients))
   148  	sessions = append(sessions, sm.deletedClients...)
   149  	return sessions
   150  }
   151  
   152  // message describes response to be sent.
   153  type message struct {
   154  	createAt time.Time
   155  	timeout  time.Duration
   156  	response *LogtailResponse
   157  }
   158  
   159  // morpcStream describes morpc stream.
   160  type morpcStream struct {
   161  	streamID uint64
   162  	remote   string
   163  	limit    int
   164  	logger   *log.MOLogger
   165  	cs       morpc.ClientSession
   166  	segments LogtailServerSegmentPool
   167  }
   168  
   169  // Close closes morpc client session.
   170  func (s *morpcStream) Close() error {
   171  	return s.cs.Close()
   172  }
   173  
   174  // write sends response by segment.
   175  func (s *morpcStream) write(
   176  	ctx context.Context, response *LogtailResponse,
   177  ) error {
   178  	size := response.ProtoSize()
   179  	buf := make([]byte, size)
   180  	n, err := response.MarshalToSizedBuffer(buf[:size])
   181  	if err != nil {
   182  		return err
   183  	}
   184  	chunks := Split(buf[:n], s.limit)
   185  
   186  	if len(chunks) > 1 {
   187  		s.logger.Info("send response by segment",
   188  			zap.Int("chunk-number", len(chunks)),
   189  			zap.Int("chunk-limit", s.limit),
   190  			zap.Int("message-size", size),
   191  		)
   192  	} else {
   193  		s.logger.Debug("send response by segment",
   194  			zap.Int("chunk-number", len(chunks)),
   195  			zap.Int("chunk-limit", s.limit),
   196  			zap.Int("message-size", size),
   197  		)
   198  	}
   199  
   200  	for index, chunk := range chunks {
   201  		seg := s.segments.Acquire()
   202  		seg.SetID(s.streamID)
   203  		seg.MessageSize = int32(size)
   204  		seg.Sequence = int32(index + 1)
   205  		seg.MaxSequence = int32(len(chunks))
   206  		n := copy(seg.Payload, chunk)
   207  		seg.Payload = seg.Payload[:n]
   208  
   209  		s.logger.Debug("real segment proto size", zap.Int("ProtoSize", seg.ProtoSize()))
   210  
   211  		st := time.Now()
   212  		err := s.cs.Write(ctx, seg)
   213  		v2.LogtailSendNetworkHistogram.Observe(time.Since(st).Seconds())
   214  		if err != nil {
   215  			return err
   216  		}
   217  	}
   218  
   219  	return nil
   220  }
   221  
   222  // Session manages subscription for logtail client.
   223  type Session struct {
   224  	sessionCtx context.Context
   225  	cancelFunc context.CancelFunc
   226  	wg         sync.WaitGroup
   227  
   228  	logger      *log.MOLogger
   229  	sendTimeout time.Duration
   230  	responses   LogtailResponsePool
   231  	notifier    SessionErrorNotifier
   232  
   233  	stream     morpcStream
   234  	poisonTime time.Duration
   235  	sendChan   chan message
   236  
   237  	active int32
   238  
   239  	mu     sync.RWMutex
   240  	tables map[TableID]TableState
   241  
   242  	heartbeatInterval time.Duration
   243  	heartbeatTimer    *time.Timer
   244  	exactFrom         timestamp.Timestamp
   245  	publishInit       sync.Once
   246  
   247  	deletedAt time.Time
   248  	sendMu    struct {
   249  		sync.Mutex
   250  		lastBeforeSend time.Time
   251  		lastAfterSend  time.Time
   252  	}
   253  }
   254  
   255  type SessionErrorNotifier interface {
   256  	NotifySessionError(*Session, error)
   257  }
   258  
   259  // NewSession constructs a session for logtail client.
   260  func NewSession(
   261  	rootCtx context.Context,
   262  	logger *log.MOLogger,
   263  	responses LogtailResponsePool,
   264  	notifier SessionErrorNotifier,
   265  	stream morpcStream,
   266  	sendTimeout time.Duration,
   267  	poisonTime time.Duration,
   268  	heartbeatInterval time.Duration,
   269  ) *Session {
   270  	ctx, cancel := context.WithCancel(rootCtx)
   271  	ss := &Session{
   272  		sessionCtx:        ctx,
   273  		cancelFunc:        cancel,
   274  		logger:            logger.With(zap.Uint64("stream-id", stream.streamID), zap.String("remote", stream.remote)),
   275  		sendTimeout:       sendTimeout,
   276  		responses:         responses,
   277  		notifier:          notifier,
   278  		stream:            stream,
   279  		poisonTime:        poisonTime,
   280  		sendChan:          make(chan message, responseBufferSize), // buffer response for morpc client session
   281  		tables:            make(map[TableID]TableState),
   282  		heartbeatInterval: heartbeatInterval,
   283  		heartbeatTimer:    time.NewTimer(heartbeatInterval),
   284  	}
   285  
   286  	ss.logger.Info("initialize new session for morpc stream")
   287  
   288  	sender := func() {
   289  		defer ss.wg.Done()
   290  
   291  		var cnt int64
   292  		timer := time.NewTimer(100 * time.Second)
   293  
   294  		for {
   295  			select {
   296  			case <-ss.sessionCtx.Done():
   297  				ss.logger.Error("stop session sender", zap.Error(ss.sessionCtx.Err()))
   298  				return
   299  
   300  			case <-timer.C:
   301  				ss.logger.Info("send logtail channel blocked", zap.Int64("sendRound", cnt))
   302  				if ss.TableCount() == 0 {
   303  					ss.logger.Error("no tables are subscribed yet, close this session")
   304  					ss.notifier.NotifySessionError(ss, moerr.NewInternalError(ctx, "no tables are subscribed"))
   305  					return
   306  				}
   307  				timer.Reset(10 * time.Second)
   308  
   309  			case msg, ok := <-ss.sendChan:
   310  				if !ok {
   311  					ss.logger.Info("session sender channel closed")
   312  					return
   313  				}
   314  				v2.LogTailSendQueueSizeGauge.Set(float64(len(ss.sendChan)))
   315  				sendFunc := func() error {
   316  					defer ss.responses.Release(msg.response)
   317  
   318  					ctx, cancel := context.WithTimeout(ss.sessionCtx, msg.timeout)
   319  					defer cancel()
   320  
   321  					now := time.Now()
   322  					v2.LogtailSendLatencyHistogram.Observe(float64(now.Sub(msg.createAt).Seconds()))
   323  
   324  					defer func() {
   325  						v2.LogtailSendTotalHistogram.Observe(time.Since(now).Seconds())
   326  					}()
   327  
   328  					ss.OnBeforeSend(now)
   329  					err := ss.stream.write(ctx, msg.response)
   330  					ss.OnAfterSend(now, cnt, msg.response.ProtoSize())
   331  					if err != nil {
   332  						ss.logger.Error("fail to send logtail response",
   333  							zap.Error(err),
   334  							zap.String("timeout", msg.timeout.String()),
   335  							zap.String("remote address", ss.RemoteAddress()),
   336  						)
   337  						return err
   338  					}
   339  					return nil
   340  				}
   341  
   342  				if err := sendFunc(); err != nil {
   343  					ss.notifier.NotifySessionError(ss, err)
   344  					return
   345  				}
   346  				cnt++
   347  				timer.Reset(10 * time.Second)
   348  			}
   349  		}
   350  	}
   351  
   352  	ss.wg.Add(1)
   353  	go sender()
   354  
   355  	return ss
   356  }
   357  
   358  // Drop closes sender goroutine.
   359  func (ss *Session) PostClean() {
   360  	ss.logger.Info("clean session for morpc stream")
   361  
   362  	// close morpc stream, maybe verbose
   363  	if err := ss.stream.Close(); err != nil {
   364  		ss.logger.Error("fail to close morpc client session", zap.Error(err))
   365  	}
   366  
   367  	ss.cancelFunc()
   368  	ss.wg.Wait()
   369  
   370  	left := len(ss.sendChan)
   371  
   372  	// release all left responses in sendChan
   373  	if left > 0 {
   374  		i := 0
   375  		for resp := range ss.sendChan {
   376  			ss.responses.Release(resp.response)
   377  			i++
   378  			if i >= left {
   379  				break
   380  			}
   381  		}
   382  		ss.logger.Info("release left responses", zap.Int("left", left))
   383  	}
   384  }
   385  
   386  // Register registers table for client.
   387  //
   388  // The returned true value indicates repeated subscription.
   389  func (ss *Session) Register(id TableID, table api.TableID) bool {
   390  	ss.mu.Lock()
   391  	defer ss.mu.Unlock()
   392  
   393  	if _, ok := ss.tables[id]; ok {
   394  		return true
   395  	}
   396  	ss.tables[id] = TableOnSubscription
   397  	return false
   398  }
   399  
   400  // Unsubscribe unsubscribes table.
   401  func (ss *Session) Unregister(id TableID) TableState {
   402  	ss.mu.Lock()
   403  	defer ss.mu.Unlock()
   404  
   405  	state, ok := ss.tables[id]
   406  	if !ok {
   407  		return TableNotFound
   408  	}
   409  	delete(ss.tables, id)
   410  	return state
   411  }
   412  
   413  // ListTable takes a snapshot of all
   414  func (ss *Session) ListSubscribedTable() []TableID {
   415  	ss.mu.RLock()
   416  	defer ss.mu.RUnlock()
   417  
   418  	ids := make([]TableID, 0, len(ss.tables))
   419  	for id, state := range ss.tables {
   420  		if state == TableSubscribed {
   421  			ids = append(ids, id)
   422  		}
   423  	}
   424  	return ids
   425  }
   426  
   427  // FilterLogtail selects logtail for expected tables.
   428  func (ss *Session) FilterLogtail(tails ...wrapLogtail) []logtail.TableLogtail {
   429  	ss.mu.RLock()
   430  	defer ss.mu.RUnlock()
   431  
   432  	qualified := make([]logtail.TableLogtail, 0, 4)
   433  	for _, t := range tails {
   434  		if state, ok := ss.tables[t.id]; ok && state == TableSubscribed {
   435  			qualified = append(qualified, t.tail)
   436  		} else {
   437  			ss.logger.Debug("table not subscribed, filter out",
   438  				zap.String("id", string(t.id)),
   439  				zap.String("checkpoint", t.tail.CkpLocation),
   440  			)
   441  		}
   442  	}
   443  	return qualified
   444  }
   445  
   446  // Publish publishes incremental logtail.
   447  func (ss *Session) Publish(
   448  	ctx context.Context, from, to timestamp.Timestamp, closeCB func(), wraps ...wrapLogtail,
   449  ) error {
   450  	// no need to send incremental logtail if no table subscribed
   451  	if atomic.LoadInt32(&ss.active) <= 0 {
   452  		if closeCB != nil {
   453  			closeCB()
   454  		}
   455  		return nil
   456  	}
   457  
   458  	// keep `logtail.UpdateResponse.From` monotonous
   459  	ss.publishInit.Do(func() {
   460  		ss.exactFrom = from
   461  	})
   462  
   463  	qualified := ss.FilterLogtail(wraps...)
   464  	// if there's no incremental logtail, heartbeat by interval
   465  	if len(qualified) == 0 {
   466  		select {
   467  		case <-ss.heartbeatTimer.C:
   468  			break
   469  		default:
   470  			if closeCB != nil {
   471  				closeCB()
   472  			}
   473  			return nil
   474  		}
   475  	}
   476  
   477  	sendCtx, cancel := context.WithTimeout(ctx, ss.sendTimeout)
   478  	defer cancel()
   479  
   480  	err := ss.SendUpdateResponse(sendCtx, ss.exactFrom, to, closeCB, qualified...)
   481  	if err == nil {
   482  		ss.heartbeatTimer.Reset(ss.heartbeatInterval)
   483  		ss.exactFrom = to
   484  	} else {
   485  		ss.notifier.NotifySessionError(ss, err)
   486  	}
   487  	return err
   488  }
   489  
   490  // AdvanceState marks table as subscribed.
   491  func (ss *Session) AdvanceState(id TableID) {
   492  	ss.logger.Debug("mark table as subscribed", zap.String("table-id", string(id)))
   493  
   494  	ss.mu.Lock()
   495  	defer ss.mu.Unlock()
   496  
   497  	if _, ok := ss.tables[id]; !ok {
   498  		return
   499  	}
   500  	ss.tables[id] = TableSubscribed
   501  }
   502  
   503  // SendErrorResponse sends error response to logtail client.
   504  func (ss *Session) SendErrorResponse(
   505  	sendCtx context.Context, table api.TableID, code uint16, message string,
   506  ) error {
   507  	ss.logger.Warn("send error response", zap.Any("table", table), zap.Uint16("code", code), zap.String("message", message))
   508  
   509  	resp := ss.responses.Acquire()
   510  	resp.Response = newErrorResponse(table, code, message)
   511  	return ss.SendResponse(sendCtx, resp)
   512  }
   513  
   514  // SendSubscriptionResponse sends subscription response.
   515  func (ss *Session) SendSubscriptionResponse(
   516  	sendCtx context.Context, tail logtail.TableLogtail, closeCB func(),
   517  ) error {
   518  	ss.logger.Info("send subscription response", zap.Any("table", tail.Table), zap.String("To", tail.Ts.String()))
   519  
   520  	resp := ss.responses.Acquire()
   521  	resp.closeCB = closeCB
   522  	resp.Response = newSubscritpionResponse(tail)
   523  	err := ss.SendResponse(sendCtx, resp)
   524  	if err == nil {
   525  		atomic.AddInt32(&ss.active, 1)
   526  	}
   527  	return err
   528  }
   529  
   530  // SendUnsubscriptionResponse sends unsubscription response.
   531  func (ss *Session) SendUnsubscriptionResponse(
   532  	sendCtx context.Context, table api.TableID,
   533  ) error {
   534  	ss.logger.Info("send unsubscription response", zap.Any("table", table))
   535  
   536  	resp := ss.responses.Acquire()
   537  	resp.Response = newUnsubscriptionResponse(table)
   538  	err := ss.SendResponse(sendCtx, resp)
   539  	if err == nil {
   540  		atomic.AddInt32(&ss.active, -1)
   541  	}
   542  	return err
   543  }
   544  
   545  // SendUpdateResponse sends publishment response.
   546  func (ss *Session) SendUpdateResponse(
   547  	sendCtx context.Context, from, to timestamp.Timestamp, closeCB func(), tails ...logtail.TableLogtail,
   548  ) error {
   549  	ss.logger.Debug("send incremental logtail", zap.Any("From", from.String()), zap.String("To", to.String()), zap.Int("tables", len(tails)))
   550  
   551  	resp := ss.responses.Acquire()
   552  	resp.closeCB = closeCB
   553  	resp.Response = newUpdateResponse(from, to, tails...)
   554  	return ss.SendResponse(sendCtx, resp)
   555  }
   556  
   557  // SendResponse sends response.
   558  //
   559  // If the sender of Session finished, it would block until
   560  // sendCtx/sessionCtx cancelled or timeout.
   561  func (ss *Session) SendResponse(
   562  	sendCtx context.Context, response *LogtailResponse,
   563  ) error {
   564  	select {
   565  	case <-ss.sessionCtx.Done():
   566  		ss.logger.Error("session context done", zap.Error(ss.sessionCtx.Err()))
   567  		ss.responses.Release(response)
   568  		return ss.sessionCtx.Err()
   569  	case <-sendCtx.Done():
   570  		ss.logger.Error("send context done", zap.Error(sendCtx.Err()))
   571  		ss.responses.Release(response)
   572  		return sendCtx.Err()
   573  	default:
   574  	}
   575  
   576  	select {
   577  	case <-time.After(ss.poisonTime):
   578  		ss.logger.Error("poison morpc client session detected, close it",
   579  			zap.Int("buffer-capacity", cap(ss.sendChan)),
   580  			zap.Int("buffer-length", len(ss.sendChan)),
   581  		)
   582  		ss.responses.Release(response)
   583  		if err := ss.stream.Close(); err != nil {
   584  			ss.logger.Error("fail to close poision morpc client session", zap.Error(err))
   585  		}
   586  		return moerr.NewStreamClosedNoCtx()
   587  	case ss.sendChan <- message{timeout: ContextTimeout(sendCtx, ss.sendTimeout), response: response, createAt: time.Now()}:
   588  		return nil
   589  	}
   590  }
   591  
   592  func (ss *Session) Active() int {
   593  	return int(atomic.LoadInt32(&ss.active))
   594  }
   595  
   596  func (ss *Session) Tables() map[TableID]TableState {
   597  	ss.mu.Lock()
   598  	defer ss.mu.Unlock()
   599  	tables := make(map[TableID]TableState, len(ss.tables))
   600  	for k, v := range ss.tables {
   601  		tables[k] = v
   602  	}
   603  	return tables
   604  }
   605  
   606  func (ss *Session) TableCount() int {
   607  	ss.mu.Lock()
   608  	defer ss.mu.Unlock()
   609  	return len(ss.tables)
   610  }
   611  
   612  func (ss *Session) OnBeforeSend(t time.Time) {
   613  	ss.sendMu.Lock()
   614  	defer ss.sendMu.Unlock()
   615  	ss.sendMu.lastBeforeSend = t
   616  }
   617  
   618  func (ss *Session) OnAfterSend(before time.Time, count int64, size int) {
   619  	ss.sendMu.Lock()
   620  	defer ss.sendMu.Unlock()
   621  	now := time.Now()
   622  	cost := now.Sub(before)
   623  	if cost > 10*time.Second {
   624  		ss.logger.Info("send logtail too much",
   625  			zap.Int64("sendRound", count),
   626  			zap.Duration("duration", cost),
   627  			zap.Int("msg size", size))
   628  	}
   629  	ss.sendMu.lastAfterSend = now
   630  }
   631  
   632  func (ss *Session) LastBeforeSend() time.Time {
   633  	ss.sendMu.Lock()
   634  	defer ss.sendMu.Unlock()
   635  	return ss.sendMu.lastBeforeSend
   636  }
   637  
   638  func (ss *Session) LastAfterSend() time.Time {
   639  	ss.sendMu.Lock()
   640  	defer ss.sendMu.Unlock()
   641  	return ss.sendMu.lastAfterSend
   642  }
   643  
   644  func (ss *Session) RemoteAddress() string {
   645  	ss.mu.Lock()
   646  	defer ss.mu.Unlock()
   647  	return ss.stream.remote
   648  }
   649  
   650  func (ss *Session) DeletedAt() time.Time {
   651  	ss.mu.Lock()
   652  	defer ss.mu.Unlock()
   653  	return ss.deletedAt
   654  }
   655  
   656  // newUnsubscriptionResponse constructs response for unsubscription.
   657  func newUnsubscriptionResponse(
   658  	table api.TableID,
   659  ) *logtail.LogtailResponse_UnsubscribeResponse {
   660  	return &logtail.LogtailResponse_UnsubscribeResponse{
   661  		UnsubscribeResponse: &logtail.UnSubscribeResponse{
   662  			Table: &table,
   663  		},
   664  	}
   665  }
   666  
   667  // newUpdateResponse constructs response for publishment.
   668  func newUpdateResponse(
   669  	from, to timestamp.Timestamp, tails ...logtail.TableLogtail,
   670  ) *logtail.LogtailResponse_UpdateResponse {
   671  	return &logtail.LogtailResponse_UpdateResponse{
   672  		UpdateResponse: &logtail.UpdateResponse{
   673  			From:        &from,
   674  			To:          &to,
   675  			LogtailList: tails,
   676  		},
   677  	}
   678  }
   679  
   680  // newSubscritpionResponse constructs response for subscription.
   681  func newSubscritpionResponse(
   682  	tail logtail.TableLogtail,
   683  ) *logtail.LogtailResponse_SubscribeResponse {
   684  	return &logtail.LogtailResponse_SubscribeResponse{
   685  		SubscribeResponse: &logtail.SubscribeResponse{
   686  			Logtail: tail,
   687  		},
   688  	}
   689  }
   690  
   691  // newErrorResponse constructs response for error condition.
   692  func newErrorResponse(
   693  	table api.TableID, code uint16, message string,
   694  ) *logtail.LogtailResponse_Error {
   695  	return &logtail.LogtailResponse_Error{
   696  		Error: &logtail.ErrorResponse{
   697  			Table: &table,
   698  			Status: logtail.Status{
   699  				Code:    uint32(code),
   700  				Message: message,
   701  			},
   702  		},
   703  	}
   704  }