From d6f6b4d3a74b27bb03a5567431ce3564870ed8b1 Mon Sep 17 00:00:00 2001 From: Prateek Rungta Date: Mon, 10 Jun 2024 21:30:22 -0400 Subject: [PATCH] [dbnode] Add x/encoding.Cloner with the ability to shallow clone m3db response iterators (#4271) --- src/dbnode/generated/mocks/generate.go | 3 +- src/dbnode/integration/integration.go | 2 +- .../integration/shallow_copy_single_test.go | 124 ++++++++ src/dbnode/x/encoding/shallow_cloner.go | 164 +++++++++++ src/dbnode/x/encoding/shallow_cloner_test.go | 229 +++++++++++++++ src/dbnode/x/xio/io_mock.go | 126 +++++++- src/dbnode/x/xpool/xpool_mock.go | 82 ++++++ src/x/generated/mocks/generate.go | 2 +- src/x/ident/ident_mock.go | 274 +++++++++++++++++- 9 files changed, 999 insertions(+), 7 deletions(-) create mode 100644 src/dbnode/integration/shallow_copy_single_test.go create mode 100644 src/dbnode/x/encoding/shallow_cloner.go create mode 100644 src/dbnode/x/encoding/shallow_cloner_test.go create mode 100644 src/dbnode/x/xpool/xpool_mock.go diff --git a/src/dbnode/generated/mocks/generate.go b/src/dbnode/generated/mocks/generate.go index 33242f78c7..9f6e3ead64 100644 --- a/src/dbnode/generated/mocks/generate.go +++ b/src/dbnode/generated/mocks/generate.go @@ -21,7 +21,7 @@ // mockgen rules for generating mocks for exported interfaces (reflection mode) //go:generate sh -c "mockgen -package=fs $PACKAGE/src/dbnode/persist/fs DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,StreamingWriter | genclean -pkg $PACKAGE/src/dbnode/persist/fs -out ../../persist/fs/fs_mock.go" -//go:generate sh -c "mockgen -package=xio $PACKAGE/src/dbnode/x/xio SegmentReader,SegmentReaderPool | genclean -pkg $PACKAGE/src/dbnode/x/xio -out ../../x/xio/io_mock.go" +//go:generate sh -c "mockgen -package=xio $PACKAGE/src/dbnode/x/xio ReaderSliceOfSlicesIterator,SegmentReader,SegmentReaderPool | genclean -pkg $PACKAGE/src/dbnode/x/xio -out ../../x/xio/io_mock.go" //go:generate sh -c "mockgen -package=digest -destination=../../digest/digest_mock.go $PACKAGE/src/dbnode/digest ReaderWithDigest" //go:generate sh -c "mockgen -package=series $PACKAGE/src/dbnode/storage/series DatabaseSeries,QueryableBlockRetriever | genclean -pkg $PACKAGE/src/dbnode/storage/series -out ../../storage/series/series_mock.go" //go:generate sh -c "mockgen -package=storage $PACKAGE/src/dbnode/storage IndexWriter | genclean -pkg $PACKAGE/src/dbnode/storage -out ../../storage/lookup_mock.go" @@ -44,5 +44,6 @@ //go:generate sh -c "mockgen -package=writes -destination=../../ts/writes/write_batch_mock.go -source=../../ts/writes/types.go" //go:generate sh -c "mockgen -package=index -destination=../../storage/index/index_mock.go -source=../../storage/index/types.go" //go:generate sh -c "mockgen -package=permits -destination=../../storage/limits/permits/permits_mock.go -source=../../storage/limits/permits/types.go" +//go:generate sh -c "mockgen -package=xpool -destination=../../x/xpool/xpool_mock.go -source=../../x/xpool/types.go" package mocks diff --git a/src/dbnode/integration/integration.go b/src/dbnode/integration/integration.go index e9a6559088..c3b15d92fe 100644 --- a/src/dbnode/integration/integration.go +++ b/src/dbnode/integration/integration.go @@ -60,7 +60,7 @@ import ( ) const ( - multiAddrPortStart = 9000 + multiAddrPortStart = 10000 multiAddrPortEach = 5 ) diff --git a/src/dbnode/integration/shallow_copy_single_test.go b/src/dbnode/integration/shallow_copy_single_test.go new file mode 100644 index 0000000000..2a6e00c95f --- /dev/null +++ b/src/dbnode/integration/shallow_copy_single_test.go @@ -0,0 +1,124 @@ +//go:build integration +// +build integration + +// +// Copyright (c) 2019 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 integration + +import ( + "testing" + "time" + + "github.com/m3db/m3/src/cluster/services" + "github.com/m3db/m3/src/cluster/shard" + "github.com/m3db/m3/src/dbnode/client" + "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/topology" + xenc "github.com/m3db/m3/src/dbnode/x/encoding" + "github.com/m3db/m3/src/x/ident" + xtime "github.com/m3db/m3/src/x/time" + + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +func TestShallowCopySingleSeries(t *testing.T) { + if testing.Short() { + t.SkipNow() // Just skip if we're doing a short run + } + + var ( + numShards = defaultNumShards + minShard = uint32(0) + maxShard = uint32(numShards - 1) + instances = []services.ServiceInstance{ + node(t, 0, newClusterShardsRange(minShard, maxShard, shard.Available)), + node(t, 1, newClusterShardsRange(minShard, maxShard, shard.Available)), + node(t, 2, newClusterShardsRange(minShard, maxShard, shard.Available)), + } + ) + nodes, closeFn, clientopts := makeMultiNodeSetup(t, numShards, true, true, instances) //nolint:govet + clientopts = clientopts. + SetWriteConsistencyLevel(topology.ConsistencyLevelAll). + SetReadConsistencyLevel(topology.ReadConsistencyLevelAll) + + defer closeFn() + log := nodes[0].StorageOpts().InstrumentOptions().Logger() + for _, n := range nodes { + require.NoError(t, n.StartServer()) + } + + c, err := client.NewClient(clientopts) + require.NoError(t, err) + session, err := c.NewSession() + require.NoError(t, err) + defer session.Close() + + now := xtime.ToUnixNano(nodes[0].DB().Options().ClockOptions().NowFn()()) + start := time.Now() + log.Info("starting data write") + + id := ident.StringID("foo") + + for i := 0; i < 10; i++ { + ts := now.Truncate(time.Second).Add(time.Duration(i * int(time.Second))) + writeErr := session.Write(testNamespaces[0], id, ts, float64(i), xtime.Second, nil) + require.NoError(t, writeErr) + } + log.Info("test data written", zap.Duration("took", time.Since(start))) + + expectedValues := []float64{0, 1, 2, 3, 4, 5, 6, 7, 8, 9} + + iter, err := session.Fetch(testNamespaces[0], id, now.Add(-time.Hour), now.Add(time.Hour)) + require.NoError(t, err) + + iterPools, err := session.IteratorPools() + require.NoError(t, err) + + cloner := xenc.NewShallowCloner(iterPools) + + // ensure we're able to make multiple clones and iterate them without + // affecting the ability to make further clones. + iterCopyA, err := cloner.CloneSeriesIterator(iter) + require.NoError(t, err) + require.Equal(t, expectedValues, iterToValueSlice(t, iterCopyA)) + + iterCopyB, err := cloner.CloneSeriesIterator(iter) + require.NoError(t, err) + require.Equal(t, expectedValues, iterToValueSlice(t, iterCopyB)) + + log.Info("data is readable", zap.Duration("took", time.Since(start))) +} + +func iterToValueSlice( + t *testing.T, + iter encoding.Iterator, +) []float64 { + valueSlice := make([]float64, 0, 10) + for iter.Next() { + dp, _, _ := iter.Current() + valueSlice = append(valueSlice, dp.Value) + } + require.NoError(t, iter.Err()) + iter.Close() + return valueSlice +} diff --git a/src/dbnode/x/encoding/shallow_cloner.go b/src/dbnode/x/encoding/shallow_cloner.go new file mode 100644 index 0000000000..973104f7e7 --- /dev/null +++ b/src/dbnode/x/encoding/shallow_cloner.go @@ -0,0 +1,164 @@ +// Copyright (c) 2024 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 encoding + +import ( + "time" + + enc "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/ts" + "github.com/m3db/m3/src/dbnode/x/xio" + xtime "github.com/m3db/m3/src/x/time" +) + +// Cloner makes a clone of the given encoding type. +type Cloner interface { + CloneSeriesIterator(enc.SeriesIterator) (enc.SeriesIterator, error) +} + +type shallowCloner struct { + pools enc.IteratorPools + + cloneBlockReaderFn cloneBlockReaderFn +} + +// NewShallowCloner returns a cloner which makes a shallow copy of the given type. +func NewShallowCloner(pools enc.IteratorPools) Cloner { + cloner := shallowCloner{ + pools: pools, + } + cloner.cloneBlockReaderFn = cloner.cloneBlockReader + return cloner +} + +// CloneSeriesIterator makes a shallow copy of the given series iterator. +// i.e. The returned iterator can be iterated independently of the original iterator. +// It does NOT copy the underlying tsz data, only points to the original iterator's data. +// nb: +// - The lifecycle of the returned iterator is only valid until the original iterator is valid. +// - Do NOT iterate the original iterator, as it can release resources held on to by the cloned +// iterators. +func (s shallowCloner) CloneSeriesIterator(iter enc.SeriesIterator) (enc.SeriesIterator, error) { + replicas, err := iter.Replicas() + if err != nil { + return nil, err + } + + replicaCopies := s.pools.MultiReaderIteratorArray().Get(len(replicas)) + replicaCopies = replicaCopies[:0] + for _, replica := range replicas { + replicaCopy, err := s.cloneMultiReaderIterator(replica) + if err != nil { + return nil, err + } + replicaCopies = append(replicaCopies, replicaCopy) + } + + iterCopy := s.pools.SeriesIterator().Get() + iterCopy.Reset(enc.SeriesIteratorOptions{ + ID: s.pools.ID().Clone(iter.ID()), + Tags: iter.Tags().Duplicate(), + StartInclusive: iter.Start(), + EndExclusive: iter.End(), + Replicas: replicaCopies, + }) + + return iterCopy, nil +} + +const ( + _initReadersSize = 5 +) + +func (s shallowCloner) cloneMultiReaderIterator( + iter enc.MultiReaderIterator, +) (enc.MultiReaderIterator, error) { + // TODO: pool these slice allocations + blockReaderCopies := make([][]xio.BlockReader, 0, _initReadersSize) + + // nb: the implementation below requires iteration of the blocksIters + // underlying the MultiReaderIterator. While we make sure to reset the + // state of the iterator back to what it was originally, one important + // caveat to callout: this iteration is not-threadsafe. i.e. if copies + // are going to be made by different go-routines, the synchronization + // has to be done at a level above this. + readers := iter.Readers() + initIdx := readers.Index() + + // nb: we start by assuming next=true for the first pass through the readersIterator + // as the multiReaderiterator already calls Next() on the readersIterator when + // it acquires the iter. + for next := true; next; next = readers.Next() { + currLen, currStart, currBlockSize := readers.CurrentReaders() + currentCopies := make([]xio.BlockReader, 0, currLen) + for i := 0; i < currLen; i++ { + currReader := readers.CurrentReaderAt(i) + currCopy, err := s.cloneBlockReaderFn(currReader, currStart, currBlockSize) + if err != nil { + return nil, err + } + currentCopies = append(currentCopies, currCopy) + } + blockReaderCopies = append(blockReaderCopies, currentCopies) + } + + // reset the reader to the original state + readers.RewindToIndex(initIdx) + + // TODO: pool this type + sliceOfSlicesIter := xio.NewReaderSliceOfSlicesFromBlockReadersIterator(blockReaderCopies) + multiReaderIterCopy := s.pools.MultiReaderIterator().Get() + multiReaderIterCopy.ResetSliceOfSlices(sliceOfSlicesIter, iter.Schema()) + + return multiReaderIterCopy, nil +} + +type cloneBlockReaderFn func(xio.BlockReader, xtime.UnixNano, time.Duration) (xio.BlockReader, error) + +func (s shallowCloner) cloneBlockReader( + reader xio.BlockReader, + start xtime.UnixNano, + blockSize time.Duration, +) (xio.BlockReader, error) { + // nb: we cannot rely on the reader.Segment.Clone() method as that performs a deep copy. + // i.e. it copies the underlying data []byte as well. The copy we provide only + // copies the wrapper constructs, it still points at the same []byte as the original. + + seg, err := reader.Segment() + if err != nil { + return xio.BlockReader{}, err + } + + head := s.pools.CheckedBytesWrapper().Get(seg.Head.Bytes()) + tail := s.pools.CheckedBytesWrapper().Get(seg.Tail.Bytes()) + checksum := seg.CalculateChecksum() + // nb: the FinalizeNone is of particular importance here. It ensures we don't assume ownership + // of the head/tail bytes. So when Close() is called, we nil out our pointers and stop there. + // (instead of returning the []byte to a pool, and so on). + segCopy := ts.NewSegment(head, tail, checksum, ts.FinalizeNone) + segReader := xio.NewSegmentReader(segCopy) + + return xio.BlockReader{ + SegmentReader: segReader, + Start: start, + BlockSize: blockSize, + }, nil +} diff --git a/src/dbnode/x/encoding/shallow_cloner_test.go b/src/dbnode/x/encoding/shallow_cloner_test.go new file mode 100644 index 0000000000..3ec6b683eb --- /dev/null +++ b/src/dbnode/x/encoding/shallow_cloner_test.go @@ -0,0 +1,229 @@ +// Copyright (c) 2024 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 encoding + +import ( + "fmt" + "testing" + "time" + + "github.com/golang/mock/gomock" + enc "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/ts" + "github.com/m3db/m3/src/dbnode/x/xio" + "github.com/m3db/m3/src/dbnode/x/xpool" + "github.com/m3db/m3/src/x/checked" + "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/serialize" + xtest "github.com/m3db/m3/src/x/test" + xtime "github.com/m3db/m3/src/x/time" + + "github.com/stretchr/testify/require" +) + +func TestCloneMultiReaderIterator(t *testing.T) { + f := newFixture(t) + defer f.ctrl.Finish() + + f.shallowCloner.cloneBlockReaderFn = func(b xio.BlockReader, _ xtime.UnixNano, _ time.Duration) (xio.BlockReader, error) { + return b, nil + } + + testBlockSize := time.Hour + now := time.Now() + testTimeFn := func(i int) xtime.UnixNano { return xtime.ToUnixNano(now.Add(time.Duration(i) * testBlockSize)) } + testBlock := func(start int, data int) xio.BlockReader { + rawHead := []byte(fmt.Sprintf("head-%d", data)) + rawTail := []byte(fmt.Sprintf("tail-%d", data)) + head := checked.NewBytes(rawHead, nil) + tail := checked.NewBytes(rawTail, nil) + seg := ts.NewSegment(head, tail, 0, ts.FinalizeNone) + return xio.BlockReader{ + SegmentReader: xio.NewSegmentReader(seg), + BlockSize: testBlockSize, + Start: testTimeFn(start), + } + } + + testBlocks := [][]xio.BlockReader{ + []xio.BlockReader{testBlock(0, 0)}, + []xio.BlockReader{testBlock(1, 0), testBlock(1, 1)}, + []xio.BlockReader{testBlock(2, 0), testBlock(2, 1), testBlock(2, 2)}, + } + + mockIter := enc.NewMockMultiReaderIterator(f.ctrl) + mockReaders := xio.NewMockReaderSliceOfSlicesIterator(f.ctrl) + gomock.InOrder( + mockIter.EXPECT().Readers().Return(mockReaders), + mockReaders.EXPECT().Index().Return(0), + mockReaders.EXPECT().CurrentReaders().Return(1, testTimeFn(0), testBlockSize), + mockReaders.EXPECT().CurrentReaderAt(0).Return(testBlocks[0][0]), + mockReaders.EXPECT().Next().Return(true), + mockReaders.EXPECT().CurrentReaders().Return(2, testTimeFn(1), testBlockSize), + mockReaders.EXPECT().CurrentReaderAt(0).Return(testBlocks[1][0]), + mockReaders.EXPECT().CurrentReaderAt(1).Return(testBlocks[1][1]), + mockReaders.EXPECT().Next().Return(true), + mockReaders.EXPECT().CurrentReaders().Return(3, testTimeFn(2), testBlockSize), + mockReaders.EXPECT().CurrentReaderAt(0).Return(testBlocks[2][0]), + mockReaders.EXPECT().CurrentReaderAt(1).Return(testBlocks[2][1]), + mockReaders.EXPECT().CurrentReaderAt(2).Return(testBlocks[2][2]), + mockReaders.EXPECT().Next().Return(false), + mockReaders.EXPECT().RewindToIndex(0), + f.pools.multiReaderIterator.EXPECT().Get().Return(mockIter), + mockIter.EXPECT().Schema().Return(nil), + mockIter.EXPECT().ResetSliceOfSlices(gomock.Any(), gomock.Any()).DoAndReturn( + func(readers xio.ReaderSliceOfSlicesIterator, _ namespace.SchemaDescr) { + require.True(t, readers.Next()) + l, s, b := readers.CurrentReaders() + require.Equal(t, 1, l) + require.Equal(t, testTimeFn(0), s) + require.Equal(t, testBlockSize, b) + assertBlocksAreEqualAndPointToSameData(t, testBlocks[0][0], readers.CurrentReaderAt(0)) + + require.True(t, readers.Next()) + l, s, b = readers.CurrentReaders() + require.Equal(t, 2, l) + require.Equal(t, testTimeFn(1), s) + require.Equal(t, testBlockSize, b) + assertBlocksAreEqualAndPointToSameData(t, testBlocks[1][0], readers.CurrentReaderAt(0)) + assertBlocksAreEqualAndPointToSameData(t, testBlocks[1][1], readers.CurrentReaderAt(1)) + + require.True(t, readers.Next()) + l, s, b = readers.CurrentReaders() + require.Equal(t, 3, l) + require.Equal(t, testTimeFn(2), s) + require.Equal(t, testBlockSize, b) + assertBlocksAreEqualAndPointToSameData(t, testBlocks[2][0], readers.CurrentReaderAt(0)) + assertBlocksAreEqualAndPointToSameData(t, testBlocks[2][1], readers.CurrentReaderAt(1)) + assertBlocksAreEqualAndPointToSameData(t, testBlocks[2][2], readers.CurrentReaderAt(2)) + require.False(t, readers.Next()) + }, + ), + ) + + _, err := f.shallowCloner.cloneMultiReaderIterator(mockIter) + require.NoError(t, err) +} + +func TestCloneBlockReader(t *testing.T) { + f := newFixture(t) + defer f.ctrl.Finish() + + testStart := xtime.ToUnixNano(time.Now()) + testBlockSize := time.Hour + + rawHead := []byte("stub-head") + rawTail := []byte("stub-tail") + head := checked.NewBytes(rawHead, nil) + head.IncRef() + tail := checked.NewBytes(rawTail, nil) + tail.IncRef() + + mockSegment := ts.Segment{ + Head: head, + Tail: tail, + Flags: ts.FinalizeNone, + } + mockSegmentReader := xio.NewSegmentReader(mockSegment) + + testReader := xio.BlockReader{ + SegmentReader: mockSegmentReader, + Start: testStart, + BlockSize: time.Hour, + } + + gomock.InOrder( + f.pools.checkedBytesWrapper.EXPECT().Get(rawHead).DoAndReturn(func(b []byte) checked.Bytes { + require.True(t, xtest.ByteSlicesBackedBySameData(rawHead, b)) + return checked.NewBytes(b, nil) + }), + f.pools.checkedBytesWrapper.EXPECT().Get(rawTail).DoAndReturn(func(b []byte) checked.Bytes { + require.True(t, xtest.ByteSlicesBackedBySameData(rawTail, b)) + return checked.NewBytes(b, nil) + }), + ) + clonedReader, err := f.shallowCloner.cloneBlockReader(testReader, testStart, testBlockSize) + require.NoError(t, err) + + assertBlocksAreEqualAndPointToSameData(t, testReader, clonedReader) +} + +func assertBlocksAreEqualAndPointToSameData(t *testing.T, exp xio.BlockReader, obs xio.BlockReader) { + require.Equal(t, exp.BlockSize, obs.BlockSize) + require.Equal(t, exp.Start, obs.Start) + + expSeg, err := exp.Segment() + require.NoError(t, err) + + obsSeg, err := obs.Segment() + require.NoError(t, err) + require.Equal(t, ts.FinalizeNone, obsSeg.Flags) + + require.True(t, expSeg.Equal(&obsSeg)) + require.True(t, xtest.ByteSlicesBackedBySameData(expSeg.Head.Bytes(), obsSeg.Head.Bytes())) + require.True(t, xtest.ByteSlicesBackedBySameData(expSeg.Tail.Bytes(), obsSeg.Tail.Bytes())) +} + +func newFixture(t *testing.T) fixture { + ctrl := gomock.NewController(t) + pools := &testPools{ + multiReaderIteratorArray: enc.NewMockMultiReaderIteratorArrayPool(ctrl), + multiReaderIterator: enc.NewMockMultiReaderIteratorPool(ctrl), + seriesIterator: enc.NewMockSeriesIteratorPool(ctrl), + checkedBytesWrapper: xpool.NewMockCheckedBytesWrapperPool(ctrl), + id: ident.NewMockPool(ctrl), + } + cloner := shallowCloner{pools: pools} + return fixture{ + t: t, + ctrl: ctrl, + pools: pools, + shallowCloner: cloner, + } +} + +type fixture struct { + t *testing.T + ctrl *gomock.Controller + pools *testPools + shallowCloner shallowCloner +} + +type testPools struct { + multiReaderIteratorArray *enc.MockMultiReaderIteratorArrayPool + multiReaderIterator *enc.MockMultiReaderIteratorPool + seriesIterator *enc.MockSeriesIteratorPool + checkedBytesWrapper *xpool.MockCheckedBytesWrapperPool + id *ident.MockPool +} + +var _ enc.IteratorPools = (*testPools)(nil) + +func (t *testPools) TagDecoder() serialize.TagDecoderPool { panic("unimplemented") } +func (t *testPools) TagEncoder() serialize.TagEncoderPool { panic("unimplemented") } +func (t *testPools) CheckedBytesWrapper() xpool.CheckedBytesWrapperPool { return t.checkedBytesWrapper } +func (t *testPools) ID() ident.Pool { return t.id } +func (t *testPools) MultiReaderIterator() enc.MultiReaderIteratorPool { return t.multiReaderIterator } +func (t *testPools) SeriesIterator() enc.SeriesIteratorPool { return t.seriesIterator } +func (t *testPools) MultiReaderIteratorArray() enc.MultiReaderIteratorArrayPool { + return t.multiReaderIteratorArray +} diff --git a/src/dbnode/x/xio/io_mock.go b/src/dbnode/x/xio/io_mock.go index 02fc989c5a..45072c85e7 100644 --- a/src/dbnode/x/xio/io_mock.go +++ b/src/dbnode/x/xio/io_mock.go @@ -1,7 +1,7 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/dbnode/x/xio (interfaces: SegmentReader,SegmentReaderPool) +// Source: github.com/m3db/m3/src/dbnode/x/xio (interfaces: ReaderSliceOfSlicesIterator,SegmentReader,SegmentReaderPool) -// Copyright (c) 2021 Uber Technologies, Inc. +// Copyright (c) 2024 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 @@ -26,13 +26,135 @@ package xio import ( "reflect" + "time" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/pool" + time0 "github.com/m3db/m3/src/x/time" "github.com/golang/mock/gomock" ) +// MockReaderSliceOfSlicesIterator is a mock of ReaderSliceOfSlicesIterator interface. +type MockReaderSliceOfSlicesIterator struct { + ctrl *gomock.Controller + recorder *MockReaderSliceOfSlicesIteratorMockRecorder +} + +// MockReaderSliceOfSlicesIteratorMockRecorder is the mock recorder for MockReaderSliceOfSlicesIterator. +type MockReaderSliceOfSlicesIteratorMockRecorder struct { + mock *MockReaderSliceOfSlicesIterator +} + +// NewMockReaderSliceOfSlicesIterator creates a new mock instance. +func NewMockReaderSliceOfSlicesIterator(ctrl *gomock.Controller) *MockReaderSliceOfSlicesIterator { + mock := &MockReaderSliceOfSlicesIterator{ctrl: ctrl} + mock.recorder = &MockReaderSliceOfSlicesIteratorMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockReaderSliceOfSlicesIterator) EXPECT() *MockReaderSliceOfSlicesIteratorMockRecorder { + return m.recorder +} + +// Close mocks base method. +func (m *MockReaderSliceOfSlicesIterator) Close() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Close") +} + +// Close indicates an expected call of Close. +func (mr *MockReaderSliceOfSlicesIteratorMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockReaderSliceOfSlicesIterator)(nil).Close)) +} + +// CurrentReaderAt mocks base method. +func (m *MockReaderSliceOfSlicesIterator) CurrentReaderAt(arg0 int) BlockReader { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CurrentReaderAt", arg0) + ret0, _ := ret[0].(BlockReader) + return ret0 +} + +// CurrentReaderAt indicates an expected call of CurrentReaderAt. +func (mr *MockReaderSliceOfSlicesIteratorMockRecorder) CurrentReaderAt(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CurrentReaderAt", reflect.TypeOf((*MockReaderSliceOfSlicesIterator)(nil).CurrentReaderAt), arg0) +} + +// CurrentReaders mocks base method. +func (m *MockReaderSliceOfSlicesIterator) CurrentReaders() (int, time0.UnixNano, time.Duration) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CurrentReaders") + ret0, _ := ret[0].(int) + ret1, _ := ret[1].(time0.UnixNano) + ret2, _ := ret[2].(time.Duration) + return ret0, ret1, ret2 +} + +// CurrentReaders indicates an expected call of CurrentReaders. +func (mr *MockReaderSliceOfSlicesIteratorMockRecorder) CurrentReaders() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CurrentReaders", reflect.TypeOf((*MockReaderSliceOfSlicesIterator)(nil).CurrentReaders)) +} + +// Index mocks base method. +func (m *MockReaderSliceOfSlicesIterator) Index() int { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Index") + ret0, _ := ret[0].(int) + return ret0 +} + +// Index indicates an expected call of Index. +func (mr *MockReaderSliceOfSlicesIteratorMockRecorder) Index() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Index", reflect.TypeOf((*MockReaderSliceOfSlicesIterator)(nil).Index)) +} + +// Next mocks base method. +func (m *MockReaderSliceOfSlicesIterator) Next() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Next") + ret0, _ := ret[0].(bool) + return ret0 +} + +// Next indicates an expected call of Next. +func (mr *MockReaderSliceOfSlicesIteratorMockRecorder) Next() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockReaderSliceOfSlicesIterator)(nil).Next)) +} + +// RewindToIndex mocks base method. +func (m *MockReaderSliceOfSlicesIterator) RewindToIndex(arg0 int) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "RewindToIndex", arg0) +} + +// RewindToIndex indicates an expected call of RewindToIndex. +func (mr *MockReaderSliceOfSlicesIteratorMockRecorder) RewindToIndex(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RewindToIndex", reflect.TypeOf((*MockReaderSliceOfSlicesIterator)(nil).RewindToIndex), arg0) +} + +// Size mocks base method. +func (m *MockReaderSliceOfSlicesIterator) Size() (int, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Size") + ret0, _ := ret[0].(int) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Size indicates an expected call of Size. +func (mr *MockReaderSliceOfSlicesIteratorMockRecorder) Size() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Size", reflect.TypeOf((*MockReaderSliceOfSlicesIterator)(nil).Size)) +} + // MockSegmentReader is a mock of SegmentReader interface. type MockSegmentReader struct { ctrl *gomock.Controller diff --git a/src/dbnode/x/xpool/xpool_mock.go b/src/dbnode/x/xpool/xpool_mock.go new file mode 100644 index 0000000000..0873307b7b --- /dev/null +++ b/src/dbnode/x/xpool/xpool_mock.go @@ -0,0 +1,82 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: ../../x/xpool/types.go + +// Copyright (c) 2024 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 xpool is a generated GoMock package. +package xpool + +import ( + "reflect" + + "github.com/m3db/m3/src/x/checked" + + "github.com/golang/mock/gomock" +) + +// MockCheckedBytesWrapperPool is a mock of CheckedBytesWrapperPool interface. +type MockCheckedBytesWrapperPool struct { + ctrl *gomock.Controller + recorder *MockCheckedBytesWrapperPoolMockRecorder +} + +// MockCheckedBytesWrapperPoolMockRecorder is the mock recorder for MockCheckedBytesWrapperPool. +type MockCheckedBytesWrapperPoolMockRecorder struct { + mock *MockCheckedBytesWrapperPool +} + +// NewMockCheckedBytesWrapperPool creates a new mock instance. +func NewMockCheckedBytesWrapperPool(ctrl *gomock.Controller) *MockCheckedBytesWrapperPool { + mock := &MockCheckedBytesWrapperPool{ctrl: ctrl} + mock.recorder = &MockCheckedBytesWrapperPoolMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockCheckedBytesWrapperPool) EXPECT() *MockCheckedBytesWrapperPoolMockRecorder { + return m.recorder +} + +// Get mocks base method. +func (m *MockCheckedBytesWrapperPool) Get(arg0 []byte) checked.Bytes { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Get", arg0) + ret0, _ := ret[0].(checked.Bytes) + return ret0 +} + +// Get indicates an expected call of Get. +func (mr *MockCheckedBytesWrapperPoolMockRecorder) Get(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Get", reflect.TypeOf((*MockCheckedBytesWrapperPool)(nil).Get), arg0) +} + +// Init mocks base method. +func (m *MockCheckedBytesWrapperPool) Init() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Init") +} + +// Init indicates an expected call of Init. +func (mr *MockCheckedBytesWrapperPoolMockRecorder) Init() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Init", reflect.TypeOf((*MockCheckedBytesWrapperPool)(nil).Init)) +} diff --git a/src/x/generated/mocks/generate.go b/src/x/generated/mocks/generate.go index 49faaa37ad..d6e8bacd9c 100644 --- a/src/x/generated/mocks/generate.go +++ b/src/x/generated/mocks/generate.go @@ -21,7 +21,7 @@ // mockgen rules for generating mocks for exported interfaces (reflection mode) //go:generate sh -c "mockgen -package=serialize $PACKAGE/src/x/serialize TagEncoder,TagEncoderPool,TagDecoder,TagDecoderPool,MetricTagsIterator,MetricTagsIteratorPool | genclean -pkg $PACKAGE/src/x/serialize -out ../../serialize/serialize_mock.go" -//go:generate sh -c "mockgen -package=ident $PACKAGE/src/x/ident ID,TagIterator | genclean -pkg $PACKAGE/src/x/ident -out ../../ident/ident_mock.go" +//go:generate sh -c "mockgen -package=ident $PACKAGE/src/x/ident ID,TagIterator,Pool | genclean -pkg $PACKAGE/src/x/ident -out ../../ident/ident_mock.go" //go:generate sh -c "mockgen -package=checked $PACKAGE/src/x/checked Bytes | genclean -pkg $PACKAGE/src/x/checked -out ../../checked/checked_mock.go" //go:generate sh -c "mockgen -package=pool $PACKAGE/src/x/pool CheckedBytesPool,BytesPool | genclean -pkg $PACKAGE/src/x/pool -out ../../pool/pool_mock.go" diff --git a/src/x/ident/ident_mock.go b/src/x/ident/ident_mock.go index 47b74e088b..97fee66068 100644 --- a/src/x/ident/ident_mock.go +++ b/src/x/ident/ident_mock.go @@ -1,7 +1,7 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/x/ident (interfaces: ID,TagIterator) +// Source: github.com/m3db/m3/src/x/ident (interfaces: ID,TagIterator,Pool) -// Copyright (c) 2021 Uber Technologies, Inc. +// Copyright (c) 2024 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 @@ -27,6 +27,9 @@ package ident import ( "reflect" + "github.com/m3db/m3/src/x/checked" + "github.com/m3db/m3/src/x/context" + "github.com/golang/mock/gomock" ) @@ -277,3 +280,270 @@ func (mr *MockTagIteratorMockRecorder) Rewind() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Rewind", reflect.TypeOf((*MockTagIterator)(nil).Rewind)) } + +// MockPool is a mock of Pool interface. +type MockPool struct { + ctrl *gomock.Controller + recorder *MockPoolMockRecorder +} + +// MockPoolMockRecorder is the mock recorder for MockPool. +type MockPoolMockRecorder struct { + mock *MockPool +} + +// NewMockPool creates a new mock instance. +func NewMockPool(ctrl *gomock.Controller) *MockPool { + mock := &MockPool{ctrl: ctrl} + mock.recorder = &MockPoolMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockPool) EXPECT() *MockPoolMockRecorder { + return m.recorder +} + +// BinaryID mocks base method. +func (m *MockPool) BinaryID(arg0 checked.Bytes) ID { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "BinaryID", arg0) + ret0, _ := ret[0].(ID) + return ret0 +} + +// BinaryID indicates an expected call of BinaryID. +func (mr *MockPoolMockRecorder) BinaryID(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BinaryID", reflect.TypeOf((*MockPool)(nil).BinaryID), arg0) +} + +// BinaryTag mocks base method. +func (m *MockPool) BinaryTag(arg0, arg1 checked.Bytes) Tag { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "BinaryTag", arg0, arg1) + ret0, _ := ret[0].(Tag) + return ret0 +} + +// BinaryTag indicates an expected call of BinaryTag. +func (mr *MockPoolMockRecorder) BinaryTag(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BinaryTag", reflect.TypeOf((*MockPool)(nil).BinaryTag), arg0, arg1) +} + +// Clone mocks base method. +func (m *MockPool) Clone(arg0 ID) ID { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Clone", arg0) + ret0, _ := ret[0].(ID) + return ret0 +} + +// Clone indicates an expected call of Clone. +func (mr *MockPoolMockRecorder) Clone(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Clone", reflect.TypeOf((*MockPool)(nil).Clone), arg0) +} + +// CloneTag mocks base method. +func (m *MockPool) CloneTag(arg0 Tag) Tag { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CloneTag", arg0) + ret0, _ := ret[0].(Tag) + return ret0 +} + +// CloneTag indicates an expected call of CloneTag. +func (mr *MockPoolMockRecorder) CloneTag(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CloneTag", reflect.TypeOf((*MockPool)(nil).CloneTag), arg0) +} + +// CloneTags mocks base method. +func (m *MockPool) CloneTags(arg0 Tags) Tags { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CloneTags", arg0) + ret0, _ := ret[0].(Tags) + return ret0 +} + +// CloneTags indicates an expected call of CloneTags. +func (mr *MockPoolMockRecorder) CloneTags(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CloneTags", reflect.TypeOf((*MockPool)(nil).CloneTags), arg0) +} + +// GetBinaryID mocks base method. +func (m *MockPool) GetBinaryID(arg0 context.Context, arg1 checked.Bytes) ID { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetBinaryID", arg0, arg1) + ret0, _ := ret[0].(ID) + return ret0 +} + +// GetBinaryID indicates an expected call of GetBinaryID. +func (mr *MockPoolMockRecorder) GetBinaryID(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetBinaryID", reflect.TypeOf((*MockPool)(nil).GetBinaryID), arg0, arg1) +} + +// GetBinaryTag mocks base method. +func (m *MockPool) GetBinaryTag(arg0 context.Context, arg1, arg2 checked.Bytes) Tag { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetBinaryTag", arg0, arg1, arg2) + ret0, _ := ret[0].(Tag) + return ret0 +} + +// GetBinaryTag indicates an expected call of GetBinaryTag. +func (mr *MockPoolMockRecorder) GetBinaryTag(arg0, arg1, arg2 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetBinaryTag", reflect.TypeOf((*MockPool)(nil).GetBinaryTag), arg0, arg1, arg2) +} + +// GetStringID mocks base method. +func (m *MockPool) GetStringID(arg0 context.Context, arg1 string) ID { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetStringID", arg0, arg1) + ret0, _ := ret[0].(ID) + return ret0 +} + +// GetStringID indicates an expected call of GetStringID. +func (mr *MockPoolMockRecorder) GetStringID(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetStringID", reflect.TypeOf((*MockPool)(nil).GetStringID), arg0, arg1) +} + +// GetStringTag mocks base method. +func (m *MockPool) GetStringTag(arg0 context.Context, arg1, arg2 string) Tag { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetStringTag", arg0, arg1, arg2) + ret0, _ := ret[0].(Tag) + return ret0 +} + +// GetStringTag indicates an expected call of GetStringTag. +func (mr *MockPoolMockRecorder) GetStringTag(arg0, arg1, arg2 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetStringTag", reflect.TypeOf((*MockPool)(nil).GetStringTag), arg0, arg1, arg2) +} + +// GetTagsIterator mocks base method. +func (m *MockPool) GetTagsIterator(arg0 context.Context) TagsIterator { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetTagsIterator", arg0) + ret0, _ := ret[0].(TagsIterator) + return ret0 +} + +// GetTagsIterator indicates an expected call of GetTagsIterator. +func (mr *MockPoolMockRecorder) GetTagsIterator(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTagsIterator", reflect.TypeOf((*MockPool)(nil).GetTagsIterator), arg0) +} + +// Put mocks base method. +func (m *MockPool) Put(arg0 ID) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Put", arg0) +} + +// Put indicates an expected call of Put. +func (mr *MockPoolMockRecorder) Put(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Put", reflect.TypeOf((*MockPool)(nil).Put), arg0) +} + +// PutTag mocks base method. +func (m *MockPool) PutTag(arg0 Tag) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "PutTag", arg0) +} + +// PutTag indicates an expected call of PutTag. +func (mr *MockPoolMockRecorder) PutTag(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PutTag", reflect.TypeOf((*MockPool)(nil).PutTag), arg0) +} + +// PutTags mocks base method. +func (m *MockPool) PutTags(arg0 Tags) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "PutTags", arg0) +} + +// PutTags indicates an expected call of PutTags. +func (mr *MockPoolMockRecorder) PutTags(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PutTags", reflect.TypeOf((*MockPool)(nil).PutTags), arg0) +} + +// PutTagsIterator mocks base method. +func (m *MockPool) PutTagsIterator(arg0 TagsIterator) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "PutTagsIterator", arg0) +} + +// PutTagsIterator indicates an expected call of PutTagsIterator. +func (mr *MockPoolMockRecorder) PutTagsIterator(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PutTagsIterator", reflect.TypeOf((*MockPool)(nil).PutTagsIterator), arg0) +} + +// StringID mocks base method. +func (m *MockPool) StringID(arg0 string) ID { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "StringID", arg0) + ret0, _ := ret[0].(ID) + return ret0 +} + +// StringID indicates an expected call of StringID. +func (mr *MockPoolMockRecorder) StringID(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StringID", reflect.TypeOf((*MockPool)(nil).StringID), arg0) +} + +// StringTag mocks base method. +func (m *MockPool) StringTag(arg0, arg1 string) Tag { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "StringTag", arg0, arg1) + ret0, _ := ret[0].(Tag) + return ret0 +} + +// StringTag indicates an expected call of StringTag. +func (mr *MockPoolMockRecorder) StringTag(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StringTag", reflect.TypeOf((*MockPool)(nil).StringTag), arg0, arg1) +} + +// Tags mocks base method. +func (m *MockPool) Tags() Tags { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Tags") + ret0, _ := ret[0].(Tags) + return ret0 +} + +// Tags indicates an expected call of Tags. +func (mr *MockPoolMockRecorder) Tags() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Tags", reflect.TypeOf((*MockPool)(nil).Tags)) +} + +// TagsIterator mocks base method. +func (m *MockPool) TagsIterator() TagsIterator { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TagsIterator") + ret0, _ := ret[0].(TagsIterator) + return ret0 +} + +// TagsIterator indicates an expected call of TagsIterator. +func (mr *MockPoolMockRecorder) TagsIterator() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TagsIterator", reflect.TypeOf((*MockPool)(nil).TagsIterator)) +}