github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/puller/puller.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 puller 15 16 import ( 17 "context" 18 "sync/atomic" 19 "time" 20 21 "github.com/pingcap/errors" 22 "github.com/pingcap/log" 23 "github.com/pingcap/ticdc/cdc/kv" 24 "github.com/pingcap/ticdc/cdc/model" 25 "github.com/pingcap/ticdc/cdc/puller/frontier" 26 "github.com/pingcap/ticdc/pkg/regionspan" 27 "github.com/pingcap/ticdc/pkg/txnutil" 28 "github.com/pingcap/ticdc/pkg/util" 29 tidbkv "github.com/pingcap/tidb/kv" 30 "github.com/pingcap/tidb/store/tikv" 31 "github.com/pingcap/tidb/store/tikv/oracle" 32 pd "github.com/tikv/pd/client" 33 "go.uber.org/zap" 34 "golang.org/x/sync/errgroup" 35 ) 36 37 const ( 38 defaultPullerEventChanSize = 128 39 defaultPullerOutputChanSize = 128 40 ) 41 42 // Puller pull data from tikv and push changes into a buffer 43 type Puller interface { 44 // Run the puller, continually fetch event from TiKV and add event into buffer 45 Run(ctx context.Context) error 46 GetResolvedTs() uint64 47 Output() <-chan *model.RawKVEntry 48 IsInitialized() bool 49 } 50 51 type pullerImpl struct { 52 pdCli pd.Client 53 kvCli kv.CDCKVClient 54 kvStorage tikv.Storage 55 checkpointTs uint64 56 spans []regionspan.ComparableSpan 57 outputCh chan *model.RawKVEntry 58 tsTracker frontier.Frontier 59 resolvedTs uint64 60 initialized int64 61 enableOldValue bool 62 } 63 64 // NewPuller create a new Puller fetch event start from checkpointTs 65 // and put into buf. 66 func NewPuller( 67 ctx context.Context, 68 pdCli pd.Client, 69 grpcPool kv.GrpcPool, 70 kvStorage tidbkv.Storage, 71 checkpointTs uint64, 72 spans []regionspan.Span, 73 enableOldValue bool, 74 ) Puller { 75 tikvStorage, ok := kvStorage.(tikv.Storage) 76 if !ok { 77 log.Panic("can't create puller for non-tikv storage") 78 } 79 comparableSpans := make([]regionspan.ComparableSpan, len(spans)) 80 for i := range spans { 81 comparableSpans[i] = regionspan.ToComparableSpan(spans[i]) 82 } 83 // To make puller level resolved ts initialization distinguishable, we set 84 // the initial ts for frontier to 0. Once the puller level resolved ts 85 // initialized, the ts should advance to a non-zero value. 86 tsTracker := frontier.NewFrontier(0, comparableSpans...) 87 kvCli := kv.NewCDCKVClient(ctx, pdCli, tikvStorage, grpcPool) 88 p := &pullerImpl{ 89 pdCli: pdCli, 90 kvCli: kvCli, 91 kvStorage: tikvStorage, 92 checkpointTs: checkpointTs, 93 spans: comparableSpans, 94 outputCh: make(chan *model.RawKVEntry, defaultPullerOutputChanSize), 95 tsTracker: tsTracker, 96 resolvedTs: checkpointTs, 97 initialized: 0, 98 enableOldValue: enableOldValue, 99 } 100 return p 101 } 102 103 func (p *pullerImpl) Output() <-chan *model.RawKVEntry { 104 return p.outputCh 105 } 106 107 // Run the puller, continually fetch event from TiKV and add event into buffer 108 func (p *pullerImpl) Run(ctx context.Context) error { 109 defer p.kvCli.Close() 110 111 g, ctx := errgroup.WithContext(ctx) 112 113 checkpointTs := p.checkpointTs 114 eventCh := make(chan model.RegionFeedEvent, defaultPullerEventChanSize) 115 116 lockresolver := txnutil.NewLockerResolver(p.kvStorage) 117 for _, span := range p.spans { 118 span := span 119 120 g.Go(func() error { 121 return p.kvCli.EventFeed(ctx, span, checkpointTs, p.enableOldValue, lockresolver, p, eventCh) 122 }) 123 } 124 125 captureAddr := util.CaptureAddrFromCtx(ctx) 126 changefeedID := util.ChangefeedIDFromCtx(ctx) 127 tableID, _ := util.TableIDFromCtx(ctx) 128 metricOutputChanSize := outputChanSizeHistogram.WithLabelValues(captureAddr, changefeedID) 129 metricEventChanSize := eventChanSizeHistogram.WithLabelValues(captureAddr, changefeedID) 130 metricPullerResolvedTs := pullerResolvedTsGauge.WithLabelValues(captureAddr, changefeedID) 131 metricTxnCollectCounterKv := txnCollectCounter.WithLabelValues(captureAddr, changefeedID, "kv") 132 metricTxnCollectCounterResolved := txnCollectCounter.WithLabelValues(captureAddr, changefeedID, "resolved") 133 defer func() { 134 outputChanSizeHistogram.DeleteLabelValues(captureAddr, changefeedID) 135 eventChanSizeHistogram.DeleteLabelValues(captureAddr, changefeedID) 136 memBufferSizeGauge.DeleteLabelValues(captureAddr, changefeedID) 137 pullerResolvedTsGauge.DeleteLabelValues(captureAddr, changefeedID) 138 kvEventCounter.DeleteLabelValues(captureAddr, changefeedID, "kv") 139 kvEventCounter.DeleteLabelValues(captureAddr, changefeedID, "resolved") 140 txnCollectCounter.DeleteLabelValues(captureAddr, changefeedID, "kv") 141 txnCollectCounter.DeleteLabelValues(captureAddr, changefeedID, "resolved") 142 }() 143 g.Go(func() error { 144 for { 145 select { 146 case <-ctx.Done(): 147 return nil 148 case <-time.After(15 * time.Second): 149 metricEventChanSize.Observe(float64(len(eventCh))) 150 metricOutputChanSize.Observe(float64(len(p.outputCh))) 151 metricPullerResolvedTs.Set(float64(oracle.ExtractPhysical(atomic.LoadUint64(&p.resolvedTs)))) 152 } 153 } 154 }) 155 156 lastResolvedTs := p.checkpointTs 157 g.Go(func() error { 158 output := func(raw *model.RawKVEntry) error { 159 // even after https://github.com/pingcap/ticdc/pull/2038, kv client 160 // could still miss region change notification, which leads to resolved 161 // ts update missing in puller, however resolved ts fallback here can 162 // be ignored since no late data is received and the guarantee of 163 // resolved ts is not broken. 164 if raw.CRTs < p.resolvedTs || (raw.CRTs == p.resolvedTs && raw.OpType != model.OpTypeResolved) { 165 log.Warn("The CRTs is fallen back in pulelr", 166 zap.Reflect("row", raw), 167 zap.Uint64("CRTs", raw.CRTs), 168 zap.Uint64("resolvedTs", p.resolvedTs), 169 zap.Int64("tableID", tableID)) 170 return nil 171 } 172 select { 173 case <-ctx.Done(): 174 return errors.Trace(ctx.Err()) 175 case p.outputCh <- raw: 176 } 177 return nil 178 } 179 180 start := time.Now() 181 initialized := false 182 for { 183 var e model.RegionFeedEvent 184 select { 185 case e = <-eventCh: 186 case <-ctx.Done(): 187 return errors.Trace(ctx.Err()) 188 } 189 if e.Val != nil { 190 metricTxnCollectCounterKv.Inc() 191 if err := output(e.Val); err != nil { 192 return errors.Trace(err) 193 } 194 } else if e.Resolved != nil { 195 metricTxnCollectCounterResolved.Inc() 196 if !regionspan.IsSubSpan(e.Resolved.Span, p.spans...) { 197 log.Panic("the resolved span is not in the total span", 198 zap.Reflect("resolved", e.Resolved), 199 zap.Int64("tableID", tableID), 200 zap.Reflect("spans", p.spans), 201 ) 202 } 203 // Forward is called in a single thread 204 p.tsTracker.Forward(e.Resolved.Span, e.Resolved.ResolvedTs) 205 resolvedTs := p.tsTracker.Frontier() 206 if resolvedTs > 0 && !initialized { 207 // Advancing to a non-zero value means the puller level 208 // resolved ts is initialized. 209 atomic.StoreInt64(&p.initialized, 1) 210 initialized = true 211 212 spans := make([]string, 0, len(p.spans)) 213 for i := range p.spans { 214 spans = append(spans, p.spans[i].String()) 215 } 216 log.Info("puller is initialized", 217 zap.Duration("duration", time.Since(start)), 218 zap.String("changefeed", changefeedID), 219 zap.Int64("tableID", tableID), 220 zap.Strings("spans", spans), 221 zap.Uint64("resolvedTs", resolvedTs)) 222 } 223 if !initialized || resolvedTs == lastResolvedTs { 224 continue 225 } 226 lastResolvedTs = resolvedTs 227 err := output(&model.RawKVEntry{CRTs: resolvedTs, OpType: model.OpTypeResolved, RegionID: e.RegionID}) 228 if err != nil { 229 return errors.Trace(err) 230 } 231 atomic.StoreUint64(&p.resolvedTs, resolvedTs) 232 } 233 } 234 }) 235 return g.Wait() 236 } 237 238 func (p *pullerImpl) GetResolvedTs() uint64 { 239 return atomic.LoadUint64(&p.resolvedTs) 240 } 241 242 func (p *pullerImpl) IsInitialized() bool { 243 return atomic.LoadInt64(&p.initialized) > 0 244 }