github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/flush.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 storage
    22  
    23  import (
    24  	"errors"
    25  	"fmt"
    26  	"sync"
    27  
    28  	"github.com/m3db/m3/src/dbnode/persist"
    29  	"github.com/m3db/m3/src/dbnode/persist/fs/commitlog"
    30  	"github.com/m3db/m3/src/dbnode/retention"
    31  	"github.com/m3db/m3/src/x/clock"
    32  	xerrors "github.com/m3db/m3/src/x/errors"
    33  	xtime "github.com/m3db/m3/src/x/time"
    34  
    35  	"github.com/pborman/uuid"
    36  	"github.com/uber-go/tally"
    37  	"go.uber.org/atomic"
    38  	"go.uber.org/zap"
    39  )
    40  
    41  var errFlushOperationsInProgress = errors.New("flush operations already in progress")
    42  
    43  type flushManagerState int
    44  
    45  const (
    46  	flushManagerIdle flushManagerState = iota
    47  	// flushManagerNotIdle is used to protect the flush manager from concurrent use
    48  	// when we haven't begun either a flush or snapshot.
    49  	flushManagerNotIdle
    50  	flushManagerFlushInProgress
    51  	flushManagerSnapshotInProgress
    52  	flushManagerIndexFlushInProgress
    53  )
    54  
    55  type flushManagerMetrics struct {
    56  	isFlushing      tally.Gauge
    57  	isSnapshotting  tally.Gauge
    58  	isIndexFlushing tally.Gauge
    59  	// This is a "debug" metric for making sure that the snapshotting process
    60  	// is not overly aggressive.
    61  	maxBlocksSnapshottedByNamespace tally.Gauge
    62  	dataWarmFlushDuration           tally.Timer
    63  	dataSnapshotDuration            tally.Timer
    64  	indexFlushDuration              tally.Timer
    65  	commitLogRotationDuration       tally.Timer
    66  }
    67  
    68  func newFlushManagerMetrics(scope tally.Scope) flushManagerMetrics {
    69  	return flushManagerMetrics{
    70  		isFlushing:                      scope.Gauge("flush"),
    71  		isSnapshotting:                  scope.Gauge("snapshot"),
    72  		isIndexFlushing:                 scope.Gauge("index-flush"),
    73  		maxBlocksSnapshottedByNamespace: scope.Gauge("max-blocks-snapshotted-by-namespace"),
    74  		dataWarmFlushDuration:           scope.Timer("data-warm-flush-duration"),
    75  		dataSnapshotDuration:            scope.Timer("data-snapshot-duration"),
    76  		indexFlushDuration:              scope.Timer("index-flush-duration"),
    77  		commitLogRotationDuration:       scope.Timer("commit-log-rotation-duration"),
    78  	}
    79  }
    80  
    81  type flushManager struct {
    82  	sync.RWMutex
    83  
    84  	database  database
    85  	commitlog commitlog.CommitLog
    86  	opts      Options
    87  	pm        persist.Manager
    88  	// state is used to protect the flush manager against concurrent use,
    89  	// while flushInProgress and snapshotInProgress are more granular and
    90  	// are used for emitting granular gauges.
    91  	state   flushManagerState
    92  	metrics flushManagerMetrics
    93  
    94  	lastSuccessfulSnapshotStartTime atomic.Int64 // == xtime.UnixNano
    95  
    96  	logger *zap.Logger
    97  	nowFn  clock.NowFn
    98  }
    99  
   100  func newFlushManager(
   101  	database database,
   102  	commitlog commitlog.CommitLog,
   103  	scope tally.Scope,
   104  ) databaseFlushManager {
   105  	opts := database.Options()
   106  	return &flushManager{
   107  		database:  database,
   108  		commitlog: commitlog,
   109  		opts:      opts,
   110  		pm:        opts.PersistManager(),
   111  		metrics:   newFlushManagerMetrics(scope),
   112  		logger:    opts.InstrumentOptions().Logger(),
   113  		nowFn:     opts.ClockOptions().NowFn(),
   114  	}
   115  }
   116  
   117  func (m *flushManager) Flush(startTime xtime.UnixNano) error {
   118  	// ensure only a single flush is happening at a time
   119  	m.Lock()
   120  	if m.state != flushManagerIdle {
   121  		m.Unlock()
   122  		return errFlushOperationsInProgress
   123  	}
   124  	m.state = flushManagerNotIdle
   125  	m.Unlock()
   126  
   127  	defer m.setState(flushManagerIdle)
   128  
   129  	namespaces, err := m.database.OwnedNamespaces()
   130  	if err != nil {
   131  		return err
   132  	}
   133  
   134  	// Perform two separate loops through all the namespaces so that we can
   135  	// emit better gauges, i.e. all the flushing for all the namespaces happens
   136  	// at once then all the snapshotting. This is
   137  	// also slightly better semantically because flushing should take priority
   138  	// over snapshotting.
   139  	//
   140  	// In addition, we need to make sure that for any given shard/blockStart
   141  	// combination, we attempt a flush before a snapshot as the snapshotting process
   142  	// will attempt to snapshot blocks w/ unflushed data which would be wasteful if
   143  	// the block is already flushable.
   144  	multiErr := xerrors.NewMultiError()
   145  	if err := m.dataWarmFlush(namespaces, startTime); err != nil {
   146  		multiErr = multiErr.Add(err)
   147  	}
   148  
   149  	start := m.nowFn()
   150  	rotatedCommitlogID, err := m.commitlog.RotateLogs()
   151  	m.metrics.commitLogRotationDuration.Record(m.nowFn().Sub(start))
   152  	if err == nil {
   153  		if err = m.dataSnapshot(namespaces, startTime, rotatedCommitlogID); err != nil {
   154  			multiErr = multiErr.Add(err)
   155  		}
   156  	} else {
   157  		multiErr = multiErr.Add(fmt.Errorf("error rotating commitlog in mediator tick: %v", err))
   158  	}
   159  
   160  	if err := m.indexFlush(namespaces); err != nil {
   161  		multiErr = multiErr.Add(err)
   162  	}
   163  
   164  	return multiErr.FinalError()
   165  }
   166  
   167  func (m *flushManager) dataWarmFlush(
   168  	namespaces []databaseNamespace,
   169  	startTime xtime.UnixNano,
   170  ) error {
   171  	flushPersist, err := m.pm.StartFlushPersist()
   172  	if err != nil {
   173  		return err
   174  	}
   175  
   176  	m.setState(flushManagerFlushInProgress)
   177  	var (
   178  		start    = m.nowFn()
   179  		multiErr = xerrors.NewMultiError()
   180  	)
   181  	for _, ns := range namespaces {
   182  		// Flush first because we will only snapshot if there are no outstanding flushes.
   183  		flushTimes, err := m.namespaceFlushTimes(ns, startTime)
   184  		if err != nil {
   185  			multiErr = multiErr.Add(err)
   186  			continue
   187  		}
   188  		if err := m.flushNamespaceWithTimes(ns, flushTimes, flushPersist); err != nil {
   189  			multiErr = multiErr.Add(err)
   190  		}
   191  	}
   192  
   193  	err = flushPersist.DoneFlush()
   194  	if err != nil {
   195  		multiErr = multiErr.Add(err)
   196  	}
   197  
   198  	m.metrics.dataWarmFlushDuration.Record(m.nowFn().Sub(start))
   199  	return multiErr.FinalError()
   200  }
   201  
   202  func (m *flushManager) dataSnapshot(
   203  	namespaces []databaseNamespace,
   204  	startTime xtime.UnixNano,
   205  	rotatedCommitlogID persist.CommitLogFile,
   206  ) error {
   207  	snapshotID := uuid.NewUUID()
   208  
   209  	snapshotPersist, err := m.pm.StartSnapshotPersist(snapshotID)
   210  	if err != nil {
   211  		return err
   212  	}
   213  
   214  	m.setState(flushManagerSnapshotInProgress)
   215  	var (
   216  		start                           = m.nowFn()
   217  		maxBlocksSnapshottedByNamespace = 0
   218  		multiErr                        = xerrors.NewMultiError()
   219  	)
   220  	for _, ns := range namespaces {
   221  		snapshotBlockStarts := m.namespaceSnapshotTimes(ns, startTime)
   222  		if len(snapshotBlockStarts) > maxBlocksSnapshottedByNamespace {
   223  			maxBlocksSnapshottedByNamespace = len(snapshotBlockStarts)
   224  		}
   225  
   226  		err := ns.Snapshot(snapshotBlockStarts, startTime, snapshotPersist)
   227  		if err != nil {
   228  			detailedErr := fmt.Errorf(
   229  				"namespace %s failed to snapshot data for some blocks: %w",
   230  				ns.ID().String(), err)
   231  			multiErr = multiErr.Add(detailedErr)
   232  			continue
   233  		}
   234  	}
   235  	m.metrics.maxBlocksSnapshottedByNamespace.Update(float64(maxBlocksSnapshottedByNamespace))
   236  
   237  	err = snapshotPersist.DoneSnapshot(snapshotID, rotatedCommitlogID)
   238  	multiErr = multiErr.Add(err)
   239  
   240  	finalErr := multiErr.FinalError()
   241  	if finalErr == nil {
   242  		m.lastSuccessfulSnapshotStartTime.Store(int64(startTime))
   243  	}
   244  	m.metrics.dataSnapshotDuration.Record(m.nowFn().Sub(start))
   245  	return finalErr
   246  }
   247  
   248  func (m *flushManager) indexFlush(
   249  	namespaces []databaseNamespace,
   250  ) error {
   251  	indexFlush, err := m.pm.StartIndexPersist()
   252  	if err != nil {
   253  		return err
   254  	}
   255  
   256  	m.setState(flushManagerIndexFlushInProgress)
   257  	var (
   258  		start    = m.nowFn()
   259  		multiErr = xerrors.NewMultiError()
   260  	)
   261  	for _, ns := range namespaces {
   262  		var (
   263  			indexOpts    = ns.Options().IndexOptions()
   264  			indexEnabled = indexOpts.Enabled()
   265  		)
   266  		if !indexEnabled {
   267  			continue
   268  		}
   269  
   270  		if err := ns.FlushIndex(indexFlush); err != nil {
   271  			multiErr = multiErr.Add(err)
   272  		}
   273  	}
   274  	multiErr = multiErr.Add(indexFlush.DoneIndex())
   275  
   276  	m.metrics.indexFlushDuration.Record(m.nowFn().Sub(start))
   277  	return multiErr.FinalError()
   278  }
   279  
   280  func (m *flushManager) Report() {
   281  	m.RLock()
   282  	state := m.state
   283  	m.RUnlock()
   284  
   285  	if state == flushManagerFlushInProgress {
   286  		m.metrics.isFlushing.Update(1)
   287  	} else {
   288  		m.metrics.isFlushing.Update(0)
   289  	}
   290  
   291  	if state == flushManagerSnapshotInProgress {
   292  		m.metrics.isSnapshotting.Update(1)
   293  	} else {
   294  		m.metrics.isSnapshotting.Update(0)
   295  	}
   296  
   297  	if state == flushManagerIndexFlushInProgress {
   298  		m.metrics.isIndexFlushing.Update(1)
   299  	} else {
   300  		m.metrics.isIndexFlushing.Update(0)
   301  	}
   302  }
   303  
   304  func (m *flushManager) setState(state flushManagerState) {
   305  	m.Lock()
   306  	m.state = state
   307  	m.Unlock()
   308  }
   309  
   310  func (m *flushManager) flushRange(rOpts retention.Options, t xtime.UnixNano) (xtime.UnixNano, xtime.UnixNano) {
   311  	return retention.FlushTimeStart(rOpts, t), retention.FlushTimeEnd(rOpts, t)
   312  }
   313  
   314  func (m *flushManager) namespaceFlushTimes(ns databaseNamespace, curr xtime.UnixNano) ([]xtime.UnixNano, error) {
   315  	var (
   316  		rOpts            = ns.Options().RetentionOptions()
   317  		blockSize        = rOpts.BlockSize()
   318  		earliest, latest = m.flushRange(rOpts, curr)
   319  	)
   320  
   321  	candidateTimes := timesInRange(earliest, latest, blockSize)
   322  	var loopErr error
   323  	return filterTimes(candidateTimes, func(t xtime.UnixNano) bool {
   324  		needsFlush, err := ns.NeedsFlush(t, t)
   325  		if err != nil {
   326  			loopErr = err
   327  			return false
   328  		}
   329  		return needsFlush
   330  	}), loopErr
   331  }
   332  
   333  func (m *flushManager) namespaceSnapshotTimes(ns databaseNamespace, curr xtime.UnixNano) []xtime.UnixNano {
   334  	var (
   335  		rOpts     = ns.Options().RetentionOptions()
   336  		blockSize = rOpts.BlockSize()
   337  		// Earliest possible snapshottable block is the earliest possible flushable
   338  		// blockStart which is the first block in the retention period.
   339  		earliest = retention.FlushTimeStart(rOpts, curr)
   340  		// Latest possible snapshotting block is either the current block OR the
   341  		// next block if the current time and bufferFuture configuration would
   342  		// allow writes to be written into the next block. Note that "current time"
   343  		// here is defined as "tick start time" because all the guarantees about
   344  		// snapshotting are based around the tick start time, now the current time.
   345  		latest = curr.Add(rOpts.BufferFuture()).Truncate(blockSize)
   346  	)
   347  
   348  	candidateTimes := timesInRange(earliest, latest, blockSize)
   349  	return filterTimes(candidateTimes, func(xtime.UnixNano) bool {
   350  		// NB(bodu): Snapshot everything since to account for cold writes/blocks.
   351  		return true
   352  	})
   353  }
   354  
   355  // flushWithTime flushes in-memory data for a given namespace, at a given
   356  // time, returning any error encountered during flushing
   357  func (m *flushManager) flushNamespaceWithTimes(
   358  	ns databaseNamespace,
   359  	times []xtime.UnixNano,
   360  	flushPreparer persist.FlushPreparer,
   361  ) error {
   362  	multiErr := xerrors.NewMultiError()
   363  	for _, t := range times {
   364  		// NB(xichen): we still want to proceed if a namespace fails to flush its data.
   365  		// Probably want to emit a counter here, but for now just log it.
   366  		if err := ns.WarmFlush(t, flushPreparer); err != nil {
   367  			detailedErr := fmt.Errorf("namespace %s failed to flush data: %v",
   368  				ns.ID().String(), err)
   369  			multiErr = multiErr.Add(detailedErr)
   370  		}
   371  	}
   372  	return multiErr.FinalError()
   373  }
   374  
   375  func (m *flushManager) LastSuccessfulSnapshotStartTime() (xtime.UnixNano, bool) {
   376  	snapTime := xtime.UnixNano(m.lastSuccessfulSnapshotStartTime.Load())
   377  	return snapTime, snapTime > 0
   378  }