github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sourcemanager/sorter/mounted_iter.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 14 package sorter 15 16 import ( 17 "context" 18 "time" 19 20 "github.com/pingcap/tiflow/cdc/entry" 21 "github.com/pingcap/tiflow/cdc/model" 22 "github.com/pingcap/tiflow/cdc/processor/memquota" 23 "github.com/prometheus/client_golang/prometheus" 24 ) 25 26 // MountedEventIter is just like EventIterator, but returns mounted events. 27 type MountedEventIter struct { 28 iter EventIterator 29 mg entry.MounterGroup 30 quota *memquota.MemQuota 31 32 rawEvents []rawEvent 33 rawEventBuffer rawEvent 34 nextToEmit int 35 savedIterError error 36 37 mountWaitDuration prometheus.Observer 38 } 39 40 // NewMountedEventIter creates a MountedEventIter instance. 41 func NewMountedEventIter( 42 changefeedID model.ChangeFeedID, 43 iter EventIterator, 44 mg entry.MounterGroup, 45 maxBatchSize int, 46 quota *memquota.MemQuota, 47 ) *MountedEventIter { 48 return &MountedEventIter{ 49 iter: iter, 50 mg: mg, 51 quota: quota, 52 rawEvents: make([]rawEvent, 0, maxBatchSize), 53 54 mountWaitDuration: mountWaitDuration.WithLabelValues(changefeedID.Namespace, changefeedID.ID), 55 } 56 } 57 58 // Next returns the next mounted event. 59 func (i *MountedEventIter) Next(ctx context.Context) (event *model.PolymorphicEvent, txnFinished Position, err error) { 60 // There are no events in mounting. Fetch more events and mounting them. 61 // The batch size is determined by `maxBatchSize`. 62 if i.nextToEmit >= len(i.rawEvents) { 63 if err = i.readBatch(ctx); err != nil { 64 return 65 } 66 } 67 68 // Check whether there are events in mounting or not. 69 if i.nextToEmit < len(i.rawEvents) { 70 idx := i.nextToEmit 71 72 startWait := time.Now() 73 if err = i.rawEvents[idx].event.WaitFinished(ctx); err != nil { 74 return 75 } 76 i.mountWaitDuration.Observe(time.Since(startWait).Seconds()) 77 i.quota.Refund(uint64(i.rawEvents[idx].size)) 78 79 event = i.rawEvents[idx].event 80 txnFinished = i.rawEvents[idx].txnFinished 81 i.nextToEmit += 1 82 } 83 return 84 } 85 86 func (i *MountedEventIter) readBatch(ctx context.Context) error { 87 if i.mg == nil || i.iter == nil { 88 return nil 89 } 90 91 i.nextToEmit = 0 92 i.rawEvents = i.rawEvents[:0] 93 94 if i.rawEventBuffer.event != nil { 95 i.rawEventBuffer.event.SetUpFinishedCh() 96 if err := i.mg.AddEvent(ctx, i.rawEventBuffer.event); err != nil { 97 i.mg = nil 98 return err 99 } 100 i.rawEvents = append(i.rawEvents, i.rawEventBuffer) 101 i.rawEventBuffer.event = nil 102 } 103 104 keepFetching := true 105 for keepFetching && len(i.rawEvents) < cap(i.rawEvents) { 106 event, txnFinished, err := i.iter.Next() 107 if err != nil { 108 return err 109 } 110 if event == nil { 111 i.savedIterError = i.iter.Close() 112 i.iter = nil 113 break 114 } 115 116 var size int64 117 if event.RawKV != nil { 118 size = event.RawKV.ApproximateDataSize() 119 } 120 keepFetching = i.quota.TryAcquire(uint64(size)) 121 if !keepFetching { 122 i.quota.ForceAcquire(uint64(size)) 123 } 124 125 var mountStarted bool 126 event.SetUpFinishedCh() 127 if len(i.rawEvents) > 0 { 128 mountStarted, err = i.mg.TryAddEvent(ctx, event) 129 keepFetching = keepFetching && mountStarted 130 } else { 131 err = i.mg.AddEvent(ctx, event) 132 mountStarted = true 133 } 134 if err != nil { 135 i.mg = nil 136 return err 137 } 138 if mountStarted { 139 i.rawEvents = append(i.rawEvents, rawEvent{event, txnFinished, size}) 140 } else { 141 i.rawEventBuffer.event = event 142 i.rawEventBuffer.txnFinished = txnFinished 143 i.rawEventBuffer.size = size 144 } 145 } 146 return nil 147 } 148 149 // Close implements sorter.EventIterator. 150 func (i *MountedEventIter) Close() error { 151 for idx := i.nextToEmit; idx < len(i.rawEvents); idx++ { 152 if i.rawEvents[idx].size != 0 { 153 i.quota.Refund(uint64(i.rawEvents[idx].size)) 154 } 155 } 156 if i.savedIterError != nil { 157 return i.savedIterError 158 } 159 if i.iter != nil { 160 return i.iter.Close() 161 } 162 return nil 163 } 164 165 type rawEvent struct { 166 event *model.PolymorphicEvent 167 txnFinished Position 168 size int64 169 }