github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/index_insert_queue.go (about)

     1  // Copyright (c) 2018 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  	"strconv"
    26  	"sync"
    27  	"time"
    28  
    29  	"github.com/m3db/m3/src/dbnode/namespace"
    30  	"github.com/m3db/m3/src/dbnode/storage/index"
    31  	"github.com/m3db/m3/src/dbnode/ts/writes"
    32  	"github.com/m3db/m3/src/x/clock"
    33  	xsync "github.com/m3db/m3/src/x/sync"
    34  
    35  	"github.com/uber-go/tally"
    36  )
    37  
    38  var (
    39  	errIndexInsertQueueNotOpen             = errors.New("index insert queue is not open")
    40  	errIndexInsertQueueAlreadyOpenOrClosed = errors.New("index insert queue already open or is closed")
    41  )
    42  
    43  type nsIndexInsertQueueState int
    44  
    45  const (
    46  	nsIndexInsertQueueStateNotOpen nsIndexInsertQueueState = iota
    47  	nsIndexInsertQueueStateOpen
    48  	nsIndexInsertQueueStateClosed
    49  
    50  	// TODO(prateek): runtime options for this stuff
    51  	defaultIndexBatchBackoff = 2 * time.Millisecond
    52  
    53  	indexResetAllInsertsEvery = 3 * time.Minute
    54  )
    55  
    56  type nsIndexInsertQueue struct {
    57  	sync.RWMutex
    58  
    59  	namespaceMetadata namespace.Metadata
    60  
    61  	state nsIndexInsertQueueState
    62  
    63  	// rate limits
    64  	indexBatchBackoff time.Duration
    65  
    66  	// active batch pending execution
    67  	currBatch *nsIndexInsertBatch
    68  
    69  	indexBatchFn nsIndexInsertBatchFn
    70  	nowFn        clock.NowFn
    71  	sleepFn      func(time.Duration)
    72  	coreFn       xsync.CoreFn
    73  	notifyInsert chan struct{}
    74  	closeCh      chan struct{}
    75  
    76  	scope tally.Scope
    77  
    78  	metrics nsIndexInsertQueueMetrics
    79  }
    80  
    81  type newNamespaceIndexInsertQueueFn func(
    82  	nsIndexInsertBatchFn, namespace.Metadata, clock.NowFn, xsync.CoreFn, tally.Scope) namespaceIndexInsertQueue
    83  
    84  // newNamespaceIndexInsertQueue returns a new index insert queue.
    85  // Note: No limit appears on the index insert queue since any items making
    86  // it into the index insert queue must first pass through the shard insert
    87  // queue which has it's own limits in place.
    88  // Any error returned from this queue would cause the series to not be indexed
    89  // and there is no way to return this error to the client over the network
    90  // (unlike the shard insert queue at which point if an error is returned
    91  // is returned all the way back to the DB node client).
    92  // FOLLOWUP(prateek): subsequent PR to wire up rate limiting to runtime.Options
    93  func newNamespaceIndexInsertQueue(
    94  	indexBatchFn nsIndexInsertBatchFn,
    95  	namespaceMetadata namespace.Metadata,
    96  	nowFn clock.NowFn,
    97  	coreFn xsync.CoreFn,
    98  	scope tally.Scope,
    99  ) namespaceIndexInsertQueue {
   100  	subscope := scope.SubScope("insert-queue")
   101  	q := &nsIndexInsertQueue{
   102  		namespaceMetadata: namespaceMetadata,
   103  		indexBatchBackoff: defaultIndexBatchBackoff,
   104  		indexBatchFn:      indexBatchFn,
   105  		nowFn:             nowFn,
   106  		sleepFn:           time.Sleep,
   107  		coreFn:            coreFn,
   108  		// NB(r): Use 2 * num cores so that each CPU insert queue which
   109  		// is 1 per num CPU core can always enqueue a notification without
   110  		// it being lost.
   111  		notifyInsert: make(chan struct{}, 2*xsync.NumCores()),
   112  		closeCh:      make(chan struct{}, 1),
   113  		scope:        subscope,
   114  		metrics:      newNamespaceIndexInsertQueueMetrics(subscope),
   115  	}
   116  	q.currBatch = q.newBatch(newBatchOptions{instrumented: true})
   117  	return q
   118  }
   119  
   120  type newBatchOptions struct {
   121  	instrumented bool
   122  }
   123  
   124  func (q *nsIndexInsertQueue) newBatch(opts newBatchOptions) *nsIndexInsertBatch {
   125  	scope := tally.NoopScope
   126  	if opts.instrumented {
   127  		scope = q.scope
   128  	}
   129  	return newNsIndexInsertBatch(q.namespaceMetadata, q.nowFn, scope)
   130  }
   131  
   132  func (q *nsIndexInsertQueue) insertLoop() {
   133  	defer func() {
   134  		close(q.closeCh)
   135  	}()
   136  
   137  	var lastInsert time.Time
   138  	batch := q.newBatch(newBatchOptions{})
   139  	for range q.notifyInsert {
   140  		// Check if inserting too fast
   141  		elapsedSinceLastInsert := q.nowFn().Sub(lastInsert)
   142  
   143  		// Rotate batches
   144  		var (
   145  			state   nsIndexInsertQueueState
   146  			backoff time.Duration
   147  		)
   148  		q.Lock()
   149  		state = q.state
   150  		if elapsedSinceLastInsert < q.indexBatchBackoff {
   151  			// Need to backoff before rotate and insert
   152  			backoff = q.indexBatchBackoff - elapsedSinceLastInsert
   153  		}
   154  		q.Unlock()
   155  
   156  		if backoff > 0 {
   157  			q.sleepFn(backoff)
   158  		}
   159  
   160  		// Rotate after backoff
   161  		batchWg := q.currBatch.Rotate(batch)
   162  
   163  		all := batch.AllInserts()
   164  		if all.Len() > 0 {
   165  			q.indexBatchFn(all)
   166  		}
   167  
   168  		batchWg.Done()
   169  
   170  		lastInsert = q.nowFn()
   171  
   172  		if state != nsIndexInsertQueueStateOpen {
   173  			return // Break if the queue closed
   174  		}
   175  	}
   176  }
   177  
   178  func (q *nsIndexInsertQueue) InsertBatch(
   179  	batch *index.WriteBatch,
   180  ) (*sync.WaitGroup, error) {
   181  	batchLen := batch.Len()
   182  
   183  	// Choose the queue relevant to current CPU index.
   184  	// Note: since inserts by CPU core is allocated when
   185  	// nsIndexInsertBatch is constructed and then never modified
   186  	// it is safe to concurently read (but not modify obviously).
   187  	inserts := q.currBatch.insertsByCPUCore[q.coreFn()]
   188  	inserts.Lock()
   189  	firstInsert := len(inserts.shardInserts) == 0
   190  	inserts.shardInserts = append(inserts.shardInserts, batch)
   191  	wg := inserts.wg
   192  	inserts.Unlock()
   193  
   194  	// Notify insert loop, only required if first to insert for this
   195  	// this CPU core.
   196  	if firstInsert {
   197  		select {
   198  		case q.notifyInsert <- struct{}{}:
   199  		default:
   200  			// Loop busy, already ready to consume notification.
   201  		}
   202  	}
   203  
   204  	q.metrics.numPending.Inc(int64(batchLen))
   205  	return wg, nil
   206  }
   207  
   208  func (q *nsIndexInsertQueue) InsertPending(
   209  	pending []writes.PendingIndexInsert,
   210  ) (*sync.WaitGroup, error) {
   211  	batchLen := len(pending)
   212  
   213  	// Choose the queue relevant to current CPU index.
   214  	// Note: since inserts by CPU core is allocated when
   215  	// nsIndexInsertBatch is constructed and then never modified
   216  	// it is safe to concurently read (but not modify obviously).
   217  	inserts := q.currBatch.insertsByCPUCore[q.coreFn()]
   218  	inserts.Lock()
   219  	firstInsert := len(inserts.batchInserts) == 0
   220  	inserts.batchInserts = append(inserts.batchInserts, pending...)
   221  	wg := inserts.wg
   222  	inserts.Unlock()
   223  
   224  	// Notify insert loop, only required if first to insert for this
   225  	// this CPU core.
   226  	if firstInsert {
   227  		select {
   228  		case q.notifyInsert <- struct{}{}:
   229  		default:
   230  			// Loop busy, already ready to consume notification.
   231  		}
   232  	}
   233  
   234  	q.metrics.numPending.Inc(int64(batchLen))
   235  	return wg, nil
   236  }
   237  
   238  func (q *nsIndexInsertQueue) Start() error {
   239  	q.Lock()
   240  	defer q.Unlock()
   241  
   242  	if q.state != nsIndexInsertQueueStateNotOpen {
   243  		return errIndexInsertQueueAlreadyOpenOrClosed
   244  	}
   245  
   246  	q.state = nsIndexInsertQueueStateOpen
   247  	go q.insertLoop()
   248  	return nil
   249  }
   250  
   251  func (q *nsIndexInsertQueue) Stop() error {
   252  	q.Lock()
   253  
   254  	if q.state != nsIndexInsertQueueStateOpen {
   255  		q.Unlock()
   256  		return errIndexInsertQueueNotOpen
   257  	}
   258  
   259  	q.state = nsIndexInsertQueueStateClosed
   260  	q.Unlock()
   261  
   262  	// Final flush
   263  	select {
   264  	case q.notifyInsert <- struct{}{}:
   265  	default:
   266  		// Loop busy, already ready to consume notification
   267  	}
   268  
   269  	// wait till other go routine is done
   270  	<-q.closeCh
   271  
   272  	return nil
   273  }
   274  
   275  type nsIndexInsertBatchFn func(inserts *index.WriteBatch)
   276  
   277  type nsIndexInsertBatch struct {
   278  	namespace namespace.Metadata
   279  	nowFn     clock.NowFn
   280  	wg        *sync.WaitGroup
   281  	// Note: since inserts by CPU core is allocated when
   282  	// nsIndexInsertBatch is constructed and then never modified
   283  	// it is safe to concurently read (but not modify obviously).
   284  	insertsByCPUCore    []*nsIndexInsertsByCPUCore
   285  	allInserts          *index.WriteBatch
   286  	allInsertsLastReset time.Time
   287  }
   288  
   289  type nsIndexInsertsByCPUCore struct {
   290  	sync.Mutex
   291  	shardInserts []*index.WriteBatch
   292  	batchInserts []writes.PendingIndexInsert
   293  	wg           *sync.WaitGroup
   294  	metrics      nsIndexInsertsByCPUCoreMetrics
   295  }
   296  
   297  type nsIndexInsertsByCPUCoreMetrics struct {
   298  	rotateInsertsShard   tally.Counter
   299  	rotateInsertsPending tally.Counter
   300  }
   301  
   302  func newNamespaceIndexInsertsByCPUCoreMetrics(
   303  	cpuIndex int,
   304  	scope tally.Scope,
   305  ) nsIndexInsertsByCPUCoreMetrics {
   306  	scope = scope.Tagged(map[string]string{
   307  		"cpu-index": strconv.Itoa(cpuIndex),
   308  	})
   309  
   310  	const rotate = "rotate-inserts"
   311  	return nsIndexInsertsByCPUCoreMetrics{
   312  		rotateInsertsShard: scope.Tagged(map[string]string{
   313  			"rotate-type": "shard-insert",
   314  		}).Counter(rotate),
   315  		rotateInsertsPending: scope.Tagged(map[string]string{
   316  			"rotate-type": "pending-insert",
   317  		}).Counter(rotate),
   318  	}
   319  }
   320  
   321  func newNsIndexInsertBatch(
   322  	namespace namespace.Metadata,
   323  	nowFn clock.NowFn,
   324  	scope tally.Scope,
   325  ) *nsIndexInsertBatch {
   326  	b := &nsIndexInsertBatch{
   327  		namespace: namespace,
   328  		nowFn:     nowFn,
   329  	}
   330  	numCores := xsync.NumCores()
   331  	for i := 0; i < numCores; i++ {
   332  		b.insertsByCPUCore = append(b.insertsByCPUCore, &nsIndexInsertsByCPUCore{
   333  			metrics: newNamespaceIndexInsertsByCPUCoreMetrics(i, scope),
   334  		})
   335  	}
   336  
   337  	b.allocateAllInserts()
   338  	b.Rotate(nil)
   339  	return b
   340  }
   341  
   342  func (b *nsIndexInsertBatch) allocateAllInserts() {
   343  	b.allInserts = index.NewWriteBatch(index.WriteBatchOptions{
   344  		IndexBlockSize: b.namespace.Options().IndexOptions().BlockSize(),
   345  	})
   346  	b.allInsertsLastReset = b.nowFn()
   347  }
   348  
   349  func (b *nsIndexInsertBatch) AllInserts() *index.WriteBatch {
   350  	b.allInserts.Reset()
   351  	for _, inserts := range b.insertsByCPUCore {
   352  		inserts.Lock()
   353  		for _, shardInserts := range inserts.shardInserts {
   354  			b.allInserts.AppendAll(shardInserts)
   355  		}
   356  		for _, insert := range inserts.batchInserts {
   357  			b.allInserts.Append(insert.Entry, insert.Document)
   358  		}
   359  		inserts.Unlock()
   360  	}
   361  	return b.allInserts
   362  }
   363  
   364  func (b *nsIndexInsertBatch) Rotate(target *nsIndexInsertBatch) *sync.WaitGroup {
   365  	prevWg := b.wg
   366  
   367  	// We always expect to be waiting for an index.
   368  	b.wg = &sync.WaitGroup{}
   369  	b.wg.Add(1)
   370  
   371  	// Rotate to target if we need to.
   372  	for idx, inserts := range b.insertsByCPUCore {
   373  		if target == nil {
   374  			// No target to rotate with.
   375  			inserts.Lock()
   376  			// Reset
   377  			inserts.shardInserts = inserts.shardInserts[:0]
   378  			inserts.batchInserts = inserts.batchInserts[:0]
   379  			// Use new wait group.
   380  			inserts.wg = b.wg
   381  			inserts.Unlock()
   382  			continue
   383  		}
   384  
   385  		// First prepare the target to take the current batch's inserts.
   386  		targetInserts := target.insertsByCPUCore[idx]
   387  		targetInserts.Lock()
   388  
   389  		// Reset the target inserts since we'll take ref to them in a second.
   390  		for i := range targetInserts.shardInserts {
   391  			// TODO(prateek): if we start pooling `[]index.WriteBatchEntry`, then we could return to the pool here.
   392  			targetInserts.shardInserts[i] = nil
   393  		}
   394  		prevTargetShardInserts := targetInserts.shardInserts[:0]
   395  
   396  		// memset optimization
   397  		var zero writes.PendingIndexInsert
   398  		for i := range targetInserts.batchInserts {
   399  			targetInserts.batchInserts[i] = zero
   400  		}
   401  		prevTargetBatchInserts := targetInserts.batchInserts[:0]
   402  
   403  		// Lock the current batch inserts now ready to rotate to the target.
   404  		inserts.Lock()
   405  
   406  		// Update current slice refs to take target's inserts.
   407  		targetInserts.shardInserts = inserts.shardInserts
   408  		targetInserts.batchInserts = inserts.batchInserts
   409  		targetInserts.wg = inserts.wg
   410  
   411  		// Reuse the target's old slices.
   412  		inserts.shardInserts = prevTargetShardInserts
   413  		inserts.batchInserts = prevTargetBatchInserts
   414  
   415  		// Use new wait group.
   416  		inserts.wg = b.wg
   417  
   418  		// Unlock as early as possible for writes to keep enqueuing.
   419  		inserts.Unlock()
   420  
   421  		numTargetInsertsShard := len(targetInserts.shardInserts)
   422  		numTargetInsertsPending := len(targetInserts.batchInserts)
   423  
   424  		// Now can unlock target inserts too.
   425  		targetInserts.Unlock()
   426  
   427  		if n := numTargetInsertsShard; n > 0 {
   428  			inserts.metrics.rotateInsertsShard.Inc(int64(n))
   429  		}
   430  		if n := numTargetInsertsPending; n > 0 {
   431  			inserts.metrics.rotateInsertsPending.Inc(int64(n))
   432  		}
   433  	}
   434  
   435  	if b.nowFn().Sub(b.allInsertsLastReset) > indexResetAllInsertsEvery {
   436  		// NB(r): Sometimes this can grow very high, so we reset it relatively frequently
   437  		b.allocateAllInserts()
   438  	}
   439  
   440  	return prevWg
   441  }
   442  
   443  type nsIndexInsertQueueMetrics struct {
   444  	numPending tally.Counter
   445  }
   446  
   447  func newNamespaceIndexInsertQueueMetrics(
   448  	scope tally.Scope,
   449  ) nsIndexInsertQueueMetrics {
   450  	subScope := scope.SubScope("index-queue")
   451  	return nsIndexInsertQueueMetrics{
   452  		numPending: subScope.Counter("num-pending"),
   453  	}
   454  }