github.com/matrixorigin/matrixone@v1.2.0/pkg/backup/tae.go (about) 1 // Copyright 2023 Matrix Origin 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 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package backup 16 17 import ( 18 "context" 19 "crypto/sha256" 20 "encoding/json" 21 "fmt" 22 "github.com/matrixorigin/matrixone/pkg/common/moerr" 23 "github.com/matrixorigin/matrixone/pkg/common/runtime" 24 "github.com/matrixorigin/matrixone/pkg/container/types" 25 "github.com/matrixorigin/matrixone/pkg/container/vector" 26 "github.com/matrixorigin/matrixone/pkg/fileservice" 27 "github.com/matrixorigin/matrixone/pkg/logutil" 28 "github.com/matrixorigin/matrixone/pkg/objectio" 29 "github.com/matrixorigin/matrixone/pkg/sql/plan/function/ctl" 30 "github.com/matrixorigin/matrixone/pkg/util/executor" 31 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio" 32 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 33 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/checkpoint" 34 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail" 35 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks" 36 37 "io" 38 "os" 39 "path" 40 runtime2 "runtime" 41 "strconv" 42 "strings" 43 "sync" 44 "time" 45 ) 46 47 func getFileNames(ctx context.Context, retBytes [][][]byte) ([]string, error) { 48 var err error 49 cr := ctl.Result{} 50 err = json.Unmarshal(retBytes[0][0], &cr) 51 if err != nil { 52 return nil, err 53 } 54 rsSlice, ok := cr.Data.([]interface{}) 55 if !ok { 56 return nil, moerr.NewInternalError(ctx, "invalid ctl result") 57 } 58 var fileName []string 59 for _, rs := range rsSlice { 60 str, ok := rs.(string) 61 if !ok { 62 return nil, moerr.NewInternalError(ctx, "invalid ctl string") 63 } 64 65 for _, x := range strings.Split(str, ";") { 66 if len(x) == 0 { 67 continue 68 } 69 fileName = append(fileName, x) 70 } 71 } 72 return fileName, err 73 } 74 75 func BackupData(ctx context.Context, srcFs, dstFs fileservice.FileService, dir string, config *Config) error { 76 v, ok := runtime.ProcessLevelRuntime().GetGlobalVariables(runtime.InternalSQLExecutor) 77 if !ok { 78 return moerr.NewNotSupported(ctx, "no implement sqlExecutor") 79 } 80 exec := v.(executor.SQLExecutor) 81 opts := executor.Options{} 82 sql := "select mo_ctl('dn','Backup','')" 83 res, err := exec.Exec(ctx, sql, opts) 84 if err != nil { 85 return err 86 } 87 88 var retByts [][][]byte 89 res.ReadRows(func(_ int, cols []*vector.Vector) bool { 90 retByts = append(retByts, executor.GetBytesRows(cols[0])) 91 return true 92 }) 93 res.Close() 94 95 fileName, err := getFileNames(ctx, retByts) 96 if err != nil { 97 return err 98 } 99 count := config.Parallelism 100 return execBackup(ctx, srcFs, dstFs, fileName, int(count), config.BackupTs, config.BackupType) 101 } 102 103 func getParallelCount(count int) int { 104 if count > 0 && count < 512 { 105 return count 106 } 107 cupNum := runtime2.NumCPU() 108 if cupNum < 8 { 109 return 50 110 } else if cupNum < 16 { 111 return 80 112 } else if cupNum < 32 { 113 return 128 114 } else if cupNum < 64 { 115 return 256 116 } 117 return 512 118 } 119 120 // parallelCopyData copy data from srcFs to dstFs in parallel 121 func parallelCopyData(srcFs, dstFs fileservice.FileService, 122 files map[string]*objectio.BackupObject, 123 parallelCount int, 124 gcFileMap map[string]string, 125 ) ([]*taeFile, error) { 126 var copyCount, skipCount, copySize int64 127 var printMutex, fileMutex sync.Mutex 128 stopPrint := false 129 defer func() { 130 printMutex.Lock() 131 if !stopPrint { 132 stopPrint = true 133 } 134 printMutex.Unlock() 135 }() 136 // record files 137 taeFileList := make([]*taeFile, 0, len(files)) 138 errC := make(chan error, 1) 139 defer close(errC) 140 jobScheduler := tasks.NewParallelJobScheduler(parallelCount) 141 defer jobScheduler.Stop() 142 go func() { 143 for { 144 printMutex.Lock() 145 if stopPrint { 146 printMutex.Unlock() 147 break 148 } 149 printMutex.Unlock() 150 fileMutex.Lock() 151 logutil.Info("backup", common.OperationField("copy file"), 152 common.AnyField("copy file size", copySize), 153 common.AnyField("copy file num", copyCount), 154 common.AnyField("skip file num", skipCount), 155 common.AnyField("total file num", len(files))) 156 fileMutex.Unlock() 157 time.Sleep(time.Second * 5) 158 } 159 }() 160 161 backupJobs := make([]*tasks.Job, len(files)) 162 getJob := func(srcFs, dstFs fileservice.FileService, backupObject *objectio.BackupObject) *tasks.Job { 163 job := new(tasks.Job) 164 job.Init(context.Background(), backupObject.Location.Name().String(), tasks.JTAny, 165 func(_ context.Context) *tasks.JobResult { 166 167 name := backupObject.Location.Name().String() 168 size := backupObject.Location.Extent().End() + objectio.FooterSize 169 if !backupObject.NeedCopy { 170 fileMutex.Lock() 171 copyCount++ 172 copySize += int64(size) 173 taeFileList = append(taeFileList, &taeFile{ 174 path: name, 175 size: int64(size), 176 needCopy: false, 177 ts: backupObject.CrateTS, 178 }) 179 fileMutex.Unlock() 180 return &tasks.JobResult{ 181 Res: nil, 182 } 183 } 184 checksum, err := CopyFileWithRetry(context.Background(), srcFs, dstFs, backupObject.Location.Name().String(), "") 185 if err != nil { 186 if moerr.IsMoErrCode(err, moerr.ErrFileNotFound) { 187 // TODO: handle file not found, maybe GC 188 fileMutex.Lock() 189 skipCount++ 190 fileMutex.Unlock() 191 return &tasks.JobResult{ 192 Res: nil, 193 } 194 } else { 195 errC <- err 196 return &tasks.JobResult{ 197 Err: err, 198 Res: nil, 199 } 200 } 201 } 202 fileMutex.Lock() 203 copyCount++ 204 copySize += int64(size) 205 taeFileList = append(taeFileList, &taeFile{ 206 path: name, 207 size: int64(size), 208 checksum: checksum, 209 needCopy: backupObject.NeedCopy, 210 ts: backupObject.CrateTS, 211 }) 212 fileMutex.Unlock() 213 return &tasks.JobResult{ 214 Res: nil, 215 } 216 }) 217 return job 218 } 219 220 idx := 0 221 for n := range files { 222 backupJobs[idx] = getJob(srcFs, dstFs, files[n]) 223 idx++ 224 } 225 226 for n := range backupJobs { 227 err := jobScheduler.Schedule(backupJobs[n]) 228 if err != nil { 229 logutil.Infof("schedule job failed %v", err.Error()) 230 return nil, err 231 } 232 select { 233 case err = <-errC: 234 logutil.Infof("copy file failed %v", err.Error()) 235 return nil, err 236 default: 237 } 238 } 239 240 for n := range backupJobs { 241 ret := backupJobs[n].WaitDone() 242 if ret.Err != nil { 243 logutil.Infof("wait job done failed %v", ret.Err.Error()) 244 return nil, ret.Err 245 } 246 } 247 248 logutil.Info("backup", common.OperationField("copy file"), 249 common.AnyField("copy file size", copySize), 250 common.AnyField("copy file num", copyCount), 251 common.AnyField("skip file num", skipCount), 252 common.AnyField("total file num", len(files))) 253 return taeFileList, nil 254 } 255 256 func execBackup( 257 ctx context.Context, 258 srcFs, dstFs fileservice.FileService, 259 names []string, 260 count int, 261 ts types.TS, 262 typ string, 263 ) error { 264 backupTime := names[0] 265 trimInfo := names[1] 266 names = names[1:] 267 files := make(map[string]*objectio.BackupObject, 0) 268 gcFileMap := make(map[string]string) 269 softDeletes := make(map[string]bool) 270 var loadDuration, copyDuration, reWriteDuration time.Duration 271 var oNames []*objectio.BackupObject 272 parallelNum := getParallelCount(count) 273 logutil.Info("backup", common.OperationField("start backup"), 274 common.AnyField("backup time", backupTime), 275 common.AnyField("checkpoint num", len(names)), 276 common.AnyField("parallel num", parallelNum)) 277 defer func() { 278 logutil.Info("backup", common.OperationField("end backup"), 279 common.AnyField("load checkpoint cost", loadDuration), 280 common.AnyField("copy file cost", copyDuration), 281 common.AnyField("rewrite checkpoint cost", reWriteDuration)) 282 }() 283 now := time.Now() 284 baseTS := ts 285 for i, name := range names { 286 if len(name) == 0 { 287 continue 288 } 289 ckpStr := strings.Split(name, ":") 290 if len(ckpStr) != 2 && i > 0 { 291 return moerr.NewInternalError(ctx, fmt.Sprintf("invalid checkpoint string: %v", ckpStr)) 292 } 293 metaLoc := ckpStr[0] 294 version, err := strconv.ParseUint(ckpStr[1], 10, 32) 295 if err != nil { 296 return err 297 } 298 key, err := blockio.EncodeLocationFromString(metaLoc) 299 if err != nil { 300 return err 301 } 302 var oneNames []*objectio.BackupObject 303 var data *logtail.CheckpointData 304 if i == 0 { 305 oneNames, data, err = logtail.LoadCheckpointEntriesFromKey(ctx, srcFs, key, uint32(version), nil, &baseTS) 306 } else { 307 oneNames, data, err = logtail.LoadCheckpointEntriesFromKey(ctx, srcFs, key, uint32(version), &softDeletes, &baseTS) 308 } 309 if err != nil { 310 return err 311 } 312 defer data.Close() 313 oNames = append(oNames, oneNames...) 314 } 315 loadDuration += time.Since(now) 316 now = time.Now() 317 for _, oName := range oNames { 318 if files[oName.Location.Name().String()] == nil { 319 files[oName.Location.Name().String()] = oName 320 } 321 } 322 323 // trim checkpoint and block 324 var cnLoc, tnLoc, mergeStart, mergeEnd string 325 var end, start types.TS 326 var version uint64 327 if trimInfo != "" { 328 var err error 329 ckpStr := strings.Split(trimInfo, ":") 330 if len(ckpStr) != 5 { 331 return moerr.NewInternalError(ctx, fmt.Sprintf("invalid checkpoint string: %v", ckpStr)) 332 } 333 cnLoc = ckpStr[0] 334 mergeEnd = ckpStr[2] 335 tnLoc = ckpStr[3] 336 mergeStart = ckpStr[4] 337 end = types.StringToTS(mergeEnd) 338 start = types.StringToTS(mergeStart) 339 version, err = strconv.ParseUint(ckpStr[1], 10, 32) 340 if err != nil { 341 return err 342 } 343 } 344 345 // copy data 346 taeFileList, err := parallelCopyData(srcFs, dstFs, files, parallelNum, gcFileMap) 347 if err != nil { 348 return err 349 } 350 351 // copy checkpoint and gc meta 352 sizeList, err := CopyDir(ctx, srcFs, dstFs, "ckp", start) 353 if err != nil { 354 return err 355 } 356 taeFileList = append(taeFileList, sizeList...) 357 sizeList, err = CopyDir(ctx, srcFs, dstFs, "gc", start) 358 if err != nil { 359 return err 360 } 361 copyDuration += time.Since(now) 362 taeFileList = append(taeFileList, sizeList...) 363 now = time.Now() 364 if trimInfo != "" { 365 cnLocation, err := blockio.EncodeLocationFromString(cnLoc) 366 if err != nil { 367 return err 368 } 369 tnLocation, err := blockio.EncodeLocationFromString(tnLoc) 370 if err != nil { 371 return err 372 } 373 var checkpointFiles []string 374 cnLocation, tnLocation, checkpointFiles, err = logtail.ReWriteCheckpointAndBlockFromKey(ctx, srcFs, dstFs, 375 cnLocation, tnLocation, uint32(version), start, softDeletes) 376 for _, name := range checkpointFiles { 377 dentry, err := dstFs.StatFile(ctx, name) 378 if err != nil { 379 return err 380 } 381 taeFileList = append(taeFileList, &taeFile{ 382 path: dentry.Name, 383 size: dentry.Size, 384 needCopy: true, 385 ts: start, 386 }) 387 } 388 if err != nil { 389 return err 390 } 391 file, err := checkpoint.MergeCkpMeta(ctx, dstFs, cnLocation, tnLocation, start, end) 392 if err != nil { 393 return err 394 } 395 dentry, err := dstFs.StatFile(ctx, file) 396 if err != nil { 397 return err 398 } 399 taeFileList = append(taeFileList, &taeFile{ 400 path: "ckp/" + dentry.Name, 401 size: dentry.Size, 402 needCopy: true, 403 ts: start, 404 }) 405 } 406 reWriteDuration += time.Since(now) 407 //save tae files size 408 err = saveTaeFilesList(ctx, dstFs, taeFileList, backupTime, start.ToString(), typ) 409 if err != nil { 410 return err 411 } 412 return nil 413 } 414 415 func CopyDir(ctx context.Context, srcFs, dstFs fileservice.FileService, dir string, backup types.TS) ([]*taeFile, error) { 416 var checksum []byte 417 files, err := srcFs.List(ctx, dir) 418 if err != nil { 419 return nil, err 420 } 421 taeFileList := make([]*taeFile, 0, len(files)) 422 423 for _, file := range files { 424 if file.IsDir { 425 panic("not support dir") 426 } 427 start, _ := blockio.DecodeCheckpointMetadataFileName(file.Name) 428 if !backup.IsEmpty() && start.GreaterEq(&backup) { 429 logutil.Infof("[Backup] skip file %v", file.Name) 430 continue 431 } 432 checksum, err = CopyFileWithRetry(ctx, srcFs, dstFs, file.Name, dir) 433 if err != nil { 434 return nil, err 435 } 436 taeFileList = append(taeFileList, &taeFile{ 437 path: dir + string(os.PathSeparator) + file.Name, 438 size: file.Size, 439 checksum: checksum, 440 needCopy: true, 441 ts: backup, 442 }) 443 } 444 return taeFileList, nil 445 } 446 447 func CopyFileWithRetry(ctx context.Context, srcFs, dstFs fileservice.FileService, name, dstDir string) ([]byte, error) { 448 return fileservice.DoWithRetry( 449 "CopyFile", 450 func() ([]byte, error) { 451 return CopyFile(ctx, srcFs, dstFs, name, dstDir) 452 }, 453 64, 454 fileservice.IsRetryableError, 455 ) 456 } 457 458 // CopyFile copy file from srcFs to dstFs and return checksum of the written file. 459 func CopyFile(ctx context.Context, srcFs, dstFs fileservice.FileService, name, dstDir string) ([]byte, error) { 460 if dstDir != "" { 461 name = path.Join(dstDir, name) 462 } 463 var reader io.ReadCloser 464 ioVec := &fileservice.IOVector{ 465 FilePath: name, 466 Entries: []fileservice.IOEntry{ 467 { 468 ReadCloserForRead: &reader, 469 Offset: 0, 470 Size: -1, 471 }, 472 }, 473 Policy: fileservice.SkipAllCache, 474 } 475 476 err := srcFs.Read(ctx, ioVec) 477 if err != nil { 478 return nil, err 479 } 480 defer reader.Close() 481 // hash 482 hasher := sha256.New() 483 hashingReader := io.TeeReader(reader, hasher) 484 dstIoVec := fileservice.IOVector{ 485 FilePath: name, 486 Entries: []fileservice.IOEntry{ 487 { 488 ReaderForWrite: hashingReader, 489 Offset: 0, 490 Size: -1, 491 }, 492 }, 493 Policy: fileservice.SkipAllCache, 494 } 495 496 err = dstFs.Write(ctx, dstIoVec) 497 if err != nil { 498 return nil, err 499 } 500 return hasher.Sum(nil), nil 501 }