github.com/matrixorigin/matrixone@v0.7.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 "time" 20 21 "github.com/fagongzi/goetty/v2" 22 "github.com/google/uuid" 23 "go.uber.org/zap" 24 25 "github.com/matrixorigin/matrixone/pkg/common/log" 26 "github.com/matrixorigin/matrixone/pkg/common/moerr" 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 "github.com/matrixorigin/matrixone/pkg/txn/clock" 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 ) 37 38 const ( 39 LogtailServiceRPCName = "logtail-push-rpc" 40 41 // minimal duration to detect slow morpc stream 42 minStreamPoisionTime = 5 * time.Millisecond 43 ) 44 45 // TableID is type for api.TableID 46 type TableID string 47 48 type ServerOption func(*LogtailServer) 49 50 // WithServerMaxMessageSize sets max rpc message size 51 func WithServerMaxMessageSize(maxMessageSize int64) ServerOption { 52 return func(s *LogtailServer) { 53 s.cfg.RpcMaxMessageSize = maxMessageSize 54 } 55 } 56 57 // WithServerPayloadCopyBufferSize sets payload copy buffer size 58 func WithServerPayloadCopyBufferSize(size int64) ServerOption { 59 return func(s *LogtailServer) { 60 s.cfg.RpcPayloadCopyBufferSize = size 61 } 62 } 63 64 // WithServerEnableChecksum enables checksum 65 func WithServerEnableChecksum(enable bool) ServerOption { 66 return func(s *LogtailServer) { 67 s.cfg.RpcEnableChecksum = enable 68 } 69 } 70 71 // WithServerCollectInterval sets logtail collection interval. 72 func WithServerCollectInterval(interval time.Duration) ServerOption { 73 return func(s *LogtailServer) { 74 s.cfg.LogtailCollectInterval = interval 75 } 76 } 77 78 func WithServerSendTimeout(timeout time.Duration) ServerOption { 79 return func(s *LogtailServer) { 80 s.cfg.ResponseSendTimeout = timeout 81 } 82 } 83 84 func WithServerMaxLogtailFetchFailure(max int) ServerOption { 85 return func(s *LogtailServer) { 86 s.cfg.MaxLogtailFetchFailure = max 87 } 88 } 89 90 // tableLogtail describes a table's logtail. 91 type wrapLogtail struct { 92 id TableID 93 tail logtail.TableLogtail 94 } 95 96 // sessionError describes error when writing via morpc client session. 97 type sessionError struct { 98 session *Session 99 err error 100 } 101 102 // subscription describes new subscription. 103 type subscription struct { 104 timeout time.Duration 105 tableID TableID 106 req *logtail.SubscribeRequest 107 session *Session 108 } 109 110 // LogtailServer handles logtail push logic. 111 type LogtailServer struct { 112 pool struct { 113 requests RequestPool 114 responses ResponsePool 115 segments SegmentPool 116 } 117 maxChunkSize int 118 119 rt runtime.Runtime 120 logger *log.MOLogger 121 clock clock.Clock 122 123 cfg *options.LogtailServerCfg 124 125 ssmgr *SessionManager 126 waterline *Waterliner 127 subscribed *TableStacker 128 129 errChan chan sessionError // errChan has no buffer in order to improve sensitivity. 130 subChan chan subscription 131 132 logtail taelogtail.Logtailer 133 134 rpc morpc.RPCServer 135 136 rootCtx context.Context 137 cancelFunc context.CancelFunc 138 stopper *stopper.Stopper 139 } 140 141 // NewLogtailServer initializes a server for logtail push model. 142 func NewLogtailServer( 143 address string, cfg *options.LogtailServerCfg, logtail taelogtail.Logtailer, rt runtime.Runtime, opts ...ServerOption, 144 ) (*LogtailServer, error) { 145 s := &LogtailServer{ 146 rt: rt, 147 logger: rt.Logger(), 148 clock: rt.Clock(), 149 cfg: cfg, 150 ssmgr: NewSessionManager(), 151 waterline: NewWaterliner(rt.Clock()), 152 subscribed: NewTableStacker(), 153 errChan: make(chan sessionError), 154 subChan: make(chan subscription, 10), 155 logtail: logtail, 156 } 157 158 for _, opt := range opts { 159 opt(s) 160 } 161 162 s.logger = s.logger.Named(LogtailServiceRPCName). 163 With(zap.String("server-id", uuid.NewString())) 164 165 s.pool.requests = NewRequestPool() 166 s.pool.responses = NewResponsePool() 167 s.pool.segments = NewSegmentPool(int(s.cfg.RpcMaxMessageSize)) 168 s.maxChunkSize = s.pool.segments.LeastEffectiveCapacity() 169 if s.maxChunkSize <= 0 { 170 panic("rpc max message size isn't enough") 171 } 172 173 s.logger.Debug("max data chunk size for segment", zap.Int("value", s.maxChunkSize)) 174 175 codecOpts := []morpc.CodecOption{ 176 morpc.WithCodecMaxBodySize(int(s.cfg.RpcMaxMessageSize)), 177 } 178 if s.cfg.RpcEnableChecksum { 179 codecOpts = append(codecOpts, morpc.WithCodecEnableChecksum()) 180 } 181 codec := morpc.NewMessageCodec(s.pool.requests.Acquire, codecOpts...) 182 183 rpc, err := morpc.NewRPCServer(LogtailServiceRPCName, address, codec, 184 morpc.WithServerLogger(s.logger.RawLogger()), 185 morpc.WithServerGoettyOptions( 186 goetty.WithSessionReleaseMsgFunc(func(v interface{}) { 187 msg := v.(morpc.RPCMessage) 188 if !msg.InternalMessage() { 189 s.pool.segments.Release(msg.Message.(*LogtailResponseSegment)) 190 } 191 }), 192 ), 193 ) 194 if err != nil { 195 return nil, err 196 } 197 198 rpc.RegisterRequestHandler(s.onMessage) 199 s.rpc = rpc 200 201 // control background goroutines 202 ctx, cancel := context.WithCancel(context.Background()) 203 s.rootCtx = ctx 204 s.cancelFunc = cancel 205 s.stopper = stopper.NewStopper( 206 LogtailServiceRPCName, stopper.WithLogger(s.logger.RawLogger()), 207 ) 208 209 return s, nil 210 } 211 212 // onMessage is the handler for morpc client session. 213 func (s *LogtailServer) onMessage( 214 ctx context.Context, request morpc.Message, seq uint64, cs morpc.ClientSession, 215 ) error { 216 ctx, span := trace.Debug(ctx, "LogtailServer.onMessage") 217 defer span.End() 218 219 logger := s.logger 220 221 msg, ok := request.(*LogtailRequest) 222 if !ok { 223 logger.Fatal("receive invalid message", zap.Any("message", request)) 224 } 225 defer s.pool.requests.Release(msg) 226 227 select { 228 case <-ctx.Done(): 229 return nil 230 default: 231 } 232 233 stream := morpcStream{ 234 streamID: msg.RequestId, 235 limit: s.maxChunkSize, 236 logger: s.logger, 237 cs: cs, 238 segments: s.pool.segments, 239 } 240 241 if req := msg.GetSubscribeTable(); req != nil { 242 logger.Debug("on subscritpion", zap.Any("request", req)) 243 return s.onSubscription(ctx, stream, req) 244 } 245 246 if req := msg.GetUnsubscribeTable(); req != nil { 247 logger.Debug("on unsubscritpion", zap.Any("request", req)) 248 return s.onUnsubscription(ctx, stream, req) 249 } 250 251 return moerr.NewInvalidArg(ctx, "request", msg) 252 } 253 254 // onSubscription handls subscription. 255 func (s *LogtailServer) onSubscription( 256 sendCtx context.Context, stream morpcStream, req *logtail.SubscribeRequest, 257 ) error { 258 logger := s.logger 259 260 tableID := TableID(req.Table.String()) 261 session := s.ssmgr.GetSession( 262 s.rootCtx, logger, s.cfg.ResponseSendTimeout, s.pool.responses, s, stream, s.streamPoisionTime(), 263 ) 264 265 repeated := session.Register(tableID, *req.Table) 266 if repeated { 267 return nil 268 } 269 270 sub := subscription{ 271 timeout: ContextTimeout(sendCtx, s.cfg.ResponseSendTimeout), 272 tableID: tableID, 273 req: req, 274 session: session, 275 } 276 277 select { 278 case <-s.rootCtx.Done(): 279 logger.Error("logtail server context done", zap.Error(s.rootCtx.Err())) 280 return s.rootCtx.Err() 281 case <-sendCtx.Done(): 282 logger.Error("request context done", zap.Error(sendCtx.Err())) 283 return sendCtx.Err() 284 case s.subChan <- sub: 285 } 286 287 return nil 288 } 289 290 // onUnsubscription sends response for unsubscription. 291 func (s *LogtailServer) onUnsubscription( 292 sendCtx context.Context, stream morpcStream, req *logtail.UnsubscribeRequest, 293 ) error { 294 tableID := TableID(req.Table.String()) 295 session := s.ssmgr.GetSession( 296 s.rootCtx, s.logger, s.cfg.ResponseSendTimeout, s.pool.responses, s, stream, s.streamPoisionTime(), 297 ) 298 299 state := session.Unregister(tableID) 300 if state == TableNotFound { 301 return nil 302 } 303 304 if state == TableSubscribed { 305 s.subscribed.Unregister(tableID) 306 } 307 308 return session.SendUnsubscriptionResponse(sendCtx, *req.Table) 309 } 310 311 // streamPoisionTime returns poision duration for stream. 312 func (s *LogtailServer) streamPoisionTime() time.Duration { 313 duration := s.cfg.LogtailCollectInterval * 2 314 if duration < minStreamPoisionTime { 315 duration = minStreamPoisionTime 316 } 317 return duration 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 { 349 e.session.PostClean() 350 s.ssmgr.DeleteSession(e.session.stream) 351 s.subscribed.Unregister(e.session.ListSubscribedTable()...) 352 } 353 } 354 } 355 } 356 357 // logtailSender sends total or additional logtail. 358 func (s *LogtailServer) logtailSender(ctx context.Context) { 359 logger := s.logger 360 361 publishTicker := time.NewTicker(s.cfg.LogtailCollectInterval) 362 defer publishTicker.Stop() 363 364 risk := 0 365 for { 366 select { 367 case <-ctx.Done(): 368 logger.Error("stop subscription handler", zap.Error(ctx.Err())) 369 return 370 371 case sub, ok := <-s.subChan: 372 if !ok { 373 logger.Info("subscription channel closed") 374 return 375 } 376 377 logger.Debug("handle subscription asynchronously", zap.Any("table", sub.req.Table)) 378 379 subscriptionFunc := func(sub subscription) { 380 sendCtx, cancel := context.WithTimeout(ctx, sub.timeout) 381 defer cancel() 382 383 var subErr error 384 defer func() { 385 if subErr != nil { 386 sub.session.Unregister(sub.tableID) 387 } 388 }() 389 390 table := *sub.req.Table 391 from := timestamp.Timestamp{} 392 to := s.waterline.Waterline() 393 394 // fetch total logtail for table 395 tail, subErr := s.logtail.TableLogtail(sendCtx, table, from, to) 396 if subErr != nil { 397 logger.Error("fail to fetch table total logtail", zap.Error(subErr), zap.Any("table", table)) 398 if err := sub.session.SendErrorResponse( 399 sendCtx, table, moerr.ErrInternal, "fail to fetch table total logtail", 400 ); err != nil { 401 logger.Error("fail to send error response", zap.Error(err)) 402 } 403 return 404 } 405 406 logger.Debug("send subscription response", zap.Any("table", sub.req.Table), zap.Any("To", to.String())) 407 408 // send subscription response 409 subErr = sub.session.SendSubscriptionResponse(sendCtx, tail) 410 if subErr != nil { 411 logger.Error("fail to send subscription response", zap.Error(subErr)) 412 return 413 } 414 415 // mark table as subscribed 416 sub.session.AdvanceState(sub.tableID) 417 418 // register subscribed table 419 s.subscribed.Register(sub.tableID, table) 420 } 421 422 subscriptionFunc(sub) 423 424 case <-publishTicker.C: 425 publishmentFunc := func() { 426 defer func() { 427 if risk >= s.cfg.MaxLogtailFetchFailure { 428 panic("fail to fetch additional logtail many times") 429 } 430 }() 431 432 from := s.waterline.Waterline() 433 to, _ := s.clock.Now() 434 435 tables := s.subscribed.ListTable() 436 wraps := make([]wrapLogtail, 0, len(tables)) 437 for _, t := range tables { 438 tail, err := s.logtail.TableLogtail(ctx, t.table, from, to) 439 if err != nil { 440 logger.Error("fail to fetch additional logtail", zap.Error(err), zap.Any("table", t.table)) 441 risk += 1 442 return 443 } 444 // skip empty logtail 445 if tail.CkpLocation == "" && len(tail.Commands) == 0 { 446 continue 447 } 448 wraps = append(wraps, wrapLogtail{id: t.id, tail: tail}) 449 } 450 451 logger.Debug("publish additional logtail", zap.Any("From", from.String()), zap.Any("To", to.String())) 452 453 // publish additional logtail for all subscribed tables 454 for _, session := range s.ssmgr.ListSession() { 455 if err := session.Publish(ctx, from, to, wraps...); err != nil { 456 logger.Error("fail to publish additional logtail", zap.Error(err), zap.Uint64("stream-id", session.stream.streamID)) 457 continue 458 } 459 } 460 461 // update waterline for all subscribed tables 462 s.waterline.Advance(to) 463 } 464 465 publishmentFunc() 466 } 467 } 468 } 469 470 // Close closes api server. 471 func (s *LogtailServer) Close() error { 472 s.cancelFunc() 473 s.stopper.Stop() 474 return s.rpc.Close() 475 } 476 477 // Start starts logtail publishment service. 478 func (s *LogtailServer) Start() error { 479 logger := s.logger 480 481 if err := s.stopper.RunNamedTask("session error handler", s.sessionErrorHandler); err != nil { 482 logger.Error("fail to start session error handler", zap.Error(err)) 483 return err 484 } 485 486 if err := s.stopper.RunNamedTask("logtail sender", s.logtailSender); err != nil { 487 logger.Error("fail to start logtail sender", zap.Error(err)) 488 return err 489 } 490 491 return s.rpc.Start() 492 }