github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sourcemanager/manager.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 sourcemanager 15 16 import ( 17 "context" 18 "time" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/log" 22 "github.com/pingcap/tiflow/cdc/entry" 23 "github.com/pingcap/tiflow/cdc/kv" 24 "github.com/pingcap/tiflow/cdc/kv/sharedconn" 25 "github.com/pingcap/tiflow/cdc/model" 26 "github.com/pingcap/tiflow/cdc/processor/memquota" 27 "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter" 28 "github.com/pingcap/tiflow/cdc/processor/tablepb" 29 "github.com/pingcap/tiflow/cdc/puller" 30 "github.com/pingcap/tiflow/pkg/config" 31 cerrors "github.com/pingcap/tiflow/pkg/errors" 32 "github.com/pingcap/tiflow/pkg/retry" 33 "github.com/pingcap/tiflow/pkg/txnutil" 34 "github.com/pingcap/tiflow/pkg/upstream" 35 "github.com/tikv/client-go/v2/oracle" 36 "github.com/tikv/client-go/v2/tikv" 37 pd "github.com/tikv/pd/client" 38 "go.uber.org/zap" 39 ) 40 41 const defaultMaxBatchSize = 256 42 43 // SourceManager is the manager of the source engine and puller. 44 type SourceManager struct { 45 ready chan struct{} 46 47 // changefeedID is the changefeed ID. 48 // We use it to create the puller and log. 49 changefeedID model.ChangeFeedID 50 // up is the upstream of the puller. 51 up *upstream.Upstream 52 // mg is the mounter group for mount the raw kv entry. 53 mg entry.MounterGroup 54 // engine is the source engine. 55 engine sorter.SortEngine 56 // Used to indicate whether the changefeed is in BDR mode. 57 bdrMode bool 58 startTs model.Ts 59 60 enableTableMonitor bool 61 puller *puller.MultiplexingPuller 62 } 63 64 // New creates a new source manager. 65 func New( 66 changefeedID model.ChangeFeedID, 67 up *upstream.Upstream, 68 mg entry.MounterGroup, 69 engine sorter.SortEngine, 70 bdrMode bool, 71 enableTableMonitor bool, 72 safeModeAtStart bool, 73 ) *SourceManager { 74 return newSourceManager(changefeedID, up, mg, engine, bdrMode, enableTableMonitor, safeModeAtStart) 75 } 76 77 // NewForTest creates a new source manager for testing. 78 func NewForTest( 79 changefeedID model.ChangeFeedID, 80 up *upstream.Upstream, 81 mg entry.MounterGroup, 82 engine sorter.SortEngine, 83 bdrMode bool, 84 ) *SourceManager { 85 return &SourceManager{ 86 ready: make(chan struct{}), 87 changefeedID: changefeedID, 88 up: up, 89 mg: mg, 90 engine: engine, 91 bdrMode: bdrMode, 92 } 93 } 94 95 func isOldUpdateKVEntry(raw *model.RawKVEntry, thresholdTs model.Ts) bool { 96 return raw != nil && raw.IsUpdate() && raw.CRTs < thresholdTs 97 } 98 99 func splitUpdateKVEntry(raw *model.RawKVEntry) (*model.RawKVEntry, *model.RawKVEntry, error) { 100 if raw == nil { 101 return nil, nil, errors.New("nil event cannot be split") 102 } 103 deleteKVEntry := *raw 104 deleteKVEntry.Value = nil 105 106 insertKVEntry := *raw 107 insertKVEntry.OldValue = nil 108 109 return &deleteKVEntry, &insertKVEntry, nil 110 } 111 112 func newSourceManager( 113 changefeedID model.ChangeFeedID, 114 up *upstream.Upstream, 115 mg entry.MounterGroup, 116 engine sorter.SortEngine, 117 bdrMode bool, 118 enableTableMonitor bool, 119 safeModeAtStart bool, 120 ) *SourceManager { 121 mgr := &SourceManager{ 122 ready: make(chan struct{}), 123 changefeedID: changefeedID, 124 up: up, 125 mg: mg, 126 engine: engine, 127 bdrMode: bdrMode, 128 enableTableMonitor: enableTableMonitor, 129 } 130 131 serverConfig := config.GetGlobalServerConfig() 132 grpcPool := sharedconn.NewConnAndClientPool(mgr.up.SecurityConfig, kv.GetGlobalGrpcMetrics()) 133 client := kv.NewSharedClient( 134 mgr.changefeedID, serverConfig, mgr.bdrMode, 135 mgr.up.PDClient, grpcPool, mgr.up.RegionCache, mgr.up.PDClock, 136 txnutil.NewLockerResolver(mgr.up.KVStorage.(tikv.Storage), mgr.changefeedID), 137 ) 138 139 // consume add raw kv entry to the engine. 140 // It will be called by the puller when new raw kv entry is received. 141 consume := func(ctx context.Context, raw *model.RawKVEntry, spans []tablepb.Span) error { 142 if len(spans) > 1 { 143 log.Panic("DML puller subscribes multiple spans", 144 zap.String("namespace", mgr.changefeedID.Namespace), 145 zap.String("changefeed", mgr.changefeedID.ID)) 146 } 147 if raw != nil { 148 if safeModeAtStart && isOldUpdateKVEntry(raw, mgr.startTs) { 149 deleteKVEntry, insertKVEntry, err := splitUpdateKVEntry(raw) 150 if err != nil { 151 return err 152 } 153 deleteEvent := model.NewPolymorphicEvent(deleteKVEntry) 154 insertEvent := model.NewPolymorphicEvent(insertKVEntry) 155 mgr.engine.Add(spans[0], deleteEvent, insertEvent) 156 } else { 157 pEvent := model.NewPolymorphicEvent(raw) 158 mgr.engine.Add(spans[0], pEvent) 159 } 160 } 161 return nil 162 } 163 slots, hasher := mgr.engine.SlotsAndHasher() 164 165 mgr.puller = puller.NewMultiplexingPuller( 166 mgr.changefeedID, 167 client, 168 up.PDClock, 169 consume, 170 slots, 171 hasher, 172 int(serverConfig.KVClient.FrontierConcurrent)) 173 174 return mgr 175 } 176 177 // AddTable adds a table to the source manager. Start puller and register table to the engine. 178 func (m *SourceManager) AddTable(span tablepb.Span, tableName string, startTs model.Ts) { 179 // Add table to the engine first, so that the engine can receive the events from the puller. 180 m.engine.AddTable(span, startTs) 181 182 // Only nil in unit tests. 183 if m.puller != nil { 184 m.puller.Subscribe([]tablepb.Span{span}, startTs, tableName) 185 } 186 } 187 188 // RemoveTable removes a table from the source manager. Stop puller and unregister table from the engine. 189 func (m *SourceManager) RemoveTable(span tablepb.Span) { 190 m.puller.Unsubscribe([]tablepb.Span{span}) 191 m.engine.RemoveTable(span) 192 } 193 194 // OnResolve just wrap the engine's OnResolve method. 195 func (m *SourceManager) OnResolve(action func(tablepb.Span, model.Ts)) { 196 m.engine.OnResolve(action) 197 } 198 199 // FetchByTable just wrap the engine's FetchByTable method. 200 func (m *SourceManager) FetchByTable( 201 span tablepb.Span, lowerBound, upperBound sorter.Position, 202 quota *memquota.MemQuota, 203 ) *sorter.MountedEventIter { 204 iter := m.engine.FetchByTable(span, lowerBound, upperBound) 205 return sorter.NewMountedEventIter(m.changefeedID, iter, m.mg, defaultMaxBatchSize, quota) 206 } 207 208 // CleanByTable just wrap the engine's CleanByTable method. 209 func (m *SourceManager) CleanByTable(span tablepb.Span, upperBound sorter.Position) error { 210 return m.engine.CleanByTable(span, upperBound) 211 } 212 213 // GetTablePullerStats returns the puller stats of the table. 214 func (m *SourceManager) GetTablePullerStats(span tablepb.Span) puller.Stats { 215 return m.puller.Stats(span) 216 } 217 218 // GetTableSorterStats returns the sorter stats of the table. 219 func (m *SourceManager) GetTableSorterStats(span tablepb.Span) sorter.TableStats { 220 return m.engine.GetStatsByTable(span) 221 } 222 223 // Run implements util.Runnable. 224 func (m *SourceManager) Run(ctx context.Context, _ ...chan<- error) error { 225 close(m.ready) 226 // Only nil in unit tests. 227 if m.puller == nil { 228 return nil 229 } 230 startTs, err := getCurrentTs(ctx, m.up.PDClient) 231 if err != nil { 232 return err 233 } 234 m.startTs = startTs 235 return m.puller.Run(ctx) 236 } 237 238 // WaitForReady implements util.Runnable. 239 func (m *SourceManager) WaitForReady(ctx context.Context) { 240 select { 241 case <-ctx.Done(): 242 case <-m.ready: 243 } 244 } 245 246 // Close closes the source manager. Stop all pullers and close the engine. 247 // It also implements util.Runnable. 248 func (m *SourceManager) Close() { 249 log.Info("Closing source manager", 250 zap.String("namespace", m.changefeedID.Namespace), 251 zap.String("changefeed", m.changefeedID.ID)) 252 253 start := time.Now() 254 255 log.Info("All pullers have been closed", 256 zap.String("namespace", m.changefeedID.Namespace), 257 zap.String("changefeed", m.changefeedID.ID), 258 zap.Duration("cost", time.Since(start))) 259 260 if err := m.engine.Close(); err != nil { 261 log.Panic("Fail to close sort engine", 262 zap.String("namespace", m.changefeedID.Namespace), 263 zap.String("changefeed", m.changefeedID.ID), 264 zap.Error(err)) 265 } 266 log.Info("Closed source manager", 267 zap.String("namespace", m.changefeedID.Namespace), 268 zap.String("changefeed", m.changefeedID.ID), 269 zap.Duration("cost", time.Since(start))) 270 } 271 272 // Add adds events to the engine. It is used for testing. 273 func (m *SourceManager) Add(span tablepb.Span, events ...*model.PolymorphicEvent) { 274 m.engine.Add(span, events...) 275 } 276 277 func getCurrentTs(ctx context.Context, pdClient pd.Client) (model.Ts, error) { 278 backoffBaseDelayInMs := int64(100) 279 totalRetryDuration := 10 * time.Second 280 var replicateTs model.Ts 281 err := retry.Do(ctx, func() error { 282 phy, logic, err := pdClient.GetTS(ctx) 283 if err != nil { 284 return errors.Trace(err) 285 } 286 replicateTs = oracle.ComposeTS(phy, logic) 287 return nil 288 }, retry.WithBackoffBaseDelay(backoffBaseDelayInMs), 289 retry.WithTotalRetryDuratoin(totalRetryDuration), 290 retry.WithIsRetryableErr(cerrors.IsRetryableError)) 291 if err != nil { 292 return model.Ts(0), errors.Trace(err) 293 } 294 return replicateTs, nil 295 }