github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/cloudstorage/defragmenter.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 package cloudstorage 14 15 import ( 16 "context" 17 18 "github.com/pingcap/tiflow/pkg/chann" 19 "github.com/pingcap/tiflow/pkg/errors" 20 "github.com/pingcap/tiflow/pkg/hash" 21 ) 22 23 // defragmenter is used to handle event fragments which can be registered 24 // out of order. 25 type defragmenter struct { 26 lastDispatchedSeq uint64 27 future map[uint64]eventFragment 28 inputCh <-chan eventFragment 29 outputChs []*chann.DrainableChann[eventFragment] 30 hasher *hash.PositionInertia 31 } 32 33 func newDefragmenter( 34 inputCh <-chan eventFragment, 35 outputChs []*chann.DrainableChann[eventFragment], 36 ) *defragmenter { 37 return &defragmenter{ 38 future: make(map[uint64]eventFragment), 39 inputCh: inputCh, 40 outputChs: outputChs, 41 hasher: hash.NewPositionInertia(), 42 } 43 } 44 45 func (d *defragmenter) run(ctx context.Context) error { 46 defer d.close() 47 for { 48 select { 49 case <-ctx.Done(): 50 d.future = nil 51 return errors.Trace(ctx.Err()) 52 case frag, ok := <-d.inputCh: 53 if !ok { 54 return nil 55 } 56 // check whether to write messages to output channel right now 57 next := d.lastDispatchedSeq + 1 58 if frag.seqNumber == next { 59 d.writeMsgsConsecutive(ctx, frag) 60 } else if frag.seqNumber > next { 61 d.future[frag.seqNumber] = frag 62 } else { 63 return nil 64 } 65 } 66 } 67 } 68 69 func (d *defragmenter) writeMsgsConsecutive( 70 ctx context.Context, 71 start eventFragment, 72 ) { 73 d.dispatchFragToDMLWorker(start) 74 75 // try to dispatch more fragments to DML workers 76 for { 77 select { 78 case <-ctx.Done(): 79 return 80 default: 81 } 82 next := d.lastDispatchedSeq + 1 83 if frag, ok := d.future[next]; ok { 84 delete(d.future, next) 85 d.dispatchFragToDMLWorker(frag) 86 } else { 87 return 88 } 89 } 90 } 91 92 func (d *defragmenter) dispatchFragToDMLWorker(frag eventFragment) { 93 tableName := frag.versionedTable.TableNameWithPhysicTableID 94 d.hasher.Reset() 95 d.hasher.Write([]byte(tableName.Schema), []byte(tableName.Table)) 96 workerID := d.hasher.Sum32() % uint32(len(d.outputChs)) 97 d.outputChs[workerID].In() <- frag 98 d.lastDispatchedSeq = frag.seqNumber 99 } 100 101 func (d *defragmenter) close() { 102 for _, ch := range d.outputChs { 103 ch.CloseAndDrain() 104 } 105 }