github.com/NVIDIA/aistore@v1.3.23-0.20240517131212-7df6609be51d/ext/dsort/dsort.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  	"bufio"
     9  	"context"
    10  	"encoding/binary"
    11  	"fmt"
    12  	"io"
    13  	"math"
    14  	"net/http"
    15  	"net/url"
    16  	"path/filepath"
    17  	"sort"
    18  	"strconv"
    19  	"strings"
    20  	"sync"
    21  	"time"
    22  
    23  	"github.com/NVIDIA/aistore/api/apc"
    24  	"github.com/NVIDIA/aistore/cmn"
    25  	"github.com/NVIDIA/aistore/cmn/archive"
    26  	"github.com/NVIDIA/aistore/cmn/cos"
    27  	"github.com/NVIDIA/aistore/cmn/debug"
    28  	"github.com/NVIDIA/aistore/cmn/nlog"
    29  	"github.com/NVIDIA/aistore/core"
    30  	"github.com/NVIDIA/aistore/core/meta"
    31  	"github.com/NVIDIA/aistore/ext/dsort/shard"
    32  	"github.com/NVIDIA/aistore/fs"
    33  	"github.com/NVIDIA/aistore/stats"
    34  	"github.com/NVIDIA/aistore/transport"
    35  	"github.com/OneOfOne/xxhash"
    36  	jsoniter "github.com/json-iterator/go"
    37  	"github.com/pkg/errors"
    38  	"github.com/tinylib/msgp/msgp"
    39  	"golang.org/x/sync/errgroup"
    40  )
    41  
    42  const PrefixJobID = "srt-"
    43  
    44  type (
    45  	dsorter interface {
    46  		shard.ContentLoader
    47  
    48  		name() string
    49  		init() error
    50  		start() error
    51  		postExtraction()
    52  		postRecordDistribution()
    53  		createShardsLocally() (err error)
    54  		preShardCreation(shardName string, mi *fs.Mountpath) error
    55  		postShardCreation(mi *fs.Mountpath)
    56  		cleanup()
    57  		finalCleanup() error
    58  		preShardExtraction(expectedUncompressedSize uint64) (toDisk bool)
    59  		postShardExtraction(expectedUncompressedSize uint64)
    60  		onAbort()
    61  	}
    62  )
    63  
    64  var js = jsoniter.ConfigFastest
    65  
    66  func (m *Manager) finish() {
    67  	if cmn.Rom.FastV(4, cos.SmoduleDsort) {
    68  		nlog.Infof("%s: %s finished", core.T, m.ManagerUUID)
    69  	}
    70  	m.lock()
    71  	m.setInProgressTo(false)
    72  	m.unlock()
    73  
    74  	// Trigger decrement reference counter. If it is already 0 it will
    75  	// trigger cleanup because progress is set to false. Otherwise, the
    76  	// cleanup will be triggered by decrementRef in load content handlers.
    77  	m.decrementRef(0)
    78  }
    79  
    80  func (m *Manager) start() (err error) {
    81  	defer m.finish()
    82  
    83  	if err := m.startDsorter(); err != nil {
    84  		return err
    85  	}
    86  
    87  	// Phase 1.
    88  	nlog.Infof("%s: %s started extraction stage", core.T, m.ManagerUUID)
    89  	if err := m.extractLocalShards(); err != nil {
    90  		return err
    91  	}
    92  
    93  	s := binary.BigEndian.Uint64(m.Pars.TargetOrderSalt)
    94  	targetOrder := _torder(s, m.smap.Tmap)
    95  	if cmn.Rom.FastV(4, cos.SmoduleDsort) {
    96  		nlog.Infof("%s: %s final target in targetOrder => URL: %s, tid %s", core.T, m.ManagerUUID,
    97  			targetOrder[len(targetOrder)-1].PubNet.URL, targetOrder[len(targetOrder)-1].ID())
    98  	}
    99  
   100  	// Phase 2.
   101  	nlog.Infof("%s: %s started sort stage", core.T, m.ManagerUUID)
   102  	curTargetIsFinal, err := m.participateInRecordDistribution(targetOrder)
   103  	if err != nil {
   104  		return err
   105  	}
   106  
   107  	// Phase 3. - run only by the final target
   108  	if curTargetIsFinal {
   109  		// assuming uniform distribution estimate avg. output shard size
   110  		ratio := m.compressionRatio()
   111  		if cmn.Rom.FastV(4, cos.SmoduleDsort) {
   112  			nlog.Infof("%s [dsort] %s phase3: ratio=%f", core.T, m.ManagerUUID, ratio)
   113  		}
   114  		debug.Assertf(shard.IsCompressed(m.Pars.InputExtension) || ratio == 1, "tar ratio=%f, ext=%q",
   115  			ratio, m.Pars.InputExtension)
   116  
   117  		shardSize := int64(float64(m.Pars.OutputShardSize) / ratio)
   118  		nlog.Infof("%s: [dsort] %s started phase 3: ratio=%f, shard size (%d, %d)",
   119  			core.T, m.ManagerUUID, ratio, shardSize, m.Pars.OutputShardSize)
   120  		if err := m.phase3(shardSize); err != nil {
   121  			nlog.Errorf("%s: [dsort] %s phase3 err: %v", core.T, m.ManagerUUID, err)
   122  			return err
   123  		}
   124  	}
   125  
   126  	// Wait for signal to start shard creations. This will happen when manager
   127  	// notice that the specification for shards to be created locally was received.
   128  	select {
   129  	case <-m.startShardCreation:
   130  		break
   131  	case <-m.listenAborted():
   132  		return m.newErrAborted()
   133  	}
   134  
   135  	// After each target participates in the cluster-wide record distribution,
   136  	// start listening for the signal to start creating shards locally.
   137  	nlog.Infof("%s: %s started creation stage", core.T, m.ManagerUUID)
   138  	if err := m.dsorter.createShardsLocally(); err != nil {
   139  		return err
   140  	}
   141  
   142  	nlog.Infof("%s: %s finished successfully", core.T, m.ManagerUUID)
   143  	return nil
   144  }
   145  
   146  // returns a slice of targets in a pseudorandom order
   147  func _torder(salt uint64, tmap meta.NodeMap) []*meta.Snode {
   148  	var (
   149  		targets = make(map[uint64]*meta.Snode, len(tmap))
   150  		keys    = make([]uint64, 0, len(tmap))
   151  	)
   152  	for i, d := range tmap {
   153  		if d.InMaintOrDecomm() {
   154  			continue
   155  		}
   156  		c := xxhash.Checksum64S(cos.UnsafeB(i), salt)
   157  		targets[c] = d
   158  		keys = append(keys, c)
   159  	}
   160  	sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] })
   161  
   162  	t := make(meta.Nodes, len(keys))
   163  	for i, k := range keys {
   164  		t[i] = targets[k]
   165  	}
   166  	return t
   167  }
   168  
   169  func (m *Manager) startDsorter() error {
   170  	defer m.markStarted()
   171  	if err := m.initStreams(); err != nil {
   172  		return err
   173  	}
   174  	nlog.Infof("%s: %s starting with dsorter: %q", core.T, m.ManagerUUID, m.dsorter.name())
   175  	return m.dsorter.start()
   176  }
   177  
   178  func (m *Manager) extractLocalShards() (err error) {
   179  	m.extractionPhase.adjuster.start()
   180  	m.Metrics.Extraction.begin()
   181  
   182  	// compare with xact/xs/multiobj.go
   183  	group, ctx := errgroup.WithContext(context.Background())
   184  	switch {
   185  	case m.Pars.Pit.isRange():
   186  		err = m.iterRange(ctx, group)
   187  	case m.Pars.Pit.isList():
   188  		err = m.iterList(ctx, group)
   189  	default:
   190  		debug.Assert(m.Pars.Pit.isPrefix())
   191  		debug.Assert(false, "not implemented yet") // TODO -- FIXME
   192  	}
   193  
   194  	m.dsorter.postExtraction()
   195  	m.Metrics.Extraction.finish()
   196  	m.extractionPhase.adjuster.stop()
   197  	if err == nil {
   198  		m.incrementRef(int64(m.recm.Records.TotalObjectCount()))
   199  	}
   200  	return
   201  }
   202  
   203  func (m *Manager) iterRange(ctx context.Context, group *errgroup.Group) error {
   204  	var (
   205  		metrics = m.Metrics.Extraction
   206  		pt      = m.Pars.Pit.Template
   207  	)
   208  	metrics.mu.Lock()
   209  	metrics.TotalCnt = pt.Count()
   210  	metrics.mu.Unlock()
   211  	pt.InitIter()
   212  outer:
   213  	for name, hasNext := pt.Next(); hasNext; name, hasNext = pt.Next() {
   214  		select {
   215  		case <-m.listenAborted():
   216  			group.Wait()
   217  			return m.newErrAborted()
   218  		case <-ctx.Done():
   219  			break outer // context canceled: we have an error
   220  		default:
   221  		}
   222  
   223  		m.extractionPhase.adjuster.acquireGoroutineSema()
   224  		es := &extractShard{m, metrics, name, true /*is-range*/}
   225  		group.Go(es.do)
   226  	}
   227  	return group.Wait()
   228  }
   229  
   230  func (m *Manager) iterList(ctx context.Context, group *errgroup.Group) error {
   231  	metrics := m.Metrics.Extraction
   232  	metrics.mu.Lock()
   233  	metrics.TotalCnt = int64(len(m.Pars.Pit.ObjNames))
   234  	metrics.mu.Unlock()
   235  outer:
   236  	for _, name := range m.Pars.Pit.ObjNames {
   237  		select {
   238  		case <-m.listenAborted():
   239  			group.Wait()
   240  			return m.newErrAborted()
   241  		case <-ctx.Done():
   242  			break outer // context canceled: we have an error
   243  		default:
   244  		}
   245  
   246  		m.extractionPhase.adjuster.acquireGoroutineSema()
   247  		es := &extractShard{m, metrics, name, false /*is-range*/}
   248  		group.Go(es.do)
   249  	}
   250  	return group.Wait()
   251  }
   252  
   253  func (m *Manager) createShard(s *shard.Shard, lom *core.LOM) (err error) {
   254  	var (
   255  		metrics   = m.Metrics.Creation
   256  		shardName = s.Name
   257  		errCh     = make(chan error, 2)
   258  	)
   259  	if err = lom.InitBck(&m.Pars.OutputBck); err != nil {
   260  		return
   261  	}
   262  	lom.SetAtimeUnix(time.Now().UnixNano())
   263  
   264  	if m.aborted() {
   265  		return m.newErrAborted()
   266  	}
   267  
   268  	if err := m.dsorter.preShardCreation(s.Name, lom.Mountpath()); err != nil {
   269  		return err
   270  	}
   271  	defer m.dsorter.postShardCreation(lom.Mountpath())
   272  
   273  	cs := fs.Cap()
   274  	if err = cs.Err(); err != nil {
   275  		m.abort(err)
   276  		return
   277  	}
   278  
   279  	beforeCreation := time.Now()
   280  
   281  	var (
   282  		wg   = &sync.WaitGroup{}
   283  		r, w = io.Pipe()
   284  	)
   285  	wg.Add(1)
   286  	go func() {
   287  		var err error
   288  		if !m.Pars.DryRun {
   289  			params := core.AllocPutParams()
   290  			{
   291  				params.WorkTag = "dsort"
   292  				params.Cksum = nil
   293  				params.Atime = beforeCreation
   294  
   295  				// NOTE: cannot have `PutObject` closing the original reader
   296  				// on error as it'll cause writer (below) to panic
   297  				params.Reader = io.NopCloser(r)
   298  
   299  				// TODO: params.Xact - in part, to count PUTs and bytes in a generic fashion
   300  				// (vs metrics.ShardCreationStats.updateThroughput - see below)
   301  
   302  				// TODO: add params.Size = (size resulting from shardRW.Create below)
   303  			}
   304  			err = core.T.PutObject(lom, params)
   305  			core.FreePutParams(params)
   306  		} else {
   307  			_, err = io.Copy(io.Discard, r)
   308  		}
   309  		errCh <- err
   310  		wg.Done()
   311  	}()
   312  
   313  	// may reshard into a different format
   314  	shardRW := m.shardRW
   315  	//
   316  	// TODO -- FIXME: compare with extractShard._do()
   317  	//
   318  	if !m.Pars.DryRun && m.Pars.OutputExtension != m.Pars.InputExtension {
   319  		debug.Assert(m.Pars.OutputExtension != "")
   320  		shardRW = shard.RWs[m.Pars.OutputExtension]
   321  		debug.Assert(shardRW != nil, m.Pars.OutputExtension)
   322  	}
   323  
   324  	_, err = shardRW.Create(s, w, m.dsorter)
   325  	w.CloseWithError(err)
   326  	if err != nil {
   327  		r.CloseWithError(err)
   328  		return err
   329  	}
   330  
   331  	select {
   332  	case err = <-errCh:
   333  		if err != nil {
   334  			r.CloseWithError(err)
   335  			w.CloseWithError(err)
   336  		}
   337  	case <-m.listenAborted():
   338  		err = m.newErrAborted()
   339  		r.CloseWithError(err)
   340  		w.CloseWithError(err)
   341  	}
   342  
   343  	wg.Wait()
   344  	close(errCh)
   345  
   346  	if err != nil {
   347  		return err
   348  	}
   349  
   350  	si, err := m.smap.HrwHash2T(lom.Digest())
   351  	if err != nil {
   352  		return err
   353  	}
   354  
   355  	// If the newly created shard belongs on a different target
   356  	// according to HRW, send it there. Since it doesn't really matter
   357  	// if we have an extra copy of the object local to this target, we
   358  	// optimize for performance by not removing the object now.
   359  	if si.ID() != core.T.SID() && !m.Pars.DryRun {
   360  		lom.Lock(false)
   361  		defer lom.Unlock(false)
   362  
   363  		// Need to make sure that the object is still there.
   364  		if err := lom.Load(false /*cache it*/, true /*locked*/); err != nil {
   365  			return err
   366  		}
   367  
   368  		if lom.SizeBytes() <= 0 {
   369  			goto exit
   370  		}
   371  
   372  		file, err := cos.NewFileHandle(lom.FQN)
   373  		if err != nil {
   374  			return err
   375  		}
   376  
   377  		o := transport.AllocSend()
   378  		o.Hdr = transport.ObjHdr{
   379  			ObjName:  shardName,
   380  			ObjAttrs: cmn.ObjAttrs{Size: lom.SizeBytes(), Cksum: lom.Checksum()},
   381  		}
   382  		o.Hdr.Bck.Copy(lom.Bucket())
   383  
   384  		// Make send synchronous.
   385  		streamWg := &sync.WaitGroup{}
   386  		errCh := make(chan error, 1)
   387  		o.Callback = func(_ *transport.ObjHdr, _ io.ReadCloser, _ any, err error) {
   388  			errCh <- err
   389  			streamWg.Done()
   390  		}
   391  		streamWg.Add(1)
   392  		err = m.streams.shards.Send(o, file, si)
   393  		if err != nil {
   394  			return err
   395  		}
   396  		streamWg.Wait()
   397  		if err := <-errCh; err != nil {
   398  			return err
   399  		}
   400  	}
   401  
   402  exit:
   403  	metrics.mu.Lock()
   404  	metrics.CreatedCnt++
   405  	if si.ID() != core.T.SID() {
   406  		metrics.MovedShardCnt++
   407  	}
   408  	metrics.mu.Unlock()
   409  
   410  	return nil
   411  }
   412  
   413  // participateInRecordDistribution coordinates the distributed merging and
   414  // sorting of each target's SortedRecords based on the order defined by
   415  // targetOrder. It returns a bool, currentTargetIsFinal, which is true iff the
   416  // current target is the final target in targetOrder, which by construction of
   417  // the algorithm, should contain the final, complete, sorted slice of Record
   418  // structs.
   419  //
   420  // The algorithm uses the following premise: for a target T at index i in
   421  // targetOrder, if i is even, then T will send its FileMeta slice to the target
   422  // at index i+1 in targetOrder. If i is odd, then it will do a blocking receive
   423  // on the FileMeta slice from the target at index i-1 in targetOrder, and will
   424  // remove all even-indexed targets in targetOrder after receiving. This pattern
   425  // repeats until len(targetOrder) == 1, in which case the single target in the
   426  // slice is the final target with the final, complete, sorted slice of Record
   427  // structs.
   428  func (m *Manager) participateInRecordDistribution(targetOrder meta.Nodes) (currentTargetIsFinal bool, err error) {
   429  	var (
   430  		i           int
   431  		d           *meta.Snode
   432  		dummyTarget *meta.Snode // dummy target is represented as nil value
   433  	)
   434  
   435  	// Metrics
   436  	metrics := m.Metrics.Sorting
   437  	metrics.begin()
   438  	defer metrics.finish()
   439  
   440  	expectedReceived := int32(1)
   441  	for len(targetOrder) > 1 {
   442  		if len(targetOrder)%2 == 1 {
   443  			// For simplicity, we always work with an even-length slice of targets. If len(targetOrder) is odd,
   444  			// we put a "dummy target" into the slice at index len(targetOrder)-2 which simulates sending its
   445  			// metadata to the next target in targetOrder (which is actually itself).
   446  			targetOrder = append(
   447  				targetOrder[:len(targetOrder)-1],
   448  				dummyTarget,
   449  				targetOrder[len(targetOrder)-1],
   450  			)
   451  		}
   452  
   453  		for i, d = range targetOrder {
   454  			if d != dummyTarget && d.ID() == core.T.SID() {
   455  				break
   456  			}
   457  		}
   458  
   459  		if i%2 == 0 {
   460  			m.dsorter.postRecordDistribution()
   461  
   462  			var (
   463  				beforeSend = time.Now()
   464  				group      = &errgroup.Group{}
   465  				r, w       = io.Pipe()
   466  			)
   467  			group.Go(func() error {
   468  				var (
   469  					buf, slab = g.mm.AllocSize(serializationBufSize)
   470  					msgpw     = msgp.NewWriterBuf(w, buf)
   471  				)
   472  				defer slab.Free(buf)
   473  
   474  				if err := m.recm.Records.EncodeMsg(msgpw); err != nil {
   475  					w.CloseWithError(err)
   476  					return errors.Errorf("failed to marshal msgp: %v", err)
   477  				}
   478  				err := msgpw.Flush()
   479  				w.CloseWithError(err)
   480  				if err != nil {
   481  					return errors.Errorf("failed to flush msgp: %v", err)
   482  				}
   483  				return nil
   484  			})
   485  			group.Go(func() error {
   486  				var (
   487  					query  = url.Values{}
   488  					sendTo = targetOrder[i+1]
   489  				)
   490  				query.Add(apc.QparamTotalCompressedSize, strconv.FormatInt(m.totalShardSize(), 10))
   491  				query.Add(apc.QparamTotalUncompressedSize, strconv.FormatInt(m.totalExtractedSize(), 10))
   492  				query.Add(apc.QparamTotalInputShardsExtracted, strconv.Itoa(m.recm.Records.Len()))
   493  				reqArgs := &cmn.HreqArgs{
   494  					Method: http.MethodPost,
   495  					Base:   sendTo.URL(cmn.NetIntraData),
   496  					Path:   apc.URLPathdSortRecords.Join(m.ManagerUUID),
   497  					Query:  query,
   498  					BodyR:  r,
   499  				}
   500  				err := m._do(reqArgs, sendTo, "send sorted records")
   501  				r.CloseWithError(err)
   502  				return err
   503  			})
   504  			if err := group.Wait(); err != nil {
   505  				return false, err
   506  			}
   507  
   508  			m.recm.Records.Drain() // we do not need it anymore
   509  
   510  			metrics.mu.Lock()
   511  			metrics.SentStats.updateTime(time.Since(beforeSend))
   512  			metrics.mu.Unlock()
   513  			return
   514  		}
   515  
   516  		beforeRecv := time.Now()
   517  
   518  		// i%2 == 1
   519  		receiveFrom := targetOrder[i-1]
   520  		if receiveFrom == dummyTarget { // dummy target
   521  			m.incrementReceived()
   522  		}
   523  
   524  		for m.received.count.Load() < expectedReceived {
   525  			select {
   526  			case <-m.listenReceived():
   527  			case <-m.listenAborted():
   528  				err = m.newErrAborted()
   529  				return
   530  			}
   531  		}
   532  		expectedReceived++
   533  
   534  		metrics.mu.Lock()
   535  		metrics.RecvStats.updateTime(time.Since(beforeRecv))
   536  		metrics.mu.Unlock()
   537  
   538  		t := targetOrder[:0]
   539  		for i, d = range targetOrder {
   540  			if i%2 == 1 {
   541  				t = append(t, d)
   542  			}
   543  		}
   544  		targetOrder = t
   545  
   546  		m.recm.MergeEnqueuedRecords()
   547  	}
   548  
   549  	err = sortRecords(m.recm.Records, m.Pars.Algorithm)
   550  	m.dsorter.postRecordDistribution()
   551  	return true, err
   552  }
   553  
   554  func (m *Manager) generateShardsWithTemplate(maxSize int64) ([]*shard.Shard, error) {
   555  	var (
   556  		start           int
   557  		curShardSize    int64
   558  		n               = m.recm.Records.Len()
   559  		pt              = m.Pars.Pot.Template
   560  		shardCount      = pt.Count()
   561  		shards          = make([]*shard.Shard, 0)
   562  		numLocalRecords = make(map[string]int, m.smap.CountActiveTs())
   563  	)
   564  	pt.InitIter()
   565  
   566  	if maxSize <= 0 {
   567  		// Heuristic: shard size when maxSize not specified.
   568  		maxSize = int64(math.Ceil(float64(m.totalExtractedSize()) / float64(shardCount)))
   569  	}
   570  
   571  	for i, r := range m.recm.Records.All() {
   572  		numLocalRecords[r.DaemonID]++
   573  		curShardSize += r.TotalSize()
   574  		if curShardSize < maxSize && i < n-1 {
   575  			continue
   576  		}
   577  
   578  		name, hasNext := pt.Next()
   579  		if !hasNext {
   580  			// no more shard names are available
   581  			return nil, errors.Errorf("number of shards to be created exceeds expected number of shards (%d)", shardCount)
   582  		}
   583  		shard := &shard.Shard{
   584  			Name: name,
   585  		}
   586  		ext, err := archive.Mime("", name)
   587  		if err == nil {
   588  			debug.Assert(m.Pars.OutputExtension == ext)
   589  		} else {
   590  			shard.Name = name + m.Pars.OutputExtension
   591  		}
   592  
   593  		shard.Size = curShardSize
   594  		shard.Records = m.recm.Records.Slice(start, i+1)
   595  		shards = append(shards, shard)
   596  
   597  		start = i + 1
   598  		curShardSize = 0
   599  		for k := range numLocalRecords {
   600  			numLocalRecords[k] = 0
   601  		}
   602  	}
   603  
   604  	return shards, nil
   605  }
   606  
   607  func (m *Manager) generateShardsWithOrderingFile(maxSize int64) ([]*shard.Shard, error) {
   608  	var (
   609  		shards         = make([]*shard.Shard, 0)
   610  		externalKeyMap = make(map[string]string)
   611  		shardsBuilder  = make(map[string][]*shard.Shard)
   612  	)
   613  	if maxSize <= 0 {
   614  		return nil, fmt.Errorf(fmtErrInvalidMaxSize, maxSize)
   615  	}
   616  	parsedURL, err := url.Parse(m.Pars.OrderFileURL)
   617  	if err != nil {
   618  		return nil, fmt.Errorf(fmtErrOrderURL, m.Pars.OrderFileURL, err)
   619  	}
   620  
   621  	req, err := http.NewRequest(http.MethodGet, m.Pars.OrderFileURL, http.NoBody)
   622  	if err != nil {
   623  		return nil, err
   624  	}
   625  	// is intra-call
   626  	tsi := core.T.Snode()
   627  	req.Header.Set(apc.HdrCallerID, tsi.ID())
   628  	req.Header.Set(apc.HdrCallerName, tsi.String())
   629  
   630  	resp, err := m.client.Do(req) //nolint:bodyclose // closed by cos.Close below
   631  	if err != nil {
   632  		return nil, err
   633  	}
   634  	defer cos.Close(resp.Body)
   635  	if resp.StatusCode != http.StatusOK {
   636  		return nil, fmt.Errorf(
   637  			"unexpected status code (%d) when requesting order file from %q",
   638  			resp.StatusCode, m.Pars.OrderFileURL,
   639  		)
   640  	}
   641  
   642  	// TODO: handle very large files > GB - in case the file is very big we
   643  	//  need to save file to the disk and operate on the file directly rather
   644  	//  than keeping everything in memory.
   645  
   646  	switch filepath.Ext(parsedURL.Path) {
   647  	case ".json":
   648  		var ekm map[string][]string
   649  		if err := jsoniter.NewDecoder(resp.Body).Decode(&ekm); err != nil {
   650  			return nil, err
   651  		}
   652  
   653  		for shardNameFmt, recordKeys := range ekm {
   654  			for _, recordKey := range recordKeys {
   655  				externalKeyMap[recordKey] = shardNameFmt
   656  			}
   657  		}
   658  	default:
   659  		lineReader := bufio.NewReader(resp.Body)
   660  		for idx := 0; ; idx++ {
   661  			l, _, err := lineReader.ReadLine()
   662  			if err == io.EOF {
   663  				break
   664  			}
   665  			if err != nil {
   666  				return nil, err
   667  			}
   668  
   669  			line := strings.TrimSpace(string(l))
   670  			if line == "" {
   671  				continue
   672  			}
   673  
   674  			parts := strings.Split(line, m.Pars.OrderFileSep)
   675  			if len(parts) != 2 {
   676  				msg := fmt.Sprintf("malformed line (%d) in external key map: %s", idx, line)
   677  				if err := m.react(m.Pars.EKMMalformedLine, msg); err != nil {
   678  					return nil, err
   679  				}
   680  			}
   681  
   682  			recordKey, shardNameFmt := parts[0], parts[1]
   683  			externalKeyMap[recordKey] = shardNameFmt
   684  		}
   685  	}
   686  
   687  	for _, r := range m.recm.Records.All() {
   688  		key := fmt.Sprintf("%v", r.Key)
   689  		shardNameFmt, ok := externalKeyMap[key]
   690  		if !ok {
   691  			msg := fmt.Sprintf("record %q doesn't belong in external key map", key)
   692  			if err := m.react(m.Pars.EKMMissingKey, msg); err != nil {
   693  				return nil, err
   694  			}
   695  		}
   696  
   697  		shards := shardsBuilder[shardNameFmt]
   698  		recordSize := r.TotalSize() + m.shardRW.MetadataSize()*int64(len(r.Objects))
   699  		shardCount := len(shards)
   700  		if shardCount == 0 || shards[shardCount-1].Size > maxSize {
   701  			shard := &shard.Shard{
   702  				Name:    fmt.Sprintf(shardNameFmt, shardCount),
   703  				Size:    recordSize,
   704  				Records: shard.NewRecords(1),
   705  			}
   706  			shard.Records.Insert(r)
   707  			shardsBuilder[shardNameFmt] = append(shardsBuilder[shardNameFmt], shard)
   708  		} else {
   709  			// Append records
   710  			lastShard := shards[shardCount-1]
   711  			lastShard.Size += recordSize
   712  			lastShard.Records.Insert(r)
   713  		}
   714  	}
   715  
   716  	for _, s := range shardsBuilder {
   717  		shards = append(shards, s...)
   718  	}
   719  
   720  	return shards, nil
   721  }
   722  
   723  // Create `maxSize` output shard structures in the order defined by dsortManager.Records.
   724  // Each output shard structure is "distributed" (via m._dist below)
   725  // to one of the targets - to create the corresponding output shard.
   726  // The logic to map output shard => target:
   727  //  1. By HRW (not using compression)
   728  //  2. By locality (using compression),
   729  //     using two maps:
   730  //     i) shardsToTarget - tracks the total number of shards creation requests sent to each target URL
   731  //     ii) numLocalRecords - tracks the number of records in the current shardMeta each target has locally
   732  //     The target is determined firstly by locality (i.e. the target with the most local records)
   733  //     and secondly (if there is a tie), by least load
   734  //     (i.e. the target with the least number of pending shard creation requests).
   735  func (m *Manager) phase3(maxSize int64) error {
   736  	var (
   737  		shards         []*shard.Shard
   738  		err            error
   739  		shardsToTarget = make(map[*meta.Snode][]*shard.Shard, m.smap.CountActiveTs())
   740  		sendOrder      = make(map[string]map[string]*shard.Shard, m.smap.CountActiveTs())
   741  		errCh          = make(chan error, m.smap.CountActiveTs())
   742  	)
   743  	for _, d := range m.smap.Tmap {
   744  		if m.smap.InMaintOrDecomm(d) {
   745  			continue
   746  		}
   747  		shardsToTarget[d] = nil
   748  		if m.dsorter.name() == MemType {
   749  			sendOrder[d.ID()] = make(map[string]*shard.Shard, 100)
   750  		}
   751  	}
   752  	if m.Pars.OrderFileURL != "" {
   753  		shards, err = m.generateShardsWithOrderingFile(maxSize)
   754  	} else {
   755  		shards, err = m.generateShardsWithTemplate(maxSize)
   756  	}
   757  	if err != nil {
   758  		return err
   759  	}
   760  
   761  	bck := meta.CloneBck(&m.Pars.OutputBck)
   762  	if err := bck.Init(core.T.Bowner()); err != nil {
   763  		return err
   764  	}
   765  	for _, s := range shards {
   766  		si, err := m.smap.HrwName2T(bck.MakeUname(s.Name))
   767  		if err != nil {
   768  			return err
   769  		}
   770  		shardsToTarget[si] = append(shardsToTarget[si], s)
   771  
   772  		if m.dsorter.name() == MemType {
   773  			singleSendOrder := make(map[string]*shard.Shard)
   774  			for _, record := range s.Records.All() {
   775  				shrd, ok := singleSendOrder[record.DaemonID]
   776  				if !ok {
   777  					shrd = &shard.Shard{
   778  						Name:    s.Name,
   779  						Records: shard.NewRecords(100),
   780  					}
   781  					singleSendOrder[record.DaemonID] = shrd
   782  				}
   783  				shrd.Records.Insert(record)
   784  			}
   785  
   786  			for tid, shard := range singleSendOrder {
   787  				sendOrder[tid][shard.Name] = shard
   788  			}
   789  		}
   790  	}
   791  
   792  	m.recm.Records.Drain()
   793  
   794  	wg := cos.NewLimitedWaitGroup(cmn.MaxParallelism(), len(shardsToTarget))
   795  	for si, s := range shardsToTarget {
   796  		wg.Add(1)
   797  		go m._dist(si, s, sendOrder[si.ID()], errCh, wg)
   798  	}
   799  
   800  	wg.Wait()
   801  	close(errCh)
   802  
   803  	for err := range errCh {
   804  		nlog.Errorf("%s: [dsort] %s err while sending shards: %v", core.T, m.ManagerUUID, err)
   805  		return err
   806  	}
   807  	nlog.Infof("%s: [dsort] %s finished sending shards", core.T, m.ManagerUUID)
   808  	return nil
   809  }
   810  
   811  func (m *Manager) _dist(si *meta.Snode, s []*shard.Shard, order map[string]*shard.Shard, errCh chan error, wg cos.WG) {
   812  	var (
   813  		group = &errgroup.Group{}
   814  		r, w  = io.Pipe()
   815  	)
   816  	group.Go(func() error {
   817  		var (
   818  			buf, slab = g.mm.AllocSize(serializationBufSize)
   819  			msgpw     = msgp.NewWriterBuf(w, buf)
   820  			md        = &CreationPhaseMetadata{Shards: s, SendOrder: order}
   821  		)
   822  		err := md.EncodeMsg(msgpw)
   823  		if err == nil {
   824  			err = msgpw.Flush()
   825  		}
   826  		w.CloseWithError(err)
   827  		slab.Free(buf)
   828  		return err
   829  	})
   830  	group.Go(func() error {
   831  		query := m.Pars.InputBck.NewQuery()
   832  		reqArgs := &cmn.HreqArgs{
   833  			Method: http.MethodPost,
   834  			Base:   si.URL(cmn.NetIntraData),
   835  			Path:   apc.URLPathdSortShards.Join(m.ManagerUUID),
   836  			Query:  query,
   837  			BodyR:  r,
   838  		}
   839  		err := m._do(reqArgs, si, "distribute shards")
   840  		r.CloseWithError(err)
   841  		return err
   842  	})
   843  
   844  	if err := group.Wait(); err != nil {
   845  		errCh <- err
   846  	}
   847  	wg.Done()
   848  }
   849  
   850  func (m *Manager) _do(reqArgs *cmn.HreqArgs, tsi *meta.Snode, act string) error {
   851  	req, errV := reqArgs.Req()
   852  	if errV != nil {
   853  		return errV
   854  	}
   855  	resp, err := m.client.Do(req) //nolint:bodyclose // cos.Close below
   856  	if err != nil {
   857  		return err
   858  	}
   859  	if resp.StatusCode != http.StatusOK {
   860  		var b []byte
   861  		b, err = io.ReadAll(resp.Body)
   862  		if err == nil {
   863  			err = fmt.Errorf("%s: %s failed to %s: %s", core.T, m.ManagerUUID, act, strings.TrimSuffix(string(b), "\n"))
   864  		} else {
   865  			err = fmt.Errorf("%s: %s failed to %s: got %v(%d) from %s", core.T, m.ManagerUUID, act, err,
   866  				resp.StatusCode, tsi.StringEx())
   867  		}
   868  	}
   869  	cos.Close(resp.Body)
   870  	return err
   871  }
   872  
   873  //////////////////
   874  // extractShard //
   875  //////////////////
   876  
   877  type extractShard struct {
   878  	m       *Manager
   879  	metrics *LocalExtraction
   880  	name    string
   881  	isRange bool
   882  }
   883  
   884  func (es *extractShard) do() (err error) {
   885  	m := es.m
   886  	shardName := es.name
   887  	if es.isRange && m.Pars.InputExtension != "" {
   888  		ext, errV := archive.Mime("", es.name) // from filename
   889  		if errV == nil {
   890  			if !archive.EqExt(ext, m.Pars.InputExtension) {
   891  				if cmn.Rom.FastV(4, cos.SmoduleDsort) {
   892  					nlog.Infof("%s: %s skipping %s: %q vs %q", core.T, m.ManagerUUID,
   893  						es.name, ext, m.Pars.InputExtension)
   894  				}
   895  				return
   896  			}
   897  		} else {
   898  			shardName = es.name + m.Pars.InputExtension
   899  		}
   900  	}
   901  	lom := core.AllocLOM(shardName)
   902  
   903  	err = es._do(lom)
   904  
   905  	core.FreeLOM(lom)
   906  	phaseInfo := &m.extractionPhase
   907  	phaseInfo.adjuster.releaseGoroutineSema()
   908  	return
   909  }
   910  
   911  func (es *extractShard) _do(lom *core.LOM) error {
   912  	var (
   913  		m                        = es.m
   914  		estimateTotalRecordsSize uint64
   915  		warnOOM                  bool
   916  	)
   917  	if err := lom.InitBck(&m.Pars.InputBck); err != nil {
   918  		return err
   919  	}
   920  	if _, local, err := lom.HrwTarget(m.smap); err != nil || !local {
   921  		return err
   922  	}
   923  	if err := lom.Load(false /*cache it*/, false /*locked*/); err != nil {
   924  		if cmn.IsErrObjNought(err) {
   925  			msg := fmt.Sprintf("shard.do: %q does not exist", lom.Cname())
   926  			return m.react(m.Pars.MissingShards, msg)
   927  		}
   928  		return err
   929  	}
   930  
   931  	shardRW := m.shardRW
   932  	if shardRW == nil {
   933  		debug.Assert(!m.Pars.DryRun)
   934  		ext, err := archive.Mime("", lom.FQN)
   935  		if err != nil {
   936  			return nil // skip
   937  		}
   938  		shardRW = shard.RWs[ext]
   939  		debug.Assert(shardRW != nil, ext)
   940  	}
   941  
   942  	phaseInfo := &m.extractionPhase
   943  	phaseInfo.adjuster.acquireSema(lom.Mountpath())
   944  	if m.aborted() {
   945  		phaseInfo.adjuster.releaseSema(lom.Mountpath())
   946  		return m.newErrAborted()
   947  	}
   948  
   949  	cs := fs.Cap()
   950  	if err := cs.Err(); err != nil {
   951  		phaseInfo.adjuster.releaseSema(lom.Mountpath())
   952  		m.abort(err)
   953  		return err
   954  	}
   955  
   956  	lom.Lock(false)
   957  	fh, err := lom.OpenFile()
   958  	if err != nil {
   959  		phaseInfo.adjuster.releaseSema(lom.Mountpath())
   960  		lom.Unlock(false)
   961  		return errors.Errorf("unable to open %s: %v", lom.Cname(), err)
   962  	}
   963  
   964  	expectedExtractedSize := uint64(float64(lom.SizeBytes()) / m.compressionRatio())
   965  	toDisk := m.dsorter.preShardExtraction(expectedExtractedSize)
   966  
   967  	extractedSize, extractedCount, err := shardRW.Extract(lom, fh, m.recm, toDisk)
   968  	cos.Close(fh)
   969  
   970  	m.addSizes(lom.SizeBytes(), extractedSize) // update compression rate
   971  
   972  	phaseInfo.adjuster.releaseSema(lom.Mountpath())
   973  	lom.Unlock(false)
   974  
   975  	m.dsorter.postShardExtraction(expectedExtractedSize) // schedule freeing reserved memory on next memory update
   976  	if err != nil {
   977  		return errors.Errorf("failed to extract shard %s: %v", lom.Cname(), err)
   978  	}
   979  
   980  	if toDisk {
   981  		g.tstats.Add(stats.DsortExtractShardDskCnt, 1)
   982  	} else {
   983  		g.tstats.Add(stats.DsortExtractShardMemCnt, 1)
   984  	}
   985  	g.tstats.Add(stats.DsortExtractShardSize, extractedSize)
   986  
   987  	//
   988  	// update metrics, check OOM
   989  	//
   990  
   991  	metrics := es.metrics
   992  	metrics.mu.Lock()
   993  	metrics.ExtractedRecordCnt += int64(extractedCount)
   994  	metrics.ExtractedCnt++
   995  	if metrics.ExtractedCnt == 1 && extractedCount > 0 {
   996  		// After extracting the _first_ shard estimate how much memory
   997  		// will be required to keep all records in memory. One node
   998  		// will eventually have all records from all shards so we
   999  		// don't calculate estimates only for single node.
  1000  		recordSize := int(m.recm.Records.RecordMemorySize())
  1001  		estimateTotalRecordsSize = uint64(metrics.TotalCnt * int64(extractedCount*recordSize))
  1002  		if estimateTotalRecordsSize > m.freeMemory() {
  1003  			warnOOM = true
  1004  		}
  1005  	}
  1006  	metrics.ExtractedSize += extractedSize
  1007  	if toDisk {
  1008  		metrics.ExtractedToDiskCnt++
  1009  		metrics.ExtractedToDiskSize += extractedSize
  1010  	}
  1011  	metrics.mu.Unlock()
  1012  
  1013  	if warnOOM {
  1014  		msg := fmt.Sprintf("(estimated) total size of records (%d) will possibly exceed available memory (%s) during sorting phase",
  1015  			estimateTotalRecordsSize, m.Pars.MaxMemUsage)
  1016  		return m.react(cmn.WarnReaction, msg)
  1017  	}
  1018  	return nil
  1019  }