github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logtail/storage_usage.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 logtail 16 17 import ( 18 "bytes" 19 "context" 20 "fmt" 21 "math" 22 "math/rand" 23 "sort" 24 "sync" 25 "sync/atomic" 26 "time" 27 "unsafe" 28 29 pkgcatalog "github.com/matrixorigin/matrixone/pkg/catalog" 30 "github.com/matrixorigin/matrixone/pkg/common/moerr" 31 "github.com/matrixorigin/matrixone/pkg/common/mpool" 32 "github.com/matrixorigin/matrixone/pkg/container/batch" 33 "github.com/matrixorigin/matrixone/pkg/container/types" 34 "github.com/matrixorigin/matrixone/pkg/container/vector" 35 "github.com/matrixorigin/matrixone/pkg/fileservice" 36 "github.com/matrixorigin/matrixone/pkg/logutil" 37 "github.com/matrixorigin/matrixone/pkg/objectio" 38 v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 39 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio" 40 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" 41 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" 42 "github.com/tidwall/btree" 43 "go.uber.org/zap" 44 ) 45 46 // 1. show accounts 47 // 48 // internal `show accounts` -------\ 49 // |<=====> cn cache <====> (missed or expired) ===> tn cache 50 // mysql client `show accounts` -------/ ^ 51 // ________________| 52 // | update | 53 // | | 54 // incremental ckp | 55 // | 56 // when tn restart --> replay from ckps --------| 57 // 2. collecting storage usage 58 // 59 // when doing incremental ckp 60 // 61 // 1. collect phase: 62 // 1. collect database deletes 63 // 2. collect table deletes 64 // 3. collect segment deletes and inserts 65 // 66 // 2. apply phase 67 // 1. apply all deletes to tn cache and ckp batch[StorageUsageDel] 68 // 2. apply segment inserts to cache and batch[StorageUsageIns] 69 // 70 // 71 // when doing global checkpoint 72 // 73 // 1. replay all changes stored in tn cache into the global ckp batch 74 // 75 // 76 // when tn restart 77 // 78 // 1. replay all changes stored in ckp batches into the tn cache 79 // 80 // 81 82 const ( 83 UsageAccID uint8 = iota 84 UsageDBID 85 UsageTblID 86 UsageObjID 87 UsageSize 88 89 UsageMAX 90 ) 91 92 const StorageUsageMagic uint64 = 0x1A2B3C4D5E6F 93 94 type UsageData struct { 95 AccId uint64 96 DbId uint64 97 TblId uint64 98 Size uint64 99 } 100 101 var zeroUsageData UsageData = UsageData{math.MaxUint32, math.MaxUint64, math.MaxUint64, math.MaxInt64} 102 103 // MockUsageData generates accCnt * dbCnt * tblCnt UsageDatas. 104 // the accIds, dbIds and tblIds are random produced. 105 // this func ensure that all ids are different. 106 func MockUsageData(accCnt, dbCnt, tblCnt int, allocator *atomic.Uint64) (result []UsageData) { 107 for x := 0; x < accCnt; x++ { 108 accId := allocator.Add(1) 109 110 for y := 0; y < dbCnt; y++ { 111 dbId := allocator.Add(1) 112 113 for z := 0; z < tblCnt; z++ { 114 result = append(result, UsageData{ 115 AccId: accId, 116 DbId: dbId, 117 TblId: allocator.Add(1), 118 Size: uint64(rand.Int63() % 0x3fff), 119 }) 120 } 121 } 122 } 123 124 return 125 } 126 127 func (u UsageData) String() string { 128 return fmt.Sprintf("account id = %d; database id = %d; table id = %d; size = %d", 129 u.AccId, u.DbId, u.TblId, u.Size) 130 } 131 132 func (u UsageData) IsZero() bool { 133 return u == zeroUsageData 134 } 135 136 func usageLess(a UsageData, b UsageData) bool { 137 if a.AccId != b.AccId { 138 return a.AccId < b.AccId 139 } 140 141 if a.DbId != b.DbId { 142 return a.DbId < b.DbId 143 } 144 145 return a.TblId < b.TblId 146 } 147 148 type StorageUsageCache struct { 149 // when two requests happens within [lastUpdate, lastUpdate + lazyThreshold], 150 // it will reuse the cached result, no new query to TN. 151 sync.Mutex 152 lazyThreshold time.Duration 153 lastUpdate time.Time 154 // accId -> dbId -> [tblId, size] 155 data *btree.BTreeG[UsageData] 156 lessFunc func(a UsageData, b UsageData) bool 157 } 158 159 type StorageUsageCacheOption = func(c *StorageUsageCache) 160 161 // WithLazyThreshold sets lazyThreshold to lazy seconds 162 func WithLazyThreshold(lazy int) StorageUsageCacheOption { 163 return StorageUsageCacheOption(func(c *StorageUsageCache) { 164 c.lazyThreshold = time.Second * time.Duration(lazy) 165 }) 166 } 167 168 func WithLessFunc(less func(a UsageData, b UsageData) bool) StorageUsageCacheOption { 169 return StorageUsageCacheOption(func(c *StorageUsageCache) { 170 c.lessFunc = less 171 }) 172 } 173 174 func NewStorageUsageCache(opts ...StorageUsageCacheOption) *StorageUsageCache { 175 cache := new(StorageUsageCache) 176 177 cache.fillDefault() 178 179 for _, opt := range opts { 180 opt(cache) 181 } 182 183 cache.data = btree.NewBTreeG[UsageData](cache.lessFunc) 184 cache.data.Clear() 185 186 return cache 187 } 188 189 func (c *StorageUsageCache) fillDefault() { 190 c.lessFunc = usageLess 191 c.lazyThreshold = 0 192 } 193 194 func (c *StorageUsageCache) LessFunc() func(a UsageData, b UsageData) bool { 195 return c.lessFunc 196 } 197 198 func (c *StorageUsageCache) CacheLen() int { 199 return c.data.Len() 200 } 201 202 // MemUsed returns the memory used in megabytes 203 func (c *StorageUsageCache) MemUsed() float64 { 204 itemCnt := uint64(c.data.Len()) 205 206 treeCost := uint64(unsafe.Sizeof(btree.BTreeG[UsageData]{})) + itemCnt/2*12 207 itemCost := itemCnt * uint64(unsafe.Sizeof(UsageData{})) 208 209 return math.Round(float64(treeCost+itemCost)/1048576.0*10e6) / 10e6 210 } 211 212 func (c *StorageUsageCache) Iter() btree.IterG[UsageData] { 213 return c.data.Iter() 214 } 215 216 func (c *StorageUsageCache) IsExpired() bool { 217 if c.lastUpdate.IsZero() || c.lazyThreshold == 0 { 218 return true 219 } 220 221 return time.Since(c.lastUpdate).Seconds() >= c.lazyThreshold.Seconds() 222 } 223 224 func (c *StorageUsageCache) String() string { 225 return fmt.Sprintf("lazy threshold = %f s, last update = %s", 226 c.lazyThreshold.Seconds(), c.lastUpdate.String()) 227 } 228 229 func (c *StorageUsageCache) setUpdateTime(t time.Time) { 230 c.lastUpdate = t 231 } 232 233 func (c *StorageUsageCache) SetOrReplace(usage UsageData) { 234 c.data.Set(usage) 235 c.setUpdateTime(time.Now()) 236 } 237 238 func (c *StorageUsageCache) ClearForUpdate() { 239 c.data.Clear() 240 } 241 242 func (c *StorageUsageCache) GatherAllAccSize() (usages map[uint64]uint64) { 243 usages = make(map[uint64]uint64) 244 c.data.Scan(func(item UsageData) bool { 245 usages[item.AccId] += item.Size 246 return true 247 }) 248 249 return 250 } 251 252 func (c *StorageUsageCache) GatherAccountSize(id uint64) (size uint64, exist bool) { 253 iter := c.data.Iter() 254 defer iter.Release() 255 256 piovt := UsageData{AccId: id} 257 258 if found := iter.Seek(piovt); !found { 259 return 260 } 261 262 if iter.Item().AccId != id { 263 return 264 } 265 266 size += iter.Item().Size 267 for iter.Next() && iter.Item().AccId == id { 268 size += iter.Item().Size 269 } 270 271 exist = true 272 return 273 } 274 275 func (c *StorageUsageCache) Get(usage UsageData) (ret UsageData, exist bool) { 276 return c.data.Get(usage) 277 } 278 279 func (c *StorageUsageCache) Delete(usage UsageData) { 280 c.data.Delete(usage) 281 c.setUpdateTime(time.Now()) 282 } 283 284 type TNUsageMemo struct { 285 sync.Mutex 286 cache *StorageUsageCache 287 // has update 288 pending bool 289 reqTrace []struct { 290 hint string 291 timeStamp time.Time 292 accountId uint64 293 totalSize uint64 294 } 295 296 newAccCache *StorageUsageCache 297 298 pendingReplay struct { 299 datas []*CheckpointData 300 vers []uint32 301 delayed map[uint64]UsageData 302 } 303 } 304 305 func NewTNUsageMemo() *TNUsageMemo { 306 memo := new(TNUsageMemo) 307 memo.cache = NewStorageUsageCache() 308 memo.newAccCache = NewStorageUsageCache() 309 return memo 310 } 311 312 func (m *TNUsageMemo) PrepareReplay(datas []*CheckpointData, vers []uint32) { 313 m.pendingReplay.datas = datas 314 m.pendingReplay.vers = vers 315 } 316 317 func (m *TNUsageMemo) GetDelayed() map[uint64]UsageData { 318 return m.pendingReplay.delayed 319 } 320 321 func (m *TNUsageMemo) GetNewAccCacheLatestUpdate() types.TS { 322 return types.BuildTS(m.newAccCache.lastUpdate.UnixNano(), 0) 323 } 324 325 func (m *TNUsageMemo) UpdateNewAccCache(usage UsageData, del bool) { 326 m.updateHelper(m.newAccCache, usage, del) 327 } 328 329 func (m *TNUsageMemo) ClearNewAccCache() { 330 m.newAccCache.ClearForUpdate() 331 } 332 333 func (m *TNUsageMemo) AddReqTrace(accountId uint64, tSize uint64, t time.Time, hint string) { 334 m.reqTrace = append(m.reqTrace, 335 struct { 336 hint string 337 timeStamp time.Time 338 accountId uint64 339 totalSize uint64 340 }{ 341 hint: hint, 342 timeStamp: t, 343 accountId: accountId, 344 totalSize: tSize, 345 }) 346 } 347 348 func (m *TNUsageMemo) GetAllReqTrace() (accountIds []uint64, timestamps []time.Time, sizes []uint64, hints []string) { 349 m.EnterProcessing() 350 defer m.LeaveProcessing() 351 352 for idx := range m.reqTrace { 353 timestamps = append(timestamps, m.reqTrace[idx].timeStamp) 354 accountIds = append(accountIds, m.reqTrace[idx].accountId) 355 sizes = append(sizes, m.reqTrace[idx].totalSize) 356 hints = append(hints, m.reqTrace[idx].hint) 357 } 358 return 359 } 360 361 func (m *TNUsageMemo) Clear() { 362 m.EnterProcessing() 363 defer m.LeaveProcessing() 364 365 m.reqTrace = m.reqTrace[:0] 366 m.cache.ClearForUpdate() 367 m.pending = false 368 m.cache.setUpdateTime(time.Time{}) 369 m.ClearNewAccCache() 370 } 371 372 func (m *TNUsageMemo) GetCache() *StorageUsageCache { 373 return m.cache 374 } 375 376 func (m *TNUsageMemo) Get(usage UsageData) (old UsageData, exist bool) { 377 return m.cache.Get(usage) 378 } 379 380 func (m *TNUsageMemo) CacheLen() int { 381 return m.cache.CacheLen() 382 } 383 384 func (m *TNUsageMemo) MemoryUsed() float64 { 385 cacheUsed := m.cache.MemUsed() + m.newAccCache.MemUsed() 386 memoUsed := int(unsafe.Sizeof(TNUsageMemo{})) + len(m.reqTrace)*(12+int(unsafe.Sizeof(time.Time{}))) 387 extraUsed := int(unsafe.Sizeof(UsageData{})) * (len(summaryLog[0]) + len(summaryLog[1])) 388 389 memoUsed += extraUsed 390 return cacheUsed + (float64(memoUsed)/1048576.0*1e6)/1e6 391 } 392 393 func (m *TNUsageMemo) EnterProcessing() { 394 m.cache.Lock() 395 m.Lock() 396 } 397 398 func (m *TNUsageMemo) LeaveProcessing() { 399 m.cache.Unlock() 400 m.Unlock() 401 } 402 403 func (m *TNUsageMemo) HasUpdate() bool { 404 return m.pending 405 } 406 407 func (m *TNUsageMemo) gatherAccountSizeHelper(cache *StorageUsageCache, id uint64) (size uint64, exist bool) { 408 return cache.GatherAccountSize(id) 409 } 410 411 func (m *TNUsageMemo) GatherAccountSize(id uint64) (size uint64, exist bool) { 412 return m.gatherAccountSizeHelper(m.cache, id) 413 } 414 415 func (m *TNUsageMemo) GatherNewAccountSize(id uint64) (size uint64, exist bool) { 416 return m.gatherAccountSizeHelper(m.newAccCache, id) 417 } 418 419 func (m *TNUsageMemo) GatherAllAccSize() (usages map[uint64]uint64) { 420 return m.cache.GatherAllAccSize() 421 } 422 423 func (m *TNUsageMemo) updateHelper(cache *StorageUsageCache, usage UsageData, del bool) { 424 size := uint64(0) 425 if old, found := cache.Get(usage); found { 426 size = old.Size 427 } 428 429 if del { 430 if usage.Size > size { 431 //panic("what the A !") 432 usage.Size = size 433 } 434 usage.Size = size - usage.Size 435 } else { 436 //if size < 0 { 437 // panic("what the B !") 438 //} 439 usage.Size = size + usage.Size 440 } 441 442 cache.SetOrReplace(usage) 443 } 444 445 // DeltaUpdate does setting or updating with delta size (delta.Size) 446 func (m *TNUsageMemo) DeltaUpdate(delta UsageData, del bool) { 447 m.pending = true 448 m.updateHelper(m.cache, delta, del) 449 } 450 451 // Replace replaces the old usage with newUsage 452 func (m *TNUsageMemo) Replace(new UsageData) { 453 m.pending = true 454 m.cache.SetOrReplace(new) 455 } 456 457 func (m *TNUsageMemo) Delete(usage UsageData) { 458 m.pending = true 459 m.cache.data.Delete(usage) 460 } 461 462 func (m *TNUsageMemo) applyDeletes( 463 deletes []interface{}, 464 ckpData *CheckpointData, 465 mp *mpool.MPool) string { 466 467 var buf bytes.Buffer 468 var dbs []*catalog.DBEntry 469 470 for _, del := range deletes { 471 switch e := del.(type) { 472 case *catalog.DBEntry: 473 dbs = append(dbs, e) 474 case *catalog.TableEntry: 475 piovt := UsageData{ 476 uint64(e.GetDB().GetTenantID()), 477 e.GetDB().GetID(), e.GetID(), 0} 478 if usage, exist := m.cache.Get(piovt); exist { 479 appendToStorageUsageBat(ckpData, usage, true, mp) 480 m.Delete(usage) 481 buf.WriteString(fmt.Sprintf("[d-tbl]%s_%d_%d_%d_%d; ", 482 e.GetFullName(), usage.AccId, usage.DbId, usage.TblId, usage.Size)) 483 } 484 } 485 } 486 487 isSameDBFunc := func(a UsageData, db *catalog.DBEntry) bool { 488 return a.AccId == uint64(db.GetTenantID()) && a.DbId == db.ID 489 } 490 491 usages := make([]UsageData, 0) 492 for _, db := range dbs { 493 iter := m.cache.Iter() 494 iter.Seek(UsageData{uint64(db.GetTenantID()), db.ID, 0, 0}) 495 496 if !isSameDBFunc(iter.Item(), db) { 497 iter.Release() 498 // db not found in cache 499 buf.WriteString(fmt.Sprintf("[n-db]%s_%d_%d_%d; ", 500 db.GetFullName(), db.GetTenantID(), db.GetID(), 0)) 501 continue 502 } 503 504 usages = append(usages, iter.Item()) 505 for iter.Next() && isSameDBFunc(iter.Item(), db) { 506 usages = append(usages, iter.Item()) 507 } 508 509 iter.Release() 510 511 totalSize := uint64(0) 512 for idx := 0; idx < len(usages); idx++ { 513 m.cache.Delete(usages[idx]) 514 appendToStorageUsageBat(ckpData, usages[idx], true, mp) 515 totalSize += usages[idx].Size 516 } 517 518 buf.WriteString(fmt.Sprintf("[d-db]%s_%d_%d_%d; ", 519 db.GetFullName(), db.GetTenantID(), db.GetID(), totalSize)) 520 521 usages = usages[:0] 522 } 523 524 return buf.String() 525 } 526 527 func (m *TNUsageMemo) applySegInserts(inserts []UsageData, ckpData *CheckpointData, mp *mpool.MPool) { 528 for _, usage := range inserts { 529 appendToStorageUsageBat(ckpData, usage, false, mp) 530 m.DeltaUpdate(usage, false) 531 } 532 } 533 534 func (m *TNUsageMemo) applySegDeletes(deletes []UsageData, ckpData *CheckpointData, mp *mpool.MPool) { 535 for _, usage := range deletes { 536 // can not delete a non-exist usage, right? 537 if _, exist := m.cache.Get(usage); exist { 538 appendToStorageUsageBat(ckpData, usage, true, mp) 539 m.DeltaUpdate(usage, true) 540 } 541 } 542 } 543 544 func (m *TNUsageMemo) replayIntoGCKP(collector *GlobalCollector) { 545 iter := m.cache.data.Iter() 546 for iter.Next() { 547 usage := iter.Item() 548 appendToStorageUsageBat(collector.data, usage, false, collector.Allocator()) 549 } 550 iter.Release() 551 } 552 553 func try2RemoveStaleData(usage UsageData, c *catalog.Catalog) (UsageData, string, bool) { 554 if c == nil { 555 return usage, "", false 556 } 557 558 var err error 559 var dbEntry *catalog.DBEntry 560 var tblEntry *catalog.TableEntry 561 562 dbEntry, err = c.GetDatabaseByID(usage.DbId) 563 if err != nil || dbEntry.HasDropCommitted() { 564 // the db has been deleted 565 name := "deleted" 566 if dbEntry != nil { 567 name = dbEntry.GetName() 568 } 569 log := fmt.Sprintf("[d-db]%s_%d_%d_%d; ", name, usage.AccId, usage.DbId, usage.Size) 570 return usage, log, true 571 } 572 573 tblEntry, err = dbEntry.GetTableEntryByID(usage.TblId) 574 if err != nil || tblEntry.HasDropCommitted() { 575 // the tbl has been deleted 576 name := "deleted" 577 if tblEntry != nil { 578 name = tblEntry.GetFullName() 579 } 580 log := fmt.Sprintf("[d-tbl]%s_%d_%d_%d_%d; ", name, usage.AccId, usage.DbId, usage.TblId, usage.Size) 581 return usage, log, true 582 } 583 584 return usage, "", false 585 } 586 587 func (m *TNUsageMemo) deleteAccount(accId uint64) (size uint64) { 588 trash := make([]UsageData, 0) 589 povit := UsageData{accId, 0, 0, 0} 590 591 iter := m.cache.Iter() 592 593 iter.Seek(povit) 594 595 if iter.Item().AccId != accId { 596 iter.Release() 597 return 598 } 599 600 trash = append(trash, iter.Item()) 601 for iter.Next() { 602 if iter.Item().AccId != accId { 603 break 604 } 605 606 trash = append(trash, iter.Item()) 607 } 608 609 iter.Release() 610 611 for idx := range trash { 612 size += trash[idx].Size 613 m.Delete(trash[idx]) 614 } 615 return 616 } 617 618 func (m *TNUsageMemo) ClearDroppedAccounts(reserved map[uint64]struct{}) string { 619 if reserved == nil { 620 return "" 621 } 622 623 var buf bytes.Buffer 624 625 usages := m.GatherAllAccSize() 626 for accId := range usages { 627 if _, ok := reserved[accId]; !ok { 628 // this account has been deleted 629 size := m.deleteAccount(accId) 630 buf.WriteString(fmt.Sprintf("%d_%d; ", accId, size)) 631 } 632 } 633 return buf.String() 634 } 635 636 // EstablishFromCKPs replays usage info which stored in ckps into the tn cache 637 func (m *TNUsageMemo) EstablishFromCKPs(c *catalog.Catalog) { 638 m.EnterProcessing() 639 defer m.LeaveProcessing() 640 641 var buf bytes.Buffer 642 643 defer func() { 644 for idx := range m.pendingReplay.datas { 645 if m.pendingReplay.datas[idx] != nil { 646 m.pendingReplay.datas[idx].Close() 647 } 648 m.pendingReplay.datas[idx] = nil 649 } 650 logutil.Info("[storage usage] replay:", 651 zap.String("remove old deleted db/tbl", buf.String()), 652 zap.Int("delayed tbl", len(m.pendingReplay.delayed))) 653 }() 654 655 for x := range m.pendingReplay.datas { 656 if m.pendingReplay.vers[x] < CheckpointVersion9 { 657 // haven't StorageUsageIns batch 658 // haven't StorageUsageDel batch 659 continue 660 } 661 662 insVecs := getStorageUsageBatVectors(m.pendingReplay.datas[x].bats[StorageUsageInsIDX]) 663 accCol, dbCol, tblCol, sizeCol := getStorageUsageVectorCols(insVecs) 664 665 var skip bool 666 var log string 667 for y := 0; y < len(accCol); y++ { 668 usage := UsageData{accCol[y], dbCol[y], tblCol[y], sizeCol[y]} 669 670 // these ckps, older than version 11, haven't del bat, we need clear the 671 // usage data which belongs the deleted databases or tables. 672 // 673 // (if a table or db recreate, it's id will change) 674 // 675 if m.pendingReplay.vers[x] < CheckpointVersion11 { 676 // here only remove the deleted db and table. 677 // if table has deletes, we update it in gckp 678 usage, log, skip = try2RemoveStaleData(usage, c) 679 if skip { 680 buf.WriteString(log) 681 continue 682 } 683 if m.pendingReplay.delayed == nil { 684 m.pendingReplay.delayed = make(map[uint64]UsageData) 685 } 686 if old, ok := m.pendingReplay.delayed[usage.TblId]; !ok { 687 m.pendingReplay.delayed[usage.TblId] = usage 688 } else { 689 old.Size += usage.Size 690 m.pendingReplay.delayed[usage.TblId] = old 691 } 692 } 693 m.DeltaUpdate(usage, false) 694 } 695 696 if m.pendingReplay.vers[x] < CheckpointVersion11 { 697 // haven't StorageUsageDel batch 698 continue 699 } 700 701 delVecs := getStorageUsageBatVectors(m.pendingReplay.datas[x].bats[StorageUsageDelIDX]) 702 accCol, dbCol, tblCol, sizeCol = getStorageUsageVectorCols(delVecs) 703 704 for y := 0; y < len(accCol); y++ { 705 usage := UsageData{accCol[y], dbCol[y], tblCol[y], sizeCol[y]} 706 m.DeltaUpdate(usage, true) 707 } 708 } 709 710 } 711 712 // the returned order: 713 // [account_id_vec, db_id_vec, tbl_id_vec, size_vec] 714 func getStorageUsageBatVectors(bat *containers.Batch) []*vector.Vector { 715 accIDVec := bat.GetVectorByName(pkgcatalog.SystemColAttr_AccID).GetDownstreamVector() 716 dbIDVec := bat.GetVectorByName(SnapshotAttr_DBID).GetDownstreamVector() 717 tblIDVec := bat.GetVectorByName(SnapshotAttr_TID).GetDownstreamVector() 718 sizeVec := bat.GetVectorByName(CheckpointMetaAttr_ObjectSize).GetDownstreamVector() 719 720 return []*vector.Vector{ 721 accIDVec, dbIDVec, tblIDVec, nil, sizeVec, 722 } 723 } 724 725 func getStorageUsageVectorCols(vecs []*vector.Vector) ( 726 accCol []uint64, dbCol []uint64, tblCol []uint64, sizeCol []uint64) { 727 728 dbCol = vector.MustFixedCol[uint64](vecs[UsageDBID]) 729 accCol = vector.MustFixedCol[uint64](vecs[UsageAccID]) 730 tblCol = vector.MustFixedCol[uint64](vecs[UsageTblID]) 731 sizeCol = vector.MustFixedCol[uint64](vecs[UsageSize]) 732 733 return 734 } 735 736 const UsageBatMetaTableId uint64 = StorageUsageMagic 737 738 var lastInsUsage UsageData = zeroUsageData 739 var lastDelUsage UsageData = zeroUsageData 740 741 // 0: insert, 1: delete 742 var summaryLog [2][]UsageData 743 744 // this function will accumulate all size of one table into one row. 745 // [acc1, db1, table1, size1] \ 746 // [acc1, db1, table1, size2] ===> [acc1, db1, table1, size1 + size2 + size3] 747 // [acc1, db1, table1, size3] / 748 func appendToStorageUsageBat(data *CheckpointData, usage UsageData, del bool, mp *mpool.MPool) { 749 appendFunc := func(vecs []*vector.Vector) { 750 vector.AppendFixed[uint64](vecs[UsageSize], usage.Size, false, mp) 751 vector.AppendFixed[uint64](vecs[UsageAccID], usage.AccId, false, mp) 752 vector.AppendFixed[uint64](vecs[UsageDBID], usage.DbId, false, mp) 753 vector.AppendFixed[uint64](vecs[UsageTblID], usage.TblId, false, mp) 754 755 // new table 756 if del { 757 summaryLog[1] = append(summaryLog[1], usage) 758 } else { 759 summaryLog[0] = append(summaryLog[0], usage) 760 } 761 } 762 763 updateFunc := func(vecs []*vector.Vector, size uint64) { 764 vector.SetFixedAt[uint64](vecs[UsageSize], vecs[UsageSize].Length()-1, size) 765 766 if del { 767 summaryLog[1][len(summaryLog[1])-1].Size = size 768 } else { 769 summaryLog[0][len(summaryLog[0])-1].Size = size 770 } 771 } 772 773 tableChanged := func(last UsageData) bool { 774 changed := !(last.AccId == usage.AccId && last.DbId == usage.DbId && last.TblId == usage.TblId) 775 return changed 776 } 777 778 entranceFunc := func(last *UsageData, batIdx uint16) { 779 vecs := getStorageUsageBatVectors(data.bats[batIdx]) 780 781 start := vecs[UsageSize].Length() 782 783 // append new row to batch only when table changed 784 if last.IsZero() || tableChanged(*last) { 785 *last = usage 786 appendFunc(vecs) 787 } else { 788 // table not change, only updates the size val 789 last.Size += usage.Size 790 updateFunc(vecs, last.Size) 791 } 792 793 end := vecs[UsageSize].Length() 794 updateStorageUsageMeta(data, UsageBatMetaTableId, int32(start), int32(end), del) 795 } 796 797 if del { 798 entranceFunc(&lastDelUsage, StorageUsageDelIDX) 799 } else { 800 entranceFunc(&lastInsUsage, StorageUsageInsIDX) 801 } 802 } 803 804 func objects2Usages(objs []*catalog.ObjectEntry) (usages []UsageData) { 805 toUsage := func(obj *catalog.ObjectEntry) UsageData { 806 return UsageData{ 807 DbId: obj.GetTable().GetDB().GetID(), 808 Size: uint64(obj.GetCompSize()), 809 TblId: obj.GetTable().GetID(), 810 AccId: uint64(obj.GetTable().GetDB().GetTenantID()), 811 } 812 } 813 814 for idx := range objs { 815 usages = append(usages, toUsage(objs[idx])) 816 } 817 818 return 819 } 820 821 // prepare for storing the storage usage bat location into ckp table meta 822 func updateStorageUsageMeta(data *CheckpointData, tid uint64, start, end int32, del bool) { 823 if del { 824 data.updateTableMeta(tid, StorageUsageDel, start, end) 825 } else { 826 data.updateTableMeta(tid, StorageUsageIns, start, end) 827 } 828 } 829 830 // putCacheBack2Track correct the margin of error happened in incremental checkpoint 831 func putCacheBack2Track(collector *BaseCollector) (string, int) { 832 memo := collector.UsageMemo 833 834 var buf bytes.Buffer 835 836 tblChanges := make(map[[3]uint64]int64) 837 838 usages := objects2Usages(collector.Usage.ObjDeletes) 839 for idx := range usages { 840 uniqueTbl := [3]uint64{usages[idx].AccId, usages[idx].DbId, usages[idx].TblId} 841 tblChanges[uniqueTbl] -= int64(usages[idx].Size) 842 } 843 844 usages = objects2Usages(collector.Usage.ObjInserts) 845 for idx := range usages { 846 uniqueTbl := [3]uint64{usages[idx].AccId, usages[idx].DbId, usages[idx].TblId} 847 tblChanges[uniqueTbl] += int64(usages[idx].Size) 848 } 849 850 delDbs := make(map[uint64]struct{}) 851 delTbls := make(map[uint64]struct{}) 852 for _, del := range collector.Usage.Deletes { 853 switch e := del.(type) { 854 case *catalog.DBEntry: 855 delDbs[e.ID] = struct{}{} 856 case *catalog.TableEntry: 857 delTbls[e.ID] = struct{}{} 858 } 859 } 860 861 if len(tblChanges) == 0 { 862 return "", 0 863 } 864 865 memo.GetCache().ClearForUpdate() 866 867 for uniqueTbl, size := range tblChanges { 868 if size <= 0 { 869 size = 0 870 } 871 872 if _, ok := delDbs[uniqueTbl[1]]; ok { 873 continue 874 } 875 876 if _, ok := delTbls[uniqueTbl[2]]; ok { 877 continue 878 } 879 880 memo.Replace(UsageData{ 881 Size: uint64(size), 882 TblId: uniqueTbl[2], 883 DbId: uniqueTbl[1], 884 AccId: uniqueTbl[0], 885 }) 886 887 if len(memo.pendingReplay.delayed) == 0 { 888 continue 889 } 890 891 if usage, ok := memo.pendingReplay.delayed[uniqueTbl[2]]; ok { 892 buf.WriteString(fmt.Sprintf("[u-tbl]%d_%d_%d_(o)%d_(n)%d; ", 893 usage.AccId, usage.DbId, usage.TblId, usage.Size, size)) 894 895 delete(memo.pendingReplay.delayed, uniqueTbl[2]) 896 } 897 } 898 899 return buf.String(), memo.CacheLen() 900 } 901 902 func applyChanges(collector *BaseCollector, tnUsageMemo *TNUsageMemo) string { 903 tnUsageMemo.newAccCache.ClearForUpdate() 904 905 // must apply seg insert first 906 // step 1: apply seg insert (non-appendable, committed) 907 usage := objects2Usages(collector.Usage.ObjInserts) 908 tnUsageMemo.applySegInserts(usage, collector.data, collector.Allocator()) 909 910 // step 2: apply db, tbl deletes 911 log := tnUsageMemo.applyDeletes(collector.Usage.Deletes, collector.data, collector.Allocator()) 912 913 // step 3: apply seg deletes 914 usage = objects2Usages(collector.Usage.ObjDeletes) 915 tnUsageMemo.applySegDeletes(usage, collector.data, collector.Allocator()) 916 917 return log 918 } 919 920 func doSummary(ckp string, fields ...zap.Field) { 921 defer func() { 922 summaryLog[0] = summaryLog[0][:0] 923 summaryLog[1] = summaryLog[1][:0] 924 925 lastInsUsage = zeroUsageData 926 lastDelUsage = zeroUsageData 927 }() 928 929 sort.Slice(summaryLog[0], func(i, j int) bool { return usageLess(summaryLog[0][i], summaryLog[0][j]) }) 930 sort.Slice(summaryLog[1], func(i, j int) bool { return usageLess(summaryLog[1][i], summaryLog[1][j]) }) 931 932 var buf bytes.Buffer 933 buf.WriteString(fmt.Sprintf("\nCKP[%s]\t%s\n", ckp, time.Now().UTC().String())) 934 935 format := "\t%19d\t%19d\t%19d\t%19.6fmb" 936 accumulated := int64(0) 937 938 for idx := range summaryLog[0] { 939 buf.WriteString(fmt.Sprintf(format+" -> i\n", 940 summaryLog[0][idx].AccId, 941 summaryLog[0][idx].DbId, 942 summaryLog[0][idx].TblId, 943 float64(summaryLog[0][idx].Size)/(1024*1024))) 944 945 accumulated += int64(summaryLog[0][idx].Size) 946 } 947 948 for idx := range summaryLog[1] { 949 buf.WriteString(fmt.Sprintf(format+" -> d\n", 950 summaryLog[1][idx].AccId, 951 summaryLog[1][idx].DbId, 952 summaryLog[1][idx].TblId, 953 float64(summaryLog[1][idx].Size)/(1024*1024))) 954 955 accumulated -= int64(summaryLog[1][idx].Size) 956 } 957 958 buf.WriteString(fmt.Sprintf("accumulated size in this ckp: %19.6fmb, ", 959 float64(accumulated)/(1024*1024))) 960 961 fields = append(fields, zap.String("storage usage summary when ckp", buf.String())) 962 logutil.Info(fmt.Sprintf("storage usage [%s]", ckp), fields...) 963 } 964 965 func FillUsageBatOfGlobal(collector *GlobalCollector) { 966 start := time.Now() 967 968 collector.UsageMemo.EnterProcessing() 969 defer func() { 970 v2.TaskGCkpCollectUsageDurationHistogram.Observe(time.Since(start).Seconds()) 971 collector.UsageMemo.LeaveProcessing() 972 }() 973 974 log1, cnt := putCacheBack2Track(collector.BaseCollector) 975 log2 := collector.UsageMemo.ClearDroppedAccounts(collector.Usage.ReservedAccIds) 976 collector.UsageMemo.replayIntoGCKP(collector) 977 978 doSummary("G", 979 zap.String("update old data", log1), 980 zap.Int("tables back to track", cnt), 981 zap.String("accounts cleaned", log2)) 982 } 983 984 func FillUsageBatOfIncremental(collector *IncrementalCollector) { 985 start := time.Now() 986 var memoryUsed float64 987 988 collector.UsageMemo.EnterProcessing() 989 defer func() { 990 v2.TaskStorageUsageCacheMemUsedGauge.Set(memoryUsed) 991 v2.TaskICkpCollectUsageDurationHistogram.Observe(time.Since(start).Seconds()) 992 collector.UsageMemo.LeaveProcessing() 993 }() 994 995 log1 := applyChanges(collector.BaseCollector, collector.UsageMemo) 996 //log2 := applyTransfer(collector.BaseCollector, collector.UsageMemo) 997 998 memoryUsed = collector.UsageMemo.MemoryUsed() 999 doSummary("I", 1000 zap.Float64("cache mem used", memoryUsed), 1001 zap.String("applied deletes", log1)) 1002 } 1003 1004 // GetStorageUsageHistory is for debug to show these storage usage changes. 1005 // 1006 // 1. load each ckp meta batch. 1007 // 1008 // 2. load the specified storage usage ins/del batch using locations storing in meta batch. 1009 func GetStorageUsageHistory( 1010 ctx context.Context, 1011 locations []objectio.Location, versions []uint32, 1012 fs fileservice.FileService, mp *mpool.MPool) ([][]UsageData, [][]UsageData, error) { 1013 1014 var err error 1015 1016 // 1. load each ckp meta batch 1017 datas, selectedVers, selectedLocs, err := loadMetaBat(ctx, versions, locations, fs, mp) 1018 if err != nil { 1019 return nil, nil, err 1020 } 1021 1022 usageInsData := make([][]UsageData, len(selectedVers)) 1023 usageDelData := make([][]UsageData, len(selectedVers)) 1024 1025 var usageInsBat []*batch.Batch 1026 var usageDelBat []*batch.Batch 1027 1028 for idx := 0; idx < len(datas); idx++ { 1029 datas[idx].GetTableMeta(UsageBatMetaTableId, selectedVers[idx], selectedLocs[idx]) 1030 usageMeta := datas[idx].meta[UsageBatMetaTableId] 1031 1032 if usageMeta == nil { 1033 continue 1034 } 1035 1036 // 2.1. load storage usage ins bat 1037 if usageInsBat, err = loadStorageUsageBatch( 1038 ctx, usageMeta.tables[StorageUsageIns].locations, 1039 selectedVers[idx], StorageUsageInsIDX, fs, mp); err != nil { 1040 return nil, nil, err 1041 } 1042 1043 // 2.2. load storage usage del bat 1044 if usageDelBat, err = loadStorageUsageBatch( 1045 ctx, usageMeta.tables[StorageUsageDel].locations, 1046 selectedVers[idx], StorageUsageDelIDX, fs, mp); err != nil { 1047 return nil, nil, err 1048 } 1049 1050 // 3. collect usage data from these batches 1051 for _, bat := range usageInsBat { 1052 if ret := cnBatchToUsageDatas(bat); len(ret) != 0 { 1053 usageInsData[idx] = append(usageInsData[idx], ret...) 1054 } 1055 } 1056 1057 for _, bat := range usageDelBat { 1058 if ret := cnBatchToUsageDatas(bat); len(ret) != 0 { 1059 usageDelData[idx] = append(usageDelData[idx], ret...) 1060 } 1061 } 1062 } 1063 1064 // release memory 1065 for idx := range datas { 1066 datas[idx].bats[MetaIDX].Clean(mp) 1067 } 1068 1069 for idx := range usageInsBat { 1070 usageInsBat[idx].Clean(mp) 1071 } 1072 1073 for idx := range usageDelBat { 1074 usageDelBat[idx].Clean(mp) 1075 } 1076 1077 return usageInsData, usageDelData, nil 1078 } 1079 1080 func cnBatchToUsageDatas(bat *batch.Batch) []UsageData { 1081 accCol := vector.MustFixedCol[uint64](bat.GetVector(2)) 1082 dbCol := vector.MustFixedCol[uint64](bat.GetVector(3)) 1083 tblCol := vector.MustFixedCol[uint64](bat.GetVector(4)) 1084 sizeCol := vector.MustFixedCol[uint64](bat.GetVector(6)) 1085 1086 usages := make([]UsageData, 0) 1087 1088 for idx := range accCol { 1089 usages = append(usages, UsageData{ 1090 accCol[idx], 1091 dbCol[idx], 1092 tblCol[idx], 1093 sizeCol[idx], 1094 }) 1095 } 1096 return usages 1097 } 1098 1099 func loadMetaBat( 1100 ctx context.Context, 1101 versions []uint32, locations []objectio.Location, 1102 fs fileservice.FileService, mp *mpool.MPool) ( 1103 datas []*CNCheckpointData, 1104 selectedVers []uint32, 1105 selectedLocs []objectio.Location, 1106 err error) { 1107 1108 var idxes []uint16 1109 1110 for idx := 0; idx < len(locations); idx++ { 1111 if versions[idx] < CheckpointVersion11 { 1112 // start with version 11, storage usage ins/del bat's locations is recorded in meta bat. 1113 continue 1114 } 1115 1116 data := NewCNCheckpointData() 1117 1118 // 1.1. prefetch meta bat 1119 meteIdxSchema := checkpointDataReferVersions[versions[idx]][MetaIDX] 1120 for attrIdx := range meteIdxSchema.attrs { 1121 idxes = append(idxes, uint16(attrIdx)) 1122 } 1123 1124 data.PrefetchMetaIdx(ctx, versions[idx], idxes, locations[idx], fs) 1125 1126 // 1.2. read meta bat 1127 reader, err := blockio.NewObjectReader(fs, locations[idx]) 1128 if err != nil { 1129 return nil, nil, nil, err 1130 } 1131 1132 data.InitMetaIdx(ctx, versions[idx], reader, locations[idx], mp) 1133 1134 idxes = idxes[:0] 1135 datas = append(datas, data) 1136 selectedVers = append(selectedVers, versions[idx]) 1137 selectedLocs = append(selectedLocs, locations[idx]) 1138 } 1139 1140 return datas, selectedVers, selectedLocs, nil 1141 } 1142 1143 func loadStorageUsageBatch( 1144 ctx context.Context, locations BlockLocations, version uint32, 1145 batIdx uint16, fs fileservice.FileService, mp *mpool.MPool) ([]*batch.Batch, error) { 1146 1147 var bats []*batch.Batch 1148 1149 it := locations.MakeIterator() 1150 for it.HasNext() { 1151 block := it.Next() 1152 schema := checkpointDataReferVersions[version][uint32(batIdx)] 1153 reader, err := blockio.NewObjectReader(fs, block.GetLocation()) 1154 if err != nil { 1155 return nil, err 1156 } 1157 1158 if bat, err := LoadCNSubBlkColumnsByMetaWithId( 1159 ctx, schema.types, schema.attrs, batIdx, 1160 block.GetID(), version, reader, mp); err != nil { 1161 return nil, err 1162 } else { 1163 bats = append(bats, bat) 1164 } 1165 } 1166 return bats, nil 1167 } 1168 1169 func PairAccountVsDB(c *catalog.Catalog) map[uint64]uint64 { 1170 pairs := make(map[uint64]uint64) 1171 1172 processor := new(catalog.LoopProcessor) 1173 processor.DatabaseFn = func(entry *catalog.DBEntry) error { 1174 pairs[entry.GetID()] = uint64(entry.GetTenantID()) 1175 return nil 1176 } 1177 1178 c.RecurLoop(processor) 1179 return pairs 1180 } 1181 1182 func CorrectUsageWrongPlacement(c *catalog.Catalog) (int, float64, error) { 1183 memo := c.GetUsageMemo().(*TNUsageMemo) 1184 if memo == nil { 1185 return 0, 0, moerr.NewInternalErrorNoCtx("tn usage cache is nil") 1186 } 1187 1188 var buf bytes.Buffer 1189 1190 pairs := PairAccountVsDB(c) 1191 1192 memo.EnterProcessing() 1193 defer memo.LeaveProcessing() 1194 1195 usages := memo.cache.data.Items() 1196 1197 anyTransferred := int(0) 1198 transferredSize := float64(0) 1199 for idx := range usages { 1200 if pairs[usages[idx].DbId] != usages[idx].AccId { 1201 anyTransferred++ 1202 transferredSize += float64(usages[idx].Size) 1203 1204 memo.DeltaUpdate(usages[idx], true) 1205 buf.WriteString(fmt.Sprintf("[td-tbl]%d_%d_%d_%d; ", 1206 usages[idx].AccId, usages[idx].DbId, usages[idx].TblId, usages[idx].Size)) 1207 //memo.pendingTransfer.deletes = append(memo.pendingTransfer.deletes, usages[idx]) 1208 1209 usages[idx].AccId = pairs[usages[idx].DbId] 1210 memo.DeltaUpdate(usages[idx], false) 1211 buf.WriteString(fmt.Sprintf("[ti-tbl]%d_%d_%d_%d; ", 1212 usages[idx].AccId, usages[idx].DbId, usages[idx].TblId, usages[idx].Size)) 1213 //memo.pendingTransfer.inserts = append(memo.pendingTransfer.inserts, usages[idx]) 1214 } 1215 } 1216 1217 transferredSize /= 1024 * 1024 1218 logutil.Info("[storage usage] apply transfer: ", 1219 zap.String(fmt.Sprintf("transferred %d tbl, %f mb", anyTransferred, transferredSize), buf.String())) 1220 1221 return anyTransferred, transferredSize, nil 1222 } 1223 1224 func EliminateErrorsOnCache(c *catalog.Catalog, end types.TS) int { 1225 collector := BaseCollector{} 1226 loop := catalog.LoopProcessor{} 1227 loop.ObjectFn = func(entry *catalog.ObjectEntry) error { 1228 if entry.GetTable().GetDB().HasDropCommitted() || entry.GetTable().HasDropCommitted() { 1229 return nil 1230 } 1231 1232 // PXU TODO 1233 if entry.IsAppendable() || !entry.IsCommittedLocked() { 1234 return nil 1235 } 1236 1237 entry.Lock() 1238 createTS := entry.GetCreatedAtLocked() 1239 if createTS.GreaterEq(&end) { 1240 entry.Unlock() 1241 return nil 1242 } 1243 entry.Unlock() 1244 1245 if entry.HasDropCommitted() { 1246 collector.Usage.ObjDeletes = append(collector.Usage.ObjDeletes, entry) 1247 } else { 1248 collector.Usage.ObjInserts = append(collector.Usage.ObjInserts, entry) 1249 } 1250 1251 return nil 1252 } 1253 1254 c.RecurLoop(&loop) 1255 1256 collector.UsageMemo = c.GetUsageMemo().(*TNUsageMemo) 1257 1258 collector.UsageMemo.EnterProcessing() 1259 defer collector.UsageMemo.LeaveProcessing() 1260 _, cnt := putCacheBack2Track(&collector) 1261 1262 return cnt 1263 }