github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/shard_insert_queue_test.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  	"sync"
    25  	"sync/atomic"
    26  	"testing"
    27  	"time"
    28  
    29  	"github.com/fortytw2/leaktest"
    30  	"github.com/stretchr/testify/assert"
    31  	"github.com/stretchr/testify/require"
    32  	"github.com/uber-go/tally"
    33  	"go.uber.org/zap"
    34  
    35  	xsync "github.com/m3db/m3/src/x/sync"
    36  )
    37  
    38  func TestShardInsertQueueBatchBackoff(t *testing.T) {
    39  	defer leaktest.CheckTimeout(t, time.Second)()
    40  
    41  	var (
    42  		inserts  [][]dbShardInsert
    43  		currTime = time.Now()
    44  		timeLock = sync.Mutex{}
    45  		addTime  = func(d time.Duration) {
    46  			timeLock.Lock()
    47  			defer timeLock.Unlock()
    48  			currTime = currTime.Add(d)
    49  		}
    50  		backoff           = 10 * time.Millisecond
    51  		insertWgs         [3]sync.WaitGroup
    52  		insertProgressWgs [3]sync.WaitGroup
    53  	)
    54  	for i := range insertWgs {
    55  		insertWgs[i].Add(1)
    56  	}
    57  	for i := range insertProgressWgs {
    58  		insertProgressWgs[i].Add(1)
    59  	}
    60  	q := newDatabaseShardInsertQueue(func(value []dbShardInsert) error {
    61  		if len(inserts) == len(insertWgs) {
    62  			return nil // Overflow.
    63  		}
    64  
    65  		inserts = append(inserts, value)
    66  		insertWgs[len(inserts)-1].Done()
    67  		insertProgressWgs[len(inserts)-1].Wait()
    68  		return nil
    69  	}, func() time.Time {
    70  		timeLock.Lock()
    71  		defer timeLock.Unlock()
    72  		return currTime
    73  	}, xsync.CPUCore, tally.NoopScope, zap.NewNop())
    74  
    75  	q.insertBatchBackoff = backoff
    76  
    77  	var slept time.Duration
    78  	var numSleeps int
    79  	q.sleepFn = func(d time.Duration) {
    80  		assert.Equal(t, backoff, d)
    81  
    82  		slept += d
    83  		numSleeps++
    84  		addTime(d)
    85  	}
    86  
    87  	require.NoError(t, q.Start())
    88  	defer func() {
    89  		require.NoError(t, q.Stop())
    90  	}()
    91  
    92  	// first insert
    93  	_, err := q.Insert(dbShardInsert{entry: &Entry{Index: 0}})
    94  	require.NoError(t, err)
    95  
    96  	// wait for first insert batch to complete
    97  	insertWgs[0].Wait()
    98  
    99  	// now next batch will need to wait as we haven't progressed time
   100  	_, err = q.Insert(dbShardInsert{entry: &Entry{Index: 1}})
   101  	require.NoError(t, err)
   102  	_, err = q.Insert(dbShardInsert{entry: &Entry{Index: 2}})
   103  	require.NoError(t, err)
   104  
   105  	// allow first insert to finish
   106  	insertProgressWgs[0].Done()
   107  
   108  	// wait for second batch to complete
   109  	insertWgs[1].Wait()
   110  
   111  	assert.Equal(t, backoff, slept)
   112  	assert.Equal(t, 1, numSleeps)
   113  
   114  	// insert third batch, will also need to wait
   115  	_, err = q.Insert(dbShardInsert{entry: &Entry{Index: 3}})
   116  	require.NoError(t, err)
   117  
   118  	// allow second batch to finish
   119  	insertProgressWgs[1].Done()
   120  
   121  	// wait for third batch to complete
   122  	insertWgs[2].Wait()
   123  
   124  	assert.Equal(t, 2*backoff, slept)
   125  	assert.Equal(t, 2, numSleeps)
   126  
   127  	assert.Equal(t, 3, len(inserts))
   128  
   129  	// allow third batch to complete
   130  	insertProgressWgs[2].Done()
   131  }
   132  
   133  func TestShardInsertQueueRateLimit(t *testing.T) {
   134  	defer leaktest.CheckTimeout(t, time.Second)()
   135  
   136  	var (
   137  		currTime = time.Now().Truncate(time.Second)
   138  		timeLock = sync.Mutex{}
   139  		addTime  = func(d time.Duration) {
   140  			timeLock.Lock()
   141  			defer timeLock.Unlock()
   142  			currTime = currTime.Add(d)
   143  		}
   144  	)
   145  	q := newDatabaseShardInsertQueue(func(value []dbShardInsert) error {
   146  		return nil
   147  	}, func() time.Time {
   148  		timeLock.Lock()
   149  		defer timeLock.Unlock()
   150  		return currTime
   151  	}, xsync.CPUCore, tally.NoopScope, zap.NewNop())
   152  
   153  	q.insertPerSecondLimit.Store(2)
   154  
   155  	require.NoError(t, q.Start())
   156  	defer func() {
   157  		require.NoError(t, q.Stop())
   158  	}()
   159  
   160  	_, err := q.Insert(dbShardInsert{})
   161  	require.NoError(t, err)
   162  
   163  	addTime(250 * time.Millisecond)
   164  	_, err = q.Insert(dbShardInsert{})
   165  	require.NoError(t, err)
   166  
   167  	// Consecutive should be all rate limited
   168  	for i := 0; i < 100; i++ {
   169  		_, err = q.Insert(dbShardInsert{})
   170  		require.Error(t, err)
   171  		require.Equal(t, errNewSeriesInsertRateLimitExceeded, err)
   172  	}
   173  
   174  	// Start 2nd second should not be an issue
   175  	addTime(750 * time.Millisecond)
   176  	_, err = q.Insert(dbShardInsert{})
   177  	require.NoError(t, err)
   178  
   179  	addTime(100 * time.Millisecond)
   180  	_, err = q.Insert(dbShardInsert{})
   181  	require.NoError(t, err)
   182  
   183  	addTime(100 * time.Millisecond)
   184  	_, err = q.Insert(dbShardInsert{})
   185  	require.Error(t, err)
   186  	require.Equal(t, errNewSeriesInsertRateLimitExceeded, err)
   187  
   188  	// Start 3rd second
   189  	addTime(800 * time.Millisecond)
   190  	_, err = q.Insert(dbShardInsert{})
   191  	require.NoError(t, err)
   192  
   193  	q.Lock()
   194  	expectedCurrWindow := uint64(currTime.Truncate(time.Second).UnixNano())
   195  	assert.Equal(t, expectedCurrWindow, q.insertPerSecondLimitWindowNanos.Load())
   196  	assert.Equal(t, uint64(1), q.insertPerSecondLimitWindowValues.Load())
   197  	q.Unlock()
   198  }
   199  
   200  func TestShardInsertQueueFlushedOnClose(t *testing.T) {
   201  	defer leaktest.CheckTimeout(t, 5*time.Second)()
   202  
   203  	var (
   204  		numInsertExpected = 10
   205  		numInsertObserved int64
   206  		currTime          = time.Now().Truncate(time.Second)
   207  	)
   208  
   209  	q := newDatabaseShardInsertQueue(func(value []dbShardInsert) error {
   210  		atomic.AddInt64(&numInsertObserved, int64(len(value)))
   211  		return nil
   212  	}, func() time.Time { return currTime }, xsync.CPUCore, tally.NoopScope, zap.NewNop())
   213  
   214  	require.NoError(t, q.Start())
   215  
   216  	for i := 0; i < numInsertExpected; i++ {
   217  		_, err := q.Insert(dbShardInsert{})
   218  		require.NoError(t, err)
   219  	}
   220  
   221  	require.NoError(t, q.Stop())
   222  	require.Equal(t, int64(numInsertExpected), atomic.LoadInt64(&numInsertObserved))
   223  }