github.com/pingcap/br@v5.3.0-alpha.0.20220125034240-ec59c7b6ce30+incompatible/pkg/restore/pipeline_items.go (about) 1 // Copyright 2020 PingCAP, Inc. Licensed under Apache-2.0. 2 3 package restore 4 5 import ( 6 "context" 7 "sync" 8 9 "github.com/pingcap/br/pkg/metautil" 10 11 "github.com/pingcap/errors" 12 "github.com/pingcap/log" 13 "github.com/pingcap/parser/model" 14 "go.uber.org/zap" 15 16 "github.com/pingcap/br/pkg/glue" 17 "github.com/pingcap/br/pkg/rtree" 18 ) 19 20 const ( 21 defaultChannelSize = 1024 22 ) 23 24 // TableSink is the 'sink' of restored data by a sender. 25 type TableSink interface { 26 EmitTables(tables ...CreatedTable) 27 EmitError(error) 28 Close() 29 } 30 31 type chanTableSink struct { 32 outCh chan<- []CreatedTable 33 errCh chan<- error 34 } 35 36 func (sink chanTableSink) EmitTables(tables ...CreatedTable) { 37 sink.outCh <- tables 38 } 39 40 func (sink chanTableSink) EmitError(err error) { 41 sink.errCh <- err 42 } 43 44 func (sink chanTableSink) Close() { 45 // ErrCh may has multi sender part, don't close it. 46 close(sink.outCh) 47 } 48 49 // ContextManager is the struct to manage a TiKV 'context' for restore. 50 // Batcher will call Enter when any table should be restore on batch, 51 // so you can do some prepare work here(e.g. set placement rules for online restore). 52 type ContextManager interface { 53 // Enter make some tables 'enter' this context(a.k.a., prepare for restore). 54 Enter(ctx context.Context, tables []CreatedTable) error 55 // Leave make some tables 'leave' this context(a.k.a., restore is done, do some post-works). 56 Leave(ctx context.Context, tables []CreatedTable) error 57 // Close closes the context manager, sometimes when the manager is 'killed' and should do some cleanup 58 // it would be call. 59 Close(ctx context.Context) 60 } 61 62 // NewBRContextManager makes a BR context manager, that is, 63 // set placement rules for online restore when enter(see <splitPrepareWork>), 64 // unset them when leave. 65 func NewBRContextManager(client *Client) ContextManager { 66 return &brContextManager{ 67 client: client, 68 69 hasTable: make(map[int64]CreatedTable), 70 } 71 } 72 73 type brContextManager struct { 74 client *Client 75 76 // This 'set' of table ID allow us to handle each table just once. 77 hasTable map[int64]CreatedTable 78 } 79 80 func (manager *brContextManager) Close(ctx context.Context) { 81 tbls := make([]*model.TableInfo, 0, len(manager.hasTable)) 82 for _, tbl := range manager.hasTable { 83 tbls = append(tbls, tbl.Table) 84 } 85 splitPostWork(ctx, manager.client, tbls) 86 } 87 88 func (manager *brContextManager) Enter(ctx context.Context, tables []CreatedTable) error { 89 placementRuleTables := make([]*model.TableInfo, 0, len(tables)) 90 91 for _, tbl := range tables { 92 if _, ok := manager.hasTable[tbl.Table.ID]; !ok { 93 placementRuleTables = append(placementRuleTables, tbl.Table) 94 } 95 manager.hasTable[tbl.Table.ID] = tbl 96 } 97 98 return splitPrepareWork(ctx, manager.client, placementRuleTables) 99 } 100 101 func (manager *brContextManager) Leave(ctx context.Context, tables []CreatedTable) error { 102 placementRuleTables := make([]*model.TableInfo, 0, len(tables)) 103 104 for _, table := range tables { 105 placementRuleTables = append(placementRuleTables, table.Table) 106 } 107 108 splitPostWork(ctx, manager.client, placementRuleTables) 109 log.Info("restore table done", ZapTables(tables)) 110 for _, tbl := range placementRuleTables { 111 delete(manager.hasTable, tbl.ID) 112 } 113 return nil 114 } 115 116 func splitPostWork(ctx context.Context, client *Client, tables []*model.TableInfo) { 117 err := client.ResetPlacementRules(ctx, tables) 118 if err != nil { 119 log.Warn("reset placement rules failed", zap.Error(err)) 120 return 121 } 122 } 123 124 func splitPrepareWork(ctx context.Context, client *Client, tables []*model.TableInfo) error { 125 err := client.SetupPlacementRules(ctx, tables) 126 if err != nil { 127 log.Error("setup placement rules failed", zap.Error(err)) 128 return errors.Trace(err) 129 } 130 131 err = client.WaitPlacementSchedule(ctx, tables) 132 if err != nil { 133 log.Error("wait placement schedule failed", zap.Error(err)) 134 return errors.Trace(err) 135 } 136 return nil 137 } 138 139 // CreatedTable is a table created on restore process, 140 // but not yet filled with data. 141 type CreatedTable struct { 142 RewriteRule *RewriteRules 143 Table *model.TableInfo 144 OldTable *metautil.Table 145 } 146 147 // TableWithRange is a CreatedTable that has been bind to some of key ranges. 148 type TableWithRange struct { 149 CreatedTable 150 151 Range []rtree.Range 152 } 153 154 // Exhaust drains all remaining errors in the channel, into a slice of errors. 155 func Exhaust(ec <-chan error) []error { 156 out := make([]error, 0, len(ec)) 157 for { 158 select { 159 case err := <-ec: 160 out = append(out, err) 161 default: 162 // errCh will NEVER be closed(ya see, it has multi sender-part), 163 // so we just consume the current backlog of this channel, then return. 164 return out 165 } 166 } 167 } 168 169 // BatchSender is the abstract of how the batcher send a batch. 170 type BatchSender interface { 171 // PutSink sets the sink of this sender, user to this interface promise 172 // call this function at least once before first call to `RestoreBatch`. 173 PutSink(sink TableSink) 174 // RestoreBatch will send the restore request. 175 RestoreBatch(ranges DrainResult) 176 Close() 177 } 178 179 type tikvSender struct { 180 client *Client 181 updateCh glue.Progress 182 183 sink TableSink 184 inCh chan<- DrainResult 185 186 wg *sync.WaitGroup 187 } 188 189 func (b *tikvSender) PutSink(sink TableSink) { 190 // don't worry about visibility, since we will call this before first call to 191 // RestoreBatch, which is a sync point. 192 b.sink = sink 193 } 194 195 func (b *tikvSender) RestoreBatch(ranges DrainResult) { 196 b.inCh <- ranges 197 } 198 199 // NewTiKVSender make a sender that send restore requests to TiKV. 200 func NewTiKVSender( 201 ctx context.Context, 202 cli *Client, 203 updateCh glue.Progress, 204 ) (BatchSender, error) { 205 inCh := make(chan DrainResult, defaultChannelSize) 206 midCh := make(chan DrainResult, defaultChannelSize) 207 208 sender := &tikvSender{ 209 client: cli, 210 updateCh: updateCh, 211 inCh: inCh, 212 wg: new(sync.WaitGroup), 213 } 214 215 sender.wg.Add(2) 216 go sender.splitWorker(ctx, inCh, midCh) 217 go sender.restoreWorker(ctx, midCh) 218 return sender, nil 219 } 220 221 func (b *tikvSender) splitWorker(ctx context.Context, ranges <-chan DrainResult, next chan<- DrainResult) { 222 defer log.Debug("split worker closed") 223 defer func() { 224 b.wg.Done() 225 close(next) 226 }() 227 for { 228 select { 229 case <-ctx.Done(): 230 return 231 case result, ok := <-ranges: 232 if !ok { 233 return 234 } 235 if err := SplitRanges(ctx, b.client, result.Ranges, result.RewriteRules, b.updateCh); err != nil { 236 log.Error("failed on split range", rtree.ZapRanges(result.Ranges), zap.Error(err)) 237 b.sink.EmitError(err) 238 return 239 } 240 next <- result 241 } 242 } 243 } 244 245 func (b *tikvSender) restoreWorker(ctx context.Context, ranges <-chan DrainResult) { 246 defer func() { 247 log.Debug("restore worker closed") 248 b.wg.Done() 249 b.sink.Close() 250 }() 251 for { 252 select { 253 case <-ctx.Done(): 254 return 255 case result, ok := <-ranges: 256 if !ok { 257 return 258 } 259 files := result.Files() 260 if err := b.client.RestoreFiles(ctx, files, result.RewriteRules, b.updateCh); err != nil { 261 b.sink.EmitError(err) 262 return 263 } 264 265 log.Info("restore batch done", rtree.ZapRanges(result.Ranges)) 266 b.sink.EmitTables(result.BlankTablesAfterSend...) 267 } 268 } 269 } 270 271 func (b *tikvSender) Close() { 272 close(b.inCh) 273 b.wg.Wait() 274 log.Debug("tikv sender closed") 275 }