github.com/matrixorigin/matrixone@v1.2.0/pkg/util/batchpipe/batch_pipe.go (about) 1 // Copyright 2022 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 batchpipe 16 17 import ( 18 "bytes" 19 "context" 20 "fmt" 21 "runtime" 22 "sync" 23 "sync/atomic" 24 "time" 25 26 "github.com/matrixorigin/matrixone/pkg/common/moerr" 27 "github.com/matrixorigin/matrixone/pkg/logutil" 28 ) 29 30 const chanCapConst = 10000 31 32 // HasName decides which table the owner will go to 33 type HasName interface { 34 GetName() string 35 } 36 37 // Reminder to force flush ItemBuffer 38 type Reminder interface { 39 RemindNextAfter() time.Duration 40 RemindBackOff() 41 RemindBackOffCnt() int 42 RemindReset() 43 } 44 45 // ItemBuffer Stash items and construct a batch can be stored. for instance, a sql inserting all items into a table 46 type ItemBuffer[T any, B any] interface { 47 Reminder 48 Add(item T) 49 Reset() 50 IsEmpty() bool 51 ShouldFlush() bool 52 // GetBatch use bytes.Buffer to mitigate mem allocation and the returned bytes should own its data 53 GetBatch(ctx context.Context, buf *bytes.Buffer) B 54 } 55 56 type PipeImpl[T any, B any] interface { 57 // NewItemBuffer create a new buffer for one kind of Item 58 NewItemBuffer(name string) ItemBuffer[T, B] 59 // NewItemBatchHandler handle the StoreBatch from an ItemBuffer, for example, execute an insert sql. 60 // this handle may be running on multiple goroutine 61 NewItemBatchHandler(ctx context.Context) func(batch B) 62 } 63 64 type backOffClock struct { 65 base time.Duration 66 current time.Duration 67 backOffCnt int 68 next func(int, time.Duration) time.Duration 69 } 70 71 func NewConstantClock(cycle time.Duration) Reminder { 72 return NewExpBackOffClock(cycle, cycle, 1) 73 } 74 75 func NewExpBackOffClock(init, max time.Duration, base int) Reminder { 76 next := func(_ int, current time.Duration) time.Duration { 77 if current < max { 78 current = current * time.Duration(base) 79 if current >= max { 80 current = max 81 } 82 } 83 return current 84 } 85 return NewBackOffClock(init, next) 86 } 87 88 func NewBackOffClock(base time.Duration, f func(int, time.Duration) time.Duration) Reminder { 89 return &backOffClock{ 90 base: base, 91 current: base, 92 backOffCnt: 0, 93 next: f, 94 } 95 } 96 func (c *backOffClock) RemindNextAfter() time.Duration { return c.current } 97 func (c *backOffClock) RemindReset() { c.current = c.base; c.backOffCnt = 0 } 98 func (c *backOffClock) RemindBackOffCnt() int { return c.backOffCnt } 99 func (c *backOffClock) RemindBackOff() { 100 c.backOffCnt += 1 101 if c.next != nil { 102 c.current = c.next(c.backOffCnt, c.current) 103 } 104 } 105 106 type reminderRegistry struct { 107 registry map[string]*time.Timer 108 C chan string 109 } 110 111 func newReminderRegistry() *reminderRegistry { 112 return &reminderRegistry{ 113 registry: make(map[string]*time.Timer), 114 C: make(chan string, chanCapConst), 115 } 116 } 117 118 func (r *reminderRegistry) Register(name string, after time.Duration) { 119 if after <= 0 { 120 return 121 } 122 if r.registry[name] != nil { 123 panic(fmt.Sprintf("%s already registered", name)) 124 } 125 r.registry[name] = time.AfterFunc(after, func() { r.C <- name }) 126 } 127 128 func (r *reminderRegistry) Reset(name string, after time.Duration) { 129 if after <= 0 { 130 return 131 } 132 if t := r.registry[name]; t != nil { 133 t.Reset(after) 134 } 135 } 136 137 func (r *reminderRegistry) CleanAll() { 138 for _, timer := range r.registry { 139 timer.Stop() 140 } 141 } 142 143 type baseBatchPipeOpts struct { 144 // the number of goroutines to handle buffer batch, default is runtime.NumCPU() 145 BatchWorkerNum int 146 // the number of goroutines to merge items into a buffer, default is 1 147 BufferWorkerNum int 148 // ItemNameFormatter 149 ItemNameFormatter func(HasName) string 150 } 151 152 type BaseBatchPipeOpt interface { 153 ApplyTo(*baseBatchPipeOpts) 154 } 155 156 type PipeWithBatchWorkerNum int 157 158 func (x PipeWithBatchWorkerNum) ApplyTo(o *baseBatchPipeOpts) { 159 o.BatchWorkerNum = int(x) 160 } 161 162 type PipeWithBufferWorkerNum int 163 164 func (x PipeWithBufferWorkerNum) ApplyTo(o *baseBatchPipeOpts) { 165 o.BufferWorkerNum = int(x) 166 } 167 168 type PipeWithItemNameFormatter func(HasName) string 169 170 func (x PipeWithItemNameFormatter) ApplyTo(o *baseBatchPipeOpts) { 171 o.ItemNameFormatter = x 172 } 173 174 func defaultBaseBatchPipeOpts() baseBatchPipeOpts { 175 return baseBatchPipeOpts{ 176 BatchWorkerNum: runtime.NumCPU(), 177 BufferWorkerNum: 1, 178 // default use T.GetName() 179 ItemNameFormatter: func(T HasName) string { return T.GetName() }, 180 } 181 } 182 183 type BaseBatchPipe[T HasName, B any] struct { 184 impl PipeImpl[T, B] 185 isRunning int32 186 opts baseBatchPipeOpts 187 dropped int64 188 itemCh chan T 189 batchCh chan B 190 sendingLock sync.RWMutex 191 mergeStopWg sync.WaitGroup 192 batchStopWg sync.WaitGroup 193 batchWorkerCancel context.CancelFunc 194 mergeWorkerCancel context.CancelFunc 195 } 196 197 func NewBaseBatchPipe[T HasName, B any](impl PipeImpl[T, B], opts ...BaseBatchPipeOpt) *BaseBatchPipe[T, B] { 198 initOpts := defaultBaseBatchPipeOpts() 199 for _, o := range opts { 200 o.ApplyTo(&initOpts) 201 } 202 return &BaseBatchPipe[T, B]{ 203 impl: impl, 204 opts: initOpts, 205 itemCh: make(chan T, chanCapConst), 206 batchCh: make(chan B, chanCapConst), 207 } 208 } 209 210 // SendItem returns error when pipe is closed 211 func (bc *BaseBatchPipe[T, B]) SendItem(ctx context.Context, items ...T) error { 212 if atomic.LoadInt32(&bc.isRunning) == 0 { 213 return moerr.NewWarn(ctx, "Collector has been stopped") 214 } 215 // avoid data race on itemCh between concurrent sending and closing 216 bc.sendingLock.RLock() 217 defer bc.sendingLock.RUnlock() 218 for _, item := range items { 219 select { 220 case bc.itemCh <- item: 221 default: 222 atomic.AddInt64(&bc.dropped, 1) 223 } 224 } 225 return nil 226 } 227 228 // Start kicks off the merge workers and batch workers, return false if the pipe is workinghas been called 229 func (bc *BaseBatchPipe[T, B]) Start(ctx context.Context) bool { 230 if atomic.SwapInt32(&bc.isRunning, 1) == 1 { 231 return false 232 } 233 bc.startBatchWorker(ctx) 234 bc.startMergeWorker(ctx) 235 return true 236 } 237 238 // Stop terminates all workers. If graceful asked, wait until all items are processed, 239 // otherwise, quit immediately. Caller can use the returned channel to wait Stop finish 240 func (bc *BaseBatchPipe[T, B]) Stop(graceful bool) (<-chan struct{}, bool) { 241 if atomic.SwapInt32(&bc.isRunning, 0) == 0 { 242 return nil, false 243 } 244 245 logutil.Debugf("BaseBatchPipe accept graceful(%v) Stop", graceful) 246 stopCh := make(chan struct{}) 247 if graceful { 248 go func() { 249 bc.sendingLock.Lock() 250 close(bc.itemCh) 251 bc.sendingLock.Unlock() 252 bc.mergeStopWg.Wait() 253 close(bc.batchCh) 254 bc.batchStopWg.Wait() 255 close(stopCh) 256 }() 257 } else { 258 bc.batchWorkerCancel() 259 bc.mergeWorkerCancel() 260 go func() { 261 bc.mergeStopWg.Wait() 262 bc.batchStopWg.Wait() 263 close(stopCh) 264 }() 265 } 266 return stopCh, true 267 } 268 269 func (bc *BaseBatchPipe[T, B]) startBatchWorker(inputCtx context.Context) { 270 ctx, cancel := context.WithCancel(inputCtx) 271 bc.batchWorkerCancel = cancel 272 for i := 0; i < bc.opts.BatchWorkerNum; i++ { 273 bc.batchStopWg.Add(1) 274 go bc.batchWorker(ctx) 275 } 276 } 277 278 func (bc *BaseBatchPipe[T, B]) startMergeWorker(inputCtx context.Context) { 279 ctx, cancel := context.WithCancel(inputCtx) 280 bc.mergeWorkerCancel = cancel 281 for i := 0; i < bc.opts.BufferWorkerNum; i++ { 282 bc.mergeStopWg.Add(1) 283 go bc.mergeWorker(ctx) 284 } 285 } 286 287 func (bc *BaseBatchPipe[T, B]) batchWorker(ctx context.Context) { 288 defer bc.batchStopWg.Done() 289 quitMsg := quitUnknown 290 defer logutil.Debugf("batchWorker quit: %s", &quitMsg) 291 f := bc.impl.NewItemBatchHandler(ctx) 292 for { 293 select { 294 case <-ctx.Done(): 295 quitMsg = quitCancel 296 return 297 case batch, ok := <-bc.batchCh: 298 if !ok { 299 quitMsg = quitChannelClose 300 return 301 } 302 f(batch) 303 } 304 } 305 } 306 307 type quitReason string 308 309 var ( 310 quitUnknown quitReason = "unknown" 311 quitCancel quitReason = "force cancel" 312 quitChannelClose quitReason = "closed channel" 313 ) 314 315 func (s *quitReason) String() string { return string(*s) } 316 317 func (bc *BaseBatchPipe[T, B]) mergeWorker(ctx context.Context) { 318 defer bc.mergeStopWg.Done() 319 bufByNames := make(map[string]ItemBuffer[T, B]) 320 batchbuf := new(bytes.Buffer) 321 registry := newReminderRegistry() 322 quitMsg := quitUnknown 323 defer registry.CleanAll() 324 defer logutil.Debugf("mergeWorker quit: %v", &quitMsg) 325 326 doFlush := func(name string, itembuf ItemBuffer[T, B]) { 327 batch := itembuf.GetBatch(ctx, batchbuf) 328 bc.batchCh <- batch 329 itembuf.Reset() 330 itembuf.RemindReset() 331 registry.Reset(name, itembuf.RemindNextAfter()) 332 } 333 334 for { 335 select { 336 case <-ctx.Done(): 337 quitMsg = quitCancel 338 return 339 case item, ok := <-bc.itemCh: 340 if !ok { 341 for name, buf := range bufByNames { 342 if !buf.IsEmpty() { 343 doFlush(name, buf) 344 } 345 } 346 quitMsg = quitChannelClose 347 return 348 } 349 name := bc.opts.ItemNameFormatter(item) 350 itembuf := bufByNames[name] 351 if itembuf == nil { 352 itembuf = bc.impl.NewItemBuffer(name) 353 bufByNames[name] = itembuf 354 registry.Register(name, itembuf.RemindNextAfter()) 355 } 356 357 itembuf.Add(item) 358 if itembuf.ShouldFlush() { 359 doFlush(name, itembuf) 360 } else if itembuf.RemindBackOffCnt() > 0 { 361 itembuf.RemindReset() 362 registry.Reset(name, itembuf.RemindNextAfter()) 363 } 364 case name := <-registry.C: 365 if itembuf := bufByNames[name]; itembuf != nil { 366 if itembuf.IsEmpty() { 367 itembuf.RemindBackOff() 368 registry.Reset(name, itembuf.RemindNextAfter()) 369 } else { 370 doFlush(name, itembuf) 371 } 372 } 373 } 374 } 375 }