github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/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/store/tikv/oracle" 27 "github.com/pingcap/tidb/table" 28 "github.com/pingcap/tidb/types" 29 "go.uber.org/zap" 30 31 "github.com/pingcap/tidb-lightning/lightning/common" 32 "github.com/pingcap/tidb-lightning/lightning/log" 33 "github.com/pingcap/tidb-lightning/lightning/metric" 34 "github.com/pingcap/tidb-lightning/lightning/verification" 35 ) 36 37 const ( 38 maxRetryTimes = 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 // AbstractBackend is the abstract interface behind Backend. 99 // Implementations of this interface must be goroutine safe: you can share an 100 // instance and execute any method anywhere. 101 type AbstractBackend interface { 102 // Close the connection to the backend. 103 Close() 104 105 // MakeEmptyRows creates an empty collection of encoded rows. 106 MakeEmptyRows() Rows 107 108 // RetryImportDelay returns the duration to sleep when retrying an import 109 RetryImportDelay() time.Duration 110 111 // ShouldPostProcess returns whether KV-specific post-processing should be 112 // performed for this backend. Post-processing includes checksum and analyze. 113 ShouldPostProcess() bool 114 115 // NewEncoder creates an encoder of a TiDB table. 116 NewEncoder(tbl table.Table, options *SessionOptions) (Encoder, error) 117 118 OpenEngine(ctx context.Context, engineUUID uuid.UUID) error 119 120 CloseEngine(ctx context.Context, engineUUID uuid.UUID) error 121 122 ImportEngine(ctx context.Context, engineUUID uuid.UUID) error 123 124 CleanupEngine(ctx context.Context, engineUUID uuid.UUID) error 125 126 // CheckRequirements performs the check whether the backend satisfies the 127 // version requirements 128 CheckRequirements(ctx context.Context) error 129 130 // FetchRemoteTableModels obtains the models of all tables given the schema 131 // name. The returned table info does not need to be precise if the encoder, 132 // is not requiring them, but must at least fill in the following fields for 133 // TablesFromMeta to succeed: 134 // - Name 135 // - State (must be model.StatePublic) 136 // - ID 137 // - Columns 138 // * Name 139 // * State (must be model.StatePublic) 140 // * Offset (must be 0, 1, 2, ...) 141 // - PKIsHandle (true = do not generate _tidb_rowid) 142 FetchRemoteTableModels(ctx context.Context, schemaName string) ([]*model.TableInfo, error) 143 144 // FlushEngine ensures all KV pairs written to an open engine has been 145 // synchronized, such that kill-9'ing Lightning afterwards and resuming from 146 // checkpoint can recover the exact same content. 147 // 148 // This method is only relevant for local backend, and is no-op for all 149 // other backends. 150 FlushEngine(ctx context.Context, engineUUID uuid.UUID) error 151 152 // FlushAllEngines performs FlushEngine on all opened engines. This is a 153 // very expensive operation and should only be used in some rare situation 154 // (e.g. preparing to resolve a disk quota violation). 155 FlushAllEngines(ctx context.Context) error 156 157 // EngineFileSizes obtains the size occupied locally of all engines managed 158 // by this backend. This method is used to compute disk quota. 159 // It can return nil if the content are all stored remotely. 160 EngineFileSizes() []EngineFileSize 161 162 // ResetEngine clears all written KV pairs in this opened engine. 163 ResetEngine(ctx context.Context, engineUUID uuid.UUID) error 164 165 // LocalWriter obtains a thread-local EngineWriter for writing rows into the given engine. 166 LocalWriter(ctx context.Context, engineUUID uuid.UUID, maxCacheSize int64) (EngineWriter, error) 167 } 168 169 func fetchRemoteTableModelsFromTLS(ctx context.Context, tls *common.TLS, schema string) ([]*model.TableInfo, error) { 170 var tables []*model.TableInfo 171 err := tls.GetJSON(ctx, "/schema/"+schema, &tables) 172 if err != nil { 173 return nil, errors.Annotatef(err, "cannot read schema '%s' from remote", schema) 174 } 175 return tables, nil 176 } 177 178 // Backend is the delivery target for Lightning 179 type Backend struct { 180 abstract AbstractBackend 181 } 182 183 type engine struct { 184 backend AbstractBackend 185 logger log.Logger 186 uuid uuid.UUID 187 } 188 189 // OpenedEngine is an opened engine, allowing data to be written via WriteRows. 190 // This type is goroutine safe: you can share an instance and execute any method 191 // anywhere. 192 type OpenedEngine struct { 193 engine 194 tableName string 195 ts uint64 196 } 197 198 // // import_ the data written to the engine into the target. 199 // import_(ctx context.Context) error 200 201 // // cleanup deletes the imported data. 202 // cleanup(ctx context.Context) error 203 204 // ClosedEngine represents a closed engine, allowing ingestion into the target. 205 // This type is goroutine safe: you can share an instance and execute any method 206 // anywhere. 207 type ClosedEngine struct { 208 engine 209 } 210 211 type LocalEngineWriter struct { 212 writer EngineWriter 213 tableName string 214 ts uint64 215 } 216 217 func MakeBackend(ab AbstractBackend) Backend { 218 return Backend{abstract: ab} 219 } 220 221 func (be Backend) Close() { 222 be.abstract.Close() 223 } 224 225 func (be Backend) MakeEmptyRows() Rows { 226 return be.abstract.MakeEmptyRows() 227 } 228 229 func (be Backend) NewEncoder(tbl table.Table, options *SessionOptions) (Encoder, error) { 230 return be.abstract.NewEncoder(tbl, options) 231 } 232 233 func (be Backend) ShouldPostProcess() bool { 234 return be.abstract.ShouldPostProcess() 235 } 236 237 func (be Backend) CheckRequirements(ctx context.Context) error { 238 return be.abstract.CheckRequirements(ctx) 239 } 240 241 func (be Backend) FetchRemoteTableModels(ctx context.Context, schemaName string) ([]*model.TableInfo, error) { 242 return be.abstract.FetchRemoteTableModels(ctx, schemaName) 243 } 244 245 func (be Backend) FlushAll(ctx context.Context) error { 246 return be.abstract.FlushAllEngines(ctx) 247 } 248 249 // CheckDiskQuota verifies if the total engine file size is below the given 250 // quota. If the quota is exceeded, this method returns an array of engines, 251 // which after importing can decrease the total size below quota. 252 func (be Backend) CheckDiskQuota(quota int64) ( 253 largeEngines []uuid.UUID, 254 inProgressLargeEngines int, 255 totalDiskSize int64, 256 totalMemSize int64, 257 ) { 258 sizes := be.abstract.EngineFileSizes() 259 sort.Slice(sizes, func(i, j int) bool { 260 a, b := &sizes[i], &sizes[j] 261 if a.IsImporting != b.IsImporting { 262 return a.IsImporting 263 } 264 return a.DiskSize+a.MemSize < b.DiskSize+b.MemSize 265 }) 266 for _, size := range sizes { 267 totalDiskSize += size.DiskSize 268 totalMemSize += size.MemSize 269 if totalDiskSize+totalMemSize > quota { 270 if size.IsImporting { 271 inProgressLargeEngines++ 272 } else { 273 largeEngines = append(largeEngines, size.UUID) 274 } 275 } 276 } 277 return 278 } 279 280 // UnsafeImportAndReset forces the backend to import the content of an engine 281 // into the target and then reset the engine to empty. This method will not 282 // close the engine. Make sure the engine is flushed manually before calling 283 // this method. 284 func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID) error { 285 // DO NOT call be.abstract.CloseEngine()! The engine should still be writable after 286 // calling UnsafeImportAndReset(). 287 closedEngine := ClosedEngine{ 288 engine: engine{ 289 backend: be.abstract, 290 logger: makeLogger("<import-and-reset>", engineUUID), 291 uuid: engineUUID, 292 }, 293 } 294 if err := closedEngine.Import(ctx); err != nil { 295 return err 296 } 297 return be.abstract.ResetEngine(ctx, engineUUID) 298 } 299 300 // OpenEngine opens an engine with the given table name and engine ID. 301 func (be Backend) OpenEngine(ctx context.Context, tableName string, engineID int32) (*OpenedEngine, error) { 302 tag, engineUUID := MakeUUID(tableName, engineID) 303 logger := makeLogger(tag, engineUUID) 304 305 if err := be.abstract.OpenEngine(ctx, engineUUID); err != nil { 306 return nil, err 307 } 308 309 openCounter := metric.ImporterEngineCounter.WithLabelValues("open") 310 openCounter.Inc() 311 312 logger.Info("open engine") 313 314 failpoint.Inject("FailIfEngineCountExceeds", func(val failpoint.Value) { 315 closedCounter := metric.ImporterEngineCounter.WithLabelValues("closed") 316 openCount := metric.ReadCounter(openCounter) 317 closedCount := metric.ReadCounter(closedCounter) 318 if injectValue := val.(int); openCount-closedCount > float64(injectValue) { 319 panic(fmt.Sprintf("forcing failure due to FailIfEngineCountExceeds: %v - %v >= %d", openCount, closedCount, injectValue)) 320 } 321 }) 322 323 return &OpenedEngine{ 324 engine: engine{ 325 backend: be.abstract, 326 logger: logger, 327 uuid: engineUUID, 328 }, 329 tableName: tableName, 330 ts: oracle.ComposeTS(time.Now().Unix()*1000, 0), 331 }, nil 332 } 333 334 // Close the opened engine to prepare it for importing. 335 func (engine *OpenedEngine) Close(ctx context.Context) (*ClosedEngine, error) { 336 closedEngine, err := engine.unsafeClose(ctx) 337 if err == nil { 338 metric.ImporterEngineCounter.WithLabelValues("closed").Inc() 339 } 340 return closedEngine, err 341 } 342 343 // Flush current written data for local backend 344 func (engine *OpenedEngine) Flush(ctx context.Context) error { 345 return engine.backend.FlushEngine(ctx, engine.uuid) 346 } 347 348 // WriteRows writes a collection of encoded rows into the engine. 349 func (engine *OpenedEngine) WriteRows(ctx context.Context, columnNames []string, rows Rows) error { 350 writer, err := engine.backend.LocalWriter(ctx, engine.uuid, LocalMemoryTableSize) 351 if err != nil { 352 return err 353 } 354 if err = writer.AppendRows(ctx, engine.tableName, columnNames, engine.ts, rows); err != nil { 355 writer.Close() 356 return err 357 } 358 return writer.Close() 359 } 360 361 func (engine *OpenedEngine) LocalWriter(ctx context.Context, maxCacheSize int64) (*LocalEngineWriter, error) { 362 w, err := engine.backend.LocalWriter(ctx, engine.uuid, maxCacheSize) 363 if err != nil { 364 return nil, err 365 } 366 return &LocalEngineWriter{writer: w, ts: engine.ts, tableName: engine.tableName}, nil 367 } 368 369 // WriteRows writes a collection of encoded rows into the engine. 370 func (w *LocalEngineWriter) WriteRows(ctx context.Context, columnNames []string, rows Rows) error { 371 return w.writer.AppendRows(ctx, w.tableName, columnNames, w.ts, rows) 372 } 373 374 func (w *LocalEngineWriter) Close() error { 375 return w.writer.Close() 376 } 377 378 // UnsafeCloseEngine closes the engine without first opening it. 379 // This method is "unsafe" as it does not follow the normal operation sequence 380 // (Open -> Write -> Close -> Import). This method should only be used when one 381 // knows via other ways that the engine has already been opened, e.g. when 382 // resuming from a checkpoint. 383 func (be Backend) UnsafeCloseEngine(ctx context.Context, tableName string, engineID int32) (*ClosedEngine, error) { 384 tag, engineUUID := MakeUUID(tableName, engineID) 385 return be.UnsafeCloseEngineWithUUID(ctx, tag, engineUUID) 386 } 387 388 // UnsafeCloseEngineWithUUID closes the engine without first opening it. 389 // This method is "unsafe" as it does not follow the normal operation sequence 390 // (Open -> Write -> Close -> Import). This method should only be used when one 391 // knows via other ways that the engine has already been opened, e.g. when 392 // resuming from a checkpoint. 393 func (be Backend) UnsafeCloseEngineWithUUID(ctx context.Context, tag string, engineUUID uuid.UUID) (*ClosedEngine, error) { 394 return engine{ 395 backend: be.abstract, 396 logger: makeLogger(tag, engineUUID), 397 uuid: engineUUID, 398 }.unsafeClose(ctx) 399 } 400 401 func (en engine) unsafeClose(ctx context.Context) (*ClosedEngine, error) { 402 task := en.logger.Begin(zap.InfoLevel, "engine close") 403 err := en.backend.CloseEngine(ctx, en.uuid) 404 task.End(zap.ErrorLevel, err) 405 if err != nil { 406 return nil, err 407 } 408 return &ClosedEngine{engine: en}, nil 409 } 410 411 // Import the data written to the engine into the target. 412 func (engine *ClosedEngine) Import(ctx context.Context) error { 413 var err error 414 415 for i := 0; i < maxRetryTimes; i++ { 416 task := engine.logger.With(zap.Int("retryCnt", i)).Begin(zap.InfoLevel, "import") 417 err = engine.backend.ImportEngine(ctx, engine.uuid) 418 if !common.IsRetryableError(err) { 419 task.End(zap.ErrorLevel, err) 420 return err 421 } 422 task.Warn("import spuriously failed, going to retry again", log.ShortError(err)) 423 time.Sleep(engine.backend.RetryImportDelay()) 424 } 425 426 return errors.Annotatef(err, "[%s] import reach max retry %d and still failed", engine.uuid, maxRetryTimes) 427 } 428 429 // Cleanup deletes the intermediate data from target. 430 func (engine *ClosedEngine) Cleanup(ctx context.Context) error { 431 task := engine.logger.Begin(zap.InfoLevel, "cleanup") 432 err := engine.backend.CleanupEngine(ctx, engine.uuid) 433 task.End(zap.WarnLevel, err) 434 return err 435 } 436 437 func (engine *ClosedEngine) Logger() log.Logger { 438 return engine.logger 439 } 440 441 // Encoder encodes a row of SQL values into some opaque type which can be 442 // consumed by OpenEngine.WriteEncoded. 443 type Encoder interface { 444 // Close the encoder. 445 Close() 446 447 // Encode encodes a row of SQL values into a backend-friendly format. 448 Encode( 449 logger log.Logger, 450 row []types.Datum, 451 rowID int64, 452 columnPermutation []int, 453 ) (Row, error) 454 } 455 456 // Row represents a single encoded row. 457 type Row interface { 458 // ClassifyAndAppend separates the data-like and index-like parts of the 459 // encoded row, and appends these parts into the existing buffers and 460 // checksums. 461 ClassifyAndAppend( 462 data *Rows, 463 dataChecksum *verification.KVChecksum, 464 indices *Rows, 465 indexChecksum *verification.KVChecksum, 466 ) 467 } 468 469 // Rows represents a collection of encoded rows. 470 type Rows interface { 471 // SplitIntoChunks splits the rows into multiple consecutive parts, each 472 // part having total byte size less than `splitSize`. The meaning of "byte 473 // size" should be consistent with the value used in `Row.ClassifyAndAppend`. 474 SplitIntoChunks(splitSize int) []Rows 475 476 // Clear returns a new collection with empty content. It may share the 477 // capacity with the current instance. The typical usage is `x = x.Clear()`. 478 Clear() Rows 479 } 480 481 type EngineWriter interface { 482 AppendRows( 483 ctx context.Context, 484 tableName string, 485 columnNames []string, 486 commitTS uint64, 487 rows Rows, 488 ) error 489 Close() error 490 }