github.com/NVIDIA/aistore@v1.3.23-0.20240517131212-7df6609be51d/ext/dsort/dsort_general.go (about) 1 // Package dsort provides distributed massively parallel resharding for very large datasets. 2 /* 3 * Copyright (c) 2018-2023, NVIDIA CORPORATION. All rights reserved. 4 */ 5 package dsort 6 7 import ( 8 "context" 9 "fmt" 10 "io" 11 "os" 12 "sync" 13 14 "github.com/NVIDIA/aistore/api/apc" 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/mono" 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/shard" 24 "github.com/NVIDIA/aistore/fs" 25 "github.com/NVIDIA/aistore/memsys" 26 "github.com/NVIDIA/aistore/stats" 27 "github.com/NVIDIA/aistore/sys" 28 "github.com/NVIDIA/aistore/transport" 29 "github.com/NVIDIA/aistore/transport/bundle" 30 jsoniter "github.com/json-iterator/go" 31 "github.com/pkg/errors" 32 "golang.org/x/sync/errgroup" 33 ) 34 35 // This is general implementation of dsort, for all types of workloads. 36 // Historically, it is the first implementation of dsorter ever created. 37 // It tries to use memory in extraction phase to pull as most of the data 38 // into memory as possible. This way, later, dsorter will use it later in 39 // creation phase. It means that if the data takes less space than the available 40 // memory in the cluster, we can extract whole data into memory and make 41 // the creation phase super fast. 42 43 const ( 44 GeneralType = "dsort_general" 45 ) 46 47 type ( 48 dsorterGeneral struct { 49 m *Manager 50 mw *memoryWatcher 51 52 streams struct { 53 cleanupDone atomic.Bool 54 request *bundle.Streams 55 response *bundle.Streams 56 } 57 58 creationPhase struct { 59 adjuster *concAdjuster 60 61 streamWriters struct { 62 mu sync.Mutex 63 writers map[string]*streamWriter 64 } 65 } 66 } 67 68 streamWriter struct { 69 w io.Writer 70 n int64 71 err error 72 wg *cos.TimeoutGroup 73 } 74 75 remoteRequest struct { 76 Record *shard.Record `json:"r"` 77 RecordObj *shard.RecordObj `json:"o"` 78 } 79 ) 80 81 // interface guard 82 var _ dsorter = (*dsorterGeneral)(nil) 83 84 func newDsorterGeneral(m *Manager) (*dsorterGeneral, error) { 85 var mem sys.MemStat 86 if err := mem.Get(); err != nil { 87 return nil, err 88 } 89 maxMemoryToUse := calcMaxMemoryUsage(m.Pars.MaxMemUsage, &mem) 90 ds := &dsorterGeneral{ 91 m: m, 92 mw: newMemoryWatcher(m, maxMemoryToUse), 93 } 94 ds.creationPhase.streamWriters.writers = make(map[string]*streamWriter, 10000) 95 return ds, nil 96 } 97 98 func (ds *dsorterGeneral) newStreamWriter(pathToContents string, w io.Writer) *streamWriter { 99 writer := &streamWriter{ 100 w: w, 101 wg: cos.NewTimeoutGroup(), 102 } 103 writer.wg.Add(1) 104 ds.creationPhase.streamWriters.mu.Lock() 105 ds.creationPhase.streamWriters.writers[pathToContents] = writer 106 ds.creationPhase.streamWriters.mu.Unlock() 107 return writer 108 } 109 110 func (ds *dsorterGeneral) pullStreamWriter(objName string) *streamWriter { 111 ds.creationPhase.streamWriters.mu.Lock() 112 writer := ds.creationPhase.streamWriters.writers[objName] 113 delete(ds.creationPhase.streamWriters.writers, objName) 114 ds.creationPhase.streamWriters.mu.Unlock() 115 return writer 116 } 117 118 func (*dsorterGeneral) name() string { return GeneralType } 119 120 func (ds *dsorterGeneral) init() error { 121 ds.creationPhase.adjuster = newConcAdjuster( 122 ds.m.Pars.CreateConcMaxLimit, 123 1, /*goroutineLimitCoef*/ 124 ) 125 return nil 126 } 127 128 func (ds *dsorterGeneral) start() error { 129 // Requests are usually small packets, no more 1KB that is why we want to 130 // utilize intraControl network. 131 config := cmn.GCO.Get() 132 reqNetwork := cmn.NetIntraControl 133 // Responses to the other targets are objects that is why we want to use 134 // intraData network. 135 respNetwork := cmn.NetIntraData 136 137 client := transport.NewIntraDataClient() 138 139 trname := fmt.Sprintf(recvReqStreamNameFmt, ds.m.ManagerUUID) 140 reqSbArgs := bundle.Args{ 141 Multiplier: ds.m.Pars.SbundleMult, 142 Net: reqNetwork, 143 Trname: trname, 144 Ntype: core.Targets, 145 Extra: &transport.Extra{ 146 Config: config, 147 }, 148 } 149 if err := transport.Handle(trname, ds.recvReq); err != nil { 150 return errors.WithStack(err) 151 } 152 153 trname = fmt.Sprintf(recvRespStreamNameFmt, ds.m.ManagerUUID) 154 respSbArgs := bundle.Args{ 155 Multiplier: ds.m.Pars.SbundleMult, 156 Net: respNetwork, 157 Trname: trname, 158 Ntype: core.Targets, 159 Extra: &transport.Extra{ 160 Compression: config.Dsort.Compression, 161 Config: config, 162 }, 163 } 164 if err := transport.Handle(trname, ds.recvResp); err != nil { 165 return errors.WithStack(err) 166 } 167 168 ds.streams.request = bundle.New(client, reqSbArgs) 169 ds.streams.response = bundle.New(client, respSbArgs) 170 171 // start watching memory 172 return ds.mw.watch() 173 } 174 175 func (ds *dsorterGeneral) cleanupStreams() (err error) { 176 if !ds.streams.cleanupDone.CAS(false, true) { 177 return nil 178 } 179 180 if ds.streams.request != nil { 181 trname := fmt.Sprintf(recvReqStreamNameFmt, ds.m.ManagerUUID) 182 if unhandleErr := transport.Unhandle(trname); unhandleErr != nil { 183 err = errors.WithStack(unhandleErr) 184 } 185 } 186 187 if ds.streams.response != nil { 188 trname := fmt.Sprintf(recvRespStreamNameFmt, ds.m.ManagerUUID) 189 if unhandleErr := transport.Unhandle(trname); unhandleErr != nil { 190 err = errors.WithStack(unhandleErr) 191 } 192 } 193 194 for _, streamBundle := range []*bundle.Streams{ds.streams.request, ds.streams.response} { 195 if streamBundle != nil { 196 // NOTE: We don't want stream to send a message at this point as the 197 // receiver might have closed its corresponding stream. 198 streamBundle.Close(false /*gracefully*/) 199 } 200 } 201 202 return err 203 } 204 205 func (ds *dsorterGeneral) cleanup() { 206 ds.mw.stop() 207 } 208 209 func (ds *dsorterGeneral) finalCleanup() error { 210 return ds.cleanupStreams() 211 } 212 213 func (ds *dsorterGeneral) postRecordDistribution() { 214 // In shard creation we should not expect memory increase (at least 215 // not from dsort). Also it would be really hard to have concurrent 216 // sends and memory cleanup. We must stop before sending records 217 // because it affects content of the records. 218 ds.mw.stopWatchingExcess() 219 } 220 221 // createShardsLocally waits until it's given the signal to start creating 222 // shards, then creates shards in parallel. 223 func (ds *dsorterGeneral) createShardsLocally() (err error) { 224 phaseInfo := &ds.m.creationPhase 225 226 ds.creationPhase.adjuster.start() 227 defer ds.creationPhase.adjuster.stop() 228 229 metrics := ds.m.Metrics.Creation 230 metrics.begin() 231 defer metrics.finish() 232 metrics.mu.Lock() 233 metrics.ToCreate = int64(len(phaseInfo.metadata.Shards)) 234 metrics.mu.Unlock() 235 236 group, ctx := errgroup.WithContext(context.Background()) 237 238 outer: 239 for _, s := range phaseInfo.metadata.Shards { 240 select { 241 case <-ds.m.listenAborted(): 242 _ = group.Wait() 243 return ds.m.newErrAborted() 244 case <-ctx.Done(): 245 break outer // context was canceled, therefore we have an error 246 default: 247 } 248 249 ds.creationPhase.adjuster.acquireGoroutineSema() 250 cs := &dsgCreateShard{ds, s} 251 group.Go(cs.do) 252 } 253 254 return group.Wait() 255 } 256 257 func (ds *dsorterGeneral) preShardCreation(_ string, mi *fs.Mountpath) error { 258 ds.creationPhase.adjuster.acquireSema(mi) 259 return nil 260 } 261 262 func (ds *dsorterGeneral) postShardCreation(mi *fs.Mountpath) { 263 ds.creationPhase.adjuster.releaseSema(mi) 264 } 265 266 // loads content from disk or memory, local or remote 267 func (ds *dsorterGeneral) Load(w io.Writer, rec *shard.Record, obj *shard.RecordObj) (int64, error) { 268 if ds.m.aborted() { 269 return 0, ds.m.newErrAborted() 270 } 271 if rec.DaemonID != core.T.SID() { 272 return ds.loadRemote(w, rec, obj) 273 } 274 return ds.loadLocal(w, obj) 275 } 276 277 func (ds *dsorterGeneral) loadLocal(w io.Writer, obj *shard.RecordObj) (written int64, err error) { 278 var ( 279 slab *memsys.Slab 280 buf []byte 281 storeType = obj.StoreType 282 ) 283 284 if storeType != shard.SGLStoreType { // SGL does not need buffer as it is buffer itself 285 buf, slab = g.mm.AllocSize(obj.Size) 286 } 287 288 defer func() { 289 if storeType != shard.SGLStoreType { 290 slab.Free(buf) 291 } 292 ds.m.decrementRef(1) 293 }() 294 295 fullContentPath := ds.m.recm.FullContentPath(obj) 296 297 if ds.m.Pars.DryRun { 298 r := cos.NopReader(obj.MetadataSize + obj.Size) 299 written, err = io.CopyBuffer(w, r, buf) 300 return 301 } 302 303 var n int64 304 switch storeType { 305 case shard.OffsetStoreType: 306 f, err := os.Open(fullContentPath) // TODO: it should be open always 307 if err != nil { 308 return written, errors.WithMessage(err, "(offset) open local content failed") 309 } 310 defer cos.Close(f) 311 _, err = f.Seek(obj.Offset-obj.MetadataSize, io.SeekStart) 312 if err != nil { 313 return written, errors.WithMessage(err, "(offset) seek local content failed") 314 } 315 if n, err = io.CopyBuffer(w, io.LimitReader(f, obj.MetadataSize+obj.Size), buf); err != nil { 316 return written, errors.WithMessage(err, "(offset) copy local content failed") 317 } 318 case shard.SGLStoreType: 319 debug.Assert(buf == nil) 320 v, ok := ds.m.recm.RecordContents().Load(fullContentPath) 321 debug.Assert(ok, fullContentPath) 322 ds.m.recm.RecordContents().Delete(fullContentPath) 323 sgl := v.(*memsys.SGL) 324 defer sgl.Free() 325 326 // No need for `io.CopyBuffer` since SGL implements `io.WriterTo`. 327 if n, err = io.Copy(w, sgl); err != nil { 328 return written, errors.WithMessage(err, "(sgl) copy local content failed") 329 } 330 case shard.DiskStoreType: 331 f, err := os.Open(fullContentPath) 332 if err != nil { 333 return written, errors.WithMessage(err, "(disk) open local content failed") 334 } 335 defer cos.Close(f) 336 if n, err = io.CopyBuffer(w, f, buf); err != nil { 337 return written, errors.WithMessage(err, "(disk) copy local content failed") 338 } 339 default: 340 debug.Assert(false, storeType) 341 } 342 343 debug.Assert(n > 0) 344 written += n 345 return 346 } 347 348 func (ds *dsorterGeneral) loadRemote(w io.Writer, rec *shard.Record, obj *shard.RecordObj) (int64, error) { 349 var ( 350 tid = rec.DaemonID 351 tsi = ds.m.smap.GetTarget(tid) 352 writer = ds.newStreamWriter(rec.MakeUniqueName(obj), w) 353 ) 354 if tsi == nil { 355 return 0, errors.Errorf("cannot send request to node %q - not present in %s", tid, ds.m.smap) 356 } 357 req := remoteRequest{ 358 Record: rec, 359 RecordObj: obj, 360 } 361 opaque := cos.MustMarshal(req) 362 o := transport.AllocSend() 363 o.Hdr = transport.ObjHdr{Opaque: opaque} 364 o.Callback, o.CmplArg = ds.sentCallback, &req 365 366 if err := ds.streams.request.Send(o, nil, tsi); err != nil { 367 return 0, errors.WithStack(err) 368 } 369 370 // May happen that the target we are trying to contact was 371 // aborted or for some reason is not responding. Thus we need to do 372 // some precaution and wait for the content only for limited time or 373 // until we receive abort signal. 374 var ( 375 beforeRecv = mono.NanoTime() 376 pulled bool 377 ) 378 timed, stopped := writer.wg.WaitTimeoutWithStop(ds.m.callTimeout, ds.m.listenAborted()) 379 if timed || stopped { 380 // In case of timeout or abort we need to pull the writer to 381 // avoid concurrent Close and Write on `writer.w`. 382 pulled = ds.pullStreamWriter(rec.MakeUniqueName(obj)) != nil 383 } else { 384 // stats 385 delta := mono.Since(beforeRecv) 386 g.tstats.AddMany( 387 cos.NamedVal64{Name: stats.DsortCreationRespCount, Value: 1}, 388 cos.NamedVal64{Name: stats.DsortCreationRespLatency, Value: int64(delta)}, 389 ) 390 } 391 392 // If we timed out or were stopped but failed to pull the 393 // writer then someone else should've done it and we barely 394 // missed. In this case we should wait for the job to finish 395 // (when stopped, we should receive an error anyway). 396 397 if pulled { // managed to pull the writer, can safely return error 398 var err error 399 switch { 400 case stopped: 401 err = cmn.NewErrAborted("wait for remote content", "", nil) 402 case timed: 403 err = errors.Errorf("wait for remote content timed out (%q was waiting for %q)", core.T.SID(), tid) 404 default: 405 debug.Assert(false, "pulled but not stopped or timed?") 406 } 407 return 0, err 408 } 409 410 if timed || stopped { 411 writer.wg.Wait() 412 } 413 414 return writer.n, writer.err 415 } 416 417 func (ds *dsorterGeneral) sentCallback(_ *transport.ObjHdr, _ io.ReadCloser, arg any, err error) { 418 if err == nil { 419 g.tstats.Add(stats.DsortCreationReqCount, 1) 420 return 421 } 422 req := arg.(*remoteRequest) 423 nlog.Errorf("%s: [dsort] %s failed to send remore-req %s: %v", 424 core.T, ds.m.ManagerUUID, req.Record.MakeUniqueName(req.RecordObj), err) 425 } 426 427 func (ds *dsorterGeneral) errHandler(err error, node *meta.Snode, o *transport.Obj) { 428 *o = transport.Obj{Hdr: o.Hdr} 429 o.Hdr.Opaque = []byte(err.Error()) 430 o.Hdr.ObjAttrs.Size = 0 431 if err = ds.streams.response.Send(o, nil, node); err != nil { 432 ds.m.abort(err) 433 } 434 } 435 436 // implements receiver i/f 437 func (ds *dsorterGeneral) recvReq(hdr *transport.ObjHdr, objReader io.Reader, err error) error { 438 ds.m.inFlightInc() 439 defer func() { 440 ds.m.inFlightDec() 441 transport.FreeRecv(objReader) 442 }() 443 req := remoteRequest{} 444 if err := jsoniter.Unmarshal(hdr.Opaque, &req); err != nil { 445 err := fmt.Errorf(cmn.FmtErrUnmarshal, apc.ActDsort, "recv request", cos.BHead(hdr.Opaque), err) 446 ds.m.abort(err) 447 return err 448 } 449 450 fromNode := ds.m.smap.GetTarget(hdr.SID) 451 if fromNode == nil { 452 err := fmt.Errorf("received request (%v) from %q not present in the %s", req.Record, hdr.SID, ds.m.smap) 453 return err 454 } 455 456 if err != nil { 457 ds.errHandler(err, fromNode, &transport.Obj{Hdr: *hdr}) 458 return err 459 } 460 461 if ds.m.aborted() { 462 return ds.m.newErrAborted() 463 } 464 465 o := transport.AllocSend() 466 o.Hdr = transport.ObjHdr{ObjName: req.Record.MakeUniqueName(req.RecordObj)} 467 o.Callback = ds.responseCallback 468 469 fullContentPath := ds.m.recm.FullContentPath(req.RecordObj) 470 471 if ds.m.Pars.DryRun { 472 lr := cos.NopReader(req.RecordObj.MetadataSize + req.RecordObj.Size) 473 r := cos.NopOpener(io.NopCloser(lr)) 474 o.Hdr.ObjAttrs.Size = req.RecordObj.MetadataSize + req.RecordObj.Size 475 ds.streams.response.Send(o, r, fromNode) 476 return nil 477 } 478 479 switch req.RecordObj.StoreType { 480 case shard.OffsetStoreType: 481 o.Hdr.ObjAttrs.Size = req.RecordObj.MetadataSize + req.RecordObj.Size 482 offset := req.RecordObj.Offset - req.RecordObj.MetadataSize 483 r, err := cos.NewFileSectionHandle(fullContentPath, offset, o.Hdr.ObjAttrs.Size) 484 if err != nil { 485 ds.errHandler(err, fromNode, o) 486 return err 487 } 488 ds.streams.response.Send(o, r, fromNode) 489 case shard.SGLStoreType: 490 v, ok := ds.m.recm.RecordContents().Load(fullContentPath) 491 debug.Assert(ok, fullContentPath) 492 ds.m.recm.RecordContents().Delete(fullContentPath) 493 sgl := v.(*memsys.SGL) 494 o.Hdr.ObjAttrs.Size = sgl.Size() 495 ds.streams.response.Send(o, sgl, fromNode) 496 case shard.DiskStoreType: 497 f, err := cos.NewFileHandle(fullContentPath) 498 if err != nil { 499 ds.errHandler(err, fromNode, o) 500 return err 501 } 502 fi, err := f.Stat() 503 if err != nil { 504 cos.Close(f) 505 ds.errHandler(err, fromNode, o) 506 return err 507 } 508 o.Hdr.ObjAttrs.Size = fi.Size() 509 ds.streams.response.Send(o, f, fromNode) 510 default: 511 debug.Assert(false) 512 } 513 return nil 514 } 515 516 func (ds *dsorterGeneral) responseCallback(hdr *transport.ObjHdr, rc io.ReadCloser, _ any, err error) { 517 if sgl, ok := rc.(*memsys.SGL); ok { 518 sgl.Free() 519 } 520 ds.m.decrementRef(1) 521 if err != nil { 522 nlog.Errorf("%s: [dsort] %s failed to send rsp %s (size %d): %v - aborting...", 523 core.T, ds.m.ManagerUUID, hdr.ObjName, hdr.ObjAttrs.Size, err) 524 ds.m.abort(err) 525 } 526 } 527 528 func (ds *dsorterGeneral) postExtraction() { 529 ds.mw.stopWatchingReserved() 530 } 531 532 func (ds *dsorterGeneral) recvResp(hdr *transport.ObjHdr, object io.Reader, err error) error { 533 ds.m.inFlightInc() 534 defer func() { 535 transport.DrainAndFreeReader(object) 536 ds.m.inFlightDec() 537 }() 538 539 if err != nil { 540 ds.m.abort(err) 541 return err 542 } 543 544 if ds.m.aborted() { 545 return ds.m.newErrAborted() 546 } 547 548 writer := ds.pullStreamWriter(hdr.ObjName) 549 if writer == nil { // was removed after timing out 550 return nil 551 } 552 553 if len(hdr.Opaque) > 0 { 554 writer.n, writer.err = 0, errors.New(string(hdr.Opaque)) 555 writer.wg.Done() 556 return nil 557 } 558 559 buf, slab := g.mm.AllocSize(hdr.ObjAttrs.Size) 560 writer.n, writer.err = io.CopyBuffer(writer.w, object, buf) 561 writer.wg.Done() 562 slab.Free(buf) 563 564 return nil 565 } 566 567 func (ds *dsorterGeneral) preShardExtraction(expectedUncompressedSize uint64) bool { 568 return ds.mw.reserveMem(expectedUncompressedSize) 569 } 570 571 func (ds *dsorterGeneral) postShardExtraction(expectedUncompressedSize uint64) { 572 ds.mw.unreserveMem(expectedUncompressedSize) 573 } 574 575 func (ds *dsorterGeneral) onAbort() { 576 _ = ds.cleanupStreams() 577 } 578 579 //////////////////// 580 // dsgCreateShard // 581 //////////////////// 582 583 type dsgCreateShard struct { 584 ds *dsorterGeneral 585 shard *shard.Shard 586 } 587 588 func (cs *dsgCreateShard) do() (err error) { 589 lom := core.AllocLOM(cs.shard.Name) 590 err = cs.ds.m.createShard(cs.shard, lom) 591 core.FreeLOM(lom) 592 cs.ds.creationPhase.adjuster.releaseGoroutineSema() 593 return 594 }