github.com/pingcap/br@v5.3.0-alpha.0.20220125034240-ec59c7b6ce30+incompatible/pkg/lightning/backend/backend.go (about) 1 // Copyright 2019 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 backend 15 16 import ( 17 "context" 18 "fmt" 19 "sort" 20 "time" 21 22 "github.com/google/uuid" 23 "github.com/pingcap/errors" 24 "github.com/pingcap/failpoint" 25 "github.com/pingcap/parser/model" 26 "github.com/pingcap/tidb/table" 27 "go.uber.org/zap" 28 29 "github.com/pingcap/br/pkg/lightning/backend/kv" 30 "github.com/pingcap/br/pkg/lightning/checkpoints" 31 "github.com/pingcap/br/pkg/lightning/common" 32 "github.com/pingcap/br/pkg/lightning/log" 33 "github.com/pingcap/br/pkg/lightning/metric" 34 "github.com/pingcap/br/pkg/lightning/mydump" 35 ) 36 37 const ( 38 importMaxRetryTimes = 3 // tikv-importer has done retry internally. so we don't retry many times. 39 ) 40 41 /* 42 43 Usual workflow: 44 45 1. Create a `Backend` for the whole process. 46 47 2. For each table, 48 49 i. Split into multiple "batches" consisting of data files with roughly equal total size. 50 51 ii. For each batch, 52 53 a. Create an `OpenedEngine` via `backend.OpenEngine()` 54 55 b. For each chunk, deliver data into the engine via `engine.WriteRows()` 56 57 c. When all chunks are written, obtain a `ClosedEngine` via `engine.Close()` 58 59 d. Import data via `engine.Import()` 60 61 e. Cleanup via `engine.Cleanup()` 62 63 3. Close the connection via `backend.Close()` 64 65 */ 66 67 func makeTag(tableName string, engineID int32) string { 68 return fmt.Sprintf("%s:%d", tableName, engineID) 69 } 70 71 func makeLogger(tag string, engineUUID uuid.UUID) log.Logger { 72 return log.With( 73 zap.String("engineTag", tag), 74 zap.Stringer("engineUUID", engineUUID), 75 ) 76 } 77 78 func MakeUUID(tableName string, engineID int32) (string, uuid.UUID) { 79 tag := makeTag(tableName, engineID) 80 engineUUID := uuid.NewSHA1(engineNamespace, []byte(tag)) 81 return tag, engineUUID 82 } 83 84 var engineNamespace = uuid.MustParse("d68d6abe-c59e-45d6-ade8-e2b0ceb7bedf") 85 86 type EngineFileSize struct { 87 // UUID is the engine's UUID. 88 UUID uuid.UUID 89 // DiskSize is the estimated total file size on disk right now. 90 DiskSize int64 91 // MemSize is the total memory size used by the engine. This is the 92 // estimated additional size saved onto disk after calling Flush(). 93 MemSize int64 94 // IsImporting indicates whether the engine performing Import(). 95 IsImporting bool 96 } 97 98 // LocalWriterConfig defines the configuration to open a LocalWriter 99 type LocalWriterConfig struct { 100 // is the chunk KV written to this LocalWriter sent in order 101 IsKVSorted bool 102 } 103 104 // EngineConfig defines configuration used for open engine 105 type EngineConfig struct { 106 // TableInfo is the corresponding tidb table info 107 TableInfo *checkpoints.TidbTableInfo 108 // local backend specified configuration 109 Local *LocalEngineConfig 110 } 111 112 // LocalEngineConfig is the configuration used for local backend in OpenEngine. 113 type LocalEngineConfig struct { 114 // compact small SSTs before ingest into pebble 115 Compact bool 116 // raw kvs size threshold to trigger compact 117 CompactThreshold int64 118 // compact routine concurrency 119 CompactConcurrency int 120 } 121 122 // CheckCtx contains all parameters used in CheckRequirements 123 type CheckCtx struct { 124 DBMetas []*mydump.MDDatabaseMeta 125 } 126 127 // AbstractBackend is the abstract interface behind Backend. 128 // Implementations of this interface must be goroutine safe: you can share an 129 // instance and execute any method anywhere. 130 type AbstractBackend interface { 131 // Close the connection to the backend. 132 Close() 133 134 // MakeEmptyRows creates an empty collection of encoded rows. 135 MakeEmptyRows() kv.Rows 136 137 // RetryImportDelay returns the duration to sleep when retrying an import 138 RetryImportDelay() time.Duration 139 140 // ShouldPostProcess returns whether KV-specific post-processing should be 141 // performed for this backend. Post-processing includes checksum and analyze. 142 ShouldPostProcess() bool 143 144 // NewEncoder creates an encoder of a TiDB table. 145 NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) 146 147 OpenEngine(ctx context.Context, config *EngineConfig, engineUUID uuid.UUID) error 148 149 CloseEngine(ctx context.Context, config *EngineConfig, engineUUID uuid.UUID) error 150 151 // ImportEngine imports engine data to the backend. If it returns ErrDuplicateDetected, 152 // it means there is duplicate detected. For this situation, all data in the engine must be imported. 153 // It's safe to reset or cleanup this engine. 154 ImportEngine(ctx context.Context, engineUUID uuid.UUID) error 155 156 CleanupEngine(ctx context.Context, engineUUID uuid.UUID) error 157 158 // CheckRequirements performs the check whether the backend satisfies the 159 // version requirements 160 CheckRequirements(ctx context.Context, checkCtx *CheckCtx) error 161 162 // FetchRemoteTableModels obtains the models of all tables given the schema 163 // name. The returned table info does not need to be precise if the encoder, 164 // is not requiring them, but must at least fill in the following fields for 165 // TablesFromMeta to succeed: 166 // - Name 167 // - State (must be model.StatePublic) 168 // - ID 169 // - Columns 170 // * Name 171 // * State (must be model.StatePublic) 172 // * Offset (must be 0, 1, 2, ...) 173 // - PKIsHandle (true = do not generate _tidb_rowid) 174 FetchRemoteTableModels(ctx context.Context, schemaName string) ([]*model.TableInfo, error) 175 176 // FlushEngine ensures all KV pairs written to an open engine has been 177 // synchronized, such that kill-9'ing Lightning afterwards and resuming from 178 // checkpoint can recover the exact same content. 179 // 180 // This method is only relevant for local backend, and is no-op for all 181 // other backends. 182 FlushEngine(ctx context.Context, engineUUID uuid.UUID) error 183 184 // FlushAllEngines performs FlushEngine on all opened engines. This is a 185 // very expensive operation and should only be used in some rare situation 186 // (e.g. preparing to resolve a disk quota violation). 187 FlushAllEngines(ctx context.Context) error 188 189 // EngineFileSizes obtains the size occupied locally of all engines managed 190 // by this backend. This method is used to compute disk quota. 191 // It can return nil if the content are all stored remotely. 192 EngineFileSizes() []EngineFileSize 193 194 // ResetEngine clears all written KV pairs in this opened engine. 195 ResetEngine(ctx context.Context, engineUUID uuid.UUID) error 196 197 // LocalWriter obtains a thread-local EngineWriter for writing rows into the given engine. 198 LocalWriter(ctx context.Context, cfg *LocalWriterConfig, engineUUID uuid.UUID) (EngineWriter, error) 199 200 // CollectLocalDuplicateRows collect duplicate keys from local db. We will store the duplicate keys which 201 // may be repeated with other keys in local data source. 202 CollectLocalDuplicateRows(ctx context.Context, tbl table.Table) error 203 204 // CollectLocalDuplicateRows collect duplicate keys from remote TiKV storage. This keys may be duplicate with 205 // the data import by other lightning. 206 CollectRemoteDuplicateRows(ctx context.Context, tbl table.Table) error 207 } 208 209 // Backend is the delivery target for Lightning 210 type Backend struct { 211 abstract AbstractBackend 212 } 213 214 type engine struct { 215 backend AbstractBackend 216 logger log.Logger 217 uuid uuid.UUID 218 } 219 220 // OpenedEngine is an opened engine, allowing data to be written via WriteRows. 221 // This type is goroutine safe: you can share an instance and execute any method 222 // anywhere. 223 type OpenedEngine struct { 224 engine 225 tableName string 226 } 227 228 // // import_ the data written to the engine into the target. 229 // import_(ctx context.Context) error 230 231 // // cleanup deletes the imported data. 232 // cleanup(ctx context.Context) error 233 234 // ClosedEngine represents a closed engine, allowing ingestion into the target. 235 // This type is goroutine safe: you can share an instance and execute any method 236 // anywhere. 237 type ClosedEngine struct { 238 engine 239 } 240 241 type LocalEngineWriter struct { 242 writer EngineWriter 243 tableName string 244 } 245 246 func MakeBackend(ab AbstractBackend) Backend { 247 return Backend{abstract: ab} 248 } 249 250 func (be Backend) Close() { 251 be.abstract.Close() 252 } 253 254 func (be Backend) MakeEmptyRows() kv.Rows { 255 return be.abstract.MakeEmptyRows() 256 } 257 258 func (be Backend) NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { 259 return be.abstract.NewEncoder(tbl, options) 260 } 261 262 func (be Backend) ShouldPostProcess() bool { 263 return be.abstract.ShouldPostProcess() 264 } 265 266 func (be Backend) CheckRequirements(ctx context.Context, checkCtx *CheckCtx) error { 267 return be.abstract.CheckRequirements(ctx, checkCtx) 268 } 269 270 func (be Backend) FetchRemoteTableModels(ctx context.Context, schemaName string) ([]*model.TableInfo, error) { 271 return be.abstract.FetchRemoteTableModels(ctx, schemaName) 272 } 273 274 func (be Backend) FlushAll(ctx context.Context) error { 275 return be.abstract.FlushAllEngines(ctx) 276 } 277 278 // CheckDiskQuota verifies if the total engine file size is below the given 279 // quota. If the quota is exceeded, this method returns an array of engines, 280 // which after importing can decrease the total size below quota. 281 func (be Backend) CheckDiskQuota(quota int64) ( 282 largeEngines []uuid.UUID, 283 inProgressLargeEngines int, 284 totalDiskSize int64, 285 totalMemSize int64, 286 ) { 287 sizes := be.abstract.EngineFileSizes() 288 sort.Slice(sizes, func(i, j int) bool { 289 a, b := &sizes[i], &sizes[j] 290 if a.IsImporting != b.IsImporting { 291 return a.IsImporting 292 } 293 return a.DiskSize+a.MemSize < b.DiskSize+b.MemSize 294 }) 295 for _, size := range sizes { 296 totalDiskSize += size.DiskSize 297 totalMemSize += size.MemSize 298 if totalDiskSize+totalMemSize > quota { 299 if size.IsImporting { 300 inProgressLargeEngines++ 301 } else { 302 largeEngines = append(largeEngines, size.UUID) 303 } 304 } 305 } 306 return 307 } 308 309 // UnsafeImportAndReset forces the backend to import the content of an engine 310 // into the target and then reset the engine to empty. This method will not 311 // close the engine. Make sure the engine is flushed manually before calling 312 // this method. 313 func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID) error { 314 // DO NOT call be.abstract.CloseEngine()! The engine should still be writable after 315 // calling UnsafeImportAndReset(). 316 closedEngine := ClosedEngine{ 317 engine: engine{ 318 backend: be.abstract, 319 logger: makeLogger("<import-and-reset>", engineUUID), 320 uuid: engineUUID, 321 }, 322 } 323 if err := closedEngine.Import(ctx); err != nil { 324 return err 325 } 326 return be.abstract.ResetEngine(ctx, engineUUID) 327 } 328 329 // OpenEngine opens an engine with the given table name and engine ID. 330 func (be Backend) OpenEngine(ctx context.Context, config *EngineConfig, tableName string, engineID int32) (*OpenedEngine, error) { 331 tag, engineUUID := MakeUUID(tableName, engineID) 332 logger := makeLogger(tag, engineUUID) 333 334 if err := be.abstract.OpenEngine(ctx, config, engineUUID); err != nil { 335 return nil, err 336 } 337 338 openCounter := metric.ImporterEngineCounter.WithLabelValues("open") 339 openCounter.Inc() 340 341 logger.Info("open engine") 342 343 failpoint.Inject("FailIfEngineCountExceeds", func(val failpoint.Value) { 344 closedCounter := metric.ImporterEngineCounter.WithLabelValues("closed") 345 openCount := metric.ReadCounter(openCounter) 346 closedCount := metric.ReadCounter(closedCounter) 347 if injectValue := val.(int); openCount-closedCount > float64(injectValue) { 348 panic(fmt.Sprintf("forcing failure due to FailIfEngineCountExceeds: %v - %v >= %d", openCount, closedCount, injectValue)) 349 } 350 }) 351 352 return &OpenedEngine{ 353 engine: engine{ 354 backend: be.abstract, 355 logger: logger, 356 uuid: engineUUID, 357 }, 358 tableName: tableName, 359 }, nil 360 } 361 362 func (be Backend) CollectLocalDuplicateRows(ctx context.Context, tbl table.Table) error { 363 return be.abstract.CollectLocalDuplicateRows(ctx, tbl) 364 } 365 366 func (be Backend) CollectRemoteDuplicateRows(ctx context.Context, tbl table.Table) error { 367 return be.abstract.CollectRemoteDuplicateRows(ctx, tbl) 368 } 369 370 // Close the opened engine to prepare it for importing. 371 func (engine *OpenedEngine) Close(ctx context.Context, cfg *EngineConfig) (*ClosedEngine, error) { 372 closedEngine, err := engine.unsafeClose(ctx, cfg) 373 if err == nil { 374 metric.ImporterEngineCounter.WithLabelValues("closed").Inc() 375 } 376 return closedEngine, err 377 } 378 379 // Flush current written data for local backend 380 func (engine *OpenedEngine) Flush(ctx context.Context) error { 381 return engine.backend.FlushEngine(ctx, engine.uuid) 382 } 383 384 func (engine *OpenedEngine) LocalWriter(ctx context.Context, cfg *LocalWriterConfig) (*LocalEngineWriter, error) { 385 w, err := engine.backend.LocalWriter(ctx, cfg, engine.uuid) 386 if err != nil { 387 return nil, err 388 } 389 return &LocalEngineWriter{writer: w, tableName: engine.tableName}, nil 390 } 391 392 // WriteRows writes a collection of encoded rows into the engine. 393 func (w *LocalEngineWriter) WriteRows(ctx context.Context, columnNames []string, rows kv.Rows) error { 394 return w.writer.AppendRows(ctx, w.tableName, columnNames, rows) 395 } 396 397 func (w *LocalEngineWriter) Close(ctx context.Context) (ChunkFlushStatus, error) { 398 return w.writer.Close(ctx) 399 } 400 401 func (w *LocalEngineWriter) IsSynced() bool { 402 return w.writer.IsSynced() 403 } 404 405 // UnsafeCloseEngine closes the engine without first opening it. 406 // This method is "unsafe" as it does not follow the normal operation sequence 407 // (Open -> Write -> Close -> Import). This method should only be used when one 408 // knows via other ways that the engine has already been opened, e.g. when 409 // resuming from a checkpoint. 410 func (be Backend) UnsafeCloseEngine(ctx context.Context, cfg *EngineConfig, tableName string, engineID int32) (*ClosedEngine, error) { 411 tag, engineUUID := MakeUUID(tableName, engineID) 412 return be.UnsafeCloseEngineWithUUID(ctx, cfg, tag, engineUUID) 413 } 414 415 // UnsafeCloseEngineWithUUID closes the engine without first opening it. 416 // This method is "unsafe" as it does not follow the normal operation sequence 417 // (Open -> Write -> Close -> Import). This method should only be used when one 418 // knows via other ways that the engine has already been opened, e.g. when 419 // resuming from a checkpoint. 420 func (be Backend) UnsafeCloseEngineWithUUID(ctx context.Context, cfg *EngineConfig, tag string, engineUUID uuid.UUID) (*ClosedEngine, error) { 421 return engine{ 422 backend: be.abstract, 423 logger: makeLogger(tag, engineUUID), 424 uuid: engineUUID, 425 }.unsafeClose(ctx, cfg) 426 } 427 428 func (en engine) unsafeClose(ctx context.Context, cfg *EngineConfig) (*ClosedEngine, error) { 429 task := en.logger.Begin(zap.InfoLevel, "engine close") 430 err := en.backend.CloseEngine(ctx, cfg, en.uuid) 431 task.End(zap.ErrorLevel, err) 432 if err != nil { 433 return nil, err 434 } 435 return &ClosedEngine{engine: en}, nil 436 } 437 438 // Import the data written to the engine into the target. 439 func (engine *ClosedEngine) Import(ctx context.Context) error { 440 var err error 441 442 for i := 0; i < importMaxRetryTimes; i++ { 443 task := engine.logger.With(zap.Int("retryCnt", i)).Begin(zap.InfoLevel, "import") 444 err = engine.backend.ImportEngine(ctx, engine.uuid) 445 if !common.IsRetryableError(err) { 446 task.End(zap.ErrorLevel, err) 447 return err 448 } 449 task.Warn("import spuriously failed, going to retry again", log.ShortError(err)) 450 time.Sleep(engine.backend.RetryImportDelay()) 451 } 452 453 return errors.Annotatef(err, "[%s] import reach max retry %d and still failed", engine.uuid, importMaxRetryTimes) 454 } 455 456 // Cleanup deletes the intermediate data from target. 457 func (engine *ClosedEngine) Cleanup(ctx context.Context) error { 458 task := engine.logger.Begin(zap.InfoLevel, "cleanup") 459 err := engine.backend.CleanupEngine(ctx, engine.uuid) 460 task.End(zap.WarnLevel, err) 461 return err 462 } 463 464 func (engine *ClosedEngine) Logger() log.Logger { 465 return engine.logger 466 } 467 468 type ChunkFlushStatus interface { 469 Flushed() bool 470 } 471 472 type EngineWriter interface { 473 AppendRows( 474 ctx context.Context, 475 tableName string, 476 columnNames []string, 477 rows kv.Rows, 478 ) error 479 IsSynced() bool 480 Close(ctx context.Context) (ChunkFlushStatus, error) 481 }