github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/puller/memorysorter/entry_sorter.go (about) 1 // Copyright 2020 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 memorysorter 15 16 import ( 17 "context" 18 "sort" 19 "sync" 20 "sync/atomic" 21 "time" 22 23 "github.com/pingcap/errors" 24 "github.com/pingcap/tiflow/cdc/model" 25 "github.com/pingcap/tiflow/pkg/notify" 26 "golang.org/x/sync/errgroup" 27 ) 28 29 // DDLPullerTableName is the fake table name for ddl puller 30 const DDLPullerTableName = "DDL_PULLER" 31 32 // EntrySorter accepts out-of-order raw kv entries and output sorted entries. 33 // For now, it only uses for DDL puller and test. 34 type EntrySorter struct { 35 unsorted []*model.PolymorphicEvent 36 lock sync.Mutex 37 resolvedTsGroup []uint64 38 closed int32 39 40 outputCh chan *model.PolymorphicEvent 41 resolvedNotifier *notify.Notifier 42 changeFeedID model.ChangeFeedID 43 } 44 45 // NewEntrySorter creates a new EntrySorter 46 func NewEntrySorter(changeFeedID model.ChangeFeedID) *EntrySorter { 47 return &EntrySorter{ 48 resolvedNotifier: new(notify.Notifier), 49 outputCh: make(chan *model.PolymorphicEvent, 128000), 50 changeFeedID: changeFeedID, 51 } 52 } 53 54 // Run runs EntrySorter 55 func (es *EntrySorter) Run(ctx context.Context) error { 56 changefeedID := es.changeFeedID 57 tableName := DDLPullerTableName 58 metricEntrySorterResolvedChanSizeGauge := entrySorterResolvedChanSizeGauge. 59 WithLabelValues(changefeedID.Namespace, changefeedID.ID, tableName) 60 metricEntrySorterOutputChanSizeGauge := entrySorterOutputChanSizeGauge. 61 WithLabelValues(changefeedID.Namespace, changefeedID.ID, tableName) 62 metricEntryUnsortedSizeGauge := entrySorterUnsortedSizeGauge. 63 WithLabelValues(changefeedID.Namespace, changefeedID.ID, tableName) 64 metricEntrySorterSortDuration := entrySorterSortDuration. 65 WithLabelValues(changefeedID.Namespace, changefeedID.ID, tableName) 66 metricEntrySorterMergeDuration := entrySorterMergeDuration. 67 WithLabelValues(changefeedID.Namespace, changefeedID.ID, tableName) 68 69 output := func(ctx context.Context, entry *model.PolymorphicEvent) { 70 select { 71 case <-ctx.Done(): 72 return 73 case es.outputCh <- entry: 74 } 75 } 76 77 errg, ctx := errgroup.WithContext(ctx) 78 receiver, err := es.resolvedNotifier.NewReceiver(1000 * time.Millisecond) 79 if err != nil { 80 return err 81 } 82 defer es.resolvedNotifier.Close() 83 errg.Go(func() error { 84 var sorted []*model.PolymorphicEvent 85 for { 86 select { 87 case <-ctx.Done(): 88 atomic.StoreInt32(&es.closed, 1) 89 close(es.outputCh) 90 return errors.Trace(ctx.Err()) 91 case <-time.After(defaultMetricInterval): 92 metricEntrySorterOutputChanSizeGauge.Set(float64(len(es.outputCh))) 93 es.lock.Lock() 94 metricEntrySorterResolvedChanSizeGauge.Set(float64(len(es.resolvedTsGroup))) 95 metricEntryUnsortedSizeGauge.Set(float64(len(es.unsorted))) 96 es.lock.Unlock() 97 case <-receiver.C: 98 es.lock.Lock() 99 if len(es.resolvedTsGroup) == 0 { 100 es.lock.Unlock() 101 continue 102 } 103 resolvedTsGroup := es.resolvedTsGroup 104 es.resolvedTsGroup = nil 105 toSort := es.unsorted 106 es.unsorted = nil 107 es.lock.Unlock() 108 109 resEvents := make([]*model.PolymorphicEvent, len(resolvedTsGroup)) 110 for i, rts := range resolvedTsGroup { 111 // regionID = 0 means the event is produced by TiCDC 112 resEvents[i] = model.NewResolvedPolymorphicEvent(0, rts) 113 } 114 toSort = append(toSort, resEvents...) 115 startTime := time.Now() 116 sort.Slice(toSort, func(i, j int) bool { 117 return eventLess(toSort[i], toSort[j]) 118 }) 119 metricEntrySorterSortDuration.Observe(time.Since(startTime).Seconds()) 120 maxResolvedTs := resolvedTsGroup[len(resolvedTsGroup)-1] 121 122 startTime = time.Now() 123 var merged []*model.PolymorphicEvent 124 mergeEvents(toSort, sorted, func(entry *model.PolymorphicEvent) { 125 if entry.CRTs <= maxResolvedTs { 126 output(ctx, entry) 127 } else { 128 merged = append(merged, entry) 129 } 130 }) 131 metricEntrySorterMergeDuration.Observe(time.Since(startTime).Seconds()) 132 sorted = merged 133 } 134 } 135 }) 136 return errg.Wait() 137 } 138 139 // AddEntry adds an RawKVEntry to the EntryGroup 140 func (es *EntrySorter) AddEntry(_ context.Context, entry *model.PolymorphicEvent) { 141 if atomic.LoadInt32(&es.closed) != 0 { 142 return 143 } 144 es.lock.Lock() 145 defer es.lock.Unlock() 146 if entry.IsResolved() { 147 es.resolvedTsGroup = append(es.resolvedTsGroup, entry.CRTs) 148 es.resolvedNotifier.Notify() 149 } else { 150 es.unsorted = append(es.unsorted, entry) 151 } 152 } 153 154 // Output returns the sorted raw kv output channel 155 func (es *EntrySorter) Output() <-chan *model.PolymorphicEvent { 156 return es.outputCh 157 } 158 159 func eventLess(i *model.PolymorphicEvent, j *model.PolymorphicEvent) bool { 160 return model.ComparePolymorphicEvents(i, j) 161 } 162 163 func mergeEvents(kvsA []*model.PolymorphicEvent, kvsB []*model.PolymorphicEvent, output func(*model.PolymorphicEvent)) { 164 var i, j int 165 for i < len(kvsA) && j < len(kvsB) { 166 if eventLess(kvsA[i], kvsB[j]) { 167 output(kvsA[i]) 168 i++ 169 } else { 170 output(kvsB[j]) 171 j++ 172 } 173 } 174 for ; i < len(kvsA); i++ { 175 output(kvsA[i]) 176 } 177 for ; j < len(kvsB); j++ { 178 output(kvsB[j]) 179 } 180 }