github.com/m3db/m3@v1.5.0/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go (about)

     1  // Copyright (c) 2016 Uber Technologies, Inc.
     2  //
     3  // Permission is hereby granted, free of charge, to any person obtaining a copy
     4  // of this software and associated documentation files (the "Software"), to deal
     5  // in the Software without restriction, including without limitation the rights
     6  // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
     7  // copies of the Software, and to permit persons to whom the Software is
     8  // furnished to do so, subject to the following conditions:
     9  //
    10  // The above copyright notice and this permission notice shall be included in
    11  // all copies or substantial portions of the Software.
    12  //
    13  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
    14  // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
    15  // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
    16  // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
    17  // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
    18  // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
    19  // THE SOFTWARE.
    20  
    21  package commitlog
    22  
    23  import (
    24  	"bytes"
    25  	"context"
    26  	"errors"
    27  	"fmt"
    28  	"io"
    29  	"sync"
    30  	"time"
    31  
    32  	"github.com/m3db/m3/src/cluster/shard"
    33  	"github.com/m3db/m3/src/dbnode/namespace"
    34  	"github.com/m3db/m3/src/dbnode/persist"
    35  	"github.com/m3db/m3/src/dbnode/persist/fs"
    36  	"github.com/m3db/m3/src/dbnode/persist/fs/commitlog"
    37  	"github.com/m3db/m3/src/dbnode/storage/block"
    38  	"github.com/m3db/m3/src/dbnode/storage/bootstrap"
    39  	"github.com/m3db/m3/src/dbnode/storage/bootstrap/result"
    40  	"github.com/m3db/m3/src/dbnode/storage/series"
    41  	"github.com/m3db/m3/src/dbnode/topology"
    42  	"github.com/m3db/m3/src/dbnode/ts"
    43  	"github.com/m3db/m3/src/x/checked"
    44  	xcontext "github.com/m3db/m3/src/x/context"
    45  	"github.com/m3db/m3/src/x/ident"
    46  	"github.com/m3db/m3/src/x/instrument"
    47  	xtime "github.com/m3db/m3/src/x/time"
    48  
    49  	"github.com/opentracing/opentracing-go"
    50  	"github.com/uber-go/tally"
    51  	"go.uber.org/zap"
    52  	"golang.org/x/sync/errgroup"
    53  )
    54  
    55  const (
    56  	workerChannelSize                 = 256
    57  	readSeriesBlocksWorkerChannelSize = 512
    58  )
    59  
    60  type newIteratorFn func(opts commitlog.IteratorOpts) (
    61  	iter commitlog.Iterator, corruptFiles []commitlog.ErrorWithPath, err error)
    62  type snapshotFilesFn func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error)
    63  
    64  type bootstrapNamespace struct {
    65  	namespaceID             []byte
    66  	bootstrapping           bool
    67  	dataAndIndexShardRanges result.ShardTimeRanges
    68  	namespace               namespace.Metadata
    69  	namespaceContext        namespace.Context
    70  	dataBlockSize           time.Duration
    71  	accumulator             bootstrap.NamespaceDataAccumulator
    72  }
    73  
    74  type seriesMapKey struct {
    75  	fileReadID  uint64
    76  	uniqueIndex uint64
    77  }
    78  
    79  type seriesMapEntry struct {
    80  	namespace          *bootstrapNamespace
    81  	series             bootstrap.CheckoutSeriesResult
    82  	shardNoLongerOwned bool
    83  }
    84  
    85  // accumulateArg contains all the information a worker go-routine needs to
    86  // accumulate a write for encoding into the database.
    87  type accumulateArg struct {
    88  	namespace *bootstrapNamespace
    89  	series    bootstrap.CheckoutSeriesResult
    90  	shard     uint32
    91  	dp        ts.Datapoint
    92  	unit      xtime.Unit
    93  
    94  	// longAnnotation stores the annotation value in case it does not fit in shortAnnotation.
    95  	longAnnotation ts.Annotation
    96  
    97  	// shortAnnotation is a predefined buffer for passing small allocations around instead of allocating.
    98  	shortAnnotation    [ts.OptimizedAnnotationLen]byte
    99  	shortAnnotationLen uint8
   100  }
   101  
   102  type accumulateWorker struct {
   103  	inputCh        chan accumulateArg
   104  	datapointsRead int
   105  	numErrors      int
   106  }
   107  
   108  type seriesBlock struct {
   109  	resolver bootstrap.SeriesRefResolver
   110  	block    block.DatabaseBlock
   111  }
   112  
   113  type readSeriesBlocksWorker struct {
   114  	dataCh      chan seriesBlock
   115  	reader      fs.DataFileSetReader
   116  	shard       uint32
   117  	accumulator bootstrap.NamespaceDataAccumulator
   118  	blocksPool  block.DatabaseBlockPool
   119  	blockStart  xtime.UnixNano
   120  	blockSize   time.Duration
   121  	nsCtx       namespace.Context
   122  }
   123  
   124  func (w *readSeriesBlocksWorker) readSeriesBlocks(ctx context.Context) error {
   125  	defer close(w.dataCh)
   126  	numSeriesRead := 0
   127  	for {
   128  		id, tags, data, expectedChecksum, err := w.reader.Read()
   129  		if err != nil && !errors.Is(err, io.EOF) {
   130  			return err
   131  		}
   132  		if errors.Is(err, io.EOF) {
   133  			break
   134  		}
   135  		numSeriesRead++
   136  
   137  		dbBlock := w.blocksPool.Get()
   138  		dbBlock.Reset(w.blockStart, w.blockSize,
   139  			ts.NewSegment(data, nil, 0, ts.FinalizeHead), w.nsCtx)
   140  
   141  		// Resetting the block will trigger a checksum calculation, so use
   142  		// that instead of calculating it twice.
   143  		checksum, err := dbBlock.Checksum()
   144  		if err != nil {
   145  			return err
   146  		}
   147  		if checksum != expectedChecksum {
   148  			return fmt.Errorf("checksum for series: %s was %d but expected %d",
   149  				id, checksum, expectedChecksum)
   150  		}
   151  
   152  		res, owned, err := w.accumulator.CheckoutSeriesWithoutLock(w.shard, id, tags)
   153  		if err != nil {
   154  			if !owned {
   155  				// Skip bootstrapping this series if we don't own it.
   156  				continue
   157  			}
   158  			return err
   159  		}
   160  
   161  		w.dataCh <- seriesBlock{
   162  			resolver: res.Resolver,
   163  			block:    dbBlock,
   164  		}
   165  
   166  		id.Finalize()
   167  		tags.Close()
   168  
   169  		// check if context was not canceled on a regular basis.
   170  		if numSeriesRead%1024 == 0 {
   171  			select {
   172  			case <-ctx.Done():
   173  				return nil
   174  			default:
   175  				// do not block.
   176  			}
   177  		}
   178  	}
   179  	return nil
   180  }
   181  
   182  type readNamespaceResult struct {
   183  	namespace               bootstrap.Namespace
   184  	dataAndIndexShardRanges result.ShardTimeRanges
   185  }
   186  
   187  type commitLogResult struct {
   188  	shouldReturnUnfulfilled bool
   189  	// ensures we only read the commit log once
   190  	read bool
   191  }
   192  
   193  type commitLogSourceMetrics struct {
   194  	corruptCommitlogFile tally.Counter
   195  	bootstrapping        tally.Gauge
   196  	commitLogEntriesRead tally.Counter
   197  }
   198  
   199  func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics {
   200  	return commitLogSourceMetrics{
   201  		corruptCommitlogFile: scope.SubScope("commitlog").Counter("corrupt"),
   202  		bootstrapping:        scope.SubScope("status").Gauge("bootstrapping"),
   203  		commitLogEntriesRead: scope.SubScope("commitlog").Counter("entries-read"),
   204  	}
   205  }
   206  
   207  type gaugeLoopCloserFn func()
   208  
   209  func (m commitLogSourceMetrics) emitBootstrapping() gaugeLoopCloserFn {
   210  	doneCh := make(chan struct{})
   211  	go func() {
   212  		for {
   213  			select {
   214  			case <-doneCh:
   215  				m.bootstrapping.Update(0)
   216  				return
   217  			default:
   218  				m.bootstrapping.Update(1)
   219  				time.Sleep(time.Second)
   220  			}
   221  		}
   222  	}()
   223  
   224  	return func() { close(doneCh) }
   225  }
   226  
   227  type commitLogSource struct {
   228  	opts  Options
   229  	log   *zap.Logger
   230  	nowFn func() time.Time
   231  
   232  	// Filesystem inspection capture before node was started.
   233  	inspection fs.Inspection
   234  
   235  	newIteratorFn   newIteratorFn
   236  	snapshotFilesFn snapshotFilesFn
   237  	newReaderFn     fs.NewReaderFn
   238  
   239  	metrics commitLogSourceMetrics
   240  	// Cache the results of reading the commit log between passes. The commit log is not sharded by time range, so the
   241  	// entire log needs to be read irrespective of the configured time ranges for the pass. The commit log only needs
   242  	// to be read once (during the first pass) and the results can be subsequently cached and returned on future passes.
   243  	// Since the bootstrapper is single threaded this does not need to be guarded with a mutex.
   244  	commitLogResult commitLogResult
   245  
   246  	instrumentation *instrumentation
   247  }
   248  
   249  func newCommitLogSource(
   250  	opts Options,
   251  	inspection fs.Inspection,
   252  ) bootstrap.Source {
   253  	scope := opts.
   254  		ResultOptions().
   255  		InstrumentOptions().
   256  		MetricsScope().
   257  		SubScope("bootstrapper-commitlog")
   258  
   259  	log := opts.
   260  		ResultOptions().
   261  		InstrumentOptions().
   262  		Logger().
   263  		With(zap.String("bootstrapper", "commitlog"))
   264  
   265  	return &commitLogSource{
   266  		opts:  opts,
   267  		log:   log,
   268  		nowFn: opts.ResultOptions().ClockOptions().NowFn(),
   269  
   270  		inspection: inspection,
   271  
   272  		newIteratorFn:   commitlog.NewIterator,
   273  		snapshotFilesFn: fs.SnapshotFiles,
   274  		newReaderFn:     fs.NewReader,
   275  
   276  		metrics:         newCommitLogSourceMetrics(scope),
   277  		instrumentation: newInstrumentation(opts, scope, log),
   278  	}
   279  }
   280  
   281  func (s *commitLogSource) AvailableData(
   282  	ns namespace.Metadata,
   283  	shardsTimeRanges result.ShardTimeRanges,
   284  	_ bootstrap.Cache,
   285  	runOpts bootstrap.RunOptions,
   286  ) (result.ShardTimeRanges, error) {
   287  	return s.availability(ns, shardsTimeRanges, runOpts)
   288  }
   289  
   290  func (s *commitLogSource) AvailableIndex(
   291  	ns namespace.Metadata,
   292  	shardsTimeRanges result.ShardTimeRanges,
   293  	_ bootstrap.Cache,
   294  	runOpts bootstrap.RunOptions,
   295  ) (result.ShardTimeRanges, error) {
   296  	return s.availability(ns, shardsTimeRanges, runOpts)
   297  }
   298  
   299  // Read will read all commitlog files on disk, as well as as the latest snapshot for
   300  // each shard/block combination (if it exists) and merge them.
   301  // TODO(rartoul): Make this take the SnapshotMetadata files into account to reduce the
   302  // number of commitlogs / snapshots that we need to read.
   303  func (s *commitLogSource) Read(
   304  	ctx xcontext.Context,
   305  	namespaces bootstrap.Namespaces,
   306  	cache bootstrap.Cache,
   307  ) (bootstrap.NamespaceResults, error) {
   308  	instrCtx := s.instrumentation.commitLogBootstrapperSourceReadStarted(ctx)
   309  	defer instrCtx.finish()
   310  
   311  	var (
   312  		// Emit bootstrapping gauge for duration of ReadData.
   313  		doneReadingData = s.metrics.emitBootstrapping()
   314  		fsOpts          = s.opts.CommitLogOptions().FilesystemOptions()
   315  		filePathPrefix  = fsOpts.FilePathPrefix()
   316  		namespaceIter   = namespaces.Namespaces.Iter()
   317  	)
   318  	defer doneReadingData()
   319  
   320  	instrCtx.bootstrapSnapshotsStarted()
   321  	for _, elem := range namespaceIter {
   322  		ns := elem.Value()
   323  		accumulator := ns.DataAccumulator
   324  
   325  		// NB(r): Combine all shard time ranges across data and index
   326  		// so we can do in one go.
   327  		shardTimeRanges := result.NewShardTimeRanges()
   328  		// NB(bodu): Use TargetShardTimeRanges which covers the entire original target shard range
   329  		// since the commitlog bootstrapper should run for the entire bootstrappable range per shard.
   330  		shardTimeRanges.AddRanges(ns.DataRunOptions.TargetShardTimeRanges)
   331  		if ns.Metadata.Options().IndexOptions().Enabled() {
   332  			shardTimeRanges.AddRanges(ns.IndexRunOptions.TargetShardTimeRanges)
   333  		}
   334  
   335  		// Determine which snapshot files are available.
   336  		snapshotFilesByShard, err := s.snapshotFilesByShard(
   337  			ns.Metadata.ID(), filePathPrefix, shardTimeRanges)
   338  		if err != nil {
   339  			return bootstrap.NamespaceResults{}, err
   340  		}
   341  
   342  		mostRecentCompleteSnapshotByBlockShard, err := s.mostRecentSnapshotByBlockShard(
   343  			ns.Metadata, shardTimeRanges, snapshotFilesByShard)
   344  		if err != nil {
   345  			return bootstrap.NamespaceResults{}, err
   346  		}
   347  
   348  		// Start by reading any available snapshot files.
   349  		blockSize := ns.Metadata.Options().RetentionOptions().BlockSize()
   350  		for shard, tr := range shardTimeRanges.Iter() {
   351  			err := s.bootstrapShardSnapshots(
   352  				ns.Metadata, accumulator, shard, tr, blockSize,
   353  				mostRecentCompleteSnapshotByBlockShard, cache)
   354  			if err != nil {
   355  				return bootstrap.NamespaceResults{}, err
   356  			}
   357  		}
   358  	}
   359  	instrCtx.bootstrapSnapshotsCompleted()
   360  
   361  	instrCtx.readCommitLogStarted()
   362  	if !s.commitLogResult.read {
   363  		var err error
   364  		s.commitLogResult, err = s.readCommitLog(namespaces, instrCtx.span)
   365  		if err != nil {
   366  			return bootstrap.NamespaceResults{}, err
   367  		}
   368  	} else {
   369  		s.log.Debug("commit log already read in a previous pass, using previous result.")
   370  	}
   371  
   372  	bootstrapResult := bootstrap.NamespaceResults{
   373  		Results: bootstrap.NewNamespaceResultsMap(bootstrap.NamespaceResultsMapOptions{}),
   374  	}
   375  	for _, elem := range namespaceIter {
   376  		ns := elem.Value()
   377  		id := ns.Metadata.ID()
   378  		dataResult := result.NewDataBootstrapResult()
   379  		if s.commitLogResult.shouldReturnUnfulfilled {
   380  			shardTimeRanges := ns.DataRunOptions.ShardTimeRanges
   381  			dataResult = shardTimeRanges.ToUnfulfilledDataResult()
   382  		}
   383  		var indexResult result.IndexBootstrapResult
   384  		if ns.Metadata.Options().IndexOptions().Enabled() {
   385  			indexResult = result.NewIndexBootstrapResult()
   386  			if s.commitLogResult.shouldReturnUnfulfilled {
   387  				shardTimeRanges := ns.IndexRunOptions.ShardTimeRanges
   388  				indexResult = shardTimeRanges.ToUnfulfilledIndexResult()
   389  			}
   390  		}
   391  		bootstrapResult.Results.Set(id, bootstrap.NamespaceResult{
   392  			Metadata:    ns.Metadata,
   393  			Shards:      ns.Shards,
   394  			DataResult:  dataResult,
   395  			IndexResult: indexResult,
   396  		})
   397  	}
   398  	instrCtx.readCommitLogCompleted()
   399  	return bootstrapResult, nil
   400  }
   401  
   402  func (s *commitLogSource) readCommitLog(namespaces bootstrap.Namespaces, span opentracing.Span) (commitLogResult, error) {
   403  	// Setup the series accumulator pipeline.
   404  	var (
   405  		numWorkers = s.opts.AccumulateConcurrency()
   406  		workers    = make([]*accumulateWorker, 0, numWorkers)
   407  	)
   408  	for i := 0; i < numWorkers; i++ {
   409  		worker := &accumulateWorker{
   410  			inputCh: make(chan accumulateArg, workerChannelSize),
   411  		}
   412  		workers = append(workers, worker)
   413  	}
   414  	closedWorkerChannels := false
   415  	closeWorkerChannels := func() {
   416  		if closedWorkerChannels {
   417  			return
   418  		}
   419  		closedWorkerChannels = true
   420  		for _, worker := range workers {
   421  			close(worker.inputCh)
   422  		}
   423  	}
   424  	// NB(r): Ensure that channels always get closed.
   425  	defer closeWorkerChannels()
   426  
   427  	var (
   428  		namespaceIter           = namespaces.Namespaces.Iter()
   429  		namespaceResults        = make(map[string]*readNamespaceResult, len(namespaceIter))
   430  		setInitialTopologyState bool
   431  		initialTopologyState    *topology.StateSnapshot
   432  	)
   433  	for _, elem := range namespaceIter {
   434  		ns := elem.Value()
   435  
   436  		// NB(r): Combine all shard time ranges across data and index
   437  		// so we can do in one go.
   438  		shardTimeRanges := result.NewShardTimeRanges()
   439  		// NB(bodu): Use TargetShardTimeRanges which covers the entire original target shard range
   440  		// since the commitlog bootstrapper should run for the entire bootstrappable range per shard.
   441  		shardTimeRanges.AddRanges(ns.DataRunOptions.TargetShardTimeRanges)
   442  		if ns.Metadata.Options().IndexOptions().Enabled() {
   443  			shardTimeRanges.AddRanges(ns.IndexRunOptions.TargetShardTimeRanges)
   444  		}
   445  
   446  		namespaceResults[ns.Metadata.ID().String()] = &readNamespaceResult{
   447  			namespace:               ns,
   448  			dataAndIndexShardRanges: shardTimeRanges,
   449  		}
   450  
   451  		// Make the initial topology state available.
   452  		if !setInitialTopologyState {
   453  			setInitialTopologyState = true
   454  			initialTopologyState = ns.DataRunOptions.RunOptions.InitialTopologyState()
   455  		}
   456  	}
   457  
   458  	// Setup the commit log iterator.
   459  	var (
   460  		iterOpts = commitlog.IteratorOpts{
   461  			CommitLogOptions:    s.opts.CommitLogOptions(),
   462  			FileFilterPredicate: s.readCommitLogFilePredicate,
   463  			// NB(r): ReturnMetadataAsRef used to all series metadata as
   464  			// references instead of pulling from pool and allocating,
   465  			// which means need to not hold onto any references returned
   466  			// from a call to the commit log read log entry call.
   467  			ReturnMetadataAsRef: true,
   468  		}
   469  		datapointsSkippedNotBootstrappingNamespace = 0
   470  		datapointsSkippedNotBootstrappingShard     = 0
   471  		datapointsSkippedShardNoLongerOwned        = 0
   472  		startCommitLogsRead                        = s.nowFn()
   473  		encounteredCorruptData                     = false
   474  	)
   475  	s.log.Info("read commit logs start")
   476  	span.LogEvent("read_commitlogs_start")
   477  	defer func() {
   478  		datapointsRead := 0
   479  		for _, worker := range workers {
   480  			datapointsRead += worker.datapointsRead
   481  		}
   482  		s.log.Info("read commit logs done",
   483  			zap.Duration("took", s.nowFn().Sub(startCommitLogsRead)),
   484  			zap.Int("datapointsRead", datapointsRead),
   485  			zap.Int("datapointsSkippedNotBootstrappingNamespace", datapointsSkippedNotBootstrappingNamespace),
   486  			zap.Int("datapointsSkippedNotBootstrappingShard", datapointsSkippedNotBootstrappingShard),
   487  			zap.Int("datapointsSkippedShardNoLongerOwned", datapointsSkippedShardNoLongerOwned))
   488  		span.LogEvent("read_commitlogs_done")
   489  	}()
   490  
   491  	iter, corruptFiles, err := s.newIteratorFn(iterOpts)
   492  	if err != nil {
   493  		err = fmt.Errorf("unable to create commit log iterator: %v", err)
   494  		return commitLogResult{}, err
   495  	}
   496  
   497  	if len(corruptFiles) > 0 {
   498  		s.logAndEmitCorruptFiles(corruptFiles)
   499  		encounteredCorruptData = true
   500  	}
   501  
   502  	defer iter.Close()
   503  
   504  	// Spin up numWorkers background go-routines to handle accumulation. This must
   505  	// happen before we start reading to prevent infinitely blocking writes to
   506  	// the worker channels.
   507  	var wg sync.WaitGroup
   508  	for _, worker := range workers {
   509  		worker := worker
   510  		wg.Add(1)
   511  		go func() {
   512  			s.startAccumulateWorker(worker)
   513  			wg.Done()
   514  		}()
   515  	}
   516  
   517  	var (
   518  		// NB(r): Use pointer type for the namespaces so we don't have to
   519  		// memcopy the large namespace context struct to the work channel and
   520  		// can pass by pointer.
   521  		// For the commit log series map we use by value since it grows
   522  		// large in size and we want to avoid allocating a struct per series
   523  		// read and just have a by value struct stored in the map (also makes
   524  		// reusing memory set aside on a per series level between commit
   525  		// log files much easier to do).
   526  		commitLogNamespaces    []*bootstrapNamespace
   527  		commitLogSeries        = make(map[seriesMapKey]seriesMapEntry)
   528  		workerEnqueue          = 0
   529  		tagDecoder             = s.opts.CommitLogOptions().FilesystemOptions().TagDecoderPool().Get()
   530  		tagDecoderCheckedBytes = checked.NewBytes(nil, nil)
   531  	)
   532  	tagDecoderCheckedBytes.IncRef()
   533  
   534  	// Read and accumulate all the log entries in the commit log that we need
   535  	// to read.
   536  	var lastFileReadID uint64
   537  	for iter.Next() {
   538  		s.metrics.commitLogEntriesRead.Inc(1)
   539  		entry := iter.Current()
   540  
   541  		currFileReadID := entry.Metadata.FileReadID
   542  		if currFileReadID != lastFileReadID {
   543  			// NB(r): If switched between files, we can reuse slice and
   544  			// map which is useful so map doesn't grow infinitely.
   545  			for k := range commitLogSeries {
   546  				delete(commitLogSeries, k)
   547  			}
   548  			lastFileReadID = currFileReadID
   549  		}
   550  
   551  		// First lookup series, if not found we are guaranteed to have
   552  		// the series metadata returned by the commit log reader.
   553  		seriesKey := seriesMapKey{
   554  			fileReadID:  entry.Metadata.FileReadID,
   555  			uniqueIndex: entry.Metadata.SeriesUniqueIndex,
   556  		}
   557  
   558  		seriesEntry, ok := commitLogSeries[seriesKey]
   559  		if !ok {
   560  			// Resolve the namespace.
   561  			var (
   562  				nsID      = entry.Series.Namespace
   563  				nsIDBytes = nsID.Bytes()
   564  				ns        *bootstrapNamespace
   565  			)
   566  			for _, elem := range commitLogNamespaces {
   567  				if bytes.Equal(elem.namespaceID, nsIDBytes) {
   568  					ns = elem
   569  					break
   570  				}
   571  			}
   572  			if ns == nil {
   573  				// NB(r): Need to create an entry into our namespaces, this will happen
   574  				// at most once per commit log file read and unique namespace.
   575  				nsResult, ok := namespaceResults[nsID.String()]
   576  				// Take a copy so that not taking ref to reused bytes from the commit log.
   577  				nsIDCopy := append([]byte(nil), nsIDBytes...)
   578  				if !ok {
   579  					// Not bootstrapping this namespace.
   580  					ns = &bootstrapNamespace{
   581  						namespaceID:   nsIDCopy,
   582  						bootstrapping: false,
   583  					}
   584  				} else {
   585  					// Bootstrapping this namespace.
   586  					nsMetadata := nsResult.namespace.Metadata
   587  					ns = &bootstrapNamespace{
   588  						namespaceID:             nsIDCopy,
   589  						bootstrapping:           true,
   590  						dataAndIndexShardRanges: nsResult.dataAndIndexShardRanges,
   591  						namespace:               nsMetadata,
   592  						namespaceContext:        namespace.NewContextFrom(nsMetadata),
   593  						dataBlockSize:           nsMetadata.Options().RetentionOptions().BlockSize(),
   594  						accumulator:             nsResult.namespace.DataAccumulator,
   595  					}
   596  				}
   597  				// Append for quick re-lookup with other series.
   598  				commitLogNamespaces = append(commitLogNamespaces, ns)
   599  			}
   600  			if !ns.bootstrapping {
   601  				// NB(r): Just set the series map entry to the memoized
   602  				// fact that we are not bootstrapping this namespace.
   603  				seriesEntry = seriesMapEntry{
   604  					namespace: ns,
   605  				}
   606  			} else {
   607  				// Resolve the series in the accumulator.
   608  				accumulator := ns.accumulator
   609  
   610  				var tagIter ident.TagIterator
   611  				if len(entry.Series.EncodedTags) > 0 {
   612  					tagDecoderCheckedBytes.Reset(entry.Series.EncodedTags)
   613  					tagDecoder.Reset(tagDecoderCheckedBytes)
   614  					tagIter = tagDecoder
   615  				} else {
   616  					// NB(r): Always expect a tag iterator in checkout series.
   617  					tagIter = ident.EmptyTagIterator
   618  				}
   619  
   620  				// Check out the series for writing, no need for concurrency
   621  				// as commit log bootstrapper does not perform parallel
   622  				// checking out of series.
   623  				series, owned, err := accumulator.CheckoutSeriesWithoutLock(
   624  					entry.Series.Shard,
   625  					entry.Series.ID,
   626  					tagIter)
   627  				if err != nil {
   628  					if !owned {
   629  						// If we encounter a log entry for a shard that we're
   630  						// not responsible for, skip this entry. This can occur
   631  						// when a topology change happens and we bootstrap from
   632  						// a commit log which contains this data.
   633  						commitLogSeries[seriesKey] = seriesMapEntry{shardNoLongerOwned: true}
   634  						continue
   635  					}
   636  					return commitLogResult{}, err
   637  				}
   638  
   639  				seriesEntry = seriesMapEntry{
   640  					namespace: ns,
   641  					series:    series,
   642  				}
   643  			}
   644  
   645  			commitLogSeries[seriesKey] = seriesEntry
   646  		}
   647  
   648  		// If series is no longer owned, then we can safely skip trying to
   649  		// bootstrap the result.
   650  		if seriesEntry.shardNoLongerOwned {
   651  			datapointsSkippedShardNoLongerOwned++
   652  			continue
   653  		}
   654  
   655  		// If not bootstrapping this namespace then skip this result.
   656  		if !seriesEntry.namespace.bootstrapping {
   657  			datapointsSkippedNotBootstrappingNamespace++
   658  			continue
   659  		}
   660  
   661  		// If not bootstrapping shard for this series then also skip.
   662  		// NB(r): This can occur when a topology change happens then we
   663  		// bootstrap from the commit log data that the node no longer owns.
   664  		shard := seriesEntry.series.Shard
   665  		_, ok = seriesEntry.namespace.dataAndIndexShardRanges.Get(shard)
   666  		if !ok {
   667  			datapointsSkippedNotBootstrappingShard++
   668  			continue
   669  		}
   670  
   671  		arg := accumulateArg{
   672  			namespace: seriesEntry.namespace,
   673  			series:    seriesEntry.series,
   674  			shard:     seriesEntry.series.Shard,
   675  			dp:        entry.Datapoint,
   676  			unit:      entry.Unit,
   677  		}
   678  
   679  		annotationLen := len(entry.Annotation)
   680  		if annotationLen > 0 {
   681  			// Use the predefined buffer if the annotation fits in it.
   682  			if annotationLen <= len(arg.shortAnnotation) {
   683  				copy(arg.shortAnnotation[:], entry.Annotation)
   684  				arg.shortAnnotationLen = uint8(annotationLen)
   685  			} else {
   686  				// Otherwise allocate.
   687  				arg.longAnnotation = append(make([]byte, 0, annotationLen), entry.Annotation...)
   688  			}
   689  		}
   690  
   691  		// Distribute work.
   692  		// NB(r): In future we could batch a few points together before sending
   693  		// to a channel to alleviate lock contention/stress on the channels.
   694  		workerEnqueue++
   695  		worker := workers[workerEnqueue%numWorkers]
   696  		worker.inputCh <- arg
   697  	}
   698  
   699  	if iterErr := iter.Err(); iterErr != nil {
   700  		// Log the error and mark that we encountered corrupt data, but don't
   701  		// return the error because we want to give the peers bootstrapper the
   702  		// opportunity to repair the data instead of failing the bootstrap
   703  		// altogether.
   704  		s.log.Error("error in commitlog iterator", zap.Error(iterErr))
   705  		s.metrics.corruptCommitlogFile.Inc(1)
   706  		encounteredCorruptData = true
   707  	}
   708  
   709  	// Close the worker channels since we've enqueued all required data.
   710  	closeWorkerChannels()
   711  
   712  	// Block until all required data from the commit log has been read and
   713  	// accumulated by the worker goroutines.
   714  	wg.Wait()
   715  
   716  	// Log the outcome and calculate if required to return unfulfilled.
   717  	s.logAccumulateOutcome(workers, iter)
   718  	shouldReturnUnfulfilled, err := s.shouldReturnUnfulfilled(
   719  		workers, encounteredCorruptData, initialTopologyState)
   720  	if err != nil {
   721  		return commitLogResult{}, err
   722  	}
   723  	return commitLogResult{shouldReturnUnfulfilled: shouldReturnUnfulfilled, read: true}, nil
   724  }
   725  
   726  func (s *commitLogSource) snapshotFilesByShard(
   727  	nsID ident.ID,
   728  	filePathPrefix string,
   729  	shardsTimeRanges result.ShardTimeRanges,
   730  ) (map[uint32]fs.FileSetFilesSlice, error) {
   731  	snapshotFilesByShard := map[uint32]fs.FileSetFilesSlice{}
   732  	for shard := range shardsTimeRanges.Iter() {
   733  		snapshotFiles, err := s.snapshotFilesFn(filePathPrefix, nsID, shard)
   734  		if err != nil {
   735  			return nil, err
   736  		}
   737  		snapshotFilesByShard[shard] = snapshotFiles
   738  	}
   739  
   740  	return snapshotFilesByShard, nil
   741  }
   742  
   743  // mostRecentCompleteSnapshotByBlockShard returns a
   744  // map[xtime.UnixNano]map[uint32]fs.FileSetFile with the contract that
   745  // for each shard/block combination in shardsTimeRanges, an entry will
   746  // exist in the map such that FileSetFile.CachedSnapshotTime is the
   747  // actual cached snapshot time, or the blockStart.
   748  func (s *commitLogSource) mostRecentCompleteSnapshotByBlockShard(
   749  	shardsTimeRanges result.ShardTimeRanges,
   750  	blockSize time.Duration,
   751  	snapshotFilesByShard map[uint32]fs.FileSetFilesSlice,
   752  	fsOpts fs.Options,
   753  ) map[xtime.UnixNano]map[uint32]fs.FileSetFile {
   754  	var (
   755  		minBlock, maxBlock              = shardsTimeRanges.MinMax()
   756  		mostRecentSnapshotsByBlockShard = map[xtime.UnixNano]map[uint32]fs.FileSetFile{}
   757  	)
   758  
   759  	for currBlockStart := minBlock.Truncate(blockSize); currBlockStart.Before(maxBlock); currBlockStart = currBlockStart.Add(blockSize) {
   760  		for shard := range shardsTimeRanges.Iter() {
   761  			// Anonymous func for easier clean up using defer.
   762  			func() {
   763  				var (
   764  					mostRecentSnapshot fs.FileSetFile
   765  				)
   766  
   767  				defer func() {
   768  					existing := mostRecentSnapshotsByBlockShard[currBlockStart]
   769  					if existing == nil {
   770  						existing = map[uint32]fs.FileSetFile{}
   771  					}
   772  
   773  					if mostRecentSnapshot.IsZero() {
   774  						// If we were unable to determine the most recent snapshot time for a given
   775  						// shard/blockStart combination, then just fall back to using the blockStart
   776  						// time as that will force us to read the entire commit log for that duration.
   777  						mostRecentSnapshot.CachedSnapshotTime = currBlockStart
   778  					}
   779  					existing[shard] = mostRecentSnapshot
   780  					mostRecentSnapshotsByBlockShard[currBlockStart] = existing
   781  				}()
   782  
   783  				snapshotFiles, ok := snapshotFilesByShard[shard]
   784  				if !ok {
   785  					// If there are no snapshot files for this shard, then rely on
   786  					// the defer to fallback to using the block start time.
   787  					return
   788  				}
   789  
   790  				mostRecentSnapshotVolume, ok := snapshotFiles.LatestVolumeForBlock(currBlockStart)
   791  				if !ok {
   792  					// If there are no complete snapshot files for this block, then rely on
   793  					// the defer to fallback to using the block start time.
   794  					return
   795  				}
   796  
   797  				// Make sure we're able to read the snapshot time. This will also set the
   798  				// CachedSnapshotTime field so that we can rely upon it from here on out.
   799  				_, _, err := mostRecentSnapshotVolume.SnapshotTimeAndID()
   800  				if err != nil {
   801  					namespace := mostRecentSnapshot.ID.Namespace
   802  					if namespace == nil {
   803  						namespace = ident.StringID("<nil>")
   804  					}
   805  					s.log.
   806  						With(
   807  							zap.Stringer("namespace", namespace),
   808  							zap.Time("blockStart", mostRecentSnapshot.ID.BlockStart.ToTime()),
   809  							zap.Uint32("shard", mostRecentSnapshot.ID.Shard),
   810  							zap.Int("index", mostRecentSnapshot.ID.VolumeIndex),
   811  							zap.Strings("filepaths", mostRecentSnapshot.AbsoluteFilePaths),
   812  							zap.Error(err),
   813  						).
   814  						Error("error resolving snapshot time for snapshot file")
   815  
   816  					// If we couldn't determine the snapshot time for the snapshot file, then rely
   817  					// on the defer to fallback to using the block start time.
   818  					return
   819  				}
   820  
   821  				mostRecentSnapshot = mostRecentSnapshotVolume
   822  			}()
   823  		}
   824  	}
   825  
   826  	return mostRecentSnapshotsByBlockShard
   827  }
   828  
   829  func (s *commitLogSource) bootstrapShardSnapshots(
   830  	ns namespace.Metadata,
   831  	accumulator bootstrap.NamespaceDataAccumulator,
   832  	shard uint32,
   833  	shardTimeRanges xtime.Ranges,
   834  	blockSize time.Duration,
   835  	mostRecentCompleteSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile,
   836  	cache bootstrap.Cache,
   837  ) error {
   838  	// NB(bodu): We use info files on disk to check if a snapshot should be loaded in as cold or warm.
   839  	// We do this instead of cross refing blockstarts and current time to handle the case of bootstrapping a
   840  	// once warm block start after a node has been shut down for a long time. We consider all block starts we
   841  	// haven't flushed data for yet a warm block start.
   842  	readInfoFilesResults, err := cache.InfoFilesForShard(ns, shard)
   843  	if err != nil {
   844  		return err
   845  	}
   846  	shardBlockStartsOnDisk := make(map[xtime.UnixNano]struct{})
   847  	for _, result := range readInfoFilesResults {
   848  		if err := result.Err.Error(); err != nil {
   849  			// If we couldn't read the info files then keep going to be consistent
   850  			// with the way the db shard updates its flush states in UpdateFlushStates().
   851  			s.log.Error("unable to read info files in commit log bootstrap",
   852  				zap.Uint32("shard", shard),
   853  				zap.Stringer("namespace", ns.ID()),
   854  				zap.String("filepath", result.Err.Filepath()),
   855  				zap.Error(err))
   856  			continue
   857  		}
   858  		info := result.Info
   859  		at := xtime.FromNanoseconds(info.BlockStart)
   860  		shardBlockStartsOnDisk[xtime.ToUnixNano(at)] = struct{}{}
   861  	}
   862  
   863  	rangeIter := shardTimeRanges.Iter()
   864  	for rangeIter.Next() {
   865  		var (
   866  			currRange             = rangeIter.Value()
   867  			currRangeDuration     = currRange.End.Sub(currRange.Start)
   868  			isMultipleOfBlockSize = currRangeDuration%blockSize == 0
   869  		)
   870  
   871  		if !isMultipleOfBlockSize {
   872  			return fmt.Errorf(
   873  				"received bootstrap range that is not multiple of blockSize, blockSize: %d, start: %s, end: %s",
   874  				blockSize, currRange.End.String(), currRange.Start.String(),
   875  			)
   876  		}
   877  
   878  		for blockStart := currRange.Start.Truncate(blockSize); blockStart.Before(currRange.End); blockStart = blockStart.Add(blockSize) {
   879  			snapshotsForBlock := mostRecentCompleteSnapshotByBlockShard[blockStart]
   880  			mostRecentCompleteSnapshotForShardBlock := snapshotsForBlock[shard]
   881  
   882  			if mostRecentCompleteSnapshotForShardBlock.CachedSnapshotTime.Equal(blockStart) ||
   883  				// Should never happen
   884  				mostRecentCompleteSnapshotForShardBlock.IsZero() {
   885  				// There is no snapshot file for this time, and even if there was, there would
   886  				// be no point in reading it. In this specific case its not an error scenario
   887  				// because the fact that snapshotTime == blockStart means we already accounted
   888  				// for the fact that this snapshot did not exist when we were deciding which
   889  				// commit logs to read.
   890  				s.log.Debug("no snapshots for shard and blockStart",
   891  					zap.Uint32("shard", shard), zap.Time("blockStart", blockStart.ToTime()))
   892  				continue
   893  			}
   894  
   895  			writeType := series.WarmWrite
   896  			if _, ok := shardBlockStartsOnDisk[blockStart]; ok {
   897  				writeType = series.ColdWrite
   898  			}
   899  			if err := s.bootstrapShardBlockSnapshot(
   900  				ns, accumulator, shard, blockStart, blockSize,
   901  				mostRecentCompleteSnapshotForShardBlock, writeType); err != nil {
   902  				return err
   903  			}
   904  		}
   905  	}
   906  
   907  	return nil
   908  }
   909  
   910  func (s *commitLogSource) bootstrapShardBlockSnapshot(
   911  	ns namespace.Metadata,
   912  	accumulator bootstrap.NamespaceDataAccumulator,
   913  	shard uint32,
   914  	blockStart xtime.UnixNano,
   915  	blockSize time.Duration,
   916  	mostRecentCompleteSnapshot fs.FileSetFile,
   917  	writeType series.WriteType,
   918  ) error {
   919  	var (
   920  		bOpts      = s.opts.ResultOptions()
   921  		blOpts     = bOpts.DatabaseBlockOptions()
   922  		blocksPool = blOpts.DatabaseBlockPool()
   923  		bytesPool  = blOpts.BytesPool()
   924  		fsOpts     = s.opts.CommitLogOptions().FilesystemOptions()
   925  		nsCtx      = namespace.NewContextFrom(ns)
   926  		numWorkers = s.opts.AccumulateConcurrency()
   927  	)
   928  
   929  	// Bootstrap the snapshot file.
   930  	reader, err := s.newReaderFn(bytesPool, fsOpts)
   931  	if err != nil {
   932  		return err
   933  	}
   934  
   935  	err = reader.Open(fs.DataReaderOpenOptions{
   936  		Identifier: fs.FileSetFileIdentifier{
   937  			Namespace:   ns.ID(),
   938  			BlockStart:  blockStart,
   939  			Shard:       shard,
   940  			VolumeIndex: mostRecentCompleteSnapshot.ID.VolumeIndex,
   941  		},
   942  		FileSetType: persist.FileSetSnapshotType,
   943  	})
   944  	if err != nil {
   945  		return err
   946  	}
   947  	defer func() {
   948  		err := reader.Close()
   949  		if err != nil {
   950  			s.log.Error("error closing reader for shard",
   951  				zap.Uint32("shard", shard),
   952  				zap.Time("blockStart", blockStart.ToTime()),
   953  				zap.Int("volume", mostRecentCompleteSnapshot.ID.VolumeIndex),
   954  				zap.Error(err))
   955  		}
   956  	}()
   957  
   958  	s.log.Debug("reading snapshot for shard",
   959  		zap.Uint32("shard", shard),
   960  		zap.Time("blockStart", blockStart.ToTime()),
   961  		zap.Int("volume", mostRecentCompleteSnapshot.ID.VolumeIndex))
   962  
   963  	worker := &readSeriesBlocksWorker{
   964  		dataCh:      make(chan seriesBlock, readSeriesBlocksWorkerChannelSize),
   965  		reader:      reader,
   966  		shard:       shard,
   967  		accumulator: accumulator,
   968  		blocksPool:  blocksPool,
   969  		blockStart:  blockStart,
   970  		blockSize:   blockSize,
   971  		nsCtx:       nsCtx,
   972  	}
   973  
   974  	errs, ctx := errgroup.WithContext(context.Background())
   975  	errs.Go(func() error {
   976  		return worker.readSeriesBlocks(ctx)
   977  	})
   978  
   979  	for i := 0; i < numWorkers; i++ {
   980  		errs.Go(func() error {
   981  			return s.loadBlocks(worker.dataCh, writeType)
   982  		})
   983  	}
   984  
   985  	return errs.Wait()
   986  }
   987  
   988  func (s *commitLogSource) loadBlocks(dataCh <-chan seriesBlock, writeType series.WriteType) error {
   989  	for seriesBlock := range dataCh {
   990  		// Load into series.
   991  		seriesRef, err := seriesBlock.resolver.SeriesRef()
   992  		if err != nil {
   993  			return fmt.Errorf("(commitlog) unable to resolve series ref: %w", err)
   994  		}
   995  
   996  		if err := seriesRef.LoadBlock(seriesBlock.block, writeType); err != nil {
   997  			return err
   998  		}
   999  	}
  1000  	return nil
  1001  }
  1002  
  1003  func (s *commitLogSource) mostRecentSnapshotByBlockShard(
  1004  	ns namespace.Metadata,
  1005  	shardsTimeRanges result.ShardTimeRanges,
  1006  	snapshotFilesByShard map[uint32]fs.FileSetFilesSlice,
  1007  ) (
  1008  	map[xtime.UnixNano]map[uint32]fs.FileSetFile,
  1009  	error,
  1010  ) {
  1011  	blockSize := ns.Options().RetentionOptions().BlockSize()
  1012  
  1013  	mostRecentCompleteSnapshotByBlockShard := s.mostRecentCompleteSnapshotByBlockShard(
  1014  		shardsTimeRanges, blockSize, snapshotFilesByShard, s.opts.CommitLogOptions().FilesystemOptions())
  1015  	for block, mostRecentByShard := range mostRecentCompleteSnapshotByBlockShard {
  1016  		for shard, mostRecent := range mostRecentByShard {
  1017  
  1018  			if mostRecent.CachedSnapshotTime.IsZero() {
  1019  				// Should never happen.
  1020  				return nil, instrument.InvariantErrorf(
  1021  					"shard: %d and block: %s had zero value for most recent snapshot time",
  1022  					shard, block.ToTime().String())
  1023  			}
  1024  
  1025  			s.log.Debug("most recent snapshot for block",
  1026  				zap.Time("blockStart", block.ToTime()),
  1027  				zap.Uint32("shard", shard),
  1028  				zap.Time("mostRecent", mostRecent.CachedSnapshotTime.ToTime()))
  1029  		}
  1030  	}
  1031  
  1032  	return mostRecentCompleteSnapshotByBlockShard, nil
  1033  }
  1034  
  1035  // TODO(rartoul): Refactor this to take the SnapshotMetadata files into account to reduce
  1036  // the number of commitlog files that need to be read.
  1037  func (s *commitLogSource) readCommitLogFilePredicate(f commitlog.FileFilterInfo) bool {
  1038  	// Read all the commitlog files that were available on disk before the node started
  1039  	// accepting writes.
  1040  	commitlogFilesPresentBeforeStart := s.inspection.CommitLogFilesSet()
  1041  	if f.IsCorrupt {
  1042  		// Corrupt files that existed on disk before the node started should be included so
  1043  		// that the commitlog bootstrapper can detect them and determine if it will return
  1044  		// unfulfilled or ignore them.
  1045  		//
  1046  		// Corrupt files that did not exist on disk before the node started should always be
  1047  		// ignored since they have no impact on the bootstrapping process and likely only
  1048  		// appear corrupt because they were just created recently by the current node as
  1049  		// its alreadying accepting writes at this point.
  1050  		_, ok := commitlogFilesPresentBeforeStart[f.Err.Path()]
  1051  		return ok
  1052  	}
  1053  	// Only attempt to read commitlog files that were present on disk before the node started.
  1054  	// If a commitlog file was not present when the node started then it was created once the
  1055  	// node began accepting writes and the data is already in memory.
  1056  	_, ok := commitlogFilesPresentBeforeStart[f.File.FilePath]
  1057  	return ok
  1058  }
  1059  
  1060  func (s *commitLogSource) startAccumulateWorker(worker *accumulateWorker) {
  1061  	ctx := xcontext.NewBackground()
  1062  	defer ctx.Close()
  1063  
  1064  	reusableAnnotation := make([]byte, 0, ts.OptimizedAnnotationLen)
  1065  
  1066  	for input := range worker.inputCh {
  1067  		var (
  1068  			namespace = input.namespace
  1069  			entry     = input.series
  1070  			dp        = input.dp
  1071  			unit      = input.unit
  1072  		)
  1073  
  1074  		annotation := input.longAnnotation
  1075  		if input.shortAnnotationLen > 0 {
  1076  			reusableAnnotation = append(reusableAnnotation[:0], input.shortAnnotation[:input.shortAnnotationLen]...)
  1077  			annotation = reusableAnnotation
  1078  		}
  1079  		worker.datapointsRead++
  1080  
  1081  		ref, err := entry.Resolver.SeriesRef()
  1082  		if err != nil {
  1083  			if worker.numErrors == 0 {
  1084  				s.log.Error("failed to resolve series ref", zap.Error(err))
  1085  			} else {
  1086  				// Always write a debug log, most of these will go nowhere if debug
  1087  				// logging not enabled however.
  1088  				s.log.Debug("failed to resolve series ref", zap.Error(err))
  1089  			}
  1090  			worker.numErrors++
  1091  			continue
  1092  		}
  1093  
  1094  		_, _, err = ref.Write(ctx, dp.TimestampNanos, dp.Value,
  1095  			unit, annotation, series.WriteOptions{
  1096  				SchemaDesc:         namespace.namespaceContext.Schema,
  1097  				BootstrapWrite:     true,
  1098  				SkipOutOfRetention: true,
  1099  			})
  1100  		if err != nil {
  1101  			// NB(r): Only log first error per worker since this could be very
  1102  			// noisy if it actually fails for "most" writes.
  1103  			if worker.numErrors == 0 {
  1104  				s.log.Error("failed to write commit log entry", zap.Error(err))
  1105  			} else {
  1106  				// Always write a debug log, most of these will go nowhere if debug
  1107  				// logging not enabled however.
  1108  				s.log.Debug("failed to write commit log entry", zap.Error(err))
  1109  			}
  1110  			worker.numErrors++
  1111  		}
  1112  	}
  1113  }
  1114  
  1115  func (s *commitLogSource) logAccumulateOutcome(
  1116  	workers []*accumulateWorker,
  1117  	iter commitlog.Iterator,
  1118  ) {
  1119  	errs := 0
  1120  	for _, worker := range workers {
  1121  		errs += worker.numErrors
  1122  	}
  1123  	if errs > 0 {
  1124  		s.log.Error("error bootstrapping from commit log", zap.Int("accumulateErrors", errs))
  1125  	}
  1126  	if err := iter.Err(); err != nil {
  1127  		s.log.Error("error reading commit log", zap.Error(err))
  1128  	}
  1129  }
  1130  
  1131  // If we encountered any corrupt data and there is a possibility of the
  1132  // peers bootstrapper being able to correct it, we want to mark the entire range
  1133  // as unfulfilled so the peers bootstrapper can attempt a repair, but keep
  1134  // the data we read from the commit log as well in case the peers
  1135  // bootstrapper is unable to satisfy the bootstrap because all peers are
  1136  // down or if the commitlog contained data that the peers do not have.
  1137  func (s commitLogSource) shouldReturnUnfulfilled(
  1138  	workers []*accumulateWorker,
  1139  	encounteredCorruptData bool,
  1140  	initialTopologyState *topology.StateSnapshot,
  1141  ) (bool, error) {
  1142  	errs := 0
  1143  	for _, worker := range workers {
  1144  		errs += worker.numErrors
  1145  	}
  1146  	if errs > 0 {
  1147  		return true, fmt.Errorf("return unfulfilled: %d accumulate errors", errs)
  1148  	}
  1149  
  1150  	if !s.opts.ReturnUnfulfilledForCorruptCommitLogFiles() {
  1151  		s.log.Info("returning not-unfulfilled: ReturnUnfulfilledForCorruptCommitLogFiles is false")
  1152  		return false, nil
  1153  	}
  1154  
  1155  	if !encounteredCorruptData {
  1156  		s.log.Info("returning not-unfulfilled: no corrupt data encountered")
  1157  		return false, nil
  1158  	}
  1159  
  1160  	shardsReplicated := s.shardsReplicated(initialTopologyState)
  1161  	if !shardsReplicated {
  1162  		s.log.Info("returning not-unfulfilled: replication is not enabled")
  1163  	}
  1164  	return shardsReplicated, nil
  1165  }
  1166  
  1167  func (s *commitLogSource) logAndEmitCorruptFiles(
  1168  	corruptFiles []commitlog.ErrorWithPath) {
  1169  	for _, f := range corruptFiles {
  1170  		s.log.Error("opting to skip commit log due to corruption", zap.String("error", f.Error()))
  1171  		s.metrics.corruptCommitlogFile.Inc(1)
  1172  	}
  1173  }
  1174  
  1175  // The commitlog bootstrapper determines availability primarily by checking if the
  1176  // origin host has ever reached the "Available" state for the shard that is being
  1177  // bootstrapped. If not, then it can't provide data for that shard because it doesn't
  1178  // have all of it by definition.
  1179  func (s *commitLogSource) availability(
  1180  	ns namespace.Metadata,
  1181  	shardsTimeRanges result.ShardTimeRanges,
  1182  	runOpts bootstrap.RunOptions,
  1183  ) (result.ShardTimeRanges, error) {
  1184  	var (
  1185  		topoState                = runOpts.InitialTopologyState()
  1186  		availableShardTimeRanges = result.NewShardTimeRanges()
  1187  	)
  1188  
  1189  	for shardIDUint := range shardsTimeRanges.Iter() {
  1190  		shardID := topology.ShardID(shardIDUint)
  1191  		hostShardStates, ok := topoState.ShardStates[shardID]
  1192  		if !ok {
  1193  			// This shard was not part of the topology when the bootstrapping
  1194  			// process began.
  1195  			continue
  1196  		}
  1197  
  1198  		originHostShardState, ok := hostShardStates[topology.HostID(topoState.Origin.ID())]
  1199  		if !ok {
  1200  			errMsg := fmt.Sprintf("initial topology state does not contain shard state for origin node and shard: %d", shardIDUint)
  1201  			iOpts := s.opts.CommitLogOptions().InstrumentOptions()
  1202  			instrument.EmitAndLogInvariantViolation(iOpts, func(l *zap.Logger) {
  1203  				l.Error(errMsg)
  1204  			})
  1205  			return nil, errors.New(errMsg)
  1206  		}
  1207  
  1208  		originShardState := originHostShardState.ShardState
  1209  		switch originShardState {
  1210  		// In the Initializing state we have to assume that the commit log
  1211  		// is missing data and can't satisfy the bootstrap request.
  1212  		case shard.Initializing:
  1213  		// In the Leaving and Available case, we assume that the commit log contains
  1214  		// all the data required to satisfy the bootstrap request because the node
  1215  		// had (at some point) been completely bootstrapped for the requested shard.
  1216  		// This doesn't mean that the node can't be missing any data or wasn't down
  1217  		// for some period of time and missing writes in a multi-node deployment, it
  1218  		// only means that technically the node has successfully taken ownership of
  1219  		// the data for this shard and made it to a "bootstrapped" state which is
  1220  		// all that is required to maintain our cluster-level consistency guarantees.
  1221  		case shard.Leaving:
  1222  			fallthrough
  1223  		case shard.Available:
  1224  			// Assume that we can bootstrap any requested time range, which is valid as
  1225  			// long as the FS bootstrapper precedes the commit log bootstrapper.
  1226  			// TODO(rartoul): Once we make changes to the bootstrapping interfaces
  1227  			// to distinguish between "unfulfilled" data and "corrupt" data, then
  1228  			// modify this to only say the commit log bootstrapper can fullfil
  1229  			// "unfulfilled" data, but not corrupt data.
  1230  			if tr, ok := shardsTimeRanges.Get(shardIDUint); ok {
  1231  				availableShardTimeRanges.Set(shardIDUint, tr)
  1232  			}
  1233  		case shard.Unknown:
  1234  			fallthrough
  1235  		default:
  1236  			return result.NewShardTimeRanges(), fmt.Errorf("unknown shard state: %v", originShardState)
  1237  		}
  1238  	}
  1239  
  1240  	return availableShardTimeRanges, nil
  1241  }
  1242  
  1243  func (s *commitLogSource) shardsReplicated(
  1244  	initialTopologyState *topology.StateSnapshot,
  1245  ) bool {
  1246  	// In any situation where we could actually stream data from our peers
  1247  	// the replication factor would be 2 or larger which means that the
  1248  	// value of majorityReplicas would be 2 or larger also. This heuristic can
  1249  	// only be used to infer whether the replication factor is 1 or larger, but
  1250  	// cannot be used to determine what the actual replication factor is in all
  1251  	// situations because it can be ambiguous. For example, both R.F 2 and 3 will
  1252  	// have majority replica values of 2.
  1253  	majorityReplicas := initialTopologyState.MajorityReplicas
  1254  	return majorityReplicas > 1
  1255  }