github.com/NVIDIA/aistore@v1.3.23-0.20240517131212-7df6609be51d/ext/dsort/manager.go (about) 1 // Package dsort provides distributed massively parallel resharding for very large datasets. 2 /* 3 * Copyright (c) 2018-2024, NVIDIA CORPORATION. All rights reserved. 4 */ 5 package dsort 6 7 import ( 8 "fmt" 9 "io" 10 "net/http" 11 "os" 12 "sync" 13 "time" 14 15 "github.com/NVIDIA/aistore/cmn" 16 "github.com/NVIDIA/aistore/cmn/atomic" 17 "github.com/NVIDIA/aistore/cmn/cos" 18 "github.com/NVIDIA/aistore/cmn/debug" 19 "github.com/NVIDIA/aistore/cmn/kvdb" 20 "github.com/NVIDIA/aistore/cmn/nlog" 21 "github.com/NVIDIA/aistore/core" 22 "github.com/NVIDIA/aistore/core/meta" 23 "github.com/NVIDIA/aistore/ext/dsort/ct" 24 "github.com/NVIDIA/aistore/ext/dsort/shard" 25 "github.com/NVIDIA/aistore/fs" 26 "github.com/NVIDIA/aistore/memsys" 27 "github.com/NVIDIA/aistore/stats" 28 "github.com/NVIDIA/aistore/sys" 29 "github.com/NVIDIA/aistore/transport" 30 "github.com/NVIDIA/aistore/transport/bundle" 31 "github.com/NVIDIA/aistore/xact/xreg" 32 "github.com/pkg/errors" 33 ) 34 35 const ( 36 // Stream names 37 recvReqStreamNameFmt = "recv-%sq" 38 recvRespStreamNameFmt = "recv-%sp" 39 shardStreamNameFmt = "shrd-%ss" 40 ) 41 42 // State of the cleans - see `cleanup` and `finalCleanup` 43 const ( 44 noCleanedState = iota 45 initiallyCleanedState 46 finallyCleanedState 47 ) 48 49 const ( 50 // Size of the buffer used for serialization of the shards/records. 51 serializationBufSize = 10 * cos.MiB 52 ) 53 54 type ( 55 global struct { 56 tstats stats.Tracker 57 mm *memsys.MMSA 58 } 59 buildingShardInfo struct { 60 shardName string 61 } 62 63 // progressState abstracts all information meta information about progress of 64 // the job. 65 progressState struct { 66 cleanWait *sync.Cond // waiting room for `cleanup` and `finalCleanup` method so then can run in correct order 67 wg *sync.WaitGroup 68 // doneCh is closed when the job is aborted so that goroutines know when 69 // they need to stop. 70 doneCh chan struct{} 71 inProgress atomic.Bool 72 aborted atomic.Bool 73 cleaned uint8 // current state of the cleanliness - no cleanup, initial cleanup, final cleanup 74 } 75 76 // Manager maintains all the state required for a single run of a distributed archive file shuffle. 77 Manager struct { 78 // tagged fields are the only fields persisted once dsort finishes 79 ManagerUUID string `json:"manager_uuid"` 80 Metrics *Metrics `json:"metrics"` 81 Pars *parsedReqSpec `json:"pars"` 82 83 mg *ManagerGroup // parent 84 mu sync.Mutex 85 smap *meta.Smap 86 recm *shard.RecordManager 87 shardRW shard.RW 88 startShardCreation chan struct{} 89 client *http.Client // Client for sending records metadata 90 compression struct { 91 totalShardSize atomic.Int64 92 totalExtractedSize atomic.Int64 93 } 94 received struct { 95 count atomic.Int32 // Number of FileMeta slices received, defining what step in the sort target is in. 96 ch chan int32 97 } 98 refCount atomic.Int64 // Refcount to cleanup. 99 inFlight atomic.Int64 // Refcount in-flight stream requests 100 state progressState 101 extractionPhase struct { 102 adjuster *concAdjuster 103 } 104 streams struct { 105 shards *bundle.Streams 106 } 107 creationPhase struct { 108 metadata CreationPhaseMetadata 109 } 110 finishedAck struct { 111 mu sync.Mutex 112 m map[string]struct{} // finished acks: tid -> ack 113 } 114 dsorter dsorter 115 dsorterStarted sync.WaitGroup 116 callTimeout time.Duration // max time to wait for another node to respond 117 config *cmn.Config 118 xctn *xaction 119 } 120 ) 121 122 var g global 123 124 // interface guard 125 var ( 126 _ cos.Packer = (*buildingShardInfo)(nil) 127 _ cos.Unpacker = (*buildingShardInfo)(nil) 128 _ core.Xact = (*xaction)(nil) 129 ) 130 131 func Pinit(si core.Node, config *cmn.Config) { 132 psi = si 133 newBcastClient(config) 134 } 135 136 func Tinit(tstats stats.Tracker, db kvdb.Driver, config *cmn.Config) { 137 Managers = NewManagerGroup(db, false) 138 139 xreg.RegBckXact(&factory{}) 140 141 debug.Assert(g.mm == nil) // only once 142 { 143 g.tstats = tstats 144 g.mm = core.T.PageMM() 145 } 146 fs.CSM.Reg(ct.DsortFileType, &ct.DsortFile{}) 147 fs.CSM.Reg(ct.DsortWorkfileType, &ct.DsortFile{}) 148 149 newBcastClient(config) 150 } 151 152 func newBcastClient(config *cmn.Config) { 153 cargs := cmn.TransportArgs{Timeout: config.Timeout.MaxHostBusy.D()} 154 if config.Net.HTTP.UseHTTPS { 155 bcastClient = cmn.NewIntraClientTLS(cargs, config) 156 } else { 157 bcastClient = cmn.NewClient(cargs) 158 } 159 } 160 161 ///////////// 162 // Manager // 163 ///////////// 164 165 func (m *Manager) String() string { return m.ManagerUUID } 166 func (m *Manager) lock() { m.mu.Lock() } 167 func (m *Manager) unlock() { m.mu.Unlock() } 168 169 // init initializes all necessary fields. 170 // PRECONDITION: `m.mu` must be locked. 171 func (m *Manager) init(pars *parsedReqSpec) error { 172 m.smap = core.T.Sowner().Get() 173 174 targetCount := m.smap.CountActiveTs() 175 176 m.Pars = pars 177 m.Metrics = newMetrics(pars.Description) 178 m.startShardCreation = make(chan struct{}, 1) 179 180 if err := m.setDsorter(); err != nil { 181 return err 182 } 183 184 if err := m.dsorter.init(); err != nil { 185 return err 186 } 187 188 if err := m.setRW(); err != nil { 189 return err 190 } 191 192 // NOTE: Total size of the records metadata can sometimes be large 193 // and so this is why we need such a long timeout. 194 m.config = cmn.GCO.Get() 195 196 cargs := cmn.TransportArgs{ 197 DialTimeout: m.config.Client.Timeout.D(), 198 Timeout: m.config.Client.TimeoutLong.D(), 199 } 200 if m.config.Net.HTTP.UseHTTPS { 201 m.client = cmn.NewIntraClientTLS(cargs, m.config) 202 } else { 203 m.client = cmn.NewClient(cargs) 204 } 205 206 m.received.ch = make(chan int32, 10) 207 208 m.compression.totalShardSize.Store(1) 209 m.compression.totalExtractedSize.Store(1) 210 211 // Concurrency: 212 // Number of goroutines should be larger than number of concurrency limit 213 // but it should not be: 214 // * too small - we don't want to artificially bottleneck the phases. 215 // * too large - we don't want too much goroutines in the system, it can cause 216 // too much overhead on context switching and managing the goroutines. 217 // Also for large workloads goroutines can take a lot of memory. 218 // 219 // Coefficient for extraction should be larger and depends on target count 220 // because we will skip a lot shards (which do not belong to us). 221 m.extractionPhase.adjuster = newConcAdjuster( 222 pars.ExtractConcMaxLimit, 223 2*targetCount, /*goroutineLimitCoef*/ 224 ) 225 226 // Fill ack map with current daemons. Once the finished ack is received from 227 // another daemon we will remove it from the map until len(ack) == 0 (then 228 // we will know that all daemons have finished operation). 229 m.finishedAck.m = make(map[string]struct{}, targetCount) 230 for sid, si := range m.smap.Tmap { 231 if m.smap.InMaintOrDecomm(si) { 232 continue 233 } 234 m.finishedAck.m[sid] = struct{}{} 235 } 236 237 m.setInProgressTo(true) 238 m.setAbortedTo(false) 239 m.state.cleanWait = sync.NewCond(&m.mu) 240 241 m.callTimeout = m.config.Dsort.CallTimeout.D() 242 return nil 243 } 244 245 // TODO -- FIXME: create on demand and reuse streams across jobs 246 // (in re: global rebalance and EC) 247 func (m *Manager) initStreams() error { 248 config := cmn.GCO.Get() 249 250 // Responses to the other targets are objects that is why we want to use 251 // intraData network. 252 respNetwork := cmn.NetIntraData 253 trname := fmt.Sprintf(shardStreamNameFmt, m.ManagerUUID) 254 shardsSbArgs := bundle.Args{ 255 Multiplier: config.Dsort.SbundleMult, 256 Net: respNetwork, 257 Trname: trname, 258 Ntype: core.Targets, 259 Extra: &transport.Extra{ 260 Compression: config.Dsort.Compression, 261 Config: config, 262 WorkChBurst: 1024, 263 }, 264 } 265 if err := transport.Handle(trname, m.recvShard); err != nil { 266 return errors.WithStack(err) 267 } 268 client := transport.NewIntraDataClient() 269 m.streams.shards = bundle.New(client, shardsSbArgs) 270 return nil 271 } 272 273 func (m *Manager) cleanupStreams() (err error) { 274 if m.streams.shards != nil { 275 trname := fmt.Sprintf(shardStreamNameFmt, m.ManagerUUID) 276 if unhandleErr := transport.Unhandle(trname); unhandleErr != nil { 277 err = errors.WithStack(unhandleErr) 278 } 279 } 280 281 for _, streamBundle := range []*bundle.Streams{m.streams.shards} { 282 if streamBundle != nil { 283 // NOTE: We don't want stream to send a message at this point as the 284 // receiver might have closed its corresponding stream. 285 streamBundle.Close(false /*gracefully*/) 286 } 287 } 288 289 return err 290 } 291 292 // cleanup removes all memory allocated and removes all files created during sort run. 293 // PRECONDITION: manager must be not in progress state (either actual finish or abort). 294 // NOTE: If cleanup is invoked during the run it is treated as abort. 295 func (m *Manager) cleanup() { 296 nlog.Infof("[dsort] %s started cleanup", m.ManagerUUID) 297 m.lock() 298 if m.state.cleaned != noCleanedState { 299 m.unlock() 300 return // Do not clean if already scheduled. 301 } 302 303 m.dsorter.cleanup() 304 now := time.Now() 305 306 defer func() { 307 m.state.cleaned = initiallyCleanedState 308 m.state.cleanWait.Signal() 309 m.unlock() 310 nlog.Infof("[dsort] %s finished cleanup in %v", m.ManagerUUID, time.Since(now)) 311 }() 312 313 debug.Assertf(!m.inProgress(), "%s: was still in progress", m.ManagerUUID) 314 315 m.shardRW = nil 316 m.client = nil 317 318 if !m.aborted() { 319 m.updateFinishedAck(core.T.SID()) 320 m.xctn.Finish() 321 } 322 } 323 324 // finalCleanup is invoked only when all targets confirm finishing. 325 // To ensure that finalCleanup is not invoked before regular 326 // cleanup is finished, we also ack ourselves. 327 // 328 // finalCleanup can be invoked only after cleanup and this is ensured by 329 // maintaining current state of the cleanliness and having conditional variable 330 // on which finalCleanup will sleep if needed. Note that it is hard (or even 331 // impossible) to ensure that cleanup and finalCleanup will be invoked in order 332 // without having ordering mechanism since cleanup and finalCleanup are invoked 333 // in goroutines (there is a possibility that finalCleanup would start before 334 // cleanup) - this cannot happen with current ordering mechanism. 335 func (m *Manager) finalCleanup() { 336 nlog.Infof("%s: [dsort] %s started final cleanup", core.T, m.ManagerUUID) 337 338 m.lock() 339 for m.state.cleaned != initiallyCleanedState { 340 if m.state.cleaned == finallyCleanedState { 341 m.unlock() 342 return // Do not clean if already cleaned. 343 } 344 if m.state.cleaned == noCleanedState { 345 // Wait for wake up from `cleanup` or other `finalCleanup` method. 346 m.state.cleanWait.Wait() 347 } 348 } 349 350 now := time.Now() 351 352 if err := m.cleanupStreams(); err != nil { 353 nlog.Errorln(err) 354 } 355 356 // Wait for all in-flight stream requests after cleaning up streams. 357 m.waitForInFlight() 358 359 if err := m.dsorter.finalCleanup(); err != nil { 360 nlog.Errorln(err) 361 } 362 363 // The reason why this is not in regular cleanup is because we are only sure 364 // that this can be freed once we cleanup streams - streams are asynchronous 365 // and we may have race between in-flight request and cleanup. 366 // Also, NOTE: 367 // recm.Cleanup => gmm.freeMemToOS => cos.FreeMemToOS to forcefully free memory to the OS 368 m.recm.Cleanup() 369 370 m.creationPhase.metadata.SendOrder = nil 371 m.creationPhase.metadata.Shards = nil 372 373 m.finishedAck.m = nil 374 375 // Update clean state. 376 m.state.cleaned = finallyCleanedState 377 // If there is another `finalCleanup` waiting it should be woken up to check the state and exit. 378 m.state.cleanWait.Signal() 379 m.unlock() 380 381 m.mg.persist(m.ManagerUUID) 382 nlog.Infof("%s: [dsort] %s finished final cleanup in %v", core.T, m.ManagerUUID, time.Since(now)) 383 } 384 385 // stop this job and free associated resources 386 func (m *Manager) abort(err error) { 387 if m.aborted() { // do not abort if already aborted 388 return 389 } 390 // serialize 391 m.lock() 392 if m.aborted() { // do not abort if already aborted 393 m.unlock() 394 return 395 } 396 if err != nil { 397 m.Metrics.lock() 398 m.Metrics.Errors = append(m.Metrics.Errors, err.Error()) 399 m.Metrics.unlock() 400 } 401 m.setAbortedTo(true) 402 m.xctn.Base.Abort(err) // notice Base, compare w/ xaction.Abort (xact.go) 403 inProgress := m.inProgress() 404 m.unlock() 405 406 nlog.Infof("%s: [dsort] %s aborted", core.T, m.ManagerUUID) 407 408 // If job has already finished we just free resources, otherwise we must wait 409 // for it to finish. 410 if inProgress { 411 if cmn.Rom.FastV(4, cos.SmoduleDsort) { 412 nlog.Infof("[dsort] %s is in progress, waiting for finish", m.ManagerUUID) 413 } 414 // Wait for dsorter to initialize all the resources. 415 m.waitToStart() 416 417 m.dsorter.onAbort() 418 m.waitForFinish() 419 if cmn.Rom.FastV(4, cos.SmoduleDsort) { 420 nlog.Infof("[dsort] %s was in progress and finished", m.ManagerUUID) 421 } 422 } 423 424 go func() { 425 m.cleanup() 426 m.finalCleanup() // on abort always perform final cleanup 427 }() 428 } 429 430 // setDsorter sets what type of dsorter implementation should be used 431 func (m *Manager) setDsorter() (err error) { 432 switch m.Pars.DsorterType { 433 case GeneralType: 434 m.dsorter, err = newDsorterGeneral(m) 435 case MemType: 436 m.dsorter = newDsorterMem(m) 437 default: 438 debug.Assertf(false, "dsorter type is invalid: %q", m.Pars.DsorterType) 439 } 440 m.dsorterStarted.Add(1) 441 return 442 } 443 444 func (m *Manager) markStarted() { m.dsorterStarted.Done() } 445 func (m *Manager) waitToStart() { m.dsorterStarted.Wait() } 446 func (m *Manager) onDupRecs(msg string) error { return m.react(m.Pars.DuplicatedRecords, msg) } 447 448 // setRW sets what type of file extraction and creation is used based on the RequestSpec. 449 func (m *Manager) setRW() (err error) { 450 var ke shard.KeyExtractor 451 switch m.Pars.Algorithm.Kind { 452 case Content: 453 ke, err = shard.NewContentKeyExtractor(m.Pars.Algorithm.ContentKeyType, m.Pars.Algorithm.Ext) 454 case MD5: 455 ke, err = shard.NewMD5KeyExtractor() 456 default: 457 ke, err = shard.NewNameKeyExtractor() 458 } 459 if err != nil { 460 return errors.WithStack(err) 461 } 462 463 m.shardRW = shard.RWs[m.Pars.InputExtension] 464 if m.shardRW == nil { 465 debug.Assert(!m.Pars.DryRun, "dry-run in combination with _any_ shard extension is not supported") 466 debug.Assert(m.Pars.InputExtension == "", m.Pars.InputExtension) 467 // TODO -- FIXME: niy 468 } 469 if m.Pars.DryRun { 470 m.shardRW = shard.NopRW(m.shardRW) 471 } 472 473 m.recm = shard.NewRecordManager(m.Pars.InputBck, m.shardRW, ke, m.onDupRecs) 474 return nil 475 } 476 477 // updateFinishedAck marks tid as finished. If all daemons ack then the 478 // finalCleanup is dispatched in separate goroutine. 479 func (m *Manager) updateFinishedAck(tid string) { 480 m.finishedAck.mu.Lock() 481 delete(m.finishedAck.m, tid) 482 if len(m.finishedAck.m) == 0 { 483 go m.finalCleanup() 484 } 485 m.finishedAck.mu.Unlock() 486 } 487 488 // incrementReceived increments number of received records batches. Also puts 489 // the information in the channel so other waiting goroutine can be informed 490 // that the information has been updated. 491 func (m *Manager) incrementReceived() { 492 m.received.ch <- m.received.count.Inc() 493 } 494 495 // listenReceived returns channel on which goroutine can wait 496 // until received count value is updated (see: incrementReceived). 497 func (m *Manager) listenReceived() chan int32 { 498 return m.received.ch 499 } 500 501 func (m *Manager) addSizes(shardSize, extractedSize int64) { 502 if shardSize > extractedSize { 503 // .tar with padding or poor compression 504 shardSize = extractedSize 505 } 506 m.compression.totalShardSize.Add(shardSize) 507 m.compression.totalExtractedSize.Add(extractedSize) 508 } 509 510 func (m *Manager) totalShardSize() int64 { return m.compression.totalShardSize.Load() } 511 func (m *Manager) totalExtractedSize() int64 { return m.compression.totalExtractedSize.Load() } 512 513 func (m *Manager) compressionRatio() float64 { 514 return float64(m.totalShardSize()) / float64(m.totalExtractedSize()) 515 } 516 517 // incrementRef increments reference counter. This prevents from premature cleanup. 518 // Each increment should have corresponding decrement to prevent memory leaks. 519 // 520 // NOTE: Manager should increment ref every time some data of it is used, otherwise 521 // unexpected things can happen. 522 func (m *Manager) incrementRef(by int64) { 523 m.refCount.Add(by) 524 } 525 526 // decrementRef decrements reference counter. If it is 0 or below and dsort has 527 // already finished returns true. Otherwise, false is returned. 528 func (m *Manager) decrementRef(by int64) { 529 newRefCount := m.refCount.Sub(by) 530 if newRefCount <= 0 { 531 // When ref count is below zero or zero we should schedule cleanup 532 m.lock() 533 if !m.inProgress() { 534 m.unlock() 535 go m.cleanup() 536 return 537 } 538 m.unlock() 539 } 540 } 541 542 func (m *Manager) inFlightInc() { m.inFlight.Inc() } 543 func (m *Manager) inFlightDec() { m.inFlight.Dec() } 544 func (m *Manager) inProgress() bool { return m.state.inProgress.Load() } 545 func (m *Manager) aborted() bool { return m.state.aborted.Load() } 546 547 // listenAborted returns channel which is closed when Dsort job was aborted. 548 // This allows for the listen to be notified when job is aborted. 549 func (m *Manager) listenAborted() chan struct{} { 550 return m.state.doneCh 551 } 552 553 // waitForFinish waits for Dsort job to be finished. Note that aborted is also 554 // 'finished'. 555 func (m *Manager) waitForFinish() { 556 m.state.wg.Wait() 557 } 558 559 // waitForInFlight waits for all in-flight stream requests to finish. 560 func (m *Manager) waitForInFlight() { 561 for m.inFlight.Load() > 0 { 562 time.Sleep(200 * time.Millisecond) 563 } 564 } 565 566 // setInProgressTo updates in progress state. If inProgress is set to false and 567 // sort was aborted this means someone is waiting. Therefore the function is 568 // waking up everyone who is waiting. 569 // PRECONDITION: `m.mu` must be locked. 570 func (m *Manager) setInProgressTo(inProgress bool) { 571 // If marking as finished and job was aborted to need to free everyone 572 // who is waiting. 573 m.state.inProgress.Store(inProgress) 574 if !inProgress && m.aborted() { 575 m.state.wg.Done() 576 } 577 } 578 579 // setAbortedTo updates aborted state. If aborted is set to true and sort is not 580 // yet finished. We need to inform current phase about abort (closing channel) 581 // and mark that we will wait until it is finished. 582 // 583 // PRECONDITION: `m.mu` must be locked. 584 func (m *Manager) setAbortedTo(aborted bool) { 585 if aborted { 586 // If not finished and not yet aborted we should mark that we will wait. 587 if m.inProgress() && !m.aborted() { 588 close(m.state.doneCh) 589 m.state.wg.Add(1) 590 } 591 } else { 592 // This is invoked when starting - on start doneCh should be open and 593 // closed when aborted. wg is used to keep all waiting process on finish. 594 m.state.doneCh = make(chan struct{}) 595 m.state.wg = &sync.WaitGroup{} 596 } 597 m.state.aborted.Store(aborted) 598 m.Metrics.setAbortedTo(aborted) 599 } 600 601 func (m *Manager) recvShard(hdr *transport.ObjHdr, objReader io.Reader, err error) error { 602 defer transport.DrainAndFreeReader(objReader) 603 if err != nil { 604 m.abort(err) 605 return err 606 } 607 if m.aborted() { 608 return m.newErrAborted() 609 } 610 lom := core.AllocLOM(hdr.ObjName) 611 defer core.FreeLOM(lom) 612 if err = lom.InitBck(&hdr.Bck); err == nil { 613 err = lom.Load(false /*cache it*/, false /*locked*/) 614 } 615 if err != nil && !os.IsNotExist(err) { 616 m.abort(err) 617 return err 618 } 619 if err == nil { 620 if lom.EqCksum(hdr.ObjAttrs.Cksum) { 621 if cmn.Rom.FastV(4, cos.SmoduleDsort) { 622 nlog.Infof("[dsort] %s shard (%s) already exists and checksums are equal, skipping", 623 m.ManagerUUID, lom) 624 } 625 return nil 626 } 627 nlog.Warningf("[dsort] %s shard (%s) already exists, overriding", m.ManagerUUID, lom) 628 } 629 started := time.Now() 630 lom.SetAtimeUnix(started.UnixNano()) 631 rc := io.NopCloser(objReader) 632 633 params := core.AllocPutParams() 634 { 635 params.WorkTag = ct.WorkfileRecvShard 636 params.Reader = rc 637 params.Cksum = nil 638 params.Atime = started 639 params.Size = hdr.ObjAttrs.Size 640 } 641 erp := core.T.PutObject(lom, params) 642 core.FreePutParams(params) 643 if erp != nil { 644 m.abort(err) 645 return erp 646 } 647 return nil 648 } 649 650 func (m *Manager) freeMemory() uint64 { 651 var mem sys.MemStat 652 if err := mem.Get(); err != nil { 653 return 0 654 } 655 maxMemoryToUse := calcMaxMemoryUsage(m.Pars.MaxMemUsage, &mem) 656 return maxMemoryToUse - mem.ActualUsed 657 } 658 659 func (m *Manager) react(reaction, msg string) error { 660 switch reaction { 661 case cmn.IgnoreReaction: 662 return nil 663 case cmn.WarnReaction: 664 m.Metrics.lock() 665 m.Metrics.Warnings = append(m.Metrics.Warnings, msg) 666 m.Metrics.unlock() 667 return nil 668 case cmn.AbortReaction: 669 return fmt.Errorf("%s", msg) // (dsort job aborts and returns this error) 670 default: 671 debug.Assert(false, reaction) 672 return nil 673 } 674 } 675 676 func calcMaxMemoryUsage(maxUsage cos.ParsedQuantity, mem *sys.MemStat) uint64 { 677 switch maxUsage.Type { 678 case cos.QuantityPercent: 679 return maxUsage.Value * (mem.Total / 100) 680 case cos.QuantityBytes: 681 return min(maxUsage.Value, mem.Total) 682 default: 683 debug.Assertf(false, "mem usage type (%s) is not recognized.. something went wrong", maxUsage.Type) 684 return 0 685 } 686 } 687 688 /////////////////////// 689 // buildingShardInfo // 690 /////////////////////// 691 692 func (bsi *buildingShardInfo) Unpack(unpacker *cos.ByteUnpack) error { 693 var err error 694 bsi.shardName, err = unpacker.ReadString() 695 return err 696 } 697 698 func (bsi *buildingShardInfo) Pack(packer *cos.BytePack) { packer.WriteString(bsi.shardName) } 699 func (bsi *buildingShardInfo) PackedSize() int { return cos.SizeofLen + len(bsi.shardName) } 700 func (bsi *buildingShardInfo) NewPack(mm *memsys.MMSA) []byte { 701 var ( 702 size = bsi.PackedSize() 703 buf, _ = mm.AllocSize(int64(size)) 704 packer = cos.NewPacker(buf, size) 705 ) 706 packer.WriteAny(bsi) 707 return packer.Bytes() 708 }