From 220013194574907d80d57357353aa04e26f8aeb4 Mon Sep 17 00:00:00 2001 From: Prateek Rungta Date: Wed, 9 May 2018 15:50:58 -0400 Subject: [PATCH 01/33] Batch Shard -> Index insertions --- storage/index.go | 79 ++++++++++++++++++------------ storage/index_block_test.go | 15 +++++- storage/index_insert_queue.go | 19 +++---- storage/index_insert_queue_test.go | 37 +++++++++----- storage/index_queue_test.go | 19 ++++--- storage/shard.go | 71 +++++++++++++++++++++++++-- storage/storage_mock.go | 25 +++++----- storage/types.go | 25 +++++++--- 8 files changed, 195 insertions(+), 95 deletions(-) diff --git a/storage/index.go b/storage/index.go index e859290709..28d825818d 100644 --- a/storage/index.go +++ b/storage/index.go @@ -30,14 +30,11 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/retention" "github.com/m3db/m3db/storage/bootstrap/result" - m3dberrors "github.com/m3db/m3db/storage/errors" "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/index/convert" "github.com/m3db/m3db/storage/namespace" - "github.com/m3db/m3ninx/doc" "github.com/m3db/m3x/context" xerrors "github.com/m3db/m3x/errors" - "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" xlog "github.com/m3db/m3x/log" xtime "github.com/m3db/m3x/time" @@ -224,61 +221,79 @@ func newNamespaceIndexWithOptions( // => indexQueue.Insert() // => index.writeBatch() -func (i *nsIndex) Write( - id ident.ID, - tags ident.Tags, - timestamp time.Time, - fns index.OnIndexSeries, +func (i *nsIndex) WriteBatch( + entries []indexWriteEntry, ) error { // Ensure timestamp is not too old/new based on retention policies. now := i.nowFn() futureLimit := now.Add(1 * i.bufferFuture) pastLimit := now.Add(-1 * i.bufferPast) - if !futureLimit.After(timestamp) { - fns.OnIndexFinalize() - return m3dberrors.ErrTooFuture - } + // TODO(prateek): pool this type + queueBatch := make([]index.WriteBatchEntry, 0, len(entries)) + + for _, entry := range entries { + var ( + id = entry.id + tags = entry.tags + timestamp = entry.timestamp + fns = entry.fns + ) + if !futureLimit.After(timestamp) { + fns.OnIndexFinalize() + continue + // TODO(prateek): return m3dberrors.ErrTooFuture + } - if !pastLimit.Before(timestamp) { - fns.OnIndexFinalize() - return m3dberrors.ErrTooPast - } + if !pastLimit.Before(timestamp) { + fns.OnIndexFinalize() + continue + // TODO(prateek): return m3dberrors.ErrTooPast + } - // Ensure metric can be converted to a valid document - d, err := convert.FromMetric(id, tags) - if err != nil { - fns.OnIndexFinalize() - return err - } + // Ensure metric can be converted to a valid document + d, err := convert.FromMetric(id, tags) + if err != nil { + fns.OnIndexFinalize() + continue + // TODO(prateek): return err + } - indexBlockStart := timestamp.Truncate(i.blockSize) - return i.queueWrite(indexBlockStart, d, fns) + indexBlockStart := xtime.ToUnixNano(timestamp.Truncate(i.blockSize)) + queueBatch = append(queueBatch, index.WriteBatchEntry{ + BlockStart: indexBlockStart, + Document: d, + OnIndexSeries: fns, + }) + } + return i.enqueueBatch(queueBatch) } -func (i *nsIndex) queueWrite( - indexBlockStart time.Time, - d doc.Document, - fns index.OnIndexSeries, +func (i *nsIndex) enqueueBatch( + entries []index.WriteBatchEntry, ) error { i.state.RLock() if !i.isOpenWithRLock() { i.state.RUnlock() i.metrics.InsertAfterClose.Inc(1) - fns.OnIndexFinalize() + for _, entry := range entries { + entry.OnIndexSeries.OnIndexFinalize() + } return errDbIndexUnableToWriteClosed } // NB(prateek): retrieving insertMode here while we have the RLock. insertMode := i.state.runtimeOpts.insertMode - wg, err := i.state.insertQueue.Insert(indexBlockStart, d, fns) + wg, err := i.state.insertQueue.InsertBatch(entries) // release the lock because we don't need it past this point. i.state.RUnlock() // if we're unable to index, we still have to finalize the reference we hold. if err != nil { - fns.OnIndexFinalize() + for _, entry := range entries { + entry.OnIndexSeries.OnIndexFinalize() + } return err } // once the write has been queued in the indexInsertQueue, it assumes @@ -356,7 +371,7 @@ func (i *nsIndex) writeBatchForBlockStartWithRLock( i.metrics.AsyncInsertErrors.Inc(numErr) } if err != nil { - i.logger.Errorf("unable to write to index, dropping inserts. [%v]", err) + // i.logger.Errorf("unable to write to index, dropping inserts. [%v]", err) } } diff --git a/storage/index_block_test.go b/storage/index_block_test.go index b8f5d49cce..4bddb5ca4a 100644 --- a/storage/index_block_test.go +++ b/storage/index_block_test.go @@ -41,6 +41,17 @@ import ( "github.com/stretchr/testify/require" ) +func testIndexWriteEntry(id ident.ID, tags ident.Tags, timestamp time.Time, fns index.OnIndexSeries) []indexWriteEntry { + return []indexWriteEntry{ + indexWriteEntry{ + id: id, + tags: tags, + timestamp: timestamp, + fns: fns, + }, + } +} + func testNamespaceMetadata(blockSize, period time.Duration) namespace.Metadata { nopts := namespace.NewOptions(). SetRetentionOptions(retention.NewOptions(). @@ -139,7 +150,7 @@ func TestNamespaceIndexWrite(t *testing.T) { OnIndexSeries: lifecycle, }, }).Return(index.WriteBatchResult{}, nil) - require.NoError(t, idx.Write(id, tags, now, lifecycle)) + require.NoError(t, idx.WriteBatch(testIndexWriteEntry(id, tags, now, lifecycle))) } func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { @@ -192,7 +203,7 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { now = now.Add(blockSize) nowLock.Unlock() - require.NoError(t, idx.Write(id, tags, now, lifecycle)) + require.NoError(t, idx.WriteBatch(testIndexWriteEntry(id, tags, now, lifecycle))) } func TestNamespaceIndexBootstrap(t *testing.T) { diff --git a/storage/index_insert_queue.go b/storage/index_insert_queue.go index 7de093e5e1..2e1be36689 100644 --- a/storage/index_insert_queue.go +++ b/storage/index_insert_queue.go @@ -27,8 +27,6 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/storage/index" - "github.com/m3db/m3ninx/doc" - xtime "github.com/m3db/m3x/time" "github.com/uber-go/tally" ) @@ -48,8 +46,9 @@ const ( ) var ( - defaultIndexBatchBackoff = time.Second - defaultIndexPerSecondLimit = 10000 + // TODO(prateek): undo this stuff + defaultIndexBatchBackoff = time.Millisecond + defaultIndexPerSecondLimit = 1000000 ) type nsIndexInsertQueue struct { @@ -154,10 +153,8 @@ func (q *nsIndexInsertQueue) insertLoop() { } } -func (q *nsIndexInsertQueue) Insert( - blockStart time.Time, - d doc.Document, - fns index.OnIndexSeries, +func (q *nsIndexInsertQueue) InsertBatch( + entries []index.WriteBatchEntry, ) (*sync.WaitGroup, error) { windowNanos := q.nowFn().Truncate(time.Second).UnixNano() @@ -178,11 +175,7 @@ func (q *nsIndexInsertQueue) Insert( return nil, errNewSeriesIndexRateLimitExceeded } } - q.currBatch.inserts = append(q.currBatch.inserts, index.WriteBatchEntry{ - BlockStart: xtime.ToUnixNano(blockStart), - Document: d, - OnIndexSeries: fns, - }) + q.currBatch.inserts = append(q.currBatch.inserts, entries...) wg := q.currBatch.wg q.Unlock() diff --git a/storage/index_insert_queue_test.go b/storage/index_insert_queue_test.go index 48dafdb72a..f99f9868fb 100644 --- a/storage/index_insert_queue_test.go +++ b/storage/index_insert_queue_test.go @@ -28,6 +28,7 @@ import ( "github.com/m3db/m3db/storage/index" "github.com/m3db/m3ninx/doc" + xtime "github.com/m3db/m3x/time" "github.com/fortytw2/leaktest" "github.com/golang/mock/gomock" @@ -36,6 +37,16 @@ import ( "github.com/uber-go/tally" ) +func testWriteBatchEntry(ts time.Time, d doc.Document, fn index.OnIndexSeries) []index.WriteBatchEntry { + return []index.WriteBatchEntry{ + index.WriteBatchEntry{ + BlockStart: xtime.ToUnixNano(ts), + Document: d, + OnIndexSeries: fn, + }, + } +} + func newTestIndexInsertQueue() *nsIndexInsertQueue { var ( nsIndexInsertBatchFn = func(inserts []index.WriteBatchEntry) {} @@ -100,7 +111,7 @@ func TestIndexInsertQueueCallback(t *testing.T) { defer q.Stop() now := time.Now() - wg, err := q.Insert(now, d, callback) + wg, err := q.InsertBatch(testWriteBatchEntry(now, d, callback)) assert.NoError(t, err) wg.Wait() @@ -139,37 +150,37 @@ func TestIndexInsertQueueRateLimit(t *testing.T) { assert.NoError(t, q.Stop()) }() - _, err := q.Insert(time.Time{}, testDoc(1), callback) + _, err := q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(1), callback)) assert.NoError(t, err) addTime(250 * time.Millisecond) - _, err = q.Insert(time.Time{}, testDoc(2), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(2), callback)) assert.NoError(t, err) // Consecutive should be all rate limited for i := 0; i < 100; i++ { - _, err = q.Insert(time.Time{}, testDoc(i+2), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(i+2), callback)) assert.Error(t, err) assert.Equal(t, errNewSeriesIndexRateLimitExceeded, err) } // Start 2nd second should not be an issue addTime(750 * time.Millisecond) - _, err = q.Insert(time.Time{}, testDoc(110), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(110), callback)) assert.NoError(t, err) addTime(100 * time.Millisecond) - _, err = q.Insert(time.Time{}, testDoc(111), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(111), callback)) assert.NoError(t, err) addTime(100 * time.Millisecond) - _, err = q.Insert(time.Time{}, testDoc(112), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(112), callback)) assert.Error(t, err) assert.Equal(t, errNewSeriesIndexRateLimitExceeded, err) // Start 3rd second addTime(800 * time.Millisecond) - _, err = q.Insert(time.Time{}, testDoc(113), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(113), callback)) assert.NoError(t, err) q.Lock() @@ -232,16 +243,16 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { }() // first insert - _, err := q.Insert(time.Time{}, testDoc(0), callback) + _, err := q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(0), callback)) require.NoError(t, err) // wait for first insert batch to complete insertWgs[0].Wait() // now next batch will need to wait as we haven't progressed time - _, err = q.Insert(time.Time{}, testDoc(1), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(1), callback)) require.NoError(t, err) - _, err = q.Insert(time.Time{}, testDoc(2), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(2), callback)) require.NoError(t, err) // allow first insert to finish @@ -254,7 +265,7 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { assert.Equal(t, 1, numSleeps) // insert third batch, will also need to wait - _, err = q.Insert(time.Time{}, testDoc(3), callback) + _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(3), callback)) require.NoError(t, err) // allow second batch to finish @@ -288,7 +299,7 @@ func TestIndexInsertQueueFlushedOnClose(t *testing.T) { require.NoError(t, q.Start()) for i := 0; i < numInsertExpected; i++ { - _, err := q.Insert(time.Time{}, doc.Document{}, nil) + _, err := q.InsertBatch(testWriteBatchEntry(time.Time{}, doc.Document{}, nil)) require.NoError(t, err) } diff --git a/storage/index_queue_test.go b/storage/index_queue_test.go index a3553fb2d6..63e927b9b1 100644 --- a/storage/index_queue_test.go +++ b/storage/index_queue_test.go @@ -30,7 +30,6 @@ import ( "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/index/convert" "github.com/m3db/m3db/storage/namespace" - "github.com/m3db/m3ninx/doc" m3ninxidx "github.com/m3db/m3ninx/idx" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" @@ -130,7 +129,7 @@ func TestNamespaceIndexInvalidDocWrite(t *testing.T) { lifecycle := index.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize() - assert.Error(t, idx.Write(id, tags, time.Time{}, lifecycle)) + assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, time.Time{}, lifecycle))) } func TestNamespaceIndexWriteAfterClose(t *testing.T) { @@ -151,7 +150,7 @@ func TestNamespaceIndexWriteAfterClose(t *testing.T) { lifecycle := index.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize() - assert.Error(t, idx.Write(id, tags, time.Time{}, lifecycle)) + assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, time.Time{}, lifecycle))) } func TestNamespaceIndexWriteQueueError(t *testing.T) { @@ -170,9 +169,9 @@ func TestNamespaceIndexWriteQueueError(t *testing.T) { lifecycle := index.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize() q.EXPECT(). - Insert(gomock.Any(), gomock.Any(), lifecycle). + InsertBatch(gomock.Any()). Return(nil, fmt.Errorf("random err")) - assert.Error(t, idx.Write(id, tags, time.Now(), lifecycle)) + assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, time.Now(), lifecycle))) } func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { @@ -216,11 +215,11 @@ func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { tooOld := now.Add(-1 * idx.bufferPast).Add(-1 * time.Second) lifecycle.EXPECT().OnIndexFinalize() - assert.Error(t, idx.Write(id, tags, tooOld, lifecycle)) + assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, tooOld, lifecycle))) tooNew := now.Add(1 * idx.bufferFuture).Add(1 * time.Second) lifecycle.EXPECT().OnIndexFinalize() - assert.Error(t, idx.Write(id, tags, tooNew, lifecycle)) + assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, tooNew, lifecycle))) } func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { @@ -244,8 +243,8 @@ func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { var wg sync.WaitGroup lifecycle := index.NewMockOnIndexSeries(ctrl) - q.EXPECT().Insert(gomock.Any(), doc.NewDocumentMatcher(d), gomock.Any()).Return(&wg, nil) - assert.NoError(t, idx.Write(id, tags, now, lifecycle)) + q.EXPECT().InsertBatch(gomock.Any()).Return(&wg, nil) + assert.NoError(t, idx.WriteBatch(testIndexWriteEntry(id, tags, now, lifecycle))) } func TestNamespaceIndexInsertQuery(t *testing.T) { @@ -279,7 +278,7 @@ func TestNamespaceIndexInsertQuery(t *testing.T) { lifecycleFns.EXPECT().OnIndexFinalize() lifecycleFns.EXPECT().OnIndexSuccess(ts) - assert.NoError(t, idx.Write(id, tags, now, lifecycleFns)) + assert.NoError(t, idx.WriteBatch(testIndexWriteEntry(id, tags, now, lifecycleFns))) reQuery, err := m3ninxidx.NewRegexpQuery([]byte("name"), []byte("val.*")) assert.NoError(t, err) diff --git a/storage/shard.go b/storage/shard.go index 9cfff469c5..f84a8af800 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -128,6 +128,9 @@ type dbShard struct { newSeriesBootstrapped bool ticking bool shard uint32 + + indexWriteEntryBatchLock sync.Mutex + indexWriteEntryBatch []indexWriteEntry } // NB(r): dbShardRuntimeOptions does not contain its own @@ -838,9 +841,9 @@ func (s *dbShard) writeAndIndex( commitLogSeriesUniqueIndex = entry.index needsIndex := shouldReverseIndex && entry.needsIndexUpdate(timestamp) if err == nil && needsIndex { - entry.onIndexPrepare() - err = s.reverseIndex.Write(entry.series.ID(), entry.series.Tags(), timestamp, entry) + err = s.insertSeriesForIndexing(entry, timestamp, opts.writeNewSeriesAsync) } + // release the reference we got on entry from `writableSeries` entry.decrementReaderWriterCount() if err != nil { return err @@ -1027,6 +1030,44 @@ type insertAsyncResult struct { entry *dbShardEntry } +func (s *dbShard) insertSeriesForIndexing( + entry *dbShardEntry, + timestamp time.Time, + async bool, +) error { + // // inc a ref on the entry to ensure it's valid until the queue acts upon it. + // entry.onIndexPrepare() + wg, err := s.insertQueue.Insert(dbShardInsert{ + entry: entry, + opts: dbShardInsertAsyncOptions{ + hasPendingIndexing: true, + pendingIndex: dbShardPendingIndex{ + timestamp: timestamp, + }, + }, + }) + + // i.e. unable to enqueue into shard insert queue + if err != nil { + // entry.OnIndexFinalize() // release any reference's we've held for indexing + return err + } + + // if operating in async mode, we're done + if async { + return nil + } + + // if indexing in sync mode, wait till we're done and ensure we have have indexed the entry + wg.Wait() + if entry.needsIndexUpdate(timestamp) { + // i.e. indexing failed + return fmt.Errorf("internal error: unable to index series") + } + + return nil +} + func (s *dbShard) insertSeriesAsyncBatched( id ident.ID, tags ident.TagIterator, @@ -1127,6 +1168,17 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { anyPendingAction := false s.Lock() + s.indexWriteEntryBatchLock.Lock() + // reset s.indexWriteEntryBatch from all exit points + defer func() { + var empty indexWriteEntry + for i := range s.indexWriteEntryBatch { + s.indexWriteEntryBatch[i] = empty + } + s.indexWriteEntryBatch = s.indexWriteEntryBatch[:0] + s.indexWriteEntryBatchLock.Unlock() + }() + for i := range inserts { entry, _, err := s.lookupEntryWithLock(inserts[i].entry.series.ID()) if entry != nil { @@ -1200,7 +1252,12 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // only index any entry that hasn't crossed the nextIndexTime if entry.needsIndexUpdate(pendingIndex.timestamp) { entry.onIndexPrepare() - s.reverseIndex.Write(entry.series.ID(), entry.series.Tags(), pendingIndex.timestamp, entry) + s.indexWriteEntryBatch = append(s.indexWriteEntryBatch, indexWriteEntry{ + id: entry.series.ID(), + tags: entry.series.Tags(), + timestamp: pendingIndex.timestamp, + fns: entry, + }) } } @@ -1215,10 +1272,16 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { } } + var err error + // index all requested entries in batch. + if len(s.indexWriteEntryBatch) != 0 { + err = s.reverseIndex.WriteBatch(s.indexWriteEntryBatch) + } + // Avoid goroutine spinning up to close this context ctx.BlockingClose() - return nil + return err } func (s *dbShard) FetchBlocks( diff --git a/storage/storage_mock.go b/storage/storage_mock.go index eb3fa47631..7e83c74251 100644 --- a/storage/storage_mock.go +++ b/storage/storage_mock.go @@ -44,7 +44,6 @@ import ( "github.com/m3db/m3db/storage/series" "github.com/m3db/m3db/x/xcounter" "github.com/m3db/m3db/x/xio" - "github.com/m3db/m3ninx/doc" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" @@ -1383,16 +1382,16 @@ func (m *MocknamespaceIndex) EXPECT() *MocknamespaceIndexMockRecorder { return m.recorder } -// Write mocks base method -func (m *MocknamespaceIndex) Write(id ident.ID, tags ident.Tags, timestamp time.Time, fns index.OnIndexSeries) error { - ret := m.ctrl.Call(m, "Write", id, tags, timestamp, fns) +// WriteBatch mocks base method +func (m *MocknamespaceIndex) WriteBatch(entries []indexWriteEntry) error { + ret := m.ctrl.Call(m, "WriteBatch", entries) ret0, _ := ret[0].(error) return ret0 } -// Write indicates an expected call of Write -func (mr *MocknamespaceIndexMockRecorder) Write(id, tags, timestamp, fns interface{}) *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Write", reflect.TypeOf((*MocknamespaceIndex)(nil).Write), id, tags, timestamp, fns) +// WriteBatch indicates an expected call of WriteBatch +func (mr *MocknamespaceIndexMockRecorder) WriteBatch(entries interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteBatch", reflect.TypeOf((*MocknamespaceIndex)(nil).WriteBatch), entries) } // Query mocks base method @@ -1492,17 +1491,17 @@ func (mr *MocknamespaceIndexInsertQueueMockRecorder) Stop() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stop", reflect.TypeOf((*MocknamespaceIndexInsertQueue)(nil).Stop)) } -// Insert mocks base method -func (m *MocknamespaceIndexInsertQueue) Insert(blockStart time.Time, d doc.Document, s index.OnIndexSeries) (*sync.WaitGroup, error) { - ret := m.ctrl.Call(m, "Insert", blockStart, d, s) +// InsertBatch mocks base method +func (m *MocknamespaceIndexInsertQueue) InsertBatch(entries []index.WriteBatchEntry) (*sync.WaitGroup, error) { + ret := m.ctrl.Call(m, "InsertBatch", entries) ret0, _ := ret[0].(*sync.WaitGroup) ret1, _ := ret[1].(error) return ret0, ret1 } -// Insert indicates an expected call of Insert -func (mr *MocknamespaceIndexInsertQueueMockRecorder) Insert(blockStart, d, s interface{}) *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Insert", reflect.TypeOf((*MocknamespaceIndexInsertQueue)(nil).Insert), blockStart, d, s) +// InsertBatch indicates an expected call of InsertBatch +func (mr *MocknamespaceIndexInsertQueueMockRecorder) InsertBatch(entries interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InsertBatch", reflect.TypeOf((*MocknamespaceIndexInsertQueue)(nil).InsertBatch), entries) } // MockdatabaseBootstrapManager is a mock of databaseBootstrapManager interface diff --git a/storage/types.go b/storage/types.go index 19cbbf0d8e..94b0fe4283 100644 --- a/storage/types.go +++ b/storage/types.go @@ -40,7 +40,6 @@ import ( "github.com/m3db/m3db/storage/series" "github.com/m3db/m3db/x/xcounter" "github.com/m3db/m3db/x/xio" - "github.com/m3db/m3ninx/doc" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" @@ -429,12 +428,9 @@ type databaseShard interface { // namespaceIndex indexes namespace writes. type namespaceIndex interface { - // Write indexes timeseries ID by provided Tags. - Write( - id ident.ID, - tags ident.Tags, - timestamp time.Time, - fns index.OnIndexSeries, + // WriteBatch indexes the provided entries. + WriteBatch( + entries []indexWriteEntry, ) error // Query resolves the given query into known IDs. @@ -457,6 +453,13 @@ type namespaceIndex interface { Close() error } +type indexWriteEntry struct { + id ident.ID + tags ident.Tags + timestamp time.Time + fns index.OnIndexSeries +} + // namespaceIndexTickResult are details about the work performed by the namespaceIndex // during a Tick(). type namespaceIndexTickResult struct { @@ -480,7 +483,13 @@ type namespaceIndexInsertQueue interface { // inserts to the index asynchronously. It executes the provided callbacks // based on the result of the execution. The returned wait group can be used // if the insert is required to be synchronous. - Insert(blockStart time.Time, d doc.Document, s index.OnIndexSeries) (*sync.WaitGroup, error) + // Insert(blockStart time.Time, d doc.Document, s index.OnIndexSeries) (*sync.WaitGroup, error) + + // InsertBatch inserts the provided documents to the index queue which processes + // inserts to the index asynchronously. It executes the provided callbacks + // based on the result of the execution. The returned wait group can be used + // if the insert is required to be synchronous. + InsertBatch(entries []index.WriteBatchEntry) (*sync.WaitGroup, error) } // databaseBootstrapManager manages the bootstrap process. From e06de11d4e01733a457671e14b7c5d5a4a21de8f Mon Sep 17 00:00:00 2001 From: Prateek Rungta Date: Thu, 10 May 2018 18:16:47 -0400 Subject: [PATCH 02/33] use a batch of batches in the index insert queue --- storage/index.go | 10 +++++++++- storage/index_insert_queue.go | 13 ++++++++----- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/storage/index.go b/storage/index.go index 28d825818d..386ebf9667 100644 --- a/storage/index.go +++ b/storage/index.go @@ -187,7 +187,7 @@ func newNamespaceIndexWithOptions( } // allocate indexing queue and start it up. - queue := newIndexQueueFn(idx.writeBatch, nowFn, opts.InstrumentOptions().MetricsScope()) + queue := newIndexQueueFn(idx.writeBatches, nowFn, opts.InstrumentOptions().MetricsScope()) if err := queue.Start(); err != nil { return nil, err } @@ -321,6 +321,14 @@ func (i *nsIndex) enqueueBatch( return nil } +func (i *nsIndex) writeBatches( + batches [][]index.WriteBatchEntry, +) { + for _, batch := range batches { + i.writeBatch(batch) + } +} + // NB: this function is called by the namespaceIndexInsertQueue. // FOLLOWUP(prateek): propagate error back up from here to the indexInsertQueue // so that we can notify users of success/failure correctly in the case of diff --git a/storage/index_insert_queue.go b/storage/index_insert_queue.go index 2e1be36689..4e021630a3 100644 --- a/storage/index_insert_queue.go +++ b/storage/index_insert_queue.go @@ -175,7 +175,7 @@ func (q *nsIndexInsertQueue) InsertBatch( return nil, errNewSeriesIndexRateLimitExceeded } } - q.currBatch.inserts = append(q.currBatch.inserts, entries...) + q.currBatch.inserts = append(q.currBatch.inserts, entries) wg := q.currBatch.wg q.Unlock() @@ -227,11 +227,11 @@ func (q *nsIndexInsertQueue) Stop() error { return nil } -type nsIndexInsertBatchFn func(inserts []index.WriteBatchEntry) +type nsIndexInsertBatchFn func(inserts [][]index.WriteBatchEntry) type nsIndexInsertBatch struct { wg *sync.WaitGroup - inserts []index.WriteBatchEntry + inserts [][]index.WriteBatchEntry } var nsIndexInsertZeroed index.WriteBatchEntry @@ -240,8 +240,11 @@ func (b *nsIndexInsertBatch) Reset() { b.wg = &sync.WaitGroup{} // We always expect to be waiting for an index b.wg.Add(1) - for i := range b.inserts { - b.inserts[i] = nsIndexInsertZeroed + for i, group := range b.inserts { + for j := range group { + group[j] = nsIndexInsertZeroed + } + b.inserts[i] = group[:0] } b.inserts = b.inserts[:0] } From 2c503fdb05f9b2bbd5b9abbfd09f39c00f75c189 Mon Sep 17 00:00:00 2001 From: Prateek Rungta Date: Fri, 11 May 2018 01:05:59 -0400 Subject: [PATCH 03/33] delay ident -> doc conversion --- storage/index.go | 92 +++++++++---------------- storage/index/block.go | 124 +++++++++++++++------------------- storage/index/index_mock.go | 2 +- storage/index/types.go | 109 +++++++++++++++++++++++------- storage/index_insert_queue.go | 12 ++-- storage/shard.go | 35 ++++------ storage/storage_mock.go | 2 +- storage/types.go | 15 +--- 8 files changed, 193 insertions(+), 198 deletions(-) diff --git a/storage/index.go b/storage/index.go index 386ebf9667..a2ea3c4950 100644 --- a/storage/index.go +++ b/storage/index.go @@ -31,7 +31,6 @@ import ( "github.com/m3db/m3db/retention" "github.com/m3db/m3db/storage/bootstrap/result" "github.com/m3db/m3db/storage/index" - "github.com/m3db/m3db/storage/index/convert" "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3x/context" xerrors "github.com/m3db/m3x/errors" @@ -222,51 +221,37 @@ func newNamespaceIndexWithOptions( // => index.writeBatch() func (i *nsIndex) WriteBatch( - entries []indexWriteEntry, + entries []index.WriteBatchEntry, ) error { // Ensure timestamp is not too old/new based on retention policies. now := i.nowFn() futureLimit := now.Add(1 * i.bufferFuture) pastLimit := now.Add(-1 * i.bufferPast) - // TODO(prateek): pool this type - queueBatch := make([]index.WriteBatchEntry, 0, len(entries)) - - for _, entry := range entries { + var emptyEntry index.WriteBatchEntry + for j := range entries { var ( - id = entry.id - tags = entry.tags - timestamp = entry.timestamp - fns = entry.fns + timestamp = entries[j].Timestamp + onIndexFn = entries[j].OnIndexSeries ) if !futureLimit.After(timestamp) { - fns.OnIndexFinalize() + onIndexFn.OnIndexFinalize() + entries[j] = emptyEntry // indicate we don't need to index this. + // TODO(prateek): capture that this needs to return m3dberrors.ErrTooFuture continue - // TODO(prateek): return m3dberrors.ErrTooFuture } if !pastLimit.Before(timestamp) { - fns.OnIndexFinalize() + onIndexFn.OnIndexFinalize() + entries[j] = emptyEntry // indicate we don't need to index this. + // TODO(prateek): capture that this needs to return m3dberrors.ErrTooPast continue - // TODO(prateek): return m3dberrors.ErrTooPast } - // Ensure metric can be converted to a valid document - d, err := convert.FromMetric(id, tags) - if err != nil { - fns.OnIndexFinalize() - continue - // TODO(prateek): return err - } - - indexBlockStart := xtime.ToUnixNano(timestamp.Truncate(i.blockSize)) - queueBatch = append(queueBatch, index.WriteBatchEntry{ - BlockStart: indexBlockStart, - Document: d, - OnIndexSeries: fns, - }) + // update the timestamp to the blockstart for the block it needs to be sent to + entries[j].Timestamp = timestamp.Truncate(i.blockSize) } - return i.enqueueBatch(queueBatch) + return i.enqueueBatch(entries) } func (i *nsIndex) enqueueBatch( @@ -276,9 +261,7 @@ func (i *nsIndex) enqueueBatch( if !i.isOpenWithRLock() { i.state.RUnlock() i.metrics.InsertAfterClose.Inc(1) - for _, entry := range entries { - entry.OnIndexSeries.OnIndexFinalize() - } + index.WriteBatchEntriesFinalizer(entries).Finalize() return errDbIndexUnableToWriteClosed } @@ -291,9 +274,7 @@ func (i *nsIndex) enqueueBatch( // if we're unable to index, we still have to finalize the reference we hold. if err != nil { - for _, entry := range entries { - entry.OnIndexSeries.OnIndexFinalize() - } + index.WriteBatchEntriesFinalizer(entries).Finalize() return err } // once the write has been queued in the indexInsertQueue, it assumes @@ -321,47 +302,40 @@ func (i *nsIndex) enqueueBatch( return nil } -func (i *nsIndex) writeBatches( - batches [][]index.WriteBatchEntry, -) { - for _, batch := range batches { - i.writeBatch(batch) - } -} - -// NB: this function is called by the namespaceIndexInsertQueue. +// WriteBatches is called by the indexInsertQueue. // FOLLOWUP(prateek): propagate error back up from here to the indexInsertQueue // so that we can notify users of success/failure correctly in the case of // sync'd inserts. -func (i *nsIndex) writeBatch(inserts []index.WriteBatchEntry) { +func (i *nsIndex) writeBatches( + batches [][]index.WriteBatchEntry, +) { // NB(prateek): we use a read lock to guard against mutation of the // indexBlocks, mutations within the underlying blocks are guarded // by primitives internal to it. i.state.RLock() defer i.state.RUnlock() - if !i.isOpenWithRLock() { // NB(prateek): deliberately skip calling any of the `OnIndexFinalize` methods // on the provided inserts to terminate quicker during shutdown. - i.metrics.InsertAfterClose.Inc(int64(len(inserts))) + return } - // we sort the inserts by which block they're applicable for, and do the inserts - // for each block. - writesByBlockStart := index.WriteBatchEntryByBlockStart(inserts) - sort.Sort(writesByBlockStart) - writesByBlockStart.ForEachBlockStart(i.writeBatchForBlockStartWithRLock) + for _, batch := range batches { + // we sort the inserts by which block they're applicable for, and do the inserts + // for each block. + writesByBlockStart := index.WriteBatchEntryByBlockStartAndID(batch) + sort.Sort(writesByBlockStart) + writesByBlockStart.ForEachBlockStart(i.writeBatchForBlockStartWithRLock) + } } func (i *nsIndex) writeBatchForBlockStartWithRLock( - blockStart xtime.UnixNano, inserts []index.WriteBatchEntry, + blockStart time.Time, inserts index.WriteBatchEntryByBlockStartAndID, ) { // ensure we have an index block for the specified blockStart. - block, err := i.ensureBlockPresentWithRLock(blockStart.ToTime()) + block, err := i.ensureBlockPresentWithRLock(blockStart) if err != nil { - for _, insert := range inserts { - insert.OnIndexSeries.OnIndexFinalize() - } + index.WriteBatchEntriesFinalizer(inserts).Finalize() i.logger.WithFields( xlog.NewField("blockStart", blockStart), xlog.NewField("numWrites", len(inserts)), @@ -373,14 +347,12 @@ func (i *nsIndex) writeBatchForBlockStartWithRLock( // i.e. we have the block and the inserts, perform the writes. result, err := block.WriteBatch(inserts) + // NB: we don't need to do anything to the OnIndexSeries refs in `inserts` at this point, // the index.Block WriteBatch assumes responsibility for calling the appropriate methods. if numErr := result.NumError; numErr != 0 { i.metrics.AsyncInsertErrors.Inc(numErr) } - if err != nil { - // i.logger.Errorf("unable to write to index, dropping inserts. [%v]", err) - } } // Bootstrap bootstraps the index with the provide blocks. diff --git a/storage/index/block.go b/storage/index/block.go index 6541332936..7a6200b09d 100644 --- a/storage/index/block.go +++ b/storage/index/block.go @@ -26,7 +26,7 @@ import ( "sync" "time" - "github.com/m3db/m3ninx/doc" + "github.com/m3db/m3db/storage/index/convert" m3ninxindex "github.com/m3db/m3ninx/index" "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3ninx/index/segment/mem" @@ -65,7 +65,6 @@ type block struct { state blockState bootstrappedSegments []segment.Segment segment segment.MutableSegment - writeBatch m3ninxindex.Batch newExecutorFn newExecutorFn startTime time.Time @@ -90,9 +89,6 @@ func NewBlock( b := &block{ state: blockStateOpen, segment: seg, - writeBatch: m3ninxindex.Batch{ - AllowPartialUpdates: true, - }, startTime: startTime, endTime: startTime.Add(blockSize), @@ -111,86 +107,74 @@ func (b *block) EndTime() time.Time { return b.endTime } -func (b *block) updateWriteBatchWithLock(inserts []WriteBatchEntry) { - for _, insert := range inserts { - b.writeBatch.Docs = append(b.writeBatch.Docs, insert.Document) - } -} - -func (b *block) resetWriteBatchWithLock() { - var emptyDoc doc.Document - for i := range b.writeBatch.Docs { - b.writeBatch.Docs[i] = emptyDoc - } - b.writeBatch.Docs = b.writeBatch.Docs[:0] -} - -func (b *block) WriteBatch(inserts []WriteBatchEntry) (WriteBatchResult, error) { - b.Lock() - defer func() { - b.resetWriteBatchWithLock() - b.Unlock() - }() +func (b *block) WriteBatch(inserts WriteBatchEntryByBlockStartAndID) (WriteBatchResult, error) { + b.RLock() + defer b.RUnlock() if b.state != blockStateOpen { // NB: releasing all references to inserts - for _, insert := range inserts { - insert.OnIndexSeries.OnIndexFinalize() - } + WriteBatchEntriesFinalizer(inserts).Finalize() return WriteBatchResult{ NumError: int64(len(inserts)), }, b.writeBatchErrorInvalidState(b.state) } - b.updateWriteBatchWithLock(inserts) - err := b.segment.InsertBatch(b.writeBatch) - if err == nil { - for _, insert := range inserts { - insert.OnIndexSeries.OnIndexSuccess(b.endTime) - insert.OnIndexSeries.OnIndexFinalize() + var ( + multiErr xerrors.MultiError + result WriteBatchResult + ) + inserts.ForEachID(func(writesForID WriteBatchEntryByBlockStartAndID) { + // all writes are guaranteed to have the same ID by this point, and further + // we're guaranteed that at least a single element exists in the slice. + id := writesForID[0].ID + tags := writesForID[0].Tags + + if id == nil { + return } - return WriteBatchResult{ - NumSuccess: int64(len(inserts)), - }, nil - } - partialErr, ok := err.(*m3ninxindex.BatchPartialError) - if !ok { // should never happen - err := b.unknownWriteBatchInvariantError(err) - // NB: marking all the inserts as failure, cause we don't know which ones failed - for _, insert := range inserts { - insert.OnIndexSeries.OnIndexFinalize() - insert.Document = doc.Document{} - insert.OnIndexSeries = nil + // define some helper functions to help keep the code below cleaner. + failFn := func(err error) { + multiErr = multiErr.Add(err) + result.NumError += int64(len(writesForID)) + // finalize all refs + WriteBatchEntriesFinalizer(writesForID).Finalize() + } + successFn := func() { + result.NumSuccess += int64(len(writesForID)) + // mark the first ref success (can mark any ref success here, because they're backed by + // by the same entry). Could also mark all of them success but it wouldn't buy us anything. + writesForID[0].OnIndexSeries.OnIndexSuccess(b.endTime) + // we do need to finalize all refs as each is an extra inc we need to dec + WriteBatchEntriesFinalizer(writesForID).Finalize() } - return WriteBatchResult{NumError: int64(len(inserts))}, err - } - // first finalize all the responses which were errors, and mark them - // nil to indicate they're done. - numErr := len(partialErr.Indices()) - for _, idx := range partialErr.Indices() { - inserts[idx].OnIndexSeries.OnIndexFinalize() - inserts[idx].OnIndexSeries = nil - inserts[idx].Document = doc.Document{} - } + contains, err := b.segment.ContainsID(id.Bytes()) + if contains && err == nil { + // can early terminate as the active segment already has the ID + successFn() + return + } - // mark all non-error inserts success, so we don't repeatedly index them, - // and then finalize any held references. - for _, insert := range inserts { - if insert.OnIndexSeries == nil { - continue + // NB(prateek): we delay the conversion from ident types -> doc as we want to minimize the allocs + // of idents until we're sure we actually need to index a series. This helps keep memory usage low + // when we receive a large spike of new metrics. + d, err := convert.FromMetric(id, tags) + if err != nil { + failFn(err) + return } - insert.OnIndexSeries.OnIndexSuccess(b.endTime) - insert.OnIndexSeries.OnIndexFinalize() - insert.OnIndexSeries = nil - insert.Document = doc.Document{} - } - return WriteBatchResult{ - NumSuccess: int64(len(inserts) - numErr), - NumError: int64(numErr), - }, partialErr + // now actually perform the insert + if _, err := b.segment.Insert(d); err != nil { + failFn(err) + return + } + + successFn() + }) + + return result, multiErr.FinalError() } func (b *block) executorWithRLock() (search.Executor, error) { diff --git a/storage/index/index_mock.go b/storage/index/index_mock.go index 72866d26aa..8c5037b17c 100644 --- a/storage/index/index_mock.go +++ b/storage/index/index_mock.go @@ -251,7 +251,7 @@ func (mr *MockBlockMockRecorder) Tick(arg0 interface{}) *gomock.Call { } // WriteBatch mocks base method -func (m *MockBlock) WriteBatch(arg0 []WriteBatchEntry) (WriteBatchResult, error) { +func (m *MockBlock) WriteBatch(arg0 WriteBatchEntryByBlockStartAndID) (WriteBatchResult, error) { ret := m.ctrl.Call(m, "WriteBatch", arg0) ret0, _ := ret[0].(WriteBatchResult) ret1, _ := ret[1].(error) diff --git a/storage/index/types.go b/storage/index/types.go index 2e2ef061ca..b260aaeff6 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -21,6 +21,7 @@ package index import ( + "bytes" "time" "github.com/m3db/m3db/clock" @@ -32,7 +33,6 @@ import ( "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" "github.com/m3db/m3x/pool" - xtime "github.com/m3db/m3x/time" ) var ( @@ -136,7 +136,7 @@ type Block interface { EndTime() time.Time // WriteBatch writes a batch of provided entries. - WriteBatch([]WriteBatchEntry) (WriteBatchResult, error) + WriteBatch(WriteBatchEntryByBlockStartAndID) (WriteBatchResult, error) // Query resolves the given query into known IDs. Query( @@ -164,13 +164,6 @@ type Block interface { Close() error } -// WriteBatchEntry captures a document to index, and the lifecycle hooks to call thereafter. -type WriteBatchEntry struct { - BlockStart xtime.UnixNano - Document doc.Document - OnIndexSeries OnIndexSeries -} - // WriteBatchResult returns statistics about the WriteBatch execution. type WriteBatchResult struct { NumSuccess int64 @@ -183,41 +176,111 @@ type BlockTickResult struct { NumDocs int64 } +// WriteBatchEntry captures a document to index, and the lifecycle hooks to call thereafter. +type WriteBatchEntry struct { + ID ident.ID + Tags ident.Tags + Timestamp time.Time + OnIndexSeries OnIndexSeries +} + +// WriteBatchEntriesFinalizer is a utility type to provide syntactic sugar to finalize references in the slice. +type WriteBatchEntriesFinalizer []WriteBatchEntry + +// Finalize finalizes all the references in the provided slice. +func (w WriteBatchEntriesFinalizer) Finalize() { + for _, entry := range w { + if entry.OnIndexSeries != nil { + entry.OnIndexSeries.OnIndexFinalize() + } + } +} + // WriteBatchEntryByBlockStart implements sort.Interface for WriteBatchEntry slices -// based on the BlockStart field. -type WriteBatchEntryByBlockStart []WriteBatchEntry +// based on the Timestamp and ID fields. +type WriteBatchEntryByBlockStartAndID []WriteBatchEntry + +func (w WriteBatchEntryByBlockStartAndID) Len() int { return len(w) } +func (w WriteBatchEntryByBlockStartAndID) Swap(i, j int) { w[i], w[j] = w[j], w[i] } +func (w WriteBatchEntryByBlockStartAndID) Less(i, j int) bool { + if !w[i].Timestamp.Equal(w[j].Timestamp) { + return w[i].Timestamp.Before(w[j].Timestamp) + } -func (w WriteBatchEntryByBlockStart) Len() int { return len(w) } -func (w WriteBatchEntryByBlockStart) Swap(i, j int) { w[i], w[j] = w[j], w[i] } -func (w WriteBatchEntryByBlockStart) Less(i, j int) bool { return w[i].BlockStart < w[j].BlockStart } + if w[j].ID == nil { + return false + } + + if w[i].ID == nil { // i.e. w[j] != nil + return true + } + + // i.e. both w[i] and w[j] are != nil + return bytes.Compare(w[i].ID.Bytes(), w[j].ID.Bytes()) < 0 +} // ForEachBlockStartFn is lambda to iterate over WriteBatchEntry(s) a single blockStart at a time. -type ForEachBlockStartFn func(blockStart xtime.UnixNano, writes []WriteBatchEntry) +type ForEachBlockStartFn func(timestamp time.Time, writes WriteBatchEntryByBlockStartAndID) -// ForEachBlockStart iterates over the provided WriteBatchEntryByBlockStart, and calls `fn` on each +// ForEachIDFn is lambda to iterate over WriteBatchEntry(s) a single ID at a time. +type ForEachIDFn func(writes WriteBatchEntryByBlockStartAndID) + +// ForEachBlockStart iterates over the provided WriteBatchEntryByBlockStartAndID, and calls `fn` on each // group of elements with the same blockStart. -func (w WriteBatchEntryByBlockStart) ForEachBlockStart(fn ForEachBlockStartFn) { +func (w WriteBatchEntryByBlockStartAndID) ForEachBlockStart(fn ForEachBlockStartFn) { var ( - startIdx = 0 - lastNanos xtime.UnixNano + startIdx = 0 + lastTime time.Time ) for i := 0; i < len(w); i++ { elem := w[i] - if elem.BlockStart != lastNanos { - lastNanos = elem.BlockStart + if !elem.Timestamp.Equal(lastTime) { + lastTime = elem.Timestamp // We only want to call the the ForEachBlockStartFn once we have calculated the entire group, // i.e. once we have gone past the last element for a given blockStart, but the first element // in the slice is a special case because we are always starting a new group at that point. if i == 0 { continue } - fn(w[startIdx].BlockStart, w[startIdx:i]) + fn(w[startIdx].Timestamp, w[startIdx:i]) + startIdx = i + } + } + // spill over + if startIdx < len(w) { + fn(w[startIdx].Timestamp, w[startIdx:]) + } +} + +// ForEachID iterates over the provided WriteBatchEntryByBlockStartAndID, and calls `fn` on each +// group of elements with the same ID. +func (w WriteBatchEntryByBlockStartAndID) ForEachID(fn ForEachIDFn) { + var ( + startIdx = 0 + lastBytes []byte + ) + for i := 0; i < len(w); i++ { + elem := w[i] + var elemBytes []byte + if elem.ID != nil { + elemBytes = elem.ID.Bytes() + } + // TODO(prateek): need to write a test to ensure this handles slice with nil IDs correctly + if !bytes.Equal(lastBytes, elemBytes) { + lastBytes = elemBytes + // We only want to call the the ForEachID once we have calculated the entire group, + // i.e. once we have gone past the last element for a given ID, but the first element + // in the slice is a special case because we are always starting a new group at that point. + if i == 0 { + continue + } + fn(w[startIdx:i]) startIdx = i } } // spill over if startIdx < len(w) { - fn(w[startIdx].BlockStart, w[startIdx:]) + fn(w[startIdx:]) } } diff --git a/storage/index_insert_queue.go b/storage/index_insert_queue.go index 4e021630a3..0b01aa40e0 100644 --- a/storage/index_insert_queue.go +++ b/storage/index_insert_queue.go @@ -186,7 +186,7 @@ func (q *nsIndexInsertQueue) InsertBatch( // Loop busy, already ready to consume notification } - q.metrics.numPending.Inc(1) + q.metrics.numPending.Inc(int64(len(entries))) return wg, nil } @@ -234,17 +234,13 @@ type nsIndexInsertBatch struct { inserts [][]index.WriteBatchEntry } -var nsIndexInsertZeroed index.WriteBatchEntry - func (b *nsIndexInsertBatch) Reset() { b.wg = &sync.WaitGroup{} // We always expect to be waiting for an index b.wg.Add(1) - for i, group := range b.inserts { - for j := range group { - group[j] = nsIndexInsertZeroed - } - b.inserts[i] = group[:0] + for i := range b.inserts { + // TODO(prateek): if we start pooling `[]index.WriteBatchEntry`, then we could return to the pool here. + b.inserts[i] = nil } b.inserts = b.inserts[:0] } diff --git a/storage/shard.go b/storage/shard.go index f84a8af800..465419dc27 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -128,9 +128,6 @@ type dbShard struct { newSeriesBootstrapped bool ticking bool shard uint32 - - indexWriteEntryBatchLock sync.Mutex - indexWriteEntryBatch []indexWriteEntry } // NB(r): dbShardRuntimeOptions does not contain its own @@ -1166,19 +1163,9 @@ func (s *dbShard) insertNewShardEntryWithLock(entry *dbShardEntry) { func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { anyPendingAction := false + numPendingIndexing := 0 s.Lock() - s.indexWriteEntryBatchLock.Lock() - // reset s.indexWriteEntryBatch from all exit points - defer func() { - var empty indexWriteEntry - for i := range s.indexWriteEntryBatch { - s.indexWriteEntryBatch[i] = empty - } - s.indexWriteEntryBatch = s.indexWriteEntryBatch[:0] - s.indexWriteEntryBatchLock.Unlock() - }() - for i := range inserts { entry, _, err := s.lookupEntryWithLock(inserts[i].entry.series.ID()) if entry != nil { @@ -1195,6 +1182,10 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { anyPendingAction = anyPendingAction || hasPendingWrite || hasPendingRetrievedBlock || hasPendingIndexing + if hasPendingIndexing { + numPendingIndexing++ + } + if hasPendingIndexing || hasPendingWrite || hasPendingRetrievedBlock { // We're definitely writing a value, ensure that the pending write is // visible before we release the lookup write lock @@ -1230,6 +1221,8 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // Perform any indexing, pending writes or pending retrieved blocks outside of lock ctx := s.contextPool.Get() + // TODO(prateek): pool this type + indexBatch := make([]index.WriteBatchEntry, 0, numPendingIndexing) for i := range inserts { var ( entry = inserts[i].entry @@ -1252,11 +1245,11 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // only index any entry that hasn't crossed the nextIndexTime if entry.needsIndexUpdate(pendingIndex.timestamp) { entry.onIndexPrepare() - s.indexWriteEntryBatch = append(s.indexWriteEntryBatch, indexWriteEntry{ - id: entry.series.ID(), - tags: entry.series.Tags(), - timestamp: pendingIndex.timestamp, - fns: entry, + indexBatch = append(indexBatch, index.WriteBatchEntry{ + ID: entry.series.ID(), + Tags: entry.series.Tags(), + Timestamp: pendingIndex.timestamp, + OnIndexSeries: entry, }) } } @@ -1274,8 +1267,8 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { var err error // index all requested entries in batch. - if len(s.indexWriteEntryBatch) != 0 { - err = s.reverseIndex.WriteBatch(s.indexWriteEntryBatch) + if len(indexBatch) != 0 { + err = s.reverseIndex.WriteBatch(indexBatch) } // Avoid goroutine spinning up to close this context diff --git a/storage/storage_mock.go b/storage/storage_mock.go index 7e83c74251..4d4b81cebf 100644 --- a/storage/storage_mock.go +++ b/storage/storage_mock.go @@ -1383,7 +1383,7 @@ func (m *MocknamespaceIndex) EXPECT() *MocknamespaceIndexMockRecorder { } // WriteBatch mocks base method -func (m *MocknamespaceIndex) WriteBatch(entries []indexWriteEntry) error { +func (m *MocknamespaceIndex) WriteBatch(entries []index.WriteBatchEntry) error { ret := m.ctrl.Call(m, "WriteBatch", entries) ret0, _ := ret[0].(error) return ret0 diff --git a/storage/types.go b/storage/types.go index 94b0fe4283..245f8356b9 100644 --- a/storage/types.go +++ b/storage/types.go @@ -430,7 +430,7 @@ type databaseShard interface { type namespaceIndex interface { // WriteBatch indexes the provided entries. WriteBatch( - entries []indexWriteEntry, + entries []index.WriteBatchEntry, ) error // Query resolves the given query into known IDs. @@ -453,13 +453,6 @@ type namespaceIndex interface { Close() error } -type indexWriteEntry struct { - id ident.ID - tags ident.Tags - timestamp time.Time - fns index.OnIndexSeries -} - // namespaceIndexTickResult are details about the work performed by the namespaceIndex // during a Tick(). type namespaceIndexTickResult struct { @@ -479,12 +472,6 @@ type namespaceIndexInsertQueue interface { // Stop stops accepting writes in the queue. Stop() error - // Insert inserts the provided document to the index queue which processes - // inserts to the index asynchronously. It executes the provided callbacks - // based on the result of the execution. The returned wait group can be used - // if the insert is required to be synchronous. - // Insert(blockStart time.Time, d doc.Document, s index.OnIndexSeries) (*sync.WaitGroup, error) - // InsertBatch inserts the provided documents to the index queue which processes // inserts to the index asynchronously. It executes the provided callbacks // based on the result of the execution. The returned wait group can be used From b8f118a67f168af51f6b1b08019c9bfc9a0e36c8 Mon Sep 17 00:00:00 2001 From: Prateek Rungta Date: Sat, 12 May 2018 03:35:17 -0400 Subject: [PATCH 04/33] Track index write attempt state within dbShardEntry --- storage/index.go | 15 ++- storage/index/block.go | 3 +- storage/index/types.go | 10 +- storage/shard.go | 183 ++++++++++++++++++++++++++++------ storage/shard_insert_queue.go | 9 +- storage/types.go | 6 ++ 6 files changed, 182 insertions(+), 44 deletions(-) diff --git a/storage/index.go b/storage/index.go index a2ea3c4950..ff3c074c16 100644 --- a/storage/index.go +++ b/storage/index.go @@ -203,6 +203,10 @@ func newNamespaceIndexWithOptions( return idx, nil } +func (i *nsIndex) BlockStartForWriteTime(writeTime time.Time) xtime.UnixNano { + return xtime.ToUnixNano(writeTime.Truncate(i.blockSize)) +} + // NB(prateek): including the call chains leading to this point: // // - For new entry (previously unseen in the shard): @@ -231,25 +235,26 @@ func (i *nsIndex) WriteBatch( var emptyEntry index.WriteBatchEntry for j := range entries { var ( - timestamp = entries[j].Timestamp - onIndexFn = entries[j].OnIndexSeries + timestamp = entries[j].Timestamp + onIndexFn = entries[j].OnIndexSeries + blockStart = i.BlockStartForWriteTime(timestamp) ) if !futureLimit.After(timestamp) { - onIndexFn.OnIndexFinalize() + onIndexFn.OnIndexFinalize(blockStart) entries[j] = emptyEntry // indicate we don't need to index this. // TODO(prateek): capture that this needs to return m3dberrors.ErrTooFuture continue } if !pastLimit.Before(timestamp) { - onIndexFn.OnIndexFinalize() + onIndexFn.OnIndexFinalize(blockStart) entries[j] = emptyEntry // indicate we don't need to index this. // TODO(prateek): capture that this needs to return m3dberrors.ErrTooPast continue } // update the timestamp to the blockstart for the block it needs to be sent to - entries[j].Timestamp = timestamp.Truncate(i.blockSize) + entries[j].Timestamp = blockStart.ToTime() } return i.enqueueBatch(entries) } diff --git a/storage/index/block.go b/storage/index/block.go index 7a6200b09d..2df4dce820 100644 --- a/storage/index/block.go +++ b/storage/index/block.go @@ -36,6 +36,7 @@ import ( "github.com/m3db/m3x/context" xerrors "github.com/m3db/m3x/errors" "github.com/m3db/m3x/instrument" + xtime "github.com/m3db/m3x/time" ) var ( @@ -144,7 +145,7 @@ func (b *block) WriteBatch(inserts WriteBatchEntryByBlockStartAndID) (WriteBatch result.NumSuccess += int64(len(writesForID)) // mark the first ref success (can mark any ref success here, because they're backed by // by the same entry). Could also mark all of them success but it wouldn't buy us anything. - writesForID[0].OnIndexSeries.OnIndexSuccess(b.endTime) + writesForID[0].OnIndexSeries.OnIndexSuccess(xtime.ToUnixNano(b.endTime)) // we do need to finalize all refs as each is an extra inc we need to dec WriteBatchEntriesFinalizer(writesForID).Finalize() } diff --git a/storage/index/types.go b/storage/index/types.go index b260aaeff6..bbbcf43d56 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -33,6 +33,7 @@ import ( "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" "github.com/m3db/m3x/pool" + xtime "github.com/m3db/m3x/time" ) var ( @@ -118,12 +119,13 @@ type OnIndexSeries interface { // OnIndexSuccess is executed when an entry is successfully indexed. The // provided value for `indexEntryExpiry` describes the TTL for the indexed // entry. - OnIndexSuccess(indexEntryExpiry time.Time) + OnIndexSuccess(indexEntryExpiry xtime.UnixNano) // OnIndexFinalize is executed when the index no longer holds any references // to the provided resources. It can be used to cleanup any resources held - // during the course of indexing. - OnIndexFinalize() + // during the course of indexing. `blockStart` is the startTime of the index + // block for which the write was attempted. + OnIndexFinalize(blockStart xtime.UnixNano) } // Block represents a collection of segments. Each `Block` is a complete reverse @@ -191,7 +193,7 @@ type WriteBatchEntriesFinalizer []WriteBatchEntry func (w WriteBatchEntriesFinalizer) Finalize() { for _, entry := range w { if entry.OnIndexSeries != nil { - entry.OnIndexSeries.OnIndexFinalize() + entry.OnIndexSeries.OnIndexFinalize(xtime.ToUnixNano(entry.Timestamp)) } } } diff --git a/storage/shard.go b/storage/shard.go index 465419dc27..42c831b6ca 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -180,6 +180,20 @@ type dbShardEntry struct { index uint64 curReadWriters int32 reverseIndex struct { + // NB(prateek): writeAttemptedBlockstart[1-2]Nanos is used to indicate the index block start(s) + // for which this entry has a pending indexing operation (if any). The value for + // the blockstart is computed based upon timestamp of writes to the entry. When + // a write comes in for series, we compute this token and compare against the value + // set on the entry. If the entry has the same value, then we know there is already + // a write pending for the entry in the index for the given block start, and don't + // need to attempt another one. If not, we need to index it. + // The following semantics hold for this type: + // - `writeAttemptedBlockstartNanos == 0` indicates no write is pending for this entry. + // - `writeAttemptedBlockStartNanos != 0` indicates a write is pending for this entry. + // We actually require two blockStarts, because there can be writes issued for two blocks at the + // same time, and we need to track both of them. + writeAttemptedBlockStart1Nanos int64 + writeAttemptedBlockStart2Nanos int64 // NB(prateek): nextWriteTimeNanos is the UnixNanos until // the next index write is required. We use an atomic instead // of a time.Time to avoid using a Mutex to guard it. @@ -199,20 +213,105 @@ func (entry *dbShardEntry) decrementReaderWriterCount() { atomic.AddInt32(&entry.curReadWriters, -1) } -func (entry *dbShardEntry) needsIndexUpdate(writeTime time.Time) bool { - return atomic.LoadInt64(&entry.reverseIndex.nextWriteTimeNanos) < writeTime.UnixNano() +func (entry *dbShardEntry) hasIndexTTLGreaterThan(writeTime time.Time) bool { + nextWriteNanos := atomic.LoadInt64(&entry.reverseIndex.nextWriteTimeNanos) + return nextWriteNanos > int64(xtime.ToUnixNano(writeTime)) +} + +// NB(prateek): needsIndexUpdate is a CAS, i.e. when this method returns true, it +// also sets state on the entry to indicate that a write for the given blockStart +// is going to be sent to the index, and other go routines should not attempt the +// same write. Callers are expected to ensure they follow this guideline. +// Further, every call to needsIndexUpdate which returns true needs to have a corresponding +// OnIndexFinalze() call. This is reqiured for correct lifecycle maintenance. +func (entry *dbShardEntry) needsIndexUpdate(indexblockStartForWrite xtime.UnixNano) bool { + // check if the entry has a TTL indicating it does not need a write till a + // time later than the provided windexblockStartForWrite. If so, we know the + // entry does not need to be indexed. More on this in a NB at the end of this function. + nextWriteNanos := atomic.LoadInt64(&entry.reverseIndex.nextWriteTimeNanos) + if nextWriteNanos > int64(indexblockStartForWrite) { + return false + } + + var ( + entryBlock1 = &entry.reverseIndex.writeAttemptedBlockStart1Nanos + entryBlock2 = &entry.reverseIndex.writeAttemptedBlockStart1Nanos + ) + + // i.e. no TTL has been marked on the entry yet. check if a write has been attempted for the entry at all. + if atomic.CompareAndSwapInt64(entryBlock1, int64(indexblockStartForWrite), int64(indexblockStartForWrite)) { + // i.e. we're already attempting a write for this block start and are tracking this at entryBlock1 + return false + } + + if atomic.CompareAndSwapInt64(entryBlock2, int64(indexblockStartForWrite), int64(indexblockStartForWrite)) { + // i.e. we're already attempting a write for this block start and are tracking this at entryBlock2 + return false + } + + // now we attempt to grab either of the slots and indicate success if so + if atomic.CompareAndSwapInt64(entryBlock1, 0, int64(indexblockStartForWrite)) { + // i.e. we're are now attempting a write for this block start and are tracking this at entryBlock1 + return true + } + + if atomic.CompareAndSwapInt64(entryBlock2, 0, int64(indexblockStartForWrite)) { + // i.e. we're are now attempting a write for this block start and are tracking this at entryBlock2 + return true + } + + // i.e. both the slots are occupied. we can reach here in a couple of ways: + // (1) There are lots of writes for this series, and another go routine has taken the slots. This is + // expected, and we can indicate false. + return false + + // (2) There's a code bug, where we haven't called OnIndexFinalize() despite this method returning + // true. We can possibly check for this by seeing if the values in entryBlock1/2 are more than + // 2 * indexBlockSize away from the provided blockStart. Should I do that here? // TODO(prateek): <--- + + // NB(prateek): There's still a design flaw in case of out of order, delayed writes. + // Consider an index block size of 2h, and buffer past of 10m. + // Say a write comes in at 2.05p (wallclock) for 2.05p (timestamp in the write), we'd index the + // entry, and update the entry to have a nextWriteTimeNanos of 4p. Now imagine another write + // comes in at 2.06p (wallclock) for 1.57p (timestamp in the write). The current design would + // see 1.57p points to the 12p blockStart, which is before the nextWriteTimeNanos of 4p and + // decide that we have already indexed this entry for the earlier block. i.e. we'd drop the + // indexing write. + // This can be addresssed by doing somthing like keeping 4 atomics (instead of the current 3), to + // capture any possible blocks that could be written to and/or if such a write has been attempted. + // IMO the complexity of that isn't worth maintaining. Much simpler to explicitly call out that out + // out order writes for the same ids have this issue. Will bring up with other devs in the PR. } // ensure dbShardEntry satisfies the `index.OnIndexSeries` interface. var _ index.OnIndexSeries = &dbShardEntry{} -func (entry *dbShardEntry) OnIndexSuccess(nextWriteTime time.Time) { - atomic.StoreInt64(&entry.reverseIndex.nextWriteTimeNanos, nextWriteTime.UnixNano()) +func (entry *dbShardEntry) OnIndexSuccess(nextWriteTime xtime.UnixNano) { + atomic.StoreInt64(&entry.reverseIndex.nextWriteTimeNanos, int64(nextWriteTime)) } -func (entry *dbShardEntry) OnIndexFinalize() { +func (entry *dbShardEntry) OnIndexFinalize(blockStartNanos xtime.UnixNano) { // indicate the index has released held reference for provided write entry.decrementReaderWriterCount() + + var ( + entryBlock1 = &entry.reverseIndex.writeAttemptedBlockStart1Nanos + entryBlock2 = &entry.reverseIndex.writeAttemptedBlockStart1Nanos + ) + + if atomic.CompareAndSwapInt64(entryBlock1, int64(blockStartNanos), 0) { + // i.e. we were attempting a write for this block start at entryBlock1 and have cleared it. + return + } + + if atomic.CompareAndSwapInt64(entryBlock2, int64(blockStartNanos), 0) { + // i.e. we were attempting a write for this block start at entryBlock2 and have cleared it. + return + } + + // TODO(prateek): capture some kind of log for this failure mode. + // should never get here, on index finalize should only be called for a blockstart we're + // tracking. } func (entry *dbShardEntry) onIndexPrepare() { @@ -836,7 +935,7 @@ func (s *dbShard) writeAndIndex( commitLogSeriesID = entry.series.ID() commitLogSeriesTags = entry.series.Tags() commitLogSeriesUniqueIndex = entry.index - needsIndex := shouldReverseIndex && entry.needsIndexUpdate(timestamp) + needsIndex := shouldReverseIndex && entry.needsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) if err == nil && needsIndex { err = s.insertSeriesForIndexing(entry, timestamp, opts.writeNewSeriesAsync) } @@ -1032,8 +1131,8 @@ func (s *dbShard) insertSeriesForIndexing( timestamp time.Time, async bool, ) error { - // // inc a ref on the entry to ensure it's valid until the queue acts upon it. - // entry.onIndexPrepare() + // inc a ref on the entry to ensure it's valid until the queue acts upon it. + entry.onIndexPrepare() wg, err := s.insertQueue.Insert(dbShardInsert{ entry: entry, opts: dbShardInsertAsyncOptions{ @@ -1041,12 +1140,16 @@ func (s *dbShard) insertSeriesForIndexing( pendingIndex: dbShardPendingIndex{ timestamp: timestamp, }, + // indicate we already have inc'd the entry's ref count, so we can correctly + // handle the ref counting semantics in `insertSeriesBatch`. + entryRefCountIncremented: true, }, }) // i.e. unable to enqueue into shard insert queue if err != nil { - // entry.OnIndexFinalize() // release any reference's we've held for indexing + indexBlockStart := s.reverseIndex.BlockStartForWriteTime(timestamp) + entry.OnIndexFinalize(indexBlockStart) // release any reference's we've held for indexing return err } @@ -1057,7 +1160,7 @@ func (s *dbShard) insertSeriesForIndexing( // if indexing in sync mode, wait till we're done and ensure we have have indexed the entry wg.Wait() - if entry.needsIndexUpdate(timestamp) { + if entry.hasIndexTTLGreaterThan(timestamp) { // i.e. indexing failed return fmt.Errorf("internal error: unable to index series") } @@ -1162,20 +1265,16 @@ func (s *dbShard) insertNewShardEntryWithLock(entry *dbShardEntry) { } func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { - anyPendingAction := false - numPendingIndexing := 0 + var ( + anyPendingAction = false + numPendingIndexing = 0 + ) s.Lock() for i := range inserts { - entry, _, err := s.lookupEntryWithLock(inserts[i].entry.series.ID()) - if entry != nil { - // Already exists so update the entry we're pointed at for this insert - inserts[i].entry = entry - } - // If we are going to write to this entry then increment the // writer count so it does not look empty immediately after - // we release the write lock + // we release the write lock. hasPendingWrite := inserts[i].opts.hasPendingWrite hasPendingIndexing := inserts[i].opts.hasPendingIndexing hasPendingRetrievedBlock := inserts[i].opts.hasPendingRetrievedBlock @@ -1186,11 +1285,30 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { numPendingIndexing++ } + // we don't need to inc the entry ref count if we already have a ref on the entry. check if + // that's the case. + if inserts[i].opts.entryRefCountIncremented { + // don't need to inc a ref on the entry, we were given as writable entry as input. + continue + } + + // i.e. we don't have a ref on provided entry, so we check if between the operation being + // enqueue in the shard insert queue, and this function executing, an entry was created + // for the same ID. + entry, _, err := s.lookupEntryWithLock(inserts[i].entry.series.ID()) + if entry != nil { + // Already exists so update the entry we're pointed at for this insert + inserts[i].entry = entry + } + if hasPendingIndexing || hasPendingWrite || hasPendingRetrievedBlock { // We're definitely writing a value, ensure that the pending write is // visible before we release the lookup write lock inserts[i].entry.incrementReaderWriterCount() + // also indicate that we have a ref count on this entry for this operation + inserts[i].opts.entryRefCountIncremented = true } + if err == nil { // Already inserted continue @@ -1199,6 +1317,9 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { if err != errShardEntryNotFound { // Shard is not taking inserts s.Unlock() + // FOLLOWUP(prateek): is this an existing bug? why don't we need to release any ref's we've inc'd + // on entries in the loop before this point, i.e. in range [0, i). Otherwise, how are those entries + // going to get cleaned up? s.metrics.insertAsyncInsertErrors.Inc(int64(len(inserts) - i)) return err } @@ -1226,11 +1347,10 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { for i := range inserts { var ( entry = inserts[i].entry - releaseEntryRef = false + releaseEntryRef = inserts[i].opts.entryRefCountIncremented ) if inserts[i].opts.hasPendingWrite { - releaseEntryRef = true write := inserts[i].opts.pendingWrite err := entry.series.Write(ctx, write.timestamp, write.value, write.unit, write.annotation) @@ -1241,21 +1361,18 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { if inserts[i].opts.hasPendingIndexing { pendingIndex := inserts[i].opts.pendingIndex - releaseEntryRef = true - // only index any entry that hasn't crossed the nextIndexTime - if entry.needsIndexUpdate(pendingIndex.timestamp) { - entry.onIndexPrepare() - indexBatch = append(indexBatch, index.WriteBatchEntry{ - ID: entry.series.ID(), - Tags: entry.series.Tags(), - Timestamp: pendingIndex.timestamp, - OnIndexSeries: entry, - }) - } + // increment the ref on the entry, as the original one was transferred to the + // this method (insertSeriesBatch) via `entryRefCountIncremented` mechanism. + entry.onIndexPrepare() + indexBatch = append(indexBatch, index.WriteBatchEntry{ + ID: entry.series.ID(), + Tags: entry.series.Tags(), + Timestamp: pendingIndex.timestamp, + OnIndexSeries: entry, + }) } if inserts[i].opts.hasPendingRetrievedBlock { - releaseEntryRef = true block := inserts[i].opts.pendingRetrievedBlock entry.series.OnRetrieveBlock(block.id, block.tags, block.start, block.segment) } diff --git a/storage/shard_insert_queue.go b/storage/shard_insert_queue.go index 432df3462a..467a3b4a55 100644 --- a/storage/shard_insert_queue.go +++ b/storage/shard_insert_queue.go @@ -101,8 +101,15 @@ type dbShardInsertAsyncOptions struct { pendingIndex dbShardPendingIndex hasPendingWrite bool - hasPendingIndexing bool hasPendingRetrievedBlock bool + hasPendingIndexing bool + + // NB(prateek): `entryRefCountIncremented` indicates if the + // entry provided along with the dbShardInsertAsyncOptions + // already has it's ref count incremented. It's used to + // correctly manage the lifecycle of the entry across the + // shard -> shard Queue -> shard boundaries. + entryRefCountIncremented bool } type dbShardInsert struct { diff --git a/storage/types.go b/storage/types.go index 245f8356b9..27b1404bfd 100644 --- a/storage/types.go +++ b/storage/types.go @@ -428,6 +428,12 @@ type databaseShard interface { // namespaceIndex indexes namespace writes. type namespaceIndex interface { + // BlockStartForWriteTime returns the index block start + // time for the given writeTime. + BlockStartForWriteTime( + writeTime time.Time, + ) xtime.UnixNano + // WriteBatch indexes the provided entries. WriteBatch( entries []index.WriteBatchEntry, From 493098b640da202e7e4861a56b693253718d86cf Mon Sep 17 00:00:00 2001 From: Prateek Rungta Date: Sat, 12 May 2018 16:16:47 -0400 Subject: [PATCH 05/33] Re-work dbShardEntry --- storage/index/index_mock.go | 11 +- storage/index/types.go | 6 +- storage/series/lookup/entry.go | 239 +++++++++++++++ storage/series/lookup/entry_blackbox_test.go | 112 +++++++ storage/series/lookup/entry_whitebox_test.go | 55 ++++ storage/shard.go | 307 +++++-------------- storage/shard_insert_queue.go | 3 +- storage/storage_mock.go | 12 + 8 files changed, 510 insertions(+), 235 deletions(-) create mode 100644 storage/series/lookup/entry.go create mode 100644 storage/series/lookup/entry_blackbox_test.go create mode 100644 storage/series/lookup/entry_whitebox_test.go diff --git a/storage/index/index_mock.go b/storage/index/index_mock.go index 8c5037b17c..9d153f1acc 100644 --- a/storage/index/index_mock.go +++ b/storage/index/index_mock.go @@ -32,6 +32,7 @@ import ( "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" + time0 "github.com/m3db/m3x/time" "github.com/golang/mock/gomock" ) @@ -287,17 +288,17 @@ func (m *MockOnIndexSeries) EXPECT() *MockOnIndexSeriesMockRecorder { } // OnIndexFinalize mocks base method -func (m *MockOnIndexSeries) OnIndexFinalize() { - m.ctrl.Call(m, "OnIndexFinalize") +func (m *MockOnIndexSeries) OnIndexFinalize(arg0 time0.UnixNano) { + m.ctrl.Call(m, "OnIndexFinalize", arg0) } // OnIndexFinalize indicates an expected call of OnIndexFinalize -func (mr *MockOnIndexSeriesMockRecorder) OnIndexFinalize() *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexFinalize", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexFinalize)) +func (mr *MockOnIndexSeriesMockRecorder) OnIndexFinalize(arg0 interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexFinalize", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexFinalize), arg0) } // OnIndexSuccess mocks base method -func (m *MockOnIndexSeries) OnIndexSuccess(arg0 time.Time) { +func (m *MockOnIndexSeries) OnIndexSuccess(arg0 time0.UnixNano) { m.ctrl.Call(m, "OnIndexSuccess", arg0) } diff --git a/storage/index/types.go b/storage/index/types.go index bbbcf43d56..da4e06b859 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -117,9 +117,9 @@ type MutableSegmentAllocator func() (segment.MutableSegment, error) // to do lifecycle management of any resources retained during indexing. type OnIndexSeries interface { // OnIndexSuccess is executed when an entry is successfully indexed. The - // provided value for `indexEntryExpiry` describes the TTL for the indexed - // entry. - OnIndexSuccess(indexEntryExpiry xtime.UnixNano) + // provided value for `blockStart` is the blockStart for which the write + // was indexed. + OnIndexSuccess(blockStart xtime.UnixNano) // OnIndexFinalize is executed when the index no longer holds any references // to the provided resources. It can be used to cleanup any resources held diff --git a/storage/series/lookup/entry.go b/storage/series/lookup/entry.go new file mode 100644 index 0000000000..106ec72ccd --- /dev/null +++ b/storage/series/lookup/entry.go @@ -0,0 +1,239 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package lookup + +import ( + "sync" + "sync/atomic" + + "github.com/m3db/m3db/storage/index" + "github.com/m3db/m3db/storage/series" + xtime "github.com/m3db/m3x/time" +) + +const ( + maxUint64 = ^uint64(0) + maxInt64 = int64(maxUint64 >> 1) +) + +// Entry is the entry in the shard ident.ID -> series map. It has additional +// members to track lifecycle and minimize indexing overhead. +// NB: users are expected to use `NewEntry` to construct these objects. +type Entry struct { + Series series.DatabaseSeries + Index uint64 + curReadWriters int32 + reverseIndex entryIndexState +} + +// ensure Entry satisfies the `index.OnIndexSeries` interface. +var _ index.OnIndexSeries = &Entry{} + +// NewEntry returns a new Entry. +func NewEntry(series series.DatabaseSeries, index uint64) *Entry { + entry := &Entry{ + Series: series, + Index: index, + } + entry.reverseIndex.states = entry.reverseIndex._staticAloc[:0] + return entry +} + +// ReaderWriterCount returns the current ref count on the Entry. +func (entry *Entry) ReaderWriterCount() int32 { + return atomic.LoadInt32(&entry.curReadWriters) +} + +// IncrementReaderWriterCount increments the ref count on the Entry. +func (entry *Entry) IncrementReaderWriterCount() { + atomic.AddInt32(&entry.curReadWriters, 1) +} + +// DecrementReaderWriterCount decrements the ref count on the Entry. +func (entry *Entry) DecrementReaderWriterCount() { + atomic.AddInt32(&entry.curReadWriters, -1) +} + +// IndexedForBlockStart returns a bool to indicate if the Entry has been successfully +// indexed for the given index blockstart. +func (entry *Entry) IndexedForBlockStart(indexBlockStart xtime.UnixNano) bool { + entry.reverseIndex.RLock() + isIndexed := entry.reverseIndex.indexedWithRLock(indexBlockStart) + entry.reverseIndex.RUnlock() + return isIndexed +} + +// NeedsIndexUpdate returns a bool to indicate if the Entry requires to be indexed +// for the provided blockStart. It only allows a single index attempt at a time +// for a single entry. +// NB(prateek): NeedsIndexUpdate is a CAS, i.e. when this method returns true, it +// also sets state on the entry to indicate that a write for the given blockStart +// is going to be sent to the index, and other go routines should not attempt the +// same write. Callers are expected to ensure they follow this guideline. +// Further, every call to NeedsIndexUpdate which returns true needs to have a corresponding +// OnIndexFinalze() call. This is reqiured for correct lifecycle maintenance. +func (entry *Entry) NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) bool { + // first we try the low-cost path: acquire a RLock and see if the given block start + // has been marked successfull or that we've attempted it. + entry.reverseIndex.RLock() + alreadyIndexedOrAttempted := entry.reverseIndex.indexedOrAttemptedWithRLock(indexBlockStartForWrite) + entry.reverseIndex.RUnlock() + if alreadyIndexedOrAttempted { + // if so, the entry does not need to be indexed. + return false + } + + // now acquire a write lock and set that we're going to attempt to do this so we don't try + // multiple times. + entry.reverseIndex.Lock() + // NB(prateek): not defer-ing here, need to avoid the the extra ~150ns to minimize contention. + + // but first, we have to ensure no one has done so since we released the read lock + alreadyIndexedOrAttempted = entry.reverseIndex.indexedOrAttemptedWithRLock(indexBlockStartForWrite) + if alreadyIndexedOrAttempted { + entry.reverseIndex.Unlock() + return false + } + + entry.reverseIndex.setAttemptWithWLock(indexBlockStartForWrite, true) + entry.reverseIndex.Unlock() + return true +} + +// OnIndexPrepare prepares the Entry to be handed off to the indexing sub-system. +// NB(prateek): we retain the ref count on the entry while the indexing is pending, +// the callback executed on the entry once the indexing is completed releases this +// reference. +func (entry *Entry) OnIndexPrepare() { + entry.IncrementReaderWriterCount() +} + +// OnIndexSuccess marks the given block start as successfully indexed. +func (entry *Entry) OnIndexSuccess(blockStartNanos xtime.UnixNano) { + entry.reverseIndex.Lock() + entry.reverseIndex.setSuccessWithWLock(blockStartNanos) + entry.reverseIndex.Unlock() +} + +// OnIndexFinalize marks any attempt for the given block start is finished. +func (entry *Entry) OnIndexFinalize(blockStartNanos xtime.UnixNano) { + entry.reverseIndex.Lock() + entry.reverseIndex.setAttemptWithWLock(blockStartNanos, false) + entry.reverseIndex.Unlock() + // indicate the index has released held reference for provided write + entry.DecrementReaderWriterCount() +} + +// entryIndexState is used to capture the state of indexing for a single shard +// entry. It's used to prevent redundant indexing operations. +// NB(prateek): We need this amount of state because in the worst case, as we can have 3 active blocks being +// written to. Albeit that's an edge case due to bad configuration. Even outside of that, 2 blocks can +// be written to due to delayed, out of order writes. Consider an index block size of 2h, and buffer +// past of 10m. Say a write comes in at 2.05p (wallclock) for 2.05p (timestamp in the write), we'd index +// the entry, and update the entry to have a success for 4p. Now imagine another write +// comes in at 2.06p (wallclock) for 1.57p (timestamp in the write). We need to differentiate that we don't +// have a write for the 12-2p block from the 2-4p block, or we'd drop the late write. +type entryIndexState struct { + sync.RWMutex + states []entryIndexBlockState + + // NB(prateek): we alloc an array (not slice) of size 3, as that is + // the most we will need (only 3 blocks should ever be written to + // simultaneously in the worst case). We allocate it like we're doing + // to ensure it's along side the rest of the struct in memory. But + // we only access it through `states`, to ensure that it can be + // grown/shrunk as needed. Do not acccess it directly. + _staticAloc [3]entryIndexBlockState +} + +// entryIndexBlockState is used to capture the state of indexing for a single shard +// entry for a given index block start. It's used to prevent attempts at double indexing +// for the same block start. +type entryIndexBlockState struct { + blockStart xtime.UnixNano + attempt bool + success bool +} + +func (s *entryIndexState) indexedWithRLock(t xtime.UnixNano) bool { + for i := range s.states { + if s.states[i].blockStart.Equal(t) { + return s.states[i].success + } + } + return false +} + +func (s *entryIndexState) indexedOrAttemptedWithRLock(t xtime.UnixNano) bool { + for i := range s.states { + if s.states[i].blockStart.Equal(t) { + return s.states[i].success || s.states[i].attempt + } + } + return false +} + +func (s *entryIndexState) setSuccessWithWLock(t xtime.UnixNano) { + for i := range s.states { + if s.states[i].blockStart.Equal(t) { + s.states[i].success = true + break + } + } +} + +func (s *entryIndexState) setAttemptWithWLock(t xtime.UnixNano, attempt bool) { + // first check if we have the block start in the slice already + for i := range s.states { + if s.states[i].blockStart.Equal(t) { + s.states[i].attempt = attempt + return + } + } + + // i.e. we don't have the block start in the slice + // if we have less than 3 elements, we can just insert an element to the slice. + if len(s.states) < 3 { + s.states = append(s.states, entryIndexBlockState{ + blockStart: t, + attempt: attempt, + }) + return + } + + // i.e. len(s.states) == 3, in this case, we update the entry with the lowest block start + // as we know only 3 writes can be active at any point. Think of this as a lazy compaction. + var ( + minIdx = -1 + minBlockStart = xtime.UnixNano(maxInt64) + ) + for idx, blockState := range s.states { + if blockState.blockStart < minBlockStart { + minIdx = idx + minBlockStart = blockState.blockStart + } + } + + s.states[minIdx] = entryIndexBlockState{ + blockStart: t, + attempt: attempt, + } +} diff --git a/storage/series/lookup/entry_blackbox_test.go b/storage/series/lookup/entry_blackbox_test.go new file mode 100644 index 0000000000..728e8eeff2 --- /dev/null +++ b/storage/series/lookup/entry_blackbox_test.go @@ -0,0 +1,112 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package lookup_test + +import ( + "sync" + "testing" + "time" + + "github.com/fortytw2/leaktest" + "github.com/m3db/m3db/storage/series/lookup" + xtime "github.com/m3db/m3x/time" + + "github.com/stretchr/testify/require" +) + +var ( + initTime = time.Date(2018, time.May, 12, 15, 55, 0, 0, time.UTC) + testBlockSize = 24 * time.Hour +) + +func newTime(n int) xtime.UnixNano { + t := initTime.Truncate(testBlockSize).Add(time.Duration(n) * testBlockSize) + return xtime.ToUnixNano(t) +} + +func TestEntryReaderWriterCount(t *testing.T) { + e := lookup.NewEntry(nil, 0) + require.Equal(t, int32(0), e.ReaderWriterCount()) + + e.IncrementReaderWriterCount() + require.Equal(t, int32(1), e.ReaderWriterCount()) + + e.DecrementReaderWriterCount() + require.Equal(t, int32(0), e.ReaderWriterCount()) +} + +func TestEntryIndexSuccessPath(t *testing.T) { + e := lookup.NewEntry(nil, 0) + t0 := newTime(0) + require.False(t, e.IndexedForBlockStart(t0)) + + require.True(t, e.NeedsIndexUpdate(t0)) + e.OnIndexPrepare() + e.OnIndexSuccess(t0) + e.OnIndexFinalize(t0) + + require.True(t, e.IndexedForBlockStart(t0)) + require.Equal(t, int32(0), e.ReaderWriterCount()) + require.False(t, e.NeedsIndexUpdate(t0)) +} + +func TestEntryIndexFailPath(t *testing.T) { + e := lookup.NewEntry(nil, 0) + t0 := newTime(0) + require.False(t, e.IndexedForBlockStart(t0)) + + require.True(t, e.NeedsIndexUpdate(t0)) + e.OnIndexPrepare() + e.OnIndexFinalize(t0) + + require.False(t, e.IndexedForBlockStart(t0)) + require.Equal(t, int32(0), e.ReaderWriterCount()) + require.True(t, e.NeedsIndexUpdate(t0)) +} + +func TestEntryMultipleGoroutinesRaceIndexUpdate(t *testing.T) { + defer leaktest.CheckTimeout(t, time.Second)() + + e := lookup.NewEntry(nil, 0) + t0 := newTime(0) + require.False(t, e.IndexedForBlockStart(t0)) + + var ( + r1, r2 bool + wg sync.WaitGroup + ) + wg.Add(2) + + go func() { + r1 = e.NeedsIndexUpdate(t0) + wg.Done() + }() + + go func() { + r2 = e.NeedsIndexUpdate(t0) + wg.Done() + }() + + wg.Wait() + + require.False(t, r1 && r2) + require.True(t, r1 || r2) +} diff --git a/storage/series/lookup/entry_whitebox_test.go b/storage/series/lookup/entry_whitebox_test.go new file mode 100644 index 0000000000..ff9431736c --- /dev/null +++ b/storage/series/lookup/entry_whitebox_test.go @@ -0,0 +1,55 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package lookup + +import ( + "testing" + "time" + + xtime "github.com/m3db/m3x/time" + "github.com/stretchr/testify/require" +) + +var ( + initTime = time.Date(2018, time.May, 12, 15, 55, 0, 0, time.UTC) + testBlockSize = 24 * time.Hour +) + +func newTime(n int) xtime.UnixNano { + t := initTime.Truncate(testBlockSize).Add(time.Duration(n) * testBlockSize) + return xtime.ToUnixNano(t) +} + +func TestEntryIndexAttemptRotatesSlice(t *testing.T) { + e := NewEntry(nil, 0) + require.Equal(t, 3, cap(e.reverseIndex.states)) + for i := 0; i < 10; i++ { + ti := newTime(i) + require.True(t, e.NeedsIndexUpdate(ti)) + require.Equal(t, 3, cap(e.reverseIndex.states)) + } + + // ensure only the latest ones are held on to + for i := 9; i >= 7; i-- { + ti := newTime(i) + require.False(t, e.NeedsIndexUpdate(ti)) + } +} diff --git a/storage/shard.go b/storage/shard.go index 42c831b6ca..5297d8c42a 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -28,7 +28,6 @@ import ( "math" "sort" "sync" - "sync/atomic" "time" "github.com/m3db/m3db/clock" @@ -44,6 +43,7 @@ import ( "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3db/storage/repair" "github.com/m3db/m3db/storage/series" + "github.com/m3db/m3db/storage/series/lookup" "github.com/m3db/m3db/ts" "github.com/m3db/m3db/x/xio" xclose "github.com/m3db/m3x/close" @@ -175,155 +175,9 @@ func newDatabaseShardMetrics(scope tally.Scope) dbShardMetrics { } } -type dbShardEntry struct { - series series.DatabaseSeries - index uint64 - curReadWriters int32 - reverseIndex struct { - // NB(prateek): writeAttemptedBlockstart[1-2]Nanos is used to indicate the index block start(s) - // for which this entry has a pending indexing operation (if any). The value for - // the blockstart is computed based upon timestamp of writes to the entry. When - // a write comes in for series, we compute this token and compare against the value - // set on the entry. If the entry has the same value, then we know there is already - // a write pending for the entry in the index for the given block start, and don't - // need to attempt another one. If not, we need to index it. - // The following semantics hold for this type: - // - `writeAttemptedBlockstartNanos == 0` indicates no write is pending for this entry. - // - `writeAttemptedBlockStartNanos != 0` indicates a write is pending for this entry. - // We actually require two blockStarts, because there can be writes issued for two blocks at the - // same time, and we need to track both of them. - writeAttemptedBlockStart1Nanos int64 - writeAttemptedBlockStart2Nanos int64 - // NB(prateek): nextWriteTimeNanos is the UnixNanos until - // the next index write is required. We use an atomic instead - // of a time.Time to avoid using a Mutex to guard it. - nextWriteTimeNanos int64 - } -} - -func (entry *dbShardEntry) readerWriterCount() int32 { - return atomic.LoadInt32(&entry.curReadWriters) -} - -func (entry *dbShardEntry) incrementReaderWriterCount() { - atomic.AddInt32(&entry.curReadWriters, 1) -} - -func (entry *dbShardEntry) decrementReaderWriterCount() { - atomic.AddInt32(&entry.curReadWriters, -1) -} - -func (entry *dbShardEntry) hasIndexTTLGreaterThan(writeTime time.Time) bool { - nextWriteNanos := atomic.LoadInt64(&entry.reverseIndex.nextWriteTimeNanos) - return nextWriteNanos > int64(xtime.ToUnixNano(writeTime)) -} - -// NB(prateek): needsIndexUpdate is a CAS, i.e. when this method returns true, it -// also sets state on the entry to indicate that a write for the given blockStart -// is going to be sent to the index, and other go routines should not attempt the -// same write. Callers are expected to ensure they follow this guideline. -// Further, every call to needsIndexUpdate which returns true needs to have a corresponding -// OnIndexFinalze() call. This is reqiured for correct lifecycle maintenance. -func (entry *dbShardEntry) needsIndexUpdate(indexblockStartForWrite xtime.UnixNano) bool { - // check if the entry has a TTL indicating it does not need a write till a - // time later than the provided windexblockStartForWrite. If so, we know the - // entry does not need to be indexed. More on this in a NB at the end of this function. - nextWriteNanos := atomic.LoadInt64(&entry.reverseIndex.nextWriteTimeNanos) - if nextWriteNanos > int64(indexblockStartForWrite) { - return false - } +type dbShardEntryWorkFn func(entry *lookup.Entry) bool - var ( - entryBlock1 = &entry.reverseIndex.writeAttemptedBlockStart1Nanos - entryBlock2 = &entry.reverseIndex.writeAttemptedBlockStart1Nanos - ) - - // i.e. no TTL has been marked on the entry yet. check if a write has been attempted for the entry at all. - if atomic.CompareAndSwapInt64(entryBlock1, int64(indexblockStartForWrite), int64(indexblockStartForWrite)) { - // i.e. we're already attempting a write for this block start and are tracking this at entryBlock1 - return false - } - - if atomic.CompareAndSwapInt64(entryBlock2, int64(indexblockStartForWrite), int64(indexblockStartForWrite)) { - // i.e. we're already attempting a write for this block start and are tracking this at entryBlock2 - return false - } - - // now we attempt to grab either of the slots and indicate success if so - if atomic.CompareAndSwapInt64(entryBlock1, 0, int64(indexblockStartForWrite)) { - // i.e. we're are now attempting a write for this block start and are tracking this at entryBlock1 - return true - } - - if atomic.CompareAndSwapInt64(entryBlock2, 0, int64(indexblockStartForWrite)) { - // i.e. we're are now attempting a write for this block start and are tracking this at entryBlock2 - return true - } - - // i.e. both the slots are occupied. we can reach here in a couple of ways: - // (1) There are lots of writes for this series, and another go routine has taken the slots. This is - // expected, and we can indicate false. - return false - - // (2) There's a code bug, where we haven't called OnIndexFinalize() despite this method returning - // true. We can possibly check for this by seeing if the values in entryBlock1/2 are more than - // 2 * indexBlockSize away from the provided blockStart. Should I do that here? // TODO(prateek): <--- - - // NB(prateek): There's still a design flaw in case of out of order, delayed writes. - // Consider an index block size of 2h, and buffer past of 10m. - // Say a write comes in at 2.05p (wallclock) for 2.05p (timestamp in the write), we'd index the - // entry, and update the entry to have a nextWriteTimeNanos of 4p. Now imagine another write - // comes in at 2.06p (wallclock) for 1.57p (timestamp in the write). The current design would - // see 1.57p points to the 12p blockStart, which is before the nextWriteTimeNanos of 4p and - // decide that we have already indexed this entry for the earlier block. i.e. we'd drop the - // indexing write. - // This can be addresssed by doing somthing like keeping 4 atomics (instead of the current 3), to - // capture any possible blocks that could be written to and/or if such a write has been attempted. - // IMO the complexity of that isn't worth maintaining. Much simpler to explicitly call out that out - // out order writes for the same ids have this issue. Will bring up with other devs in the PR. -} - -// ensure dbShardEntry satisfies the `index.OnIndexSeries` interface. -var _ index.OnIndexSeries = &dbShardEntry{} - -func (entry *dbShardEntry) OnIndexSuccess(nextWriteTime xtime.UnixNano) { - atomic.StoreInt64(&entry.reverseIndex.nextWriteTimeNanos, int64(nextWriteTime)) -} - -func (entry *dbShardEntry) OnIndexFinalize(blockStartNanos xtime.UnixNano) { - // indicate the index has released held reference for provided write - entry.decrementReaderWriterCount() - - var ( - entryBlock1 = &entry.reverseIndex.writeAttemptedBlockStart1Nanos - entryBlock2 = &entry.reverseIndex.writeAttemptedBlockStart1Nanos - ) - - if atomic.CompareAndSwapInt64(entryBlock1, int64(blockStartNanos), 0) { - // i.e. we were attempting a write for this block start at entryBlock1 and have cleared it. - return - } - - if atomic.CompareAndSwapInt64(entryBlock2, int64(blockStartNanos), 0) { - // i.e. we were attempting a write for this block start at entryBlock2 and have cleared it. - return - } - - // TODO(prateek): capture some kind of log for this failure mode. - // should never get here, on index finalize should only be called for a blockstart we're - // tracking. -} - -func (entry *dbShardEntry) onIndexPrepare() { - // NB(prateek): we retain the ref count on the entry while the indexing is pending, - // the callback executed on the entry once the indexing is completed releases this - // reference. - entry.incrementReaderWriterCount() -} - -type dbShardEntryWorkFn func(entry *dbShardEntry) bool - -type dbShardEntryBatchWorkFn func(entries []*dbShardEntry) bool +type dbShardEntryBatchWorkFn func(entries []*lookup.Entry) bool type shardListElement *list.Element @@ -474,8 +328,8 @@ func (s *dbShard) OnRetrieveBlock( s.RLock() entry, _, err := s.lookupEntryWithLock(id) if entry != nil { - entry.incrementReaderWriterCount() - defer entry.decrementReaderWriterCount() + entry.IncrementReaderWriterCount() + defer entry.DecrementReaderWriterCount() } s.RUnlock() @@ -484,7 +338,7 @@ func (s *dbShard) OnRetrieveBlock( } if entry != nil { - entry.series.OnRetrieveBlock(id, tags, startTime, segment) + entry.Series.OnRetrieveBlock(id, tags, startTime, segment) return } @@ -500,9 +354,9 @@ func (s *dbShard) OnRetrieveBlock( // NB(r): Do not need to specify that needs to be indexed as series would // have been already been indexed when it was written - copiedID := entry.series.ID() + copiedID := entry.Series.ID() // TODO(r): Pool the slice iterators here. - copiedTags := ident.NewTagSliceIterator(entry.series.Tags()) + copiedTags := ident.NewTagSliceIterator(entry.Series.Tags()) s.insertQueue.Insert(dbShardInsert{ entry: entry, opts: dbShardInsertAsyncOptions{ @@ -540,11 +394,11 @@ func (s *dbShard) OnEvictedFromWiredList(id ident.ID, blockStart time.Time) { return } - entry.series.OnEvictedFromWiredList(id, blockStart) + entry.Series.OnEvictedFromWiredList(id, blockStart) } func (s *dbShard) forEachShardEntry(entryFn dbShardEntryWorkFn) error { - return s.forEachShardEntryBatch(func(currEntries []*dbShardEntry) bool { + return s.forEachShardEntryBatch(func(currEntries []*lookup.Entry) bool { for _, entry := range currEntries { if continueForEach := entryFn(entry); !continueForEach { return false @@ -573,11 +427,11 @@ func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) if e == nil { return } - e.Value.(*dbShardEntry).decrementReaderWriterCount() + e.Value.(*lookup.Entry).DecrementReaderWriterCount() } var ( - currEntries = make([]*dbShardEntry, 0, batchSize) + currEntries = make([]*lookup.Entry, 0, batchSize) first = true nextElem *list.Element ) @@ -597,8 +451,8 @@ func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) elem := nextElem for ticked := 0; ticked < batchSize && elem != nil; ticked++ { nextElem = elem.Next() - entry := elem.Value.(*dbShardEntry) - entry.incrementReaderWriterCount() + entry := elem.Value.(*lookup.Entry) + entry.IncrementReaderWriterCount() currEntries = append(currEntries, entry) elem = nextElem } @@ -606,13 +460,13 @@ func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) // NB(prateek): inc a reference to the next element while we have a lock, // to guarantee the element pointer cannot be changed from under us. if nextElem != nil { - nextElem.Value.(*dbShardEntry).incrementReaderWriterCount() + nextElem.Value.(*lookup.Entry).IncrementReaderWriterCount() } s.RUnlock() continueExecution := entriesBatchFn(currEntries) for i := range currEntries { - currEntries[i].decrementReaderWriterCount() + currEntries[i].DecrementReaderWriterCount() currEntries[i] = nil } currEntries = currEntries[:0] @@ -724,13 +578,13 @@ func (s *dbShard) tickAndExpire( terminatedTickingDueToClosing bool i int slept time.Duration - expired []*dbShardEntry + expired []*lookup.Entry ) s.RLock() tickSleepBatch := s.currRuntimeOptions.tickSleepSeriesBatchSize tickSleepPerSeries := s.currRuntimeOptions.tickSleepPerSeries s.RUnlock() - s.forEachShardEntryBatch(func(currEntries []*dbShardEntry) bool { + s.forEachShardEntryBatch(func(currEntries []*lookup.Entry) bool { // re-using `expired` to amortize allocs, still need to reset it // to be safe for re-use. for i := range expired { @@ -764,7 +618,7 @@ func (s *dbShard) tickAndExpire( ) switch policy { case tickPolicyRegular: - result, err = entry.series.Tick() + result, err = entry.Series.Tick() case tickPolicyCloseShard: err = series.ErrSeriesAllDatapointsExpired } @@ -811,24 +665,24 @@ func (s *dbShard) tickAndExpire( // Currently, this function is only called by the lambda inside `tickAndExpire`'s `forEachShardEntryBatch` // call. This satisfies the contract of all entries it operating upon being guaranteed to have a // readerWriterEntryCount of at least 1, by virtue of the implementation of `forEachShardEntryBatch`. -func (s *dbShard) purgeExpiredSeries(expiredEntries []*dbShardEntry) { +func (s *dbShard) purgeExpiredSeries(expiredEntries []*lookup.Entry) { // Remove all expired series from lookup and list. s.Lock() for _, entry := range expiredEntries { - series := entry.series + series := entry.Series id := series.ID() elem, exists := s.lookup.Get(id) if !exists { continue } - count := entry.readerWriterCount() + count := entry.ReaderWriterCount() // The contract requires all entries to have count >= 1. if count < 1 { s.logger.WithFields( xlog.NewField("series", series.ID().String()), - xlog.NewField("readerWriterCount", count), - ).Errorf("observed series with invalid readerWriterCount in `purgeExpiredSeries`") + xlog.NewField("ReaderWriterCount", count), + ).Errorf("observed series with invalid ReaderWriterCount in `purgeExpiredSeries`") continue } // If this series is currently being written to or read from, we don't @@ -925,22 +779,23 @@ func (s *dbShard) writeAndIndex( ) if writable { // Perform write - err = entry.series.Write(ctx, timestamp, value, unit, annotation) + err = entry.Series.Write(ctx, timestamp, value, unit, annotation) // Load series metadata before decrementing the writer count // to ensure this metadata is snapshotted at a consistent state // NB(r): We explicitly do not place the series ID back into a // pool as high frequency users of series IDs such // as the commit log need to use the reference without the // overhead of ownership tracking. This makes taking a ref here safe. - commitLogSeriesID = entry.series.ID() - commitLogSeriesTags = entry.series.Tags() - commitLogSeriesUniqueIndex = entry.index - needsIndex := shouldReverseIndex && entry.needsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) - if err == nil && needsIndex { - err = s.insertSeriesForIndexing(entry, timestamp, opts.writeNewSeriesAsync) + commitLogSeriesID = entry.Series.ID() + commitLogSeriesTags = entry.Series.Tags() + commitLogSeriesUniqueIndex = entry.Index + if err == nil && shouldReverseIndex { + if entry.NeedsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) { + err = s.insertSeriesForIndexing(entry, timestamp, opts.writeNewSeriesAsync) + } } // release the reference we got on entry from `writableSeries` - entry.decrementReaderWriterCount() + entry.DecrementReaderWriterCount() if err != nil { return err } @@ -969,7 +824,7 @@ func (s *dbShard) writeAndIndex( // (i.e. registering a dependency on the context) is too expensive. commitLogSeriesID = result.copiedID commitLogSeriesTags = result.copiedTags - commitLogSeriesUniqueIndex = result.entry.index + commitLogSeriesUniqueIndex = result.entry.Index } // Write commit log @@ -1000,8 +855,8 @@ func (s *dbShard) ReadEncoded( if entry != nil { // NB(r): Ensure readers have consistent view of this series, do // not expire the series while being read from. - entry.incrementReaderWriterCount() - defer entry.decrementReaderWriterCount() + entry.IncrementReaderWriterCount() + defer entry.DecrementReaderWriterCount() } s.RUnlock() @@ -1019,7 +874,7 @@ func (s *dbShard) ReadEncoded( } if entry != nil { - return entry.series.ReadEncoded(ctx, start, end) + return entry.Series.ReadEncoded(ctx, start, end) } retriever := s.seriesBlockRetriever @@ -1030,7 +885,7 @@ func (s *dbShard) ReadEncoded( } // lookupEntryWithLock returns the entry for a given id while holding a read lock or a write lock. -func (s *dbShard) lookupEntryWithLock(id ident.ID) (*dbShardEntry, *list.Element, error) { +func (s *dbShard) lookupEntryWithLock(id ident.ID) (*lookup.Entry, *list.Element, error) { if s.state != dbShardStateOpen { // NB(r): Return an invalid params error here so any upstream // callers will not retry this operation @@ -1040,10 +895,10 @@ func (s *dbShard) lookupEntryWithLock(id ident.ID) (*dbShardEntry, *list.Element if !exists { return nil, nil, errShardEntryNotFound } - return elem.Value.(*dbShardEntry), elem, nil + return elem.Value.(*lookup.Entry), elem, nil } -func (s *dbShard) writableSeries(id ident.ID, tags ident.TagIterator) (*dbShardEntry, error) { +func (s *dbShard) writableSeries(id ident.ID, tags ident.TagIterator) (*lookup.Entry, error) { for { entry, _, err := s.tryRetrieveWritableSeries(id) if entry != nil { @@ -1069,7 +924,7 @@ type writableSeriesOptions struct { } func (s *dbShard) tryRetrieveWritableSeries(id ident.ID) ( - *dbShardEntry, + *lookup.Entry, writableSeriesOptions, error, ) { @@ -1078,7 +933,7 @@ func (s *dbShard) tryRetrieveWritableSeries(id ident.ID) ( writeNewSeriesAsync: s.currRuntimeOptions.writeNewSeriesAsync, } if entry, _, err := s.lookupEntryWithLock(id); err == nil { - entry.incrementReaderWriterCount() + entry.IncrementReaderWriterCount() s.RUnlock() return entry, opts, nil } else if err != errShardEntryNotFound { @@ -1089,7 +944,7 @@ func (s *dbShard) tryRetrieveWritableSeries(id ident.ID) ( return nil, opts, nil } -func (s *dbShard) newShardEntry(id ident.ID, tags ident.TagIterator) (*dbShardEntry, error) { +func (s *dbShard) newShardEntry(id ident.ID, tags ident.TagIterator) (*lookup.Entry, error) { clonedTags, err := s.cloneTags(tags) if err != nil { return nil, err @@ -1099,7 +954,7 @@ func (s *dbShard) newShardEntry(id ident.ID, tags ident.TagIterator) (*dbShardEn series.Reset(clonedID, clonedTags, s.seriesBlockRetriever, s.seriesOnRetrieveBlock, s, s.seriesOpts) uniqueIndex := s.increasingIndex.nextIndex() - return &dbShardEntry{series: series, index: uniqueIndex}, nil + return lookup.NewEntry(series, uniqueIndex), nil } func (s *dbShard) cloneTags(tags ident.TagIterator) (ident.Tags, error) { @@ -1123,16 +978,17 @@ type insertAsyncResult struct { // entry is not guaranteed to be the final entry // inserted into the shard map in case there is already // an existing entry waiting in the insert queue - entry *dbShardEntry + entry *lookup.Entry } func (s *dbShard) insertSeriesForIndexing( - entry *dbShardEntry, + entry *lookup.Entry, timestamp time.Time, async bool, ) error { + indexBlockStart := s.reverseIndex.BlockStartForWriteTime(timestamp) // inc a ref on the entry to ensure it's valid until the queue acts upon it. - entry.onIndexPrepare() + entry.OnIndexPrepare() wg, err := s.insertQueue.Insert(dbShardInsert{ entry: entry, opts: dbShardInsertAsyncOptions{ @@ -1148,7 +1004,6 @@ func (s *dbShard) insertSeriesForIndexing( // i.e. unable to enqueue into shard insert queue if err != nil { - indexBlockStart := s.reverseIndex.BlockStartForWriteTime(timestamp) entry.OnIndexFinalize(indexBlockStart) // release any reference's we've held for indexing return err } @@ -1160,7 +1015,7 @@ func (s *dbShard) insertSeriesForIndexing( // if indexing in sync mode, wait till we're done and ensure we have have indexed the entry wg.Wait() - if entry.hasIndexTTLGreaterThan(timestamp) { + if entry.IndexedForBlockStart(indexBlockStart) { // i.e. indexing failed return fmt.Errorf("internal error: unable to index series") } @@ -1185,8 +1040,8 @@ func (s *dbShard) insertSeriesAsyncBatched( return insertAsyncResult{ wg: wg, // Make sure to return the copied ID from the new series - copiedID: entry.series.ID(), - copiedTags: entry.series.Tags(), + copiedID: entry.Series.ID(), + copiedTags: entry.Series.Tags(), entry: entry, }, err } @@ -1203,9 +1058,9 @@ func (s *dbShard) insertSeriesSync( id ident.ID, tags ident.TagIterator, insertType insertSyncType, -) (*dbShardEntry, error) { +) (*lookup.Entry, error) { var ( - entry *dbShardEntry + entry *lookup.Entry err error ) @@ -1215,7 +1070,7 @@ func (s *dbShard) insertSeriesSync( // to increment the writer count so it's visible when we release // the lock if entry != nil && insertType == insertSyncIncReaderWriterCount { - entry.incrementReaderWriterCount() + entry.IncrementReaderWriterCount() } s.Unlock() }() @@ -1241,7 +1096,7 @@ func (s *dbShard) insertSeriesSync( } if s.newSeriesBootstrapped { - _, err := entry.series.Bootstrap(nil) + _, err := entry.Series.Bootstrap(nil) if err != nil { entry = nil // Don't increment the writer count for this series return nil, err @@ -1252,11 +1107,11 @@ func (s *dbShard) insertSeriesSync( return entry, nil } -func (s *dbShard) insertNewShardEntryWithLock(entry *dbShardEntry) { +func (s *dbShard) insertNewShardEntryWithLock(entry *lookup.Entry) { // Set the lookup value, we use the copied ID and since it is GC'd // we explicitly set it with options to not copy the key and not to // finalize it - copiedID := entry.series.ID() + copiedID := entry.Series.ID() listElem := s.list.PushBack(entry) s.lookup.SetUnsafe(copiedID, listElem, shardMapSetUnsafeOptions{ NoCopyKey: true, @@ -1295,7 +1150,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // i.e. we don't have a ref on provided entry, so we check if between the operation being // enqueue in the shard insert queue, and this function executing, an entry was created // for the same ID. - entry, _, err := s.lookupEntryWithLock(inserts[i].entry.series.ID()) + entry, _, err := s.lookupEntryWithLock(inserts[i].entry.Series.ID()) if entry != nil { // Already exists so update the entry we're pointed at for this insert inserts[i].entry = entry @@ -1304,7 +1159,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { if hasPendingIndexing || hasPendingWrite || hasPendingRetrievedBlock { // We're definitely writing a value, ensure that the pending write is // visible before we release the lookup write lock - inserts[i].entry.incrementReaderWriterCount() + inserts[i].entry.IncrementReaderWriterCount() // also indicate that we have a ref count on this entry for this operation inserts[i].opts.entryRefCountIncremented = true } @@ -1327,7 +1182,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // Insert still pending, perform the insert entry = inserts[i].entry if s.newSeriesBootstrapped { - _, err := entry.series.Bootstrap(nil) + _, err := entry.Series.Bootstrap(nil) if err != nil { s.metrics.insertAsyncBootstrapErrors.Inc(1) } @@ -1352,7 +1207,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { if inserts[i].opts.hasPendingWrite { write := inserts[i].opts.pendingWrite - err := entry.series.Write(ctx, write.timestamp, write.value, + err := entry.Series.Write(ctx, write.timestamp, write.value, write.unit, write.annotation) if err != nil { s.metrics.insertAsyncWriteErrors.Inc(1) @@ -1363,10 +1218,10 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { pendingIndex := inserts[i].opts.pendingIndex // increment the ref on the entry, as the original one was transferred to the // this method (insertSeriesBatch) via `entryRefCountIncremented` mechanism. - entry.onIndexPrepare() + entry.OnIndexPrepare() indexBatch = append(indexBatch, index.WriteBatchEntry{ - ID: entry.series.ID(), - Tags: entry.series.Tags(), + ID: entry.Series.ID(), + Tags: entry.Series.Tags(), Timestamp: pendingIndex.timestamp, OnIndexSeries: entry, }) @@ -1374,11 +1229,11 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { if inserts[i].opts.hasPendingRetrievedBlock { block := inserts[i].opts.pendingRetrievedBlock - entry.series.OnRetrieveBlock(block.id, block.tags, block.start, block.segment) + entry.Series.OnRetrieveBlock(block.id, block.tags, block.start, block.segment) } if releaseEntryRef { - entry.decrementReaderWriterCount() + entry.DecrementReaderWriterCount() } } @@ -1404,8 +1259,8 @@ func (s *dbShard) FetchBlocks( if entry != nil { // NB(r): Ensure readers have consistent view of this series, do // not expire the series while being read from. - entry.incrementReaderWriterCount() - defer entry.decrementReaderWriterCount() + entry.IncrementReaderWriterCount() + defer entry.DecrementReaderWriterCount() } s.RUnlock() @@ -1423,7 +1278,7 @@ func (s *dbShard) FetchBlocks( } if entry != nil { - return entry.series.FetchBlocks(ctx, starts) + return entry.Series.FetchBlocks(ctx, starts) } retriever := s.seriesBlockRetriever @@ -1450,23 +1305,23 @@ func (s *dbShard) fetchActiveBlocksMetadata( ) var loopErr error - s.forEachShardEntry(func(entry *dbShardEntry) bool { + s.forEachShardEntry(func(entry *lookup.Entry) bool { // Break out of the iteration loop once we've accumulated enough entries. if int64(len(res.Results())) >= limit { - next := int64(entry.index) + next := int64(entry.Index) nextIndexCursor = &next return false } // Fast forward past indexes lower than page token - if int64(entry.index) < indexCursor { + if int64(entry.Index) < indexCursor { return true } // Use a temporary context here so the stream readers can be returned to // pool after we finish fetching the metadata for this series. tmpCtx.Reset() - metadata, err := entry.series.FetchBlocksMetadata(tmpCtx, start, end, opts) + metadata, err := entry.Series.FetchBlocksMetadata(tmpCtx, start, end, opts) tmpCtx.BlockingClose() if err != nil { loopErr = err @@ -1795,14 +1650,14 @@ func (s *dbShard) Bootstrap( } } - bsResult, err := entry.series.Bootstrap(dbBlocks.Blocks) + bsResult, err := entry.Series.Bootstrap(dbBlocks.Blocks) if err != nil { multiErr = multiErr.Add(err) } shardBootstrapResult.update(bsResult) // Always decrement the writer count, avoid continue on bootstrap error - entry.decrementReaderWriterCount() + entry.DecrementReaderWriterCount() } s.emitBootstrapResult(shardBootstrapResult) @@ -1817,8 +1672,8 @@ func (s *dbShard) Bootstrap( // Find the series with no data within the retention period but has // buffered data points since server start. Any new series added // after this will be marked as bootstrapped. - s.forEachShardEntry(func(entry *dbShardEntry) bool { - series := entry.series + s.forEachShardEntry(func(entry *lookup.Entry) bool { + series := entry.Series if series.IsBootstrapped() { return true } @@ -1890,8 +1745,8 @@ func (s *dbShard) Flush( tmpCtx := context.NewContext() flushResult := dbShardFlushResult{} - s.forEachShardEntry(func(entry *dbShardEntry) bool { - curr := entry.series + s.forEachShardEntry(func(entry *lookup.Entry) bool { + curr := entry.Series // Use a temporary context here so the stream readers can be returned to // the pool after we finish fetching flushing the series. tmpCtx.Reset() @@ -1959,8 +1814,8 @@ func (s *dbShard) Snapshot( } tmpCtx := context.NewContext() - s.forEachShardEntry(func(entry *dbShardEntry) bool { - series := entry.series + s.forEachShardEntry(func(entry *lookup.Entry) bool { + series := entry.Series // Use a temporary context here so the stream readers can be returned to // pool after we finish fetching flushing the series tmpCtx.Reset() diff --git a/storage/shard_insert_queue.go b/storage/shard_insert_queue.go index 467a3b4a55..f2b2011d1d 100644 --- a/storage/shard_insert_queue.go +++ b/storage/shard_insert_queue.go @@ -27,6 +27,7 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/runtime" + "github.com/m3db/m3db/storage/series/lookup" "github.com/m3db/m3db/ts" "github.com/m3db/m3x/ident" xtime "github.com/m3db/m3x/time" @@ -113,7 +114,7 @@ type dbShardInsertAsyncOptions struct { } type dbShardInsert struct { - entry *dbShardEntry + entry *lookup.Entry opts dbShardInsertAsyncOptions } diff --git a/storage/storage_mock.go b/storage/storage_mock.go index 4d4b81cebf..233047cf04 100644 --- a/storage/storage_mock.go +++ b/storage/storage_mock.go @@ -1382,6 +1382,18 @@ func (m *MocknamespaceIndex) EXPECT() *MocknamespaceIndexMockRecorder { return m.recorder } +// BlockStartForWriteTime mocks base method +func (m *MocknamespaceIndex) BlockStartForWriteTime(writeTime time.Time) time0.UnixNano { + ret := m.ctrl.Call(m, "BlockStartForWriteTime", writeTime) + ret0, _ := ret[0].(time0.UnixNano) + return ret0 +} + +// BlockStartForWriteTime indicates an expected call of BlockStartForWriteTime +func (mr *MocknamespaceIndexMockRecorder) BlockStartForWriteTime(writeTime interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BlockStartForWriteTime", reflect.TypeOf((*MocknamespaceIndex)(nil).BlockStartForWriteTime), writeTime) +} + // WriteBatch mocks base method func (m *MocknamespaceIndex) WriteBatch(entries []index.WriteBatchEntry) error { ret := m.ctrl.Call(m, "WriteBatch", entries) From 1191750e515cffdb1dad948ab835863633f547ab Mon Sep 17 00:00:00 2001 From: Prateek Rungta Date: Sat, 12 May 2018 18:05:51 -0400 Subject: [PATCH 06/33] fix some tests and the startTime v endTime issue --- storage/index/block.go | 2 +- storage/index_block_test.go | 28 +++++---- storage/index_insert_queue_test.go | 82 ++++++++++--------------- storage/index_queue_test.go | 27 ++++---- storage/shard_foreachentry_prop_test.go | 13 ++-- storage/shard_index_test.go | 36 ++++++----- storage/shard_insert_queue_test.go | 1 + storage/shard_ref_count_test.go | 4 +- 8 files changed, 95 insertions(+), 98 deletions(-) diff --git a/storage/index/block.go b/storage/index/block.go index 2df4dce820..c56e51bb64 100644 --- a/storage/index/block.go +++ b/storage/index/block.go @@ -145,7 +145,7 @@ func (b *block) WriteBatch(inserts WriteBatchEntryByBlockStartAndID) (WriteBatch result.NumSuccess += int64(len(writesForID)) // mark the first ref success (can mark any ref success here, because they're backed by // by the same entry). Could also mark all of them success but it wouldn't buy us anything. - writesForID[0].OnIndexSeries.OnIndexSuccess(xtime.ToUnixNano(b.endTime)) + writesForID[0].OnIndexSeries.OnIndexSuccess(xtime.ToUnixNano(b.startTime)) // we do need to finalize all refs as each is an extra inc we need to dec WriteBatchEntriesFinalizer(writesForID).Finalize() } diff --git a/storage/index_block_test.go b/storage/index_block_test.go index 4bddb5ca4a..4fb20fff71 100644 --- a/storage/index_block_test.go +++ b/storage/index_block_test.go @@ -41,13 +41,13 @@ import ( "github.com/stretchr/testify/require" ) -func testIndexWriteEntry(id ident.ID, tags ident.Tags, timestamp time.Time, fns index.OnIndexSeries) []indexWriteEntry { - return []indexWriteEntry{ - indexWriteEntry{ - id: id, - tags: tags, - timestamp: timestamp, - fns: fns, +func testWriteBatchEntry(id ident.ID, tags ident.Tags, timestamp time.Time, fns index.OnIndexSeries) []index.WriteBatchEntry { + return []index.WriteBatchEntry{ + index.WriteBatchEntry{ + ID: id, + Tags: tags, + Timestamp: timestamp, + OnIndexSeries: fns, }, } } @@ -145,12 +145,13 @@ func TestNamespaceIndexWrite(t *testing.T) { lifecycle := index.NewMockOnIndexSeries(ctrl) mockBlock.EXPECT().WriteBatch([]index.WriteBatchEntry{ index.WriteBatchEntry{ - BlockStart: blockStart, - Document: testDoc1(), + Timestamp: blockStart.ToTime(), + ID: id, + Tags: tags, OnIndexSeries: lifecycle, }, }).Return(index.WriteBatchResult{}, nil) - require.NoError(t, idx.WriteBatch(testIndexWriteEntry(id, tags, now, lifecycle))) + require.NoError(t, idx.WriteBatch(testWriteBatchEntry(id, tags, now, lifecycle))) } func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { @@ -193,8 +194,9 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { lifecycle := index.NewMockOnIndexSeries(ctrl) b1.EXPECT().WriteBatch([]index.WriteBatchEntry{ index.WriteBatchEntry{ - BlockStart: t1Nanos, - Document: testDoc1(), + Timestamp: t1, + ID: id, + Tags: tags, OnIndexSeries: lifecycle, }, }).Return(index.WriteBatchResult{}, nil) @@ -203,7 +205,7 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { now = now.Add(blockSize) nowLock.Unlock() - require.NoError(t, idx.WriteBatch(testIndexWriteEntry(id, tags, now, lifecycle))) + require.NoError(t, idx.WriteBatch(testWriteBatchEntry(id, tags, now, lifecycle))) } func TestNamespaceIndexBootstrap(t *testing.T) { diff --git a/storage/index_insert_queue_test.go b/storage/index_insert_queue_test.go index f99f9868fb..f5f16fd107 100644 --- a/storage/index_insert_queue_test.go +++ b/storage/index_insert_queue_test.go @@ -21,14 +21,14 @@ package storage import ( + "fmt" "sync" "sync/atomic" "testing" "time" "github.com/m3db/m3db/storage/index" - "github.com/m3db/m3ninx/doc" - xtime "github.com/m3db/m3x/time" + "github.com/m3db/m3x/ident" "github.com/fortytw2/leaktest" "github.com/golang/mock/gomock" @@ -37,19 +37,9 @@ import ( "github.com/uber-go/tally" ) -func testWriteBatchEntry(ts time.Time, d doc.Document, fn index.OnIndexSeries) []index.WriteBatchEntry { - return []index.WriteBatchEntry{ - index.WriteBatchEntry{ - BlockStart: xtime.ToUnixNano(ts), - Document: d, - OnIndexSeries: fn, - }, - } -} - func newTestIndexInsertQueue() *nsIndexInsertQueue { var ( - nsIndexInsertBatchFn = func(inserts []index.WriteBatchEntry) {} + nsIndexInsertBatchFn = func(inserts [][]index.WriteBatchEntry) {} nowFn = time.Now scope = tally.NoopScope ) @@ -59,15 +49,11 @@ func newTestIndexInsertQueue() *nsIndexInsertQueue { return q } -func testDoc(i int) doc.Document { - return doc.Document{ - Fields: []doc.Field{ - doc.Field{ - Name: []byte("foo"), - Value: []byte("bar"), - }, - }, - } +func testID(i int) ident.ID { + return ident.StringID(fmt.Sprintf("foo%d", i)) +} +func testTags(i int) ident.Tags { + return ident.Tags{ident.Tag{testID(i), testID(i)}} } func TestIndexInsertQueueStopBeforeStart(t *testing.T) { @@ -95,31 +81,31 @@ func TestIndexInsertQueueCallback(t *testing.T) { defer ctrl.Finish() var ( - q = newTestIndexInsertQueue() - insertLock sync.Mutex - insertedDocs []index.WriteBatchEntry - callback = index.NewMockOnIndexSeries(ctrl) + q = newTestIndexInsertQueue() + insertLock sync.Mutex + insertedBatches [][]index.WriteBatchEntry + callback = index.NewMockOnIndexSeries(ctrl) ) - q.indexBatchFn = func(inserts []index.WriteBatchEntry) { + q.indexBatchFn = func(inserts [][]index.WriteBatchEntry) { insertLock.Lock() - insertedDocs = append(insertedDocs, inserts...) + insertedBatches = append(insertedBatches, inserts...) insertLock.Unlock() } - d := testDoc(1) assert.NoError(t, q.Start()) defer q.Stop() now := time.Now() - wg, err := q.InsertBatch(testWriteBatchEntry(now, d, callback)) + wg, err := q.InsertBatch(testWriteBatchEntry(testID(1), testTags(1), now, callback)) assert.NoError(t, err) wg.Wait() insertLock.Lock() defer insertLock.Unlock() - assert.Len(t, insertedDocs, 1) - assert.Equal(t, d, insertedDocs[0].Document) - assert.Equal(t, now.UnixNano(), int64(insertedDocs[0].BlockStart)) + assert.Len(t, insertedBatches, 1) + assert.Len(t, insertedBatches[0], 1) + // assert.Equal(t, d, insertedBatches[0][0].Document) + // assert.Equal(t, now.UnixNano(), int64(insertedBatches[0][0].BlockStart)) } func TestIndexInsertQueueRateLimit(t *testing.T) { @@ -150,37 +136,37 @@ func TestIndexInsertQueueRateLimit(t *testing.T) { assert.NoError(t, q.Stop()) }() - _, err := q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(1), callback)) + _, err := q.InsertBatch(testWriteBatchEntry(testID(1), testTags(1), time.Time{}, callback)) assert.NoError(t, err) addTime(250 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(2), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(2), testTags(2), time.Time{}, callback)) assert.NoError(t, err) // Consecutive should be all rate limited for i := 0; i < 100; i++ { - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(i+2), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(i+2), testTags(i+2), time.Time{}, callback)) assert.Error(t, err) assert.Equal(t, errNewSeriesIndexRateLimitExceeded, err) } // Start 2nd second should not be an issue addTime(750 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(110), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(110), testTags(100), time.Time{}, callback)) assert.NoError(t, err) addTime(100 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(111), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(111), testTags(111), time.Time{}, callback)) assert.NoError(t, err) addTime(100 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(112), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(112), testTags(112), time.Time{}, callback)) assert.Error(t, err) assert.Equal(t, errNewSeriesIndexRateLimitExceeded, err) // Start 3rd second addTime(800 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(113), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(113), testTags(113), time.Time{}, callback)) assert.NoError(t, err) q.Lock() @@ -194,7 +180,7 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() var ( - inserts [][]index.WriteBatchEntry + inserts [][][]index.WriteBatchEntry currTime = time.Now() timeLock = sync.Mutex{} addTime = func(d time.Duration) { @@ -218,7 +204,7 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { defer timeLock.Unlock() return currTime } - q.indexBatchFn = func(value []index.WriteBatchEntry) { + q.indexBatchFn = func(value [][]index.WriteBatchEntry) { inserts = append(inserts, value) insertWgs[len(inserts)-1].Done() insertProgressWgs[len(inserts)-1].Wait() @@ -243,16 +229,16 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { }() // first insert - _, err := q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(0), callback)) + _, err := q.InsertBatch(testWriteBatchEntry(testID(0), testTags(0), time.Time{}, callback)) require.NoError(t, err) // wait for first insert batch to complete insertWgs[0].Wait() // now next batch will need to wait as we haven't progressed time - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(1), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(1), testTags(1), time.Time{}, callback)) require.NoError(t, err) - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(2), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(2), testTags(2), time.Time{}, callback)) require.NoError(t, err) // allow first insert to finish @@ -265,7 +251,7 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { assert.Equal(t, 1, numSleeps) // insert third batch, will also need to wait - _, err = q.InsertBatch(testWriteBatchEntry(time.Time{}, testDoc(3), callback)) + _, err = q.InsertBatch(testWriteBatchEntry(testID(3), testTags(3), time.Time{}, callback)) require.NoError(t, err) // allow second batch to finish @@ -292,14 +278,14 @@ func TestIndexInsertQueueFlushedOnClose(t *testing.T) { currTime = time.Now().Truncate(time.Second) ) - q := newNamespaceIndexInsertQueue(func(value []index.WriteBatchEntry) { + q := newNamespaceIndexInsertQueue(func(value [][]index.WriteBatchEntry) { atomic.AddInt64(&numInsertObserved, int64(len(value))) }, func() time.Time { return currTime }, tally.NoopScope) require.NoError(t, q.Start()) for i := 0; i < numInsertExpected; i++ { - _, err := q.InsertBatch(testWriteBatchEntry(time.Time{}, doc.Document{}, nil)) + _, err := q.InsertBatch(testWriteBatchEntry(testID(1), testTags(1), time.Time{}, nil)) require.NoError(t, err) } diff --git a/storage/index_queue_test.go b/storage/index_queue_test.go index 63e927b9b1..7a4b80c28e 100644 --- a/storage/index_queue_test.go +++ b/storage/index_queue_test.go @@ -128,8 +128,8 @@ func TestNamespaceIndexInvalidDocWrite(t *testing.T) { } lifecycle := index.NewMockOnIndexSeries(ctrl) - lifecycle.EXPECT().OnIndexFinalize() - assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, time.Time{}, lifecycle))) + lifecycle.EXPECT().OnIndexFinalize(time.Time{}) + assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, time.Time{}, lifecycle))) } func TestNamespaceIndexWriteAfterClose(t *testing.T) { @@ -149,8 +149,8 @@ func TestNamespaceIndexWriteAfterClose(t *testing.T) { assert.NoError(t, idx.Close()) lifecycle := index.NewMockOnIndexSeries(ctrl) - lifecycle.EXPECT().OnIndexFinalize() - assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, time.Time{}, lifecycle))) + lifecycle.EXPECT().OnIndexFinalize(time.Time{}) + assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, time.Time{}, lifecycle))) } func TestNamespaceIndexWriteQueueError(t *testing.T) { @@ -166,12 +166,13 @@ func TestNamespaceIndexWriteQueueError(t *testing.T) { ident.StringTag("name", "value"), } + n := time.Now() lifecycle := index.NewMockOnIndexSeries(ctrl) - lifecycle.EXPECT().OnIndexFinalize() + lifecycle.EXPECT().OnIndexFinalize(n) q.EXPECT(). InsertBatch(gomock.Any()). Return(nil, fmt.Errorf("random err")) - assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, time.Now(), lifecycle))) + assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, n, lifecycle))) } func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { @@ -214,12 +215,12 @@ func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { ) tooOld := now.Add(-1 * idx.bufferPast).Add(-1 * time.Second) - lifecycle.EXPECT().OnIndexFinalize() - assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, tooOld, lifecycle))) + lifecycle.EXPECT().OnIndexFinalize(tooOld) + assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, tooOld, lifecycle))) tooNew := now.Add(1 * idx.bufferFuture).Add(1 * time.Second) - lifecycle.EXPECT().OnIndexFinalize() - assert.Error(t, idx.WriteBatch(testIndexWriteEntry(id, tags, tooNew, lifecycle))) + lifecycle.EXPECT().OnIndexFinalize(tooOld) + assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, tooNew, lifecycle))) } func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { @@ -244,7 +245,7 @@ func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { var wg sync.WaitGroup lifecycle := index.NewMockOnIndexSeries(ctrl) q.EXPECT().InsertBatch(gomock.Any()).Return(&wg, nil) - assert.NoError(t, idx.WriteBatch(testIndexWriteEntry(id, tags, now, lifecycle))) + assert.NoError(t, idx.WriteBatch(testWriteBatchEntry(id, tags, now, lifecycle))) } func TestNamespaceIndexInsertQuery(t *testing.T) { @@ -276,9 +277,9 @@ func TestNamespaceIndexInsertQuery(t *testing.T) { lifecycleFns = index.NewMockOnIndexSeries(ctrl) ) - lifecycleFns.EXPECT().OnIndexFinalize() + lifecycleFns.EXPECT().OnIndexFinalize(ts) lifecycleFns.EXPECT().OnIndexSuccess(ts) - assert.NoError(t, idx.WriteBatch(testIndexWriteEntry(id, tags, now, lifecycleFns))) + assert.NoError(t, idx.WriteBatch(testWriteBatchEntry(id, tags, now, lifecycleFns))) reQuery, err := m3ninxidx.NewRegexpQuery([]byte("name"), []byte("val.*")) assert.NoError(t, err) diff --git a/storage/shard_foreachentry_prop_test.go b/storage/shard_foreachentry_prop_test.go index 373136b5d1..c4ceb89f87 100644 --- a/storage/shard_foreachentry_prop_test.go +++ b/storage/shard_foreachentry_prop_test.go @@ -28,6 +28,7 @@ import ( "testing" "time" + "github.com/m3db/m3db/storage/series/lookup" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" @@ -199,14 +200,14 @@ func shardEntriesAreEqual(shard *dbShard, expectedEntries []shardEntryState) err return fmt.Errorf("expected to have %d idx, but did not see anything", idx) } nextElem := elem.Next() - entry := elem.Value.(*dbShardEntry) - if !entry.series.ID().Equal(expectedEntry.id) { + entry := elem.Value.(*lookup.Entry) + if !entry.Series.ID().Equal(expectedEntry.id) { return fmt.Errorf("expected id: %s at %d, observed: %s", - expectedEntry.id.String(), idx, entry.series.ID().String()) + expectedEntry.id.String(), idx, entry.Series.ID().String()) } - if entry.readerWriterCount() != expectedEntry.refCount { + if entry.ReaderWriterCount() != expectedEntry.refCount { return fmt.Errorf("expected id: %s at %d to have ref count %d, observed: %d", - entry.series.ID().String(), idx, expectedEntry.refCount, entry.readerWriterCount()) + entry.Series.ID().String(), idx, expectedEntry.refCount, entry.ReaderWriterCount()) } elem = nextElem } @@ -247,7 +248,7 @@ func genBatchWorkFn() gopter.Gen { return gen.UInt8(). Map(func(n uint8) dbShardEntryBatchWorkFn { i := uint8(0) - return func([]*dbShardEntry) bool { + return func([]*lookup.Entry) bool { i++ return i < n } diff --git a/storage/shard_index_test.go b/storage/shard_index_test.go index 6eb53029b0..c793f7f3d9 100644 --- a/storage/shard_index_test.go +++ b/storage/shard_index_test.go @@ -45,19 +45,22 @@ func TestShardInsertNamespaceIndex(t *testing.T) { opts := testDatabaseOptions() lock := sync.Mutex{} - indexWrites := []testIndexWrite{} - later := time.Now().Add(time.Hour) + indexWrites := []index.WriteBatchEntry{} + var blockStart xtime.UnixNano ctrl := gomock.NewController(t) defer ctrl.Finish() idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().Write(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do( - func(id ident.ID, tags ident.Tags, ts time.Time, onIdx index.OnIndexSeries) { + idx.EXPECT().BlockStartForWriteTime(gomock.Any()).Return(blockStart).AnyTimes() + idx.EXPECT().WriteBatch(gomock.Any()).Do( + func(entries []index.WriteBatchEntry) { lock.Lock() - indexWrites = append(indexWrites, testIndexWrite{id: id, tags: tags}) + indexWrites = append(indexWrites, entries...) lock.Unlock() - onIdx.OnIndexSuccess(later) - onIdx.OnIndexFinalize() + for _, e := range entries { + e.OnIndexSeries.OnIndexSuccess(blockStart) + e.OnIndexSeries.OnIndexFinalize(blockStart) + } }).Return(nil).AnyTimes() shard := testDatabaseShardWithIndexFn(t, opts, idx) @@ -84,9 +87,9 @@ func TestShardInsertNamespaceIndex(t *testing.T) { defer lock.Unlock() require.Len(t, indexWrites, 1) - require.Equal(t, "foo", indexWrites[0].id.String()) - require.Equal(t, "name", indexWrites[0].tags[0].Name.String()) - require.Equal(t, "value", indexWrites[0].tags[0].Value.String()) + require.Equal(t, "foo", indexWrites[0].ID.String()) + require.Equal(t, "name", indexWrites[0].Tags[0].Name.String()) + require.Equal(t, "value", indexWrites[0].Tags[0].Value.String()) } func TestShardAsyncInsertNamespaceIndex(t *testing.T) { @@ -230,14 +233,17 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().Write(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do( - func(id ident.ID, tags ident.Tags, ts time.Time, onIdx index.OnIndexSeries) { + idx.EXPECT().BlockStartForWriteTime(gomock.Any()).Return() + idx.EXPECT().WriteBatch(gomock.Any()).Do( + func(batch []index.WriteBatchEntry) { nowLock.Lock() now = now.Add(time.Hour) nowLock.Unlock() - onIdx.OnIndexSuccess(now) - onIdx.OnIndexFinalize() - atomic.AddInt32(&numCalls, 1) + for _, b := range batch { + b.OnIndexSeries.OnIndexSuccess(now) + b.OnIndexSeries.OnIndexFinalize() + atomic.AddInt32(&numCalls, 1) + } }).Return(nil).AnyTimes() shard := testDatabaseShardWithIndexFn(t, opts, idx) diff --git a/storage/shard_insert_queue_test.go b/storage/shard_insert_queue_test.go index 31a4616d03..6530f2e090 100644 --- a/storage/shard_insert_queue_test.go +++ b/storage/shard_insert_queue_test.go @@ -1,3 +1,4 @@ +// +build disable_for_now // Copyright (c) 2016 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy diff --git a/storage/shard_ref_count_test.go b/storage/shard_ref_count_test.go index f9eba2ab4c..0f56e9192d 100644 --- a/storage/shard_ref_count_test.go +++ b/storage/shard_ref_count_test.go @@ -82,7 +82,7 @@ func TestShardWriteSyncRefCount(t *testing.T) { entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) - assert.Equal(t, int32(0), entry.readerWriterCount(), id) + assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) } // write already inserted series' @@ -105,7 +105,7 @@ func TestShardWriteSyncRefCount(t *testing.T) { entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) - assert.Equal(t, int32(0), entry.readerWriterCount(), id) + assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) } } From dfc4c6ae6da3b2637decbf56ee6daaf3b4c7091d Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 12:14:25 -0400 Subject: [PATCH 07/33] Pull m3x and use new Tags type --- ...h_tagged_results_accumulator_merge_test.go | 12 +- ...ch_tagged_results_accumulator_misc_test.go | 6 +- ...etch_tagged_results_index_iterator_test.go | 9 +- client/iterator_matcher_test.go | 12 +- client/options.go | 6 +- client/session.go | 8 +- client/session_fetch_bulk_blocks_test.go | 4 +- client/session_fetch_metadata_test.go | 22 ++-- client/session_write_tagged_test.go | 32 +++--- client/write_tagged_test.go | 2 +- .../series_iterator_split_into_blocks_test.go | 2 +- generated-source-files.mk | 12 -- glide.lock | 28 ++--- glide.yaml | 2 +- integration/integration_data_verify.go | 6 +- integration/integration_index_verify.go | 2 +- integration/setup.go | 2 +- .../tchannelthrift/convert/convert_test.go | 9 +- network/server/tchannelthrift/node/service.go | 2 +- persist/fs/commitlog/options.go | 3 +- persist/fs/commitlog/reader.go | 6 +- persist/fs/commitlog/writer.go | 6 +- persist/fs/persist_manager_test.go | 14 +-- persist/fs/retriever_options.go | 2 +- persist/fs/write.go | 4 +- serialize/decoder.go | 2 +- services/m3dbnode/config/pooling.go | 7 +- services/m3dbnode/server/server.go | 27 ++--- storage/block/metadata.go | 5 +- storage/block/result.go | 35 ++++-- storage/block/result_test.go | 7 +- storage/block/types.go | 3 + storage/bootstrap/bootstrapper/fs/options.go | 2 +- storage/bootstrap/bootstrapper/fs/source.go | 4 +- storage/index/convert/convert.go | 4 +- storage/index/options.go | 19 +-- storage/index/results.go | 21 ++-- storage/index/tag_arraypool_gen.go | 108 ------------------ storage/index/types.go | 6 - storage/options.go | 22 ++-- storage/repair.go | 5 +- storage/repair/metadata.go | 4 +- storage/repair/metadata_test.go | 3 +- storage/repair/types.go | 2 +- storage/series/options.go | 2 +- storage/series/series.go | 41 ++++--- storage/shard.go | 27 +++-- 47 files changed, 229 insertions(+), 340 deletions(-) delete mode 100644 storage/index/tag_arraypool_gen.go diff --git a/client/fetch_tagged_results_accumulator_merge_test.go b/client/fetch_tagged_results_accumulator_merge_test.go index 61d6e2366e..2bce321462 100644 --- a/client/fetch_tagged_results_accumulator_merge_test.go +++ b/client/fetch_tagged_results_accumulator_merge_test.go @@ -394,7 +394,7 @@ func newTestSeries(i int) testSeries { return testSeries{ ns: ident.StringID("testNs"), id: ident.StringID(fmt.Sprintf("id%03d", i)), - tags: ident.Tags{ + tags: ident.NewTags( ident.StringTag( fmt.Sprintf("tagName0%d", i), fmt.Sprintf("tagValue0%d", i), @@ -403,7 +403,7 @@ func newTestSeries(i int) testSeries { fmt.Sprintf("tagName1%d", i), fmt.Sprintf("tagValue1%d", i), ), - }, + ), } } @@ -433,13 +433,13 @@ func (ts testSeries) assertMatchesEncodingIter(t *testing.T, iter encoding.Serie require.Equal(t, ts.ns.String(), iter.Namespace().String()) require.Equal(t, ts.id.String(), iter.ID().String()) require.True(t, ident.NewTagIterMatcher( - ident.NewTagSliceIterator(ts.tags)).Matches(iter.Tags())) + ident.NewTagsIterator(ts.tags)).Matches(iter.Tags())) ts.datapoints.assertMatchesEncodingIter(t, iter) } func (ts testSeries) matcherOption() TaggedIDsIteratorMatcherOption { - tags := make([]string, 0, len(ts.tags)*2) - for _, t := range ts.tags { + tags := make([]string, 0, len(ts.tags.Values())*2) + for _, t := range ts.tags.Values() { tags = append(tags, t.Name.String(), t.Value.String()) } return TaggedIDsIteratorMatcherOption{ @@ -506,7 +506,7 @@ func (td testDatapoints) toRPCSegments(th testFetchTaggedHelper, start time.Time func (th testFetchTaggedHelper) encodeTags(tags ident.Tags) []byte { enc := th.tagEncPool.Get() - iter := ident.NewTagSliceIterator(tags) + iter := ident.NewTagsIterator(tags) require.NoError(th.t, enc.Encode(iter)) data, ok := enc.Data() require.True(th.t, ok) diff --git a/client/fetch_tagged_results_accumulator_misc_test.go b/client/fetch_tagged_results_accumulator_misc_test.go index 6dff37564a..e507218880 100644 --- a/client/fetch_tagged_results_accumulator_misc_test.go +++ b/client/fetch_tagged_results_accumulator_misc_test.go @@ -235,7 +235,11 @@ func newTestFetchTaggedPools() testFetchTaggedPools { pools.multiReaderIteratorArray = encoding.NewMultiReaderIteratorArrayPool(nil) pools.multiReaderIteratorArray.Init() - pools.id = ident.NewPool(nil, opts) + pools.id = ident.NewPool(nil, ident.PoolOptions{ + IDPoolOptions: opts, + TagsPoolOptions: opts, + TagsIteratorPoolOptions: opts, + }) pools.checkedBytesWrapper = xpool.NewCheckedBytesWrapperPool(opts) pools.checkedBytesWrapper.Init() diff --git a/client/fetch_tagged_results_index_iterator_test.go b/client/fetch_tagged_results_index_iterator_test.go index 1e84d46718..0b6e22be50 100644 --- a/client/fetch_tagged_results_index_iterator_test.go +++ b/client/fetch_tagged_results_index_iterator_test.go @@ -49,9 +49,12 @@ func TestFetchTaggedResultsIndexIterator(t *testing.T) { []ident.ID{ident.StringID("ns0"), ident.StringID("ns1"), ident.StringID("ns2")}, []ident.ID{ident.StringID("id0"), ident.StringID("id1"), ident.StringID("id2")}, []ident.TagIterator{ - ident.NewTagIterator(ident.StringTag("tn0", "tv0")), - ident.NewTagIterator(ident.StringTag("tn0", "tv0"), ident.StringTag("tn1", "tv1")), - ident.NewTagIterator(ident.StringTag("tn0", "tv0"), ident.StringTag("tn1", "tv1"), ident.StringTag("tn2", "tv2")), + ident.NewTagsIterator(ident.NewTags( + ident.StringTag("tn0", "tv0"))), + ident.NewTagsIterator(ident.NewTags( + ident.StringTag("tn0", "tv0"), ident.StringTag("tn1", "tv1"))), + ident.NewTagsIterator(ident.NewTags( + ident.StringTag("tn0", "tv0"), ident.StringTag("tn1", "tv1"), ident.StringTag("tn2", "tv2"))), }, }, } { diff --git a/client/iterator_matcher_test.go b/client/iterator_matcher_test.go index ff25e9d2a5..a2988ab2b6 100644 --- a/client/iterator_matcher_test.go +++ b/client/iterator_matcher_test.go @@ -40,13 +40,13 @@ func TestTaggedIDsIteratorMatcherMatches(t *testing.T) { mIter.EXPECT().Current().Return( ident.StringID("ns"), ident.StringID("id0"), - ident.NewTagIterator(ident.StringTag("fgh", "ijk")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("fgh", "ijk"))), ), mIter.EXPECT().Next().Return(true), mIter.EXPECT().Current().Return( ident.StringID("ns"), ident.StringID("id1"), - ident.NewTagIterator(ident.StringTag("fgh", "ijk")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("fgh", "ijk"))), ), mIter.EXPECT().Next().Return(false), mIter.EXPECT().Next().Return(false), @@ -76,7 +76,7 @@ func TestTaggedIDsIteratorMatcherDoesNotMatchTooFew(t *testing.T) { mIter.EXPECT().Current().Return( ident.StringID("ns"), ident.StringID("id0"), - ident.NewTagIterator(ident.StringTag("fgh", "ijk")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("fgh", "ijk"))), ), mIter.EXPECT().Next().Return(false), mIter.EXPECT().Next().Return(false), @@ -106,19 +106,19 @@ func TestTaggedIDsIteratorMatcherDoesNotMatchTooMany(t *testing.T) { mIter.EXPECT().Current().Return( ident.StringID("ns"), ident.StringID("id0"), - ident.NewTagIterator(ident.StringTag("fgh", "ijk")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("fgh", "ijk"))), ), mIter.EXPECT().Next().Return(true), mIter.EXPECT().Current().Return( ident.StringID("ns"), ident.StringID("id1"), - ident.NewTagIterator(ident.StringTag("fgh", "ijk")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("fgh", "ijk"))), ), mIter.EXPECT().Next().Return(true), mIter.EXPECT().Current().Return( ident.StringID("ns"), ident.StringID("id2"), - ident.NewTagIterator(ident.StringTag("fgh", "ijk")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("fgh", "ijk"))), ), ) diff --git a/client/options.go b/client/options.go index 883af6c66a..eb258fa699 100644 --- a/client/options.go +++ b/client/options.go @@ -255,7 +255,11 @@ func newOptions() *options { poolOpts := pool.NewObjectPoolOptions(). SetSize(defaultIdentifierPoolSize) - idPool := ident.NewPool(bytesPool, poolOpts) + idPool := ident.NewPool(bytesPool, ident.PoolOptions{ + IDPoolOptions: poolOpts, + TagsPoolOptions: poolOpts, + TagsIteratorPoolOptions: poolOpts, + }) contextPool := context.NewPool(context.NewOptions(). SetContextPoolOptions(poolOpts). diff --git a/client/session.go b/client/session.go index 52414d0424..d91ea94a90 100644 --- a/client/session.go +++ b/client/session.go @@ -3867,22 +3867,22 @@ func newTagsFromEncodedTags( idPool ident.Pool, ) (ident.Tags, error) { if encodedTags == nil { - return nil, nil + return ident.Tags{}, nil } encodedTags.IncRef() tagDecoder.Reset(encodedTags) - tags := make(ident.Tags, 0, tagDecoder.Remaining()) + tags := idPool.Tags() for tagDecoder.Next() { curr := tagDecoder.Current() - tags = append(tags, idPool.CloneTag(curr)) + tags.Append(idPool.CloneTag(curr)) } encodedTags.DecRef() if err := tagDecoder.Err(); err != nil { - return nil, err + return ident.Tags{}, err } return tags, nil diff --git a/client/session_fetch_bulk_blocks_test.go b/client/session_fetch_bulk_blocks_test.go index b6cc317816..4dd8cabe4a 100644 --- a/client/session_fetch_bulk_blocks_test.go +++ b/client/session_fetch_bulk_blocks_test.go @@ -69,7 +69,7 @@ var ( testIDPool = NewOptions().IdentifierPool() fooID = ident.StringID("foo") fooTags checked.Bytes - fooDecodedTags = ident.Tags{ident.StringTag("aaa", "bbb")} + fooDecodedTags = ident.NewTags(ident.StringTag("aaa", "bbb")) barID = ident.StringID("bar") bazID = ident.StringID("baz") testHost = topology.NewHost("testhost", "testhost:9000") @@ -79,7 +79,7 @@ func init() { testTagDecodingPool.Init() testTagEncodingPool.Init() tagEncoder := testTagEncodingPool.Get() - err := tagEncoder.Encode(ident.NewTagSliceIterator(fooDecodedTags)) + err := tagEncoder.Encode(ident.NewTagsIterator(fooDecodedTags)) if err != nil { panic(err) } diff --git a/client/session_fetch_metadata_test.go b/client/session_fetch_metadata_test.go index f41a4eb5c1..58afb6982d 100644 --- a/client/session_fetch_metadata_test.go +++ b/client/session_fetch_metadata_test.go @@ -55,7 +55,7 @@ func TestPeerBlockMetadataIter(t *testing.T) { { peer: peer, id: ident.StringID("foo"), - encodedTags: mustEncodeTags(t, ident.Tags{ident.StringTag("aaa", "bbb")}), + encodedTags: mustEncodeTags(t, ident.NewTags(ident.StringTag("aaa", "bbb"))), block: blockMetadata{ start: now, size: int64(1), checksum: &checksums[0], }, @@ -63,7 +63,7 @@ func TestPeerBlockMetadataIter(t *testing.T) { { peer: peer, id: ident.StringID("foo"), - encodedTags: mustEncodeTags(t, ident.Tags{ident.StringTag("aaa", "bbb")}), + encodedTags: mustEncodeTags(t, ident.NewTags(ident.StringTag("aaa", "bbb"))), block: blockMetadata{ start: now.Add(time.Second), size: int64(2), checksum: &checksums[1], lastRead: lastRead, }, @@ -71,7 +71,7 @@ func TestPeerBlockMetadataIter(t *testing.T) { { peer: peer, id: ident.StringID("bar"), - encodedTags: mustEncodeTags(t, ident.Tags{ident.StringTag("ccc", "ddd")}), + encodedTags: mustEncodeTags(t, ident.NewTags(ident.StringTag("ccc", "ddd"))), block: blockMetadata{ start: now, size: int64(3), checksum: &checksums[2], lastRead: lastRead, }, @@ -79,7 +79,7 @@ func TestPeerBlockMetadataIter(t *testing.T) { { peer: peer, id: ident.StringID("baz"), - encodedTags: mustEncodeTags(t, ident.Tags{ident.StringTag("eee", "fff")}), + encodedTags: mustEncodeTags(t, ident.NewTags(ident.StringTag("eee", "fff"))), block: blockMetadata{ start: now, size: int64(4), checksum: nil, lastRead: lastRead, }, @@ -106,19 +106,19 @@ func TestPeerBlockMetadataIter(t *testing.T) { expected := []testHostBlock{ {h, block.NewMetadata(ident.StringID("foo"), - ident.Tags{ident.StringTag("aaa", "bbb")}, + ident.NewTags(ident.StringTag("aaa", "bbb")), inputs[0].block.start, inputs[0].block.size, inputs[0].block.checksum, inputs[0].block.lastRead)}, {h, block.NewMetadata(ident.StringID("foo"), - ident.Tags{ident.StringTag("aaa", "bbb")}, + ident.NewTags(ident.StringTag("aaa", "bbb")), inputs[1].block.start, inputs[1].block.size, inputs[1].block.checksum, inputs[1].block.lastRead)}, {h, block.NewMetadata(ident.StringID("bar"), - ident.Tags{ident.StringTag("ccc", "ddd")}, + ident.NewTags(ident.StringTag("ccc", "ddd")), inputs[2].block.start, inputs[2].block.size, inputs[2].block.checksum, inputs[2].block.lastRead)}, {h, block.NewMetadata(ident.StringID("baz"), - ident.Tags{ident.StringTag("eee", "fff")}, + ident.NewTags(ident.StringTag("eee", "fff")), inputs[3].block.start, inputs[3].block.size, inputs[3].block.checksum, inputs[3].block.lastRead)}, } @@ -128,8 +128,8 @@ func TestPeerBlockMetadataIter(t *testing.T) { actual := actual[i] assert.Equal(t, expected.host.String(), actual.host.String()) assert.True(t, expected.block.ID.Equal(actual.block.ID)) - tagMatcher := ident.NewTagIterMatcher(ident.NewTagSliceIterator(expected.block.Tags)) - assert.True(t, tagMatcher.Matches(ident.NewTagSliceIterator(actual.block.Tags))) + tagMatcher := ident.NewTagIterMatcher(ident.NewTagsIterator(expected.block.Tags)) + assert.True(t, tagMatcher.Matches(ident.NewTagsIterator(actual.block.Tags))) assert.True(t, expected.block.Start.Equal(actual.block.Start)) assert.Equal(t, expected.block.Size, actual.block.Size) assert.Equal(t, expected.block.Checksum, actual.block.Checksum) @@ -139,7 +139,7 @@ func TestPeerBlockMetadataIter(t *testing.T) { func mustEncodeTags(t *testing.T, tags ident.Tags) checked.Bytes { encoder := testTagEncodingPool.Get() - err := encoder.Encode(ident.NewTagSliceIterator(tags)) + err := encoder.Encode(ident.NewTagsIterator(tags)) require.NoError(t, err) data, ok := encoder.Data() require.True(t, ok) diff --git a/client/session_write_tagged_test.go b/client/session_write_tagged_test.go index e7619bed68..34bb72b457 100644 --- a/client/session_write_tagged_test.go +++ b/client/session_write_tagged_test.go @@ -99,7 +99,7 @@ func TestSessionWriteTagged(t *testing.T) { var writeWg sync.WaitGroup writeWg.Add(1) go func() { - resultErr = session.WriteTagged(w.ns, w.id, ident.NewTagSliceIterator(w.tags), + resultErr = session.WriteTagged(w.ns, w.id, ident.NewTagsIterator(w.tags), w.t, w.value, w.unit, w.annotation) writeWg.Done() }() @@ -132,11 +132,9 @@ func TestSessionWriteTaggedBadUnitErr(t *testing.T) { unit xtime.Unit annotation []byte }{ - ns: ident.StringID("testNs"), - id: ident.StringID("foo"), - tags: ident.Tags{ - ident.StringTag("abc", "def"), - }, + ns: ident.StringID("testNs"), + id: ident.StringID("foo"), + tags: ident.NewTags(ident.StringTag("abc", "def")), value: 1.0, t: time.Now(), unit: xtime.Unit(byte(255)), @@ -148,7 +146,7 @@ func TestSessionWriteTaggedBadUnitErr(t *testing.T) { assert.NoError(t, session.Open()) assert.Error(t, session.WriteTagged( - w.ns, w.id, ident.NewTagSliceIterator(w.tags), w.t, w.value, w.unit, w.annotation)) + w.ns, w.id, ident.NewTagsIterator(w.tags), w.t, w.value, w.unit, w.annotation)) assert.NoError(t, session.Close()) } @@ -203,11 +201,9 @@ func TestSessionWriteTaggedBadRequestErrorIsNonRetryable(t *testing.T) { unit xtime.Unit annotation []byte }{ - ns: ident.StringID("testNs"), - id: ident.StringID("foo"), - tags: ident.Tags{ - ident.StringTag("abc", "def"), - }, + ns: ident.StringID("testNs"), + id: ident.StringID("foo"), + tags: ident.NewTags(ident.StringTag("abc", "def")), value: 1.0, t: time.Now(), unit: xtime.Second, @@ -234,7 +230,7 @@ func TestSessionWriteTaggedBadRequestErrorIsNonRetryable(t *testing.T) { session.state.RUnlock() err := session.WriteTagged( - w.ns, w.id, ident.NewTagSliceIterator(w.tags), w.t, w.value, w.unit, w.annotation) + w.ns, w.id, ident.NewTagsIterator(w.tags), w.t, w.value, w.unit, w.annotation) assert.Error(t, err) assert.True(t, xerrors.IsNonRetryableError(err)) @@ -299,7 +295,7 @@ func TestSessionWriteTaggedRetry(t *testing.T) { writeWg.Add(1) go func() { resultErr = session.WriteTagged( - w.ns, w.id, ident.NewTagSliceIterator(w.tags), w.t, w.value, w.unit, w.annotation) + w.ns, w.id, ident.NewTagsIterator(w.tags), w.t, w.value, w.unit, w.annotation) writeWg.Done() }() @@ -389,7 +385,7 @@ func testWriteTaggedConsistencyLevel( writeWg.Add(1) go func() { resultErr = session.WriteTagged( - w.ns, w.id, ident.NewTagSliceIterator(w.tags), w.t, w.value, w.unit, w.annotation) + w.ns, w.id, ident.NewTagsIterator(w.tags), w.t, w.value, w.unit, w.annotation) writeWg.Done() }() @@ -474,10 +470,10 @@ func newWriteTaggedStub() writeTaggedStub { return writeTaggedStub{ ns: ident.StringID("testNs"), id: ident.StringID("foo"), - tags: ident.Tags{ + tags: ident.NewTags( ident.StringTag("foo", "bar"), ident.StringTag("baz", "bah"), - }, + ), value: 1.0, t: time.Now(), unit: xtime.Second, @@ -487,7 +483,7 @@ func newWriteTaggedStub() writeTaggedStub { func testEncodeTags(tags ident.Tags) checked.Bytes { m := make(map[string]string) - for _, t := range tags { + for _, t := range tags.Values() { m[t.Name.String()] = t.Value.String() } b := testEncode(m) diff --git a/client/write_tagged_test.go b/client/write_tagged_test.go index 80a549123c..3c32bf1626 100644 --- a/client/write_tagged_test.go +++ b/client/write_tagged_test.go @@ -165,7 +165,7 @@ func writeTaggedTestSetup(t *testing.T, writeWg *sync.WaitGroup) (*writeState, * // Begin write writeWg.Add(1) go func() { - s.WriteTagged(w.ns, w.id, ident.NewTagSliceIterator(w.tags), w.t, w.value, w.unit, w.annotation) + s.WriteTagged(w.ns, w.id, ident.NewTagsIterator(w.tags), w.t, w.value, w.unit, w.annotation) writeWg.Done() }() diff --git a/encoding/series_iterator_split_into_blocks_test.go b/encoding/series_iterator_split_into_blocks_test.go index 4b1082eb34..863dd767a2 100644 --- a/encoding/series_iterator_split_into_blocks_test.go +++ b/encoding/series_iterator_split_into_blocks_test.go @@ -83,7 +83,7 @@ func TestDeconstructAndReconstruct(t *testing.T) { multiReader.Reset([]xio.SegmentReader{reader}, blockStart, blockSize) orig := encoding.NewSeriesIterator(ident.StringID("foo"), ident.StringID("namespace"), - ident.NewTagSliceIterator(ident.Tags{}), start, end, []encoding.MultiReaderIterator{multiReader}, nil) + ident.NewTagsIterator(ident.Tags{}), start, end, []encoding.MultiReaderIterator{multiReader}, nil) // Construct a per block view of the series series := Series{ diff --git a/generated-source-files.mk b/generated-source-files.mk index 583a9cfb10..43be0d111d 100644 --- a/generated-source-files.mk +++ b/generated-source-files.mk @@ -157,18 +157,6 @@ genny-arraypool-node-segments: install-m3x-repo rename_type_middle=Segments \ rename_constructor=newSegmentsArrayPool -# arraypool generation rule for ./storage/index/tagArrayPool -.PHONY: genny-arraypool-storage-index-tag -genny-arraypool-storage-index-tag: install-m3x-repo - cd $(m3x_package_path) && make genny-arraypool \ - pkg=index \ - elem_type=ident.Tag \ - target_package=$(m3db_package)/storage/index \ - out_file=tag_arraypool_gen.go \ - rename_type_prefix=Tag \ - rename_type_middle=Tag \ - rename_constructor=NewTagArrayPool - # generation rule for all generated leakcheckpools .PHONY: genny-leakcheckpool-all genny-leakcheckpool-all: genny-leakcheckpool-fetch-tagged-attempt \ diff --git a/glide.lock b/glide.lock index d648d7eade..fcfd22d688 100644 --- a/glide.lock +++ b/glide.lock @@ -1,5 +1,5 @@ -hash: 713fed9cf8519794dc959f8c18d63c0da69d3ba057a340644f0d2db12aa04dd1 -updated: 2018-05-11T13:10:38.674476-04:00 +hash: ae522bcbc20f0ddb69902e6e73f1b42499a361d4fda9e4c1f9f32b148a253d19 +updated: 2018-05-13T10:07:27.359065-04:00 imports: - name: github.com/alecthomas/template version: a0175ee3bccc567396460bf5acd36800cb10c49c @@ -112,19 +112,21 @@ imports: - spew - name: github.com/dgrijalva/jwt-go version: d2709f9f1f31ebcda9651b03077758c1f3a0018c +- name: github.com/dgryski/go-bits + version: 2ad8d707cc05b1815ce6ff2543bb5e8d8f9298ef - name: github.com/ghodss/yaml version: 0ca9ea5df5451ffdf184b4428c902747c2c11cd7 - name: github.com/glycerine/go-unsnap-stream version: 62a9a9eb44fd8932157b1a8ace2149eff5971af6 - name: github.com/go-kit/kit - version: 91c58cc7d6d770b0285fdd69949a23d817a57d13 + version: 04dd4f741c6e76cc170a4d7913f4c625952e6f58 subpackages: - log - log/level - name: github.com/go-logfmt/logfmt version: 390ab7935ee28ec6b286364bba9b4dd6410cb3d5 - name: github.com/go-stack/stack - version: 259ab82a6cad3992b4e21ff5cac294ccb06474bc + version: 54be5f394ed2c3e19dac9134a40a95ba5a017f7b - name: github.com/gogo/protobuf version: 100ba4e885062801d56799d78530b73b178a78f3 subpackages: @@ -153,7 +155,7 @@ imports: - name: github.com/gorilla/context version: 08b5f424b9271eedf6f9f0ce86cb9396ed337a42 - name: github.com/gorilla/mux - version: 53c1911da2b537f792e7cafcb446b05ffe33b996 + version: e3702bed27f0d39777b0b37b664b6280e8ef8fbf - name: github.com/grpc-ecosystem/go-grpc-prometheus version: 6b7015e65d366bf3f19b2b2a000a831940f0f7e0 - name: github.com/grpc-ecosystem/grpc-gateway @@ -236,7 +238,7 @@ imports: - search/query - search/searcher - name: github.com/m3db/m3x - version: 6148700dde75adcdcc27d16fb68cee2d9d9126d8 + version: df82cd7aacfd8439586821ecd6e073da35bfbb02 vcs: git subpackages: - checked @@ -270,11 +272,11 @@ imports: - name: github.com/mschoch/smat version: 90eadee771aeab36e8bf796039b8c261bebebe4f - name: github.com/nightlyone/lockfile - version: 6a197d5ea61168f2ac821de2b7f011b250904900 + version: 1d49c987357a327b5b03aa84cbddd582c328615d - name: github.com/nu7hatch/gouuid version: 179d4d0c4d8d407a32af483c2354df1d2c91e6c3 - name: github.com/oklog/ulid - version: 7e1bd8a9a281b5abdf651a430b918bb0ecdbceda + version: 66bb6560562feca7045b23db1ae85b01260f87c5 - name: github.com/opentracing/opentracing-go version: 855519783f479520497c6b3445611b05fc42f009 subpackages: @@ -286,7 +288,7 @@ imports: - name: github.com/philhofer/fwd version: bb6d471dc95d4fe11e432687f8b70ff496cf3136 - name: github.com/pkg/errors - version: 816c9085562cd7ee03e7f8188a1cfd942858cded + version: 248dadf4e9068a0b3e79f02ed0a610d935de5302 - name: github.com/pkg/profile version: 5b67d428864e92711fcbd2f8629456121a56d91f - name: github.com/pmezard/go-difflib @@ -294,7 +296,7 @@ imports: subpackages: - difflib - name: github.com/prometheus/client_golang - version: c5b7fccd204277076155f10851dad72b76a49317 + version: 967789050ba94deca04a5e84cce8ad472ce313c1 subpackages: - prometheus - name: github.com/prometheus/client_model @@ -323,12 +325,10 @@ imports: - util/strutil - util/testutil - name: github.com/prometheus/tsdb - version: def6e5a57439cffe7b44a619c05bce4ac513a63e + version: 706602daed1487f7849990678b4ece4599745905 subpackages: - - chunkenc - chunks - fileutil - - index - labels - name: github.com/RoaringBitmap/roaring version: 4bbba8874933b985c9160558fddea525dde73e63 @@ -420,7 +420,7 @@ imports: - lex/httplex - trace - name: golang.org/x/sync - version: fd80eb99c8f653c847d294a001bdf2a3a6f768f5 + version: 450f422ab23cf9881c94e2db30cac0eb1b7cf80c subpackages: - errgroup - name: golang.org/x/sys diff --git a/glide.yaml b/glide.yaml index 3fe2717d64..0c822ecc95 100644 --- a/glide.yaml +++ b/glide.yaml @@ -1,7 +1,7 @@ package: github.com/m3db/m3db import: - package: github.com/m3db/m3x - version: 6148700dde75adcdcc27d16fb68cee2d9d9126d8 + version: df82cd7aacfd8439586821ecd6e073da35bfbb02 vcs: git subpackages: - checked diff --git a/integration/integration_data_verify.go b/integration/integration_data_verify.go index 0c0c1dc370..6d68afe5e4 100644 --- a/integration/integration_data_verify.go +++ b/integration/integration_data_verify.go @@ -143,7 +143,7 @@ func verifySeriesMapForRange( expected, ok := expectedMetadata[id] require.True(t, ok, fmt.Sprintf("unexpected ID: %s", id)) - expectedTagsIter := ident.NewTagSliceIterator(expected.Tags) + expectedTagsIter := ident.NewTagsIterator(expected.Tags) actualTagsIter := actual.Tags.Duplicate() tagMatcher := ident.NewTagIterMatcher(expectedTagsIter) tagsMatch := tagMatcher.Matches(actualTagsIter) @@ -188,8 +188,8 @@ func writeVerifyDebugOutput(t *testing.T, filePath string, start, end time.Time, list := make(readableSeriesList, 0, len(series)) for i := range series { - tags := make([]readableSeriesTag, len(series[i].Tags)) - for _, tag := range series[i].Tags { + tags := make([]readableSeriesTag, len(series[i].Tags.Values())) + for _, tag := range series[i].Tags.Values() { tags = append(tags, readableSeriesTag{ Name: tag.Name.String(), Value: tag.Value.String(), diff --git a/integration/integration_index_verify.go b/integration/integration_index_verify.go index fa66a948f7..06c6578918 100644 --- a/integration/integration_index_verify.go +++ b/integration/integration_index_verify.go @@ -71,7 +71,7 @@ func verifyQueryMetadataResults( require.True(t, ok, fmt.Sprintf("not expecting ID: %s", idStr)) - expectedTagsIter := ident.NewTagSliceIterator(result.series.Tags) + expectedTagsIter := ident.NewTagsIterator(result.series.Tags) matcher := ident.NewTagIterMatcher(expectedTagsIter) assert.True(t, matcher.Matches(tags), fmt.Sprintf("tags not matching for ID: %s", idStr)) diff --git a/integration/setup.go b/integration/setup.go index 42c24e1144..91bb297d1e 100644 --- a/integration/setup.go +++ b/integration/setup.go @@ -151,7 +151,7 @@ func newTestSetup(t *testing.T, opts testOptions, fsOpts fs.Options) (*testSetup storageOpts = storageOpts.SetBytesPool(bytesPool) - idPool := ident.NewNativePool(bytesPool, nil) + idPool := ident.NewNativePool(bytesPool, ident.PoolOptions{}) storageOpts = storageOpts.SetIdentifierPool(idPool) } diff --git a/network/server/tchannelthrift/convert/convert_test.go b/network/server/tchannelthrift/convert/convert_test.go index cfbe95218e..cd8f3f333d 100644 --- a/network/server/tchannelthrift/convert/convert_test.go +++ b/network/server/tchannelthrift/convert/convert_test.go @@ -156,8 +156,13 @@ type testPools struct { } func newTestPools() *testPools { - id := ident.NewPool(nil, nil) - wrapper := xpool.NewCheckedBytesWrapperPool(pool.NewObjectPoolOptions().SetSize(1)) + poolOpts := pool.NewObjectPoolOptions().SetSize(1) + id := ident.NewPool(nil, ident.PoolOptions{ + IDPoolOptions: poolOpts, + TagsPoolOptions: poolOpts, + TagsIteratorPoolOptions: poolOpts, + }) + wrapper := xpool.NewCheckedBytesWrapperPool(poolOpts) wrapper.Init() return &testPools{id, wrapper} } diff --git a/network/server/tchannelthrift/node/service.go b/network/server/tchannelthrift/node/service.go index 8c027a4a94..a821cc503b 100644 --- a/network/server/tchannelthrift/node/service.go +++ b/network/server/tchannelthrift/node/service.go @@ -293,7 +293,7 @@ func (s *service) FetchTagged(tctx thrift.Context, req *rpc.FetchTaggedRequest) } results := queryResult.Results nsID := results.Namespace() - tagsIter := ident.NewTagSliceIterator(nil) + tagsIter := ident.NewTagsIterator(ident.Tags{}) for _, entry := range results.Map().Iter() { tsID := entry.Key() tags := entry.Value() diff --git a/persist/fs/commitlog/options.go b/persist/fs/commitlog/options.go index f6c2a66f0b..6908296c7c 100644 --- a/persist/fs/commitlog/options.go +++ b/persist/fs/commitlog/options.go @@ -98,8 +98,7 @@ func NewOptions() Options { readConcurrency: defaultReadConcurrency, } o.bytesPool.Init() - - o.identPool = ident.NewPool(o.bytesPool, pool.NewObjectPoolOptions()) + o.identPool = ident.NewPool(o.bytesPool, ident.PoolOptions{}) return o } diff --git a/persist/fs/commitlog/reader.go b/persist/fs/commitlog/reader.go index b2051e5dd8..511e68528f 100644 --- a/persist/fs/commitlog/reader.go +++ b/persist/fs/commitlog/reader.go @@ -426,11 +426,11 @@ func (r *reader) decodeAndHandleMetadata( tagDecoderCheckedBytes.Reset(decoded.EncodedTags) tagDecoder.Reset(tagDecoderCheckedBytes) - tags = make(ident.Tags, 0, tagDecoder.Remaining()) + idPool := r.opts.IdentifierPool() + tags = idPool.Tags() for tagDecoder.Next() { curr := tagDecoder.Current() - clone := r.opts.IdentifierPool().CloneTag(curr) - tags = append(tags, clone) + tags.Append(idPool.CloneTag(curr)) } err = tagDecoder.Err() if err != nil { diff --git a/persist/fs/commitlog/writer.go b/persist/fs/commitlog/writer.go index edaf354f6e..0507f5dbf1 100644 --- a/persist/fs/commitlog/writer.go +++ b/persist/fs/commitlog/writer.go @@ -97,7 +97,7 @@ type writer struct { logEncoder *msgpack.Encoder metadataEncoder *msgpack.Encoder tagEncoder serialize.TagEncoder - tagSliceIter ident.TagSliceIterator + tagSliceIter ident.TagsIterator } func newCommitLogWriter( @@ -119,7 +119,7 @@ func newCommitLogWriter( logEncoder: msgpack.NewEncoder(), metadataEncoder: msgpack.NewEncoder(), tagEncoder: opts.FilesystemOptions().TagEncoderPool().Get(), - tagSliceIter: ident.NewTagSliceIterator(nil), + tagSliceIter: ident.NewTagsIterator(ident.Tags{}), } } @@ -181,7 +181,7 @@ func (w *writer) Write( encodedTags []byte ) - if tags != nil { + if tags.Values() != nil { w.tagSliceIter.Reset(tags) w.tagEncoder.Reset() err := w.tagEncoder.Encode(w.tagSliceIter) diff --git a/persist/fs/persist_manager_test.go b/persist/fs/persist_manager_test.go index b684d6abc1..eb35a3dcbc 100644 --- a/persist/fs/persist_manager_test.go +++ b/persist/fs/persist_manager_test.go @@ -207,7 +207,7 @@ func TestPersistenceManagerPrepareSuccess(t *testing.T) { var ( id = ident.StringID("foo") - tags = ident.Tags{ident.StringTag("bar", "baz")} + tags = ident.NewTags(ident.StringTag("bar", "baz")) head = checked.NewBytes([]byte{0x1, 0x2}, nil) tail = checked.NewBytes([]byte{0x3, 0x4}, nil) segment = ts.NewSegment(head, tail, ts.FinalizeNone) @@ -279,7 +279,7 @@ func TestPersistenceManagerNoRateLimit(t *testing.T) { now time.Time slept time.Duration id = ident.StringID("foo") - tags = ident.Tags{ident.StringTag("bar", "baz")} + tags = ident.NewTags(ident.StringTag("bar", "baz")) head = checked.NewBytes([]byte{0x1, 0x2}, nil) tail = checked.NewBytes([]byte{0x3}, nil) segment = ts.NewSegment(head, tail, ts.FinalizeNone) @@ -392,21 +392,21 @@ func TestPersistenceManagerWithRateLimit(t *testing.T) { // Start persistence now = time.Now() - require.NoError(t, prepared.Persist(id, nil, segment, checksum)) + require.NoError(t, prepared.Persist(id, ident.Tags{}, segment, checksum)) // Assert we don't rate limit if the count is not enough yet - require.NoError(t, prepared.Persist(id, nil, segment, checksum)) + require.NoError(t, prepared.Persist(id, ident.Tags{}, segment, checksum)) require.Equal(t, time.Duration(0), slept) // Advance time and check we rate limit if the disk throughput exceeds the limit now = now.Add(time.Microsecond) - require.NoError(t, prepared.Persist(id, nil, segment, checksum)) + require.NoError(t, prepared.Persist(id, ident.Tags{}, segment, checksum)) require.Equal(t, time.Duration(1861), slept) // Advance time and check we don't rate limit if the disk throughput is below the limit - require.NoError(t, prepared.Persist(id, nil, segment, checksum)) + require.NoError(t, prepared.Persist(id, ident.Tags{}, segment, checksum)) now = now.Add(time.Second - time.Microsecond) - require.NoError(t, prepared.Persist(id, nil, segment, checksum)) + require.NoError(t, prepared.Persist(id, ident.Tags{}, segment, checksum)) require.Equal(t, time.Duration(1861), slept) require.Equal(t, int64(15), pm.bytesWritten) diff --git a/persist/fs/retriever_options.go b/persist/fs/retriever_options.go index 25d30599dd..e28f5ab3c8 100644 --- a/persist/fs/retriever_options.go +++ b/persist/fs/retriever_options.go @@ -53,7 +53,7 @@ func NewBlockRetrieverOptions() BlockRetrieverOptions { bytesPool: bytesPool, segmentReaderPool: xio.NewSegmentReaderPool(nil), fetchConcurrency: defaultFetchConcurrency, - identifierPool: ident.NewPool(bytesPool, nil), + identifierPool: ident.NewPool(bytesPool, ident.PoolOptions{}), } o.segmentReaderPool.Init() return o diff --git a/persist/fs/write.go b/persist/fs/write.go index c05d4d6dac..ff76c9e6e5 100644 --- a/persist/fs/write.go +++ b/persist/fs/write.go @@ -416,7 +416,7 @@ func (w *writer) writeIndexFileContents( var ( offset int64 prevID []byte - tagsIter = ident.NewTagSliceIterator(nil) + tagsIter = ident.NewTagsIterator(ident.Tags{}) tagsEncoder = w.tagEncoderPool.Get() ) defer tagsEncoder.Finalize() @@ -429,7 +429,7 @@ func (w *writer) writeIndexFileContents( } var encodedTags []byte - if tags := w.indexEntries[i].tags; tags != nil { + if tags := w.indexEntries[i].tags; tags.Values() != nil { tagsIter.Reset(tags) tagsEncoder.Reset() if err := tagsEncoder.Encode(tagsIter); err != nil { diff --git a/serialize/decoder.go b/serialize/decoder.go index e4a78a3775..c7dfddeac4 100644 --- a/serialize/decoder.go +++ b/serialize/decoder.go @@ -118,7 +118,7 @@ func (d *decoder) decodeTag() (ident.Tag, error) { return ident.Tag{}, err } - return ident.Tag{name, value}, nil + return ident.Tag{Name: name, Value: value}, nil } func (d *decoder) decodeID() (ident.ID, error) { diff --git a/services/m3dbnode/config/pooling.go b/services/m3dbnode/config/pooling.go index 1c362636a8..fb9351d978 100644 --- a/services/m3dbnode/config/pooling.go +++ b/services/m3dbnode/config/pooling.go @@ -93,8 +93,11 @@ type PoolingPolicy struct { // The policy for the BlocksMetadataSlice pool BlocksMetadataSlicePool CapacityPoolPolicy `yaml:"blocksMetadataSlicePool"` - // The policy for the tagArray pool - TagArrayPool MaxCapacityPoolPolicy `yaml:"tagArrayPool"` + // The policy for the tags pool + TagsPool MaxCapacityPoolPolicy `yaml:"tagsPool"` + + // The policy for the tags iterator pool + TagsIteratorPool PoolPolicy `yaml:"tagIteratorPool"` // The policy for the index.ResultsPool IndexResultsPool PoolPolicy `yaml:"indexResultsPool"` diff --git a/services/m3dbnode/server/server.go b/services/m3dbnode/server/server.go index b9dd957d82..c2ffc23500 100644 --- a/services/m3dbnode/server/server.go +++ b/services/m3dbnode/server/server.go @@ -910,17 +910,13 @@ func withEncodingAndPoolingOptions( multiIteratorPool := encoding.NewMultiReaderIteratorPool( poolOptions(policy.IteratorPool, scope.SubScope("multi-iterator-pool"))) - var identifierPool ident.Pool - switch policy.Type { - case "simple": - identifierPool = ident.NewPool( - bytesPool, - poolOptions(policy.IdentifierPool, scope.SubScope("identifier-pool"))) - case "native": - identifierPool = ident.NewNativePool( - bytesPool, - poolOptions(policy.IdentifierPool, scope.SubScope("identifier-pool"))) - } + identifierPool := ident.NewPool(bytesPool, ident.PoolOptions{ + IDPoolOptions: poolOptions(policy.IdentifierPool, scope.SubScope("identifier-pool")), + TagsPoolOptions: maxCapacityPoolOptions(policy.TagsPool, scope.SubScope("tags-pool")), + TagsCapacity: policy.TagsPool.Capacity, + TagsMaxCapacity: policy.TagsPool.MaxCapacity, + TagsIteratorPoolOptions: poolOptions(policy.TagsIteratorPool, scope.SubScope("tags-iterator-pool")), + }) fetchBlockMetadataResultsPool := block.NewFetchBlockMetadataResultsPool( capacityPoolOptions(policy.FetchBlockMetadataResultsPool, @@ -996,14 +992,6 @@ func withEncodingAndPoolingOptions( SetBytesPool(bytesPool). SetIdentifierPool(identifierPool)) - // options related to the indexing sub-system - tagArrPool := index.NewTagArrayPool(index.TagArrayPoolOpts{ - Options: maxCapacityPoolOptions(policy.TagArrayPool, scope.SubScope("tag-array-pool")), - Capacity: policy.TagArrayPool.Capacity, - MaxCapacity: policy.TagArrayPool.MaxCapacity, - }) - tagArrPool.Init() - resultsPool := index.NewResultsPool(poolOptions(policy.IndexResultsPool, scope.SubScope("index-results-pool"))) indexOpts := opts.IndexOptions(). @@ -1012,7 +1000,6 @@ func withEncodingAndPoolingOptions( opts.IndexOptions().MemSegmentOptions().SetInstrumentOptions(iopts)). SetIdentifierPool(identifierPool). SetCheckedBytesPool(bytesPool). - SetTagArrayPool(tagArrPool). SetResultsPool(resultsPool) resultsPool.Init(func() index.Results { return index.NewResults(indexOpts) }) diff --git a/storage/block/metadata.go b/storage/block/metadata.go index adece274a1..50e1ecd850 100644 --- a/storage/block/metadata.go +++ b/storage/block/metadata.go @@ -51,8 +51,5 @@ func (m *Metadata) Finalize() { m.ID.Finalize() m.ID = nil } - if m.Tags != nil { - m.Tags.Finalize() - m.Tags = nil - } + m.Tags.Finalize() } diff --git a/storage/block/result.go b/storage/block/result.go index 6c4006a8e6..6b5a5d7939 100644 --- a/storage/block/result.go +++ b/storage/block/result.go @@ -164,11 +164,9 @@ func (s *fetchBlocksMetadataResults) Reset() { func (s *fetchBlocksMetadataResults) Close() { for i := range s.results { - // NB(r): We explicitly do not finalize ID or Tags as - // some of them are refs to series in memory right now. - // For ID and Tags coming from disk callers can use the context - // to register finalizers for these types. + s.results[i].ID.Finalize() s.results[i].ID = nil + s.results[i].Tags.Close() s.results[i].Tags = nil s.results[i].Blocks.Close() s.results[i].Blocks = nil @@ -184,11 +182,11 @@ type filteredBlocksMetadataIter struct { metadata Metadata resIdx int blockIdx int + err error } // NewFilteredBlocksMetadataIter creates a new filtered blocks metadata -// iterator, currently this will not propagate tags as there is no efficient -// way to go from a tags iterator back to tags. +// iterator, there's no pooling of the tags returned currently. // Only the repair process uses this currently which is unoptimized. func NewFilteredBlocksMetadataIter( res FetchBlocksMetadataResults, @@ -197,6 +195,9 @@ func NewFilteredBlocksMetadataIter( } func (it *filteredBlocksMetadataIter) Next() bool { + if it.err != nil { + return false + } if it.resIdx >= len(it.res) { return false } @@ -204,8 +205,8 @@ func (it *filteredBlocksMetadataIter) Next() bool { for it.blockIdx < len(blocks) { block := blocks[it.blockIdx] if block.Err != nil { - it.blockIdx++ - continue + it.err = block.Err + return false } break } @@ -216,8 +217,18 @@ func (it *filteredBlocksMetadataIter) Next() bool { } it.id = it.res[it.resIdx].ID block := blocks[it.blockIdx] - // TODO(r): When reviving the end to end repair process, propagate tags. - it.metadata = NewMetadata(it.id, nil, block.Start, + tags := ident.NewTags() + tagsIter := it.res[it.resIdx].Tags + for tagsIter.Next() { + curr := tagsIter.Current() + tags.Append(ident.StringTag(curr.Name.String(), curr.Value.String())) + } + if err := tagsIter.Err(); err != nil { + it.err = err + return false + } + tagsIter.Close() + it.metadata = NewMetadata(it.id, tags, block.Start, block.Size, block.Checksum, block.LastRead) it.blockIdx++ return true @@ -226,3 +237,7 @@ func (it *filteredBlocksMetadataIter) Next() bool { func (it *filteredBlocksMetadataIter) Current() (ident.ID, Metadata) { return it.id, it.metadata } + +func (it *filteredBlocksMetadataIter) Err() error { + return it.err +} diff --git a/storage/block/result_test.go b/storage/block/result_test.go index 5e121a6670..8e2737a7c4 100644 --- a/storage/block/result_test.go +++ b/storage/block/result_test.go @@ -92,13 +92,14 @@ func TestFilteredBlocksMetadataIter(t *testing.T) { _, metadata := iter.Current() actual = append(actual, metadata) } + require.NoError(t, iter.Err()) expected := []Metadata{ - NewMetadata(ident.StringID("foo"), nil, now.Add(-time.Second), + NewMetadata(ident.StringID("foo"), ident.Tags{}, now.Add(-time.Second), sizes[0], &checksums[0], lastRead), - NewMetadata(ident.StringID("bar"), nil, now, + NewMetadata(ident.StringID("bar"), ident.Tags{}, now, sizes[1], &checksums[1], lastRead), - NewMetadata(ident.StringID("bar"), nil, now.Add(2*time.Second), + NewMetadata(ident.StringID("bar"), ident.Tags{}, now.Add(2*time.Second), int64(0), nil, lastRead), } diff --git a/storage/block/types.go b/storage/block/types.go index 0ead919b4e..6e80500afe 100644 --- a/storage/block/types.go +++ b/storage/block/types.go @@ -61,6 +61,9 @@ type FilteredBlocksMetadataIter interface { // Current returns the current id and block metadata Current() (ident.ID, Metadata) + + // Error returns an error if encountered + Err() error } // FetchBlockResult captures the block start time, the readers for the underlying streams, the diff --git a/storage/bootstrap/bootstrapper/fs/options.go b/storage/bootstrap/bootstrapper/fs/options.go index 6f72d525dc..3a9729e8fe 100644 --- a/storage/bootstrap/bootstrapper/fs/options.go +++ b/storage/bootstrap/bootstrapper/fs/options.go @@ -49,7 +49,7 @@ func NewOptions() Options { return pool.NewBytesPool(s, nil) }) bytesPool.Init() - idPool := ident.NewPool(bytesPool, pool.NewObjectPoolOptions()) + idPool := ident.NewPool(bytesPool, ident.PoolOptions{}) return &options{ resultOpts: result.NewOptions(), fsOpts: fs.NewOptions(), diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index 92b26110fb..1aeaddab39 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -349,10 +349,10 @@ func (s *fileSystemSource) markRunResultErrorsAndUnfulfilled( func (s *fileSystemSource) tagsFromTagsIter( iter ident.TagIterator, ) (ident.Tags, error) { - tags := make(ident.Tags, 0, iter.Remaining()) + tags := s.idPool.Tags() for iter.Next() { curr := iter.Current() - tags = append(tags, s.idPool.CloneTag(curr)) + tags.Append(s.idPool.CloneTag(curr)) } return tags, iter.Err() } diff --git a/storage/index/convert/convert.go b/storage/index/convert/convert.go index c3f7de52d2..837fb98e67 100644 --- a/storage/index/convert/convert.go +++ b/storage/index/convert/convert.go @@ -42,8 +42,8 @@ var ( // FromMetric converts the provided metric id+tags into a document. func FromMetric(id ident.ID, tags ident.Tags) (doc.Document, error) { - fields := make([]doc.Field, 0, len(tags)) - for _, tag := range tags { + fields := make([]doc.Field, 0, len(tags.Values())) + for _, tag := range tags.Values() { if bytes.Equal(ReservedFieldNameID, tag.Name.Bytes()) { return doc.Document{}, errUnableToConvertReservedFieldName } diff --git a/storage/index/options.go b/storage/index/options.go index 5c45a5dfbd..ababe3786e 100644 --- a/storage/index/options.go +++ b/storage/index/options.go @@ -53,7 +53,6 @@ type opts struct { idPool ident.Pool bytesPool pool.CheckedBytesPool resultsPool ResultsPool - tagArrayPool TagArrayPool } var undefinedUUIDFn = func() ([]byte, error) { return nil, errIDGenerationDisabled } @@ -61,16 +60,11 @@ var undefinedUUIDFn = func() ([]byte, error) { return nil, errIDGenerationDisabl // NewOptions returns a new index.Options object with default properties. func NewOptions() Options { resultsPool := NewResultsPool(pool.NewObjectPoolOptions()) - tagArrayPool := NewTagArrayPool(TagArrayPoolOpts{ - Capacity: defaultTagArrayPoolInitCapacity, - MaxCapacity: defaultTagArrayPoolMaxCapacity, - }) - tagArrayPool.Init() bytesPool := pool.NewCheckedBytesPool(nil, nil, func(s []pool.Bucket) pool.BytesPool { return pool.NewBytesPool(s, nil) }) bytesPool.Init() - idPool := ident.NewPool(bytesPool, nil) + idPool := ident.NewPool(bytesPool, ident.PoolOptions{}) opts := &opts{ insertMode: defaultIndexInsertMode, clockOpts: clock.NewOptions(), @@ -78,7 +72,6 @@ func NewOptions() Options { memOpts: mem.NewOptions().SetNewUUIDFn(undefinedUUIDFn), bytesPool: bytesPool, idPool: idPool, - tagArrayPool: tagArrayPool, resultsPool: resultsPool, } resultsPool.Init(func() Results { return NewResults(opts) }) @@ -169,13 +162,3 @@ func (o *opts) SetResultsPool(value ResultsPool) Options { func (o *opts) ResultsPool() ResultsPool { return o.resultsPool } - -func (o *opts) SetTagArrayPool(value TagArrayPool) Options { - opts := *o - opts.tagArrayPool = value - return &opts -} - -func (o *opts) TagArrayPool() TagArrayPool { - return o.tagArrayPool -} diff --git a/storage/index/results.go b/storage/index/results.go index f9db83770d..59648f6c18 100644 --- a/storage/index/results.go +++ b/storage/index/results.go @@ -37,9 +37,8 @@ type results struct { size int resultsMap *ResultsMap - idPool ident.Pool - bytesPool pool.CheckedBytesPool - tagArrayPool TagArrayPool + idPool ident.Pool + bytesPool pool.CheckedBytesPool pool ResultsPool } @@ -47,11 +46,10 @@ type results struct { // NewResults returns a new results object. func NewResults(opts Options) Results { return &results{ - resultsMap: newResultsMap(opts.IdentifierPool()), - idPool: opts.IdentifierPool(), - bytesPool: opts.CheckedBytesPool(), - tagArrayPool: opts.TagArrayPool(), - pool: opts.ResultsPool(), + resultsMap: newResultsMap(opts.IdentifierPool()), + idPool: opts.IdentifierPool(), + bytesPool: opts.CheckedBytesPool(), + pool: opts.ResultsPool(), } } @@ -83,10 +81,10 @@ func (r *results) Add(d doc.Document) (added bool, size int, err error) { return added, r.size, nil } -func (r *results) tags(fields doc.Fields) []ident.Tag { - tags := r.tagArrayPool.Get() +func (r *results) tags(fields doc.Fields) ident.Tags { + tags := r.idPool.Tags() for _, f := range fields { - tags = append(tags, ident.Tag{ + tags.Append(ident.Tag{ Name: r.copyBytes(f.Name), Value: r.copyBytes(f.Value), }) @@ -132,7 +130,6 @@ func (r *results) Reset(nsID ident.ID) { for _, entry := range r.resultsMap.Iter() { tags := entry.Value() tags.Finalize() - r.tagArrayPool.Put(tags) } // reset all keys in the map next diff --git a/storage/index/tag_arraypool_gen.go b/storage/index/tag_arraypool_gen.go deleted file mode 100644 index c1b1d2b3ed..0000000000 --- a/storage/index/tag_arraypool_gen.go +++ /dev/null @@ -1,108 +0,0 @@ -// This file was automatically generated by genny. -// Any changes will be lost if this file is regenerated. -// see https://github.com/mauricelam/genny - -package index - -import ( - "github.com/m3db/m3x/ident" - "github.com/m3db/m3x/pool" -) - -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -// TagArrayPool provides a pool for identTag slices. -type TagArrayPool interface { - // Init initializes the array pool, it needs to be called - // before Get/Put use. - Init() - - // Get returns the a slice from the pool. - Get() []ident.Tag - - // Put returns the provided slice to the pool. - Put(elems []ident.Tag) -} - -type TagFinalizeFn func([]ident.Tag) []ident.Tag - -type TagArrayPoolOpts struct { - Options pool.ObjectPoolOptions - Capacity int - MaxCapacity int - FinalizeFn TagFinalizeFn -} - -type TagArrPool struct { - opts TagArrayPoolOpts - pool pool.ObjectPool -} - -func NewTagArrayPool(opts TagArrayPoolOpts) TagArrayPool { - if opts.FinalizeFn == nil { - opts.FinalizeFn = defaultTagFinalizerFn - } - p := pool.NewObjectPool(opts.Options) - return &TagArrPool{opts, p} -} - -func (p *TagArrPool) Init() { - p.pool.Init(func() interface{} { - return make([]ident.Tag, 0, p.opts.Capacity) - }) -} - -func (p *TagArrPool) Get() []ident.Tag { - return p.pool.Get().([]ident.Tag) -} - -func (p *TagArrPool) Put(arr []ident.Tag) { - arr = p.opts.FinalizeFn(arr) - if max := p.opts.MaxCapacity; max > 0 && cap(arr) > max { - return - } - p.pool.Put(arr) -} - -func defaultTagFinalizerFn(elems []ident.Tag) []ident.Tag { - var empty ident.Tag - for i := range elems { - elems[i] = empty - } - elems = elems[:0] - return elems -} - -type TagArr []ident.Tag - -func (elems TagArr) grow(n int) []ident.Tag { - if cap(elems) < n { - elems = make([]ident.Tag, n) - } - elems = elems[:n] - // following compiler optimized memcpy impl - // https://github.com/golang/go/wiki/CompilerOptimizations#optimized-memclr - var empty ident.Tag - for i := range elems { - elems[i] = empty - } - return elems -} diff --git a/storage/index/types.go b/storage/index/types.go index 2e2ef061ca..c96b11000f 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -267,10 +267,4 @@ type Options interface { // ResultsPool returns the results pool. ResultsPool() ResultsPool - - // SetTagArrayPool updates the tag array pool. - SetTagArrayPool(value TagArrayPool) Options - - // TagArrayPool updates the tag array pool. - TagArrayPool() TagArrayPool } diff --git a/storage/options.go b/storage/options.go index 51022c0810..0ac21842d8 100644 --- a/storage/options.go +++ b/storage/options.go @@ -175,15 +175,19 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { contextPool: context.NewPool(context.NewOptions(). SetContextPoolOptions(poolOpts). SetFinalizerPoolOptions(poolOpts)), - seriesCachePolicy: series.DefaultCachePolicy, - seriesOpts: seriesOpts, - seriesPool: series.NewDatabaseSeriesPool(poolOpts), - bytesPool: bytesPool, - encoderPool: encoding.NewEncoderPool(poolOpts), - segmentReaderPool: xio.NewSegmentReaderPool(poolOpts), - readerIteratorPool: encoding.NewReaderIteratorPool(poolOpts), - multiReaderIteratorPool: encoding.NewMultiReaderIteratorPool(poolOpts), - identifierPool: ident.NewPool(bytesPool, poolOpts), + seriesCachePolicy: series.DefaultCachePolicy, + seriesOpts: seriesOpts, + seriesPool: series.NewDatabaseSeriesPool(poolOpts), + bytesPool: bytesPool, + encoderPool: encoding.NewEncoderPool(poolOpts), + segmentReaderPool: xio.NewSegmentReaderPool(poolOpts), + readerIteratorPool: encoding.NewReaderIteratorPool(poolOpts), + multiReaderIteratorPool: encoding.NewMultiReaderIteratorPool(poolOpts), + identifierPool: ident.NewPool(bytesPool, ident.PoolOptions{ + IDPoolOptions: poolOpts, + TagsPoolOptions: poolOpts, + TagsIteratorPoolOptions: poolOpts, + }), fetchBlockMetadataResultsPool: block.NewFetchBlockMetadataResultsPool(poolOpts, 0), fetchBlocksMetadataResultsPool: block.NewFetchBlocksMetadataResultsPool(poolOpts, 0), } diff --git a/storage/repair.go b/storage/repair.go index 0c5e92da0e..e15dbe68ba 100644 --- a/storage/repair.go +++ b/storage/repair.go @@ -115,7 +115,10 @@ func (r shardRepairer) Repair( ctx.RegisterCloser(localMetadata) localIter := block.NewFilteredBlocksMetadataIter(localMetadata) - metadata.AddLocalMetadata(origin, localIter) + err = metadata.AddLocalMetadata(origin, localIter) + if err != nil { + return repair.MetadataComparisonResult{}, err + } // Add peer metadata level := r.rpopts.RepairConsistencyLevel() diff --git a/storage/repair/metadata.go b/storage/repair/metadata.go index d4bb654ec8..625131a266 100644 --- a/storage/repair/metadata.go +++ b/storage/repair/metadata.go @@ -174,7 +174,7 @@ func NewReplicaMetadataComparer(replicas int, opts Options) ReplicaMetadataCompa } } -func (m replicaMetadataComparer) AddLocalMetadata(origin topology.Host, localIter block.FilteredBlocksMetadataIter) { +func (m replicaMetadataComparer) AddLocalMetadata(origin topology.Host, localIter block.FilteredBlocksMetadataIter) error { for localIter.Next() { id, block := localIter.Current() blocks := m.metadata.GetOrAdd(id) @@ -184,6 +184,8 @@ func (m replicaMetadataComparer) AddLocalMetadata(origin topology.Host, localIte Checksum: block.Checksum, }) } + + return localIter.Err() } func (m replicaMetadataComparer) AddPeerMetadata(peerIter client.PeerBlockMetadataIter) error { diff --git a/storage/repair/metadata_test.go b/storage/repair/metadata_test.go index 7a23ca44be..c4b3f90401 100644 --- a/storage/repair/metadata_test.go +++ b/storage/repair/metadata_test.go @@ -146,7 +146,8 @@ func TestReplicaMetadataComparerAddLocalMetadata(t *testing.T) { ) m := NewReplicaMetadataComparer(3, testRepairOptions()).(replicaMetadataComparer) - m.AddLocalMetadata(origin, localIter) + err := m.AddLocalMetadata(origin, localIter) + require.NoError(t, err) expected := []testBlock{ {inputBlocks[0].ID, inputBlocks[0].Start, []HostBlockMetadata{{origin, inputBlocks[0].Size, inputBlocks[0].Checksum}}}, diff --git a/storage/repair/types.go b/storage/repair/types.go index 30d6f78e2b..6ddfcf5eb7 100644 --- a/storage/repair/types.go +++ b/storage/repair/types.go @@ -121,7 +121,7 @@ type ReplicaSeriesBlocksMetadata struct { // ReplicaMetadataComparer compares metadata from hosts in a replica set type ReplicaMetadataComparer interface { // AddLocalMetadata adds metadata from local host - AddLocalMetadata(origin topology.Host, localIter block.FilteredBlocksMetadataIter) + AddLocalMetadata(origin topology.Host, localIter block.FilteredBlocksMetadataIter) error // AddPeerMetadata adds metadata from peers AddPeerMetadata(peerIter client.PeerBlockMetadataIter) error diff --git a/storage/series/options.go b/storage/series/options.go index 21516bf092..f21d007590 100644 --- a/storage/series/options.go +++ b/storage/series/options.go @@ -62,7 +62,7 @@ func NewOptions() Options { encoderPool: encoding.NewEncoderPool(nil), multiReaderIteratorPool: encoding.NewMultiReaderIteratorPool(nil), fetchBlockMetadataResultsPool: block.NewFetchBlockMetadataResultsPool(nil, 0), - identifierPool: ident.NewPool(bytesPool, nil), + identifierPool: ident.NewPool(bytesPool, ident.PoolOptions{}), } } diff --git a/storage/series/series.go b/storage/series/series.go index 206cb3ce80..afb32227b4 100644 --- a/storage/series/series.go +++ b/storage/series/series.go @@ -404,8 +404,9 @@ func (s *dbSeries) FetchBlocksMetadata( // NB(r): Since ID and Tags are garbage collected we can safely // return refs. - return block.NewFetchBlocksMetadataResult(s.id, - ident.NewTagSliceIterator(s.tags), res), nil + tagsIter := s.opts.IdentifierPool().TagsIterator() + tagsIter.Reset(s.tags) + return block.NewFetchBlocksMetadataResult(s.id, tagsIter, res), nil } func (s *dbSeries) bufferDrained(newBlock block.DatabaseBlock) { @@ -676,23 +677,9 @@ func (s *dbSeries) Close() { s.Lock() defer s.Unlock() - // NB(r): We explicitly do not place this ID back into an - // existing pool as high frequency users of series IDs such - // as the commit log need to use the reference without the - // overhead of ownership tracking. In addition, the blocks - // themselves have a reference to the ID which is required - // for the LRU/WiredList caching strategy eviction process. - // Since the wired list can still have a reference to a - // DatabaseBlock for which the corresponding DatabaseSeries - // has been closed, its important that the ID itself is still - // available because the process of kicking a DatabaseBlock - // out of the WiredList requires the ID. - // - // Since series are purged so infrequently the overhead - // of not releasing back an ID to a pool is amortized over - // a long period of time. + // See Reset() for why these aren't finalized s.id = nil - s.tags = nil + s.tags = ident.Tags{} switch s.opts.CachePolicy() { case CacheLRU: @@ -730,8 +717,26 @@ func (s *dbSeries) Reset( s.Lock() defer s.Unlock() + // NB(r): We explicitly do not place this ID back into an + // existing pool as high frequency users of series IDs such + // as the commit log need to use the reference without the + // overhead of ownership tracking. In addition, the blocks + // themselves have a reference to the ID which is required + // for the LRU/WiredList caching strategy eviction process. + // Since the wired list can still have a reference to a + // DatabaseBlock for which the corresponding DatabaseSeries + // has been closed, its important that the ID itself is still + // available because the process of kicking a DatabaseBlock + // out of the WiredList requires the ID. + // + // Since series are purged so infrequently the overhead + // of not releasing back an ID to a pool is amortized over + // a long period of time. s.id = id + s.id.NoFinalize() s.tags = tags + s.tags.NoFinalize() + s.blocks.Reset() s.buffer.Reset(opts) s.opts = opts diff --git a/storage/shard.go b/storage/shard.go index 9cfff469c5..febe5708b0 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -402,15 +402,15 @@ func (s *dbShard) OnRetrieveBlock( // NB(r): Do not need to specify that needs to be indexed as series would // have been already been indexed when it was written copiedID := entry.series.ID() - // TODO(r): Pool the slice iterators here. - copiedTags := ident.NewTagSliceIterator(entry.series.Tags()) + copiedTagsIter := s.identifierPool.TagsIterator() + copiedTagsIter.Reset(entry.series.Tags()) s.insertQueue.Insert(dbShardInsert{ entry: entry, opts: dbShardInsertAsyncOptions{ hasPendingRetrievedBlock: true, pendingRetrievedBlock: dbShardPendingRetrievedBlock{ id: copiedID, - tags: copiedTags, + tags: copiedTagsIter, start: startTime, segment: segment, }, @@ -1005,14 +1005,14 @@ func (s *dbShard) newShardEntry(id ident.ID, tags ident.TagIterator) (*dbShardEn func (s *dbShard) cloneTags(tags ident.TagIterator) (ident.Tags, error) { tags = tags.Duplicate() - clone := make(ident.Tags, 0, tags.Remaining()) + clone := s.identifierPool.Tags() defer tags.Close() for tags.Next() { t := tags.Current() - clone = append(clone, s.identifierPool.CloneTag(t)) + clone.Append(s.identifierPool.CloneTag(t)) } if err := tags.Err(); err != nil { - return nil, err + return ident.Tags{}, err } return clone, nil } @@ -1533,10 +1533,6 @@ func (s *dbShard) FetchBlocksMetadataV2( blockStart, err) } - // Make sure ID and tags get cleaned up after read is done - ctx.RegisterFinalizer(id) - ctx.RegisterCloser(tags) - blockResult := s.opts.FetchBlockMetadataResultsPool().Get() value := block.FetchBlockMetadataResult{ Start: blockStart, @@ -1613,8 +1609,9 @@ func (s *dbShard) Bootstrap( if entry == nil { // Synchronously insert to avoid waiting for // the insert queue potential delayed insert - entry, err = s.insertSeriesSync(dbBlocks.ID, - ident.NewTagSliceIterator(dbBlocks.Tags), + tagsIter := s.identifierPool.TagsIterator() + tagsIter.Reset(dbBlocks.Tags) + entry, err = s.insertSeriesSync(dbBlocks.ID, tagsIter, insertSyncIncReaderWriterCount) if err != nil { multiErr = multiErr.Add(err) @@ -1622,6 +1619,12 @@ func (s *dbShard) Bootstrap( } } + // No longer require ID or tags + dbBlocks.ID.Finalize() + dbBlocks.Tags.Finalize() + + // Cannot close blocks once done as series takes ref to these, + // however we copy ID and Tags bsResult, err := entry.series.Bootstrap(dbBlocks.Blocks) if err != nil { multiErr = multiErr.Add(err) From 555a3fbda32bd0b123afc80cddbee5804087dae8 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 12:44:13 -0400 Subject: [PATCH 08/33] Fix more test build errors --- .../tchannelthrift/node/service_test.go | 22 ++++++------- persist/fs/clone/cloner_test.go | 4 +-- persist/fs/commitlog/commit_log_test.go | 14 ++++---- persist/fs/persist_manager_test.go | 2 +- persist/fs/read_test.go | 12 +++---- persist/fs/read_write_test.go | 20 ++++++----- persist/fs/retriever_concurrent_test.go | 4 +-- persist/fs/seek_test.go | 33 +++++++------------ serialize/bench_test.go | 6 ++-- serialize/decoder_lifecycle_prop_test.go | 15 ++++++--- serialize/encode_decode_prop_test.go | 13 +++++--- serialize/encoder_pool_test.go | 2 +- serialize/encoder_test.go | 14 ++++---- storage/block/result.go | 22 ++++++++----- 14 files changed, 94 insertions(+), 89 deletions(-) diff --git a/network/server/tchannelthrift/node/service_test.go b/network/server/tchannelthrift/node/service_test.go index 8a6f9d00f1..c61f943d4f 100644 --- a/network/server/tchannelthrift/node/service_test.go +++ b/network/server/tchannelthrift/node/service_test.go @@ -509,10 +509,10 @@ func TestServiceFetchBlocksMetadataEndpointV2Raw(t *testing.T) { }{ "foo": { // Check with tags - tags: ident.Tags{ + tags: ident.NewTags( ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd"), - }, + ), data: []testBlock{ {start.Add(0 * time.Hour), 16, 111, time.Now().Add(-time.Minute)}, {start.Add(2 * time.Hour), 32, 222, time.Time{}}, @@ -534,7 +534,7 @@ func TestServiceFetchBlocksMetadataEndpointV2Raw(t *testing.T) { ids = append(ids, []byte(id)) blocks := block.NewFetchBlockMetadataResults() metadata := block.NewFetchBlocksMetadataResult(ident.StringID(id), - ident.NewTagSliceIterator(s.tags), blocks) + ident.NewTagsIterator(s.tags), blocks) for _, v := range s.data { numBlocks++ entry := v @@ -584,14 +584,14 @@ func TestServiceFetchBlocksMetadataEndpointV2Raw(t *testing.T) { expectedBlocks := series[string(block.ID)] - if len(expectedBlocks.tags) == 0 { + if len(expectedBlocks.tags.Values()) == 0 { require.Equal(t, 0, len(block.EncodedTags)) } else { encodedTags := checked.NewBytes(block.EncodedTags, nil) decoder := service.pools.tagDecoder.Get() decoder.Reset(encodedTags) - expectedTags := ident.NewTagSliceIterator(expectedBlocks.tags) + expectedTags := ident.NewTagsIterator(expectedBlocks.tags) require.True(t, ident.NewTagIterMatcher(expectedTags).Matches(decoder)) decoder.Close() @@ -672,14 +672,14 @@ func TestServiceFetchTagged(t *testing.T) { resMap := index.NewResults(index.NewOptions()) resMap.Reset(ident.StringID(nsID)) - resMap.Map().Set(ident.StringID("foo"), ident.Tags{ + resMap.Map().Set(ident.StringID("foo"), ident.NewTags( ident.StringTag("foo", "bar"), ident.StringTag("baz", "dxk"), - }) - resMap.Map().Set(ident.StringID("bar"), ident.Tags{ + )) + resMap.Map().Set(ident.StringID("bar"), ident.NewTags( ident.StringTag("foo", "bar"), ident.StringTag("dzk", "baz"), - }) + )) mockDB.EXPECT().QueryIDs( ctx, @@ -766,8 +766,8 @@ func TestServiceFetchTaggedNoData(t *testing.T) { resMap := index.NewResults(index.NewOptions()) resMap.Reset(ident.StringID(nsID)) - resMap.Map().Set(ident.StringID("foo"), nil) - resMap.Map().Set(ident.StringID("bar"), nil) + resMap.Map().Set(ident.StringID("foo"), ident.Tags{}) + resMap.Map().Set(ident.StringID("bar"), ident.Tags{}) mockDB.EXPECT().QueryIDs( ctx, ident.NewIDMatcher(nsID), diff --git a/persist/fs/clone/cloner_test.go b/persist/fs/clone/cloner_test.go index b76869bf04..91e6c71c73 100644 --- a/persist/fs/clone/cloner_test.go +++ b/persist/fs/clone/cloner_test.go @@ -149,10 +149,10 @@ func writeTestData(t *testing.T, bs time.Duration, src FileSetID, opts Options) id := ident.StringID(fmt.Sprintf("test-series.%d", i)) var tags ident.Tags if i%2 == 0 { - tags = ident.Tags{ + tags = ident.NewTags( ident.StringTag("foo", "bar"), ident.StringTag("qux", "qaz"), - } + ) } require.NoError(t, w.Write(id, tags, testBytes, 1234)) } diff --git a/persist/fs/commitlog/commit_log_test.go b/persist/fs/commitlog/commit_log_test.go index ce239f2b18..dab77b69f1 100644 --- a/persist/fs/commitlog/commit_log_test.go +++ b/persist/fs/commitlog/commit_log_test.go @@ -134,7 +134,7 @@ func (w testWrite) assert( require.True(t, w.series.ID.Equal(series.ID), fmt.Sprintf("write ID '%s' does not match actual ID '%s'", w.series.ID.String(), series.ID.String())) require.Equal(t, w.series.Shard, series.Shard) - require.Equal(t, len(w.series.Tags), len(series.Tags)) + // ident.Tags.Equal will compare length require.True(t, w.series.Tags.Equal(series.Tags)) require.True(t, w.t.Equal(datapoint.Timestamp)) @@ -354,8 +354,8 @@ func TestCommitLogWrite(t *testing.T) { commitLog := newTestCommitLog(t, opts) writes := []testWrite{ - {testSeries(0, "foo.bar", ident.Tags{ident.StringTag("name1", "val1")}, 127), time.Now(), 123.456, xtime.Second, []byte{1, 2, 3}, nil}, - {testSeries(1, "foo.baz", ident.Tags{ident.StringTag("name2", "val2")}, 150), time.Now(), 456.789, xtime.Second, nil, nil}, + {testSeries(0, "foo.bar", ident.NewTags(ident.StringTag("name1", "val1")), 127), time.Now(), 123.456, xtime.Second, []byte{1, 2, 3}, nil}, + {testSeries(1, "foo.baz", ident.NewTags(ident.StringTag("name2", "val2")), 150), time.Now(), 456.789, xtime.Second, nil, nil}, } // Call write sync @@ -396,7 +396,7 @@ func TestReadCommitLogMissingMetadata(t *testing.T) { allSeries = append(allSeries, testSeries( uint64(i), "hax", - ident.Tags{ident.StringTag("name", "val")}, + ident.NewTags(ident.StringTag("name", "val")), uint32(i%100), )) if willNotHaveMetadata { @@ -826,7 +826,7 @@ var ( testTag2 = ident.StringTag("name2", "val2") testTag3 = ident.StringTag("name3", "val3") - testTags1 = ident.Tags{testTag1} - testTags2 = ident.Tags{testTag2} - testTags3 = ident.Tags{testTag3} + testTags1 = ident.NewTags(testTag1) + testTags2 = ident.NewTags(testTag2) + testTags3 = ident.NewTags(testTag3) ) diff --git a/persist/fs/persist_manager_test.go b/persist/fs/persist_manager_test.go index eb35a3dcbc..0a332a3a4c 100644 --- a/persist/fs/persist_manager_test.go +++ b/persist/fs/persist_manager_test.go @@ -353,7 +353,7 @@ func TestPersistenceManagerWithRateLimit(t *testing.T) { BlockSize: testBlockSize, } writer.EXPECT().Open(writerOpts).Return(nil).Times(iter) - writer.EXPECT().WriteAll(id, nil, pm.segmentHolder, checksum).Return(nil).AnyTimes() + writer.EXPECT().WriteAll(id, ident.Tags{}, pm.segmentHolder, checksum).Return(nil).AnyTimes() writer.EXPECT().Close().Times(iter) // Enable rate limiting diff --git a/persist/fs/read_test.go b/persist/fs/read_test.go index ff8e7a99d7..14ce30436e 100644 --- a/persist/fs/read_test.go +++ b/persist/fs/read_test.go @@ -170,8 +170,7 @@ func TestReadDataError(t *testing.T) { err = w.Open(writerOpts) require.NoError(t, err) require.NoError(t, w.Write( - ident.StringID("foo"), - nil, + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) require.NoError(t, w.Close()) @@ -226,8 +225,7 @@ func TestReadDataUnexpectedSize(t *testing.T) { dataFile := w.(*writer).dataFdWithDigest.Fd().Name() assert.NoError(t, w.Write( - ident.StringID("foo"), - nil, + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -308,8 +306,7 @@ func testReadOpen(t *testing.T, fileData map[string][]byte) { assert.NoError(t, w.Open(writerOpts)) assert.NoError(t, w.Write( - ident.StringID("foo"), - nil, + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{0x1}), digest.Checksum([]byte{0x1}))) assert.NoError(t, w.Close()) @@ -402,8 +399,7 @@ func TestReadValidate(t *testing.T) { require.NoError(t, w.Open(writerOpts)) assert.NoError(t, w.Write( - ident.StringID("foo"), - nil, + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{0x1}), digest.Checksum([]byte{0x1}))) require.NoError(t, w.Close()) diff --git a/persist/fs/read_write_test.go b/persist/fs/read_write_test.go index c810f0a556..1a30817e0b 100644 --- a/persist/fs/read_write_test.go +++ b/persist/fs/read_write_test.go @@ -51,7 +51,7 @@ func (e testEntry) ID() ident.ID { func (e testEntry) Tags() ident.Tags { if e.tags == nil { - return nil + return ident.Tags{} } // Return in sorted order for deterministic order @@ -63,7 +63,7 @@ func (e testEntry) Tags() ident.Tags { var tags ident.Tags for _, key := range keys { - tags = append(tags, ident.StringTag(key, e.tags[key])) + tags.Append(ident.StringTag(key, e.tags[key])) } return tags @@ -157,7 +157,7 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim assert.Equal(t, entries[i].id, id.String()) // Assert tags - tagMatcher := ident.NewTagIterMatcher(ident.NewTagSliceIterator(entries[i].Tags())) + tagMatcher := ident.NewTagIterMatcher(ident.NewTagsIterator(entries[i].Tags())) assert.True(t, tagMatcher.Matches(tags)) assert.True(t, bytes.Equal(entries[i].data, data.Bytes())) @@ -181,7 +181,7 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim assert.True(t, id.Equal(id)) // Assert tags - tagMatcher := ident.NewTagIterMatcher(ident.NewTagSliceIterator(entries[i].Tags())) + tagMatcher := ident.NewTagIterMatcher(ident.NewTagsIterator(entries[i].Tags())) assert.True(t, tagMatcher.Matches(tags)) assert.Equal(t, digest.Checksum(entries[i].data), checksum) @@ -411,11 +411,13 @@ func TestWriterOnlyWritesNonNilBytes(t *testing.T) { } require.NoError(t, w.Open(writerOpts)) - w.WriteAll(ident.StringID("foo"), nil, []checked.Bytes{ - checkedBytes([]byte{1, 2, 3}), - nil, - checkedBytes([]byte{4, 5, 6}), - }, digest.Checksum([]byte{1, 2, 3, 4, 5, 6})) + w.WriteAll(ident.StringID("foo"), ident.Tags{}, + []checked.Bytes{ + checkedBytes([]byte{1, 2, 3}), + nil, + checkedBytes([]byte{4, 5, 6}), + }, + digest.Checksum([]byte{1, 2, 3, 4, 5, 6})) assert.NoError(t, w.Close()) diff --git a/persist/fs/retriever_concurrent_test.go b/persist/fs/retriever_concurrent_test.go index 177c5f5bba..d99fc630dd 100644 --- a/persist/fs/retriever_concurrent_test.go +++ b/persist/fs/retriever_concurrent_test.go @@ -182,7 +182,7 @@ func testBlockRetrieverHighConcurrentSeeks(t *testing.T, shouldCacheShardIndices } shardData[shard][idString][xtime.ToUnixNano(blockStart)] = data - err := w.Write(id, nil, data, digest.Checksum(data.Bytes())) + err := w.Write(id, ident.Tags{}, data, digest.Checksum(data.Bytes())) require.NoError(t, err) } closer() @@ -290,7 +290,7 @@ func TestBlockRetrieverIDDoesNotExist(t *testing.T) { data := checked.NewBytes([]byte("Hello world!"), nil) data.IncRef() defer data.DecRef() - err = w.Write(ident.StringID("exists"), nil, data, digest.Checksum(data.Bytes())) + err = w.Write(ident.StringID("exists"), ident.Tags{}, data, digest.Checksum(data.Bytes())) assert.NoError(t, err) closer() diff --git a/persist/fs/seek_test.go b/persist/fs/seek_test.go index 1185341033..b2997d4891 100644 --- a/persist/fs/seek_test.go +++ b/persist/fs/seek_test.go @@ -100,8 +100,7 @@ func TestSeekDataUnexpectedSize(t *testing.T) { dataFile := w.(*writer).dataFdWithDigest.Fd().Name() assert.NoError(t, w.Write( - ident.StringID("foo"), - ident.Tags(nil), + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -142,8 +141,7 @@ func TestSeekBadChecksum(t *testing.T) { // Write data with wrong checksum assert.NoError(t, w.Write( - ident.StringID("foo"), - ident.Tags(nil), + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 4}))) assert.NoError(t, w.Close()) @@ -182,17 +180,17 @@ func TestSeek(t *testing.T) { assert.NoError(t, err) assert.NoError(t, w.Write( ident.StringID("foo1"), - ident.Tags{ident.StringTag("num", "1")}, + ident.NewTags(ident.StringTag("num", "1")), bytesRefd([]byte{1, 2, 1}), digest.Checksum([]byte{1, 2, 1}))) assert.NoError(t, w.Write( ident.StringID("foo2"), - ident.Tags{ident.StringTag("num", "2")}, + ident.NewTags(ident.StringTag("num", "2")), bytesRefd([]byte{1, 2, 2}), digest.Checksum([]byte{1, 2, 2}))) assert.NoError(t, w.Write( ident.StringID("foo3"), - ident.Tags{ident.StringTag("num", "3")}, + ident.NewTags(ident.StringTag("num", "3")), bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -251,18 +249,15 @@ func TestSeekIDNotExists(t *testing.T) { err = w.Open(writerOpts) assert.NoError(t, err) assert.NoError(t, w.Write( - ident.StringID("foo10"), - ident.Tags(nil), + ident.StringID("foo10"), ident.Tags{}, bytesRefd([]byte{1, 2, 1}), digest.Checksum([]byte{1, 2, 1}))) assert.NoError(t, w.Write( - ident.StringID("foo20"), - ident.Tags(nil), + ident.StringID("foo20"), ident.Tags{}, bytesRefd([]byte{1, 2, 2}), digest.Checksum([]byte{1, 2, 2}))) assert.NoError(t, w.Write( - ident.StringID("foo30"), - ident.Tags(nil), + ident.StringID("foo30"), ident.Tags{}, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -307,8 +302,7 @@ func TestReuseSeeker(t *testing.T) { err = w.Open(writerOpts) assert.NoError(t, err) assert.NoError(t, w.Write( - ident.StringID("foo"), - ident.Tags(nil), + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{1, 2, 1}), digest.Checksum([]byte{1, 2, 1}))) assert.NoError(t, w.Close()) @@ -324,8 +318,7 @@ func TestReuseSeeker(t *testing.T) { err = w.Open(writerOpts) assert.NoError(t, err) assert.NoError(t, w.Write( - ident.StringID("foo"), - ident.Tags(nil), + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -373,8 +366,7 @@ func TestCloneSeeker(t *testing.T) { err = w.Open(writerOpts) assert.NoError(t, err) assert.NoError(t, w.Write( - ident.StringID("foo"), - ident.Tags(nil), + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{1, 2, 1}), digest.Checksum([]byte{1, 2, 1}))) assert.NoError(t, w.Close()) @@ -390,8 +382,7 @@ func TestCloneSeeker(t *testing.T) { err = w.Open(writerOpts) assert.NoError(t, err) assert.NoError(t, w.Write( - ident.StringID("foo"), - ident.Tags(nil), + ident.StringID("foo"), ident.Tags{}, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) diff --git a/serialize/bench_test.go b/serialize/bench_test.go index a4c2a5cc15..1277048d5d 100644 --- a/serialize/bench_test.go +++ b/serialize/bench_test.go @@ -27,15 +27,15 @@ import ( ) func benchmarkTags() ident.Tags { - return ident.Tags{ + return ident.NewTags( ident.StringTag("abc", "Def"), ident.StringTag("ghifsdf", "andson"), - } + ) } func BenchmarkCustomReadWrite(b *testing.B) { tags := benchmarkTags() - iter := ident.NewTagSliceIterator(tags) + iter := ident.NewTagsIterator(tags) enc := newTagEncoder(defaultNewCheckedBytesFn, newTestEncoderOpts(), nil) dec := newTagDecoder(testDecodeOpts, nil) diff --git a/serialize/decoder_lifecycle_prop_test.go b/serialize/decoder_lifecycle_prop_test.go index 041077dd1e..f2bf82c99c 100644 --- a/serialize/decoder_lifecycle_prop_test.go +++ b/serialize/decoder_lifecycle_prop_test.go @@ -183,7 +183,7 @@ var currentCmd = &commands.ProtoCommand{ return &gopter.PropResult{Status: gopter.PropTrue} } observedTag := res - expectedTag := decState.tags[decState.primary.numNextCalls-1] + expectedTag := decState.tags.Values()[decState.primary.numNextCalls-1] if !observedTag.Name.Equal(expectedTag.Name) || !observedTag.Value.Equal(expectedTag.Value) { return &gopter.PropResult{ @@ -424,7 +424,7 @@ func newDecoderState() gopter.Gen { return anyASCIITags().Map( func(tags ident.Tags) *multiDecoderState { enc := newTestTagEncoder() - if err := enc.Encode(ident.NewTagSliceIterator(tags)); err != nil { + if err := enc.Encode(ident.NewTagsIterator(tags)); err != nil { return nil } b, ok := enc.Data() @@ -437,7 +437,7 @@ func newDecoderState() gopter.Gen { initBytes: data, numRefs: 1, primary: decoderState{ - numTags: len(tags), + numTags: len(tags.Values()), }, } }, @@ -453,7 +453,12 @@ func anyASCIITag() gopter.Gen { }) } -func anyASCIITags() gopter.Gen { return gen.SliceOf(anyASCIITag()) } +func anyASCIITags() gopter.Gen { + return gen.SliceOf(anyASCIITag()). + Map(func(tags []ident.Tag) ident.Tags { + return ident.NewTags(tags...) + }) +} func (d decoderState) String() string { return fmt.Sprintf("[ numTags=%d, closed=%v, numNextCalls=%d ]", @@ -494,7 +499,7 @@ func (d *multiDecoderState) String() string { func tagsToString(tags ident.Tags) string { var tagBuffer bytes.Buffer - for i, t := range tags { + for i, t := range tags.Values() { if i != 0 { tagBuffer.WriteString(", ") } diff --git a/serialize/encode_decode_prop_test.go b/serialize/encode_decode_prop_test.go index 73886d6ed6..8493baf2fa 100644 --- a/serialize/encode_decode_prop_test.go +++ b/serialize/encode_decode_prop_test.go @@ -45,7 +45,7 @@ func TestPropertySerializationBijective(t *testing.T) { properties := gopter.NewProperties(testParams) properties.Property("serialization is bijiective", prop.ForAll( func(x string) (bool, error) { - tags := ident.NewTagIterator(ident.StringTag(x, "")) + tags := ident.NewTagsIterator(ident.NewTags(ident.StringTag(x, ""))) copy, err := encodeAndDecode(tags) if err != nil { return false, err @@ -61,7 +61,7 @@ func TestPropertyAnyStringsDontCollide(t *testing.T) { properties := gopter.NewProperties(testParams) properties.Property("no collisions during string concat", prop.ForAll( func(tag ident.Tag) (bool, error) { - tags := ident.NewTagIterator(tag) + tags := ident.NewTagsIterator(ident.NewTags(tag)) copy, err := encodeAndDecode(tags) if err != nil { return false, err @@ -77,7 +77,7 @@ func TestPropertyAnyReasonableTagSlicesAreAight(t *testing.T) { properties := gopter.NewProperties(testParams) properties.Property("tags of reasonable length are handled fine", prop.ForAll( func(tags ident.Tags) (bool, error) { - iter := ident.NewTagSliceIterator(tags) + iter := ident.NewTagsIterator(tags) copy, err := encodeAndDecode(iter) if err != nil { return false, err @@ -161,4 +161,9 @@ func anyTag() gopter.Gen { }) } -func anyTags() gopter.Gen { return gen.SliceOf(anyTag()) } +func anyTags() gopter.Gen { + return gen.SliceOf(anyTag()). + Map(func(tags []ident.Tag) ident.Tags { + return ident.NewTags(tags...) + }) +} diff --git a/serialize/encoder_pool_test.go b/serialize/encoder_pool_test.go index 23faabf0b9..1a2c26990c 100644 --- a/serialize/encoder_pool_test.go +++ b/serialize/encoder_pool_test.go @@ -40,7 +40,7 @@ func TestTagEncoderPool(t *testing.T) { e.Reset() require.NoError(t, e.Encode( - ident.NewTagIterator(ident.StringTag("hey", "jude")))) + ident.NewTagsIterator(ident.NewTags(ident.StringTag("hey", "jude"))))) e.Finalize() } diff --git a/serialize/encoder_test.go b/serialize/encoder_test.go index d4dfe418d1..54a5176c92 100644 --- a/serialize/encoder_test.go +++ b/serialize/encoder_test.go @@ -68,10 +68,10 @@ func TestInuseEncode(t *testing.T) { func TestTagEncoderLeavesOriginalIterator(t *testing.T) { e := newTestTagEncoder() - tags := ident.NewTagIterator( + tags := ident.NewTagsIterator(ident.NewTags( ident.StringTag("abc", "defg"), ident.StringTag("x", "bar"), - ) + )) require.Equal(t, 2, tags.Remaining()) require.NoError(t, e.Encode(tags)) @@ -81,10 +81,10 @@ func TestTagEncoderLeavesOriginalIterator(t *testing.T) { func TestSimpleEncode(t *testing.T) { e := newTestTagEncoder() - tags := ident.NewTagIterator( + tags := ident.NewTagsIterator(ident.NewTags( ident.StringTag("abc", "defg"), ident.StringTag("x", "bar"), - ) + )) require.NoError(t, e.Encode(tags)) bc, ok := e.Data() @@ -113,10 +113,10 @@ func TestTagEncoderErrorEncoding(t *testing.T) { opts := NewTagEncoderOptions() e := newTagEncoder(defaultNewCheckedBytesFn, opts, nil) maxLiteralLen := opts.TagSerializationLimits().MaxTagLiteralLength() - tags := ident.NewTagIterator( + tags := ident.NewTagsIterator(ident.NewTags( ident.StringTag("abc", "defg"), ident.StringTag("x", nstring(1+int(maxLiteralLen))), - ) + )) require.Error(t, e.Encode(tags)) d, ok := e.Data() @@ -124,7 +124,7 @@ func TestTagEncoderErrorEncoding(t *testing.T) { require.False(t, ok) e.Reset() - tags = ident.NewTagIterator(ident.StringTag("abc", "defg")) + tags = ident.NewTagsIterator(ident.NewTags(ident.StringTag("abc", "defg"))) require.NoError(t, e.Encode(tags)) } diff --git a/storage/block/result.go b/storage/block/result.go index 6b5a5d7939..885a2d2b89 100644 --- a/storage/block/result.go +++ b/storage/block/result.go @@ -164,12 +164,18 @@ func (s *fetchBlocksMetadataResults) Reset() { func (s *fetchBlocksMetadataResults) Close() { for i := range s.results { - s.results[i].ID.Finalize() - s.results[i].ID = nil - s.results[i].Tags.Close() - s.results[i].Tags = nil - s.results[i].Blocks.Close() - s.results[i].Blocks = nil + if s.results[i].ID != nil { + s.results[i].ID.Finalize() + s.results[i].ID = nil + } + if s.results[i].Tags != nil { + s.results[i].Tags.Close() + s.results[i].Tags = nil + } + if s.results[i].Blocks != nil { + s.results[i].Blocks.Close() + s.results[i].Blocks = nil + } } if s.pool != nil { s.pool.Put(s) @@ -205,8 +211,8 @@ func (it *filteredBlocksMetadataIter) Next() bool { for it.blockIdx < len(blocks) { block := blocks[it.blockIdx] if block.Err != nil { - it.err = block.Err - return false + it.blockIdx++ + continue } break } From a3307fb09334aabe11be57862257605e8d8f4640 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 12:47:24 -0400 Subject: [PATCH 09/33] Revert unnecessary glide.lock changes --- glide.lock | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/glide.lock b/glide.lock index fcfd22d688..c5ef5a767f 100644 --- a/glide.lock +++ b/glide.lock @@ -112,21 +112,19 @@ imports: - spew - name: github.com/dgrijalva/jwt-go version: d2709f9f1f31ebcda9651b03077758c1f3a0018c -- name: github.com/dgryski/go-bits - version: 2ad8d707cc05b1815ce6ff2543bb5e8d8f9298ef - name: github.com/ghodss/yaml version: 0ca9ea5df5451ffdf184b4428c902747c2c11cd7 - name: github.com/glycerine/go-unsnap-stream version: 62a9a9eb44fd8932157b1a8ace2149eff5971af6 - name: github.com/go-kit/kit - version: 04dd4f741c6e76cc170a4d7913f4c625952e6f58 + version: 91c58cc7d6d770b0285fdd69949a23d817a57d13 subpackages: - log - log/level - name: github.com/go-logfmt/logfmt version: 390ab7935ee28ec6b286364bba9b4dd6410cb3d5 - name: github.com/go-stack/stack - version: 54be5f394ed2c3e19dac9134a40a95ba5a017f7b + version: 259ab82a6cad3992b4e21ff5cac294ccb06474bc - name: github.com/gogo/protobuf version: 100ba4e885062801d56799d78530b73b178a78f3 subpackages: @@ -155,7 +153,7 @@ imports: - name: github.com/gorilla/context version: 08b5f424b9271eedf6f9f0ce86cb9396ed337a42 - name: github.com/gorilla/mux - version: e3702bed27f0d39777b0b37b664b6280e8ef8fbf + version: 53c1911da2b537f792e7cafcb446b05ffe33b996 - name: github.com/grpc-ecosystem/go-grpc-prometheus version: 6b7015e65d366bf3f19b2b2a000a831940f0f7e0 - name: github.com/grpc-ecosystem/grpc-gateway @@ -272,11 +270,11 @@ imports: - name: github.com/mschoch/smat version: 90eadee771aeab36e8bf796039b8c261bebebe4f - name: github.com/nightlyone/lockfile - version: 1d49c987357a327b5b03aa84cbddd582c328615d + version: 6a197d5ea61168f2ac821de2b7f011b250904900 - name: github.com/nu7hatch/gouuid version: 179d4d0c4d8d407a32af483c2354df1d2c91e6c3 - name: github.com/oklog/ulid - version: 66bb6560562feca7045b23db1ae85b01260f87c5 + version: 7e1bd8a9a281b5abdf651a430b918bb0ecdbceda - name: github.com/opentracing/opentracing-go version: 855519783f479520497c6b3445611b05fc42f009 subpackages: @@ -288,7 +286,7 @@ imports: - name: github.com/philhofer/fwd version: bb6d471dc95d4fe11e432687f8b70ff496cf3136 - name: github.com/pkg/errors - version: 248dadf4e9068a0b3e79f02ed0a610d935de5302 + version: 816c9085562cd7ee03e7f8188a1cfd942858cded - name: github.com/pkg/profile version: 5b67d428864e92711fcbd2f8629456121a56d91f - name: github.com/pmezard/go-difflib @@ -296,7 +294,7 @@ imports: subpackages: - difflib - name: github.com/prometheus/client_golang - version: 967789050ba94deca04a5e84cce8ad472ce313c1 + version: c5b7fccd204277076155f10851dad72b76a49317 subpackages: - prometheus - name: github.com/prometheus/client_model @@ -325,10 +323,12 @@ imports: - util/strutil - util/testutil - name: github.com/prometheus/tsdb - version: 706602daed1487f7849990678b4ece4599745905 + version: def6e5a57439cffe7b44a619c05bce4ac513a63e subpackages: + - chunkenc - chunks - fileutil + - index - labels - name: github.com/RoaringBitmap/roaring version: 4bbba8874933b985c9160558fddea525dde73e63 @@ -420,7 +420,7 @@ imports: - lex/httplex - trace - name: golang.org/x/sync - version: 450f422ab23cf9881c94e2db30cac0eb1b7cf80c + version: fd80eb99c8f653c847d294a001bdf2a3a6f768f5 subpackages: - errgroup - name: golang.org/x/sys From 60b3d012a6da36381d63cbecd6f0f54fb35a0ab6 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 12:53:07 -0400 Subject: [PATCH 10/33] Fix integration tests build errors --- integration/commitlog_bootstrap_index_test.go | 8 ++++---- integration/fetch_tagged_quorum_test.go | 2 +- integration/fs_bootstrap_index_test.go | 6 +++--- integration/fs_bootstrap_tags_test.go | 8 ++++---- integration/index_multiple_node_high_concurrency_test.go | 2 +- integration/peers_bootstrap_index_test.go | 6 +++--- integration/write_tagged_quorum_test.go | 4 ++-- 7 files changed, 18 insertions(+), 18 deletions(-) diff --git a/integration/commitlog_bootstrap_index_test.go b/integration/commitlog_bootstrap_index_test.go index d20a72b90e..5f913a53ce 100644 --- a/integration/commitlog_bootstrap_index_test.go +++ b/integration/commitlog_bootstrap_index_test.go @@ -81,17 +81,17 @@ func TestCommitLogIndexBootstrap(t *testing.T) { now := setup.getNowFn() fooSeries := generate.Series{ ID: ident.StringID("foo"), - Tags: ident.Tags{ident.StringTag("city", "new_york"), ident.StringTag("foo", "foo")}, + Tags: ident.NewTags(ident.StringTag("city", "new_york"), ident.StringTag("foo", "foo")), } barSeries := generate.Series{ ID: ident.StringID("bar"), - Tags: ident.Tags{ident.StringTag("city", "new_jersey")}, + Tags: ident.NewTags(ident.StringTag("city", "new_jersey")), } bazSeries := generate.Series{ ID: ident.StringID("baz"), - Tags: ident.Tags{ident.StringTag("city", "seattle")}, + Tags: ident.NewTags(ident.StringTag("city", "seattle")), } unindexedSeries := generate.Series{ @@ -125,7 +125,7 @@ func TestCommitLogIndexBootstrap(t *testing.T) { }, { IDs: []string{unindexedSeries.ID.String()}, - Tags: nil, + Tags: ident.Tags{}, NumPoints: 1, Start: now, }, diff --git a/integration/fetch_tagged_quorum_test.go b/integration/fetch_tagged_quorum_test.go index 28c91ed9e4..542f3ba238 100644 --- a/integration/fetch_tagged_quorum_test.go +++ b/integration/fetch_tagged_quorum_test.go @@ -295,7 +295,7 @@ func startAndWriteTagged( for _, n := range nodes { require.NoError(t, n.startServer()) require.NoError(t, n.db.WriteTagged(ctx, testNamespaces[0], ident.StringID("quorumTest"), - ident.NewTagIterator(ident.StringTag("foo", "bar"), ident.StringTag("boo", "baz")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("foo", "bar"), ident.StringTag("boo", "baz"))), n.getNowFn(), 42, xtime.Second, nil)) } } diff --git a/integration/fs_bootstrap_index_test.go b/integration/fs_bootstrap_index_test.go index 9f8e7f8531..d1c4322ed3 100644 --- a/integration/fs_bootstrap_index_test.go +++ b/integration/fs_bootstrap_index_test.go @@ -86,17 +86,17 @@ func TestFilesystemBootstrapIndexWithIndexingEnabled(t *testing.T) { fooSeries := generate.Series{ ID: ident.StringID("foo"), - Tags: ident.Tags{ident.StringTag("city", "new_york"), ident.StringTag("foo", "foo")}, + Tags: ident.NewTags(ident.StringTag("city", "new_york"), ident.StringTag("foo", "foo")), } barSeries := generate.Series{ ID: ident.StringID("bar"), - Tags: ident.Tags{ident.StringTag("city", "new_jersey")}, + Tags: ident.NewTags(ident.StringTag("city", "new_jersey")), } bazSeries := generate.Series{ ID: ident.StringID("baz"), - Tags: ident.Tags{ident.StringTag("city", "seattle")}, + Tags: ident.NewTags(ident.StringTag("city", "seattle")), } seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ diff --git a/integration/fs_bootstrap_tags_test.go b/integration/fs_bootstrap_tags_test.go index aa988d5011..24030efd4d 100644 --- a/integration/fs_bootstrap_tags_test.go +++ b/integration/fs_bootstrap_tags_test.go @@ -84,25 +84,25 @@ func TestFilesystemBootstrapTagsWithIndexingDisabled(t *testing.T) { seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ { IDs: []string{"foo"}, - Tags: ident.Tags{ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd")}, + Tags: ident.NewTags(ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd")), NumPoints: 100, Start: now.Add(-blockSize), }, { IDs: []string{"bar"}, - Tags: ident.Tags{ident.StringTag("eee", "fff")}, + Tags: ident.NewTags(ident.StringTag("eee", "fff")), NumPoints: 100, Start: now.Add(-blockSize), }, { IDs: []string{"foo"}, - Tags: ident.Tags{ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd")}, + Tags: ident.NewTags(ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd")), NumPoints: 50, Start: now, }, { IDs: []string{"baz"}, - Tags: ident.Tags{ident.StringTag("ggg", "hhh")}, + Tags: ident.NewTags(ident.StringTag("ggg", "hhh")), NumPoints: 50, Start: now, }, diff --git a/integration/index_multiple_node_high_concurrency_test.go b/integration/index_multiple_node_high_concurrency_test.go index 743001094b..e2bcaba591 100644 --- a/integration/index_multiple_node_high_concurrency_test.go +++ b/integration/index_multiple_node_high_concurrency_test.go @@ -185,5 +185,5 @@ func genIDTags(i int, j int, numTags int) (ident.ID, ident.TagIterator) { tags = append(tags, ident.StringTag("commoni", fmt.Sprintf("%d", i)), ident.StringTag("shared", "shared")) - return ident.StringID(id), ident.NewTagSliceIterator(tags) + return ident.StringID(id), ident.NewTagsIterator(ident.NewTags(tags...)) } diff --git a/integration/peers_bootstrap_index_test.go b/integration/peers_bootstrap_index_test.go index 5a96096c94..02e1a88476 100644 --- a/integration/peers_bootstrap_index_test.go +++ b/integration/peers_bootstrap_index_test.go @@ -79,17 +79,17 @@ func TestPeersBootstrapIndexWithIndexingEnabled(t *testing.T) { fooSeries := generate.Series{ ID: ident.StringID("foo"), - Tags: ident.Tags{ident.StringTag("city", "new_york"), ident.StringTag("foo", "foo")}, + Tags: ident.NewTags(ident.StringTag("city", "new_york"), ident.StringTag("foo", "foo")), } barSeries := generate.Series{ ID: ident.StringID("bar"), - Tags: ident.Tags{ident.StringTag("city", "new_jersey")}, + Tags: ident.NewTags(ident.StringTag("city", "new_jersey")), } bazSeries := generate.Series{ ID: ident.StringID("baz"), - Tags: ident.Tags{ident.StringTag("city", "seattle")}, + Tags: ident.NewTags(ident.StringTag("city", "seattle")), } seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ diff --git a/integration/write_tagged_quorum_test.go b/integration/write_tagged_quorum_test.go index 7fcd9f2dd5..a5b51972ad 100644 --- a/integration/write_tagged_quorum_test.go +++ b/integration/write_tagged_quorum_test.go @@ -246,7 +246,7 @@ func makeTestWriteTagged( now := nodes[0].getNowFn().Add(time.Minute) return s.WriteTagged(testNamespaces[0], ident.StringID("quorumTest"), - ident.NewTagIterator(ident.StringTag("foo", "bar"), ident.StringTag("boo", "baz")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("foo", "bar"), ident.StringTag("boo", "baz"))), now, 42, xtime.Second, nil) } @@ -284,7 +284,7 @@ func nodeHasTaggedWrite(t *testing.T, s *testSetup) bool { require.Equal(t, testNamespaces[0].String(), results.Namespace().String()) tags, ok := results.Map().Get(ident.StringID("quorumTest")) idxFound := ok && ident.NewTagIterMatcher(ident.MustNewTagStringsIterator( - "foo", "bar", "boo", "baz")).Matches(ident.NewTagSliceIterator(tags)) + "foo", "bar", "boo", "baz")).Matches(ident.NewTagsIterator(tags)) if !idxFound { return false From 99af4ef99fa951769fd2a9cdfb447d75e2522b88 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 13:16:14 -0400 Subject: [PATCH 11/33] Fix further test build errors --- .gitignore | 4 ++ storage/block/block_mock.go | 12 ++++++ storage/block/result.go | 21 +++++----- storage/bootstrap/bootstrapper/base_test.go | 4 +- .../commitlog/source_index_test.go | 16 ++++---- .../commitlog/source_prop_test.go | 2 +- .../bootstrapper/fs/source_data_test.go | 2 +- .../bootstrapper/peers/source_data_test.go | 10 ++--- .../bootstrapper/peers/source_index_test.go | 4 +- storage/bootstrap/result/result_data_test.go | 38 +++++++++---------- storage/index/convert/convert_test.go | 22 +++++------ storage/index_block_test.go | 4 +- storage/index_queue_test.go | 26 ++++++------- storage/repair/metadata_test.go | 15 ++++---- storage/repair_test.go | 6 +-- storage/shard_fetch_blocks_metadata_test.go | 20 +++++----- storage/shard_index_test.go | 38 +++++++++---------- storage/shard_test.go | 9 ++--- 18 files changed, 135 insertions(+), 118 deletions(-) diff --git a/.gitignore b/.gitignore index 79283736f4..c7923edaff 100644 --- a/.gitignore +++ b/.gitignore @@ -24,3 +24,7 @@ debug.test # Test data test-data + +# Coverage imports +coverage_imports.go + diff --git a/storage/block/block_mock.go b/storage/block/block_mock.go index 4b2891a70c..2a93106500 100644 --- a/storage/block/block_mock.go +++ b/storage/block/block_mock.go @@ -90,6 +90,18 @@ func (mr *MockFilteredBlocksMetadataIterMockRecorder) Current() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockFilteredBlocksMetadataIter)(nil).Current)) } +// Err mocks base method +func (m *MockFilteredBlocksMetadataIter) Err() error { + ret := m.ctrl.Call(m, "Err") + ret0, _ := ret[0].(error) + return ret0 +} + +// Err indicates an expected call of Err +func (mr *MockFilteredBlocksMetadataIterMockRecorder) Err() *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockFilteredBlocksMetadataIter)(nil).Err)) +} + // MockFetchBlockMetadataResults is a mock of FetchBlockMetadataResults interface type MockFetchBlockMetadataResults struct { ctrl *gomock.Controller diff --git a/storage/block/result.go b/storage/block/result.go index 885a2d2b89..eb9f3f3f52 100644 --- a/storage/block/result.go +++ b/storage/block/result.go @@ -223,17 +223,18 @@ func (it *filteredBlocksMetadataIter) Next() bool { } it.id = it.res[it.resIdx].ID block := blocks[it.blockIdx] - tags := ident.NewTags() - tagsIter := it.res[it.resIdx].Tags - for tagsIter.Next() { - curr := tagsIter.Current() - tags.Append(ident.StringTag(curr.Name.String(), curr.Value.String())) - } - if err := tagsIter.Err(); err != nil { - it.err = err - return false + var tags ident.Tags + if tagsIter := it.res[it.resIdx].Tags; tagsIter != nil { + for tagsIter.Next() { + curr := tagsIter.Current() + tags.Append(ident.StringTag(curr.Name.String(), curr.Value.String())) + } + if err := tagsIter.Err(); err != nil { + it.err = err + return false + } + tagsIter.Close() } - tagsIter.Close() it.metadata = NewMetadata(it.id, tags, block.Start, block.Size, block.Checksum, block.LastRead) it.blockIdx++ diff --git a/storage/bootstrap/bootstrapper/base_test.go b/storage/bootstrap/bootstrapper/base_test.go index f80b7c8a80..e6200226c6 100644 --- a/storage/bootstrap/bootstrapper/base_test.go +++ b/storage/bootstrap/bootstrapper/base_test.go @@ -91,9 +91,9 @@ func shardResult(entries ...testBlockEntry) result.ShardResult { if len(entry.tags)%2 != 0 { panic(fmt.Sprintf("entry tags must be of even length: %v", entry.tags)) } - tags := make(ident.Tags, 0, len(entry.tags)) + tags := ident.NewTags() for idx := 0; idx < len(entry.tags); idx += 2 { - tags = append(tags, ident.StringTag(entry.tags[idx], entry.tags[idx+1])) + tags.Append(ident.StringTag(entry.tags[idx], entry.tags[idx+1])) } res.AddBlock(ident.StringID(entry.id), tags, block) } diff --git a/storage/bootstrap/bootstrapper/commitlog/source_index_test.go b/storage/bootstrap/bootstrapper/commitlog/source_index_test.go index 4e2575ad85..c035f202d8 100644 --- a/storage/bootstrap/bootstrapper/commitlog/source_index_test.go +++ b/storage/bootstrap/bootstrapper/commitlog/source_index_test.go @@ -59,23 +59,23 @@ func TestBootstrapIndex(t *testing.T) { now := time.Now() start := now.Truncate(indexBlockSize) - fooTags := ident.Tags{ident.StringTag("city", "ny"), ident.StringTag("conference", "monitoroma")} - barTags := ident.Tags{ident.StringTag("city", "sf")} - bazTags := ident.Tags{ident.StringTag("city", "oakland")} + fooTags := ident.NewTags(ident.StringTag("city", "ny"), ident.StringTag("conference", "monitoroma")) + barTags := ident.NewTags(ident.StringTag("city", "sf")) + bazTags := ident.NewTags(ident.StringTag("city", "oakland")) shardZero := uint32(0) foo := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero, ID: ident.StringID("foo"), Tags: fooTags} bar := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 1, ID: ident.StringID("bar"), Tags: barTags} baz := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 5, ID: ident.StringID("baz"), Tags: bazTags} // Make sure we can handle series that don't have tags. - untagged := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 5, ID: ident.StringID("untagged"), Tags: nil} + untagged := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 5, ID: ident.StringID("untagged"), Tags: ident.Tags{}} // Make sure we skip series that are not within the bootstrap range. - outOfRange := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 3, ID: ident.StringID("outOfRange"), Tags: nil} + outOfRange := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 3, ID: ident.StringID("outOfRange"), Tags: ident.Tags{}} // Make sure we skip and dont panic on writes for shards that are higher than the maximum we're trying to bootstrap. - shardTooHigh := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 100, ID: ident.StringID("shardTooHigh"), Tags: nil} + shardTooHigh := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 100, ID: ident.StringID("shardTooHigh"), Tags: ident.Tags{}} // Make sure we skip series for shards that have no requested bootstrap ranges. The shard for this write needs // to be less than the highest shard we actually plan to bootstrap. - noShardBootstrapRange := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 4, ID: ident.StringID("noShardBootstrapRange"), Tags: nil} + noShardBootstrapRange := commitlog.Series{Namespace: testNamespaceID, Shard: shardZero + 4, ID: ident.StringID("noShardBootstrapRange"), Tags: ident.Tags{}} seriesNotToExpect := map[string]struct{}{ outOfRange.ID.String(): struct{}{}, @@ -192,7 +192,7 @@ func verifyIndexResultsAreCorrect( existingTags = map[string]string{} expectedSeries[seriesID] = existingTags } - for _, tag := range value.s.Tags { + for _, tag := range value.s.Tags.Values() { existingTags[tag.Name.String()] = tag.Value.String() } } diff --git a/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 1d884013bd..148a101678 100644 --- a/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -284,7 +284,7 @@ func seriesUniqueTags(seriesID, proposedTagKey, proposedTagVal string, includeTa } if includeTags { - tags = ident.Tags{ident.StringTag(proposedTagKey, proposedTagVal)} + tags = ident.NewTags(ident.StringTag(proposedTagKey, proposedTagVal)) } seriesRegistry.idToTags[seriesID] = tags return tags diff --git a/storage/bootstrap/bootstrapper/fs/source_data_test.go b/storage/bootstrap/bootstrapper/fs/source_data_test.go index a580997d8f..167ada1671 100644 --- a/storage/bootstrap/bootstrapper/fs/source_data_test.go +++ b/storage/bootstrap/bootstrapper/fs/source_data_test.go @@ -192,7 +192,7 @@ func sortedTagsFromTagsMap(tags map[string]string) ident.Tags { } sort.Strings(tagNames) for _, name := range tagNames { - seriesTags = append(seriesTags, ident.StringTag(name, tags[name])) + seriesTags.Append(ident.StringTag(name, tags[name])) } return seriesTags } diff --git a/storage/bootstrap/bootstrapper/peers/source_data_test.go b/storage/bootstrap/bootstrapper/peers/source_data_test.go index 3a0c639c8c..546cc61052 100644 --- a/storage/bootstrap/bootstrapper/peers/source_data_test.go +++ b/storage/bootstrap/bootstrapper/peers/source_data_test.go @@ -126,7 +126,7 @@ func TestPeersSourceReturnsFulfilledAndUnfulfilled(t *testing.T) { goodResult := result.NewShardResult(0, opts.ResultOptions()) fooBlock := block.NewDatabaseBlock(start, ropts.BlockSize(), ts.Segment{}, testBlockOpts) - goodResult.AddBlock(ident.StringID("foo"), ident.Tags{ident.StringTag("foo", "oof")}, fooBlock) + goodResult.AddBlock(ident.StringID("foo"), ident.NewTags(ident.StringTag("foo", "oof")), fooBlock) badErr := fmt.Errorf("an error") mockAdminSession := client.NewMockAdminSession(ctrl) @@ -197,15 +197,15 @@ func TestPeersSourceIncrementalRun(t *testing.T) { barBlock := block.NewDatabaseBlock(start.Add(ropts.BlockSize()), ropts.BlockSize(), ts.NewSegment(checked.NewBytes([]byte{4, 5, 6}, nil), nil, ts.FinalizeNone), testBlockOpts) - shard0ResultBlock1.AddBlock(ident.StringID("foo"), ident.Tags{ident.StringTag("foo", "oof")}, fooBlock) - shard0ResultBlock2.AddBlock(ident.StringID("bar"), ident.Tags{ident.StringTag("bar", "rab")}, barBlock) + shard0ResultBlock1.AddBlock(ident.StringID("foo"), ident.NewTags(ident.StringTag("foo", "oof")), fooBlock) + shard0ResultBlock2.AddBlock(ident.StringID("bar"), ident.NewTags(ident.StringTag("bar", "rab")), barBlock) shard1ResultBlock1 := result.NewShardResult(0, opts.ResultOptions()) shard1ResultBlock2 := result.NewShardResult(0, opts.ResultOptions()) bazBlock := block.NewDatabaseBlock(start, ropts.BlockSize(), ts.NewSegment(checked.NewBytes([]byte{7, 8, 9}, nil), nil, ts.FinalizeNone), testBlockOpts) - shard1ResultBlock1.AddBlock(ident.StringID("baz"), ident.Tags{ident.StringTag("baz", "zab")}, bazBlock) + shard1ResultBlock1.AddBlock(ident.StringID("baz"), ident.NewTags(ident.StringTag("baz", "zab")), bazBlock) mockAdminSession := client.NewMockAdminSession(ctrl) mockAdminSession.EXPECT(). @@ -413,7 +413,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { results := make(map[resultsKey]result.ShardResult) addResult := func(shard uint32, id string, b block.DatabaseBlock) { r := result.NewShardResult(0, opts.ResultOptions()) - r.AddBlock(ident.StringID(id), ident.Tags{ident.StringTag(id, id)}, b) + r.AddBlock(ident.StringID(id), ident.NewTags(ident.StringTag(id, id)), b) start := b.StartTime() end := start.Add(ropts.BlockSize()) results[resultsKey{shard, start.UnixNano(), end.UnixNano()}] = r diff --git a/storage/bootstrap/bootstrapper/peers/source_index_test.go b/storage/bootstrap/bootstrapper/peers/source_index_test.go index d99039549e..b2f8f364ab 100644 --- a/storage/bootstrap/bootstrapper/peers/source_index_test.go +++ b/storage/bootstrap/bootstrapper/peers/source_index_test.go @@ -47,7 +47,7 @@ func (s testSeriesMetadata) ID() ident.ID { func (s testSeriesMetadata) Tags() ident.Tags { if s.tags == nil { - return nil + return ident.Tags{} } // Return in sorted order for deterministic order @@ -59,7 +59,7 @@ func (s testSeriesMetadata) Tags() ident.Tags { var tags ident.Tags for _, key := range keys { - tags = append(tags, ident.StringTag(key, s.tags[key])) + tags.Append(ident.StringTag(key, s.tags[key])) } return tags diff --git a/storage/bootstrap/result/result_data_test.go b/storage/bootstrap/result/result_data_test.go index fc30c82540..24412bf984 100644 --- a/storage/bootstrap/result/result_data_test.go +++ b/storage/bootstrap/result/result_data_test.go @@ -57,8 +57,8 @@ func TestDataResultAddMergesExistingShardResults(t *testing.T) { NewShardResult(0, opts), } - fooTags := ident.Tags{ident.StringTag("foo", "foe")} - barTags := ident.Tags{ident.StringTag("bar", "baz")} + fooTags := ident.NewTags(ident.StringTag("foo", "foe")) + barTags := ident.NewTags(ident.StringTag("bar", "baz")) srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[0]) srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[1]) @@ -151,8 +151,8 @@ func TestResultNumSeries(t *testing.T) { NewShardResult(0, opts), } - fooTags := ident.Tags{ident.StringTag("foo", "foe")} - barTags := ident.Tags{ident.StringTag("bar", "baz")} + fooTags := ident.NewTags(ident.StringTag("foo", "foe")) + barTags := ident.NewTags(ident.StringTag("bar", "baz")) srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[0]) srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[1]) @@ -181,8 +181,8 @@ func TestResultAddResult(t *testing.T) { NewShardResult(0, opts), NewShardResult(0, opts), } - fooTags := ident.Tags{ident.StringTag("foo", "foe")} - barTags := ident.Tags{ident.StringTag("bar", "baz")} + fooTags := ident.NewTags(ident.StringTag("foo", "foe")) + barTags := ident.NewTags(ident.StringTag("bar", "baz")) srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[0]) srs[0].AddBlock(ident.StringID("foo"), fooTags, blocks[1]) @@ -234,7 +234,7 @@ func TestShardResultIsEmpty(t *testing.T) { require.True(t, sr.IsEmpty()) block := opts.DatabaseBlockOptions().DatabaseBlockPool().Get() block.Reset(time.Now(), time.Hour, ts.Segment{}) - fooTags := ident.Tags{ident.StringTag("foo", "foe")} + fooTags := ident.NewTags(ident.StringTag("foo", "foe")) sr.AddBlock(ident.StringID("foo"), fooTags, block) require.False(t, sr.IsEmpty()) } @@ -248,9 +248,9 @@ func TestShardResultAddBlock(t *testing.T) { tags ident.Tags timestamp time.Time }{ - {"foo", ident.Tags{ident.StringTag("foo", "foe")}, start}, - {"foo", ident.Tags{ident.StringTag("foo", "foe")}, start.Add(2 * time.Hour)}, - {"bar", ident.Tags{ident.StringTag("bar", "baz")}, start}, + {"foo", ident.NewTags(ident.StringTag("foo", "foe")), start}, + {"foo", ident.NewTags(ident.StringTag("foo", "foe")), start.Add(2 * time.Hour)}, + {"bar", ident.NewTags(ident.StringTag("bar", "baz")), start}, } for _, input := range inputs { block := opts.DatabaseBlockOptions().DatabaseBlockPool().Get() @@ -276,8 +276,8 @@ func TestShardResultAddSeries(t *testing.T) { tags ident.Tags series block.DatabaseSeriesBlocks }{ - {"foo", ident.Tags{ident.StringTag("foo", "foe")}, block.NewDatabaseSeriesBlocks(0)}, - {"bar", ident.Tags{ident.StringTag("bar", "baz")}, block.NewDatabaseSeriesBlocks(0)}, + {"foo", ident.NewTags(ident.StringTag("foo", "foe")), block.NewDatabaseSeriesBlocks(0)}, + {"bar", ident.NewTags(ident.StringTag("bar", "baz")), block.NewDatabaseSeriesBlocks(0)}, } for _, input := range inputs { sr.AddSeries(ident.StringID(input.id), input.tags, input.series) @@ -286,7 +286,7 @@ func TestShardResultAddSeries(t *testing.T) { block := opts.DatabaseBlockOptions().DatabaseBlockPool().Get() block.Reset(start, time.Hour, ts.Segment{}) moreSeries.AddBlock(block) - sr.AddSeries(ident.StringID("foo"), ident.Tags{ident.StringTag("foo", "foe")}, moreSeries) + sr.AddSeries(ident.StringID("foo"), ident.NewTags(ident.StringTag("foo", "foe")), moreSeries) allSeries := sr.AllSeries() require.Equal(t, 2, allSeries.Len()) fooBlocks, ok := allSeries.Get(ident.StringID("foo")) @@ -303,8 +303,8 @@ func TestShardResultAddResult(t *testing.T) { sr.AddResult(nil) require.True(t, sr.IsEmpty()) other := NewShardResult(0, opts) - other.AddSeries(ident.StringID("foo"), ident.Tags{ident.StringTag("foo", "foe")}, block.NewDatabaseSeriesBlocks(0)) - other.AddSeries(ident.StringID("bar"), ident.Tags{ident.StringTag("bar", "baz")}, block.NewDatabaseSeriesBlocks(0)) + other.AddSeries(ident.StringID("foo"), ident.NewTags(ident.StringTag("foo", "foe")), block.NewDatabaseSeriesBlocks(0)) + other.AddSeries(ident.StringID("bar"), ident.NewTags(ident.StringTag("bar", "baz")), block.NewDatabaseSeriesBlocks(0)) sr.AddResult(other) require.Equal(t, 2, sr.AllSeries().Len()) } @@ -315,8 +315,8 @@ func TestShardResultNumSeries(t *testing.T) { sr.AddResult(nil) require.True(t, sr.IsEmpty()) other := NewShardResult(0, opts) - other.AddSeries(ident.StringID("foo"), ident.Tags{ident.StringTag("foo", "foe")}, block.NewDatabaseSeriesBlocks(0)) - other.AddSeries(ident.StringID("bar"), ident.Tags{ident.StringTag("bar", "baz")}, block.NewDatabaseSeriesBlocks(0)) + other.AddSeries(ident.StringID("foo"), ident.NewTags(ident.StringTag("foo", "foe")), block.NewDatabaseSeriesBlocks(0)) + other.AddSeries(ident.StringID("bar"), ident.NewTags(ident.StringTag("bar", "baz")), block.NewDatabaseSeriesBlocks(0)) sr.AddResult(other) require.Equal(t, int64(2), sr.NumSeries()) } @@ -329,8 +329,8 @@ func TestShardResultRemoveSeries(t *testing.T) { tags ident.Tags series block.DatabaseSeriesBlocks }{ - {"foo", ident.Tags{ident.StringTag("foo", "foe")}, block.NewDatabaseSeriesBlocks(0)}, - {"bar", ident.Tags{ident.StringTag("bar", "baz")}, block.NewDatabaseSeriesBlocks(0)}, + {"foo", ident.NewTags(ident.StringTag("foo", "foe")), block.NewDatabaseSeriesBlocks(0)}, + {"bar", ident.NewTags(ident.StringTag("bar", "baz")), block.NewDatabaseSeriesBlocks(0)}, } for _, input := range inputs { sr.AddSeries(ident.StringID(input.id), input.tags, input.series) diff --git a/storage/index/convert/convert_test.go b/storage/index/convert/convert_test.go index 42ec55b1dc..22c0c0810b 100644 --- a/storage/index/convert/convert_test.go +++ b/storage/index/convert/convert_test.go @@ -40,34 +40,34 @@ func init() { return pool.NewBytesPool(s, nil) }) bytesPool.Init() - idPool := ident.NewPool(bytesPool, nil) + idPool := ident.NewPool(bytesPool, ident.PoolOptions{}) testOpts.CheckedBytesPool = bytesPool testOpts.IdentPool = idPool } func TestFromMetricInvalid(t *testing.T) { id := ident.StringID("foo") - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag(string(convert.ReservedFieldNameID), "value"), - } + ) _, err := convert.FromMetric(id, tags) assert.Error(t, err) } func TestFromMetricIteratorInvalid(t *testing.T) { id := ident.StringID("foo") - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag(string(convert.ReservedFieldNameID), "value"), - } - _, err := convert.FromMetricIter(id, ident.NewTagSliceIterator(tags)) + ) + _, err := convert.FromMetricIter(id, ident.NewTagsIterator(tags)) assert.Error(t, err) } func TestFromMetricValid(t *testing.T) { id := ident.StringID("foo") - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag("bar", "baz"), - } + ) d, err := convert.FromMetric(id, tags) assert.NoError(t, err) assert.Equal(t, "foo", string(d.ID)) @@ -78,10 +78,10 @@ func TestFromMetricValid(t *testing.T) { func TestFromMetricIterValid(t *testing.T) { id := ident.StringID("foo") - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag("bar", "baz"), - } - d, err := convert.FromMetricIter(id, ident.NewTagSliceIterator(tags)) + ) + d, err := convert.FromMetricIter(id, ident.NewTagsIterator(tags)) assert.NoError(t, err) assert.Equal(t, "foo", string(d.ID)) assert.Len(t, d.Fields, 1) diff --git a/storage/index_block_test.go b/storage/index_block_test.go index b8f5d49cce..a5eb03ba88 100644 --- a/storage/index_block_test.go +++ b/storage/index_block_test.go @@ -130,7 +130,7 @@ func TestNamespaceIndexWrite(t *testing.T) { blockStart := xtime.ToUnixNano(now.Truncate(blockSize)) id := ident.StringID("foo") - tags := ident.Tags{ident.StringTag("name", "value")} + tags := ident.NewTags(ident.StringTag("name", "value")) lifecycle := index.NewMockOnIndexSeries(ctrl) mockBlock.EXPECT().WriteBatch([]index.WriteBatchEntry{ index.WriteBatchEntry{ @@ -178,7 +178,7 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { require.NoError(t, err) id := ident.StringID("foo") - tags := ident.Tags{ident.StringTag("name", "value")} + tags := ident.NewTags(ident.StringTag("name", "value")) lifecycle := index.NewMockOnIndexSeries(ctrl) b1.EXPECT().WriteBatch([]index.WriteBatchEntry{ index.WriteBatchEntry{ diff --git a/storage/index_queue_test.go b/storage/index_queue_test.go index a3553fb2d6..e243de8d1d 100644 --- a/storage/index_queue_test.go +++ b/storage/index_queue_test.go @@ -124,9 +124,9 @@ func TestNamespaceIndexInvalidDocWrite(t *testing.T) { assert.True(t, ok) id := ident.StringID("foo") - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag(string(index.ReservedFieldNameID), "value"), - } + ) lifecycle := index.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize() @@ -142,9 +142,9 @@ func TestNamespaceIndexWriteAfterClose(t *testing.T) { assert.True(t, ok) id := ident.StringID("foo") - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag("name", "value"), - } + ) q.EXPECT().Stop().Return(nil) assert.NoError(t, idx.Close()) @@ -163,9 +163,9 @@ func TestNamespaceIndexWriteQueueError(t *testing.T) { assert.True(t, ok) id := ident.StringID("foo") - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag("name", "value"), - } + ) lifecycle := index.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize() @@ -208,9 +208,9 @@ func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { var ( id = ident.StringID("foo") - tags = ident.Tags{ + tags = ident.NewTags( ident.StringTag("name", "value"), - } + ) lifecycle = index.NewMockOnIndexSeries(ctrl) ) @@ -233,9 +233,9 @@ func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { var ( id = ident.StringID("foo") - tags = ident.Tags{ + tags = ident.NewTags( ident.StringTag("name", "value"), - } + ) ) now := time.Now() @@ -270,9 +270,9 @@ func TestNamespaceIndexInsertQuery(t *testing.T) { ts = indexState.latestBlock.EndTime() now = time.Now() id = ident.StringID("foo") - tags = ident.Tags{ + tags = ident.NewTags( ident.StringTag("name", "value"), - } + ) ctx = context.NewContext() lifecycleFns = index.NewMockOnIndexSeries(ctrl) ) @@ -296,5 +296,5 @@ func TestNamespaceIndexInsertQuery(t *testing.T) { assert.True(t, ok) assert.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("name", "value")).Matches( - ident.NewTagSliceIterator(tags))) + ident.NewTagsIterator(tags))) } diff --git a/storage/repair/metadata_test.go b/storage/repair/metadata_test.go index c4b3f90401..e97dc1dcf9 100644 --- a/storage/repair/metadata_test.go +++ b/storage/repair/metadata_test.go @@ -130,9 +130,9 @@ func TestReplicaMetadataComparerAddLocalMetadata(t *testing.T) { now := time.Now() localIter := block.NewMockFilteredBlocksMetadataIter(ctrl) inputBlocks := []block.Metadata{ - block.NewMetadata(ident.StringID("foo"), nil, now, int64(0), new(uint32), time.Time{}), - block.NewMetadata(ident.StringID("foo"), nil, now.Add(time.Second), int64(2), new(uint32), time.Time{}), - block.NewMetadata(ident.StringID("bar"), nil, now, int64(4), nil, time.Time{}), + block.NewMetadata(ident.StringID("foo"), ident.Tags{}, now, int64(0), new(uint32), time.Time{}), + block.NewMetadata(ident.StringID("foo"), ident.Tags{}, now.Add(time.Second), int64(2), new(uint32), time.Time{}), + block.NewMetadata(ident.StringID("bar"), ident.Tags{}, now, int64(4), nil, time.Time{}), } gomock.InOrder( @@ -143,6 +143,7 @@ func TestReplicaMetadataComparerAddLocalMetadata(t *testing.T) { localIter.EXPECT().Next().Return(true), localIter.EXPECT().Current().Return(inputBlocks[2].ID, inputBlocks[2]), localIter.EXPECT().Next().Return(false), + localIter.EXPECT().Err().Return(nil), ) m := NewReplicaMetadataComparer(3, testRepairOptions()).(replicaMetadataComparer) @@ -169,22 +170,22 @@ func TestReplicaMetadataComparerAddPeerMetadata(t *testing.T) { }{ { host: topology.NewHost("1", "addr1"), - meta: block.NewMetadata(ident.StringID("foo"), nil, + meta: block.NewMetadata(ident.StringID("foo"), ident.Tags{}, now, int64(0), new(uint32), time.Time{}), }, { host: topology.NewHost("1", "addr1"), - meta: block.NewMetadata(ident.StringID("foo"), nil, + meta: block.NewMetadata(ident.StringID("foo"), ident.Tags{}, now.Add(time.Second), int64(1), new(uint32), time.Time{}), }, { host: topology.NewHost("2", "addr2"), - meta: block.NewMetadata(ident.StringID("foo"), nil, + meta: block.NewMetadata(ident.StringID("foo"), ident.Tags{}, now, int64(2), nil, time.Time{}), }, { host: topology.NewHost("2", "addr2"), - meta: block.NewMetadata(ident.StringID("bar"), nil, + meta: block.NewMetadata(ident.StringID("bar"), ident.Tags{}, now.Add(time.Second), int64(3), nil, time.Time{}), }, } diff --git a/storage/repair_test.go b/storage/repair_test.go index fac243848e..cd24066d65 100644 --- a/storage/repair_test.go +++ b/storage/repair_test.go @@ -272,11 +272,11 @@ func TestDatabaseShardRepairerRepair(t *testing.T) { host topology.Host meta block.Metadata }{ - {topology.NewHost("1", "addr1"), block.NewMetadata(ident.StringID("foo"), nil, + {topology.NewHost("1", "addr1"), block.NewMetadata(ident.StringID("foo"), ident.Tags{}, now.Add(30*time.Minute), sizes[0], &checksums[0], lastRead)}, - {topology.NewHost("1", "addr1"), block.NewMetadata(ident.StringID("foo"), nil, + {topology.NewHost("1", "addr1"), block.NewMetadata(ident.StringID("foo"), ident.Tags{}, now.Add(time.Hour), sizes[0], &checksums[1], lastRead)}, - {topology.NewHost("1", "addr1"), block.NewMetadata(ident.StringID("bar"), nil, + {topology.NewHost("1", "addr1"), block.NewMetadata(ident.StringID("bar"), ident.Tags{}, now.Add(30*time.Minute), sizes[2], &checksums[2], lastRead)}, } diff --git a/storage/shard_fetch_blocks_metadata_test.go b/storage/shard_fetch_blocks_metadata_test.go index 995a97d946..5aada40532 100644 --- a/storage/shard_fetch_blocks_metadata_test.go +++ b/storage/shard_fetch_blocks_metadata_test.go @@ -68,11 +68,11 @@ func TestShardFetchBlocksMetadata(t *testing.T) { lastRead := time.Now().Add(-time.Minute) for i := 0; i < 10; i++ { id := ident.StringID(fmt.Sprintf("foo.%d", i)) - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd"), - } - tagsIter := ident.NewTagSliceIterator(tags) + ) + tagsIter := ident.NewTagsIterator(tags) series := addMockSeries(ctrl, shard, id, tags, uint64(i)) if i == 2 { series.EXPECT(). @@ -129,11 +129,11 @@ func TestShardFetchBlocksMetadataV2WithSeriesCachePolicyCacheAll(t *testing.T) { lastRead := time.Now().Add(-time.Minute) for i := int64(0); i < 10; i++ { id := ident.StringID(fmt.Sprintf("foo.%d", i)) - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd"), - } - tagsIter := ident.NewTagSliceIterator(tags) + ) + tagsIter := ident.NewTagsIterator(tags) series := addMockSeries(ctrl, shard, id, tags, uint64(i)) if i == startCursor { series.EXPECT(). @@ -253,7 +253,7 @@ func TestShardFetchBlocksMetadataV2WithSeriesCachePolicyNotCacheAll(t *testing.T bytes := checked.NewBytes(data, nil) bytes.IncRef() - err = writer.Write(id, nil, bytes, checksum) + err = writer.Write(id, ident.Tags{}, bytes, checksum) require.NoError(t, err) blockMetadataResult := block.NewFetchBlockMetadataResult(at, @@ -273,10 +273,10 @@ func TestShardFetchBlocksMetadataV2WithSeriesCachePolicyNotCacheAll(t *testing.T lastRead := time.Now().Add(-time.Minute) for i := 0; i < numActiveSeries; i++ { id := ident.StringID(fmt.Sprintf("series+instance=%d", i)) - tags := ident.Tags{ + tags := ident.NewTags( ident.StringTag("instance", strconv.Itoa(i)), - } - tagsIter := ident.NewTagSliceIterator(tags) + ) + tagsIter := ident.NewTagsIterator(tags) series := addMockSeries(ctrl, shard, id, tags, uint64(i)) blocks := block.NewFetchBlockMetadataResults() at := mostRecentBlockStart diff --git a/storage/shard_index_test.go b/storage/shard_index_test.go index 6eb53029b0..7bc48446f8 100644 --- a/storage/shard_index_test.go +++ b/storage/shard_index_test.go @@ -69,12 +69,12 @@ func TestShardInsertNamespaceIndex(t *testing.T) { require.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), - ident.NewTagIterator(ident.StringTag("name", "value")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), time.Now(), 1.0, xtime.Second, nil)) require.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), - ident.NewTagIterator(ident.StringTag("name", "value")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), time.Now(), 2.0, xtime.Second, nil)) require.NoError(t, @@ -85,8 +85,8 @@ func TestShardInsertNamespaceIndex(t *testing.T) { require.Len(t, indexWrites, 1) require.Equal(t, "foo", indexWrites[0].id.String()) - require.Equal(t, "name", indexWrites[0].tags[0].Name.String()) - require.Equal(t, "value", indexWrites[0].tags[0].Value.String()) + require.Equal(t, "name", indexWrites[0].tags.Values()[0].Name.String()) + require.Equal(t, "value", indexWrites[0].tags.Values()[0].Value.String()) } func TestShardAsyncInsertNamespaceIndex(t *testing.T) { @@ -115,7 +115,7 @@ func TestShardAsyncInsertNamespaceIndex(t *testing.T) { assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), - ident.NewTagIterator(ident.StringTag("name", "value")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), time.Now(), 1.0, xtime.Second, nil)) assert.NoError(t, @@ -123,10 +123,10 @@ func TestShardAsyncInsertNamespaceIndex(t *testing.T) { assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("baz"), - ident.NewTagIterator( + ident.NewTagsIterator(ident.NewTags( ident.StringTag("all", "tags"), ident.StringTag("should", "be-present"), - ), + )), time.Now(), 1.0, xtime.Second, nil)) for { @@ -144,15 +144,15 @@ func TestShardAsyncInsertNamespaceIndex(t *testing.T) { assert.Len(t, indexWrites, 2) for _, w := range indexWrites { if w.id.String() == "foo" { - assert.Len(t, w.tags, 1) - assert.Equal(t, "name", w.tags[0].Name.String()) - assert.Equal(t, "value", w.tags[0].Value.String()) + assert.Equal(t, 1, len(w.tags.Values())) + assert.Equal(t, "name", w.tags.Values()[0].Name.String()) + assert.Equal(t, "value", w.tags.Values()[0].Value.String()) } else if w.id.String() == "baz" { - assert.Len(t, w.tags, 2) - assert.Equal(t, "all", w.tags[0].Name.String()) - assert.Equal(t, "tags", w.tags[0].Value.String()) - assert.Equal(t, "should", w.tags[1].Name.String()) - assert.Equal(t, "be-present", w.tags[1].Value.String()) + assert.Equal(t, 2, len(w.tags.Values())) + assert.Equal(t, "all", w.tags.Values()[0].Name.String()) + assert.Equal(t, "tags", w.tags.Values()[0].Value.String()) + assert.Equal(t, "should", w.tags.Values()[1].Name.String()) + assert.Equal(t, "be-present", w.tags.Values()[1].Value.String()) } else { assert.Fail(t, "unexpected write", w) } @@ -187,7 +187,7 @@ func TestShardAsyncIndexOnlyWhenNotIndexed(t *testing.T) { assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), - ident.NewTagIterator(ident.StringTag("name", "value")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), time.Now(), 1.0, xtime.Second, nil)) for { @@ -200,7 +200,7 @@ func TestShardAsyncIndexOnlyWhenNotIndexed(t *testing.T) { // ensure we don't index once we have already indexed assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), - ident.NewTagIterator(ident.StringTag("name", "value")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), time.Now(), 2.0, xtime.Second, nil)) l := atomic.LoadInt32(&numCalls) @@ -249,7 +249,7 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), - ident.NewTagIterator(ident.StringTag("name", "value")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), now, 1.0, xtime.Second, nil)) // wait till we're done indexing. @@ -261,7 +261,7 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { // ensure we index because it's expired assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), - ident.NewTagIterator(ident.StringTag("name", "value")), + ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), now.Add(time.Minute), 2.0, xtime.Second, nil)) // wait till we're done indexing. diff --git a/storage/shard_test.go b/storage/shard_test.go index f6592ebc46..21df7f9696 100644 --- a/storage/shard_test.go +++ b/storage/shard_test.go @@ -391,7 +391,7 @@ func addTestSeries(shard *dbShard, id ident.ID) series.DatabaseSeries { } func addTestSeriesWithCount(shard *dbShard, id ident.ID, count int32) series.DatabaseSeries { - series := series.NewDatabaseSeries(id, nil, shard.seriesOpts) + series := series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) series.Bootstrap(nil) shard.Lock() shard.insertNewShardEntryWithLock(&dbShardEntry{series: series, curReadWriters: count}) @@ -736,7 +736,7 @@ func TestPurgeExpiredSeriesWriteAfterTicking(t *testing.T) { shard := testDatabaseShard(t, opts) defer shard.Close() id := ident.StringID("foo") - s := addMockSeries(ctrl, shard, id, nil, 0) + s := addMockSeries(ctrl, shard, id, ident.Tags{}, 0) s.EXPECT().Tick().Do(func() { // Emulate a write taking place just after tick for this series s.EXPECT().Write(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) @@ -766,7 +766,7 @@ func TestPurgeExpiredSeriesWriteAfterPurging(t *testing.T) { shard := testDatabaseShard(t, opts) defer shard.Close() id := ident.StringID("foo") - s := addMockSeries(ctrl, shard, id, nil, 0) + s := addMockSeries(ctrl, shard, id, ident.Tags{}, 0) s.EXPECT().Tick().Do(func() { // Emulate a write taking place and staying open just after tick for this series var err error @@ -842,7 +842,7 @@ func TestShardFetchBlocksIDExists(t *testing.T) { shard := testDatabaseShard(t, opts) defer shard.Close() id := ident.StringID("foo") - series := addMockSeries(ctrl, shard, id, nil, 0) + series := addMockSeries(ctrl, shard, id, ident.Tags{}, 0) now := time.Now() starts := []time.Time{now} expected := []block.FetchBlockResult{block.NewFetchBlockResult(now, nil, nil)} @@ -1095,7 +1095,6 @@ func TestShardNewInvalidShardEntry(t *testing.T) { iter := ident.NewMockTagIterator(ctrl) gomock.InOrder( iter.EXPECT().Duplicate().Return(iter), - iter.EXPECT().Remaining().Return(0), iter.EXPECT().Next().Return(false), iter.EXPECT().Err().Return(fmt.Errorf("random err")), iter.EXPECT().Close(), From 9465d94b613690417b7ce02f1991efd55c23485b Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 13:26:47 -0400 Subject: [PATCH 12/33] Fix big unit tests --- persist/fs/index_lookup_prop_test.go | 2 +- services/m3dbnode/main/main_index_test.go | 8 +++-- storage/block/metadata.go | 5 ++- storage/index/block_test.go | 28 +++++++-------- storage/index/results_test.go | 10 +++--- storage/series/series_test.go | 44 +++++++++++------------ 6 files changed, 52 insertions(+), 45 deletions(-) diff --git a/persist/fs/index_lookup_prop_test.go b/persist/fs/index_lookup_prop_test.go index 466d32749f..fd049ae60f 100644 --- a/persist/fs/index_lookup_prop_test.go +++ b/persist/fs/index_lookup_prop_test.go @@ -229,7 +229,7 @@ func genWrite() gopter.Gen { return generatedWrite{ id: ident.StringID(id), - tags: tags, + tags: ident.NewTags(tags...), data: bytesRefd(data), checksum: digest.Checksum(data), } diff --git a/services/m3dbnode/main/main_index_test.go b/services/m3dbnode/main/main_index_test.go index 844aa8025c..fd639ab56e 100644 --- a/services/m3dbnode/main/main_index_test.go +++ b/services/m3dbnode/main/main_index_test.go @@ -200,8 +200,12 @@ func TestIndexEnabledServer(t *testing.T) { } for _, v := range values { - err := session.WriteTagged(ident.StringID(namespaceID), ident.StringID("foo"), - ident.NewTagIterator(ident.StringTag("foo", "bar"), ident.StringTag("baz", "foo")), + err := session.WriteTagged(ident.StringID(namespaceID), + ident.StringID("foo"), + ident.NewTagsIterator(ident.NewTags( + ident.StringTag("foo", "bar"), + ident.StringTag("baz", "foo"), + )), v.at, v.value, v.unit, nil) require.NoError(t, err) } diff --git a/storage/block/metadata.go b/storage/block/metadata.go index 50e1ecd850..9882f19f6b 100644 --- a/storage/block/metadata.go +++ b/storage/block/metadata.go @@ -51,5 +51,8 @@ func (m *Metadata) Finalize() { m.ID.Finalize() m.ID = nil } - m.Tags.Finalize() + if m.Tags.Values() != nil { + m.Tags.Finalize() + m.Tags = ident.Tags{} + } } diff --git a/storage/index/block_test.go b/storage/index/block_test.go index d798887c3a..9294022be2 100644 --- a/storage/index/block_test.go +++ b/storage/index/block_test.go @@ -427,7 +427,7 @@ func TestBlockMockQueryExecutorExecLimit(t *testing.T) { require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) } func TestBlockMockQueryExecutorExecIterCloseErr(t *testing.T) { @@ -525,7 +525,7 @@ func TestBlockMockQueryLimit(t *testing.T) { require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) } func TestBlockMockQueryLimitExhaustive(t *testing.T) { @@ -565,7 +565,7 @@ func TestBlockMockQueryLimitExhaustive(t *testing.T) { require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) } func TestBlockMockQueryMergeResultsMapLimit(t *testing.T) { @@ -607,7 +607,7 @@ func TestBlockMockQueryMergeResultsMapLimit(t *testing.T) { require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) } func TestBlockMockQueryMergeResultsDupeID(t *testing.T) { @@ -652,13 +652,13 @@ func TestBlockMockQueryMergeResultsDupeID(t *testing.T) { require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) t2, ok := rMap.Get(ident.StringID(string(testDoc2().ID))) require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz", "some", "more")).Matches( - ident.NewTagSliceIterator(t2))) + ident.NewTagsIterator(t2))) } func TestBlockBootstrapAddsSegment(t *testing.T) { @@ -830,13 +830,13 @@ func TestBlockE2EInsertQuery(t *testing.T) { require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) t2, ok := rMap.Get(ident.StringID(string(testDoc2().ID))) require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz", "some", "more")).Matches( - ident.NewTagSliceIterator(t2))) + ident.NewTagsIterator(t2))) } func TestBlockE2EInsertQueryLimit(t *testing.T) { @@ -881,7 +881,7 @@ func TestBlockE2EInsertQueryLimit(t *testing.T) { numFound++ require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) } t2, ok := rMap.Get(ident.StringID(string(testDoc2().ID))) @@ -889,7 +889,7 @@ func TestBlockE2EInsertQueryLimit(t *testing.T) { numFound++ require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz", "some", "more")).Matches( - ident.NewTagSliceIterator(t2))) + ident.NewTagsIterator(t2))) } require.Equal(t, 1, numFound) @@ -938,13 +938,13 @@ func TestBlockE2EInsertBootstrapQuery(t *testing.T) { require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) t2, ok := rMap.Get(ident.StringID(string(testDoc2().ID))) require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz", "some", "more")).Matches( - ident.NewTagSliceIterator(t2))) + ident.NewTagsIterator(t2))) } func TestBlockE2EInsertBootstrapMergeQuery(t *testing.T) { @@ -985,13 +985,13 @@ func TestBlockE2EInsertBootstrapMergeQuery(t *testing.T) { require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz")).Matches( - ident.NewTagSliceIterator(t1))) + ident.NewTagsIterator(t1))) t2, ok := rMap.Get(ident.StringID(string(testDoc2().ID))) require.True(t, ok) require.True(t, ident.NewTagIterMatcher( ident.MustNewTagStringsIterator("bar", "baz", "some", "more")).Matches( - ident.NewTagSliceIterator(t2))) + ident.NewTagsIterator(t2))) } func testSegment(t *testing.T, docs ...doc.Document) segment.Segment { diff --git a/storage/index/results_test.go b/storage/index/results_test.go index 5335bf88dd..22255782a2 100644 --- a/storage/index/results_test.go +++ b/storage/index/results_test.go @@ -71,7 +71,7 @@ func TestResultsFirstInsertWins(t *testing.T) { tags, ok := res.Map().Get(ident.StringID("abc")) require.True(t, ok) - require.Len(t, tags, 0) + require.Equal(t, 0, len(tags.Values())) d2 := doc.Document{ID: []byte("abc"), Fields: doc.Fields{ @@ -84,7 +84,7 @@ func TestResultsFirstInsertWins(t *testing.T) { tags, ok = res.Map().Get(ident.StringID("abc")) require.True(t, ok) - require.Len(t, tags, 0) + require.Equal(t, 0, len(tags.Values())) } func TestResultsInsertContains(t *testing.T) { @@ -97,7 +97,7 @@ func TestResultsInsertContains(t *testing.T) { tags, ok := res.Map().Get(ident.StringID("abc")) require.True(t, ok) - require.Len(t, tags, 0) + require.Equal(t, 0, len(tags.Values())) } func TestResultsInsertCopies(t *testing.T) { @@ -126,12 +126,12 @@ func TestResultsReset(t *testing.T) { tags, ok := res.Map().Get(ident.StringID("abc")) require.True(t, ok) - require.Len(t, tags, 0) + require.Equal(t, 0, len(tags.Values())) res.Reset(nil) _, ok = res.Map().Get(ident.StringID("abc")) require.False(t, ok) - require.Len(t, tags, 0) + require.Equal(t, 0, len(tags.Values())) require.Equal(t, 0, res.Size()) } diff --git a/storage/series/series_test.go b/storage/series/series_test.go index 4f488a1c5e..2201a1a159 100644 --- a/storage/series/series_test.go +++ b/storage/series/series_test.go @@ -78,7 +78,7 @@ func newSeriesTestOptions() Options { func TestSeriesEmpty(t *testing.T) { opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) assert.True(t, series.IsEmpty()) @@ -91,7 +91,7 @@ func TestSeriesWriteFlush(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) @@ -140,7 +140,7 @@ func TestSeriesWriteFlushRead(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) @@ -177,7 +177,7 @@ func TestSeriesWriteFlushRead(t *testing.T) { func TestSeriesReadEndBeforeStart(t *testing.T) { opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) @@ -192,7 +192,7 @@ func TestSeriesReadEndBeforeStart(t *testing.T) { func TestSeriesFlushNoBlock(t *testing.T) { opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) flushTime := time.Unix(7200, 0) @@ -206,7 +206,7 @@ func TestSeriesFlush(t *testing.T) { defer ctrl.Finish() opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) flushTime := time.Unix(7200, 0) @@ -236,7 +236,7 @@ func TestSeriesFlush(t *testing.T) { func TestSeriesTickEmptySeries(t *testing.T) { opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) _, err = series.Tick() @@ -248,7 +248,7 @@ func TestSeriesTickDrainAndResetBuffer(t *testing.T) { defer ctrl.Finish() opts := newSeriesTestOptions() - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) buffer := NewMockdatabaseBuffer(ctrl) @@ -273,7 +273,7 @@ func TestSeriesTickNeedsBlockExpiry(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) blockStart := curr.Add(-ropts.RetentionPeriod()).Add(-ropts.BlockSize()) @@ -315,7 +315,7 @@ func TestSeriesTickNotRetrieved(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever _, err := series.Bootstrap(nil) @@ -345,7 +345,7 @@ func TestSeriesTickRecentlyRead(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever _, err := series.Bootstrap(nil) @@ -406,7 +406,7 @@ func TestSeriesTickCacheLRU(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever _, err := series.Bootstrap(nil) @@ -475,7 +475,7 @@ func TestSeriesTickCacheAllMetadata(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever _, err := series.Bootstrap(nil) @@ -540,7 +540,7 @@ func TestSeriesTickCacheNone(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever _, err := series.Bootstrap(nil) @@ -576,7 +576,7 @@ func TestSeriesBootstrapWithError(t *testing.T) { now := time.Now() blockSize := 2 * time.Hour - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) bufferMin := now.Truncate(blockSize).Add(-blockSize) bufferMax := now.Truncate(blockSize).Add(2 * blockSize) @@ -643,7 +643,7 @@ func TestSeriesFetchBlocks(t *testing.T) { FetchBlocks(ctx, starts). Return([]block.FetchBlockResult{block.NewFetchBlockResult(starts[2], nil, nil)}) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) @@ -713,7 +713,7 @@ func TestSeriesFetchBlocksMetadata(t *testing.T) { FetchBlocksMetadata(ctx, start, end, fetchOpts). Return(expectedResults) - series := NewDatabaseSeries(ident.StringID("bar"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("bar"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) mockBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) @@ -768,7 +768,7 @@ func TestSeriesOutOfOrderWritesAndRotate(t *testing.T) { ctx = context.NewContext() id = ident.StringID("foo") nsID = ident.StringID("bar") - tags = ident.Tags{ident.StringTag("name", "value")} + tags = ident.NewTags(ident.StringTag("name", "value")) startValue = 1.0 blockSize = opts.RetentionOptions().BlockSize() numPoints = 10 @@ -810,7 +810,7 @@ func TestSeriesOutOfOrderWritesAndRotate(t *testing.T) { multiIt := opts.MultiReaderIteratorPool().Get() multiIt.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(encoded)) - it := encoding.NewSeriesIterator(id, nsID, ident.NewTagSliceIterator(tags), + it := encoding.NewSeriesIterator(id, nsID, ident.NewTagsIterator(tags), qStart, qEnd, []encoding.MultiReaderIterator{multiIt}, nil) defer it.Close() @@ -841,7 +841,7 @@ func TestSeriesWriteReadFromTheSameBucket(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { return curr })) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) _, err := series.Bootstrap(nil) assert.NoError(t, err) @@ -866,7 +866,7 @@ func TestSeriesCloseNonCacheLRUPolicy(t *testing.T) { opts := newSeriesTestOptions(). SetCachePolicy(CacheRecentlyRead) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) start := time.Now() blocks := block.NewDatabaseSeriesBlocks(0) @@ -885,7 +885,7 @@ func TestSeriesCloseCacheLRUPolicy(t *testing.T) { opts := newSeriesTestOptions(). SetCachePolicy(CacheLRU) - series := NewDatabaseSeries(ident.StringID("foo"), nil, opts).(*dbSeries) + series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) start := time.Now() blocks := block.NewDatabaseSeriesBlocks(0) From 7f6be88567a5732c2cc9bf514e38c804368dc337 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 13:30:01 -0400 Subject: [PATCH 13/33] Finalize just tags on bootstrap --- storage/shard.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/storage/shard.go b/storage/shard.go index febe5708b0..ae9bdd3d03 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -1619,12 +1619,12 @@ func (s *dbShard) Bootstrap( } } - // No longer require ID or tags - dbBlocks.ID.Finalize() + // No longer require tags as we copy them in insert series sync + // or if we found the series then we don't require them for insertion + // at all dbBlocks.Tags.Finalize() - // Cannot close blocks once done as series takes ref to these, - // however we copy ID and Tags + // Cannot close blocks once done as series takes ref to these bsResult, err := entry.series.Bootstrap(dbBlocks.Blocks) if err != nil { multiErr = multiErr.Add(err) From b66f6a48a6c3fcba20238b969c11d166a271d4da Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 13:38:05 -0400 Subject: [PATCH 14/33] Fix config test and remove unused consts --- services/m3dbnode/config/config_test.go | 12 ++++++++++-- storage/index/options.go | 3 --- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/services/m3dbnode/config/config_test.go b/services/m3dbnode/config/config_test.go index 02456bfd19..73d6f19a8d 100644 --- a/services/m3dbnode/config/config_test.go +++ b/services/m3dbnode/config/config_test.go @@ -186,12 +186,16 @@ pooling: capacity: 4096 lowWatermark: 0.01 highWatermark: 0.02 - tagArrayPool: + tagsPool: size: 65536 capacity: 8 maxCapacity: 32 lowWatermark: 0.01 highWatermark: 0.02 + tagIteratorPool: + size: 8192 + lowWatermark: 0.01 + highWatermark: 0.02 indexResultsPool: size: 8192 lowWatermark: 0.01 @@ -498,12 +502,16 @@ pooling: capacity: 4096 lowWatermark: 0.01 highWatermark: 0.02 - tagArrayPool: + tagsPool: size: 65536 capacity: 8 maxCapacity: 32 lowWatermark: 0.01 highWatermark: 0.02 + tagIteratorPool: + size: 8192 + lowWatermark: 0.01 + highWatermark: 0.02 indexResultsPool: size: 8192 lowWatermark: 0.01 diff --git a/storage/index/options.go b/storage/index/options.go index ababe3786e..562fce9e38 100644 --- a/storage/index/options.go +++ b/storage/index/options.go @@ -33,9 +33,6 @@ import ( const ( // defaultIndexInsertMode sets the default indexing mode to synchronous. defaultIndexInsertMode = InsertSync - - defaultTagArrayPoolInitCapacity = 8 - defaultTagArrayPoolMaxCapacity = 32 ) var ( From 42933bc6fe8d8b5ce3afd378e735d7254311afc4 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 19:11:01 -0400 Subject: [PATCH 15/33] Use new WriteBatch type to flow index batch insertion without realloc --- glide.lock | 7 +- glide.yaml | 2 +- storage/index.go | 112 +++++------ storage/index/block.go | 104 ++++------- storage/index/index_mock.go | 2 +- storage/index/types.go | 339 ++++++++++++++++++++++++++-------- storage/index_insert_queue.go | 11 +- storage/shard.go | 30 ++- storage/types.go | 4 +- 9 files changed, 384 insertions(+), 227 deletions(-) diff --git a/glide.lock b/glide.lock index fc9b06d240..727d2fb602 100644 --- a/glide.lock +++ b/glide.lock @@ -1,5 +1,5 @@ -hash: ae522bcbc20f0ddb69902e6e73f1b42499a361d4fda9e4c1f9f32b148a253d19 -updated: 2018-05-13T10:07:27.359065-04:00 +hash: 238562b5d3566785aa16f01f9dcea7d80e0281177fdb88be6036b1f60e6c8744 +updated: 2018-05-13T17:00:00.352679-04:00 imports: - name: github.com/alecthomas/template version: a0175ee3bccc567396460bf5acd36800cb10c49c @@ -219,7 +219,7 @@ imports: - metric - policy - name: github.com/m3db/m3ninx - version: 12b8ac4f173f9d539b0e94f3bca475318ab1a8db + version: 2a492ea6d91d3e2b5b9deb1b5cab343c80de373b subpackages: - doc - generated/proto/querypb @@ -228,6 +228,7 @@ imports: - index/segment - index/segment/mem - index/util + - persist - postings - postings/roaring - search diff --git a/glide.yaml b/glide.yaml index 80da9f1037..54a2ad0856 100644 --- a/glide.yaml +++ b/glide.yaml @@ -26,7 +26,7 @@ import: version: ed532baee45a440f0b08b6893c816634c6978d4d - package: github.com/m3db/m3ninx - version: 12b8ac4f173f9d539b0e94f3bca475318ab1a8db + version: 2a492ea6d91d3e2b5b9deb1b5cab343c80de373b - package: github.com/m3db/bitset version: 07973db6b78acb62ac207d0538055e874b49d90d diff --git a/storage/index.go b/storage/index.go index ff3c074c16..b1821314eb 100644 --- a/storage/index.go +++ b/storage/index.go @@ -30,8 +30,10 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/retention" "github.com/m3db/m3db/storage/bootstrap/result" + m3dberrors "github.com/m3db/m3db/storage/errors" "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/namespace" + "github.com/m3db/m3ninx/doc" "github.com/m3db/m3x/context" xerrors "github.com/m3db/m3x/errors" "github.com/m3db/m3x/instrument" @@ -211,75 +213,45 @@ func (i *nsIndex) BlockStartForWriteTime(writeTime time.Time) xtime.UnixNano { // // - For new entry (previously unseen in the shard): // shard.WriteTagged() +// => shard.insertSeriesAsyncBatched() // => shardInsertQueue.Insert() // => shard.writeBatch() -// => index.Write() +// => index.WriteBatch() // => indexQueue.Insert() // => index.writeBatch() // // - For entry which exists in the shard, but needs indexing (either past // the TTL or the last indexing hasn't happened/failed): // shard.WriteTagged() +// => shard.insertSeriesForIndexingAsyncBatched() +// => shardInsertQueue.Insert() +// => shard.writeBatch() // => index.Write() // => indexQueue.Insert() // => index.writeBatch() func (i *nsIndex) WriteBatch( - entries []index.WriteBatchEntry, -) error { - // Ensure timestamp is not too old/new based on retention policies. - now := i.nowFn() - futureLimit := now.Add(1 * i.bufferFuture) - pastLimit := now.Add(-1 * i.bufferPast) - - var emptyEntry index.WriteBatchEntry - for j := range entries { - var ( - timestamp = entries[j].Timestamp - onIndexFn = entries[j].OnIndexSeries - blockStart = i.BlockStartForWriteTime(timestamp) - ) - if !futureLimit.After(timestamp) { - onIndexFn.OnIndexFinalize(blockStart) - entries[j] = emptyEntry // indicate we don't need to index this. - // TODO(prateek): capture that this needs to return m3dberrors.ErrTooFuture - continue - } - - if !pastLimit.Before(timestamp) { - onIndexFn.OnIndexFinalize(blockStart) - entries[j] = emptyEntry // indicate we don't need to index this. - // TODO(prateek): capture that this needs to return m3dberrors.ErrTooPast - continue - } - - // update the timestamp to the blockstart for the block it needs to be sent to - entries[j].Timestamp = blockStart.ToTime() - } - return i.enqueueBatch(entries) -} - -func (i *nsIndex) enqueueBatch( - entries []index.WriteBatchEntry, + batch *index.WriteBatch, ) error { i.state.RLock() if !i.isOpenWithRLock() { i.state.RUnlock() i.metrics.InsertAfterClose.Inc(1) - index.WriteBatchEntriesFinalizer(entries).Finalize() - return errDbIndexUnableToWriteClosed + err := errDbIndexUnableToWriteClosed + batch.MarkUnmarkedEntriesError(err) + return err } // NB(prateek): retrieving insertMode here while we have the RLock. insertMode := i.state.runtimeOpts.insertMode - wg, err := i.state.insertQueue.InsertBatch(entries) + wg, err := i.state.insertQueue.InsertBatch(batch) // release the lock because we don't need it past this point. i.state.RUnlock() // if we're unable to index, we still have to finalize the reference we hold. if err != nil { - index.WriteBatchEntriesFinalizer(entries).Finalize() + batch.MarkUnmarkedEntriesError(err) return err } // once the write has been queued in the indexInsertQueue, it assumes @@ -287,21 +259,14 @@ func (i *nsIndex) enqueueBatch( // wait/terminate depending on if we are indexing synchronously or not. if insertMode != index.InsertAsync { - // FOLLOWUP(prateek): to correctly propagate indexing error to the user in - // the sync case, we need a mechanism to receive notifications from - // the index insert queue path. Some ways to do this: - // (0) alloc a slice for the errors in queue, and return a wait group - // the slice and an index into the slice. - // (1) provide an error channel as input to i.insertQueue.Insert, and - // guarantee that receives a value on success/failure in the async - // insertion code path. We could eliminate the wait group if we did that. - // (2) we could provide an OnIndexError callback within OnIndexSeries, - // again ensuring it was called upon failure, and cache the error within - // the dbShardEntry. Then we're guaranteed that the error would be set - // once wg.Wait returns, and the shard insert code-paths would be able - // to retrieve the error from the dbShardEntry. Not pretty, but probably - // a lot cheaper than (1). wg.Wait() + + // Resort the batch by initial enqueue order + if numErrs := batch.NumErrs(); numErrs > 0 { + // Restore the sort order from whene enqueued for the caller + batch.SortByEnqueued() + return fmt.Errorf("check batch: %d insert errors", numErrs) + } } return nil @@ -312,7 +277,7 @@ func (i *nsIndex) enqueueBatch( // so that we can notify users of success/failure correctly in the case of // sync'd inserts. func (i *nsIndex) writeBatches( - batches [][]index.WriteBatchEntry, + batches []*index.WriteBatch, ) { // NB(prateek): we use a read lock to guard against mutation of the // indexBlocks, mutations within the underlying blocks are guarded @@ -325,33 +290,50 @@ func (i *nsIndex) writeBatches( return } + now := i.nowFn() + futureLimit := now.Add(1 * i.bufferFuture) + pastLimit := now.Add(-1 * i.bufferPast) + writeBatchFn := i.writeBatchForBlockStartWithRLock for _, batch := range batches { + // Ensure timestamp is not too old/new based on retention policies. + batch.ForEach(func(idx int, entry index.WriteBatchEntry, + _ doc.Document, _ index.WriteBatchEntryResult) { + if !futureLimit.After(entry.Timestamp) { + batch.MarkUnmarkedEntryError(m3dberrors.ErrTooFuture, idx) + return + } + + if !pastLimit.Before(entry.Timestamp) { + batch.MarkUnmarkedEntryError(m3dberrors.ErrTooPast, idx) + return + } + }) + // we sort the inserts by which block they're applicable for, and do the inserts - // for each block. - writesByBlockStart := index.WriteBatchEntryByBlockStartAndID(batch) - sort.Sort(writesByBlockStart) - writesByBlockStart.ForEachBlockStart(i.writeBatchForBlockStartWithRLock) + // for each block, making sure to not try to insert any entries already marked + // with a result. + batch.ForEachUnmarkedBatchByBlockStart(writeBatchFn) } } func (i *nsIndex) writeBatchForBlockStartWithRLock( - blockStart time.Time, inserts index.WriteBatchEntryByBlockStartAndID, + blockStart time.Time, batch *index.WriteBatch, ) { // ensure we have an index block for the specified blockStart. block, err := i.ensureBlockPresentWithRLock(blockStart) if err != nil { - index.WriteBatchEntriesFinalizer(inserts).Finalize() + batch.MarkUnmarkedEntriesError(err) i.logger.WithFields( xlog.NewField("blockStart", blockStart), - xlog.NewField("numWrites", len(inserts)), + xlog.NewField("numWrites", batch.Len()), xlog.NewField("err", err.Error()), ).Error("unable to write to index, dropping inserts.") - i.metrics.AsyncInsertErrors.Inc(int64(len(inserts))) + i.metrics.AsyncInsertErrors.Inc(int64(batch.Len())) return } // i.e. we have the block and the inserts, perform the writes. - result, err := block.WriteBatch(inserts) + result, err := block.WriteBatch(batch) // NB: we don't need to do anything to the OnIndexSeries refs in `inserts` at this point, // the index.Block WriteBatch assumes responsibility for calling the appropriate methods. diff --git a/storage/index/block.go b/storage/index/block.go index c56e51bb64..a4f2b5a9f2 100644 --- a/storage/index/block.go +++ b/storage/index/block.go @@ -26,7 +26,6 @@ import ( "sync" "time" - "github.com/m3db/m3db/storage/index/convert" m3ninxindex "github.com/m3db/m3ninx/index" "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3ninx/index/segment/mem" @@ -36,7 +35,6 @@ import ( "github.com/m3db/m3x/context" xerrors "github.com/m3db/m3x/errors" "github.com/m3db/m3x/instrument" - xtime "github.com/m3db/m3x/time" ) var ( @@ -70,6 +68,7 @@ type block struct { newExecutorFn newExecutorFn startTime time.Time endTime time.Time + blockSize time.Duration opts Options } @@ -88,11 +87,11 @@ func NewBlock( } b := &block{ - state: blockStateOpen, - segment: seg, - + state: blockStateOpen, + segment: seg, startTime: startTime, endTime: startTime.Add(blockSize), + blockSize: blockSize, opts: opts, } b.newExecutorFn = b.executorWithRLock @@ -108,74 +107,49 @@ func (b *block) EndTime() time.Time { return b.endTime } -func (b *block) WriteBatch(inserts WriteBatchEntryByBlockStartAndID) (WriteBatchResult, error) { - b.RLock() - defer b.RUnlock() +func (b *block) WriteBatch(inserts *WriteBatch) (WriteBatchResult, error) { + b.Lock() + defer b.Unlock() if b.state != blockStateOpen { - // NB: releasing all references to inserts - WriteBatchEntriesFinalizer(inserts).Finalize() + err := b.writeBatchErrorInvalidState(b.state) + inserts.MarkUnmarkedEntriesError(err) return WriteBatchResult{ - NumError: int64(len(inserts)), - }, b.writeBatchErrorInvalidState(b.state) + NumError: int64(inserts.Len()), + }, err } - var ( - multiErr xerrors.MultiError - result WriteBatchResult - ) - inserts.ForEachID(func(writesForID WriteBatchEntryByBlockStartAndID) { - // all writes are guaranteed to have the same ID by this point, and further - // we're guaranteed that at least a single element exists in the slice. - id := writesForID[0].ID - tags := writesForID[0].Tags - - if id == nil { - return - } - - // define some helper functions to help keep the code below cleaner. - failFn := func(err error) { - multiErr = multiErr.Add(err) - result.NumError += int64(len(writesForID)) - // finalize all refs - WriteBatchEntriesFinalizer(writesForID).Finalize() - } - successFn := func() { - result.NumSuccess += int64(len(writesForID)) - // mark the first ref success (can mark any ref success here, because they're backed by - // by the same entry). Could also mark all of them success but it wouldn't buy us anything. - writesForID[0].OnIndexSeries.OnIndexSuccess(xtime.ToUnixNano(b.startTime)) - // we do need to finalize all refs as each is an extra inc we need to dec - WriteBatchEntriesFinalizer(writesForID).Finalize() - } - - contains, err := b.segment.ContainsID(id.Bytes()) - if contains && err == nil { - // can early terminate as the active segment already has the ID - successFn() - return - } - - // NB(prateek): we delay the conversion from ident types -> doc as we want to minimize the allocs - // of idents until we're sure we actually need to index a series. This helps keep memory usage low - // when we receive a large spike of new metrics. - d, err := convert.FromMetric(id, tags) - if err != nil { - failFn(err) - return - } + err := b.segment.InsertBatch(m3ninxindex.Batch{ + Docs: inserts.PendingDocs(), + AllowPartialUpdates: true, + }) + if err == nil { + inserts.MarkUnmarkedEntriesSuccess() + return WriteBatchResult{ + NumSuccess: int64(inserts.Len()), + }, nil + } - // now actually perform the insert - if _, err := b.segment.Insert(d); err != nil { - failFn(err) - return - } + partialErr, ok := err.(*m3ninxindex.BatchPartialError) + if !ok { // should never happen + err := b.unknownWriteBatchInvariantError(err) + // NB: marking all the inserts as failure, cause we don't know which ones failed + inserts.MarkUnmarkedEntriesError(err) + return WriteBatchResult{NumError: int64(inserts.Len())}, err + } - successFn() - }) + numErr := len(partialErr.Errs()) + for _, err := range partialErr.Errs() { + // Avoid marking these as success + inserts.MarkUnmarkedEntryError(err.Err, err.Idx) + } - return result, multiErr.FinalError() + // mark all non-error inserts success, so we don't repeatedly index them + inserts.MarkUnmarkedEntriesSuccess() + return WriteBatchResult{ + NumSuccess: int64(inserts.Len() - numErr), + NumError: int64(numErr), + }, partialErr } func (b *block) executorWithRLock() (search.Executor, error) { diff --git a/storage/index/index_mock.go b/storage/index/index_mock.go index 9d153f1acc..14f76e9dcf 100644 --- a/storage/index/index_mock.go +++ b/storage/index/index_mock.go @@ -252,7 +252,7 @@ func (mr *MockBlockMockRecorder) Tick(arg0 interface{}) *gomock.Call { } // WriteBatch mocks base method -func (m *MockBlock) WriteBatch(arg0 WriteBatchEntryByBlockStartAndID) (WriteBatchResult, error) { +func (m *MockBlock) WriteBatch(arg0 *WriteBatch) (WriteBatchResult, error) { ret := m.ctrl.Call(m, "WriteBatch", arg0) ret0, _ := ret[0].(WriteBatchResult) ret1, _ := ret[1].(error) diff --git a/storage/index/types.go b/storage/index/types.go index d9e92fefe1..544386cc71 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -21,7 +21,8 @@ package index import ( - "bytes" + "fmt" + "sort" "time" "github.com/m3db/m3db/clock" @@ -138,7 +139,7 @@ type Block interface { EndTime() time.Time // WriteBatch writes a batch of provided entries. - WriteBatch(WriteBatchEntryByBlockStartAndID) (WriteBatchResult, error) + WriteBatch(inserts *WriteBatch) (WriteBatchResult, error) // Query resolves the given query into known IDs. Query( @@ -178,112 +179,296 @@ type BlockTickResult struct { NumDocs int64 } -// WriteBatchEntry captures a document to index, and the lifecycle hooks to call thereafter. -type WriteBatchEntry struct { - ID ident.ID - Tags ident.Tags - Timestamp time.Time - OnIndexSeries OnIndexSeries +// WriteBatch is a batch type that allows for building of a slice of documents +// with metadata in a separate slice, this allows the documents slice to be +// passed to the segment to batch insert without having to copy into a buffer +// again. +type WriteBatch struct { + opts WriteBatchOptions + sortBy writeBatchSortBy + + entries []WriteBatchEntry + docs []doc.Document } -// WriteBatchEntriesFinalizer is a utility type to provide syntactic sugar to finalize references in the slice. -type WriteBatchEntriesFinalizer []WriteBatchEntry +type writeBatchSortBy uint -// Finalize finalizes all the references in the provided slice. -func (w WriteBatchEntriesFinalizer) Finalize() { - for _, entry := range w { - if entry.OnIndexSeries != nil { - entry.OnIndexSeries.OnIndexFinalize(xtime.ToUnixNano(entry.Timestamp)) - } - } -} +const ( + writeBatchSortByUnmarkedAndBlockStart writeBatchSortBy = iota + writeBatchSortByEnqueued +) -// WriteBatchEntryByBlockStart implements sort.Interface for WriteBatchEntry slices -// based on the Timestamp and ID fields. -type WriteBatchEntryByBlockStartAndID []WriteBatchEntry +// WriteBatchOptions is a set of options required for a write batch. +type WriteBatchOptions struct { + InitialCapacity int + IndexBlockSize time.Duration +} -func (w WriteBatchEntryByBlockStartAndID) Len() int { return len(w) } -func (w WriteBatchEntryByBlockStartAndID) Swap(i, j int) { w[i], w[j] = w[j], w[i] } -func (w WriteBatchEntryByBlockStartAndID) Less(i, j int) bool { - if !w[i].Timestamp.Equal(w[j].Timestamp) { - return w[i].Timestamp.Before(w[j].Timestamp) +// NewWriteBatch creates a new write batch. +func NewWriteBatch(opts WriteBatchOptions) *WriteBatch { + return &WriteBatch{ + opts: opts, + entries: make([]WriteBatchEntry, 0, opts.InitialCapacity), + docs: make([]doc.Document, 0, opts.InitialCapacity), } +} - if w[j].ID == nil { - return false - } +// Append appends an entry with accompanying document. +func (b *WriteBatch) Append( + entry WriteBatchEntry, + doc doc.Document, +) { + // Set private WriteBatchEntry fields + entry.enqueuedIdx = len(b.entries) + entry.result = WriteBatchEntryResult{} + + // Append + b.entries = append(b.entries, entry) + b.docs = append(b.docs, doc) +} - if w[i].ID == nil { // i.e. w[j] != nil - return true - } +// ForEachWriteBatchEntryFn allows a caller to perform an operation for each +// batch entry. +type ForEachWriteBatchEntryFn func( + idx int, + entry WriteBatchEntry, + doc doc.Document, + result WriteBatchEntryResult, +) - // i.e. both w[i] and w[j] are != nil - return bytes.Compare(w[i].ID.Bytes(), w[j].ID.Bytes()) < 0 +// ForEach allows a caller to perform an operation for each batch entry. +func (b *WriteBatch) ForEach(fn ForEachWriteBatchEntryFn) { + for idx, entry := range b.entries { + fn(idx, entry, b.docs[idx], entry.Result()) + } } -// ForEachBlockStartFn is lambda to iterate over WriteBatchEntry(s) a single blockStart at a time. -type ForEachBlockStartFn func(timestamp time.Time, writes WriteBatchEntryByBlockStartAndID) +// ForEachWriteBatchByBlockStartFn allows a caller to perform an operation with +// reference to a restricted set of the write batch for each unique block +// start. +type ForEachWriteBatchByBlockStartFn func( + blockStart time.Time, + batch *WriteBatch, +) -// ForEachIDFn is lambda to iterate over WriteBatchEntry(s) a single ID at a time. -type ForEachIDFn func(writes WriteBatchEntryByBlockStartAndID) +// ForEachUnmarkedBatchByBlockStart allows a caller to perform an operation +// with reference to a restricted set of the write batch for each unique block +// start for entries that have not been marked completed yet. +func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( + fn ForEachWriteBatchByBlockStartFn, +) { + // Ensure sorted correctly first + b.SortByUnmarkedAndIndexBlockStart() + + // What we do is a little funky but least alloc intensive, essentially we mutate + // this batch and then restore the pointers to the original docs after. + allEntries := b.entries + allDocs := b.docs + defer func() { + b.entries = allEntries + b.docs = allDocs + }() -// ForEachBlockStart iterates over the provided WriteBatchEntryByBlockStartAndID, and calls `fn` on each -// group of elements with the same blockStart. -func (w WriteBatchEntryByBlockStartAndID) ForEachBlockStart(fn ForEachBlockStartFn) { var ( - startIdx = 0 - lastTime time.Time + blockSize = b.opts.IndexBlockSize + startIdx = 0 + lastBlockStart xtime.UnixNano ) - for i := 0; i < len(w); i++ { - elem := w[i] - if !elem.Timestamp.Equal(lastTime) { - lastTime = elem.Timestamp - // We only want to call the the ForEachBlockStartFn once we have calculated the entire group, + for i := range b.entries { + if b.entries[i].OnIndexSeries == nil { + // Hit a marked done entry + b.entries = allEntries[startIdx:i] + b.docs = allDocs[startIdx:i] + if len(b.entries) != 0 { + fn(lastBlockStart.ToTime(), b) + } + return + } + + blockStart := b.entries[i].indexBlockStart(blockSize) + if !blockStart.Equal(lastBlockStart) { + prevLastBlockStart := lastBlockStart.ToTime() + lastBlockStart = blockStart + // We only want to call the the ForEachUnmarkedBatchByBlockStart once we have calculated the entire group, // i.e. once we have gone past the last element for a given blockStart, but the first element // in the slice is a special case because we are always starting a new group at that point. if i == 0 { continue } - fn(w[startIdx].Timestamp, w[startIdx:i]) + b.entries = allEntries[startIdx:i] + b.docs = allDocs[startIdx:i] + fn(prevLastBlockStart, b) startIdx = i } } // spill over - if startIdx < len(w) { - fn(w[startIdx].Timestamp, w[startIdx:]) + if startIdx < len(b.entries) { + b.entries = allEntries[startIdx:] + b.docs = allDocs[startIdx:] + fn(lastBlockStart.ToTime(), b) } } -// ForEachID iterates over the provided WriteBatchEntryByBlockStartAndID, and calls `fn` on each -// group of elements with the same ID. -func (w WriteBatchEntryByBlockStartAndID) ForEachID(fn ForEachIDFn) { - var ( - startIdx = 0 - lastBytes []byte - ) - for i := 0; i < len(w); i++ { - elem := w[i] - var elemBytes []byte - if elem.ID != nil { - elemBytes = elem.ID.Bytes() +// PendingDocs returns all the docs in this batch that are unmarked. +func (b *WriteBatch) PendingDocs() []doc.Document { + // Ensure sorted by unmarked first + b.SortByUnmarkedAndIndexBlockStart() + + numUnmarked := 0 + for i := range b.entries { + if b.entries[i].OnIndexSeries == nil { + break } - // TODO(prateek): need to write a test to ensure this handles slice with nil IDs correctly - if !bytes.Equal(lastBytes, elemBytes) { - lastBytes = elemBytes - // We only want to call the the ForEachID once we have calculated the entire group, - // i.e. once we have gone past the last element for a given ID, but the first element - // in the slice is a special case because we are always starting a new group at that point. - if i == 0 { - continue - } - fn(w[startIdx:i]) - startIdx = i + numUnmarked++ + } + return b.docs[:numUnmarked] +} + +// NumErrs returns the number of errors encountered by the batch. +func (b *WriteBatch) NumErrs() int { + errs := 0 + for _, entry := range b.entries { + if entry.result.Err != nil { + errs++ } } - // spill over - if startIdx < len(w) { - fn(w[startIdx:]) + return errs +} + +// Reset resets the batch for use. +func (b *WriteBatch) Reset() { + // Memset optimizations + var entryZeroed WriteBatchEntry + for i := range b.entries { + b.entries[i] = entryZeroed + } + b.entries = b.entries[:0] + var docZeroed doc.Document + for i := range b.docs { + b.docs[i] = docZeroed + } + b.docs = b.docs[:0] +} + +// SortByUnmarkedAndIndexBlockStart sorts the batch by unmarked first and then +// by index block start time. +func (b *WriteBatch) SortByUnmarkedAndIndexBlockStart() { + b.sortBy = writeBatchSortByUnmarkedAndBlockStart + sort.Sort(b) +} + +// SortByEnqueued sorts the entries and documents back to the sort order they +// were enqueued as. +func (b *WriteBatch) SortByEnqueued() { + b.sortBy = writeBatchSortByEnqueued + sort.Sort(b) +} + +// MarkUnmarkedEntriesSuccess marks all unmarked entries as success. +func (b *WriteBatch) MarkUnmarkedEntriesSuccess() { + for idx := range b.entries { + if b.entries[idx].OnIndexSeries != nil { + blockStart := b.entries[idx].indexBlockStart(b.opts.IndexBlockSize) + b.entries[idx].OnIndexSeries.OnIndexSuccess(blockStart) + b.entries[idx].OnIndexSeries.OnIndexFinalize(blockStart) + b.entries[idx].result = WriteBatchEntryResult{Err: nil} + } + } +} + +// MarkUnmarkedEntriesError marks all unmarked entries as error. +func (b *WriteBatch) MarkUnmarkedEntriesError(err error) { + for idx := range b.entries { + if b.entries[idx].OnIndexSeries != nil { + blockStart := b.entries[idx].indexBlockStart(b.opts.IndexBlockSize) + b.entries[idx].OnIndexSeries.OnIndexFinalize(blockStart) + b.entries[idx].OnIndexSeries = nil + b.entries[idx].result = WriteBatchEntryResult{Err: err} + } + } +} + +// MarkUnmarkedEntryError marks an unmarked entry at index as error. +func (b *WriteBatch) MarkUnmarkedEntryError( + err error, + idx int, +) { + if b.entries[idx].OnIndexSeries != nil { + blockStart := b.entries[idx].indexBlockStart(b.opts.IndexBlockSize) + b.entries[idx].OnIndexSeries.OnIndexFinalize(blockStart) + b.entries[idx].OnIndexSeries = nil + b.entries[idx].result = WriteBatchEntryResult{Err: err} + } +} + +// Ensure that WriteBatch meets the sort interface +var _ sort.Interface = (*WriteBatch)(nil) + +// Len returns the length of the batch. +func (b *WriteBatch) Len() int { + return len(b.entries) +} + +// Swap will swap two entries and the corresponding docs. +func (b *WriteBatch) Swap(i, j int) { + b.entries[i], b.entries[j] = b.entries[j], b.entries[i] + b.docs[i], b.docs[j] = b.docs[j], b.docs[i] +} + +// Less returns whether an entry appears before another depending +// on the type of sort and . +func (b *WriteBatch) Less(i, j int) bool { + if b.sortBy == writeBatchSortByEnqueued { + return b.entries[i].enqueuedIdx < b.entries[j].enqueuedIdx + } + if b.sortBy != writeBatchSortByUnmarkedAndBlockStart { + panic(fmt.Errorf("unexpected sort by: %d", b.sortBy)) + } + + if b.entries[i].OnIndexSeries != nil && b.entries[j].OnIndexSeries == nil { + // This other entry has already been marked and this hasn't + return true + } + if b.entries[i].OnIndexSeries == nil && b.entries[j].OnIndexSeries != nil { + // This entry has already been marked and other hasn't + return false } + + // They're either both unmarked or marked + blockStartI := b.entries[i].indexBlockStart(b.opts.IndexBlockSize) + blockStartJ := b.entries[j].indexBlockStart(b.opts.IndexBlockSize) + return blockStartI.Before(blockStartJ) +} + +// WriteBatchEntry represents the metadata accompanying the document that is +// being inserted. +type WriteBatchEntry struct { + // Timestamp is the timestamp that this entry should be indexed for + Timestamp time.Time + // OnIndexSeries is a listener/callback for when this entry is marked done + // it is set to nil when the entry is marked done + OnIndexSeries OnIndexSeries + // enqueuedIdx is the idx of the entry when originally enqueued by the call + // to append on the write batch + enqueuedIdx int + // result is the result for this entry which is updated when marked done + result WriteBatchEntryResult +} + +// WriteBatchEntryResult represents a result. +type WriteBatchEntryResult struct { + Err error +} + +func (e WriteBatchEntry) indexBlockStart( + indexBlockSize time.Duration, +) xtime.UnixNano { + return xtime.ToUnixNano(e.Timestamp.Truncate(indexBlockSize)) +} + +// Result returns the result for this entry. +func (e WriteBatchEntry) Result() WriteBatchEntryResult { + return e.result } // Options control the Indexing knobs. diff --git a/storage/index_insert_queue.go b/storage/index_insert_queue.go index 0b01aa40e0..12e7e5ea94 100644 --- a/storage/index_insert_queue.go +++ b/storage/index_insert_queue.go @@ -154,7 +154,7 @@ func (q *nsIndexInsertQueue) insertLoop() { } func (q *nsIndexInsertQueue) InsertBatch( - entries []index.WriteBatchEntry, + batch *index.WriteBatch, ) (*sync.WaitGroup, error) { windowNanos := q.nowFn().Truncate(time.Second).UnixNano() @@ -175,7 +175,8 @@ func (q *nsIndexInsertQueue) InsertBatch( return nil, errNewSeriesIndexRateLimitExceeded } } - q.currBatch.inserts = append(q.currBatch.inserts, entries) + batchLen := batch.Len() + q.currBatch.inserts = append(q.currBatch.inserts, batch) wg := q.currBatch.wg q.Unlock() @@ -186,7 +187,7 @@ func (q *nsIndexInsertQueue) InsertBatch( // Loop busy, already ready to consume notification } - q.metrics.numPending.Inc(int64(len(entries))) + q.metrics.numPending.Inc(int64(batchLen)) return wg, nil } @@ -227,11 +228,11 @@ func (q *nsIndexInsertQueue) Stop() error { return nil } -type nsIndexInsertBatchFn func(inserts [][]index.WriteBatchEntry) +type nsIndexInsertBatchFn func(inserts []*index.WriteBatch) type nsIndexInsertBatch struct { wg *sync.WaitGroup - inserts [][]index.WriteBatchEntry + inserts []*index.WriteBatch } func (b *nsIndexInsertBatch) Reset() { diff --git a/storage/shard.go b/storage/shard.go index 722ecf6ffc..89d60e0894 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -46,6 +46,7 @@ import ( "github.com/m3db/m3db/storage/series/lookup" "github.com/m3db/m3db/ts" "github.com/m3db/m3db/x/xio" + "github.com/m3db/m3ninx/doc" xclose "github.com/m3db/m3x/close" "github.com/m3db/m3x/context" xerrors "github.com/m3db/m3x/errors" @@ -791,7 +792,8 @@ func (s *dbShard) writeAndIndex( commitLogSeriesUniqueIndex = entry.Index if err == nil && shouldReverseIndex { if entry.NeedsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) { - err = s.insertSeriesForIndexing(entry, timestamp, opts.writeNewSeriesAsync) + err = s.insertSeriesForIndexingAsyncBatched(entry, timestamp, + opts.writeNewSeriesAsync) } } // release the reference we got on entry from `writableSeries` @@ -981,7 +983,7 @@ type insertAsyncResult struct { entry *lookup.Entry } -func (s *dbShard) insertSeriesForIndexing( +func (s *dbShard) insertSeriesForIndexingAsyncBatched( entry *lookup.Entry, timestamp time.Time, async bool, @@ -1198,7 +1200,11 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // Perform any indexing, pending writes or pending retrieved blocks outside of lock ctx := s.contextPool.Get() // TODO(prateek): pool this type - indexBatch := make([]index.WriteBatchEntry, 0, numPendingIndexing) + indexBlockSize := s.namespace.Options().IndexOptions().BlockSize() + indexBatch := index.NewWriteBatch(index.WriteBatchOptions{ + InitialCapacity: numPendingIndexing, + IndexBlockSize: indexBlockSize, + }) for i := range inserts { var ( entry = inserts[i].entry @@ -1219,12 +1225,20 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // increment the ref on the entry, as the original one was transferred to the // this method (insertSeriesBatch) via `entryRefCountIncremented` mechanism. entry.OnIndexPrepare() - indexBatch = append(indexBatch, index.WriteBatchEntry{ - ID: entry.Series.ID(), - Tags: entry.Series.Tags(), + + var d doc.Document + d.ID = entry.Series.ID().Bytes() // IDs from shard entries are always set NoFinalize + d.Fields = make(doc.Fields, 0, len(entry.Series.Tags().Values())) + for _, tag := range entry.Series.Tags().Values() { + d.Fields = append(d.Fields, doc.Field{ + Name: tag.Name.Bytes(), // Tags from shard entries are always set NoFinalize + Value: tag.Value.Bytes(), // Tags from shard entries are always set NoFinalize + }) + } + indexBatch.Append(index.WriteBatchEntry{ Timestamp: pendingIndex.timestamp, OnIndexSeries: entry, - }) + }, d) } if inserts[i].opts.hasPendingRetrievedBlock { @@ -1239,7 +1253,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { var err error // index all requested entries in batch. - if len(indexBatch) != 0 { + if indexBatch.Len() > 0 { err = s.reverseIndex.WriteBatch(indexBatch) } diff --git a/storage/types.go b/storage/types.go index 27b1404bfd..4c3d26f3d5 100644 --- a/storage/types.go +++ b/storage/types.go @@ -436,7 +436,7 @@ type namespaceIndex interface { // WriteBatch indexes the provided entries. WriteBatch( - entries []index.WriteBatchEntry, + batch *index.WriteBatch, ) error // Query resolves the given query into known IDs. @@ -482,7 +482,7 @@ type namespaceIndexInsertQueue interface { // inserts to the index asynchronously. It executes the provided callbacks // based on the result of the execution. The returned wait group can be used // if the insert is required to be synchronous. - InsertBatch(entries []index.WriteBatchEntry) (*sync.WaitGroup, error) + InsertBatch(batch *index.WriteBatch) (*sync.WaitGroup, error) } // databaseBootstrapManager manages the bootstrap process. From a7a3944a5c59a34abc15e19a670c537e7790df1c Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 19:41:41 -0400 Subject: [PATCH 16/33] Restore the logging of indexing errors --- storage/index.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/storage/index.go b/storage/index.go index b1821314eb..ed2db070a9 100644 --- a/storage/index.go +++ b/storage/index.go @@ -340,6 +340,9 @@ func (i *nsIndex) writeBatchForBlockStartWithRLock( if numErr := result.NumError; numErr != 0 { i.metrics.AsyncInsertErrors.Inc(numErr) } + if err != nil { + i.logger.Errorf("error writing to index block: %v", err) + } } // Bootstrap bootstraps the index with the provide blocks. From 716cdd716c6dacb32710a59d208943fb0fa6fd41 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 13 May 2018 22:24:42 -0400 Subject: [PATCH 17/33] Fix detecting if indexing succeeded or failed --- storage/shard.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/shard.go b/storage/shard.go index 5030efb67f..08ac03f5f6 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -1017,7 +1017,7 @@ func (s *dbShard) insertSeriesForIndexingAsyncBatched( // if indexing in sync mode, wait till we're done and ensure we have have indexed the entry wg.Wait() - if entry.IndexedForBlockStart(indexBlockStart) { + if !entry.IndexedForBlockStart(indexBlockStart) { // i.e. indexing failed return fmt.Errorf("internal error: unable to index series") } From d7b89ddc74e5afb93e19983d2546fe2d743cf917 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 01:21:04 -0400 Subject: [PATCH 18/33] Encode commit log metadata before queueing commit log write --- persist/fs/commitlog/types.go | 69 +++++++++++++++++++++++++++++++++- persist/fs/commitlog/writer.go | 43 ++++++--------------- storage/index/types.go | 8 ++-- storage/options.go | 31 +++++++++++++++ storage/shard.go | 30 +++++++++++++++ storage/types.go | 13 +++++++ 6 files changed, 157 insertions(+), 37 deletions(-) diff --git a/persist/fs/commitlog/types.go b/persist/fs/commitlog/types.go index 13ebb3f75f..69494aaea4 100644 --- a/persist/fs/commitlog/types.go +++ b/persist/fs/commitlog/types.go @@ -25,7 +25,11 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/persist/fs" + "github.com/m3db/m3db/persist/fs/msgpack" + "github.com/m3db/m3db/persist/schema" + "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/ts" + "github.com/m3db/m3x/checked" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" @@ -100,10 +104,73 @@ type Series struct { ID ident.ID // Tags are the series tags - Tags ident.Tags // FOLLOWUP(prateek): wire Tags to commit log writer + Tags ident.Tags // Shard is the shard the series belongs to Shard uint32 + + encodedLogMetadata checked.Bytes +} + +// EncodeLogMetadata eagerly encodes the log metadata and +// caches it on the Series for use by the commit log writer. +func (s *Series) EncodeLogMetadata( + encoder *msgpack.Encoder, + tagsEncoder serialize.TagEncoder, + tagsIter ident.TagsIterator, + bytesPool pool.CheckedBytesPool, +) error { + if s.encodedLogMetadata != nil { + // Already encoded + return nil + } + + var ( + tags = s.Tags + encodedTags []byte + ) + + if len(tags.Values()) > 0 { + tagsIter.Reset(tags) + tagsEncoder.Reset() + err := tagsEncoder.Encode(tagsIter) + if err != nil { + return err + } + + encodedTagsChecked, ok := tagsEncoder.Data() + if !ok { + return errTagEncoderDataNotAvailable + } + + encodedTags = encodedTagsChecked.Bytes() + } + + var metadata schema.LogMetadata + metadata.ID = s.ID.Data().Bytes() + metadata.Namespace = s.Namespace.Data().Bytes() + metadata.Shard = s.Shard + metadata.EncodedTags = encodedTags + + encoder.Reset() + if err := encoder.EncodeLogMetadata(metadata); err != nil { + return err + } + + encoded := encoder.Bytes() + bytes := bytesPool.Get(len(encoded)) + bytes.IncRef() + bytes.AppendAll(encoded) + s.encodedLogMetadata = bytes + return nil +} + +func (s *Series) finalizeEncoded() { + if s.encodedLogMetadata != nil { + s.encodedLogMetadata.DecRef() + s.encodedLogMetadata.Finalize() + s.encodedLogMetadata = nil + } } // Options represents the options for the commit log diff --git a/persist/fs/commitlog/writer.go b/persist/fs/commitlog/writer.go index 0507f5dbf1..ee3deda28f 100644 --- a/persist/fs/commitlog/writer.go +++ b/persist/fs/commitlog/writer.go @@ -36,6 +36,7 @@ import ( "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/ts" "github.com/m3db/m3x/ident" + "github.com/m3db/m3x/pool" xtime "github.com/m3db/m3x/time" ) @@ -98,6 +99,7 @@ type writer struct { metadataEncoder *msgpack.Encoder tagEncoder serialize.TagEncoder tagSliceIter ident.TagsIterator + bytesPool pool.CheckedBytesPool } func newCommitLogWriter( @@ -105,7 +107,6 @@ func newCommitLogWriter( opts Options, ) commitLogWriter { shouldFsync := opts.Strategy() == StrategyWriteWait - return &writer{ filePathPrefix: opts.FilesystemOptions().FilePathPrefix(), newFileMode: opts.FilesystemOptions().NewFileMode(), @@ -120,6 +121,7 @@ func newCommitLogWriter( metadataEncoder: msgpack.NewEncoder(), tagEncoder: opts.FilesystemOptions().TagEncoderPool().Get(), tagSliceIter: ident.NewTagsIterator(ident.Tags{}), + bytesPool: opts.BytesPool(), } } @@ -176,39 +178,13 @@ func (w *writer) Write( seen := w.seen.Test(uint(series.UniqueIndex)) if !seen { - var ( - tags = series.Tags - encodedTags []byte - ) - - if tags.Values() != nil { - w.tagSliceIter.Reset(tags) - w.tagEncoder.Reset() - err := w.tagEncoder.Encode(w.tagSliceIter) - if err != nil { - return err - } - - encodedTagsChecked, ok := w.tagEncoder.Data() - if !ok { - return errTagEncoderDataNotAvailable - } - - encodedTags = encodedTagsChecked.Bytes() - } - - // If "idx" likely hasn't been written to commit log - // yet we need to include series metadata - var metadata schema.LogMetadata - metadata.ID = series.ID.Data().Bytes() - metadata.Namespace = series.Namespace.Data().Bytes() - metadata.Shard = series.Shard - metadata.EncodedTags = encodedTags - w.metadataEncoder.Reset() - if err := w.metadataEncoder.EncodeLogMetadata(metadata); err != nil { + // Encode or use pre-encoded metadata + err := series.EncodeLogMetadata(w.metadataEncoder, w.tagEncoder, + w.tagSliceIter, w.bytesPool) + if err != nil { return err } - logEntry.Metadata = w.metadataEncoder.Bytes() + logEntry.Metadata = series.encodedLogMetadata.Bytes() } logEntry.Timestamp = datapoint.Timestamp.UnixNano() @@ -223,6 +199,9 @@ func (w *writer) Write( return err } + // Finalize any encoded metadata + // series.finalizeEncoded() + if !seen { // Record we have written this series and metadata to this commit log w.seen.Set(uint(series.UniqueIndex)) diff --git a/storage/index/types.go b/storage/index/types.go index 83ad4a3c86..7dd13c2819 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -274,8 +274,8 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( startIdx = 0 lastBlockStart xtime.UnixNano ) - for i := range b.entries { - if b.entries[i].OnIndexSeries == nil { + for i := range allEntries { + if allEntries[i].OnIndexSeries == nil { // Hit a marked done entry b.entries = allEntries[startIdx:i] b.docs = allDocs[startIdx:i] @@ -285,7 +285,7 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( return } - blockStart := b.entries[i].indexBlockStart(blockSize) + blockStart := allEntries[i].indexBlockStart(blockSize) if !blockStart.Equal(lastBlockStart) { prevLastBlockStart := lastBlockStart.ToTime() lastBlockStart = blockStart @@ -303,7 +303,7 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( } // spill over - if startIdx < len(b.entries) { + if startIdx < len(allEntries) { b.entries = allEntries[startIdx:] b.docs = allDocs[startIdx:] fn(lastBlockStart.ToTime(), b) diff --git a/storage/options.go b/storage/options.go index 0ac21842d8..3c91438142 100644 --- a/storage/options.go +++ b/storage/options.go @@ -33,6 +33,7 @@ import ( "github.com/m3db/m3db/persist/fs/commitlog" "github.com/m3db/m3db/retention" "github.com/m3db/m3db/runtime" + "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/bootstrap" "github.com/m3db/m3db/storage/index" @@ -140,6 +141,8 @@ type options struct { readerIteratorPool encoding.ReaderIteratorPool multiReaderIteratorPool encoding.MultiReaderIteratorPool identifierPool ident.Pool + tagEncoderPool serialize.TagEncoderPool + tagDecoderPool serialize.TagDecoderPool fetchBlockMetadataResultsPool block.FetchBlockMetadataResultsPool fetchBlocksMetadataResultsPool block.FetchBlocksMetadataResultsPool } @@ -154,6 +157,12 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { return pool.NewBytesPool(s, poolOpts) }) bytesPool.Init() + tagEncoderPool := serialize.NewTagEncoderPool( + serialize.NewTagEncoderOptions(), poolOpts) + tagEncoderPool.Init() + tagDecoderPool := serialize.NewTagDecoderPool( + serialize.NewTagDecoderOptions(), poolOpts) + tagDecoderPool.Init() seriesOpts := series.NewOptions() o := &options{ clockOpts: clock.NewOptions(), @@ -188,6 +197,8 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { TagsPoolOptions: poolOpts, TagsIteratorPoolOptions: poolOpts, }), + tagEncoderPool: tagEncoderPool, + tagDecoderPool: tagDecoderPool, fetchBlockMetadataResultsPool: block.NewFetchBlockMetadataResultsPool(poolOpts, 0), fetchBlocksMetadataResultsPool: block.NewFetchBlocksMetadataResultsPool(poolOpts, 0), } @@ -588,6 +599,26 @@ func (o *options) IdentifierPool() ident.Pool { return o.identifierPool } +func (o *options) SetTagEncoderPool(value serialize.TagEncoderPool) Options { + opts := *o + opts.tagEncoderPool = value + return &opts +} + +func (o *options) TagEncoderPool() serialize.TagEncoderPool { + return o.tagEncoderPool +} + +func (o *options) SetTagDecoderPool(value serialize.TagDecoderPool) Options { + opts := *o + opts.tagDecoderPool = value + return &opts +} + +func (o *options) TagDecoderPool() serialize.TagDecoderPool { + return o.tagDecoderPool +} + func (o *options) SetFetchBlockMetadataResultsPool(value block.FetchBlockMetadataResultsPool) Options { opts := *o opts.fetchBlockMetadataResultsPool = value diff --git a/storage/shard.go b/storage/shard.go index 08ac03f5f6..b45affe57c 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -35,6 +35,7 @@ import ( "github.com/m3db/m3db/persist" "github.com/m3db/m3db/persist/fs" "github.com/m3db/m3db/persist/fs/commitlog" + "github.com/m3db/m3db/persist/fs/msgpack" "github.com/m3db/m3db/retention" "github.com/m3db/m3db/runtime" "github.com/m3db/m3db/storage/block" @@ -199,6 +200,20 @@ type shardSnapshotState struct { lastSuccessfulSnapshot time.Time } +var _msgpackEncoderPool = &sync.Pool{ + New: func() interface{} { + return msgpack.NewEncoder() + }, +} + +func msgpackEncoderPoolGet() *msgpack.Encoder { + return _msgpackEncoderPool.Get().(*msgpack.Encoder) +} + +func msgpackEncoderPoolPut(v *msgpack.Encoder) { + _msgpackEncoderPool.Put(v) +} + func newDatabaseShard( namespaceMetadata namespace.Metadata, shard uint32, @@ -838,6 +853,21 @@ func (s *dbShard) writeAndIndex( Shard: s.shard, } + // Prepare the log metadata by encoding it before putting it in the queue + // so the encoding work can be done in parallel instead of synchronously + // once in the queue + encoder := msgpackEncoderPoolGet() + tagsEncoder := s.opts.TagEncoderPool().Get() + tagsIter := s.opts.IdentifierPool().TagsIterator() + err = series.EncodeLogMetadata(encoder, tagsEncoder, + tagsIter, s.opts.BytesPool()) + msgpackEncoderPoolPut(encoder) + tagsEncoder.Finalize() + tagsIter.Close() + if err != nil { + return err + } + datapoint := ts.Datapoint{ Timestamp: timestamp, Value: value, diff --git a/storage/types.go b/storage/types.go index 4c3d26f3d5..ad9ad4c339 100644 --- a/storage/types.go +++ b/storage/types.go @@ -30,6 +30,7 @@ import ( "github.com/m3db/m3db/persist" "github.com/m3db/m3db/persist/fs/commitlog" "github.com/m3db/m3db/runtime" + "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/sharding" "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/bootstrap" @@ -806,6 +807,18 @@ type Options interface { // IDPool returns the ID pool. IdentifierPool() ident.Pool + // SetTagEncoderPool sets the tag encoder pool + SetTagEncoderPool(value serialize.TagEncoderPool) Options + + // TagEncoderPool returns the tag encoder pool + TagEncoderPool() serialize.TagEncoderPool + + // SetTagDecoderPool sets the tag decoder pool + SetTagDecoderPool(value serialize.TagDecoderPool) Options + + // TagDecoderPool returns the tag decoder pool + TagDecoderPool() serialize.TagDecoderPool + // SetFetchBlockMetadataResultsPool sets the fetchBlockMetadataResultsPool SetFetchBlockMetadataResultsPool(value block.FetchBlockMetadataResultsPool) Options From 7c5363b4e07d307e277b900e1894bba21d0a601e Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 01:49:18 -0400 Subject: [PATCH 19/33] Finalize the encoded tags buffer after writing commit log entry --- persist/fs/commitlog/writer.go | 2 +- services/m3dbnode/server/server.go | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/persist/fs/commitlog/writer.go b/persist/fs/commitlog/writer.go index ee3deda28f..fbac313fc8 100644 --- a/persist/fs/commitlog/writer.go +++ b/persist/fs/commitlog/writer.go @@ -200,7 +200,7 @@ func (w *writer) Write( } // Finalize any encoded metadata - // series.finalizeEncoded() + series.finalizeEncoded() if !seen { // Record we have written this series and metadata to this commit log diff --git a/services/m3dbnode/server/server.go b/services/m3dbnode/server/server.go index c2ffc23500..535c45aa49 100644 --- a/services/m3dbnode/server/server.go +++ b/services/m3dbnode/server/server.go @@ -252,6 +252,10 @@ func Run(runOpts RunOptions) { poolOptions(policy.TagDecoderPool, scope.SubScope("tag-decoder-pool"))) tagDecoderPool.Init() + opts = opts. + SetTagEncoderPool(tagEncoderPool). + SetTagDecoderPool(tagDecoderPool) + fsopts := fs.NewOptions(). SetClockOptions(opts.ClockOptions()). SetInstrumentOptions(opts.InstrumentOptions(). From 1352f2dd7e276a243990a6202ce0cfe06e3c126f Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 02:17:07 -0400 Subject: [PATCH 20/33] Revert "Finalize the encoded tags buffer after writing commit log entry" This reverts commit 7c5363b4e07d307e277b900e1894bba21d0a601e. --- persist/fs/commitlog/writer.go | 2 +- services/m3dbnode/server/server.go | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/persist/fs/commitlog/writer.go b/persist/fs/commitlog/writer.go index fbac313fc8..ee3deda28f 100644 --- a/persist/fs/commitlog/writer.go +++ b/persist/fs/commitlog/writer.go @@ -200,7 +200,7 @@ func (w *writer) Write( } // Finalize any encoded metadata - series.finalizeEncoded() + // series.finalizeEncoded() if !seen { // Record we have written this series and metadata to this commit log diff --git a/services/m3dbnode/server/server.go b/services/m3dbnode/server/server.go index 535c45aa49..c2ffc23500 100644 --- a/services/m3dbnode/server/server.go +++ b/services/m3dbnode/server/server.go @@ -252,10 +252,6 @@ func Run(runOpts RunOptions) { poolOptions(policy.TagDecoderPool, scope.SubScope("tag-decoder-pool"))) tagDecoderPool.Init() - opts = opts. - SetTagEncoderPool(tagEncoderPool). - SetTagDecoderPool(tagDecoderPool) - fsopts := fs.NewOptions(). SetClockOptions(opts.ClockOptions()). SetInstrumentOptions(opts.InstrumentOptions(). From 3cbbd1f8329c545dfe37cfd70fdd8790f9d02967 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 02:17:12 -0400 Subject: [PATCH 21/33] Revert "Encode commit log metadata before queueing commit log write" This reverts commit d7b89ddc74e5afb93e19983d2546fe2d743cf917. --- persist/fs/commitlog/types.go | 69 +--------------------------------- persist/fs/commitlog/writer.go | 43 +++++++++++++++------ storage/index/types.go | 8 ++-- storage/options.go | 31 --------------- storage/shard.go | 30 --------------- storage/types.go | 13 ------- 6 files changed, 37 insertions(+), 157 deletions(-) diff --git a/persist/fs/commitlog/types.go b/persist/fs/commitlog/types.go index 69494aaea4..13ebb3f75f 100644 --- a/persist/fs/commitlog/types.go +++ b/persist/fs/commitlog/types.go @@ -25,11 +25,7 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/persist/fs" - "github.com/m3db/m3db/persist/fs/msgpack" - "github.com/m3db/m3db/persist/schema" - "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/ts" - "github.com/m3db/m3x/checked" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" @@ -104,73 +100,10 @@ type Series struct { ID ident.ID // Tags are the series tags - Tags ident.Tags + Tags ident.Tags // FOLLOWUP(prateek): wire Tags to commit log writer // Shard is the shard the series belongs to Shard uint32 - - encodedLogMetadata checked.Bytes -} - -// EncodeLogMetadata eagerly encodes the log metadata and -// caches it on the Series for use by the commit log writer. -func (s *Series) EncodeLogMetadata( - encoder *msgpack.Encoder, - tagsEncoder serialize.TagEncoder, - tagsIter ident.TagsIterator, - bytesPool pool.CheckedBytesPool, -) error { - if s.encodedLogMetadata != nil { - // Already encoded - return nil - } - - var ( - tags = s.Tags - encodedTags []byte - ) - - if len(tags.Values()) > 0 { - tagsIter.Reset(tags) - tagsEncoder.Reset() - err := tagsEncoder.Encode(tagsIter) - if err != nil { - return err - } - - encodedTagsChecked, ok := tagsEncoder.Data() - if !ok { - return errTagEncoderDataNotAvailable - } - - encodedTags = encodedTagsChecked.Bytes() - } - - var metadata schema.LogMetadata - metadata.ID = s.ID.Data().Bytes() - metadata.Namespace = s.Namespace.Data().Bytes() - metadata.Shard = s.Shard - metadata.EncodedTags = encodedTags - - encoder.Reset() - if err := encoder.EncodeLogMetadata(metadata); err != nil { - return err - } - - encoded := encoder.Bytes() - bytes := bytesPool.Get(len(encoded)) - bytes.IncRef() - bytes.AppendAll(encoded) - s.encodedLogMetadata = bytes - return nil -} - -func (s *Series) finalizeEncoded() { - if s.encodedLogMetadata != nil { - s.encodedLogMetadata.DecRef() - s.encodedLogMetadata.Finalize() - s.encodedLogMetadata = nil - } } // Options represents the options for the commit log diff --git a/persist/fs/commitlog/writer.go b/persist/fs/commitlog/writer.go index ee3deda28f..0507f5dbf1 100644 --- a/persist/fs/commitlog/writer.go +++ b/persist/fs/commitlog/writer.go @@ -36,7 +36,6 @@ import ( "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/ts" "github.com/m3db/m3x/ident" - "github.com/m3db/m3x/pool" xtime "github.com/m3db/m3x/time" ) @@ -99,7 +98,6 @@ type writer struct { metadataEncoder *msgpack.Encoder tagEncoder serialize.TagEncoder tagSliceIter ident.TagsIterator - bytesPool pool.CheckedBytesPool } func newCommitLogWriter( @@ -107,6 +105,7 @@ func newCommitLogWriter( opts Options, ) commitLogWriter { shouldFsync := opts.Strategy() == StrategyWriteWait + return &writer{ filePathPrefix: opts.FilesystemOptions().FilePathPrefix(), newFileMode: opts.FilesystemOptions().NewFileMode(), @@ -121,7 +120,6 @@ func newCommitLogWriter( metadataEncoder: msgpack.NewEncoder(), tagEncoder: opts.FilesystemOptions().TagEncoderPool().Get(), tagSliceIter: ident.NewTagsIterator(ident.Tags{}), - bytesPool: opts.BytesPool(), } } @@ -178,13 +176,39 @@ func (w *writer) Write( seen := w.seen.Test(uint(series.UniqueIndex)) if !seen { - // Encode or use pre-encoded metadata - err := series.EncodeLogMetadata(w.metadataEncoder, w.tagEncoder, - w.tagSliceIter, w.bytesPool) - if err != nil { + var ( + tags = series.Tags + encodedTags []byte + ) + + if tags.Values() != nil { + w.tagSliceIter.Reset(tags) + w.tagEncoder.Reset() + err := w.tagEncoder.Encode(w.tagSliceIter) + if err != nil { + return err + } + + encodedTagsChecked, ok := w.tagEncoder.Data() + if !ok { + return errTagEncoderDataNotAvailable + } + + encodedTags = encodedTagsChecked.Bytes() + } + + // If "idx" likely hasn't been written to commit log + // yet we need to include series metadata + var metadata schema.LogMetadata + metadata.ID = series.ID.Data().Bytes() + metadata.Namespace = series.Namespace.Data().Bytes() + metadata.Shard = series.Shard + metadata.EncodedTags = encodedTags + w.metadataEncoder.Reset() + if err := w.metadataEncoder.EncodeLogMetadata(metadata); err != nil { return err } - logEntry.Metadata = series.encodedLogMetadata.Bytes() + logEntry.Metadata = w.metadataEncoder.Bytes() } logEntry.Timestamp = datapoint.Timestamp.UnixNano() @@ -199,9 +223,6 @@ func (w *writer) Write( return err } - // Finalize any encoded metadata - // series.finalizeEncoded() - if !seen { // Record we have written this series and metadata to this commit log w.seen.Set(uint(series.UniqueIndex)) diff --git a/storage/index/types.go b/storage/index/types.go index 7dd13c2819..83ad4a3c86 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -274,8 +274,8 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( startIdx = 0 lastBlockStart xtime.UnixNano ) - for i := range allEntries { - if allEntries[i].OnIndexSeries == nil { + for i := range b.entries { + if b.entries[i].OnIndexSeries == nil { // Hit a marked done entry b.entries = allEntries[startIdx:i] b.docs = allDocs[startIdx:i] @@ -285,7 +285,7 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( return } - blockStart := allEntries[i].indexBlockStart(blockSize) + blockStart := b.entries[i].indexBlockStart(blockSize) if !blockStart.Equal(lastBlockStart) { prevLastBlockStart := lastBlockStart.ToTime() lastBlockStart = blockStart @@ -303,7 +303,7 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( } // spill over - if startIdx < len(allEntries) { + if startIdx < len(b.entries) { b.entries = allEntries[startIdx:] b.docs = allDocs[startIdx:] fn(lastBlockStart.ToTime(), b) diff --git a/storage/options.go b/storage/options.go index 3c91438142..0ac21842d8 100644 --- a/storage/options.go +++ b/storage/options.go @@ -33,7 +33,6 @@ import ( "github.com/m3db/m3db/persist/fs/commitlog" "github.com/m3db/m3db/retention" "github.com/m3db/m3db/runtime" - "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/bootstrap" "github.com/m3db/m3db/storage/index" @@ -141,8 +140,6 @@ type options struct { readerIteratorPool encoding.ReaderIteratorPool multiReaderIteratorPool encoding.MultiReaderIteratorPool identifierPool ident.Pool - tagEncoderPool serialize.TagEncoderPool - tagDecoderPool serialize.TagDecoderPool fetchBlockMetadataResultsPool block.FetchBlockMetadataResultsPool fetchBlocksMetadataResultsPool block.FetchBlocksMetadataResultsPool } @@ -157,12 +154,6 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { return pool.NewBytesPool(s, poolOpts) }) bytesPool.Init() - tagEncoderPool := serialize.NewTagEncoderPool( - serialize.NewTagEncoderOptions(), poolOpts) - tagEncoderPool.Init() - tagDecoderPool := serialize.NewTagDecoderPool( - serialize.NewTagDecoderOptions(), poolOpts) - tagDecoderPool.Init() seriesOpts := series.NewOptions() o := &options{ clockOpts: clock.NewOptions(), @@ -197,8 +188,6 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { TagsPoolOptions: poolOpts, TagsIteratorPoolOptions: poolOpts, }), - tagEncoderPool: tagEncoderPool, - tagDecoderPool: tagDecoderPool, fetchBlockMetadataResultsPool: block.NewFetchBlockMetadataResultsPool(poolOpts, 0), fetchBlocksMetadataResultsPool: block.NewFetchBlocksMetadataResultsPool(poolOpts, 0), } @@ -599,26 +588,6 @@ func (o *options) IdentifierPool() ident.Pool { return o.identifierPool } -func (o *options) SetTagEncoderPool(value serialize.TagEncoderPool) Options { - opts := *o - opts.tagEncoderPool = value - return &opts -} - -func (o *options) TagEncoderPool() serialize.TagEncoderPool { - return o.tagEncoderPool -} - -func (o *options) SetTagDecoderPool(value serialize.TagDecoderPool) Options { - opts := *o - opts.tagDecoderPool = value - return &opts -} - -func (o *options) TagDecoderPool() serialize.TagDecoderPool { - return o.tagDecoderPool -} - func (o *options) SetFetchBlockMetadataResultsPool(value block.FetchBlockMetadataResultsPool) Options { opts := *o opts.fetchBlockMetadataResultsPool = value diff --git a/storage/shard.go b/storage/shard.go index b45affe57c..08ac03f5f6 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -35,7 +35,6 @@ import ( "github.com/m3db/m3db/persist" "github.com/m3db/m3db/persist/fs" "github.com/m3db/m3db/persist/fs/commitlog" - "github.com/m3db/m3db/persist/fs/msgpack" "github.com/m3db/m3db/retention" "github.com/m3db/m3db/runtime" "github.com/m3db/m3db/storage/block" @@ -200,20 +199,6 @@ type shardSnapshotState struct { lastSuccessfulSnapshot time.Time } -var _msgpackEncoderPool = &sync.Pool{ - New: func() interface{} { - return msgpack.NewEncoder() - }, -} - -func msgpackEncoderPoolGet() *msgpack.Encoder { - return _msgpackEncoderPool.Get().(*msgpack.Encoder) -} - -func msgpackEncoderPoolPut(v *msgpack.Encoder) { - _msgpackEncoderPool.Put(v) -} - func newDatabaseShard( namespaceMetadata namespace.Metadata, shard uint32, @@ -853,21 +838,6 @@ func (s *dbShard) writeAndIndex( Shard: s.shard, } - // Prepare the log metadata by encoding it before putting it in the queue - // so the encoding work can be done in parallel instead of synchronously - // once in the queue - encoder := msgpackEncoderPoolGet() - tagsEncoder := s.opts.TagEncoderPool().Get() - tagsIter := s.opts.IdentifierPool().TagsIterator() - err = series.EncodeLogMetadata(encoder, tagsEncoder, - tagsIter, s.opts.BytesPool()) - msgpackEncoderPoolPut(encoder) - tagsEncoder.Finalize() - tagsIter.Close() - if err != nil { - return err - } - datapoint := ts.Datapoint{ Timestamp: timestamp, Value: value, diff --git a/storage/types.go b/storage/types.go index ad9ad4c339..4c3d26f3d5 100644 --- a/storage/types.go +++ b/storage/types.go @@ -30,7 +30,6 @@ import ( "github.com/m3db/m3db/persist" "github.com/m3db/m3db/persist/fs/commitlog" "github.com/m3db/m3db/runtime" - "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/sharding" "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/bootstrap" @@ -807,18 +806,6 @@ type Options interface { // IDPool returns the ID pool. IdentifierPool() ident.Pool - // SetTagEncoderPool sets the tag encoder pool - SetTagEncoderPool(value serialize.TagEncoderPool) Options - - // TagEncoderPool returns the tag encoder pool - TagEncoderPool() serialize.TagEncoderPool - - // SetTagDecoderPool sets the tag decoder pool - SetTagDecoderPool(value serialize.TagDecoderPool) Options - - // TagDecoderPool returns the tag decoder pool - TagDecoderPool() serialize.TagDecoderPool - // SetFetchBlockMetadataResultsPool sets the fetchBlockMetadataResultsPool SetFetchBlockMetadataResultsPool(value block.FetchBlockMetadataResultsPool) Options From a6998c74327d34df1b3ebeeccbeefd2d1b5e8fab Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 09:34:28 -0400 Subject: [PATCH 22/33] Fix the ForEachUnmarkedBatchByBlockStart logic during blocko rotation --- storage/index/types.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/storage/index/types.go b/storage/index/types.go index 83ad4a3c86..7dd13c2819 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -274,8 +274,8 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( startIdx = 0 lastBlockStart xtime.UnixNano ) - for i := range b.entries { - if b.entries[i].OnIndexSeries == nil { + for i := range allEntries { + if allEntries[i].OnIndexSeries == nil { // Hit a marked done entry b.entries = allEntries[startIdx:i] b.docs = allDocs[startIdx:i] @@ -285,7 +285,7 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( return } - blockStart := b.entries[i].indexBlockStart(blockSize) + blockStart := allEntries[i].indexBlockStart(blockSize) if !blockStart.Equal(lastBlockStart) { prevLastBlockStart := lastBlockStart.ToTime() lastBlockStart = blockStart @@ -303,7 +303,7 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( } // spill over - if startIdx < len(b.entries) { + if startIdx < len(allEntries) { b.entries = allEntries[startIdx:] b.docs = allDocs[startIdx:] fn(lastBlockStart.ToTime(), b) From 27b9e3499b548cc5386b0449a517dfe0b6edf263 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 15:08:58 -0400 Subject: [PATCH 23/33] Purge merged buffer encoders and bootstrap blocks --- storage/series/buffer.go | 20 ++------------------ storage/series/buffer_test.go | 25 ++++++++++++++++++++++--- 2 files changed, 24 insertions(+), 21 deletions(-) diff --git a/storage/series/buffer.go b/storage/series/buffer.go index 1d52ec766f..087b6ef01b 100644 --- a/storage/series/buffer.go +++ b/storage/series/buffer.go @@ -527,13 +527,10 @@ func (b *dbBufferBucket) resetTo( // Close the old context if we're resetting for use b.finalize() - ctx := b.opts.ContextPool().Get() - bopts := b.opts.DatabaseBlockOptions() encoder := bopts.EncoderPool().Get() encoder.Reset(start, bopts.DatabaseBlockAllocSize()) - b.ctx = ctx b.start = start b.encoders = append(b.encoders, inOrderEncoder{ lastWriteAt: timeZero, @@ -548,11 +545,6 @@ func (b *dbBufferBucket) resetTo( func (b *dbBufferBucket) finalize() { b.resetEncoders() b.resetBootstrapped() - if b.ctx != nil { - // Close the old context - b.ctx.Close() - } - b.ctx = nil } func (b *dbBufferBucket) canRead() bool { @@ -625,10 +617,6 @@ func (b *dbBufferBucket) write( } func (b *dbBufferBucket) streams(ctx context.Context) []xio.BlockReader { - // NB(r): Ensure we don't call any closers before the operation - // started by the passed context completes. - b.ctx.DependsOn(ctx) - streams := make([]xio.BlockReader, 0, len(b.bootstrapped)+len(b.encoders)) for i := range b.bootstrapped { @@ -680,9 +668,7 @@ func (b *dbBufferBucket) resetEncoders() { for i := range b.encoders { // Register when this bucket resets we close the encoder encoder := b.encoders[i].encoder - if b.ctx != nil { - b.ctx.RegisterCloser(encoder) - } + encoder.Close() b.encoders[i] = zeroed } b.encoders = b.encoders[:0] @@ -691,9 +677,7 @@ func (b *dbBufferBucket) resetEncoders() { func (b *dbBufferBucket) resetBootstrapped() { for i := range b.bootstrapped { bl := b.bootstrapped[i] - if b.ctx != nil { - b.ctx.RegisterCloser(bl) - } + bl.Close() } b.bootstrapped = nil } diff --git a/storage/series/buffer_test.go b/storage/series/buffer_test.go index 1d8ed715ec..f120b1ef80 100644 --- a/storage/series/buffer_test.go +++ b/storage/series/buffer_test.go @@ -22,6 +22,7 @@ package series import ( "io" + "io/ioutil" "sort" "testing" "time" @@ -676,6 +677,20 @@ func TestBufferReadEncodedValidAfterDrain(t *testing.T) { require.Equal(t, 2, len(encoders)) + var ( + expectedData []byte + streams []io.Reader + ) + for _, encoder := range encoders { + stream := encoder.Stream() + clone, err := stream.Clone() + require.NoError(t, err) + streams = append(streams, clone) + data, err := ioutil.ReadAll(stream) + require.NoError(t, err) + expectedData = append(expectedData, data...) + } + assert.Equal(t, true, buffer.NeedsDrain()) assert.Equal(t, 0, len(drained)) @@ -690,10 +705,14 @@ func TestBufferReadEncodedValidAfterDrain(t *testing.T) { assert.Equal(t, false, buffer.NeedsDrain()) assert.Equal(t, 1, len(drained)) - // Ensure all encoders still has data - for _, encoder := range encoders { - assert.NotNil(t, encoder.Stream()) + // Ensure all streams that were taken reference to still has data + var actualData []byte + for _, stream := range streams { + data, err := ioutil.ReadAll(stream) + require.NoError(t, err) + actualData = append(actualData, data...) } + assert.Equal(t, expectedData, actualData) } func TestBufferTickReordersOutOfOrderBuffers(t *testing.T) { From b5991f2dc9ff2d53e6ba726e4cc40e0fe6b8d567 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 15:57:08 -0400 Subject: [PATCH 24/33] Remove depends on pattern from database block --- storage/block/block.go | 34 ++++++--------------- storage/block/block_test.go | 61 +------------------------------------ storage/series/buffer.go | 1 - 3 files changed, 11 insertions(+), 85 deletions(-) diff --git a/storage/block/block.go b/storage/block/block.go index 9194afbfc4..640b90a444 100644 --- a/storage/block/block.go +++ b/storage/block/block.go @@ -45,7 +45,6 @@ type dbBlock struct { sync.RWMutex opts Options - ctx context.Context startUnixNanos int64 segment ts.Segment length int @@ -88,7 +87,6 @@ func NewDatabaseBlock( ) DatabaseBlock { b := &dbBlock{ opts: opts, - ctx: opts.ContextPool().Get(), startUnixNanos: start.UnixNano(), blockSize: blockSize, closed: false, @@ -109,7 +107,6 @@ func NewRetrievableDatabaseBlock( ) DatabaseBlock { b := &dbBlock{ opts: opts, - ctx: opts.ContextPool().Get(), startUnixNanos: start.UnixNano(), blockSize: blockSize, closed: false, @@ -217,7 +214,6 @@ func (b *dbBlock) Stream(blocker context.Context) (xio.BlockReader, error) { return xio.EmptyBlockReader, errReadFromClosedBlock } - b.ctx.DependsOn(blocker) stream, err := b.stream(blocker) if err != nil { return xio.EmptyBlockReader, err @@ -294,8 +290,6 @@ func (b *dbBlock) ResetRetrievable( } func (b *dbBlock) stream(ctx context.Context) (xio.BlockReader, error) { - b.ctx.DependsOn(ctx) - start := b.startWithLock() // If the block retrieve ID is set then it must be retrieved @@ -309,14 +303,21 @@ func (b *dbBlock) stream(ctx context.Context) (xio.BlockReader, error) { return xio.EmptyBlockReader, err } } else { + // Take a copy to avoid heavy depends on cycle segmentReader := b.opts.SegmentReaderPool().Get() + data := b.opts.BytesPool().Get(b.segment.Len()) + data.IncRef() + data.AppendAll(b.segment.Head.Bytes()) + data.AppendAll(b.segment.Tail.Bytes()) + data.DecRef() + segmentReader.Reset(ts.NewSegment(data, nil, ts.FinalizeHead)) + ctx.RegisterFinalizer(segmentReader) + blockReader = xio.BlockReader{ SegmentReader: segmentReader, Start: start, BlockSize: b.blockSize, } - blockReader.Reset(ts.NewSegment(b.segment.Head, b.segment.Tail, ts.FinalizeNone)) - ctx.RegisterFinalizer(segmentReader) } return blockReader, nil @@ -346,10 +347,6 @@ func (b *dbBlock) forceMergeWithLock(ctx context.Context, stream xio.SegmentRead } func (b *dbBlock) resetNewBlockStartWithLock(start time.Time, blockSize time.Duration) { - if !b.closed { - b.ctx.Close() - } - b.ctx = b.opts.ContextPool().Get() b.startUnixNanos = start.UnixNano() b.blockSize = blockSize atomic.StoreInt64(&b.lastReadUnixNanos, 0) @@ -365,8 +362,6 @@ func (b *dbBlock) resetSegmentWithLock(seg ts.Segment) { b.retriever = nil b.retrieveID = nil b.wasRetrievedFromDisk = false - - b.ctx.RegisterFinalizer(&seg) } func (b *dbBlock) resetRetrievableWithLock( @@ -391,16 +386,7 @@ func (b *dbBlock) Close() { } b.closed = true - - // NB(xichen): we use the worker pool to close the context instead doing - // an asynchronous context close to explicitly control the context closing - // concurrency. This is particularly important during a node removal because - // all the shards are removed at once, causing a goroutine explosion without - // limiting the concurrency. We also cannot do a blocking close here because - // the block may be closed before the underlying context is closed, which - // causes a deadlock if the block and the underlying context are closed - // from within the same goroutine. - b.opts.CloseContextWorkers().Go(b.ctx.BlockingClose) + b.segment.Finalize() b.resetMergeTargetWithLock() if pool := b.opts.DatabaseBlockPool(); pool != nil { diff --git a/storage/block/block_test.go b/storage/block/block_test.go index 6376b19b65..491f6ee7e7 100644 --- a/storage/block/block_test.go +++ b/storage/block/block_test.go @@ -21,7 +21,6 @@ package block import ( - "sync/atomic" "testing" "time" @@ -31,7 +30,6 @@ import ( "github.com/m3db/m3db/ts" "github.com/m3db/m3db/x/xio" "github.com/m3db/m3x/context" - "github.com/m3db/m3x/resource" xtime "github.com/m3db/m3x/time" "github.com/golang/mock/gomock" @@ -72,17 +70,6 @@ func validateBlocks(t *testing.T, blocks *databaseSeriesBlocks, minTime, maxTime } } -func closeTestDatabaseBlock(t *testing.T, block *dbBlock) { - var finished uint32 - block.ctx = block.opts.ContextPool().Get() - block.ctx.RegisterFinalizer(resource.FinalizerFn(func() { atomic.StoreUint32(&finished, 1) })) - block.Close() - // waiting for the goroutine that closes context to finish - for atomic.LoadUint32(&finished) == 0 { - time.Sleep(100 * time.Millisecond) - } -} - func TestDatabaseBlockReadFromClosedBlock(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -91,7 +78,7 @@ func TestDatabaseBlockReadFromClosedBlock(t *testing.T) { defer ctx.Close() block := testDatabaseBlock(ctrl) - closeTestDatabaseBlock(t, block) + block.Close() _, err := block.Stream(ctx) require.Equal(t, errReadFromClosedBlock, err) } @@ -112,52 +99,6 @@ type testDatabaseBlockFn func(block *dbBlock) type testDatabaseBlockAssertionFn func(t *testing.T, block *dbBlock) -func testDatabaseBlockWithDependentContext( - t *testing.T, - f testDatabaseBlockFn, - af testDatabaseBlockAssertionFn, -) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - block := testDatabaseBlock(ctrl) - depCtx := block.opts.ContextPool().Get() - - // register a dependent context here - _, err := block.Stream(depCtx) - require.NoError(t, err) - - var finished uint32 - block.ctx.RegisterFinalizer(resource.FinalizerFn(func() { - atomic.StoreUint32(&finished, 1) - })) - f(block) - - // sleep a bit to let the goroutine run - time.Sleep(200 * time.Millisecond) - require.Equal(t, uint32(0), atomic.LoadUint32(&finished)) - - // now closing the dependent context - depCtx.Close() - for atomic.LoadUint32(&finished) == 0 { - time.Sleep(200 * time.Millisecond) - } - - af(t, block) -} - -func TestDatabaseBlockResetNormalWithDependentContext(t *testing.T) { - f := func(block *dbBlock) { block.Reset(time.Now(), time.Hour, ts.Segment{}) } - af := func(t *testing.T, block *dbBlock) { require.False(t, block.closed) } - testDatabaseBlockWithDependentContext(t, f, af) -} - -func TestDatabaseBlockCloseNormalWithDependentContext(t *testing.T) { - f := func(block *dbBlock) { block.Close() } - af := func(t *testing.T, block *dbBlock) { require.True(t, block.closed) } - testDatabaseBlockWithDependentContext(t, f, af) -} - type segmentReaderFinalizeCounter struct { xio.SegmentReader // Use a pointer so we can update it from the Finalize method diff --git a/storage/series/buffer.go b/storage/series/buffer.go index 087b6ef01b..47a803ea60 100644 --- a/storage/series/buffer.go +++ b/storage/series/buffer.go @@ -506,7 +506,6 @@ func (b *dbBuffer) FetchBlocksMetadata( } type dbBufferBucket struct { - ctx context.Context opts Options start time.Time encoders []inOrderEncoder From 526375d9123b712509f8a74d5dd9d25472ff3819 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 16:14:11 -0400 Subject: [PATCH 25/33] Check buffers are not nil before blindly appending --- storage/block/block.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/storage/block/block.go b/storage/block/block.go index 640b90a444..7be451f60f 100644 --- a/storage/block/block.go +++ b/storage/block/block.go @@ -307,8 +307,12 @@ func (b *dbBlock) stream(ctx context.Context) (xio.BlockReader, error) { segmentReader := b.opts.SegmentReaderPool().Get() data := b.opts.BytesPool().Get(b.segment.Len()) data.IncRef() - data.AppendAll(b.segment.Head.Bytes()) - data.AppendAll(b.segment.Tail.Bytes()) + if b.segment.Head != nil { + data.AppendAll(b.segment.Head.Bytes()) + } + if b.segment.Tail != nil { + data.AppendAll(b.segment.Tail.Bytes()) + } data.DecRef() segmentReader.Reset(ts.NewSegment(data, nil, ts.FinalizeHead)) ctx.RegisterFinalizer(segmentReader) From 7258d1c174718bf0acf848b2c37576f846d3eb7c Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 23:15:19 -0400 Subject: [PATCH 26/33] Use slices into cloned ID when creating tags for new series --- client/host_queue.go | 4 +- client/session.go | 18 ++--- client/session_fetch_bulk_blocks_test.go | 6 +- glide.lock | 6 +- glide.yaml | 2 +- integration/client.go | 4 +- integration/generate/generate.go | 4 +- integration/integration_data_verify.go | 2 +- integration/truncate_namespace_test.go | 2 +- network/server/tchannelthrift/node/service.go | 4 +- persist/fs/commitlog/writer.go | 4 +- persist/fs/index_lookup.go | 2 +- persist/fs/index_lookup_prop_test.go | 2 +- persist/fs/read_write_test.go | 4 +- persist/fs/retriever.go | 2 +- persist/fs/seek.go | 2 +- persist/fs/write.go | 6 +- serialize/decoder.go | 4 +- serialize/decoder_test.go | 8 +-- serialize/encode_decode_prop_test.go | 4 +- serialize/encoder.go | 2 +- sharding/shardset.go | 2 +- .../bootstrapper/commitlog/source.go | 2 +- .../commitlog/source_prop_test.go | 2 +- storage/index/convert/convert.go | 2 +- storage/namespace/metadata.go | 2 +- storage/series/series.go | 2 - storage/shard.go | 66 ++++++++++++++----- storage/types.go | 2 +- tools/read_ids/main/main.go | 2 +- 30 files changed, 101 insertions(+), 73 deletions(-) diff --git a/client/host_queue.go b/client/host_queue.go index 095d36a1a7..2606252f96 100644 --- a/client/host_queue.go +++ b/client/host_queue.go @@ -286,7 +286,7 @@ func (q *queue) asyncTaggedWrite( // TODO(r): Use a worker pool to avoid creating new go routines for async writes go func() { req := q.writeTaggedBatchRawRequestPool.Get() - req.NameSpace = namespace.Data().Bytes() + req.NameSpace = namespace.Bytes() req.Elements = elems // NB(r): Defer is slow in the hot path unfortunately @@ -351,7 +351,7 @@ func (q *queue) asyncWrite( // TODO(r): Use a worker pool to avoid creating new go routines for async writes go func() { req := q.writeBatchRawRequestPool.Get() - req.NameSpace = namespace.Data().Bytes() + req.NameSpace = namespace.Bytes() req.Elements = elems // NB(r): Defer is slow in the hot path unfortunately diff --git a/client/session.go b/client/session.go index d91ea94a90..e8bc2a9a48 100644 --- a/client/session.go +++ b/client/session.go @@ -976,7 +976,7 @@ func (s *session) writeAttemptWithRLock( wop := s.pools.writeOperation.Get() wop.namespace = nsID wop.shardID = s.state.topoMap.ShardSet().Lookup(tsID) - wop.request.ID = tsID.Data().Bytes() + wop.request.ID = tsID.Bytes() wop.request.Datapoint.Value = value wop.request.Datapoint.Timestamp = timestamp wop.request.Datapoint.TimestampTimeType = timeType @@ -986,7 +986,7 @@ func (s *session) writeAttemptWithRLock( wop := s.pools.writeTaggedOperation.Get() wop.namespace = nsID wop.shardID = s.state.topoMap.ShardSet().Lookup(tsID) - wop.request.ID = tsID.Data().Bytes() + wop.request.ID = tsID.Bytes() encodedTagBytes, ok := tagEncoder.Data() if !ok { return nil, 0, 0, errUnableToEncodeTags @@ -1445,7 +1445,7 @@ func (s *session) fetchIDsAttempt( } // Append IDWithNamespace to this request - f.append(namespace.Data().Bytes(), tsID.Data().Bytes(), completionFn) + f.append(namespace.Bytes(), tsID.Bytes(), completionFn) }); err != nil { routeErr = err break @@ -1568,7 +1568,7 @@ func (s *session) Truncate(namespace ident.ID) (int64, error) { ) t := &truncateOp{} - t.request.NameSpace = namespace.Data().Bytes() + t.request.NameSpace = namespace.Bytes() t.completionFn = func(result interface{}, err error) { if err != nil { resultErrLock.Lock() @@ -2036,7 +2036,7 @@ func (s *session) streamBlocksMetadataFromPeer( attemptFn := func(client rpc.TChanNode) error { tctx, _ := thrift.NewContext(s.streamBlocksMetadataBatchTimeout) req := rpc.NewFetchBlocksMetadataRawRequest() - req.NameSpace = namespace.Data().Bytes() + req.NameSpace = namespace.Bytes() req.Shard = int32(shard) req.RangeStart = start.UnixNano() req.RangeEnd = end.UnixNano() @@ -2201,7 +2201,7 @@ func (s *session) streamBlocksMetadataFromPeerV2( attemptFn := func(client rpc.TChanNode) error { tctx, _ := thrift.NewContext(s.streamBlocksMetadataBatchTimeout) req := rpc.NewFetchBlocksMetadataRawV2Request() - req.NameSpace = namespace.Data().Bytes() + req.NameSpace = namespace.Bytes() req.Shard = int32(shard) req.RangeStart = start.UnixNano() req.RangeEnd = end.UnixNano() @@ -2800,7 +2800,7 @@ func (s *session) streamBlocksBatchFromPeer( retention = ropts.RetentionPeriod() earliestBlockStart = nowFn().Add(-retention).Truncate(ropts.BlockSize()) ) - req.NameSpace = namespaceMetadata.ID().Data().Bytes() + req.NameSpace = namespaceMetadata.ID().Bytes() req.Shard = int32(shard) req.Elements = make([]*rpc.FetchBlocksRawRequestElement, 0, len(batch)) for i := range batch { @@ -2809,7 +2809,7 @@ func (s *session) streamBlocksBatchFromPeer( continue // Fell out of retention while we were streaming blocks } req.Elements = append(req.Elements, &rpc.FetchBlocksRawRequestElement{ - ID: batch[i].id.Data().Bytes(), + ID: batch[i].id.Bytes(), Starts: []int64{blockStart.UnixNano()}, }) reqBlocksLen++ @@ -2863,7 +2863,7 @@ func (s *session) streamBlocksBatchFromPeer( } id := batch[i].id - if !bytes.Equal(id.Data().Bytes(), result.Elements[i].ID) { + if !bytes.Equal(id.Bytes(), result.Elements[i].ID) { blocksErr := fmt.Errorf( "stream blocks mismatched ID: expectedID=%s, actualID=%s, indexID=%d, peer=%s", batch[i].id.String(), id.String(), i, peer.Host().String(), diff --git a/client/session_fetch_bulk_blocks_test.go b/client/session_fetch_bulk_blocks_test.go index 4dd8cabe4a..fc9288ec58 100644 --- a/client/session_fetch_bulk_blocks_test.go +++ b/client/session_fetch_bulk_blocks_test.go @@ -2107,7 +2107,7 @@ func expectFetchMetadataAndReturn( ) for j := beginIdx; j < len(result) && j < beginIdx+batchSize; j++ { elem := &rpc.BlocksMetadata{} - elem.ID = result[j].id.Data().Bytes() + elem.ID = result[j].id.Bytes() for k := 0; k < len(result[j].blocks); k++ { bl := &rpc.BlockMetadata{} bl.Start = result[j].blocks[k].start.UnixNano() @@ -2159,7 +2159,7 @@ func expectFetchMetadataAndReturnV2( beginIdx = i * batchSize ) for j := beginIdx; j < len(result) && j < beginIdx+batchSize; j++ { - id := result[j].id.Data().Bytes() + id := result[j].id.Bytes() for k := 0; k < len(result[j].blocks); k++ { bl := &rpc.BlockMetadataV2{} bl.ID = id @@ -2311,7 +2311,7 @@ func expectFetchBlocksAndReturn( ret := &rpc.FetchBlocksRawResult_{} for _, res := range result[i] { blocks := &rpc.Blocks{} - blocks.ID = res.id.Data().Bytes() + blocks.ID = res.id.Bytes() for j := range res.blocks { bl := &rpc.Block{} bl.Start = res.blocks[j].start.UnixNano() diff --git a/glide.lock b/glide.lock index 727d2fb602..8808968aa9 100644 --- a/glide.lock +++ b/glide.lock @@ -1,5 +1,5 @@ -hash: 238562b5d3566785aa16f01f9dcea7d80e0281177fdb88be6036b1f60e6c8744 -updated: 2018-05-13T17:00:00.352679-04:00 +hash: 8f1604427ddcca0987a7771518e61562d5468253a4dde980d53cdafbc1c87637 +updated: 2018-05-14T22:38:20.088778-04:00 imports: - name: github.com/alecthomas/template version: a0175ee3bccc567396460bf5acd36800cb10c49c @@ -236,7 +236,7 @@ imports: - search/query - search/searcher - name: github.com/m3db/m3x - version: df82cd7aacfd8439586821ecd6e073da35bfbb02 + version: 4cfb56e76b0219e593809bd13c54f5a69c764f57 vcs: git subpackages: - checked diff --git a/glide.yaml b/glide.yaml index 54a2ad0856..7e179d4841 100644 --- a/glide.yaml +++ b/glide.yaml @@ -1,7 +1,7 @@ package: github.com/m3db/m3db import: - package: github.com/m3db/m3x - version: df82cd7aacfd8439586821ecd6e073da35bfbb02 + version: 4cfb56e76b0219e593809bd13c54f5a69c764f57 vcs: git subpackages: - checked diff --git a/integration/client.go b/integration/client.go index d8d023b65d..ecb93d091c 100644 --- a/integration/client.go +++ b/integration/client.go @@ -59,7 +59,7 @@ func tchannelClientWriteBatch(client rpc.TChanNode, timeout time.Duration, names for _, series := range seriesList { for _, dp := range series.Data { elem := &rpc.WriteBatchRawRequestElement{ - ID: series.ID.Data().Bytes(), + ID: series.ID.Bytes(), Datapoint: &rpc.Datapoint{ Timestamp: xtime.ToNormalizedTime(dp.Timestamp, time.Second), Value: dp.Value, @@ -72,7 +72,7 @@ func tchannelClientWriteBatch(client rpc.TChanNode, timeout time.Duration, names ctx, _ := thrift.NewContext(timeout) batchReq := &rpc.WriteBatchRawRequest{ - NameSpace: namespace.Data().Bytes(), + NameSpace: namespace.Bytes(), Elements: elems, } return client.WriteBatchRaw(ctx, batchReq) diff --git a/integration/generate/generate.go b/integration/generate/generate.go index 71dd21bed9..83451579d4 100644 --- a/integration/generate/generate.go +++ b/integration/generate/generate.go @@ -36,7 +36,7 @@ import ( func (l SeriesBlock) Len() int { return len(l) } func (l SeriesBlock) Swap(i, j int) { l[i], l[j] = l[j], l[i] } func (l SeriesBlock) Less(i, j int) bool { - return bytes.Compare(l[i].ID.Data().Bytes(), l[j].ID.Data().Bytes()) < 0 + return bytes.Compare(l[i].ID.Bytes(), l[j].ID.Bytes()) < 0 } // Block generates a SeriesBlock based on provided config @@ -120,5 +120,5 @@ func (l SeriesDataPointsByTime) Less(i, j int) bool { if !l[i].Timestamp.Equal(l[j].Timestamp) { return l[i].Timestamp.Before(l[j].Timestamp) } - return bytes.Compare(l[i].ID.Data().Bytes(), l[j].ID.Data().Bytes()) < 0 + return bytes.Compare(l[i].ID.Bytes(), l[j].ID.Bytes()) < 0 } diff --git a/integration/integration_data_verify.go b/integration/integration_data_verify.go index 6d68afe5e4..687e980975 100644 --- a/integration/integration_data_verify.go +++ b/integration/integration_data_verify.go @@ -263,7 +263,7 @@ func compareSeriesList( require.Equal(t, len(expected), len(actual)) for i := range expected { - require.Equal(t, expected[i].ID.Data().Bytes(), actual[i].ID.Data().Bytes()) + require.Equal(t, expected[i].ID.Bytes(), actual[i].ID.Bytes()) require.Equal(t, expected[i].Data, expected[i].Data) } } diff --git a/integration/truncate_namespace_test.go b/integration/truncate_namespace_test.go index f2578598ec..582fd42b87 100644 --- a/integration/truncate_namespace_test.go +++ b/integration/truncate_namespace_test.go @@ -106,7 +106,7 @@ func TestTruncateNamespace(t *testing.T) { log.Debugf("truncate namespace %s", testNamespaces[0]) truncateReq := rpc.NewTruncateRequest() - truncateReq.NameSpace = testNamespaces[0].Data().Bytes() + truncateReq.NameSpace = testNamespaces[0].Bytes() truncated, err := testSetup.truncate(truncateReq) require.NoError(t, err) require.Equal(t, int64(1), truncated) diff --git a/network/server/tchannelthrift/node/service.go b/network/server/tchannelthrift/node/service.go index a821cc503b..0ead00fa93 100644 --- a/network/server/tchannelthrift/node/service.go +++ b/network/server/tchannelthrift/node/service.go @@ -144,7 +144,7 @@ func NewService(db storage.Database, opts tchannelthrift.Options) rpc.TChanNode scope := iopts. MetricsScope(). SubScope("service"). - Tagged(map[string]string{"serviceName": "node"}) + Tagged(map[string]string{"service-name": "node"}) wrapperPoolOpts := pool.NewObjectPoolOptions(). SetSize(checkedBytesPoolSize). @@ -527,7 +527,7 @@ func (s *service) FetchBlocksMetadataRaw(tctx thrift.Context, req *rpc.FetchBloc for _, fetchedMetadata := range fetchedResults { blocksMetadata := s.pools.blocksMetadata.Get() - blocksMetadata.ID = fetchedMetadata.ID.Data().Bytes() + blocksMetadata.ID = fetchedMetadata.ID.Bytes() fetchedMetadataBlocks := fetchedMetadata.Blocks.Results() blocksMetadata.Blocks = s.pools.blockMetadataSlice.Get() diff --git a/persist/fs/commitlog/writer.go b/persist/fs/commitlog/writer.go index 0507f5dbf1..468bd631f6 100644 --- a/persist/fs/commitlog/writer.go +++ b/persist/fs/commitlog/writer.go @@ -200,8 +200,8 @@ func (w *writer) Write( // If "idx" likely hasn't been written to commit log // yet we need to include series metadata var metadata schema.LogMetadata - metadata.ID = series.ID.Data().Bytes() - metadata.Namespace = series.Namespace.Data().Bytes() + metadata.ID = series.ID.Bytes() + metadata.Namespace = series.Namespace.Bytes() metadata.Shard = series.Shard metadata.EncodedTags = encodedTags w.metadataEncoder.Reset() diff --git a/persist/fs/index_lookup.go b/persist/fs/index_lookup.go index 0681aa436d..04e2ae785d 100644 --- a/persist/fs/index_lookup.go +++ b/persist/fs/index_lookup.go @@ -85,7 +85,7 @@ func (il *nearestIndexOffsetLookup) concurrentClone() (*nearestIndexOffsetLookup // In other words, the returned offset can always be used as a starting point to // begin scanning the index file for the desired series. func (il *nearestIndexOffsetLookup) getNearestIndexFileOffset(id ident.ID) (int64, error) { - idBytes := id.Data().Bytes() + idBytes := id.Bytes() min := 0 max := len(il.summaryIDsOffsets) - 1 diff --git a/persist/fs/index_lookup_prop_test.go b/persist/fs/index_lookup_prop_test.go index fd049ae60f..2425453fa2 100644 --- a/persist/fs/index_lookup_prop_test.go +++ b/persist/fs/index_lookup_prop_test.go @@ -49,7 +49,7 @@ func TestIndexLookupWriteRead(t *testing.T) { writes := []generatedWrite{} unique := map[string]struct{}{} for _, write := range input.realWrites { - s := string(write.id.Data().Bytes()) + s := string(write.id.Bytes()) if _, ok := unique[s]; ok { continue } diff --git a/persist/fs/read_write_test.go b/persist/fs/read_write_test.go index 1a30817e0b..12034e0bcd 100644 --- a/persist/fs/read_write_test.go +++ b/persist/fs/read_write_test.go @@ -167,7 +167,7 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim // Verify that the bloomFilter was bootstrapped properly by making sure it // at least contains every ID - assert.True(t, bloomFilter.Test(id.Data().Bytes())) + assert.True(t, bloomFilter.Test(id.Bytes())) id.Finalize() tags.Close() @@ -191,7 +191,7 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim // Verify that the bloomFilter was bootstrapped properly by making sure it // at least contains every ID - assert.True(t, bloomFilter.Test(id.Data().Bytes())) + assert.True(t, bloomFilter.Test(id.Bytes())) id.Finalize() tags.Close() diff --git a/persist/fs/retriever.go b/persist/fs/retriever.go index fac71d4fd8..0b4a833d41 100644 --- a/persist/fs/retriever.go +++ b/persist/fs/retriever.go @@ -400,7 +400,7 @@ func (r *blockRetriever) Stream( // If the ID is not in the seeker's bloom filter, then it's definitely not on // disk and we can return immediately - if !bloomFilter.Test(id.Data().Bytes()) { + if !bloomFilter.Test(id.Bytes()) { // No need to call req.onRetrieve.OnRetrieveBlock if there is no data req.onRetrieved(ts.Segment{}) return req.toBlock(), nil diff --git a/persist/fs/seek.go b/persist/fs/seek.go index dd31eda20a..c563e6b200 100644 --- a/persist/fs/seek.go +++ b/persist/fs/seek.go @@ -414,7 +414,7 @@ func (s *seeker) SeekIndexEntry(id ident.ID) (IndexEntry, error) { stream := msgpack.NewDecoderStream(s.indexMmap[offset:]) s.decoder.Reset(stream) - idBytes := id.Data().Bytes() + idBytes := id.Bytes() // Prevent panic's when we're scanning to the end of the buffer for stream.Remaining() != 0 { entry, err := s.decoder.DecodeIndexEntry() diff --git a/persist/fs/write.go b/persist/fs/write.go index ff76c9e6e5..a14248ee6f 100644 --- a/persist/fs/write.go +++ b/persist/fs/write.go @@ -97,7 +97,7 @@ func (e indexEntries) Len() int { } func (e indexEntries) Less(i, j int) bool { - return bytes.Compare(e[i].id.Data().Bytes(), e[j].id.Data().Bytes()) < 0 + return bytes.Compare(e[i].id.Bytes(), e[j].id.Bytes()) < 0 } func (e indexEntries) Swap(i, j int) { @@ -421,7 +421,7 @@ func (w *writer) writeIndexFileContents( ) defer tagsEncoder.Finalize() for i := range w.indexEntries { - id := w.indexEntries[i].id.Data().Bytes() + id := w.indexEntries[i].id.Bytes() // Need to check if i > 0 or we can never write an empty string ID if i > 0 && bytes.Equal(id, prevID) { // Should never happen, Write() should only be called once per ID @@ -491,7 +491,7 @@ func (w *writer) writeSummariesFileContents( summary := schema.IndexSummary{ Index: w.indexEntries[i].index, - ID: w.indexEntries[i].id.Data().Bytes(), + ID: w.indexEntries[i].id.Bytes(), IndexEntryOffset: w.indexEntries[i].indexFileOffset, } diff --git a/serialize/decoder.go b/serialize/decoder.go index c7dfddeac4..d44eafc7aa 100644 --- a/serialize/decoder.go +++ b/serialize/decoder.go @@ -200,9 +200,9 @@ func (d *decoder) Close() { } func (d *decoder) cloneCurrent() ident.Tag { - name := d.opts.CheckedBytesWrapperPool().Get(d.current.Name.Data().Bytes()) + name := d.opts.CheckedBytesWrapperPool().Get(d.current.Name.Bytes()) d.checkedData.IncRef() - value := d.opts.CheckedBytesWrapperPool().Get(d.current.Value.Data().Bytes()) + value := d.opts.CheckedBytesWrapperPool().Get(d.current.Value.Bytes()) d.checkedData.IncRef() return ident.BinaryTag(name, value) } diff --git a/serialize/decoder_test.go b/serialize/decoder_test.go index 84116f480e..46ec372a50 100644 --- a/serialize/decoder_test.go +++ b/serialize/decoder_test.go @@ -185,8 +185,8 @@ func TestDecodeDuplicateLifecycle(t *testing.T) { for copy.Next() { tag := copy.Current() // keep looping - tag.Name.Data().Bytes() // ensure we can get values too - tag.Value.Data().Bytes() // and don't panic + tag.Name.Bytes() // ensure we can get values too + tag.Value.Bytes() // and don't panic } require.NoError(t, copy.Err()) copy.Close() @@ -206,8 +206,8 @@ func TestDecodeDuplicateIteration(t *testing.T) { for copy.Next() { tag := copy.Current() // keep looping - tag.Name.Data().Bytes() // ensure we can get values too - tag.Value.Data().Bytes() // and don't panic + tag.Name.Bytes() // ensure we can get values too + tag.Value.Bytes() // and don't panic } require.NoError(t, copy.Err()) copy.Close() diff --git a/serialize/encode_decode_prop_test.go b/serialize/encode_decode_prop_test.go index 8493baf2fa..f9e5c3a64b 100644 --- a/serialize/encode_decode_prop_test.go +++ b/serialize/encode_decode_prop_test.go @@ -121,11 +121,11 @@ func tagItersAreEqual(ti1, ti2 ident.TagIterator) (bool, error) { t1, t2 := ti1.Current(), ti2.Current() if !t1.Name.Equal(t2.Name) { return false, fmt.Errorf("tag names are un-equal: %v %v", - t1.Name.Data().Bytes(), t2.Name.Data().Bytes()) + t1.Name.Bytes(), t2.Name.Bytes()) } if !t2.Value.Equal(t2.Value) { return false, fmt.Errorf("tag values are un-equal: %v %v", - t1.Value.Data().Bytes(), t2.Value.Data().Bytes()) + t1.Value.Bytes(), t2.Value.Bytes()) } return tagItersAreEqual(ti1, ti2) diff --git a/serialize/encoder.go b/serialize/encoder.go index b39a1547bf..0d657ccd5f 100644 --- a/serialize/encoder.go +++ b/serialize/encoder.go @@ -163,7 +163,7 @@ func (e *encoder) encodeTag(t ident.Tag) error { } func (e *encoder) encodeID(i ident.ID) error { - d := i.Data().Bytes() + d := i.Bytes() max := int(e.opts.TagSerializationLimits().MaxTagLiteralLength()) if len(d) >= max { return errTagLiteralTooLong diff --git a/sharding/shardset.go b/sharding/shardset.go index 8bb044c4ca..aaee2293b7 100644 --- a/sharding/shardset.go +++ b/sharding/shardset.go @@ -161,6 +161,6 @@ func NewHashGenWithSeed(seed uint32) HashGen { // NewHashFn generates a HashFN based on murmur32 with a given seed func NewHashFn(length int, seed uint32) HashFn { return func(id ident.ID) uint32 { - return murmur3.Sum32WithSeed(id.Data().Bytes(), seed) % uint32(length) + return murmur3.Sum32WithSeed(id.Bytes(), seed) % uint32(length) } } diff --git a/storage/bootstrap/bootstrapper/commitlog/source.go b/storage/bootstrap/bootstrapper/commitlog/source.go index a3f0489957..ad07c49e53 100644 --- a/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/storage/bootstrap/bootstrapper/commitlog/source.go @@ -587,7 +587,7 @@ func (s *commitLogSource) ReadIndex( return nil, err } - exists, err := segment.ContainsID(series.ID.Data().Bytes()) + exists, err := segment.ContainsID(series.ID.Bytes()) if err != nil { return nil, err } diff --git a/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 148a101678..15938ba6cc 100644 --- a/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -292,5 +292,5 @@ func seriesUniqueTags(seriesID, proposedTagKey, proposedTagVal string, includeTa // hashIDToShard generates a HashFn based on murmur32 func hashIDToShard(id ident.ID) uint32 { - return murmur3.Sum32(id.Data().Bytes()) % uint32(maxShards) + return murmur3.Sum32(id.Bytes()) % uint32(maxShards) } diff --git a/storage/index/convert/convert.go b/storage/index/convert/convert.go index 837fb98e67..1b31cda335 100644 --- a/storage/index/convert/convert.go +++ b/storage/index/convert/convert.go @@ -86,7 +86,7 @@ func FromMetricIter(id ident.ID, tags ident.TagIterator) (doc.Document, error) { // any ids provided, as we need to maintain the lifecycle of the indexed // bytes separately from the rest of the storage subsystem. func clone(id ident.ID) []byte { - original := id.Data().Bytes() + original := id.Bytes() clone := make([]byte, len(original)) copy(clone, original) return clone diff --git a/storage/namespace/metadata.go b/storage/namespace/metadata.go index 9dae8a4e77..3bce9539cd 100644 --- a/storage/namespace/metadata.go +++ b/storage/namespace/metadata.go @@ -53,7 +53,7 @@ func NewMetadata(id ident.ID, opts Options) (Metadata, error) { } - copiedID := checked.NewBytes(append([]byte(nil), id.Data().Bytes()...), nil) + copiedID := checked.NewBytes(append([]byte(nil), id.Bytes()...), nil) return &metadata{ id: ident.BinaryID(copiedID), opts: opts, diff --git a/storage/series/series.go b/storage/series/series.go index afb32227b4..68939f0a93 100644 --- a/storage/series/series.go +++ b/storage/series/series.go @@ -733,9 +733,7 @@ func (s *dbSeries) Reset( // of not releasing back an ID to a pool is amortized over // a long period of time. s.id = id - s.id.NoFinalize() s.tags = tags - s.tags.NoFinalize() s.blocks.Reset() s.buffer.Reset(opts) diff --git a/storage/shard.go b/storage/shard.go index 08ac03f5f6..5dfeb37c56 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -21,6 +21,7 @@ package storage import ( + "bytes" "container/list" "errors" "fmt" @@ -946,33 +947,62 @@ func (s *dbShard) tryRetrieveWritableSeries(id ident.ID) ( return nil, opts, nil } -func (s *dbShard) newShardEntry(id ident.ID, tags ident.TagIterator) (*lookup.Entry, error) { - clonedTags, err := s.cloneTags(tags) +func (s *dbShard) newShardEntry( + id ident.ID, + tags ident.TagIterator, +) (*lookup.Entry, error) { + series := s.seriesPool.Get() + // NB(r): As documented in storage/series.DatabaseSeries the series IDs + // are garbage collected, hence we cast the ID to a BytesID that can't be + // finalized. + // Since series are purged so infrequently the overhead of not releasing + // back an ID to a pool is amortized over a long period of time. + clonedID := ident.BytesID(append([]byte(nil), id.Bytes()...)) + + // Inlining tag creation here so its obvious why we can safely index + // into clonedID below + tags = tags.Duplicate() + clonedTags := s.identifierPool.Tags() + + // Avoid finalizing the tags since series will let them be garbage collected + clonedTags.NoFinalize() + + for tags.Next() { + t := tags.Current() + + // NB(r): Optimization for workloads that embed the tags in the ID is to + // just take a ref to them directly, the cloned ID is frozen by casting to + // a BytesID in newShardEntry + var tag ident.Tag + + nameBytes := t.Name.Bytes() + if idx := bytes.Index(clonedID, nameBytes); idx != -1 { + tag.Name = ident.BytesID(clonedID[idx : idx+len(nameBytes)]) + } else { + tag.Name = s.identifierPool.Clone(t.Name) + } + + valueBytes := t.Value.Bytes() + if idx := bytes.Index(clonedID, valueBytes); idx != -1 { + tag.Value = ident.BytesID(clonedID[idx : idx+len(valueBytes)]) + } else { + tag.Value = s.identifierPool.Clone(t.Value) + } + + clonedTags.Append(tag) + } + err := tags.Err() + tags.Close() if err != nil { return nil, err } - series := s.seriesPool.Get() - clonedID := s.identifierPool.Clone(id) + series.Reset(clonedID, clonedTags, s.seriesBlockRetriever, s.seriesOnRetrieveBlock, s, s.seriesOpts) uniqueIndex := s.increasingIndex.nextIndex() return lookup.NewEntry(series, uniqueIndex), nil } -func (s *dbShard) cloneTags(tags ident.TagIterator) (ident.Tags, error) { - tags = tags.Duplicate() - clone := s.identifierPool.Tags() - defer tags.Close() - for tags.Next() { - t := tags.Current() - clone.Append(s.identifierPool.CloneTag(t)) - } - if err := tags.Err(); err != nil { - return ident.Tags{}, err - } - return clone, nil -} - type insertAsyncResult struct { wg *sync.WaitGroup copiedID ident.ID diff --git a/storage/types.go b/storage/types.go index 4c3d26f3d5..1fc54a5e3e 100644 --- a/storage/types.go +++ b/storage/types.go @@ -204,7 +204,7 @@ type NamespacesByID []Namespace func (n NamespacesByID) Len() int { return len(n) } func (n NamespacesByID) Swap(i, j int) { n[i], n[j] = n[j], n[i] } func (n NamespacesByID) Less(i, j int) bool { - return bytes.Compare(n[i].ID().Data().Bytes(), n[j].ID().Data().Bytes()) < 0 + return bytes.Compare(n[i].ID().Bytes(), n[j].ID().Bytes()) < 0 } type databaseNamespace interface { diff --git a/tools/read_ids/main/main.go b/tools/read_ids/main/main.go index 045b0b8f33..2e6b0b7271 100644 --- a/tools/read_ids/main/main.go +++ b/tools/read_ids/main/main.go @@ -100,7 +100,7 @@ func main() { attemptFn := func() error { tctx, _ := thrift.NewContext(60 * time.Second) req := rpc.NewFetchBlocksMetadataRawRequest() - req.NameSpace = ident.StringID(namespace).Data().Bytes() + req.NameSpace = ident.StringID(namespace).Bytes() req.Shard = int32(shard) req.RangeStart = 0 req.RangeEnd = time.Now().Add(365 * 24 * time.Hour).UnixNano() From c2d48e5cb22237baf23e5be10ba6dfaaf7738cc0 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 14 May 2018 23:32:36 -0400 Subject: [PATCH 27/33] Update m3x --- glide.lock | 6 +++--- glide.yaml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/glide.lock b/glide.lock index 8808968aa9..2911112a95 100644 --- a/glide.lock +++ b/glide.lock @@ -1,5 +1,5 @@ -hash: 8f1604427ddcca0987a7771518e61562d5468253a4dde980d53cdafbc1c87637 -updated: 2018-05-14T22:38:20.088778-04:00 +hash: 68ea275b977061da51371dc9dc5f9c1fcd90126b5f4e7b2157a76632f06f4442 +updated: 2018-05-14T23:31:13.635022-04:00 imports: - name: github.com/alecthomas/template version: a0175ee3bccc567396460bf5acd36800cb10c49c @@ -236,7 +236,7 @@ imports: - search/query - search/searcher - name: github.com/m3db/m3x - version: 4cfb56e76b0219e593809bd13c54f5a69c764f57 + version: 80455550b18244f71637a568f097ae929748e5be vcs: git subpackages: - checked diff --git a/glide.yaml b/glide.yaml index 7e179d4841..70098ab0ed 100644 --- a/glide.yaml +++ b/glide.yaml @@ -1,7 +1,7 @@ package: github.com/m3db/m3db import: - package: github.com/m3db/m3x - version: 4cfb56e76b0219e593809bd13c54f5a69c764f57 + version: 80455550b18244f71637a568f097ae929748e5be vcs: git subpackages: - checked From 1318f4b6123b71ab1dcadf9fb1473098897feaf5 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 15 May 2018 03:37:22 -0400 Subject: [PATCH 28/33] Fix a lot of tests, still some not quite there --- storage/index.go | 5 +- storage/index/block.go | 2 +- storage/index/block_test.go | 430 +++++++++++++++++++++-------- storage/index/convert/convert.go | 16 ++ storage/index/for_each_test.go | 107 ++++--- storage/index/types.go | 25 +- storage/index_block_test.go | 116 ++++++-- storage/index_insert_queue_test.go | 60 ++-- storage/index_queue_test.go | 49 ++-- storage/series/lookup/entry.go | 5 + storage/shard.go | 14 + storage/shard_index_test.go | 95 ++++--- storage/shard_ref_count_test.go | 41 ++- storage/shard_test.go | 41 +-- 14 files changed, 691 insertions(+), 315 deletions(-) diff --git a/storage/index.go b/storage/index.go index ed2db070a9..a95c86fd15 100644 --- a/storage/index.go +++ b/storage/index.go @@ -295,9 +295,10 @@ func (i *nsIndex) writeBatches( pastLimit := now.Add(-1 * i.bufferPast) writeBatchFn := i.writeBatchForBlockStartWithRLock for _, batch := range batches { - // Ensure timestamp is not too old/new based on retention policies. + // Ensure timestamp is not too old/new based on retention policies and that + // doc is valid. batch.ForEach(func(idx int, entry index.WriteBatchEntry, - _ doc.Document, _ index.WriteBatchEntryResult) { + d doc.Document, _ index.WriteBatchEntryResult) { if !futureLimit.After(entry.Timestamp) { batch.MarkUnmarkedEntryError(m3dberrors.ErrTooFuture, idx) return diff --git a/storage/index/block.go b/storage/index/block.go index a4f2b5a9f2..bc02270f87 100644 --- a/storage/index/block.go +++ b/storage/index/block.go @@ -352,7 +352,7 @@ func (b *block) writeBatchErrorInvalidState(state blockState) error { } func (b *block) unknownWriteBatchInvariantError(err error) error { - wrappedErr := fmt.Errorf("received non BatchPartialError from m3ninx InsertBatch [%T]", err) + wrappedErr := fmt.Errorf("unexpected non-BatchPartialError from m3ninx InsertBatch: %v", err) instrument.EmitInvariantViolationAndGetLogger(b.opts.InstrumentOptions()).Errorf(wrappedErr.Error()) return wrappedErr } diff --git a/storage/index/block_test.go b/storage/index/block_test.go index 9294022be2..6d3dcb2609 100644 --- a/storage/index/block_test.go +++ b/storage/index/block_test.go @@ -22,6 +22,7 @@ package index import ( "fmt" + "strings" "testing" "time" @@ -32,6 +33,7 @@ import ( "github.com/m3db/m3ninx/index/segment/mem" "github.com/m3db/m3ninx/search" "github.com/m3db/m3x/ident" + xtime "github.com/m3db/m3x/time" "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" @@ -52,59 +54,121 @@ func TestBlockWriteAfterClose(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - b, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + b, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) require.NoError(t, b.Close()) lifecycle := NewMockOnIndexSeries(ctrl) - lifecycle.EXPECT().OnIndexFinalize() - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{OnIndexSeries: lifecycle}, + lifecycle.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: lifecycle, + }, doc.Document{}) + + res, err := b.WriteBatch(batch) require.Error(t, err) require.Equal(t, int64(0), res.NumSuccess) require.Equal(t, int64(1), res.NumError) + + verified := 0 + batch.ForEach(func( + idx int, + entry WriteBatchEntry, + doc doc.Document, + result WriteBatchEntryResult, + ) { + verified++ + require.Error(t, result.Err) + require.Equal(t, errUnableToWriteBlockClosed, result.Err) + }) + require.Equal(t, 1, verified) } func TestBlockWriteAfterSeal(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - b, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + b, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) require.NoError(t, b.Seal()) lifecycle := NewMockOnIndexSeries(ctrl) - lifecycle.EXPECT().OnIndexFinalize() - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{OnIndexSeries: lifecycle}, + lifecycle.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: lifecycle, + }, doc.Document{}) + + res, err := b.WriteBatch(batch) require.Error(t, err) require.Equal(t, int64(0), res.NumSuccess) require.Equal(t, int64(1), res.NumError) + + verified := 0 + batch.ForEach(func( + idx int, + entry WriteBatchEntry, + doc doc.Document, + result WriteBatchEntryResult, + ) { + verified++ + require.Error(t, result.Err) + require.Equal(t, errUnableToWriteBlockSealed, result.Err) + }) + require.Equal(t, 1, verified) } func TestBlockWriteMockSegment(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + blk, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) - end := start.Add(time.Hour) - h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() - h1.EXPECT().OnIndexSuccess(end) + h1.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h1.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) h2 := NewMockOnIndexSeries(ctrl) - h2.EXPECT().OnIndexFinalize() - h2.EXPECT().OnIndexSuccess(end) + h2.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h2.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) seg := segment.NewMockMutableSegment(ctrl) b.segment = seg @@ -115,66 +179,115 @@ func TestBlockWriteMockSegment(t *testing.T) { }, )).Return(nil) - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, - WriteBatchEntry{Document: testDoc1DupeID(), OnIndexSeries: h2}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h1, + }, testDoc1()) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h2, + }, testDoc1DupeID()) + + res, err := b.WriteBatch(batch) require.NoError(t, err) require.Equal(t, int64(2), res.NumSuccess) require.Equal(t, int64(0), res.NumError) } -func TestBlockWriteActualSegment(t *testing.T) { +func TestBlockWriteActualSegmentPartialFailure(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + blk, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) - end := start.Add(time.Hour) - h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() - h1.EXPECT().OnIndexSuccess(end) + h1.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h1.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) h2 := NewMockOnIndexSeries(ctrl) - h2.EXPECT().OnIndexFinalize() + h2.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, - WriteBatchEntry{Document: testDoc1DupeID(), OnIndexSeries: h2}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h1, + }, testDoc1()) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h2, + }, testDoc1DupeID()) + res, err := b.WriteBatch(batch) require.Error(t, err) require.Equal(t, int64(1), res.NumSuccess) require.Equal(t, int64(1), res.NumError) + + verified := 0 + batch.ForEach(func( + idx int, + entry WriteBatchEntry, + doc doc.Document, + result WriteBatchEntryResult, + ) { + verified++ + if idx == 0 { + require.NoError(t, result.Err) + } else { + require.Error(t, result.Err) + require.Equal(t, mem.ErrDuplicateID, result.Err) + } + }) + require.Equal(t, 2, verified) } func TestBlockWriteMockSegmentPartialFailure(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + blk, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) - end := start.Add(time.Hour) seg := segment.NewMockMutableSegment(ctrl) b.segment = seg h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() - h1.EXPECT().OnIndexSuccess(end) + h1.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h1.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) h2 := NewMockOnIndexSeries(ctrl) - h2.EXPECT().OnIndexFinalize() + h2.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + + testErr := fmt.Errorf("random-err") berr := index.NewBatchPartialError() - berr.Add(fmt.Errorf("random-err"), 1) + berr.Add(index.BatchError{testErr, 1}) seg.EXPECT().InsertBatch(index.NewBatchMatcher( index.Batch{ Docs: []doc.Document{testDoc1(), testDoc1DupeID()}, @@ -182,49 +295,55 @@ func TestBlockWriteMockSegmentPartialFailure(t *testing.T) { }, )).Return(berr) - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, - WriteBatchEntry{Document: testDoc1DupeID(), OnIndexSeries: h2}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h1, + }, testDoc1()) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h2, + }, testDoc1DupeID()) + + res, err := b.WriteBatch(batch) require.Error(t, err) require.Equal(t, int64(1), res.NumSuccess) require.Equal(t, int64(1), res.NumError) -} - -func TestBlockWriteActualSegmentPartialFailure(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) - require.NoError(t, err) - b, ok := blk.(*block) - require.True(t, ok) - - end := start.Add(time.Hour) - - h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() - h1.EXPECT().OnIndexSuccess(end) - - h2 := NewMockOnIndexSeries(ctrl) - h2.EXPECT().OnIndexFinalize() - - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, - WriteBatchEntry{Document: testDoc1DupeID(), OnIndexSeries: h2}, + verified := 0 + batch.ForEach(func( + idx int, + entry WriteBatchEntry, + doc doc.Document, + result WriteBatchEntryResult, + ) { + verified++ + if idx == 0 { + require.NoError(t, result.Err) + } else { + require.Error(t, result.Err) + require.Equal(t, testErr, result.Err) + } }) - require.Error(t, err) - require.Equal(t, int64(1), res.NumSuccess) - require.Equal(t, int64(1), res.NumError) + require.Equal(t, 2, verified) } func TestBlockWriteMockSegmentUnexpectedErrorType(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + blk, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) @@ -233,24 +352,48 @@ func TestBlockWriteMockSegmentUnexpectedErrorType(t *testing.T) { b.segment = seg h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() + h1.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) h2 := NewMockOnIndexSeries(ctrl) - h2.EXPECT().OnIndexFinalize() + h2.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + + testErr := fmt.Errorf("random-err") seg.EXPECT().InsertBatch(index.NewBatchMatcher( index.Batch{ Docs: []doc.Document{testDoc1(), testDoc1DupeID()}, AllowPartialUpdates: true, }, - )).Return(fmt.Errorf("random-err")) + )).Return(testErr) - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, - WriteBatchEntry{Document: testDoc1DupeID(), OnIndexSeries: h2}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h1, + }, testDoc1()) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h2, + }, testDoc1DupeID()) + + res, err := b.WriteBatch(batch) require.Error(t, err) require.Equal(t, int64(2), res.NumError) + + verified := 0 + batch.ForEach(func( + idx int, + entry WriteBatchEntry, + doc doc.Document, + result WriteBatchEntryResult, + ) { + verified++ + require.Error(t, result.Err) + require.True(t, strings.Contains(result.Err.Error(), "unexpected non-BatchPartialError")) + }) + require.Equal(t, 2, verified) } func TestBlockQueryAfterClose(t *testing.T) { @@ -794,25 +937,41 @@ func TestBlockE2EInsertQuery(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + blk, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) - end := start.Add(time.Hour) h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() - h1.EXPECT().OnIndexSuccess(end) + h1.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h1.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) h2 := NewMockOnIndexSeries(ctrl) - h2.EXPECT().OnIndexFinalize() - h2.EXPECT().OnIndexSuccess(end) + h2.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h2.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, - WriteBatchEntry{Document: testDoc2(), OnIndexSeries: h2}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h1, + }, testDoc1()) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h2, + }, testDoc2()) + + res, err := b.WriteBatch(batch) require.NoError(t, err) require.Equal(t, int64(2), res.NumSuccess) require.Equal(t, int64(0), res.NumError) @@ -843,25 +1002,41 @@ func TestBlockE2EInsertQueryLimit(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + blk, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) - end := start.Add(time.Hour) h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() - h1.EXPECT().OnIndexSuccess(end) + h1.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h1.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) h2 := NewMockOnIndexSeries(ctrl) - h2.EXPECT().OnIndexFinalize() - h2.EXPECT().OnIndexSuccess(end) + h2.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h2.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, - WriteBatchEntry{Document: testDoc2(), OnIndexSeries: h2}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h1, + }, testDoc1()) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h2, + }, testDoc2()) + + res, err := b.WriteBatch(batch) require.NoError(t, err) require.Equal(t, int64(2), res.NumSuccess) require.Equal(t, int64(0), res.NumError) @@ -899,25 +1074,41 @@ func TestBlockE2EInsertBootstrapQuery(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + blk, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) - end := start.Add(time.Hour) h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() - h1.EXPECT().OnIndexSuccess(end) + h1.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h1.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) h2 := NewMockOnIndexSeries(ctrl) - h2.EXPECT().OnIndexFinalize() - h2.EXPECT().OnIndexSuccess(end) + h2.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h2.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, - WriteBatchEntry{Document: testDoc2(), OnIndexSeries: h2}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h1, + }, testDoc1()) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h2, + }, testDoc2()) + + res, err := b.WriteBatch(batch) require.NoError(t, err) require.Equal(t, int64(2), res.NumSuccess) require.Equal(t, int64(0), res.NumError) @@ -951,20 +1142,33 @@ func TestBlockE2EInsertBootstrapMergeQuery(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - start := time.Now().Truncate(time.Hour) - blk, err := NewBlock(start, time.Hour, testOpts) + blockSize := time.Hour + + now := time.Now() + blockStart := now.Truncate(blockSize) + + nowNotBlockStartAligned := now. + Truncate(blockSize). + Add(time.Minute) + + blk, err := NewBlock(blockStart, blockSize, testOpts) require.NoError(t, err) b, ok := blk.(*block) require.True(t, ok) - end := start.Add(time.Hour) h1 := NewMockOnIndexSeries(ctrl) - h1.EXPECT().OnIndexFinalize() - h1.EXPECT().OnIndexSuccess(end) + h1.EXPECT().OnIndexFinalize(xtime.ToUnixNano(blockStart)) + h1.EXPECT().OnIndexSuccess(xtime.ToUnixNano(blockStart)) - res, err := b.WriteBatch([]WriteBatchEntry{ - WriteBatchEntry{Document: testDoc1(), OnIndexSeries: h1}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) + batch.Append(WriteBatchEntry{ + Timestamp: nowNotBlockStartAligned, + OnIndexSeries: h1, + }, testDoc1()) + + res, err := b.WriteBatch(batch) require.NoError(t, err) require.Equal(t, int64(1), res.NumSuccess) require.Equal(t, int64(0), res.NumError) diff --git a/storage/index/convert/convert.go b/storage/index/convert/convert.go index 1b31cda335..020bd90745 100644 --- a/storage/index/convert/convert.go +++ b/storage/index/convert/convert.go @@ -40,6 +40,20 @@ var ( ReservedFieldNameID = doc.IDReservedFieldName ) +// FOLLOWUP(r): Rename ValidateMetric to ValidateSeries (metric terminiology +// is not common in the codebase) +// ValidateMetric will validate a metric for use in the m3ninx subsytem +func ValidateMetric(id ident.ID, tags ident.Tags) error { + for _, tag := range tags.Values() { + if bytes.Equal(ReservedFieldNameID, tag.Name.Bytes()) { + return errUnableToConvertReservedFieldName + } + } + return nil +} + +// FOLLOWUP(r): Rename FromMetric to FromSeries (metric terminiology +// is not common in the codebase) // FromMetric converts the provided metric id+tags into a document. func FromMetric(id ident.ID, tags ident.Tags) (doc.Document, error) { fields := make([]doc.Field, 0, len(tags.Values())) @@ -59,6 +73,8 @@ func FromMetric(id ident.ID, tags ident.Tags) (doc.Document, error) { }, nil } +// FOLLOWUP(r): Rename FromMetric to FromSeries (metric terminiology +// is not common in the codebase) // FromMetricIter converts the provided metric id+tags into a document. func FromMetricIter(id ident.ID, tags ident.TagIterator) (doc.Document, error) { fields := make([]doc.Field, 0, tags.Remaining()) diff --git a/storage/index/for_each_test.go b/storage/index/for_each_test.go index 9d2ddecea3..a630d73e70 100644 --- a/storage/index/for_each_test.go +++ b/storage/index/for_each_test.go @@ -22,43 +22,54 @@ package index import ( "fmt" - "sort" "testing" "time" "github.com/m3db/m3ninx/doc" - xtime "github.com/m3db/m3x/time" + "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" ) -func TestForEachBlockStarts(t *testing.T) { - now := time.Now().Truncate(time.Hour) - tn := func(n int64) xtime.UnixNano { - return xtime.ToUnixNano(now.Add(time.Duration(n) * time.Hour)) +func TestWriteBatchForEachUnmarkedBatchByBlockStart(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + blockSize := time.Hour + now := time.Now().Truncate(blockSize) + tn := func(n int64) time.Time { + nDur := time.Duration(n) + return now.Add(nDur * blockSize).Add(nDur * time.Minute) } - d := func(n int) doc.Document { + d := func(n int64) doc.Document { return doc.Document{ - ID: []byte(fmt.Sprintf("%d", n)), + ID: []byte(fmt.Sprintf("doc-%d", n)), } } - entries := WriteBatchEntryByBlockStart([]WriteBatchEntry{ - WriteBatchEntry{BlockStart: tn(2), Document: d(2)}, - WriteBatchEntry{BlockStart: tn(0), Document: d(0)}, - WriteBatchEntry{BlockStart: tn(1), Document: d(1)}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) - sort.Sort(entries) + for _, n := range []int64{2, 0, 1} { + batch.Append(WriteBatchEntry{ + Timestamp: tn(n), + OnIndexSeries: NewMockOnIndexSeries(ctrl), + }, d(n)) + } numCalls := 0 - entries.ForEachBlockStart(func(ts xtime.UnixNano, writes []WriteBatchEntry) { - require.Equal(t, 1, len(writes)) + // entries.ForEachBlockStart(func(ts xtime.UnixNano, writes []WriteBatchEntry) { + batch.ForEachUnmarkedBatchByBlockStart(func( + blockStart time.Time, + batch *WriteBatch, + ) { + require.Equal(t, 1, batch.Len()) switch numCalls { case 0: - require.Equal(t, "0", string(writes[0].Document.ID)) + require.Equal(t, "doc-0", string(batch.PendingDocs()[0].ID)) case 1: - require.Equal(t, "1", string(writes[0].Document.ID)) + require.Equal(t, "doc-1", string(batch.PendingDocs()[0].ID)) case 2: - require.Equal(t, "2", string(writes[0].Document.ID)) + require.Equal(t, "doc-2", string(batch.PendingDocs()[0].ID)) default: require.FailNow(t, "should never get here") } @@ -67,37 +78,55 @@ func TestForEachBlockStarts(t *testing.T) { require.Equal(t, 3, numCalls) } -func TestForEachBlockStartMore(t *testing.T) { - now := time.Now().Truncate(time.Hour) - tn := func(n int64) xtime.UnixNano { - return xtime.ToUnixNano(now.Add(time.Duration(n) * time.Hour)) +func TestWriteBatchForEachUnmarkedBatchByBlockStartMore(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + blockSize := time.Hour + now := time.Now().Truncate(blockSize) + tn := func(n int64) time.Time { + nDur := time.Duration(n) + return now.Add(nDur * blockSize).Add(nDur * time.Minute) } - d := func(n int) doc.Document { + d := func(n int64) doc.Document { return doc.Document{ - ID: []byte(fmt.Sprintf("%d", n)), + ID: []byte(fmt.Sprintf("doc-%d", n)), } } - entries := WriteBatchEntryByBlockStart([]WriteBatchEntry{ - WriteBatchEntry{BlockStart: tn(0), Document: d(0)}, - WriteBatchEntry{BlockStart: tn(1), Document: d(3)}, - WriteBatchEntry{BlockStart: tn(0), Document: d(1)}, - WriteBatchEntry{BlockStart: tn(1), Document: d(4)}, - WriteBatchEntry{BlockStart: tn(0), Document: d(2)}, + batch := NewWriteBatch(WriteBatchOptions{ + IndexBlockSize: blockSize, }) - sort.Sort(entries) + for _, v := range []struct { + nTime int64 + nDoc int64 + }{ + {0, 0}, + {1, 3}, + {0, 1}, + {1, 4}, + {0, 2}, + } { + batch.Append(WriteBatchEntry{ + Timestamp: tn(v.nTime), + OnIndexSeries: NewMockOnIndexSeries(ctrl), + }, d(v.nDoc)) + } numCalls := 0 - entries.ForEachBlockStart(func(ts xtime.UnixNano, writes []WriteBatchEntry) { + batch.ForEachUnmarkedBatchByBlockStart(func( + blockStart time.Time, + batch *WriteBatch, + ) { switch numCalls { case 0: - require.Equal(t, 3, len(writes)) - require.Equal(t, "0", string(writes[0].Document.ID)) - require.Equal(t, "1", string(writes[1].Document.ID)) - require.Equal(t, "2", string(writes[2].Document.ID)) + require.Equal(t, 3, batch.Len()) + require.Equal(t, "doc-0", string(batch.PendingDocs()[0].ID)) + require.Equal(t, "doc-1", string(batch.PendingDocs()[1].ID)) + require.Equal(t, "doc-2", string(batch.PendingDocs()[2].ID)) case 1: - require.Equal(t, 2, len(writes)) - require.Equal(t, "3", string(writes[0].Document.ID)) - require.Equal(t, "4", string(writes[1].Document.ID)) + require.Equal(t, 2, batch.Len()) + require.Equal(t, "doc-3", string(batch.PendingDocs()[0].ID)) + require.Equal(t, "doc-4", string(batch.PendingDocs()[1].ID)) default: require.FailNow(t, "should never get here") } diff --git a/storage/index/types.go b/storage/index/types.go index 7dd13c2819..199924c02f 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -285,6 +285,7 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( return } + fmt.Printf("!!! blocksize = %s\n", blockSize) blockStart := allEntries[i].indexBlockStart(blockSize) if !blockStart.Equal(lastBlockStart) { prevLastBlockStart := lastBlockStart.ToTime() @@ -310,11 +311,7 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( } } -// PendingDocs returns all the docs in this batch that are unmarked. -func (b *WriteBatch) PendingDocs() []doc.Document { - // Ensure sorted by unmarked first - b.SortByUnmarkedAndIndexBlockStart() - +func (b *WriteBatch) numPending() int { numUnmarked := 0 for i := range b.entries { if b.entries[i].OnIndexSeries == nil { @@ -322,7 +319,19 @@ func (b *WriteBatch) PendingDocs() []doc.Document { } numUnmarked++ } - return b.docs[:numUnmarked] + return numUnmarked +} + +// PendingDocs returns all the docs in this batch that are unmarked. +func (b *WriteBatch) PendingDocs() []doc.Document { + b.SortByUnmarkedAndIndexBlockStart() // Ensure sorted by unmarked first + return b.docs[:b.numPending()] +} + +// PendingEntries returns all the entries in this batch that are unmarked. +func (b *WriteBatch) PendingEntries() []WriteBatchEntry { + b.SortByUnmarkedAndIndexBlockStart() // Ensure sorted by unmarked first + return b.entries[:b.numPending()] } // NumErrs returns the number of errors encountered by the batch. @@ -355,14 +364,14 @@ func (b *WriteBatch) Reset() { // by index block start time. func (b *WriteBatch) SortByUnmarkedAndIndexBlockStart() { b.sortBy = writeBatchSortByUnmarkedAndBlockStart - sort.Sort(b) + sort.Stable(b) } // SortByEnqueued sorts the entries and documents back to the sort order they // were enqueued as. func (b *WriteBatch) SortByEnqueued() { b.sortBy = writeBatchSortByEnqueued - sort.Sort(b) + sort.Stable(b) } // MarkUnmarkedEntriesSuccess marks all unmarked entries as success. diff --git a/storage/index_block_test.go b/storage/index_block_test.go index d0acb8b548..6f69ecdd38 100644 --- a/storage/index_block_test.go +++ b/storage/index_block_test.go @@ -41,15 +41,51 @@ import ( "github.com/stretchr/testify/require" ) -func testWriteBatchEntry(id ident.ID, tags ident.Tags, timestamp time.Time, fns index.OnIndexSeries) []index.WriteBatchEntry { - return []index.WriteBatchEntry{ - index.WriteBatchEntry{ - ID: id, - Tags: tags, - Timestamp: timestamp, - OnIndexSeries: fns, - }, +type testWriteBatchOption func(index.WriteBatchOptions) index.WriteBatchOptions + +func testWriteBatchBlockSizeOption(blockSize time.Duration) testWriteBatchOption { + return func(o index.WriteBatchOptions) index.WriteBatchOptions { + o.IndexBlockSize = blockSize + return o + } +} + +func testWriteBatch( + e index.WriteBatchEntry, + d doc.Document, + opts ...testWriteBatchOption, +) *index.WriteBatch { + var options index.WriteBatchOptions + for _, opt := range opts { + options = opt(options) + } + b := index.NewWriteBatch(options) + b.Append(e, d) + return b +} + +func testWriteBatchEntry( + id ident.ID, + tags ident.Tags, + timestamp time.Time, + fns index.OnIndexSeries, +) (index.WriteBatchEntry, doc.Document) { + d := doc.Document{ID: copyBytes(id.Bytes())} + for _, tag := range tags.Values() { + d.Fields = append(d.Fields, doc.Field{ + Name: copyBytes(tag.Name.Bytes()), + Value: copyBytes(tag.Value.Bytes()), + }) } + return index.WriteBatchEntry{ + + Timestamp: timestamp, + OnIndexSeries: fns, + }, d +} + +func copyBytes(b []byte) []byte { + return append([]byte(nil), b...) } func testNamespaceMetadata(blockSize, period time.Duration) namespace.Metadata { @@ -139,19 +175,30 @@ func TestNamespaceIndexWrite(t *testing.T) { idx, err := newNamespaceIndexWithNewBlockFn(md, newBlockFn, opts) require.NoError(t, err) - blockStart := xtime.ToUnixNano(now.Truncate(blockSize)) id := ident.StringID("foo") - tags := ident.NewTags(ident.StringTag("name", "value")) + tag := ident.StringTag("name", "value") + tags := ident.NewTags(tag) lifecycle := index.NewMockOnIndexSeries(ctrl) - mockBlock.EXPECT().WriteBatch([]index.WriteBatchEntry{ - index.WriteBatchEntry{ - Timestamp: blockStart.ToTime(), - ID: id, - Tags: tags, - OnIndexSeries: lifecycle, - }, - }).Return(index.WriteBatchResult{}, nil) - require.NoError(t, idx.WriteBatch(testWriteBatchEntry(id, tags, now, lifecycle))) + mockBlock.EXPECT(). + WriteBatch(gomock.Any()). + Return(index.WriteBatchResult{}, nil). + Do(func(batch *index.WriteBatch) { + docs := batch.PendingDocs() + require.Equal(t, 1, len(docs)) + require.Equal(t, doc.Document{ + ID: id.Bytes(), + Fields: doc.Fields{{Name: tag.Name.Bytes(), Value: tag.Value.Bytes()}}, + }, docs[0]) + entries := batch.PendingEntries() + require.Equal(t, 1, len(entries)) + require.True(t, entries[0].Timestamp.Equal(now)) + require.True(t, entries[0].OnIndexSeries == lifecycle) // Just ptr equality + }) + batch := index.NewWriteBatch(index.WriteBatchOptions{ + IndexBlockSize: blockSize, + }) + batch.Append(testWriteBatchEntry(id, tags, now, lifecycle)) + require.NoError(t, idx.WriteBatch(batch)) } func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { @@ -162,7 +209,6 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { now := time.Now().Truncate(blockSize).Add(2 * time.Minute) t0 := now.Truncate(blockSize) t1 := t0.Add(blockSize) - t1Nanos := xtime.ToUnixNano(t1) var nowLock sync.Mutex nowFn := func() time.Time { nowLock.Lock() @@ -190,22 +236,32 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { require.NoError(t, err) id := ident.StringID("foo") - tags := ident.NewTags(ident.StringTag("name", "value")) + tag := ident.StringTag("name", "value") + tags := ident.NewTags(tag) lifecycle := index.NewMockOnIndexSeries(ctrl) - b1.EXPECT().WriteBatch([]index.WriteBatchEntry{ - index.WriteBatchEntry{ - Timestamp: t1, - ID: id, - Tags: tags, - OnIndexSeries: lifecycle, - }, - }).Return(index.WriteBatchResult{}, nil) + b1.EXPECT(). + WriteBatch(gomock.Any()). + Return(index.WriteBatchResult{}, nil). + Do(func(batch *index.WriteBatch) { + docs := batch.PendingDocs() + require.Equal(t, 1, len(docs)) + require.Equal(t, doc.Document{ + ID: id.Bytes(), + Fields: doc.Fields{{Name: tag.Name.Bytes(), Value: tag.Value.Bytes()}}, + }, docs[0]) + entries := batch.PendingEntries() + require.Equal(t, 1, len(entries)) + require.True(t, entries[0].Timestamp.Equal(now)) + require.True(t, entries[0].OnIndexSeries == lifecycle) // Just ptr equality + }) nowLock.Lock() now = now.Add(blockSize) nowLock.Unlock() - require.NoError(t, idx.WriteBatch(testWriteBatchEntry(id, tags, now, lifecycle))) + entry, doc := testWriteBatchEntry(id, tags, now, lifecycle) + batch := testWriteBatch(entry, doc, testWriteBatchBlockSizeOption(blockSize)) + require.NoError(t, idx.WriteBatch(batch)) } func TestNamespaceIndexBootstrap(t *testing.T) { diff --git a/storage/index_insert_queue_test.go b/storage/index_insert_queue_test.go index f5f16fd107..1899a19885 100644 --- a/storage/index_insert_queue_test.go +++ b/storage/index_insert_queue_test.go @@ -39,7 +39,7 @@ import ( func newTestIndexInsertQueue() *nsIndexInsertQueue { var ( - nsIndexInsertBatchFn = func(inserts [][]index.WriteBatchEntry) {} + nsIndexInsertBatchFn = func(inserts []*index.WriteBatch) {} nowFn = time.Now scope = tally.NoopScope ) @@ -53,7 +53,7 @@ func testID(i int) ident.ID { return ident.StringID(fmt.Sprintf("foo%d", i)) } func testTags(i int) ident.Tags { - return ident.Tags{ident.Tag{testID(i), testID(i)}} + return ident.NewTags(ident.Tag{Name: testID(i), Value: testID(i)}) } func TestIndexInsertQueueStopBeforeStart(t *testing.T) { @@ -83,10 +83,10 @@ func TestIndexInsertQueueCallback(t *testing.T) { var ( q = newTestIndexInsertQueue() insertLock sync.Mutex - insertedBatches [][]index.WriteBatchEntry + insertedBatches []*index.WriteBatch callback = index.NewMockOnIndexSeries(ctrl) ) - q.indexBatchFn = func(inserts [][]index.WriteBatchEntry) { + q.indexBatchFn = func(inserts []*index.WriteBatch) { insertLock.Lock() insertedBatches = append(insertedBatches, inserts...) insertLock.Unlock() @@ -96,16 +96,18 @@ func TestIndexInsertQueueCallback(t *testing.T) { defer q.Stop() now := time.Now() - wg, err := q.InsertBatch(testWriteBatchEntry(testID(1), testTags(1), now, callback)) + batch := index.NewWriteBatch(index.WriteBatchOptions{}) + batch.Append(testWriteBatchEntry(testID(1), testTags(1), now, callback)) + wg, err := q.InsertBatch(batch) assert.NoError(t, err) wg.Wait() insertLock.Lock() defer insertLock.Unlock() assert.Len(t, insertedBatches, 1) - assert.Len(t, insertedBatches[0], 1) - // assert.Equal(t, d, insertedBatches[0][0].Document) - // assert.Equal(t, now.UnixNano(), int64(insertedBatches[0][0].BlockStart)) + assert.Equal(t, 1, insertedBatches[0].Len()) + assert.Equal(t, testID(1).Bytes(), insertedBatches[0].PendingDocs()[0].ID) + assert.Equal(t, now.UnixNano(), int64(insertedBatches[0].PendingEntries()[0].Timestamp.UnixNano())) } func TestIndexInsertQueueRateLimit(t *testing.T) { @@ -136,37 +138,44 @@ func TestIndexInsertQueueRateLimit(t *testing.T) { assert.NoError(t, q.Stop()) }() - _, err := q.InsertBatch(testWriteBatchEntry(testID(1), testTags(1), time.Time{}, callback)) + _, err := q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(1), + testTags(1), time.Time{}, callback))) assert.NoError(t, err) addTime(250 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(testID(2), testTags(2), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(2), + testTags(2), time.Time{}, callback))) assert.NoError(t, err) // Consecutive should be all rate limited for i := 0; i < 100; i++ { - _, err = q.InsertBatch(testWriteBatchEntry(testID(i+2), testTags(i+2), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(i+2), + testTags(i+2), time.Time{}, callback))) assert.Error(t, err) assert.Equal(t, errNewSeriesIndexRateLimitExceeded, err) } // Start 2nd second should not be an issue addTime(750 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(testID(110), testTags(100), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(110), + testTags(100), time.Time{}, callback))) assert.NoError(t, err) addTime(100 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(testID(111), testTags(111), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(111), + testTags(111), time.Time{}, callback))) assert.NoError(t, err) addTime(100 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(testID(112), testTags(112), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(112), + testTags(112), time.Time{}, callback))) assert.Error(t, err) assert.Equal(t, errNewSeriesIndexRateLimitExceeded, err) // Start 3rd second addTime(800 * time.Millisecond) - _, err = q.InsertBatch(testWriteBatchEntry(testID(113), testTags(113), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(113), + testTags(113), time.Time{}, callback))) assert.NoError(t, err) q.Lock() @@ -180,7 +189,7 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() var ( - inserts [][][]index.WriteBatchEntry + inserts [][]*index.WriteBatch currTime = time.Now() timeLock = sync.Mutex{} addTime = func(d time.Duration) { @@ -204,7 +213,7 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { defer timeLock.Unlock() return currTime } - q.indexBatchFn = func(value [][]index.WriteBatchEntry) { + q.indexBatchFn = func(value []*index.WriteBatch) { inserts = append(inserts, value) insertWgs[len(inserts)-1].Done() insertProgressWgs[len(inserts)-1].Wait() @@ -229,16 +238,19 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { }() // first insert - _, err := q.InsertBatch(testWriteBatchEntry(testID(0), testTags(0), time.Time{}, callback)) + _, err := q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(0), + testTags(0), time.Time{}, callback))) require.NoError(t, err) // wait for first insert batch to complete insertWgs[0].Wait() // now next batch will need to wait as we haven't progressed time - _, err = q.InsertBatch(testWriteBatchEntry(testID(1), testTags(1), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(1), + testTags(1), time.Time{}, callback))) require.NoError(t, err) - _, err = q.InsertBatch(testWriteBatchEntry(testID(2), testTags(2), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(2), + testTags(2), time.Time{}, callback))) require.NoError(t, err) // allow first insert to finish @@ -251,7 +263,8 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { assert.Equal(t, 1, numSleeps) // insert third batch, will also need to wait - _, err = q.InsertBatch(testWriteBatchEntry(testID(3), testTags(3), time.Time{}, callback)) + _, err = q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(3), + testTags(3), time.Time{}, callback))) require.NoError(t, err) // allow second batch to finish @@ -278,14 +291,15 @@ func TestIndexInsertQueueFlushedOnClose(t *testing.T) { currTime = time.Now().Truncate(time.Second) ) - q := newNamespaceIndexInsertQueue(func(value [][]index.WriteBatchEntry) { + q := newNamespaceIndexInsertQueue(func(value []*index.WriteBatch) { atomic.AddInt64(&numInsertObserved, int64(len(value))) }, func() time.Time { return currTime }, tally.NoopScope) require.NoError(t, q.Start()) for i := 0; i < numInsertExpected; i++ { - _, err := q.InsertBatch(testWriteBatchEntry(testID(1), testTags(1), time.Time{}, nil)) + _, err := q.InsertBatch(testWriteBatch(testWriteBatchEntry(testID(1), + testTags(1), time.Time{}, nil))) require.NoError(t, err) } diff --git a/storage/index_queue_test.go b/storage/index_queue_test.go index b4bc9ce4b7..d1764e15ac 100644 --- a/storage/index_queue_test.go +++ b/storage/index_queue_test.go @@ -28,11 +28,11 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/storage/index" - "github.com/m3db/m3db/storage/index/convert" "github.com/m3db/m3db/storage/namespace" m3ninxidx "github.com/m3db/m3ninx/idx" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" + xtime "github.com/m3db/m3x/time" "github.com/fortytw2/leaktest" "github.com/golang/mock/gomock" @@ -114,24 +114,6 @@ func TestNamespaceIndexStopErr(t *testing.T) { assert.Error(t, idx.Close()) } -func TestNamespaceIndexInvalidDocWrite(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - dbIdx, _ := newTestNamespaceIndex(t, ctrl) - idx, ok := dbIdx.(*nsIndex) - assert.True(t, ok) - - id := ident.StringID("foo") - tags := ident.NewTags( - ident.StringTag(string(index.ReservedFieldNameID), "value"), - ) - - lifecycle := index.NewMockOnIndexSeries(ctrl) - lifecycle.EXPECT().OnIndexFinalize(time.Time{}) - assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, time.Time{}, lifecycle))) -} - func TestNamespaceIndexWriteAfterClose(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -150,7 +132,8 @@ func TestNamespaceIndexWriteAfterClose(t *testing.T) { lifecycle := index.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize(time.Time{}) - assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, time.Time{}, lifecycle))) + assert.Error(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, + tags, time.Time{}, lifecycle)))) } func TestNamespaceIndexWriteQueueError(t *testing.T) { @@ -172,7 +155,8 @@ func TestNamespaceIndexWriteQueueError(t *testing.T) { q.EXPECT(). InsertBatch(gomock.Any()). Return(nil, fmt.Errorf("random err")) - assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, n, lifecycle))) + assert.Error(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, + tags, n, lifecycle)))) } func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { @@ -216,11 +200,13 @@ func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { tooOld := now.Add(-1 * idx.bufferPast).Add(-1 * time.Second) lifecycle.EXPECT().OnIndexFinalize(tooOld) - assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, tooOld, lifecycle))) + assert.Error(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, + tags, tooOld, lifecycle)))) tooNew := now.Add(1 * idx.bufferFuture).Add(1 * time.Second) lifecycle.EXPECT().OnIndexFinalize(tooOld) - assert.Error(t, idx.WriteBatch(testWriteBatchEntry(id, tags, tooNew, lifecycle))) + assert.Error(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, + tags, tooNew, lifecycle)))) } func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { @@ -239,13 +225,12 @@ func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { ) now := time.Now() - d, err := convert.FromMetric(id, tags) - assert.NoError(t, err) var wg sync.WaitGroup lifecycle := index.NewMockOnIndexSeries(ctrl) q.EXPECT().InsertBatch(gomock.Any()).Return(&wg, nil) - assert.NoError(t, idx.WriteBatch(testWriteBatchEntry(id, tags, now, lifecycle))) + assert.NoError(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, + tags, now, lifecycle)))) } func TestNamespaceIndexInsertQuery(t *testing.T) { @@ -266,8 +251,9 @@ func TestNamespaceIndexInsertQuery(t *testing.T) { defer idx.Close() var ( + blockSize = idx.(*nsIndex).blockSize indexState = idx.(*nsIndex).state - ts = indexState.latestBlock.EndTime() + ts = indexState.latestBlock.StartTime() now = time.Now() id = ident.StringID("foo") tags = ident.NewTags( @@ -277,9 +263,12 @@ func TestNamespaceIndexInsertQuery(t *testing.T) { lifecycleFns = index.NewMockOnIndexSeries(ctrl) ) - lifecycleFns.EXPECT().OnIndexFinalize(ts) - lifecycleFns.EXPECT().OnIndexSuccess(ts) - assert.NoError(t, idx.WriteBatch(testWriteBatchEntry(id, tags, now, lifecycleFns))) + lifecycleFns.EXPECT().OnIndexFinalize(xtime.ToUnixNano(ts)) + lifecycleFns.EXPECT().OnIndexSuccess(xtime.ToUnixNano(ts)) + + entry, doc := testWriteBatchEntry(id, tags, now, lifecycleFns) + batch := testWriteBatch(entry, doc, testWriteBatchBlockSizeOption(blockSize)) + assert.NoError(t, idx.WriteBatch(batch)) reQuery, err := m3ninxidx.NewRegexpQuery([]byte("name"), []byte("val.*")) assert.NoError(t, err) diff --git a/storage/series/lookup/entry.go b/storage/series/lookup/entry.go index 106ec72ccd..d06a1ce11d 100644 --- a/storage/series/lookup/entry.go +++ b/storage/series/lookup/entry.go @@ -21,6 +21,7 @@ package lookup import ( + "fmt" "sync" "sync/atomic" @@ -97,6 +98,7 @@ func (entry *Entry) NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) boo alreadyIndexedOrAttempted := entry.reverseIndex.indexedOrAttemptedWithRLock(indexBlockStartForWrite) entry.reverseIndex.RUnlock() if alreadyIndexedOrAttempted { + fmt.Printf("! GOOD alreadyIndexedOrAttempted true 1 for start %v\n", indexBlockStartForWrite.ToTime().String()) // if so, the entry does not need to be indexed. return false } @@ -110,11 +112,13 @@ func (entry *Entry) NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) boo alreadyIndexedOrAttempted = entry.reverseIndex.indexedOrAttemptedWithRLock(indexBlockStartForWrite) if alreadyIndexedOrAttempted { entry.reverseIndex.Unlock() + fmt.Printf("! GOOD alreadyIndexedOrAttempted true 2 for start %v\n", indexBlockStartForWrite.ToTime().String()) return false } entry.reverseIndex.setAttemptWithWLock(indexBlockStartForWrite, true) entry.reverseIndex.Unlock() + fmt.Printf("! BAD alreadyIndexedOrAttempted false for start %v\n", indexBlockStartForWrite.ToTime().String()) return true } @@ -128,6 +132,7 @@ func (entry *Entry) OnIndexPrepare() { // OnIndexSuccess marks the given block start as successfully indexed. func (entry *Entry) OnIndexSuccess(blockStartNanos xtime.UnixNano) { + fmt.Printf("! on indexsuccess for %s at %v\n", entry.Series.ID().String(), blockStartNanos.ToTime().String()) entry.reverseIndex.Lock() entry.reverseIndex.setSuccessWithWLock(blockStartNanos) entry.reverseIndex.Unlock() diff --git a/storage/shard.go b/storage/shard.go index 28264f093d..d7b7fe7e8a 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -41,6 +41,7 @@ import ( "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/bootstrap/result" "github.com/m3db/m3db/storage/index" + "github.com/m3db/m3db/storage/index/convert" "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3db/storage/repair" "github.com/m3db/m3db/storage/series" @@ -750,6 +751,8 @@ func (s *dbShard) writeAndIndex( writable := entry != nil + fmt.Printf("!! call for %s\n", id.String()) + // If no entry and we are not writing new series asynchronously if !writable && !opts.writeNewSeriesAsync { // Avoid double lookup by enqueueing insert immediately @@ -772,6 +775,10 @@ func (s *dbShard) writeAndIndex( return err } writable = true + + fmt.Printf("!! just batched and inserted %s\n", id.String()) + // NB(r): We just indexed this series if shouldReverseIndex was true + shouldReverseIndex = false } var ( @@ -793,8 +800,11 @@ func (s *dbShard) writeAndIndex( commitLogSeriesUniqueIndex = entry.Index if err == nil && shouldReverseIndex { if entry.NeedsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) { + fmt.Printf("!! need index %s YES\n", entry.Series.ID().String()) err = s.insertSeriesForIndexingAsyncBatched(entry, timestamp, opts.writeNewSeriesAsync) + } else { + fmt.Printf("!! need index %s NO\n", entry.Series.ID().String()) } } // release the reference we got on entry from `writableSeries` @@ -997,6 +1007,10 @@ func (s *dbShard) newShardEntry( return nil, err } + if err := convert.ValidateMetric(clonedID, clonedTags); err != nil { + return nil, err + } + series.Reset(clonedID, clonedTags, s.seriesBlockRetriever, s.seriesOnRetrieveBlock, s, s.seriesOpts) uniqueIndex := s.increasingIndex.nextIndex() diff --git a/storage/shard_index_test.go b/storage/shard_index_test.go index b307974443..835ac7c427 100644 --- a/storage/shard_index_test.go +++ b/storage/shard_index_test.go @@ -21,6 +21,7 @@ package storage import ( + "fmt" "sync" "sync/atomic" "testing" @@ -29,6 +30,8 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/runtime" "github.com/m3db/m3db/storage/index" + "github.com/m3db/m3db/storage/namespace" + "github.com/m3db/m3ninx/doc" xclock "github.com/m3db/m3x/clock" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" @@ -45,21 +48,28 @@ func TestShardInsertNamespaceIndex(t *testing.T) { opts := testDatabaseOptions() lock := sync.Mutex{} - indexWrites := []index.WriteBatchEntry{} - var blockStart xtime.UnixNano + indexWrites := []doc.Document{} + + now := time.Now() + blockSize := namespace.NewIndexOptions().BlockSize() + + blockStart := xtime.ToUnixNano(now.Truncate(blockSize)) ctrl := gomock.NewController(t) defer ctrl.Finish() idx := NewMocknamespaceIndex(ctrl) idx.EXPECT().BlockStartForWriteTime(gomock.Any()).Return(blockStart).AnyTimes() idx.EXPECT().WriteBatch(gomock.Any()).Do( - func(entries []index.WriteBatchEntry) { + func(batch *index.WriteBatch) { + lock.Lock() - indexWrites = append(indexWrites, entries...) + indexWrites = append(indexWrites, batch.PendingDocs()...) lock.Unlock() - for _, e := range entries { + for i, e := range batch.PendingEntries() { + fmt.Printf("!! inserting %s\n", string(batch.PendingDocs()[i].ID)) e.OnIndexSeries.OnIndexSuccess(blockStart) e.OnIndexSeries.OnIndexFinalize(blockStart) + batch.PendingEntries()[i].OnIndexSeries = nil } }).Return(nil).AnyTimes() @@ -73,23 +83,23 @@ func TestShardInsertNamespaceIndex(t *testing.T) { require.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), - time.Now(), 1.0, xtime.Second, nil)) + now, 1.0, xtime.Second, nil)) require.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), - time.Now(), 2.0, xtime.Second, nil)) + now, 2.0, xtime.Second, nil)) require.NoError(t, - shard.Write(ctx, ident.StringID("baz"), time.Now(), 1.0, xtime.Second, nil)) + shard.Write(ctx, ident.StringID("baz"), now, 1.0, xtime.Second, nil)) lock.Lock() defer lock.Unlock() require.Len(t, indexWrites, 1) - require.Equal(t, "foo", indexWrites[0].ID.String()) - require.Equal(t, "name", indexWrites[0].Tags.Values()[0].Name.String()) - require.Equal(t, "value", indexWrites[0].Tags.Values()[0].Value.String()) + require.Equal(t, []byte("foo"), indexWrites[0].ID) + require.Equal(t, []byte("name"), indexWrites[0].Fields[0].Name) + require.Equal(t, []byte("value"), indexWrites[0].Fields[0].Value) } func TestShardAsyncInsertNamespaceIndex(t *testing.T) { @@ -97,15 +107,15 @@ func TestShardAsyncInsertNamespaceIndex(t *testing.T) { opts := testDatabaseOptions() lock := sync.RWMutex{} - indexWrites := []testIndexWrite{} + indexWrites := []doc.Document{} ctrl := gomock.NewController(t) defer ctrl.Finish() idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().Write(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do( - func(id ident.ID, tags ident.Tags, ts time.Time, onIdx index.OnIndexSeries) { + idx.EXPECT().WriteBatch(gomock.Any()).Do( + func(batch *index.WriteBatch) { lock.Lock() - indexWrites = append(indexWrites, testIndexWrite{id: id, tags: tags}) + indexWrites = append(indexWrites, batch.PendingDocs()...) lock.Unlock() }).Return(nil).AnyTimes() @@ -146,16 +156,16 @@ func TestShardAsyncInsertNamespaceIndex(t *testing.T) { assert.Len(t, indexWrites, 2) for _, w := range indexWrites { - if w.id.String() == "foo" { - assert.Equal(t, 1, len(w.tags.Values())) - assert.Equal(t, "name", w.tags.Values()[0].Name.String()) - assert.Equal(t, "value", w.tags.Values()[0].Value.String()) - } else if w.id.String() == "baz" { - assert.Equal(t, 2, len(w.tags.Values())) - assert.Equal(t, "all", w.tags.Values()[0].Name.String()) - assert.Equal(t, "tags", w.tags.Values()[0].Value.String()) - assert.Equal(t, "should", w.tags.Values()[1].Name.String()) - assert.Equal(t, "be-present", w.tags.Values()[1].Value.String()) + if string(w.ID) == "foo" { + assert.Equal(t, 1, len(w.Fields)) + assert.Equal(t, "name", string(w.Fields[0].Name)) + assert.Equal(t, "value", string(w.Fields[0].Value)) + } else if string(w.ID) == "baz" { + assert.Equal(t, 2, len(w.Fields)) + assert.Equal(t, "all", string(w.Fields[0].Name)) + assert.Equal(t, "tags", string(w.Fields[0].Value)) + assert.Equal(t, "should", string(w.Fields[1].Name)) + assert.Equal(t, "be-present", string(w.Fields[1].Value)) } else { assert.Fail(t, "unexpected write", w) } @@ -169,12 +179,17 @@ func TestShardAsyncIndexOnlyWhenNotIndexed(t *testing.T) { var numCalls int32 opts := testDatabaseOptions() - nextWriteTime := time.Now().Add(time.Hour) + now := time.Now() + nextWriteTime := now.Truncate(time.Hour) idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().Write(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do( - func(id ident.ID, tags ident.Tags, ts time.Time, onIdx index.OnIndexSeries) { - onIdx.OnIndexSuccess(nextWriteTime) // i.e. mark that the entry should not be indexed for an hour at least - onIdx.OnIndexFinalize() + idx.EXPECT().WriteBatch(gomock.Any()).Do( + func(batch *index.WriteBatch) { + if batch.Len() == 0 { + panic(fmt.Errorf("expected batch of len 1")) // panic to avoid goroutine exit from require + } + onIdx := batch.PendingEntries()[0].OnIndexSeries + onIdx.OnIndexSuccess(xtime.ToUnixNano(nextWriteTime)) // i.e. mark that the entry should not be indexed for an hour at least + onIdx.OnIndexFinalize(xtime.ToUnixNano(nextWriteTime)) current := atomic.AddInt32(&numCalls, 1) if current > 1 { panic("only need to index when not-indexed") @@ -191,7 +206,7 @@ func TestShardAsyncIndexOnlyWhenNotIndexed(t *testing.T) { assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), - time.Now(), 1.0, xtime.Second, nil)) + now, 1.0, xtime.Second, nil)) for { if l := atomic.LoadInt32(&numCalls); l == 1 { @@ -204,14 +219,14 @@ func TestShardAsyncIndexOnlyWhenNotIndexed(t *testing.T) { assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), - time.Now(), 2.0, xtime.Second, nil)) + now.Add(time.Second), 2.0, xtime.Second, nil)) l := atomic.LoadInt32(&numCalls) assert.Equal(t, int32(1), l) entry, _, err := shard.tryRetrieveWritableSeries(ident.StringID("foo")) assert.NoError(t, err) - assert.Equal(t, nextWriteTime.UnixNano(), entry.reverseIndex.nextWriteTimeNanos) + assert.True(t, entry.IndexedForBlockStart(xtime.ToUnixNano(nextWriteTime))) } func TestShardAsyncIndexIfExpired(t *testing.T) { @@ -230,18 +245,20 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { return n })) + blockStart := now.Truncate(time.Hour) + ctrl := gomock.NewController(t) defer ctrl.Finish() idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().BlockStartForWriteTime(gomock.Any()).Return() + idx.EXPECT().BlockStartForWriteTime(gomock.Any()).Return(blockStart) idx.EXPECT().WriteBatch(gomock.Any()).Do( - func(batch []index.WriteBatchEntry) { + func(batch *index.WriteBatch) { nowLock.Lock() now = now.Add(time.Hour) nowLock.Unlock() - for _, b := range batch { - b.OnIndexSeries.OnIndexSuccess(now) - b.OnIndexSeries.OnIndexFinalize() + for _, b := range batch.PendingEntries() { + b.OnIndexSeries.OnIndexSuccess(xtime.ToUnixNano(blockStart)) + b.OnIndexSeries.OnIndexFinalize(xtime.ToUnixNano(blockStart)) atomic.AddInt32(&numCalls, 1) } }).Return(nil).AnyTimes() @@ -280,7 +297,7 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { assert.NoError(t, err) nowLock.Lock() defer nowLock.Unlock() - assert.Equal(t, now.UnixNano(), entry.reverseIndex.nextWriteTimeNanos) + assert.True(t, entry.IndexedForBlockStart(xtime.ToUnixNano(blockStart))) } // TODO(prateek): wire tests above to use the field `ts` diff --git a/storage/shard_ref_count_test.go b/storage/shard_ref_count_test.go index 0f56e9192d..114604b067 100644 --- a/storage/shard_ref_count_test.go +++ b/storage/shard_ref_count_test.go @@ -112,11 +112,20 @@ func TestShardWriteSyncRefCount(t *testing.T) { func TestShardWriteTaggedSyncRefCountMockIndex(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + + blockSize := namespace.NewIndexOptions().BlockSize() + idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().Write(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do( - func(id ident.ID, tags ident.Tags, ts time.Time, onIdx index.OnIndexSeries) { - onIdx.OnIndexFinalize() + idx.EXPECT().WriteBatch(gomock.Any()).Do( + func(batch *index.WriteBatch) { + if batch.Len() != 1 { + panic("expected batch len 1") // require.Equal(...) silently kills goroutines + } + entry := batch.PendingEntries()[0] + onIdx := entry.OnIndexSeries + onIdx.OnIndexFinalize(xtime.ToUnixNano(entry.Timestamp.Truncate(blockSize))) }).Return(nil).AnyTimes() + testShardWriteTaggedSyncRefCount(t, idx) } @@ -178,7 +187,7 @@ func testShardWriteTaggedSyncRefCount(t *testing.T, idx namespaceIndex) { entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) - assert.Equal(t, int32(0), entry.readerWriterCount(), id) + assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) } // write already inserted series' @@ -201,7 +210,7 @@ func testShardWriteTaggedSyncRefCount(t *testing.T, idx namespaceIndex) { entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) - assert.Equal(t, int32(0), entry.readerWriterCount(), id) + assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) } } @@ -259,7 +268,7 @@ func TestShardWriteAsyncRefCount(t *testing.T) { entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) - assert.Equal(t, int32(0), entry.readerWriterCount(), id) + assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) } // write already inserted series' @@ -282,17 +291,25 @@ func TestShardWriteAsyncRefCount(t *testing.T) { entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) - assert.Equal(t, int32(0), entry.readerWriterCount(), id) + assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) } } func TestShardWriteTaggedAsyncRefCountMockIndex(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() + + blockSize := namespace.NewIndexOptions().BlockSize() + idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().Write(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do( - func(id ident.ID, tags ident.Tags, ts time.Time, onIdx index.OnIndexSeries) { - onIdx.OnIndexFinalize() + idx.EXPECT().WriteBatch(gomock.Any()).Do( + func(batch *index.WriteBatch) { + if batch.Len() != 1 { + panic("expected batch len 1") // require.Equal(...) silently kills goroutines + } + entry := batch.PendingEntries()[0] + onIdx := entry.OnIndexSeries + onIdx.OnIndexFinalize(xtime.ToUnixNano(entry.Timestamp.Truncate(blockSize))) }).Return(nil).AnyTimes() testShardWriteTaggedAsyncRefCount(t, idx) } @@ -371,7 +388,7 @@ func testShardWriteTaggedAsyncRefCount(t *testing.T, idx namespaceIndex) { entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) - assert.Equal(t, int32(0), entry.readerWriterCount(), id) + assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) } // write already inserted series' @@ -394,6 +411,6 @@ func testShardWriteTaggedAsyncRefCount(t *testing.T, idx namespaceIndex) { entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) - assert.Equal(t, int32(0), entry.readerWriterCount(), id) + assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) } } diff --git a/storage/shard_test.go b/storage/shard_test.go index f1b0f33017..62088c2964 100644 --- a/storage/shard_test.go +++ b/storage/shard_test.go @@ -37,6 +37,7 @@ import ( "github.com/m3db/m3db/storage/bootstrap/result" "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3db/storage/series" + "github.com/m3db/m3db/storage/series/lookup" "github.com/m3db/m3db/ts" xmetrics "github.com/m3db/m3db/x/metrics" "github.com/m3db/m3db/x/xio" @@ -83,7 +84,7 @@ func addMockSeries(ctrl *gomock.Controller, shard *dbShard, id ident.ID, tags id series.EXPECT().Tags().Return(tags).AnyTimes() series.EXPECT().IsEmpty().Return(false).AnyTimes() shard.Lock() - shard.insertNewShardEntryWithLock(&dbShardEntry{series: series, index: index}) + shard.insertNewShardEntryWithLock(lookup.NewEntry(series, index)) shard.Unlock() return series } @@ -150,8 +151,8 @@ func TestShardBootstrapWithError(t *testing.T) { barSeries.EXPECT().ID().Return(ident.StringID("bar")).AnyTimes() barSeries.EXPECT().IsEmpty().Return(false).AnyTimes() s.Lock() - s.insertNewShardEntryWithLock(&dbShardEntry{series: fooSeries}) - s.insertNewShardEntryWithLock(&dbShardEntry{series: barSeries}) + s.insertNewShardEntryWithLock(lookup.NewEntry(fooSeries, 0)) + s.insertNewShardEntryWithLock(lookup.NewEntry(barSeries, 0)) s.Unlock() fooBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) @@ -226,7 +227,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { flushed[i] = struct{}{} }). Return(series.FlushOutcomeErr, expectedErr) - s.list.PushBack(&dbShardEntry{series: curr}) + s.list.PushBack(lookup.NewEntry(curr, 0)) } err := s.Flush(blockStart, flush) @@ -288,7 +289,7 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { flushed[i] = struct{}{} }). Return(series.FlushOutcomeFlushedToDisk, nil) - s.list.PushBack(&dbShardEntry{series: curr}) + s.list.PushBack(lookup.NewEntry(curr, 0)) } err := s.Flush(blockStart, flush) @@ -362,7 +363,7 @@ func TestShardSnapshotSeriesSnapshotSuccess(t *testing.T) { snapshotted[i] = struct{}{} }). Return(nil) - s.list.PushBack(&dbShardEntry{series: series}) + s.list.PushBack(lookup.NewEntry(series, 0)) } err := s.Snapshot(blockStart, blockStart, flush) @@ -381,7 +382,7 @@ func addMockTestSeries(ctrl *gomock.Controller, shard *dbShard, id ident.ID) *se series := series.NewMockDatabaseSeries(ctrl) series.EXPECT().ID().AnyTimes().Return(id) shard.Lock() - shard.insertNewShardEntryWithLock(&dbShardEntry{series: series}) + shard.insertNewShardEntryWithLock(lookup.NewEntry(series, 0)) shard.Unlock() return series } @@ -394,7 +395,11 @@ func addTestSeriesWithCount(shard *dbShard, id ident.ID, count int32) series.Dat series := series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) series.Bootstrap(nil) shard.Lock() - shard.insertNewShardEntryWithLock(&dbShardEntry{series: series, curReadWriters: count}) + entry := lookup.NewEntry(series, 0) + for i := int32(0); i < count; i++ { + entry.IncrementReaderWriterCount() + } + shard.insertNewShardEntryWithLock(entry) shard.Unlock() return series } @@ -760,7 +765,7 @@ func TestPurgeExpiredSeriesWriteAfterPurging(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - var entry *dbShardEntry + var entry *lookup.Entry opts := testDatabaseOptions() shard := testDatabaseShard(t, opts) @@ -780,7 +785,7 @@ func TestPurgeExpiredSeriesWriteAfterPurging(t *testing.T) { require.Equal(t, 1, r.expiredSeries) require.Equal(t, 1, shard.lookup.Len()) - entry.decrementReaderWriterCount() + entry.DecrementReaderWriterCount() require.Equal(t, 1, shard.lookup.Len()) } @@ -793,14 +798,14 @@ func TestForEachShardEntry(t *testing.T) { } count := 0 - entryFn := func(entry *dbShardEntry) bool { - if entry.series.ID().String() == "foo.8" { + entryFn := func(entry *lookup.Entry) bool { + if entry.Series.ID().String() == "foo.8" { return false } // Ensure the readerwriter count is incremented while we operate // on this series - assert.Equal(t, int32(1), entry.readerWriterCount()) + assert.Equal(t, int32(1), entry.ReaderWriterCount()) count++ return true @@ -813,8 +818,8 @@ func TestForEachShardEntry(t *testing.T) { // Ensure that reader writer count gets reset shard.RLock() for elem := shard.list.Front(); elem != nil; elem = elem.Next() { - entry := elem.Value.(*dbShardEntry) - assert.Equal(t, int32(0), entry.readerWriterCount()) + entry := elem.Value.(*lookup.Entry) + assert.Equal(t, int32(0), entry.ReaderWriterCount()) } shard.RUnlock() } @@ -1069,7 +1074,7 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { continue } - if err != nil || entry.series.NumActiveBlocks() == 2 { + if err != nil || entry.Series.NumActiveBlocks() == 2 { // Expecting at least 2 active blocks and never an error break } @@ -1081,8 +1086,8 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { require.NoError(t, err) require.NotNil(t, entry) - assert.False(t, entry.series.IsEmpty()) - assert.Equal(t, 2, entry.series.NumActiveBlocks()) + assert.False(t, entry.Series.IsEmpty()) + assert.Equal(t, 2, entry.Series.NumActiveBlocks()) } func TestShardNewInvalidShardEntry(t *testing.T) { From 7ab107e1e84aa7b751e9455757aadb9c30dec279 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 15 May 2018 16:06:57 -0400 Subject: [PATCH 29/33] Fix remaining tests --- runtime/runtime_options_manager.go | 38 +++++++++++++++ storage/database_test.go | 13 +++++- storage/index.go | 2 +- storage/index/types.go | 1 - storage/index_block_test.go | 1 - storage/index_queue_test.go | 75 ++++++++++++++++++++++-------- storage/series/lookup/entry.go | 29 +++++++----- storage/shard_index_test.go | 53 +++++++++++---------- storage/shard_ref_count_test.go | 46 ++++++++++++------ 9 files changed, 184 insertions(+), 74 deletions(-) diff --git a/runtime/runtime_options_manager.go b/runtime/runtime_options_manager.go index 68659bde0a..85d9c95112 100644 --- a/runtime/runtime_options_manager.go +++ b/runtime/runtime_options_manager.go @@ -21,6 +21,8 @@ package runtime import ( + "fmt" + xclose "github.com/m3db/m3x/close" xwatch "github.com/m3db/m3x/watch" ) @@ -74,3 +76,39 @@ func (w *optionsManager) RegisterListener( func (w *optionsManager) Close() { w.watchable.Close() } + +// NewNoOpOptionsManager returns a no-op options manager that cannot +// be updated and does not spawn backround goroutines (useful for globals +// in test files). +func NewNoOpOptionsManager(opts Options) OptionsManager { + return noOpOptionsManager{opts: opts} +} + +type noOpOptionsManager struct { + opts Options +} + +func (n noOpOptionsManager) Update(value Options) error { + return fmt.Errorf("no-op options manager cannot update options") +} + +func (n noOpOptionsManager) Get() Options { + return n.opts +} + +func (n noOpOptionsManager) RegisterListener( + listener OptionsListener, +) xclose.SimpleCloser { + // noOpOptionsManager never changes its options, not worth + // registering listener + return noOpCloser{} +} + +func (n noOpOptionsManager) Close() { +} + +type noOpCloser struct{} + +func (n noOpCloser) Close() { + +} diff --git a/storage/database_test.go b/storage/database_test.go index 9b8ce6f97c..202197effb 100644 --- a/storage/database_test.go +++ b/storage/database_test.go @@ -31,6 +31,7 @@ import ( "github.com/m3db/m3db/client" "github.com/m3db/m3db/persist/fs" "github.com/m3db/m3db/retention" + "github.com/m3db/m3db/runtime" "github.com/m3db/m3db/sharding" "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/index" @@ -68,9 +69,17 @@ var ( ) func init() { - opts := newOptions(pool.NewObjectPoolOptions().SetSize(16)) + opts := newOptions(pool.NewObjectPoolOptions(). + SetSize(16)) - pm, err := fs.NewPersistManager(fs.NewOptions()) + // Use a no-op options manager to avoid spinning up a goroutine to listen + // for updates, which causes problems with leaktest in individual test + // executions + runtimeOptionsMgr := runtime.NewNoOpOptionsManager( + runtime.NewOptions()) + + pm, err := fs.NewPersistManager(fs.NewOptions(). + SetRuntimeOptionsManager(runtimeOptionsMgr)) if err != nil { panic(err) } diff --git a/storage/index.go b/storage/index.go index a95c86fd15..7240dfffa0 100644 --- a/storage/index.go +++ b/storage/index.go @@ -304,7 +304,7 @@ func (i *nsIndex) writeBatches( return } - if !pastLimit.Before(entry.Timestamp) { + if !entry.Timestamp.After(pastLimit) { batch.MarkUnmarkedEntryError(m3dberrors.ErrTooPast, idx) return } diff --git a/storage/index/types.go b/storage/index/types.go index 199924c02f..1e57f6e7c7 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -285,7 +285,6 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( return } - fmt.Printf("!!! blocksize = %s\n", blockSize) blockStart := allEntries[i].indexBlockStart(blockSize) if !blockStart.Equal(lastBlockStart) { prevLastBlockStart := lastBlockStart.ToTime() diff --git a/storage/index_block_test.go b/storage/index_block_test.go index 6f69ecdd38..01fdb0ee04 100644 --- a/storage/index_block_test.go +++ b/storage/index_block_test.go @@ -78,7 +78,6 @@ func testWriteBatchEntry( }) } return index.WriteBatchEntry{ - Timestamp: timestamp, OnIndexSeries: fns, }, d diff --git a/storage/index_queue_test.go b/storage/index_queue_test.go index d1764e15ac..3f4db7b3ba 100644 --- a/storage/index_queue_test.go +++ b/storage/index_queue_test.go @@ -27,8 +27,10 @@ import ( "time" "github.com/m3db/m3db/clock" + m3dberrors "github.com/m3db/m3db/storage/errors" "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/namespace" + "github.com/m3db/m3ninx/doc" m3ninxidx "github.com/m3db/m3ninx/idx" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" @@ -130,10 +132,14 @@ func TestNamespaceIndexWriteAfterClose(t *testing.T) { q.EXPECT().Stop().Return(nil) assert.NoError(t, idx.Close()) + now := time.Now() + lifecycle := index.NewMockOnIndexSeries(ctrl) - lifecycle.EXPECT().OnIndexFinalize(time.Time{}) - assert.Error(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, - tags, time.Time{}, lifecycle)))) + lifecycle.EXPECT(). + OnIndexFinalize(xtime.ToUnixNano(now.Truncate(idx.blockSize))) + entry, document := testWriteBatchEntry(id, tags, now, lifecycle) + assert.Error(t, idx.WriteBatch(testWriteBatch(entry, document, + testWriteBatchBlockSizeOption(idx.blockSize)))) } func TestNamespaceIndexWriteQueueError(t *testing.T) { @@ -151,12 +157,14 @@ func TestNamespaceIndexWriteQueueError(t *testing.T) { n := time.Now() lifecycle := index.NewMockOnIndexSeries(ctrl) - lifecycle.EXPECT().OnIndexFinalize(n) + lifecycle.EXPECT(). + OnIndexFinalize(xtime.ToUnixNano(n.Truncate(idx.blockSize))) q.EXPECT(). InsertBatch(gomock.Any()). Return(nil, fmt.Errorf("random err")) - assert.Error(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, - tags, n, lifecycle)))) + entry, document := testWriteBatchEntry(id, tags, n, lifecycle) + assert.Error(t, idx.WriteBatch(testWriteBatch(entry, document, + testWriteBatchBlockSizeOption(idx.blockSize)))) } func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { @@ -174,17 +182,13 @@ func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { } ) - q := NewMocknamespaceIndexInsertQueue(ctrl) - newFn := func(fn nsIndexInsertBatchFn, nowFn clock.NowFn, s tally.Scope) namespaceIndexInsertQueue { - return q - } - q.EXPECT().Start().Return(nil) md, err := namespace.NewMetadata(defaultTestNs1ID, defaultTestNs1Opts) require.NoError(t, err) - opts := testNamespaceIndexOptions() + opts := testNamespaceIndexOptions(). + SetInsertMode(index.InsertSync) opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) - dbIdx, err := newNamespaceIndexWithInsertQueueFn(md, newFn, opts) + dbIdx, err := newNamespaceIndex(md, opts) assert.NoError(t, err) idx, ok := dbIdx.(*nsIndex) @@ -199,14 +203,47 @@ func TestNamespaceIndexInsertRetentionPeriod(t *testing.T) { ) tooOld := now.Add(-1 * idx.bufferPast).Add(-1 * time.Second) - lifecycle.EXPECT().OnIndexFinalize(tooOld) - assert.Error(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, - tags, tooOld, lifecycle)))) + lifecycle.EXPECT(). + OnIndexFinalize(xtime.ToUnixNano(tooOld.Truncate(idx.blockSize))) + entry, document := testWriteBatchEntry(id, tags, tooOld, lifecycle) + batch := testWriteBatch(entry, document, testWriteBatchBlockSizeOption(idx.blockSize)) + + assert.Error(t, idx.WriteBatch(batch)) + + verified := 0 + batch.ForEach(func( + idx int, + entry index.WriteBatchEntry, + doc doc.Document, + result index.WriteBatchEntryResult, + ) { + verified++ + require.Error(t, result.Err) + require.Equal(t, m3dberrors.ErrTooPast, result.Err) + }) + require.Equal(t, 1, verified) tooNew := now.Add(1 * idx.bufferFuture).Add(1 * time.Second) - lifecycle.EXPECT().OnIndexFinalize(tooOld) - assert.Error(t, idx.WriteBatch(testWriteBatch(testWriteBatchEntry(id, - tags, tooNew, lifecycle)))) + lifecycle.EXPECT(). + OnIndexFinalize(xtime.ToUnixNano(tooNew.Truncate(idx.blockSize))) + entry, document = testWriteBatchEntry(id, tags, tooNew, lifecycle) + batch = testWriteBatch(entry, document, testWriteBatchBlockSizeOption(idx.blockSize)) + assert.Error(t, idx.WriteBatch(batch)) + + verified = 0 + batch.ForEach(func( + idx int, + entry index.WriteBatchEntry, + doc doc.Document, + result index.WriteBatchEntryResult, + ) { + verified++ + require.Error(t, result.Err) + require.Equal(t, m3dberrors.ErrTooFuture, result.Err) + }) + require.Equal(t, 1, verified) + + assert.NoError(t, dbIdx.Close()) } func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { diff --git a/storage/series/lookup/entry.go b/storage/series/lookup/entry.go index d06a1ce11d..c199409179 100644 --- a/storage/series/lookup/entry.go +++ b/storage/series/lookup/entry.go @@ -98,7 +98,6 @@ func (entry *Entry) NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) boo alreadyIndexedOrAttempted := entry.reverseIndex.indexedOrAttemptedWithRLock(indexBlockStartForWrite) entry.reverseIndex.RUnlock() if alreadyIndexedOrAttempted { - fmt.Printf("! GOOD alreadyIndexedOrAttempted true 1 for start %v\n", indexBlockStartForWrite.ToTime().String()) // if so, the entry does not need to be indexed. return false } @@ -112,13 +111,11 @@ func (entry *Entry) NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) boo alreadyIndexedOrAttempted = entry.reverseIndex.indexedOrAttemptedWithRLock(indexBlockStartForWrite) if alreadyIndexedOrAttempted { entry.reverseIndex.Unlock() - fmt.Printf("! GOOD alreadyIndexedOrAttempted true 2 for start %v\n", indexBlockStartForWrite.ToTime().String()) return false } entry.reverseIndex.setAttemptWithWLock(indexBlockStartForWrite, true) entry.reverseIndex.Unlock() - fmt.Printf("! BAD alreadyIndexedOrAttempted false for start %v\n", indexBlockStartForWrite.ToTime().String()) return true } @@ -132,7 +129,6 @@ func (entry *Entry) OnIndexPrepare() { // OnIndexSuccess marks the given block start as successfully indexed. func (entry *Entry) OnIndexSuccess(blockStartNanos xtime.UnixNano) { - fmt.Printf("! on indexsuccess for %s at %v\n", entry.Series.ID().String(), blockStartNanos.ToTime().String()) entry.reverseIndex.Lock() entry.reverseIndex.setSuccessWithWLock(blockStartNanos) entry.reverseIndex.Unlock() @@ -203,6 +199,14 @@ func (s *entryIndexState) setSuccessWithWLock(t xtime.UnixNano) { break } } + + // NB(r): If not inserted state yet that means we need to make an insertion, + // this will happen if synchronously indexing and we haven't called + // NeedIndexUpdate before we indexed the series. + s.insertBlockState(entryIndexBlockState{ + blockStart: t, + success: true, + }) } func (s *entryIndexState) setAttemptWithWLock(t xtime.UnixNano, attempt bool) { @@ -214,13 +218,17 @@ func (s *entryIndexState) setAttemptWithWLock(t xtime.UnixNano, attempt bool) { } } + s.insertBlockState(entryIndexBlockState{ + blockStart: t, + attempt: attempt, + }) +} + +func (s *entryIndexState) insertBlockState(newState entryIndexBlockState) { // i.e. we don't have the block start in the slice // if we have less than 3 elements, we can just insert an element to the slice. if len(s.states) < 3 { - s.states = append(s.states, entryIndexBlockState{ - blockStart: t, - attempt: attempt, - }) + s.states = append(s.states, newState) return } @@ -237,8 +245,5 @@ func (s *entryIndexState) setAttemptWithWLock(t xtime.UnixNano, attempt bool) { } } - s.states[minIdx] = entryIndexBlockState{ - blockStart: t, - attempt: attempt, - } + s.states[minIdx] = newState } diff --git a/storage/shard_index_test.go b/storage/shard_index_test.go index 835ac7c427..2ddc662e04 100644 --- a/storage/shard_index_test.go +++ b/storage/shard_index_test.go @@ -27,7 +27,6 @@ import ( "testing" "time" - "github.com/m3db/m3db/clock" "github.com/m3db/m3db/runtime" "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/namespace" @@ -179,9 +178,15 @@ func TestShardAsyncIndexOnlyWhenNotIndexed(t *testing.T) { var numCalls int32 opts := testDatabaseOptions() + blockSize := time.Hour now := time.Now() - nextWriteTime := now.Truncate(time.Hour) + nextWriteTime := now.Truncate(blockSize) idx := NewMocknamespaceIndex(ctrl) + idx.EXPECT().BlockStartForWriteTime(gomock.Any()). + DoAndReturn(func(t time.Time) xtime.UnixNano { + return xtime.ToUnixNano(t.Truncate(blockSize)) + }). + AnyTimes() idx.EXPECT().WriteBatch(gomock.Any()).Do( func(batch *index.WriteBatch) { if batch.Len() == 0 { @@ -233,36 +238,32 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { defer leaktest.CheckTimeout(t, 2*time.Second)() var numCalls int32 - var nowLock sync.Mutex - now := time.Now() - - opts := testDatabaseOptions(). - SetClockOptions(clock.NewOptions().SetNowFn( - func() time.Time { - nowLock.Lock() - n := now - nowLock.Unlock() - return n - })) - blockStart := now.Truncate(time.Hour) + // Make now not rounded exactly to the block size + blockSize := time.Minute + now := time.Now().Truncate(blockSize).Add(time.Second) ctrl := gomock.NewController(t) defer ctrl.Finish() idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().BlockStartForWriteTime(gomock.Any()).Return(blockStart) - idx.EXPECT().WriteBatch(gomock.Any()).Do( - func(batch *index.WriteBatch) { - nowLock.Lock() - now = now.Add(time.Hour) - nowLock.Unlock() + idx.EXPECT().BlockStartForWriteTime(gomock.Any()). + DoAndReturn(func(t time.Time) xtime.UnixNano { + return xtime.ToUnixNano(t.Truncate(blockSize)) + }). + AnyTimes() + idx.EXPECT().WriteBatch(gomock.Any()). + Return(nil). + Do(func(batch *index.WriteBatch) { for _, b := range batch.PendingEntries() { + blockStart := b.Timestamp.Truncate(blockSize) b.OnIndexSeries.OnIndexSuccess(xtime.ToUnixNano(blockStart)) b.OnIndexSeries.OnIndexFinalize(xtime.ToUnixNano(blockStart)) atomic.AddInt32(&numCalls, 1) } - }).Return(nil).AnyTimes() + }). + AnyTimes() + opts := testDatabaseOptions() shard := testDatabaseShardWithIndexFn(t, opts, idx) shard.SetRuntimeOptions(runtime.NewOptions().SetWriteNewSeriesAsync(true)) defer shard.Close() @@ -282,10 +283,11 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { assert.True(t, indexed) // ensure we index because it's expired + nextWriteTime := now.Add(blockSize) assert.NoError(t, shard.WriteTagged(ctx, ident.StringID("foo"), ident.NewTagsIterator(ident.NewTags(ident.StringTag("name", "value"))), - now.Add(time.Minute), 2.0, xtime.Second, nil)) + nextWriteTime, 2.0, xtime.Second, nil)) // wait till we're done indexing. reIndexed := xclock.WaitUntil(func() bool { @@ -295,9 +297,10 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { entry, _, err := shard.tryRetrieveWritableSeries(ident.StringID("foo")) assert.NoError(t, err) - nowLock.Lock() - defer nowLock.Unlock() - assert.True(t, entry.IndexedForBlockStart(xtime.ToUnixNano(blockStart))) + + // make sure we indexed the second write + assert.True(t, entry.IndexedForBlockStart( + xtime.ToUnixNano(nextWriteTime.Truncate(blockSize)))) } // TODO(prateek): wire tests above to use the field `ts` diff --git a/storage/shard_ref_count_test.go b/storage/shard_ref_count_test.go index 114604b067..a133db31b1 100644 --- a/storage/shard_ref_count_test.go +++ b/storage/shard_ref_count_test.go @@ -21,6 +21,7 @@ package storage import ( + "fmt" "sync/atomic" "testing" "time" @@ -116,15 +117,26 @@ func TestShardWriteTaggedSyncRefCountMockIndex(t *testing.T) { blockSize := namespace.NewIndexOptions().BlockSize() idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().WriteBatch(gomock.Any()).Do( - func(batch *index.WriteBatch) { + idx.EXPECT().BlockStartForWriteTime(gomock.Any()). + DoAndReturn(func(t time.Time) xtime.UnixNano { + return xtime.ToUnixNano(t.Truncate(blockSize)) + }). + AnyTimes() + idx.EXPECT().WriteBatch(gomock.Any()). + Return(nil). + Do(func(batch *index.WriteBatch) { if batch.Len() != 1 { - panic("expected batch len 1") // require.Equal(...) silently kills goroutines + // require.Equal(...) silently kills goroutines + panic(fmt.Sprintf("expected batch len 1: len=%d", batch.Len())) } + entry := batch.PendingEntries()[0] + blockStart := xtime.ToUnixNano(entry.Timestamp.Truncate(blockSize)) onIdx := entry.OnIndexSeries - onIdx.OnIndexFinalize(xtime.ToUnixNano(entry.Timestamp.Truncate(blockSize))) - }).Return(nil).AnyTimes() + onIdx.OnIndexSuccess(blockStart) + onIdx.OnIndexFinalize(blockStart) + }). + AnyTimes() testShardWriteTaggedSyncRefCount(t, idx) } @@ -302,15 +314,23 @@ func TestShardWriteTaggedAsyncRefCountMockIndex(t *testing.T) { blockSize := namespace.NewIndexOptions().BlockSize() idx := NewMocknamespaceIndex(ctrl) - idx.EXPECT().WriteBatch(gomock.Any()).Do( - func(batch *index.WriteBatch) { - if batch.Len() != 1 { - panic("expected batch len 1") // require.Equal(...) silently kills goroutines + idx.EXPECT().BlockStartForWriteTime(gomock.Any()). + DoAndReturn(func(t time.Time) xtime.UnixNano { + return xtime.ToUnixNano(t.Truncate(blockSize)) + }). + AnyTimes() + idx.EXPECT().WriteBatch(gomock.Any()). + Return(nil). + Do(func(batch *index.WriteBatch) { + for _, entry := range batch.PendingEntries() { + blockStart := xtime.ToUnixNano(entry.Timestamp.Truncate(blockSize)) + onIdx := entry.OnIndexSeries + onIdx.OnIndexSuccess(blockStart) + onIdx.OnIndexFinalize(blockStart) } - entry := batch.PendingEntries()[0] - onIdx := entry.OnIndexSeries - onIdx.OnIndexFinalize(xtime.ToUnixNano(entry.Timestamp.Truncate(blockSize))) - }).Return(nil).AnyTimes() + }). + AnyTimes() + testShardWriteTaggedAsyncRefCount(t, idx) } From e13652593594746473f0ff1bb93356f9f564c0be Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 15 May 2018 16:54:30 -0400 Subject: [PATCH 30/33] Fix build error --- storage/series/lookup/entry.go | 1 - 1 file changed, 1 deletion(-) diff --git a/storage/series/lookup/entry.go b/storage/series/lookup/entry.go index c199409179..3fa3c096a0 100644 --- a/storage/series/lookup/entry.go +++ b/storage/series/lookup/entry.go @@ -21,7 +21,6 @@ package lookup import ( - "fmt" "sync" "sync/atomic" From 43645e80d3c196a9ada186d2b0d3f3c23d058103 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 15 May 2018 17:44:08 -0400 Subject: [PATCH 31/33] Fix final test build error and metalint issues --- client/session_fetch_bulk_blocks_test.go | 17 +++++++++++++---- storage/block/block_test.go | 4 ---- storage/index/convert/convert.go | 6 +++--- storage/index_block_test.go | 9 --------- storage/series/lookup/entry.go | 2 +- storage/shard.go | 4 ++-- 6 files changed, 19 insertions(+), 23 deletions(-) diff --git a/client/session_fetch_bulk_blocks_test.go b/client/session_fetch_bulk_blocks_test.go index fc9288ec58..b1b20e9b4d 100644 --- a/client/session_fetch_bulk_blocks_test.go +++ b/client/session_fetch_bulk_blocks_test.go @@ -24,6 +24,7 @@ import ( "bytes" "fmt" "io" + "io/ioutil" "math" "sort" "sync" @@ -714,7 +715,7 @@ func assertFetchBlocksFromPeersResult( seg, err := stream.Segment() require.NoError(t, err) - actualData := append(seg.Head.Bytes(), seg.Tail.Bytes()...) + actualData := append(bytesFor(seg.Head), bytesFor(seg.Tail)...) // compare actual v expected data if len(expectedData) != len(actualData) { @@ -1716,8 +1717,9 @@ func TestBlocksResultAddBlockFromPeerReadMerged(t *testing.T) { require.NoError(t, err) // Assert block has data - assert.Equal(t, []byte{1, 2}, seg.Head.Bytes()) - assert.Equal(t, []byte{3}, seg.Tail.Bytes()) + data, err := ioutil.ReadAll(xio.NewSegmentReader(seg)) + require.NoError(t, err) + assert.Equal(t, []byte{1, 2, 3}, data) } func TestBlocksResultAddBlockFromPeerReadUnmerged(t *testing.T) { @@ -2468,7 +2470,7 @@ func assertFetchBootstrapBlocksResult( require.NoError(t, err) seg, err := stream.Segment() require.NoError(t, err) - actualData := append(seg.Head.Bytes(), seg.Tail.Bytes()...) + actualData := append(bytesFor(seg.Head), bytesFor(seg.Tail)...) assert.Equal(t, expectedData, actualData) } else if block.segments.unmerged != nil { assert.Fail(t, "unmerged comparison not supported") @@ -2477,6 +2479,13 @@ func assertFetchBootstrapBlocksResult( } } +func bytesFor(data checked.Bytes) []byte { + if data == nil { + return nil + } + return data.Bytes() +} + func assertEnqueueChannel( t *testing.T, expected []receivedBlockMetadata, diff --git a/storage/block/block_test.go b/storage/block/block_test.go index 491f6ee7e7..0e424cdd0f 100644 --- a/storage/block/block_test.go +++ b/storage/block/block_test.go @@ -95,10 +95,6 @@ func TestDatabaseBlockChecksum(t *testing.T) { require.Equal(t, block.checksum, checksum) } -type testDatabaseBlockFn func(block *dbBlock) - -type testDatabaseBlockAssertionFn func(t *testing.T, block *dbBlock) - type segmentReaderFinalizeCounter struct { xio.SegmentReader // Use a pointer so we can update it from the Finalize method diff --git a/storage/index/convert/convert.go b/storage/index/convert/convert.go index 020bd90745..fea8d6c2e4 100644 --- a/storage/index/convert/convert.go +++ b/storage/index/convert/convert.go @@ -40,9 +40,9 @@ var ( ReservedFieldNameID = doc.IDReservedFieldName ) +// ValidateMetric will validate a metric for use in the m3ninx subsytem // FOLLOWUP(r): Rename ValidateMetric to ValidateSeries (metric terminiology // is not common in the codebase) -// ValidateMetric will validate a metric for use in the m3ninx subsytem func ValidateMetric(id ident.ID, tags ident.Tags) error { for _, tag := range tags.Values() { if bytes.Equal(ReservedFieldNameID, tag.Name.Bytes()) { @@ -52,9 +52,9 @@ func ValidateMetric(id ident.ID, tags ident.Tags) error { return nil } +// FromMetric converts the provided metric id+tags into a document. // FOLLOWUP(r): Rename FromMetric to FromSeries (metric terminiology // is not common in the codebase) -// FromMetric converts the provided metric id+tags into a document. func FromMetric(id ident.ID, tags ident.Tags) (doc.Document, error) { fields := make([]doc.Field, 0, len(tags.Values())) for _, tag := range tags.Values() { @@ -73,9 +73,9 @@ func FromMetric(id ident.ID, tags ident.Tags) (doc.Document, error) { }, nil } +// FromMetricIter converts the provided metric id+tags into a document. // FOLLOWUP(r): Rename FromMetric to FromSeries (metric terminiology // is not common in the codebase) -// FromMetricIter converts the provided metric id+tags into a document. func FromMetricIter(id ident.ID, tags ident.TagIterator) (doc.Document, error) { fields := make([]doc.Field, 0, tags.Remaining()) for tags.Next() { diff --git a/storage/index_block_test.go b/storage/index_block_test.go index 01fdb0ee04..b107937cc1 100644 --- a/storage/index_block_test.go +++ b/storage/index_block_test.go @@ -510,12 +510,3 @@ func TestNamespaceIndexBlockQuery(t *testing.T) { _, err = idx.Query(ctx, q, qOpts) require.NoError(t, err) } - -func testDoc1() doc.Document { - return doc.Document{ - ID: []byte("foo"), - Fields: []doc.Field{ - doc.Field{[]byte("name"), []byte("value")}, - }, - } -} diff --git a/storage/series/lookup/entry.go b/storage/series/lookup/entry.go index 3fa3c096a0..c575cf5c71 100644 --- a/storage/series/lookup/entry.go +++ b/storage/series/lookup/entry.go @@ -92,7 +92,7 @@ func (entry *Entry) IndexedForBlockStart(indexBlockStart xtime.UnixNano) bool { // OnIndexFinalze() call. This is reqiured for correct lifecycle maintenance. func (entry *Entry) NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) bool { // first we try the low-cost path: acquire a RLock and see if the given block start - // has been marked successfull or that we've attempted it. + // has been marked successful or that we've attempted it. entry.reverseIndex.RLock() alreadyIndexedOrAttempted := entry.reverseIndex.indexedOrAttemptedWithRLock(indexBlockStartForWrite) entry.reverseIndex.RUnlock() diff --git a/storage/shard.go b/storage/shard.go index d7b7fe7e8a..b7a944eabd 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -987,14 +987,14 @@ func (s *dbShard) newShardEntry( nameBytes := t.Name.Bytes() if idx := bytes.Index(clonedID, nameBytes); idx != -1 { - tag.Name = ident.BytesID(clonedID[idx : idx+len(nameBytes)]) + tag.Name = clonedID[idx : idx+len(nameBytes)] } else { tag.Name = s.identifierPool.Clone(t.Name) } valueBytes := t.Value.Bytes() if idx := bytes.Index(clonedID, valueBytes); idx != -1 { - tag.Value = ident.BytesID(clonedID[idx : idx+len(valueBytes)]) + tag.Value = clonedID[idx : idx+len(valueBytes)] } else { tag.Value = s.identifierPool.Clone(t.Value) } From bdbbb6cb9f201147158e32e7a86d48d4cb37f672 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 15 May 2018 17:45:01 -0400 Subject: [PATCH 32/33] Remove debugging statements --- storage/shard.go | 5 ----- storage/shard_index_test.go | 1 - 2 files changed, 6 deletions(-) diff --git a/storage/shard.go b/storage/shard.go index b7a944eabd..bd8b932400 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -751,8 +751,6 @@ func (s *dbShard) writeAndIndex( writable := entry != nil - fmt.Printf("!! call for %s\n", id.String()) - // If no entry and we are not writing new series asynchronously if !writable && !opts.writeNewSeriesAsync { // Avoid double lookup by enqueueing insert immediately @@ -776,7 +774,6 @@ func (s *dbShard) writeAndIndex( } writable = true - fmt.Printf("!! just batched and inserted %s\n", id.String()) // NB(r): We just indexed this series if shouldReverseIndex was true shouldReverseIndex = false } @@ -800,11 +797,9 @@ func (s *dbShard) writeAndIndex( commitLogSeriesUniqueIndex = entry.Index if err == nil && shouldReverseIndex { if entry.NeedsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) { - fmt.Printf("!! need index %s YES\n", entry.Series.ID().String()) err = s.insertSeriesForIndexingAsyncBatched(entry, timestamp, opts.writeNewSeriesAsync) } else { - fmt.Printf("!! need index %s NO\n", entry.Series.ID().String()) } } // release the reference we got on entry from `writableSeries` diff --git a/storage/shard_index_test.go b/storage/shard_index_test.go index 2ddc662e04..dee0e77706 100644 --- a/storage/shard_index_test.go +++ b/storage/shard_index_test.go @@ -65,7 +65,6 @@ func TestShardInsertNamespaceIndex(t *testing.T) { indexWrites = append(indexWrites, batch.PendingDocs()...) lock.Unlock() for i, e := range batch.PendingEntries() { - fmt.Printf("!! inserting %s\n", string(batch.PendingDocs()[i].ID)) e.OnIndexSeries.OnIndexSuccess(blockStart) e.OnIndexSeries.OnIndexFinalize(blockStart) batch.PendingEntries()[i].OnIndexSeries = nil From 8dcc8504968e6d8a3fea60907efe997f48f505a1 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 16 May 2018 01:03:36 -0400 Subject: [PATCH 33/33] Address feedback --- storage/index.go | 38 ++++++--- storage/index/convert/convert.go | 22 +++-- storage/index/types.go | 20 +++-- storage/series/lookup/entry.go | 4 +- storage/series/lookup/entry_blackbox_test.go | 2 +- storage/series/lookup/entry_whitebox_test.go | 1 + storage/shard.go | 89 +++++++++++--------- storage/shard_insert_queue.go | 3 +- storage/shard_test.go | 1 + 9 files changed, 110 insertions(+), 70 deletions(-) diff --git a/storage/index.go b/storage/index.go index 7240dfffa0..eaa6d6ec90 100644 --- a/storage/index.go +++ b/storage/index.go @@ -66,8 +66,9 @@ type nsIndex struct { newBlockFn newBlockFn logger xlog.Logger opts index.Options - metrics nsIndexMetrics nsMetadata namespace.Metadata + + metrics nsIndexMetrics } type nsIndexState struct { @@ -183,8 +184,9 @@ func newNamespaceIndexWithOptions( newBlockFn: newBlockFn, opts: opts, logger: opts.InstrumentOptions().Logger(), - metrics: newNamespaceIndexMetrics(opts.InstrumentOptions().MetricsScope()), nsMetadata: nsMD, + + metrics: newNamespaceIndexMetrics(opts.InstrumentOptions()), } // allocate indexing queue and start it up. @@ -261,7 +263,7 @@ func (i *nsIndex) WriteBatch( if insertMode != index.InsertAsync { wg.Wait() - // Resort the batch by initial enqueue order + // Re-sort the batch by initial enqueue order if numErrs := batch.NumErrs(); numErrs > 0 { // Restore the sort order from whene enqueued for the caller batch.SortByEnqueued() @@ -273,9 +275,6 @@ func (i *nsIndex) WriteBatch( } // WriteBatches is called by the indexInsertQueue. -// FOLLOWUP(prateek): propagate error back up from here to the indexInsertQueue -// so that we can notify users of success/failure correctly in the case of -// sync'd inserts. func (i *nsIndex) writeBatches( batches []*index.WriteBatch, ) { @@ -310,7 +309,7 @@ func (i *nsIndex) writeBatches( } }) - // we sort the inserts by which block they're applicable for, and do the inserts + // Sort the inserts by which block they're applicable for, and do the inserts // for each block, making sure to not try to insert any entries already marked // with a result. batch.ForEachUnmarkedBatchByBlockStart(writeBatchFn) @@ -333,9 +332,19 @@ func (i *nsIndex) writeBatchForBlockStartWithRLock( return } + // NB(r): Capture pending entries so we can emit the latencies + pending := batch.PendingEntries() + // i.e. we have the block and the inserts, perform the writes. result, err := block.WriteBatch(batch) + // record the end to end indexing latency + now := i.nowFn() + for idx := range pending { + took := now.Sub(pending[idx].EnqueuedAt) + i.metrics.InsertEndToEndLatency.Record(took) + } + // NB: we don't need to do anything to the OnIndexSeries refs in `inserts` at this point, // the index.Block WriteBatch assumes responsibility for calling the appropriate methods. if numErr := result.NumError; numErr != 0 { @@ -624,12 +633,16 @@ func (i *nsIndex) unableToAllocBlockInvariantError(err error) error { } type nsIndexMetrics struct { - AsyncInsertErrors tally.Counter - InsertAfterClose tally.Counter - QueryAfterClose tally.Counter + AsyncInsertErrors tally.Counter + InsertAfterClose tally.Counter + QueryAfterClose tally.Counter + InsertEndToEndLatency tally.Timer } -func newNamespaceIndexMetrics(scope tally.Scope) nsIndexMetrics { +func newNamespaceIndexMetrics( + iopts instrument.Options, +) nsIndexMetrics { + scope := iopts.MetricsScope() return nsIndexMetrics{ AsyncInsertErrors: scope.Tagged(map[string]string{ "error_type": "async-insert", @@ -640,5 +653,8 @@ func newNamespaceIndexMetrics(scope tally.Scope) nsIndexMetrics { QueryAfterClose: scope.Tagged(map[string]string{ "error_type": "query-closed", }).Counter("query-after-error"), + InsertEndToEndLatency: instrument.MustCreateSampledTimer( + scope.Timer("insert-end-to-end-latency"), + iopts.MetricsSamplingRate()), } } diff --git a/storage/index/convert/convert.go b/storage/index/convert/convert.go index fea8d6c2e4..2b808b11f7 100644 --- a/storage/index/convert/convert.go +++ b/storage/index/convert/convert.go @@ -29,15 +29,19 @@ import ( "github.com/m3db/m3x/pool" ) -var ( - errUnableToConvertReservedFieldName = errors.New("unable to convert metric due to use of reserved fieldname") - errInvalidResultMissingID = errors.New("corrupt data, unable to extract id") -) - var ( // ReservedFieldNameID is the field name used to index the ID in the // m3ninx subsytem. ReservedFieldNameID = doc.IDReservedFieldName + + // ErrUsingReservedFieldName is the error returned when a metric + // cannot be parsed due to using a resereved field name + ErrUsingReservedFieldName = errors.New( + "unable to parse metric using reserved field name: " + + string(ReservedFieldNameID)) + + errInvalidResultMissingID = errors.New( + "corrupt data, unable to extract id") ) // ValidateMetric will validate a metric for use in the m3ninx subsytem @@ -46,7 +50,7 @@ var ( func ValidateMetric(id ident.ID, tags ident.Tags) error { for _, tag := range tags.Values() { if bytes.Equal(ReservedFieldNameID, tag.Name.Bytes()) { - return errUnableToConvertReservedFieldName + return ErrUsingReservedFieldName } } return nil @@ -59,7 +63,7 @@ func FromMetric(id ident.ID, tags ident.Tags) (doc.Document, error) { fields := make([]doc.Field, 0, len(tags.Values())) for _, tag := range tags.Values() { if bytes.Equal(ReservedFieldNameID, tag.Name.Bytes()) { - return doc.Document{}, errUnableToConvertReservedFieldName + return doc.Document{}, ErrUsingReservedFieldName } name := clone(tag.Name) fields = append(fields, doc.Field{ @@ -81,7 +85,7 @@ func FromMetricIter(id ident.ID, tags ident.TagIterator) (doc.Document, error) { for tags.Next() { tag := tags.Current() if bytes.Equal(ReservedFieldNameID, tag.Name.Bytes()) { - return doc.Document{}, errUnableToConvertReservedFieldName + return doc.Document{}, ErrUsingReservedFieldName } name := clone(tag.Name) fields = append(fields, doc.Field{ @@ -180,7 +184,7 @@ func (t *tagIter) parseNext() (hasNext bool) { // is not using the reserved ID fieldname next := t.docFields[t.currentIdx] if bytes.Equal(ReservedFieldNameID, next.Name) { - t.err = errUnableToConvertReservedFieldName + t.err = ErrUsingReservedFieldName return false } // otherwise, we're good. diff --git a/storage/index/types.go b/storage/index/types.go index 1e57f6e7c7..70782251e7 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -254,6 +254,12 @@ type ForEachWriteBatchByBlockStartFn func( // ForEachUnmarkedBatchByBlockStart allows a caller to perform an operation // with reference to a restricted set of the write batch for each unique block // start for entries that have not been marked completed yet. +// The underlying batch returned is simply the current batch but with updated +// subslices to the relevant entries and documents that are restored at the +// end of `fn` being applied. +// NOTE: This means `fn` cannot perform any asynchronous work that uses the +// arguments passed to it as the args will be invalid at the synchronous +// execution of `fn`. func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( fn ForEachWriteBatchByBlockStartFn, ) { @@ -302,7 +308,9 @@ func (b *WriteBatch) ForEachUnmarkedBatchByBlockStart( } } - // spill over + // We can unconditionally spill over here since we haven't hit any marked + // done entries yet and thanks to sort order there weren't any, therefore + // we can execute all the remaining entries we had. if startIdx < len(allEntries) { b.entries = allEntries[startIdx:] b.docs = allDocs[startIdx:] @@ -389,12 +397,7 @@ func (b *WriteBatch) MarkUnmarkedEntriesSuccess() { // MarkUnmarkedEntriesError marks all unmarked entries as error. func (b *WriteBatch) MarkUnmarkedEntriesError(err error) { for idx := range b.entries { - if b.entries[idx].OnIndexSeries != nil { - blockStart := b.entries[idx].indexBlockStart(b.opts.IndexBlockSize) - b.entries[idx].OnIndexSeries.OnIndexFinalize(blockStart) - b.entries[idx].OnIndexSeries = nil - b.entries[idx].result = WriteBatchEntryResult{Err: err} - } + b.MarkUnmarkedEntryError(err, idx) } } @@ -458,6 +461,9 @@ type WriteBatchEntry struct { // OnIndexSeries is a listener/callback for when this entry is marked done // it is set to nil when the entry is marked done OnIndexSeries OnIndexSeries + // EnqueuedAt is the timestamp that this entry was enqueued for indexing + // so that we can calculate the latency it takes to index the entry + EnqueuedAt time.Time // enqueuedIdx is the idx of the entry when originally enqueued by the call // to append on the write batch enqueuedIdx int diff --git a/storage/series/lookup/entry.go b/storage/series/lookup/entry.go index c575cf5c71..0779531353 100644 --- a/storage/series/lookup/entry.go +++ b/storage/series/lookup/entry.go @@ -81,7 +81,7 @@ func (entry *Entry) IndexedForBlockStart(indexBlockStart xtime.UnixNano) bool { return isIndexed } -// NeedsIndexUpdate returns a bool to indicate if the Entry requires to be indexed +// NeedsIndexUpdate returns a bool to indicate if the Entry needs to be indexed // for the provided blockStart. It only allows a single index attempt at a time // for a single entry. // NB(prateek): NeedsIndexUpdate is a CAS, i.e. when this method returns true, it @@ -89,7 +89,7 @@ func (entry *Entry) IndexedForBlockStart(indexBlockStart xtime.UnixNano) bool { // is going to be sent to the index, and other go routines should not attempt the // same write. Callers are expected to ensure they follow this guideline. // Further, every call to NeedsIndexUpdate which returns true needs to have a corresponding -// OnIndexFinalze() call. This is reqiured for correct lifecycle maintenance. +// OnIndexFinalze() call. This is required for correct lifecycle maintenance. func (entry *Entry) NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) bool { // first we try the low-cost path: acquire a RLock and see if the given block start // has been marked successful or that we've attempted it. diff --git a/storage/series/lookup/entry_blackbox_test.go b/storage/series/lookup/entry_blackbox_test.go index 728e8eeff2..51a3844e11 100644 --- a/storage/series/lookup/entry_blackbox_test.go +++ b/storage/series/lookup/entry_blackbox_test.go @@ -25,10 +25,10 @@ import ( "testing" "time" - "github.com/fortytw2/leaktest" "github.com/m3db/m3db/storage/series/lookup" xtime "github.com/m3db/m3x/time" + "github.com/fortytw2/leaktest" "github.com/stretchr/testify/require" ) diff --git a/storage/series/lookup/entry_whitebox_test.go b/storage/series/lookup/entry_whitebox_test.go index ff9431736c..7971b1ee10 100644 --- a/storage/series/lookup/entry_whitebox_test.go +++ b/storage/series/lookup/entry_whitebox_test.go @@ -25,6 +25,7 @@ import ( "time" xtime "github.com/m3db/m3x/time" + "github.com/stretchr/testify/require" ) diff --git a/storage/shard.go b/storage/shard.go index bd8b932400..82b99d9a1a 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -757,7 +757,8 @@ func (s *dbShard) writeAndIndex( result, err := s.insertSeriesAsyncBatched(id, tags, dbShardInsertAsyncOptions{ hasPendingIndexing: shouldReverseIndex, pendingIndex: dbShardPendingIndex{ - timestamp: timestamp, + timestamp: timestamp, + enqueuedAt: s.nowFn(), }, }) if err != nil { @@ -819,7 +820,8 @@ func (s *dbShard) writeAndIndex( }, hasPendingIndexing: shouldReverseIndex, pendingIndex: dbShardPendingIndex{ - timestamp: timestamp, + timestamp: timestamp, + enqueuedAt: s.nowFn(), }, }) if err != nil { @@ -963,47 +965,51 @@ func (s *dbShard) newShardEntry( // Since series are purged so infrequently the overhead of not releasing // back an ID to a pool is amortized over a long period of time. clonedID := ident.BytesID(append([]byte(nil), id.Bytes()...)) + clonedID.NoFinalize() - // Inlining tag creation here so its obvious why we can safely index - // into clonedID below - tags = tags.Duplicate() - clonedTags := s.identifierPool.Tags() + var clonedTags ident.Tags + if tags.Remaining() > 0 { + // Inlining tag creation here so its obvious why we can safely index + // into clonedID below + clonedTags = s.identifierPool.Tags() + tags = tags.Duplicate() - // Avoid finalizing the tags since series will let them be garbage collected - clonedTags.NoFinalize() + // Avoid finalizing the tags since series will let them be garbage collected + clonedTags.NoFinalize() - for tags.Next() { - t := tags.Current() + for tags.Next() { + t := tags.Current() - // NB(r): Optimization for workloads that embed the tags in the ID is to - // just take a ref to them directly, the cloned ID is frozen by casting to - // a BytesID in newShardEntry - var tag ident.Tag + // NB(r): Optimization for workloads that embed the tags in the ID is to + // just take a ref to them directly, the cloned ID is frozen by casting to + // a BytesID in newShardEntry + var tag ident.Tag - nameBytes := t.Name.Bytes() - if idx := bytes.Index(clonedID, nameBytes); idx != -1 { - tag.Name = clonedID[idx : idx+len(nameBytes)] - } else { - tag.Name = s.identifierPool.Clone(t.Name) - } + nameBytes := t.Name.Bytes() + if idx := bytes.Index(clonedID, nameBytes); idx != -1 { + tag.Name = clonedID[idx : idx+len(nameBytes)] + } else { + tag.Name = s.identifierPool.Clone(t.Name) + } - valueBytes := t.Value.Bytes() - if idx := bytes.Index(clonedID, valueBytes); idx != -1 { - tag.Value = clonedID[idx : idx+len(valueBytes)] - } else { - tag.Value = s.identifierPool.Clone(t.Value) - } + valueBytes := t.Value.Bytes() + if idx := bytes.Index(clonedID, valueBytes); idx != -1 { + tag.Value = clonedID[idx : idx+len(valueBytes)] + } else { + tag.Value = s.identifierPool.Clone(t.Value) + } - clonedTags.Append(tag) - } - err := tags.Err() - tags.Close() - if err != nil { - return nil, err - } + clonedTags.Append(tag) + } + err := tags.Err() + tags.Close() + if err != nil { + return nil, err + } - if err := convert.ValidateMetric(clonedID, clonedTags); err != nil { - return nil, err + if err := convert.ValidateMetric(clonedID, clonedTags); err != nil { + return nil, err + } } series.Reset(clonedID, clonedTags, s.seriesBlockRetriever, @@ -1035,7 +1041,8 @@ func (s *dbShard) insertSeriesForIndexingAsyncBatched( opts: dbShardInsertAsyncOptions{ hasPendingIndexing: true, pendingIndex: dbShardPendingIndex{ - timestamp: timestamp, + timestamp: timestamp, + enqueuedAt: s.nowFn(), }, // indicate we already have inc'd the entry's ref count, so we can correctly // handle the ref counting semantics in `insertSeriesBatch`. @@ -1265,10 +1272,13 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // this method (insertSeriesBatch) via `entryRefCountIncremented` mechanism. entry.OnIndexPrepare() + id := entry.Series.ID() + tags := entry.Series.Tags().Values() + var d doc.Document - d.ID = entry.Series.ID().Bytes() // IDs from shard entries are always set NoFinalize - d.Fields = make(doc.Fields, 0, len(entry.Series.Tags().Values())) - for _, tag := range entry.Series.Tags().Values() { + d.ID = id.Bytes() // IDs from shard entries are always set NoFinalize + d.Fields = make(doc.Fields, 0, len(tags)) + for _, tag := range tags { d.Fields = append(d.Fields, doc.Field{ Name: tag.Name.Bytes(), // Tags from shard entries are always set NoFinalize Value: tag.Value.Bytes(), // Tags from shard entries are always set NoFinalize @@ -1277,6 +1287,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { indexBatch.Append(index.WriteBatchEntry{ Timestamp: pendingIndex.timestamp, OnIndexSeries: entry, + EnqueuedAt: pendingIndex.enqueuedAt, }, d) } diff --git a/storage/shard_insert_queue.go b/storage/shard_insert_queue.go index f2b2011d1d..0c56c0540f 100644 --- a/storage/shard_insert_queue.go +++ b/storage/shard_insert_queue.go @@ -128,7 +128,8 @@ type dbShardPendingWrite struct { } type dbShardPendingIndex struct { - timestamp time.Time + timestamp time.Time + enqueuedAt time.Time } type dbShardPendingRetrievedBlock struct { diff --git a/storage/shard_test.go b/storage/shard_test.go index 62088c2964..cd5397030c 100644 --- a/storage/shard_test.go +++ b/storage/shard_test.go @@ -1099,6 +1099,7 @@ func TestShardNewInvalidShardEntry(t *testing.T) { iter := ident.NewMockTagIterator(ctrl) gomock.InOrder( + iter.EXPECT().Remaining().Return(2), iter.EXPECT().Duplicate().Return(iter), iter.EXPECT().Next().Return(false), iter.EXPECT().Err().Return(fmt.Errorf("random err")),