From 3b16eada192bc86991a0ea071027a0345ec741f9 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Thu, 9 May 2019 16:08:32 -0400 Subject: [PATCH 01/18] Cold flush logic --- src/dbnode/generated-source-files.mk | 20 +- .../server/tchannelthrift/node/service.go | 3 +- src/dbnode/retention/times.go | 3 +- src/dbnode/storage/database.go | 30 +- src/dbnode/storage/dirty_series_map_gen.go | 271 ++++++++++++++ .../storage/dirty_series_new_map_gen.go | 84 +++++ src/dbnode/storage/flush.go | 132 +++++-- src/dbnode/storage/flush_test.go | 45 ++- src/dbnode/storage/fs.go | 12 +- src/dbnode/storage/index.go | 2 +- .../index/field_terms_iterator_test.go | 4 +- src/dbnode/storage/index_block_test.go | 5 +- src/dbnode/storage/index_test.go | 16 +- src/dbnode/storage/namespace.go | 104 ++++-- src/dbnode/storage/namespace_test.go | 59 +-- src/dbnode/storage/repair.go | 2 +- src/dbnode/storage/series/buffer.go | 158 ++++++-- src/dbnode/storage/series/buffer_test.go | 65 ++-- src/dbnode/storage/series/series.go | 34 +- src/dbnode/storage/series/series_mock.go | 77 ++-- src/dbnode/storage/series/series_test.go | 42 +-- src/dbnode/storage/series/types.go | 30 +- src/dbnode/storage/shard.go | 340 +++++++++++++++--- src/dbnode/storage/shard_test.go | 38 +- src/dbnode/storage/types.go | 34 +- ..._reader_slice_from_segment_reader_slice.go | 35 ++ src/x/generated-source-files.mk | 11 +- 27 files changed, 1300 insertions(+), 356 deletions(-) create mode 100644 src/dbnode/storage/dirty_series_map_gen.go create mode 100644 src/dbnode/storage/dirty_series_new_map_gen.go create mode 100644 src/dbnode/x/xio/block_reader_slice_from_segment_reader_slice.go diff --git a/src/dbnode/generated-source-files.mk b/src/dbnode/generated-source-files.mk index f15f2b66fa..4eeae4a722 100644 --- a/src/dbnode/generated-source-files.mk +++ b/src/dbnode/generated-source-files.mk @@ -64,7 +64,8 @@ genny-map-storage-bootstrap-result: .PHONY: genny-map-storage genny-map-storage: \ genny-map-storage-database-namespaces \ - genny-map-storage-shard + genny-map-storage-shard \ + genny-map-storage-dirty-series # Map generation rule for storage/databaseNamespacesMap .PHONY: genny-map-storage-database-namespaces @@ -170,7 +171,22 @@ genny-map-storage-index-aggregation-results: genny-map-storage-index-aggregate-v # This map has a custom constructor; delete the genny generated one rm -f $(m3db_package_path)/src/dbnode/storage/index/new_map_gen.go -# generation rule for all generated arraypools +# Map generation rule for storage/DirtySeriesMap +.PHONY: genny-map-storage-dirty-series +genny-map-storage-dirty-series: + cd $(m3x_package_path) && make hashmap-gen \ + pkg=storage \ + key_type=idAndBlockStart \ + value_type=*idElement \ + value_type_alias=idElement \ + target_package=$(m3db_package)/src/dbnode/storage \ + rename_type_prefix=dirtySeries + # Rename both generated map and constructor files + mv -f $(m3db_package_path)/src/dbnode/storage/map_gen.go $(m3db_package_path)/src/dbnode/storage/dirty_series_map_gen.go + # This map has a custom constructor; delete the genny generated one + rm -f $(m3db_package_path)/src/dbnode/storage/new_map_gen.go + +# Generation rule for all generated arraypools .PHONY: genny-arraypool-all genny-arraypool-all: \ genny-arraypool-node-segments \ diff --git a/src/dbnode/network/server/tchannelthrift/node/service.go b/src/dbnode/network/server/tchannelthrift/node/service.go index 23bb094f95..b18e75653d 100644 --- a/src/dbnode/network/server/tchannelthrift/node/service.go +++ b/src/dbnode/network/server/tchannelthrift/node/service.go @@ -770,8 +770,7 @@ func (s *service) FetchBlocksRaw(tctx thrift.Context, req *rpc.FetchBlocksRawReq // Preallocate starts to maximum size since at least one element will likely // be fetching most blocks for peer bootstrapping ropts := nsMetadata.Options().RetentionOptions() - blockStarts := make([]time.Time, 0, - (ropts.RetentionPeriod()+ropts.FutureRetentionPeriod())/ropts.BlockSize()) + blockStarts := make([]time.Time, 0, ropts.RetentionPeriod()/ropts.BlockSize()) for i, request := range req.Elements { blockStarts = blockStarts[:0] diff --git a/src/dbnode/retention/times.go b/src/dbnode/retention/times.go index 8e6c733668..855286ac1e 100644 --- a/src/dbnode/retention/times.go +++ b/src/dbnode/retention/times.go @@ -34,8 +34,7 @@ func FlushTimeStartForRetentionPeriod(retentionPeriod time.Duration, blockSize t // FlushTimeEnd is the latest flushable time func FlushTimeEnd(opts Options, t time.Time) time.Time { - return FlushTimeEndForBlockSize(opts.BlockSize(), - t.Add(opts.FutureRetentionPeriod()).Add(-opts.BufferPast())) + return FlushTimeEndForBlockSize(opts.BlockSize(), t.Add(-opts.BufferPast())) } // FlushTimeEndForBlockSize is the latest flushable time diff --git a/src/dbnode/storage/database.go b/src/dbnode/storage/database.go index 9dcbd3b3a6..4e17d62f6b 100644 --- a/src/dbnode/storage/database.go +++ b/src/dbnode/storage/database.go @@ -29,12 +29,12 @@ import ( "time" "github.com/m3db/m3/src/dbnode/clock" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage/block" dberrors "github.com/m3db/m3/src/dbnode/storage/errors" "github.com/m3db/m3/src/dbnode/storage/index" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/tracepoint" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" @@ -392,6 +392,34 @@ func (d *db) addNamespacesWithLock(namespaces []namespace.Metadata) error { return nil } +func (d *db) updateNamespaceSchemasWithLock(schemaUpdates []namespace.Metadata) error { + for _, n := range schemaUpdates { + // Ensure namespace exists. + curNamepsace, ok := d.namespaces.Get(n.ID()) + if !ok { + // Should never happen. + return fmt.Errorf("non-existent namespace marked for schema update: %v", n.ID().String()) + } + curSchemaID := "none" + curSchema, found := curNamepsace.SchemaRegistry().GetLatest() + if found { + curSchemaID = curSchema.DeployId() + } + // Log schema update. + latestSchema, found := n.Options().SchemaRegistry().GetLatest() + if !found { + return fmt.Errorf("can not update namespace (%s) schema from %s to empty", n.ID().String(), curSchemaID) + } + d.log.Info("updating database namespace schema", zap.Stringer("namespace", n.ID()), + zap.String("current schema", curSchemaID), zap.String("latest schema", latestSchema.DeployId())) + err := curNamepsace.SetSchemaRegistry(n.Options().SchemaRegistry()) + if err != nil { + return xerrors.Wrapf(err, "failed to update latest schema for namespace %s", n.ID().String()) + } + } + return nil +} + func (d *db) newDatabaseNamespaceWithLock( md namespace.Metadata, ) (databaseNamespace, error) { diff --git a/src/dbnode/storage/dirty_series_map_gen.go b/src/dbnode/storage/dirty_series_map_gen.go new file mode 100644 index 0000000000..6cbef60122 --- /dev/null +++ b/src/dbnode/storage/dirty_series_map_gen.go @@ -0,0 +1,271 @@ +// 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. + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +package storage + +// 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. + +// dirtySeriesMapHash is the hash for a given map entry, this is public to support +// iterating over the map using a native Go for loop. +type dirtySeriesMapHash uint64 + +// dirtySeriesMapHashFn is the hash function to execute when hashing a key. +type dirtySeriesMapHashFn func(idAndBlockStart) dirtySeriesMapHash + +// dirtySeriesMapEqualsFn is the equals key function to execute when detecting equality of a key. +type dirtySeriesMapEqualsFn func(idAndBlockStart, idAndBlockStart) bool + +// dirtySeriesMapCopyFn is the copy key function to execute when copying the key. +type dirtySeriesMapCopyFn func(idAndBlockStart) idAndBlockStart + +// dirtySeriesMapFinalizeFn is the finalize key function to execute when finished with a key. +type dirtySeriesMapFinalizeFn func(idAndBlockStart) + +// dirtySeriesMap uses the genny package to provide a generic hash map that can be specialized +// by running the following command from this root of the repository: +// ``` +// make hashmap-gen pkg=outpkg key_type=Type value_type=Type out_dir=/tmp +// ``` +// Or if you would like to use bytes or ident.ID as keys you can use the +// partially specialized maps to generate your own maps as well: +// ``` +// make byteshashmap-gen pkg=outpkg value_type=Type out_dir=/tmp +// make idhashmap-gen pkg=outpkg value_type=Type out_dir=/tmp +// ``` +// This will output to stdout the generated source file to use for your map. +// It uses linear probing by incrementing the number of the hash created when +// hashing the identifier if there is a collision. +// dirtySeriesMap is a value type and not an interface to allow for less painful +// upgrades when adding/removing methods, it is not likely to need mocking so +// an interface would not be super useful either. +type dirtySeriesMap struct { + _dirtySeriesMapOptions + + // lookup uses hash of the identifier for the key and the MapEntry value + // wraps the value type and the key (used to ensure lookup is correct + // when dealing with collisions), we use uint64 for the hash partially + // because lookups of maps with uint64 keys has a fast path for Go. + lookup map[dirtySeriesMapHash]dirtySeriesMapEntry +} + +// _dirtySeriesMapOptions is a set of options used when creating an identifier map, it is kept +// private so that implementers of the generated map can specify their own options +// that partially fulfill these options. +type _dirtySeriesMapOptions struct { + // hash is the hash function to execute when hashing a key. + hash dirtySeriesMapHashFn + // equals is the equals key function to execute when detecting equality. + equals dirtySeriesMapEqualsFn + // copy is the copy key function to execute when copying the key. + copy dirtySeriesMapCopyFn + // finalize is the finalize key function to execute when finished with a + // key, this is optional to specify. + finalize dirtySeriesMapFinalizeFn + // initialSize is the initial size for the map, use zero to use Go's std map + // initial size and consequently is optional to specify. + initialSize int +} + +// dirtySeriesMapEntry is an entry in the map, this is public to support iterating +// over the map using a native Go for loop. +type dirtySeriesMapEntry struct { + // key is used to check equality on lookups to resolve collisions + key _dirtySeriesMapKey + // value type stored + value *idElement +} + +type _dirtySeriesMapKey struct { + key idAndBlockStart + finalize bool +} + +// Key returns the map entry key. +func (e dirtySeriesMapEntry) Key() idAndBlockStart { + return e.key.key +} + +// Value returns the map entry value. +func (e dirtySeriesMapEntry) Value() *idElement { + return e.value +} + +// _dirtySeriesMapAlloc is a non-exported function so that when generating the source code +// for the map you can supply a public constructor that sets the correct +// hash, equals, copy, finalize options without users of the map needing to +// implement them themselves. +func _dirtySeriesMapAlloc(opts _dirtySeriesMapOptions) *dirtySeriesMap { + m := &dirtySeriesMap{_dirtySeriesMapOptions: opts} + m.Reallocate() + return m +} + +func (m *dirtySeriesMap) newMapKey(k idAndBlockStart, opts _dirtySeriesMapKeyOptions) _dirtySeriesMapKey { + key := _dirtySeriesMapKey{key: k, finalize: opts.finalizeKey} + if !opts.copyKey { + return key + } + + key.key = m.copy(k) + return key +} + +func (m *dirtySeriesMap) removeMapKey(hash dirtySeriesMapHash, key _dirtySeriesMapKey) { + delete(m.lookup, hash) + if key.finalize { + m.finalize(key.key) + } +} + +// Get returns a value in the map for an identifier if found. +func (m *dirtySeriesMap) Get(k idAndBlockStart) (*idElement, bool) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + return entry.value, true + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } + var empty *idElement + return empty, false +} + +// Set will set the value for an identifier. +func (m *dirtySeriesMap) Set(k idAndBlockStart, v *idElement) { + m.set(k, v, _dirtySeriesMapKeyOptions{ + copyKey: true, + finalizeKey: m.finalize != nil, + }) +} + +// dirtySeriesMapSetUnsafeOptions is a set of options to use when setting a value with +// the SetUnsafe method. +type dirtySeriesMapSetUnsafeOptions struct { + NoCopyKey bool + NoFinalizeKey bool +} + +// SetUnsafe will set the value for an identifier with unsafe options for how +// the map treats the key. +func (m *dirtySeriesMap) SetUnsafe(k idAndBlockStart, v *idElement, opts dirtySeriesMapSetUnsafeOptions) { + m.set(k, v, _dirtySeriesMapKeyOptions{ + copyKey: !opts.NoCopyKey, + finalizeKey: !opts.NoFinalizeKey, + }) +} + +type _dirtySeriesMapKeyOptions struct { + copyKey bool + finalizeKey bool +} + +func (m *dirtySeriesMap) set(k idAndBlockStart, v *idElement, opts _dirtySeriesMapKeyOptions) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + m.lookup[hash] = dirtySeriesMapEntry{ + key: entry.key, + value: v, + } + return + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } + + m.lookup[hash] = dirtySeriesMapEntry{ + key: m.newMapKey(k, opts), + value: v, + } +} + +// Iter provides the underlying map to allow for using a native Go for loop +// to iterate the map, however callers should only ever read and not write +// the map. +func (m *dirtySeriesMap) Iter() map[dirtySeriesMapHash]dirtySeriesMapEntry { + return m.lookup +} + +// Len returns the number of map entries in the map. +func (m *dirtySeriesMap) Len() int { + return len(m.lookup) +} + +// Contains returns true if value exists for key, false otherwise, it is +// shorthand for a call to Get that doesn't return the value. +func (m *dirtySeriesMap) Contains(k idAndBlockStart) bool { + _, ok := m.Get(k) + return ok +} + +// Delete will remove a value set in the map for the specified key. +func (m *dirtySeriesMap) Delete(k idAndBlockStart) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + m.removeMapKey(hash, entry.key) + return + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } +} + +// Reset will reset the map by simply deleting all keys to avoid +// allocating a new map. +func (m *dirtySeriesMap) Reset() { + for hash, entry := range m.lookup { + m.removeMapKey(hash, entry.key) + } +} + +// Reallocate will avoid deleting all keys and reallocate a new +// map, this is useful if you believe you have a large map and +// will not need to grow back to a similar size. +func (m *dirtySeriesMap) Reallocate() { + if m.initialSize > 0 { + m.lookup = make(map[dirtySeriesMapHash]dirtySeriesMapEntry, m.initialSize) + } else { + m.lookup = make(map[dirtySeriesMapHash]dirtySeriesMapEntry) + } +} diff --git a/src/dbnode/storage/dirty_series_new_map_gen.go b/src/dbnode/storage/dirty_series_new_map_gen.go new file mode 100644 index 0000000000..6e6cdb19c9 --- /dev/null +++ b/src/dbnode/storage/dirty_series_new_map_gen.go @@ -0,0 +1,84 @@ +// 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. + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +package storage + +import ( + "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/pool" + + "github.com/cespare/xxhash" +) + +// dirtySeriesMapOptions provides options used when created the map. +type dirtySeriesMapOptions struct { + InitialSize int + KeyCopyPool pool.BytesPool +} + +// newDirtySeriesMap returns a new byte keyed map. +func newDirtySeriesMap(opts dirtySeriesMapOptions) *dirtySeriesMap { + var ( + copyFn dirtySeriesMapCopyFn + finalizeFn dirtySeriesMapFinalizeFn + ) + if pool := opts.KeyCopyPool; pool == nil { + copyFn = func(k idAndBlockStart) idAndBlockStart { + return idAndBlockStart{ + id: ident.BytesID(append([]byte(nil), k.id.Bytes()...)), + blockStart: k.blockStart, + } + } + } else { + copyFn = func(k idAndBlockStart) idAndBlockStart { + bytes := k.id.Bytes() + keyLen := len(bytes) + pooled := pool.Get(keyLen)[:keyLen] + copy(pooled, bytes) + return idAndBlockStart{ + id: ident.BytesID(pooled), + blockStart: k.blockStart, + } + } + finalizeFn = func(k idAndBlockStart) { + if slice, ok := k.id.(ident.BytesID); ok { + pool.Put(slice) + } + } + } + return _dirtySeriesMapAlloc(_dirtySeriesMapOptions{ + hash: func(k idAndBlockStart) dirtySeriesMapHash { + hash := uint64(7) + hash = 31*hash + xxhash.Sum64(k.id.Bytes()) + hash = 31*hash + uint64(k.blockStart) + return dirtySeriesMapHash(hash) + }, + equals: func(x, y idAndBlockStart) bool { + return x.id.Equal(y.id) && x.blockStart == y.blockStart + }, + copy: copyFn, + finalize: finalizeFn, + initialSize: opts.InitialSize, + }) +} diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 7c904be06e..cee0ae8f6a 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -47,6 +47,7 @@ const ( // when we haven't begun either a flush or snapshot. flushManagerNotIdle flushManagerFlushInProgress + flushManagerColdFlushInProgress flushManagerSnapshotInProgress flushManagerIndexFlushInProgress ) @@ -63,6 +64,7 @@ type flushManager struct { // are used for emitting granular gauges. state flushManagerState isFlushing tally.Gauge + isColdFlushing tally.Gauge isSnapshotting tally.Gauge isIndexFlushing tally.Gauge // This is a "debug" metric for making sure that the snapshotting process @@ -81,6 +83,7 @@ func newFlushManager( opts: opts, pm: opts.PersistManager(), isFlushing: scope.Gauge("flush"), + isColdFlushing: scope.Gauge("cold-flush"), isSnapshotting: scope.Gauge("snapshot"), isIndexFlushing: scope.Gauge("index-flush"), maxBlocksSnapshottedByNamespace: scope.Gauge("max-blocks-snapshotted-by-namespace"), @@ -102,28 +105,58 @@ func (m *flushManager) Flush( defer m.setState(flushManagerIdle) - // create flush-er - flushPersist, err := m.pm.StartFlushPersist() + namespaces, err := m.database.GetOwnedNamespaces() if err != nil { return err } - namespaces, err := m.database.GetOwnedNamespaces() + // Perform three separate loops through all the namespaces so that we can + // emit better gauges, i.e. all the flushing for all the namespaces happens + // at once, then all the cold flushes, then all the snapshotting. This is + // also slightly better semantically because flushing should take priority + // over cold flushes and snapshotting. + // + // In addition, we need to make sure that for any given shard/blockStart + // combination, we attempt a flush before a snapshot as the snapshotting + // process will attempt to snapshot any unflushed blocks which would be + // wasteful if the block is already flushable. + multiErr := xerrors.NewMultiError() + if err = m.dataWarmFlush(namespaces, tickStart, dbBootstrapStateAtTickStart); err != nil { + multiErr = multiErr.Add(err) + } + + rotatedCommitlogID, err := m.commitlog.RotateLogs() + if err == nil { + if err = m.dataColdFlush(namespaces); err != nil { + multiErr = multiErr.Add(err) + } + + if err = m.dataSnapshot(namespaces, tickStart, rotatedCommitlogID); err != nil { + multiErr = multiErr.Add(err) + } + } else { + multiErr = multiErr.Add(fmt.Errorf("error rotating commitlog in mediator tick: %v", err)) + } + + if err = m.indexFlush(namespaces); err != nil { + multiErr = multiErr.Add(err) + } + + return multiErr.FinalError() +} + +func (m *flushManager) dataWarmFlush( + namespaces []databaseNamespace, + tickStart time.Time, + dbBootstrapStateAtTickStart DatabaseBootstrapState, +) error { + flushPersist, err := m.pm.StartFlushPersist() if err != nil { return err } - // Perform two separate loops through all the namespaces so that we can emit better - // gauges I.E all the flushing for all the namespaces happens at once and then all - // the snapshotting for all the namespaces happens at once. This is also slightly - // better semantically because flushing should take priority over snapshotting. - // - // In addition, we need to make sure that for any given shard/blockStart combination, - // we attempt a flush before a snapshot as the snapshotting process will attempt to - // snapshot any unflushed blocks which would be wasteful if the block is already - // flushable. - multiErr := xerrors.NewMultiError() m.setState(flushManagerFlushInProgress) + multiErr := xerrors.NewMultiError() for _, ns := range namespaces { // Flush first because we will only snapshot if there are no outstanding flushes flushTimes := m.namespaceFlushTimes(ns, tickStart) @@ -147,42 +180,38 @@ func (m *flushManager) Flush( multiErr = multiErr.Add(err) } - err = m.rotateCommitlogAndSnapshot(namespaces, tickStart) - if err != nil { - multiErr = multiErr.Add(err) - } + return multiErr.FinalError() +} - indexFlush, err := m.pm.StartIndexPersist() +func (m *flushManager) dataColdFlush( + namespaces []databaseNamespace, +) error { + flushPersist, err := m.pm.StartFlushPersist() if err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() + return err } - m.setState(flushManagerIndexFlushInProgress) + m.setState(flushManagerColdFlushInProgress) + multiErr := xerrors.NewMultiError() for _, ns := range namespaces { - var ( - indexOpts = ns.Options().IndexOptions() - indexEnabled = indexOpts.Enabled() - ) - if !indexEnabled { - continue + if err = ns.ColdFlush(flushPersist); err != nil { + multiErr = multiErr.Add(err) } - multiErr = multiErr.Add(ns.FlushIndex(indexFlush)) } - multiErr = multiErr.Add(indexFlush.DoneIndex()) + + err = flushPersist.DoneFlush() + if err != nil { + multiErr = multiErr.Add(err) + } return multiErr.FinalError() } -func (m *flushManager) rotateCommitlogAndSnapshot( +func (m *flushManager) dataSnapshot( namespaces []databaseNamespace, tickStart time.Time, + rotatedCommitlogID persist.CommitLogFile, ) error { - rotatedCommitlogID, err := m.commitlog.RotateLogs() - if err != nil { - return fmt.Errorf("error rotating commitlog in mediator tick: %v", err) - } - snapshotID := uuid.NewUUID() snapshotPersist, err := m.pm.StartSnapshotPersist(snapshotID) @@ -224,6 +253,31 @@ func (m *flushManager) rotateCommitlogAndSnapshot( return finalErr } +func (m *flushManager) indexFlush( + namespaces []databaseNamespace, +) error { + indexFlush, err := m.pm.StartIndexPersist() + if err != nil { + return err + } + + m.setState(flushManagerIndexFlushInProgress) + multiErr := xerrors.NewMultiError() + for _, ns := range namespaces { + var ( + indexOpts = ns.Options().IndexOptions() + indexEnabled = indexOpts.Enabled() + ) + if !indexEnabled { + continue + } + multiErr = multiErr.Add(ns.FlushIndex(indexFlush)) + } + multiErr = multiErr.Add(indexFlush.DoneIndex()) + + return multiErr.FinalError() +} + func (m *flushManager) Report() { m.RLock() state := m.state @@ -235,6 +289,12 @@ func (m *flushManager) Report() { m.isFlushing.Update(0) } + if state == flushManagerColdFlushInProgress { + m.isColdFlushing.Update(1) + } else { + m.isColdFlushing.Update(0) + } + if state == flushManagerSnapshotInProgress { m.isSnapshotting.Update(1) } else { @@ -305,7 +365,7 @@ func (m *flushManager) flushNamespaceWithTimes( for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. // Probably want to emit a counter here, but for now just log it. - if err := ns.Flush(t, ShardBootstrapStates, flushPreparer); err != nil { + if err := ns.WarmFlush(t, ShardBootstrapStates, flushPreparer); err != nil { detailedErr := fmt.Errorf("namespace %s failed to flush data: %v", ns.ID().String(), err) multiErr = multiErr.Add(detailedErr) diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index 43c018322e..a62fe67931 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -31,6 +31,7 @@ import ( "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3/src/dbnode/retention" + xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" xtest "github.com/m3db/m3/src/x/test" @@ -140,6 +141,15 @@ func TestFlushManagerFlushAlreadyInProgress(t *testing.T) { // Allow the flush to finish. doneCh <- struct{}{} + // Wait until we start the compaction process. + <-startCh + + // Ensure it doesn't allow a parallel flush. + require.Equal(t, errFlushOperationsInProgress, fm.Flush(now, DatabaseBootstrapState{})) + + // Allow the compaction to finish. + doneCh <- struct{}{} + // Allow the snapshot to begin and finish. <-startCh doneCh <- struct{}{} @@ -161,8 +171,8 @@ func TestFlushManagerFlushDoneFlushError(t *testing.T) { mockSnapshotPersist = persist.NewMockSnapshotPreparer(ctrl) ) - mockFlushPersist.EXPECT().DoneFlush().Return(fakeErr) - mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil) + mockFlushPersist.EXPECT().DoneFlush().Return(fakeErr).Times(2) + mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil).Times(2) mockSnapshotPersist.EXPECT().DoneSnapshot(gomock.Any(), testCommitlogFile).Return(nil) mockPersistManager.EXPECT().StartSnapshotPersist(gomock.Any()).Return(mockSnapshotPersist, nil) @@ -183,7 +193,10 @@ func TestFlushManagerFlushDoneFlushError(t *testing.T) { fm.pm = mockPersistManager now := time.Unix(0, 0) - require.EqualError(t, fakeErr, fm.Flush(now, DatabaseBootstrapState{}).Error()) + multiErr := xerrors.NewMultiError() + multiErr = multiErr.Add(fakeErr) + multiErr = multiErr.Add(fakeErr) + require.EqualError(t, multiErr.FinalError(), fm.Flush(now, DatabaseBootstrapState{}).Error()) } // TestFlushManagerFlushDoneSnapshotError makes sure that snapshot errors do not @@ -193,14 +206,14 @@ func TestFlushManagerFlushDoneSnapshotError(t *testing.T) { defer ctrl.Finish() var ( - fakeErr = errors.New("fake error while marking flush done") + fakeErr = errors.New("fake error while marking snapshot done") mockPersistManager = persist.NewMockManager(ctrl) mockFlushPersist = persist.NewMockFlushPreparer(ctrl) mockSnapshotPersist = persist.NewMockSnapshotPreparer(ctrl) ) - mockFlushPersist.EXPECT().DoneFlush().Return(nil) - mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil) + mockFlushPersist.EXPECT().DoneFlush().Return(nil).Times(2) + mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil).Times(2) mockSnapshotPersist.EXPECT().DoneSnapshot(gomock.Any(), testCommitlogFile).Return(fakeErr) mockPersistManager.EXPECT().StartSnapshotPersist(gomock.Any()).Return(mockSnapshotPersist, nil) @@ -234,8 +247,8 @@ func TestFlushManagerFlushDoneIndexError(t *testing.T) { mockPersistManager = persist.NewMockManager(ctrl) ) - mockFlushPersist.EXPECT().DoneFlush().Return(nil) - mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil) + mockFlushPersist.EXPECT().DoneFlush().Return(nil).Times(2) + mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil).Times(2) mockSnapshotPersist.EXPECT().DoneSnapshot(gomock.Any(), testCommitlogFile).Return(nil) mockPersistManager.EXPECT().StartSnapshotPersist(gomock.Any()).Return(mockSnapshotPersist, nil) @@ -269,7 +282,8 @@ func TestFlushManagerSkipNamespaceIndexingDisabled(t *testing.T) { ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true).AnyTimes() - ns.EXPECT().Flush(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + ns.EXPECT().ColdFlush(gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() var ( @@ -278,8 +292,8 @@ func TestFlushManagerSkipNamespaceIndexingDisabled(t *testing.T) { mockPersistManager = persist.NewMockManager(ctrl) ) - mockFlushPersist.EXPECT().DoneFlush().Return(nil) - mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil) + mockFlushPersist.EXPECT().DoneFlush().Return(nil).Times(2) + mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil).Times(2) mockSnapshotPersist.EXPECT().DoneSnapshot(gomock.Any(), testCommitlogFile).Return(nil) mockPersistManager.EXPECT().StartSnapshotPersist(gomock.Any()).Return(mockSnapshotPersist, nil) @@ -317,7 +331,8 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true).AnyTimes() - ns.EXPECT().Flush(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + ns.EXPECT().ColdFlush(gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().FlushIndex(gomock.Any()).Return(nil) @@ -327,8 +342,8 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { mockPersistManager = persist.NewMockManager(ctrl) ) - mockFlushPersist.EXPECT().DoneFlush().Return(nil) - mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil) + mockFlushPersist.EXPECT().DoneFlush().Return(nil).Times(2) + mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil).Times(2) mockSnapshotPersist.EXPECT().DoneSnapshot(gomock.Any(), testCommitlogFile).Return(nil) mockPersistManager.EXPECT().StartSnapshotPersist(gomock.Any()).Return(mockSnapshotPersist, nil) @@ -483,6 +498,8 @@ func TestFlushManagerFlushSnapshot(t *testing.T) { ns.EXPECT().NeedsFlush(st, st).Return(false) } + ns.EXPECT().ColdFlush(gomock.Any()) + snapshotEnd := now.Add(bufferFuture).Truncate(blockSize) num = numIntervals(start, snapshotEnd, blockSize) for i := 0; i < num; i++ { diff --git a/src/dbnode/storage/fs.go b/src/dbnode/storage/fs.go index 14a91d7cf7..471bcc1454 100644 --- a/src/dbnode/storage/fs.go +++ b/src/dbnode/storage/fs.go @@ -39,8 +39,16 @@ const ( ) type fileOpState struct { - Status fileOpStatus - Version int + // WarmStatus is the status of data persistence for WarmWrites only. + // Each block will only be warm-flushed once, so not keeping track of a + // version here is okay. This is used in the buffer Tick to determine when + // a warm bucket is evictable from memory. + WarmStatus fileOpStatus + // ColdVersion keeps track of data persistence for ColdWrites only. + // Each block can be cold-flushed multiple times, so this tracks which + // version of the flush completed successfully. This is ultimately used in + // the buffer Tick to determine which buckets are evictable. + ColdVersion int NumFailures int } diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 4f9ac49f29..4e0a60dfde 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -799,7 +799,7 @@ func (i *nsIndex) canFlushBlock( start := block.StartTime() dataBlockSize := i.nsMetadata.Options().RetentionOptions().BlockSize() for t := start; t.Before(block.EndTime()); t = t.Add(dataBlockSize) { - if shard.FlushState(t).Status != fileOpSuccess { + if shard.FlushState(t).WarmStatus != fileOpSuccess { return false } } diff --git a/src/dbnode/storage/index/field_terms_iterator_test.go b/src/dbnode/storage/index/field_terms_iterator_test.go index 647a5168ed..31ebfc39bb 100644 --- a/src/dbnode/storage/index/field_terms_iterator_test.go +++ b/src/dbnode/storage/index/field_terms_iterator_test.go @@ -149,7 +149,7 @@ func TestFieldsTermsIteratorTermsOnly(t *testing.T) { } func TestFieldsTermsIteratorEmptyTerm(t *testing.T) { - ctrl := gomock.NewController(xtest.Reporter{t}) + ctrl := gomock.NewController(xtest.Reporter{T: t}) defer ctrl.Finish() seg := newMockSegment(ctrl, map[string][]string{ @@ -162,7 +162,7 @@ func TestFieldsTermsIteratorEmptyTerm(t *testing.T) { } func TestFieldsTermsIteratorEmptyTermInclude(t *testing.T) { - ctrl := gomock.NewController(xtest.Reporter{t}) + ctrl := gomock.NewController(xtest.Reporter{T: t}) defer ctrl.Finish() seg := newMockSegment(ctrl, map[string][]string{ diff --git a/src/dbnode/storage/index_block_test.go b/src/dbnode/storage/index_block_test.go index e794cd2ce3..9720625e62 100644 --- a/src/dbnode/storage/index_block_test.go +++ b/src/dbnode/storage/index_block_test.go @@ -751,7 +751,7 @@ func TestNamespaceIndexBlockAggregateQuery(t *testing.T) { // only queries as much as is needed (wrt to time) ctx := context.NewContext() - q := index.Query{query} + q := index.Query{Query: query} qOpts := index.QueryOptions{ StartInclusive: t0, EndExclusive: now.Add(time.Minute), @@ -862,7 +862,7 @@ func TestNamespaceIndexBlockAggregateQueryReleasingContext(t *testing.T) { // only queries as much as is needed (wrt to time) ctx := context.NewContext() - q := index.Query{query} + q := index.Query{Query: query} qOpts := index.QueryOptions{ StartInclusive: t0, EndExclusive: now.Add(time.Minute), @@ -948,6 +948,7 @@ func TestNamespaceIndexBlockAggregateQueryAggPath(t *testing.T) { // only queries as much as is needed (wrt to time) ctx := context.NewContext() + q := index.Query{Query: query} qOpts := index.QueryOptions{ StartInclusive: t0, EndExclusive: now.Add(time.Minute), diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index 5943a95046..b153f6d749 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -118,8 +118,8 @@ func TestNamespaceIndexFlushSuccess(t *testing.T) { mockShard := NewMockdatabaseShard(ctrl) mockShard.EXPECT().ID().Return(uint32(0)).AnyTimes() - mockShard.EXPECT().FlushState(blockTime).Return(fileOpState{Status: fileOpSuccess}) - mockShard.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{Status: fileOpSuccess}) + mockShard.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}) + mockShard.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}) shards := []databaseShard{mockShard} mockFlush := persist.NewMockIndexFlush(ctrl) @@ -185,8 +185,8 @@ func TestNamespaceIndexFlushShardStateNotSuccess(t *testing.T) { mockShard := NewMockdatabaseShard(ctrl) mockShard.EXPECT().ID().Return(uint32(0)).AnyTimes() - mockShard.EXPECT().FlushState(blockTime).Return(fileOpState{Status: fileOpSuccess}) - mockShard.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{Status: fileOpFailed}) + mockShard.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}) + mockShard.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpFailed}) shards := []databaseShard{mockShard} mockFlush := persist.NewMockIndexFlush(ctrl) @@ -220,13 +220,13 @@ func TestNamespaceIndexFlushSuccessMultipleShards(t *testing.T) { mockShard1 := NewMockdatabaseShard(ctrl) mockShard1.EXPECT().ID().Return(uint32(0)).AnyTimes() - mockShard1.EXPECT().FlushState(blockTime).Return(fileOpState{Status: fileOpSuccess}) - mockShard1.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{Status: fileOpSuccess}) + mockShard1.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}) + mockShard1.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}) mockShard2 := NewMockdatabaseShard(ctrl) mockShard2.EXPECT().ID().Return(uint32(1)).AnyTimes() - mockShard2.EXPECT().FlushState(blockTime).Return(fileOpState{Status: fileOpSuccess}) - mockShard2.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{Status: fileOpSuccess}) + mockShard2.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}) + mockShard2.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}) shards := []databaseShard{mockShard1, mockShard2} diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 03d944d591..163be858c6 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -151,7 +151,8 @@ type databaseNamespaceIndexStatsLastTick struct { type databaseNamespaceMetrics struct { bootstrap instrument.MethodMetrics - flush instrument.MethodMetrics + flushWarmData instrument.MethodMetrics + flushColdData instrument.MethodMetrics flushIndex instrument.MethodMetrics snapshot instrument.MethodMetrics write instrument.MethodMetrics @@ -229,7 +230,8 @@ func newDatabaseNamespaceMetrics(scope tally.Scope, samplingRate float64) databa indexStatusScope := statusScope.SubScope("index") return databaseNamespaceMetrics{ bootstrap: instrument.NewMethodMetrics(scope, "bootstrap", samplingRate), - flush: instrument.NewMethodMetrics(scope, "flush", samplingRate), + flushWarmData: instrument.NewMethodMetrics(scope, "flushWarmData", samplingRate), + flushColdData: instrument.NewMethodMetrics(scope, "flushColdData", samplingRate), flushIndex: instrument.NewMethodMetrics(scope, "flushIndex", samplingRate), snapshot: instrument.NewMethodMetrics(scope, "snapshot", samplingRate), write: instrument.NewMethodMetrics(scope, "write", overrideWriteSamplingRate), @@ -902,7 +904,7 @@ func (n *dbNamespace) Bootstrap(start time.Time, process bootstrap.Process) erro return err } -func (n *dbNamespace) Flush( +func (n *dbNamespace) WarmFlush( blockStart time.Time, shardBootstrapStatesAtTickStart ShardBootstrapStates, flushPersist persist.FlushPreparer, @@ -914,14 +916,14 @@ func (n *dbNamespace) Flush( n.RLock() if n.bootstrapState != Bootstrapped { n.RUnlock() - n.metrics.flush.ReportError(n.nowFn().Sub(callStart)) + n.metrics.flushWarmData.ReportError(n.nowFn().Sub(callStart)) return errNamespaceNotBootstrapped } nsCtx := n.nsContextWithRLock() n.RUnlock() if !n.nopts.FlushEnabled() { - n.metrics.flush.ReportSuccess(n.nowFn().Sub(callStart)) + n.metrics.flushWarmData.ReportSuccess(n.nowFn().Sub(callStart)) return nil } @@ -953,12 +955,12 @@ func (n *dbNamespace) Flush( } // skip flushing if the shard has already flushed data for the `blockStart` - if s := shard.FlushState(blockStart); s.Status == fileOpSuccess { + if s := shard.FlushState(blockStart); s.WarmStatus == fileOpSuccess { continue } // NB(xichen): we still want to proceed if a shard fails to flush its data. // Probably want to emit a counter here, but for now just log it. - if err := shard.Flush(blockStart, flushPersist, nsCtx); err != nil { + if err := shard.WarmFlush(blockStart, flushPersist, nsCtx); err != nil { detailedErr := fmt.Errorf("shard %d failed to flush data: %v", shard.ID(), err) multiErr = multiErr.Add(detailedErr) @@ -966,7 +968,78 @@ func (n *dbNamespace) Flush( } res := multiErr.FinalError() - n.metrics.flush.ReportSuccessOrError(res, n.nowFn().Sub(callStart)) + n.metrics.flushWarmData.ReportSuccessOrError(res, n.nowFn().Sub(callStart)) + return res +} + +// idAndBlockStart is the composite key for the genny map used to keep track of +// dirty series that need to be ColdFlushed. +type idAndBlockStart struct { + id ident.ID + blockStart xtime.UnixNano +} + +type coldFlushReuseableResources struct { + // dirtySeries is a map from a composite key of + // to an element in a list in the dirtySeriesToWrite map. This composite key + // is deliberately made so that this map stays one level deep, making it + // easier to share between shard loops, minimizing the need for allocations. + dirtySeries *dirtySeriesMap + // dirtySeriesToWrite is a map from block start to a list of dirty series + // that have yet to be written to disk. + dirtySeriesToWrite map[xtime.UnixNano]*idList + // idElementPool is a pool of list elements to be used when constructing + // new lists for the dirtySeriesToWrite map. + idElementPool *idElementPool + fsReader fs.DataFileSetReader +} + +func (n *dbNamespace) ColdFlush( + flushPersist persist.FlushPreparer, +) error { + // NB(rartoul): This value can be used for emitting metrics, but should not be used + // for business logic. + callStart := n.nowFn() + + n.RLock() + if n.bootstrapState != Bootstrapped { + n.RUnlock() + n.metrics.flushColdData.ReportError(n.nowFn().Sub(callStart)) + return errNamespaceNotBootstrapped + } + nsCtx := namespace.Context{Schema: n.schemaDescr} + n.RUnlock() + + if !n.nopts.ColdWritesEnabled() { + n.metrics.flushColdData.ReportSuccess(n.nowFn().Sub(callStart)) + return nil + } + + multiErr := xerrors.NewMultiError() + shards := n.GetOwnedShards() + + fsReader, err := fs.NewReader(n.opts.BytesPool(), n.opts.CommitLogOptions().FilesystemOptions()) + if err != nil { + return err + } + resources := coldFlushReuseableResources{ + dirtySeries: newDirtySeriesMap(dirtySeriesMapOptions{}), + dirtySeriesToWrite: make(map[xtime.UnixNano]*idList), + // TODO(juchan) set pool options + idElementPool: newIDElementPool(nil), + fsReader: fsReader, + } + for _, shard := range shards { + err := shard.ColdFlush(flushPersist, resources, nsCtx) + if err != nil { + detailedErr := fmt.Errorf("shard %d failed to compact: %v", shard.ID(), err) + multiErr = multiErr.Add(detailedErr) + // Continue with remaining shards + } + } + + res := multiErr.FinalError() + n.metrics.flushColdData.ReportSuccessOrError(res, n.nowFn().Sub(callStart)) return res } @@ -983,13 +1056,13 @@ func (n *dbNamespace) FlushIndex( n.RUnlock() if !n.nopts.FlushEnabled() || !n.nopts.IndexOptions().Enabled() { - n.metrics.flush.ReportSuccess(n.nowFn().Sub(callStart)) + n.metrics.flushIndex.ReportSuccess(n.nowFn().Sub(callStart)) return nil } shards := n.GetOwnedShards() err := n.reverseIndex.Flush(flush, shards) - n.metrics.flush.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) + n.metrics.flushIndex.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) return err } @@ -1024,15 +1097,6 @@ func (n *dbNamespace) Snapshot( multiErr := xerrors.NewMultiError() shards := n.GetOwnedShards() for _, shard := range shards { - isSnapshotting, _ := shard.SnapshotState() - if isSnapshotting { - // Should never happen because snapshots should never overlap - // each other (controlled by loop in flush manager) - n.log.Error("[invariant violated] tried to snapshot shard that is already snapshotting", - zap.Uint32("shard", shard.ID())) - continue - } - err := shard.Snapshot(blockStart, snapshotTime, snapshotPersist, nsCtx) if err != nil { detailedErr := fmt.Errorf("shard %d failed to snapshot: %v", shard.ID(), err) @@ -1119,7 +1183,7 @@ func (n *dbNamespace) needsFlushWithLock(alignedInclusiveStart time.Time, aligne continue } for _, blockStart := range blockStarts { - if shard.FlushState(blockStart).Status != fileOpSuccess { + if shard.FlushState(blockStart).WarmStatus != fileOpSuccess { return true } } diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index d0660dd1a9..f0022ecd9c 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -414,7 +414,7 @@ func TestNamespaceBootstrapOnlyNonBootstrappedShards(t *testing.T) { func TestNamespaceFlushNotBootstrapped(t *testing.T) { ns, closer := newTestNamespace(t) defer closer() - require.Equal(t, errNamespaceNotBootstrapped, ns.Flush(time.Now(), nil, nil)) + require.Equal(t, errNamespaceNotBootstrapped, ns.WarmFlush(time.Now(), nil, nil)) } func TestNamespaceFlushDontNeedFlush(t *testing.T) { @@ -423,7 +423,7 @@ func TestNamespaceFlushDontNeedFlush(t *testing.T) { defer close() ns.bootstrapState = Bootstrapped - require.NoError(t, ns.Flush(time.Now(), nil, nil)) + require.NoError(t, ns.WarmFlush(time.Now(), nil, nil)) } func TestNamespaceFlushSkipFlushed(t *testing.T) { @@ -440,15 +440,15 @@ func TestNamespaceFlushSkipFlushed(t *testing.T) { blockStart := time.Now().Truncate(ns.Options().RetentionOptions().BlockSize()) states := []fileOpState{ - {Status: fileOpNotStarted}, - {Status: fileOpSuccess}, + {WarmStatus: fileOpNotStarted}, + {WarmStatus: fileOpSuccess}, } for i, s := range states { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().ID().Return(testShardIDs[i].ID()) shard.EXPECT().FlushState(blockStart).Return(s) - if s.Status != fileOpSuccess { - shard.EXPECT().Flush(blockStart, gomock.Any(), gomock.Any()).Return(nil) + if s.WarmStatus != fileOpSuccess { + shard.EXPECT().WarmFlush(blockStart, gomock.Any(), gomock.Any()).Return(nil) } ns.shards[testShardIDs[i].ID()] = shard } @@ -458,7 +458,7 @@ func TestNamespaceFlushSkipFlushed(t *testing.T) { ShardBootstrapStates[testShardIDs[i].ID()] = Bootstrapped } - require.NoError(t, ns.Flush(blockStart, ShardBootstrapStates, nil)) + require.NoError(t, ns.WarmFlush(blockStart, ShardBootstrapStates, nil)) } func TestNamespaceFlushSkipShardNotBootstrappedBeforeTick(t *testing.T) { @@ -481,7 +481,7 @@ func TestNamespaceFlushSkipShardNotBootstrappedBeforeTick(t *testing.T) { shardBootstrapStates := ShardBootstrapStates{} shardBootstrapStates[testShardIDs[0].ID()] = Bootstrapping - require.NoError(t, ns.Flush(blockStart, shardBootstrapStates, nil)) + require.NoError(t, ns.WarmFlush(blockStart, shardBootstrapStates, nil)) } type snapshotTestCase struct { @@ -509,24 +509,6 @@ func TestNamespaceSnapshotNotBootstrapped(t *testing.T) { require.Equal(t, errNamespaceNotBootstrapped, ns.Snapshot(blockStart, blockStart, nil)) } -func TestNamespaceSnapshotShardIsSnapshotting(t *testing.T) { - shardMethodResults := []snapshotTestCase{ - snapshotTestCase{ - isSnapshotting: false, - expectSnapshot: true, - shardBootstrapStateBeforeTick: Bootstrapped, - shardSnapshotErr: nil, - }, - snapshotTestCase{ - isSnapshotting: true, - expectSnapshot: false, - shardBootstrapStateBeforeTick: Bootstrapped, - shardSnapshotErr: nil, - }, - } - require.NoError(t, testSnapshotWithShardSnapshotErrs(t, shardMethodResults)) -} - func TestNamespaceSnapshotAllShardsSuccess(t *testing.T) { shardMethodResults := []snapshotTestCase{ snapshotTestCase{ @@ -587,13 +569,6 @@ func testSnapshotWithShardSnapshotErrs(t *testing.T, shardMethodResults []snapsh for i, tc := range shardMethodResults { shard := NewMockdatabaseShard(ctrl) - var lastSnapshotTime time.Time - if tc.lastSnapshotTime == nil { - lastSnapshotTime = blockStart.Add(-blockSize) - } else { - lastSnapshotTime = tc.lastSnapshotTime(now, blockSize) - } - shard.EXPECT().SnapshotState().Return(tc.isSnapshotting, lastSnapshotTime) shardID := uint32(i) shard.EXPECT().ID().Return(uint32(i)).AnyTimes() if tc.expectSnapshot { @@ -795,11 +770,11 @@ func setShardExpects(ns *dbNamespace, ctrl *gomock.Controller, cases []needsFlus for t, needFlush := range cs.needsFlush { if needFlush { shard.EXPECT().FlushState(t.ToTime()).Return(fileOpState{ - Status: fileOpNotStarted, + WarmStatus: fileOpNotStarted, }).AnyTimes() } else { shard.EXPECT().FlushState(t.ToTime()).Return(fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, }).AnyTimes() } } @@ -935,7 +910,7 @@ func TestNamespaceNeedsFlushAllSuccess(t *testing.T) { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().ID().Return(s.ID()).AnyTimes() shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, }).AnyTimes() ns.shards[s.ID()] = shard } @@ -977,15 +952,15 @@ func TestNamespaceNeedsFlushAnyFailed(t *testing.T) { switch shard.ID() { case shards[0].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, }).AnyTimes() case shards[1].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, }).AnyTimes() case shards[2].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - Status: fileOpFailed, + WarmStatus: fileOpFailed, NumFailures: 999, }).AnyTimes() } @@ -1029,15 +1004,15 @@ func TestNamespaceNeedsFlushAnyNotStarted(t *testing.T) { switch shard.ID() { case shards[0].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, }).AnyTimes() case shards[1].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - Status: fileOpNotStarted, + WarmStatus: fileOpNotStarted, }).AnyTimes() case shards[2].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, }).AnyTimes() } ns.shards[s.ID()] = shard diff --git a/src/dbnode/storage/repair.go b/src/dbnode/storage/repair.go index 672356ea0a..844fe87aa8 100644 --- a/src/dbnode/storage/repair.go +++ b/src/dbnode/storage/repair.go @@ -331,7 +331,7 @@ func (r *dbRepairer) namespaceRepairTimeRanges(ns databaseNamespace) xtime.Range rtopts = ns.Options().RetentionOptions() blockSize = rtopts.BlockSize() start = now.Add(-rtopts.RetentionPeriod()).Truncate(blockSize) - end = now.Add(rtopts.FutureRetentionPeriod()).Add(-rtopts.BufferPast()).Truncate(blockSize) + end = now.Add(-rtopts.BufferPast()).Truncate(blockSize) ) targetRanges := xtime.NewRanges(xtime.Range{Start: start, End: end}) diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 1489eebdab..4e4a8df2a6 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -22,6 +22,7 @@ package series import ( "errors" + "fmt" "sort" "sync/atomic" "time" @@ -56,11 +57,17 @@ var ( const ( bucketsCacheSize = 2 - // TODO(juchan): This represents the maximum number of blocks we think we'll - // be flushing at one time. This should be revisited to after ColdWrites are - // enabled to see if this is a sane number. - evictedTimesArraySize = 8 - writableBucketVer = 0 + // optimizedTimesArraySize is the size of the internal array for the + // optimizedTimes struct. Since the size of this array determines the + // effectiveness of minimizing heap allocations, usage of this struct and/or + // changing this const should only be done after considering its current + // use cases: + // 1) The number of buckets that will be removed within a tick due to that + // block being recently flushed + // 2) The number of buckets that contain ColdWrites within a cold flush + // cycle + optimizedTimesArraySize = 8 + writableBucketVer = 0 ) type databaseBuffer interface { @@ -82,13 +89,12 @@ type databaseBuffer interface { nsCtx namespace.Context, ) error - Flush( + WarmFlush( ctx context.Context, blockStart time.Time, id ident.ID, tags ident.Tags, persistFn persist.DataFn, - version int, nsCtx namespace.Context, ) (FlushOutcome, error) @@ -98,6 +104,13 @@ type databaseBuffer interface { nsCtx namespace.Context, ) ([][]xio.BlockReader, error) + FetchBlocksForColdFlush( + ctx context.Context, + start time.Time, + version int, + nsCtx namespace.Context, + ) ([]xio.BlockReader, error) + FetchBlocks( ctx context.Context, starts []time.Time, @@ -112,6 +125,8 @@ type databaseBuffer interface { IsEmpty() bool + NeedsColdFlushBlockStarts() OptimizedTimes + Stats() bufferStats Tick(versions map[xtime.UnixNano]BlockState, nsCtx namespace.Context) bufferTickResult @@ -127,25 +142,26 @@ type bufferStats struct { type bufferTickResult struct { mergedOutOfOrderBlocks int - evictedBucketTimes evictedTimes + evictedBucketTimes OptimizedTimes } -// evictedTimes is a struct that holds an unknown number of times. This is used -// to avoid heap allocations as much as possible by trying to not allocate a -// slice of times. To do this, `evictedTimesArraySize` needs to be strategically -// sized such that for the vast majority of the time, the internal array can -// hold all the times required so that `slice` is nil. +// OptimizedTimes is a struct that holds an unknown number of times. This is +// used to avoid heap allocations as much as possible by trying to not allocate +// a slice of times. To do this, `optimizedTimesArraySize` needs to be +// strategically sized such that for the vast majority of the time, the internal +// array can hold all the times required so that `slice` is nil. // -// evictedTimes should only be interacted with via its helper functions - its +// OptimizedTimes should only be interacted with via its helper functions - its // fields should never be accessed or modified directly, which could cause an // invalid state. -type evictedTimes struct { +type OptimizedTimes struct { arrIdx int - arr [evictedTimesArraySize]xtime.UnixNano + arr [optimizedTimesArraySize]xtime.UnixNano slice []xtime.UnixNano } -func (t *evictedTimes) add(newTime xtime.UnixNano) { +// Add adds a time to this OptimizedTimes. +func (t *OptimizedTimes) Add(newTime xtime.UnixNano) { if t.arrIdx < cap(t.arr) { t.arr[t.arrIdx] = newTime t.arrIdx++ @@ -154,11 +170,13 @@ func (t *evictedTimes) add(newTime xtime.UnixNano) { } } -func (t *evictedTimes) len() int { +// Len returns the number of times in this OptimizedTimes. +func (t *OptimizedTimes) Len() int { return t.arrIdx + len(t.slice) } -func (t *evictedTimes) contains(target xtime.UnixNano) bool { +// Contains returns whether the target time is in this OptimizedTimes. +func (t *OptimizedTimes) Contains(target xtime.UnixNano) bool { for i := 0; i < t.arrIdx; i++ { if t.arr[i].Equal(target) { return true @@ -172,6 +190,16 @@ func (t *evictedTimes) contains(target xtime.UnixNano) bool { return false } +// ForEach runs the given function for each time in this OptimizedTimes. +func (t *OptimizedTimes) ForEach(fn func(t xtime.UnixNano)) { + for _, tNano := range t.arr { + fn(tNano) + } + for _, tNano := range t.slice { + fn(tNano) + } +} + type dbBuffer struct { opts Options nowFn clock.NowFn @@ -281,6 +309,21 @@ func (b *dbBuffer) IsEmpty() bool { return len(b.bucketsMap) == 0 } +func (b *dbBuffer) NeedsColdFlushBlockStarts() OptimizedTimes { + var times OptimizedTimes + + for t, bucketVersions := range b.bucketsMap { + for _, bucket := range bucketVersions.buckets { + if bucket.writeType == ColdWrite { + times.Add(t) + break + } + } + } + + return times +} + func (b *dbBuffer) Stats() bufferStats { return bufferStats{ wiredBlocks: len(b.bucketsMap), @@ -289,7 +332,7 @@ func (b *dbBuffer) Stats() bufferStats { func (b *dbBuffer) Tick(blockStates map[xtime.UnixNano]BlockState, nsCtx namespace.Context) bufferTickResult { mergedOutOfOrder := 0 - var evictedBucketTimes evictedTimes + var evictedBucketTimes OptimizedTimes for tNano, buckets := range b.bucketsMap { // The blockStates map is never written to after creation, so this // read access is safe. Since this version map is a snapshot of the @@ -300,8 +343,15 @@ func (b *dbBuffer) Tick(blockStates map[xtime.UnixNano]BlockState, nsCtx namespa // Retrievable and higher versioned buckets will be left to be // collected in the next tick. blockState := blockStates[tNano] - if blockState.Retrievable { - buckets.removeBucketsUpToVersion(blockState.Version) + if coldVersion := blockState.ColdVersion; blockState.WarmRetrievable || coldVersion > 0 { + if blockState.WarmRetrievable { + // Buckets for WarmWrites that are retrievable will only be version 1, since + // they only get successfully persisted once. + buckets.removeBucketsUpToVersion(WarmWrite, 1) + } + if coldVersion > 0 { + buckets.removeBucketsUpToVersion(ColdWrite, coldVersion) + } if buckets.streamsLen() == 0 { t := tNano.ToTime() @@ -319,7 +369,7 @@ func (b *dbBuffer) Tick(blockStates map[xtime.UnixNano]BlockState, nsCtx namespa // It's unclear whether recently flushed data would frequently be // read soon afterward, so we're choosing (1) here, since it has a // simpler implementation (just removing from a map). - evictedBucketTimes.add(tNano) + evictedBucketTimes.Add(tNano) continue } } @@ -424,13 +474,12 @@ func (b *dbBuffer) Snapshot( return persistFn(id, tags, segment, checksum) } -func (b *dbBuffer) Flush( +func (b *dbBuffer) WarmFlush( ctx context.Context, blockStart time.Time, id ident.ID, tags ident.Tags, persistFn persist.DataFn, - version int, nsCtx namespace.Context, ) (FlushOutcome, error) { buckets, exists := b.bucketVersionsAt(blockStart) @@ -489,7 +538,9 @@ func (b *dbBuffer) Flush( } if bucket, exists := buckets.writableBucket(WarmWrite); exists { - bucket.version = version + // WarmFlushes only happen once per block, so it makes sense to always + // set this to 1. + bucket.version = 1 } return FlushOutcomeFlushedToDisk, nil @@ -537,7 +588,46 @@ func (b *dbBuffer) ReadEncoded( return res, nil } +func (b *dbBuffer) FetchBlocksForColdFlush( + ctx context.Context, + start time.Time, + version int, + nsCtx namespace.Context, +) ([]xio.BlockReader, error) { + res := b.fetchBlocks(ctx, []time.Time{start}, + streamsOptions{filterWriteType: true, writeType: ColdWrite, nsCtx: nsCtx}) + if len(res) == 0 { + return nil, nil + } + // We should have at most one BlockReader slice since we are only looking + // at one block. + if len(res) > 1 { + return nil, errors.New("fetchBlocks returned more than one result for one block") + } + blocks := res[0].Blocks + + buckets, exists := b.bucketVersionsAt(start) + if !exists { + return nil, fmt.Errorf("buckets do not exist with block start %s", start) + } + if bucket, exists := buckets.writableBucket(ColdWrite); exists { + bucket.version = version + } else { + return nil, fmt.Errorf("writable bucket does not exist with block start %s", start) + } + + return blocks, nil +} + func (b *dbBuffer) FetchBlocks(ctx context.Context, starts []time.Time, nsCtx namespace.Context) []block.FetchBlockResult { + return b.fetchBlocks(ctx, starts, streamsOptions{filterWriteType: false, nsCtx: nsCtx}) +} + +func (b *dbBuffer) fetchBlocks( + ctx context.Context, + starts []time.Time, + sOpts streamsOptions, +) []block.FetchBlockResult { var res []block.FetchBlockResult for _, start := range starts { @@ -546,7 +636,7 @@ func (b *dbBuffer) FetchBlocks(ctx context.Context, starts []time.Time, nsCtx na continue } - if streams := buckets.streams(ctx, streamsOptions{filterWriteType: false, nsCtx: nsCtx}); len(streams) > 0 { + if streams := buckets.streams(ctx, sOpts); len(streams) > 0 { res = append(res, block.NewFetchBlockResult(start, streams, nil)) } } @@ -796,20 +886,16 @@ func (b *BufferBucketVersions) merge(writeType WriteType, nsCtx namespace.Contex return res, nil } -func (b *BufferBucketVersions) removeBucketsUpToVersion(version int) { - // TODO(juchan): in order to support ColdWrites, we need to keep track of - // separate bucket versions for ColdWrites and WarmWrites, since they have - // different persist cycles. This will involve storing that state in the - // shard flush state management. That state will need to be passed down - // here so that this function will know which WriteType/version is safe to - // remove. - +func (b *BufferBucketVersions) removeBucketsUpToVersion( + writeType WriteType, + version int, +) { // Avoid allocating a new backing array. nonEvictedBuckets := b.buckets[:0] for _, bucket := range b.buckets { bVersion := bucket.version - if bucket.writeType == WarmWrite && bVersion != writableBucketVer && + if bucket.writeType == writeType && bVersion != writableBucketVer && bVersion <= version { // We no longer need to keep any version which is equal to // or less than the retrievable version, since that means diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index ab0c1f1d2e..c3ed8cab17 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -37,9 +37,9 @@ import ( xtime "github.com/m3db/m3/src/x/time" "github.com/golang/mock/gomock" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/m3db/m3/src/dbnode/namespace" ) func newBufferTestOptions() Options { @@ -65,12 +65,12 @@ func newBufferTestOptions() Options { SetBufferBucketVersionsPool(bufferBucketVersionsPool) opts = opts. SetRetentionOptions(opts.RetentionOptions(). - SetBlockSize(2 * time.Minute). - SetBufferFuture(10 * time.Second). - SetBufferPast(10 * time.Second)). + SetBlockSize(2 * time.Minute). + SetBufferFuture(10 * time.Second). + SetBufferPast(10 * time.Second)). SetDatabaseBlockOptions(opts.DatabaseBlockOptions(). - SetContextPool(opts.ContextPool()). - SetEncoderPool(opts.EncoderPool())) + SetContextPool(opts.ContextPool()). + SetEncoderPool(opts.EncoderPool())) return opts } @@ -769,8 +769,8 @@ func TestBufferTickReordersOutOfOrderBuffers(t *testing.T) { blockStates := make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(start)] = BlockState{ - Retrievable: true, - Version: 1, + WarmRetrievable: true, + ColdVersion: 1, } // Perform a tick and ensure merged out of order blocks. r := buffer.Tick(blockStates, namespace.Context{}) @@ -845,8 +845,8 @@ func TestBufferRemoveBucket(t *testing.T) { // get removed from the bucket. blockStates := make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(start)] = BlockState{ - Retrievable: true, - Version: 1, + WarmRetrievable: true, + ColdVersion: 1, } bucket.version = 1 @@ -951,8 +951,8 @@ func testBufferWithEmptyEncoder(t *testing.T, testSnapshot bool) { } else { ctx = context.NewContext() defer ctx.Close() - _, err = buffer.Flush( - ctx, start, ident.StringID("some-id"), ident.Tags{}, assertPersistDataFn, 1, namespace.Context{}) + _, err = buffer.WarmFlush( + ctx, start, ident.StringID("some-id"), ident.Tags{}, assertPersistDataFn, namespace.Context{}) require.NoError(t, err) } } @@ -1110,30 +1110,45 @@ func assertTimeSlicesEqual(t *testing.T, t1, t2 []time.Time) { } } -func TestEvictedTimes(t *testing.T) { - var times evictedTimes +func TestOptimizedTimes(t *testing.T) { + var times OptimizedTimes assert.Equal(t, 0, cap(times.slice)) - assert.Equal(t, 0, times.len()) - assert.False(t, times.contains(xtime.UnixNano(0))) + assert.Equal(t, 0, times.Len()) + assert.False(t, times.Contains(xtime.UnixNano(0))) + + var expectedTimes []xtime.UnixNano // These adds should only go in the array. - for i := 0; i < evictedTimesArraySize; i++ { + for i := 0; i < optimizedTimesArraySize; i++ { tNano := xtime.UnixNano(i) - times.add(tNano) + times.Add(tNano) + expectedTimes = append(expectedTimes, tNano) assert.Equal(t, 0, cap(times.slice)) assert.Equal(t, i+1, times.arrIdx) - assert.Equal(t, i+1, times.len()) - assert.True(t, times.contains(tNano)) + assert.Equal(t, i+1, times.Len()) + assert.True(t, times.Contains(tNano)) } + numExtra := 5 // These adds don't fit in the array any more, will go to the slice. - for i := evictedTimesArraySize; i < evictedTimesArraySize+5; i++ { + for i := optimizedTimesArraySize; i < optimizedTimesArraySize+numExtra; i++ { tNano := xtime.UnixNano(i) - times.add(tNano) + times.Add(tNano) + expectedTimes = append(expectedTimes, tNano) + + assert.Equal(t, optimizedTimesArraySize, times.arrIdx) + assert.Equal(t, i+1, times.Len()) + assert.True(t, times.Contains(tNano)) + } + + var forEachTimes []xtime.UnixNano + times.ForEach(func(tNano xtime.UnixNano) { + forEachTimes = append(forEachTimes, tNano) + }) - assert.Equal(t, evictedTimesArraySize, times.arrIdx) - assert.Equal(t, i+1, times.len()) - assert.True(t, times.contains(tNano)) + require.Equal(t, len(expectedTimes), len(forEachTimes)) + for i := range expectedTimes { + assert.Equal(t, expectedTimes[i], forEachTimes[i]) } } diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 2a78f957d3..9092b15412 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -34,8 +34,8 @@ import ( "github.com/m3db/m3/src/x/instrument" xtime "github.com/m3db/m3/src/x/time" - "go.uber.org/zap" "github.com/m3db/m3/src/dbnode/namespace" + "go.uber.org/zap" ) type bootstrapState int @@ -124,7 +124,7 @@ func (s *dbSeries) Tick(blockStates map[xtime.UnixNano]BlockState, nsCtx namespa bufferResult := s.buffer.Tick(blockStates, nsCtx) r.MergedOutOfOrderBlocks = bufferResult.mergedOutOfOrderBlocks - r.EvictedBuckets = bufferResult.evictedBucketTimes.len() + r.EvictedBuckets = bufferResult.evictedBucketTimes.Len() update, err := s.updateBlocksWithLock(blockStates, bufferResult.evictedBucketTimes) if err != nil { s.Unlock() @@ -150,7 +150,7 @@ type updateBlocksResult struct { func (s *dbSeries) updateBlocksWithLock( blockStates map[xtime.UnixNano]BlockState, - evictedBucketTimes evictedTimes, + evictedBucketTimes OptimizedTimes, ) (updateBlocksResult, error) { var ( result updateBlocksResult @@ -162,7 +162,7 @@ func (s *dbSeries) updateBlocksWithLock( ) for startNano, currBlock := range s.cachedBlocks.AllBlocks() { start := startNano.ToTime() - if start.Before(expireCutoff) || evictedBucketTimes.contains(xtime.ToUnixNano(start)) { + if start.Before(expireCutoff) || evictedBucketTimes.Contains(xtime.ToUnixNano(start)) { s.cachedBlocks.RemoveBlockAt(start) // If we're using the LRU policy and the block was retrieved from disk, // then don't close the block because that is the WiredList's @@ -207,7 +207,7 @@ func (s *dbSeries) updateBlocksWithLock( // Makes sure that the block has been flushed, which // prevents us from unwiring blocks that haven't been flushed yet which // would cause data loss. - if blockState := blockStates[startNano]; blockState.Retrievable { + if blockState := blockStates[startNano]; blockState.WarmRetrievable { switch cachePolicy { case CacheNone: shouldUnwire = true @@ -301,6 +301,18 @@ func (s *dbSeries) ReadEncoded( return r, err } +func (s *dbSeries) FetchBlocksForColdFlush( + ctx context.Context, + start time.Time, + version int, + nsCtx namespace.Context, +) ([]xio.BlockReader, error) { + s.Lock() + defer s.Unlock() + + return s.buffer.FetchBlocksForColdFlush(ctx, start, version, nsCtx) +} + func (s *dbSeries) FetchBlocks( ctx context.Context, starts []time.Time, @@ -517,11 +529,10 @@ func (s *dbSeries) OnEvictedFromWiredList(id ident.ID, blockStart time.Time) { } } -func (s *dbSeries) Flush( +func (s *dbSeries) WarmFlush( ctx context.Context, blockStart time.Time, persistFn persist.DataFn, - version int, nsCtx namespace.Context, ) (FlushOutcome, error) { s.Lock() @@ -531,7 +542,7 @@ func (s *dbSeries) Flush( return FlushOutcomeErr, errSeriesNotBootstrapped } - return s.buffer.Flush(ctx, blockStart, s.id, s.tags, persistFn, version, nsCtx) + return s.buffer.WarmFlush(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) } func (s *dbSeries) Snapshot( @@ -552,6 +563,13 @@ func (s *dbSeries) Snapshot( return s.buffer.Snapshot(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) } +func (s *dbSeries) NeedsColdFlushBlockStarts() OptimizedTimes { + s.RLock() + defer s.RUnlock() + + return s.buffer.NeedsColdFlushBlockStarts() +} + func (s *dbSeries) Close() { s.Lock() defer s.Unlock() diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 5a32cf979d..1b30071240 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -1,26 +1,6 @@ // Code generated by MockGen. DO NOT EDIT. // Source: github.com/m3db/m3/src/dbnode/storage/series (interfaces: DatabaseSeries,QueryableBlockRetriever) -// 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 series is a generated GoMock package. package series @@ -105,34 +85,34 @@ func (mr *MockDatabaseSeriesMockRecorder) FetchBlocks(arg0, arg1, arg2 interface return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBlocks", reflect.TypeOf((*MockDatabaseSeries)(nil).FetchBlocks), arg0, arg1, arg2) } -// FetchBlocksMetadata mocks base method -func (m *MockDatabaseSeries) FetchBlocksMetadata(arg0 context.Context, arg1, arg2 time.Time, arg3 FetchBlocksMetadataOptions) (block.FetchBlocksMetadataResult, error) { +// FetchBlocksForColdFlush mocks base method +func (m *MockDatabaseSeries) FetchBlocksForColdFlush(arg0 context.Context, arg1 time.Time, arg2 int, arg3 namespace.Context) ([]xio.BlockReader, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchBlocksMetadata", arg0, arg1, arg2, arg3) - ret0, _ := ret[0].(block.FetchBlocksMetadataResult) + ret := m.ctrl.Call(m, "FetchBlocksForColdFlush", arg0, arg1, arg2, arg3) + ret0, _ := ret[0].([]xio.BlockReader) ret1, _ := ret[1].(error) return ret0, ret1 } -// FetchBlocksMetadata indicates an expected call of FetchBlocksMetadata -func (mr *MockDatabaseSeriesMockRecorder) FetchBlocksMetadata(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { +// FetchBlocksForColdFlush indicates an expected call of FetchBlocksForColdFlush +func (mr *MockDatabaseSeriesMockRecorder) FetchBlocksForColdFlush(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBlocksMetadata", reflect.TypeOf((*MockDatabaseSeries)(nil).FetchBlocksMetadata), arg0, arg1, arg2, arg3) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBlocksForColdFlush", reflect.TypeOf((*MockDatabaseSeries)(nil).FetchBlocksForColdFlush), arg0, arg1, arg2, arg3) } -// Flush mocks base method -func (m *MockDatabaseSeries) Flush(arg0 context.Context, arg1 time.Time, arg2 persist.DataFn, arg3 int, arg4 namespace.Context) (FlushOutcome, error) { +// FetchBlocksMetadata mocks base method +func (m *MockDatabaseSeries) FetchBlocksMetadata(arg0 context.Context, arg1, arg2 time.Time, arg3 FetchBlocksMetadataOptions) (block.FetchBlocksMetadataResult, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Flush", arg0, arg1, arg2, arg3, arg4) - ret0, _ := ret[0].(FlushOutcome) + ret := m.ctrl.Call(m, "FetchBlocksMetadata", arg0, arg1, arg2, arg3) + ret0, _ := ret[0].(block.FetchBlocksMetadataResult) ret1, _ := ret[1].(error) return ret0, ret1 } -// Flush indicates an expected call of Flush -func (mr *MockDatabaseSeriesMockRecorder) Flush(arg0, arg1, arg2, arg3, arg4 interface{}) *gomock.Call { +// FetchBlocksMetadata indicates an expected call of FetchBlocksMetadata +func (mr *MockDatabaseSeriesMockRecorder) FetchBlocksMetadata(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Flush", reflect.TypeOf((*MockDatabaseSeries)(nil).Flush), arg0, arg1, arg2, arg3, arg4) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBlocksMetadata", reflect.TypeOf((*MockDatabaseSeries)(nil).FetchBlocksMetadata), arg0, arg1, arg2, arg3) } // ID mocks base method @@ -177,6 +157,20 @@ func (mr *MockDatabaseSeriesMockRecorder) IsEmpty() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsEmpty", reflect.TypeOf((*MockDatabaseSeries)(nil).IsEmpty)) } +// NeedsColdFlushBlockStarts mocks base method +func (m *MockDatabaseSeries) NeedsColdFlushBlockStarts() OptimizedTimes { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "NeedsColdFlushBlockStarts") + ret0, _ := ret[0].(OptimizedTimes) + return ret0 +} + +// NeedsColdFlushBlockStarts indicates an expected call of NeedsColdFlushBlockStarts +func (mr *MockDatabaseSeriesMockRecorder) NeedsColdFlushBlockStarts() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NeedsColdFlushBlockStarts", reflect.TypeOf((*MockDatabaseSeries)(nil).NeedsColdFlushBlockStarts)) +} + // NumActiveBlocks mocks base method func (m *MockDatabaseSeries) NumActiveBlocks() int { m.ctrl.T.Helper() @@ -285,6 +279,21 @@ func (mr *MockDatabaseSeriesMockRecorder) Tick(arg0, arg1 interface{}) *gomock.C return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Tick", reflect.TypeOf((*MockDatabaseSeries)(nil).Tick), arg0, arg1) } +// WarmFlush mocks base method +func (m *MockDatabaseSeries) WarmFlush(arg0 context.Context, arg1 time.Time, arg2 persist.DataFn, arg3 namespace.Context) (FlushOutcome, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "WarmFlush", arg0, arg1, arg2, arg3) + ret0, _ := ret[0].(FlushOutcome) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// WarmFlush indicates an expected call of WarmFlush +func (mr *MockDatabaseSeriesMockRecorder) WarmFlush(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WarmFlush", reflect.TypeOf((*MockDatabaseSeries)(nil).WarmFlush), arg0, arg1, arg2, arg3) +} + // Write mocks base method func (m *MockDatabaseSeries) Write(arg0 context.Context, arg1 time.Time, arg2 float64, arg3 time0.Unit, arg4 []byte, arg5 WriteOptions) (bool, error) { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/series/series_test.go b/src/dbnode/storage/series/series_test.go index e5f98151ca..0a64e97594 100644 --- a/src/dbnode/storage/series/series_test.go +++ b/src/dbnode/storage/series/series_test.go @@ -41,9 +41,9 @@ import ( xtime "github.com/m3db/m3/src/x/time" "github.com/golang/mock/gomock" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/m3db/m3/src/dbnode/namespace" ) func newSeriesTestOptions() Options { @@ -239,7 +239,7 @@ func TestSeriesFlushNoBlock(t *testing.T) { _, err := series.Bootstrap(nil) assert.NoError(t, err) flushTime := time.Unix(7200, 0) - outcome, err := series.Flush(nil, flushTime, nil, 1, namespace.Context{}) + outcome, err := series.WarmFlush(nil, flushTime, nil, namespace.Context{}) require.Nil(t, err) require.Equal(t, FlushOutcomeBlockDoesNotExist, outcome) } @@ -268,7 +268,7 @@ func TestSeriesFlush(t *testing.T) { return input } ctx := context.NewContext() - outcome, err := series.Flush(ctx, curr, persistFn, 1, namespace.Context{}) + outcome, err := series.WarmFlush(ctx, curr, persistFn, namespace.Context{}) ctx.BlockingClose() require.Equal(t, input, err) if input == nil { @@ -337,12 +337,12 @@ func TestSeriesTickNeedsBlockExpiry(t *testing.T) { buffer.EXPECT().Stats().Return(bufferStats{wiredBlocks: 1}) blockStates := make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(blockStart)] = BlockState{ - Retrievable: false, - Version: 0, + WarmRetrievable: false, + ColdVersion: 0, } blockStates[xtime.ToUnixNano(curr)] = BlockState{ - Retrievable: false, - Version: 0, + WarmRetrievable: false, + ColdVersion: 0, } r, err := series.Tick(blockStates, namespace.Context{}) require.NoError(t, err) @@ -384,8 +384,8 @@ func TestSeriesTickRecentlyRead(t *testing.T) { blockStates := make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(curr)] = BlockState{ - Retrievable: true, - Version: 1, + WarmRetrievable: true, + ColdVersion: 1, } tickResult, err := series.Tick(blockStates, namespace.Context{}) require.NoError(t, err) @@ -413,8 +413,8 @@ func TestSeriesTickRecentlyRead(t *testing.T) { blockStates = make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(curr)] = BlockState{ - Retrievable: false, - Version: 0, + WarmRetrievable: false, + ColdVersion: 0, } tickResult, err = series.Tick(blockStates, namespace.Context{}) require.NoError(t, err) @@ -451,8 +451,8 @@ func TestSeriesTickCacheLRU(t *testing.T) { blockStates := make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(curr)] = BlockState{ - Retrievable: true, - Version: 1, + WarmRetrievable: true, + ColdVersion: 1, } tickResult, err := series.Tick(blockStates, namespace.Context{}) require.NoError(t, err) @@ -487,8 +487,8 @@ func TestSeriesTickCacheLRU(t *testing.T) { blockStates = make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(curr)] = BlockState{ - Retrievable: false, - Version: 0, + WarmRetrievable: false, + ColdVersion: 0, } tickResult, err = series.Tick(blockStates, namespace.Context{}) require.NoError(t, err) @@ -525,8 +525,8 @@ func TestSeriesTickCacheNone(t *testing.T) { blockStates := make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(curr)] = BlockState{ - Retrievable: true, - Version: 1, + WarmRetrievable: true, + ColdVersion: 1, } tickResult, err := series.Tick(blockStates, namespace.Context{}) require.NoError(t, err) @@ -541,8 +541,8 @@ func TestSeriesTickCacheNone(t *testing.T) { blockStates = make(map[xtime.UnixNano]BlockState) blockStates[xtime.ToUnixNano(curr)] = BlockState{ - Retrievable: false, - Version: 0, + WarmRetrievable: false, + ColdVersion: 0, } tickResult, err = series.Tick(blockStates, namespace.Context{}) require.NoError(t, err) @@ -587,9 +587,9 @@ func TestSeriesTickCachedBlockRemove(t *testing.T) { Return(bufferTickResult{ // This means that (curr - 1 block) and (curr - 2 blocks) should // be removed after the tick. - evictedBucketTimes: evictedTimes{ + evictedBucketTimes: OptimizedTimes{ arrIdx: 2, - arr: [evictedTimesArraySize]xtime.UnixNano{ + arr: [optimizedTimesArraySize]xtime.UnixNano{ xtime.ToUnixNano(curr.Add(-ropts.BlockSize())), xtime.ToUnixNano(curr.Add(-2 * ropts.BlockSize())), }, diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index 25309728a5..efbbbd5a84 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -25,10 +25,10 @@ import ( "github.com/m3db/m3/src/dbnode/clock" "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/storage/block" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" @@ -76,6 +76,16 @@ type DatabaseSeries interface { nsCtx namespace.Context, ) ([]block.FetchBlockResult, error) + // FetchBlocksForColdFlush fetches blocks for a cold flush. This function + // informs the series and the buffer that a cold flush for the specified + // block start is occurring so that it knows to update bucket versions. + FetchBlocksForColdFlush( + ctx context.Context, + start time.Time, + version int, + nsCtx namespace.Context, + ) ([]xio.BlockReader, error) + // FetchBlocksMetadata returns the blocks metadata. FetchBlocksMetadata( ctx context.Context, @@ -95,17 +105,16 @@ type DatabaseSeries interface { // Bootstrap merges the raw series bootstrapped along with any buffered data. Bootstrap(blocks block.DatabaseSeriesBlocks) (BootstrapResult, error) - // Flush flushes the data blocks of this series for a given start time - Flush( + // WarmFlush flushes the WarmWrites of this series for a given start time. + WarmFlush( ctx context.Context, blockStart time.Time, persistFn persist.DataFn, - version int, nsCtx namespace.Context, ) (FlushOutcome, error) // Snapshot snapshots the buffer buckets of this series for any data that has - // not been rotated into a block yet + // not been rotated into a block yet. Snapshot( ctx context.Context, blockStart time.Time, @@ -113,6 +122,9 @@ type DatabaseSeries interface { nsCtx namespace.Context, ) error + // NeedsColdFlushBlockStarts returns the block starts that need cold flushes. + NeedsColdFlushBlockStarts() OptimizedTimes + // Close will close the series and if pooled returned to the pool. Close() @@ -159,10 +171,10 @@ type QueryableBlockRetriever interface { BlockStatesSnapshot() map[xtime.UnixNano]BlockState } -// BlockState contains the state of a block +// BlockState contains the state of a block. type BlockState struct { - Retrievable bool - Version int + WarmRetrievable bool + ColdVersion int } // TickStatus is the status of a series for a given tick. @@ -203,7 +215,7 @@ type DatabaseSeriesPool interface { } // FlushOutcome is an enum that provides more context about the outcome -// of series.Flush() to the caller. +// of series.WarmFlush() to the caller. type FlushOutcome int const ( diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 48c3635ee7..d1f8214ce6 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -30,6 +30,8 @@ import ( "time" "github.com/m3db/m3/src/dbnode/clock" + "github.com/m3db/m3/src/dbnode/digest" + "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/generated/proto/pagetoken" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" @@ -155,7 +157,6 @@ type dbShard struct { identifierPool ident.Pool contextPool context.Pool flushState shardFlushState - snapshotState shardSnapshotState tickWg *sync.WaitGroup runtimeOptsListenClosers []xclose.SimpleCloser currRuntimeOptions dbShardRuntimeOptions @@ -231,12 +232,6 @@ func newShardFlushState() shardFlushState { } } -type shardSnapshotState struct { - sync.RWMutex - isSnapshotting bool - lastSuccessfulSnapshot time.Time -} - func newDatabaseShard( namespaceMetadata namespace.Metadata, shard uint32, @@ -347,7 +342,7 @@ func (s *dbShard) Stream( // IsBlockRetrievable implements series.QueryableBlockRetriever func (s *dbShard) IsBlockRetrievable(blockStart time.Time) bool { flushState := s.FlushState(blockStart) - return statusIsRetrievable(flushState.Status) + return statusIsRetrievable(flushState.WarmStatus) || flushState.ColdVersion > 0 } func statusIsRetrievable(status fileOpStatus) bool { @@ -364,7 +359,7 @@ func statusIsRetrievable(status fileOpStatus) bool { // RetrievableBlockVersion implements series.QueryableBlockRetriever func (s *dbShard) RetrievableBlockVersion(blockStart time.Time) int { flushState := s.FlushState(blockStart) - return flushState.Version + return flushState.ColdVersion } // BlockStatesSnapshot implements series.QueryableBlockRetriever @@ -376,8 +371,8 @@ func (s *dbShard) BlockStatesSnapshot() map[xtime.UnixNano]series.BlockState { snapshot := make(map[xtime.UnixNano]series.BlockState, len(states)) for time, state := range states { snapshot[time] = series.BlockState{ - Retrievable: statusIsRetrievable(state.Status), - Version: state.Version, + WarmRetrievable: statusIsRetrievable(state.WarmStatus), + ColdVersion: state.ColdVersion, } } @@ -1838,11 +1833,11 @@ func (s *dbShard) Bootstrap( info := result.Info at := xtime.FromNanoseconds(info.BlockStart) fs := s.FlushState(at) - if fs.Status != fileOpNotStarted { + if fs.WarmStatus != fileOpNotStarted { continue // Already recorded progress } - s.markFlushStateSuccess(at, 0) + s.markWarmFlushStateSuccess(at) } s.Lock() @@ -1852,7 +1847,7 @@ func (s *dbShard) Bootstrap( return multiErr.FinalError() } -func (s *dbShard) Flush( +func (s *dbShard) WarmFlush( blockStart time.Time, flushPreparer persist.FlushPreparer, nsCtx namespace.Context, @@ -1877,20 +1872,19 @@ func (s *dbShard) Flush( } prepared, err := flushPreparer.PrepareData(prepareOpts) if err != nil { - return s.markFlushStateSuccessOrError(blockStart, 0, err) + return s.markWarmFlushStateSuccessOrError(blockStart, err) } var multiErr xerrors.MultiError tmpCtx := context.NewContext() flushResult := dbShardFlushResult{} - version := s.RetrievableBlockVersion(blockStart) + 1 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() - flushOutcome, err := curr.Flush(tmpCtx, blockStart, prepared.Persist, version, nsCtx) + flushOutcome, err := curr.WarmFlush(tmpCtx, blockStart, prepared.Persist, nsCtx) tmpCtx.BlockingClose() if err != nil { @@ -1911,7 +1905,258 @@ func (s *dbShard) Flush( multiErr = multiErr.Add(err) } - return s.markFlushStateSuccessOrError(blockStart, version, multiErr.FinalError()) + return s.markWarmFlushStateSuccessOrError(blockStart, multiErr.FinalError()) +} + +func (s *dbShard) ColdFlush( + flushPreparer persist.FlushPreparer, + resources coldFlushReuseableResources, + nsCtx namespace.Context, +) error { + // We don't flush data when the shard is still bootstrapping. + s.RLock() + if s.bootstrapState != Bootstrapped { + s.RUnlock() + return errShardNotBootstrappedToFlush + } + s.RUnlock() + + dirtySeriesToWrite := resources.dirtySeriesToWrite + dirtySeries := resources.dirtySeries + idElementPool := resources.idElementPool + fsReader := resources.fsReader + // Reset resources. + for _, series := range dirtySeriesToWrite { + if series != nil { + series.Reset() + } + // Don't delete the empty list from the map so that other shards don't + // need to reinitialize the list for these blocks. + } + dirtySeries.Reset() + if fsReader.Status().Open { + if err := fsReader.Close(); err != nil { + return err + } + } + + var multiErr xerrors.MultiError + // First, loop through all series to capture data on which blocks have dirty + // series and add them to the resources for further processing. + s.forEachShardEntry(func(entry *lookup.Entry) bool { + curr := entry.Series + seriesID := curr.ID() + blockStarts := curr.NeedsColdFlushBlockStarts() + blockStarts.ForEach(func(t xtime.UnixNano) { + seriesList := dirtySeriesToWrite[t] + if seriesList == nil { + seriesList = newIDList(idElementPool) + dirtySeriesToWrite[t] = seriesList + } + element := seriesList.PushBack(seriesID) + + dirtySeries.Set(idAndBlockStart{blockStart: t, id: seriesID}, element) + }) + + return true + }) + + blockSize := s.namespace.Options().RetentionOptions().BlockSize() + srPool := s.opts.SegmentReaderPool() + multiIterPool := s.opts.MultiReaderIteratorPool() + encoderPool := s.opts.EncoderPool() + tmpCtx := context.NewContext() + + // Loop through each block that we know has ColdWrites. Since each block + // has its own fileset, if we encounter an error while trying to persist + // a block, we continue to try persisting other blocks. +BlockLoop: + for blockStart, seriesList := range dirtySeriesToWrite { + startTime := blockStart.ToTime() + openOpts := fs.DataReaderOpenOptions{ + Identifier: fs.FileSetFileIdentifier{ + Namespace: s.namespace.ID(), + Shard: s.ID(), + BlockStart: startTime, + }, + } + if err := fsReader.Open(openOpts); err != nil { + multiErr = multiErr.Add(err) + continue BlockLoop + } + defer fsReader.Close() + + s.flushState.RLock() + nextVersion := s.flushState.statesByTime[blockStart].ColdVersion + 1 + s.flushState.RUnlock() + + prepareOpts := persist.DataPrepareOptions{ + NamespaceMetadata: s.namespace, + Shard: s.ID(), + BlockStart: startTime, + DeleteIfExists: false, + } + prepared, err := flushPreparer.PrepareData(prepareOpts) + if err != nil { + s.incrementFlushStateFailures(startTime) + multiErr = multiErr.Add(err) + continue BlockLoop + } + + // Persistence is done in two stages. The first stage is to loop through + // series on disk and merge it with what's in memory. The second stage + // is to persist the rest of the series in memory that was not + // persisted in the first stage. + + // First stage: loop through series on disk. + for id, tagsIter, data, _, err := fsReader.Read(); err != io.EOF; id, _, data, _, err = fsReader.Read() { + if err != nil { + s.incrementFlushStateFailures(startTime) + multiErr = multiErr.Add(err) + continue BlockLoop + } + + // There will be at most two BlockReader slices: one for disk data + // and one for in memory data. + brs := make([][]xio.BlockReader, 0, 2) + + // Create BlockReader slice out of disk data. + seg := ts.NewSegment(data, nil, ts.FinalizeHead) + sr := srPool.Get() + sr.Reset(seg) + br := xio.BlockReader{ + SegmentReader: sr, + Start: startTime, + BlockSize: blockSize, + } + brs = append(brs, []xio.BlockReader{br}) + + // Check if this series is in memory (and thus requires merging). + if element, ok := dirtySeries.Get(idAndBlockStart{blockStart: blockStart, id: id}); ok { + // Series is in memory, so it will get merged with disk and + // written in this loop. Therefore, we need to remove it from + // the "to write" list so that the later loop does not rewrite + // it. + dirtySeriesToWrite[blockStart].Remove(element) + + s.RLock() + entry, _, err := s.lookupEntryWithLock(element.Value) + s.RUnlock() + + tmpCtx.Reset() + encoded, err := entry.Series.FetchBlocksForColdFlush(tmpCtx, startTime, nextVersion, nsCtx) + tmpCtx.BlockingClose() + if err != nil { + s.incrementFlushStateFailures(startTime) + multiErr = multiErr.Add(err) + continue BlockLoop + } + + if len(encoded) > 0 { + brs = append(brs, encoded) + } + } + + mergedIter := multiIterPool.Get() + mergedIter.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(brs), nsCtx.Schema) + defer mergedIter.Close() + + tags, err := convert.TagsFromTagsIter(id, tagsIter, s.opts.IdentifierPool()) + if err != nil { + s.incrementFlushStateFailures(startTime) + multiErr = multiErr.Add(err) + continue BlockLoop + } + if err := persistIter(prepared.Persist, id, tags, mergedIter, encoderPool); err != nil { + s.incrementFlushStateFailures(startTime) + multiErr = multiErr.Add(err) + continue BlockLoop + } + } + + // Second stage: loop through rest of series in memory that was not on + // disk. + for seriesElement := seriesList.Front(); seriesElement != nil; seriesElement = seriesElement.Next() { + s.RLock() + entry, _, err := s.lookupEntryWithLock(seriesElement.Value) + s.RUnlock() + if err != nil { + s.incrementFlushStateFailures(startTime) + multiErr = multiErr.Add(err) + continue BlockLoop + } + series := entry.Series + tmpCtx.Reset() + encoded, err := series.FetchBlocksForColdFlush(tmpCtx, startTime, nextVersion, nsCtx) + tmpCtx.BlockingClose() + if err != nil { + s.incrementFlushStateFailures(startTime) + multiErr = multiErr.Add(err) + continue BlockLoop + } + + if len(encoded) > 0 { + iter := multiIterPool.Get() + iter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize, nsCtx.Schema) + defer iter.Close() + if err := persistIter(prepared.Persist, series.ID(), series.Tags(), iter, encoderPool); err != nil { + s.incrementFlushStateFailures(startTime) + multiErr = multiErr.Add(err) + continue BlockLoop + } + } + } + + // Close the flush preparer, which writes the rest of the filesets. + if err := prepared.Close(); err != nil { + multiErr = multiErr.Add(err) + } + + // After writing the full block successfully, update the cold version + // in the flush state. + s.setFlushStateColdVersion(startTime, nextVersion) + } + + return multiErr.FinalError() +} + +func persistIter( + persistFn persist.DataFn, + id ident.ID, + tags ident.Tags, + it encoding.Iterator, + encoderPool encoding.EncoderPool, +) error { + encoder := encoderPool.Get() + for it.Next() { + if err := encoder.Encode(it.Current()); err != nil { + encoder.Close() + return err + } + } + if err := it.Err(); err != nil { + return err + } + + stream, ok := encoder.Stream(encoding.StreamOptions{}) + encoder.Close() + if !ok { + return nil + } + + segment, err := stream.Segment() + if err != nil { + return err + } + + checksum := digest.SegmentChecksum(segment) + + err = persistFn(id, tags, segment, checksum) + if err != nil { + return err + } + + return nil } func (s *dbShard) Snapshot( @@ -1930,11 +2175,6 @@ func (s *dbShard) Snapshot( var multiErr xerrors.MultiError - s.markIsSnapshotting() - defer func() { - s.markDoneSnapshotting(multiErr.Empty(), snapshotTime) - }() - prepareOpts := persist.DataPrepareOptions{ NamespaceMetadata: s.namespace, Shard: s.ID(), @@ -1988,40 +2228,55 @@ func (s *dbShard) FlushState(blockStart time.Time) fileOpState { state, ok := s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] if !ok { - return fileOpState{Status: fileOpNotStarted} + return fileOpState{WarmStatus: fileOpNotStarted} } return state } -func (s *dbShard) markFlushStateSuccessOrError(blockStart time.Time, version int, err error) error { +func (s *dbShard) markWarmFlushStateSuccessOrError(blockStart time.Time, err error) error { // Track flush state for block state if err == nil { - s.markFlushStateSuccess(blockStart, version) + s.markWarmFlushStateSuccess(blockStart) } else { - s.markFlushStateFail(blockStart) + s.markWarmFlushStateFail(blockStart) } return err } -func (s *dbShard) markFlushStateSuccess(blockStart time.Time, version int) { +func (s *dbShard) markWarmFlushStateSuccess(blockStart time.Time) { s.flushState.Lock() s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ - Status: fileOpSuccess, - Version: version, + WarmStatus: fileOpSuccess, } s.flushState.Unlock() } -func (s *dbShard) markFlushStateFail(blockStart time.Time) { +func (s *dbShard) markWarmFlushStateFail(blockStart time.Time) { s.flushState.Lock() state := s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] - state.Status = fileOpFailed + state.WarmStatus = fileOpFailed state.NumFailures++ s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = state s.flushState.Unlock() } +func (s *dbShard) incrementFlushStateFailures(blockStart time.Time) { + s.flushState.Lock() + state := s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] + state.NumFailures++ + s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = state + s.flushState.Unlock() +} + +func (s *dbShard) setFlushStateColdVersion(blockStart time.Time, version int) { + s.flushState.Lock() + state := s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] + state.ColdVersion = version + s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = state + s.flushState.Unlock() +} + func (s *dbShard) removeAnyFlushStatesTooEarly(tickStart time.Time) { s.flushState.Lock() earliestFlush := retention.FlushTimeStart(s.namespace.Options().RetentionOptions(), tickStart) @@ -2033,27 +2288,6 @@ func (s *dbShard) removeAnyFlushStatesTooEarly(tickStart time.Time) { s.flushState.Unlock() } -func (s *dbShard) SnapshotState() (bool, time.Time) { - s.snapshotState.RLock() - defer s.snapshotState.RUnlock() - return s.snapshotState.isSnapshotting, s.snapshotState.lastSuccessfulSnapshot -} - -func (s *dbShard) markIsSnapshotting() { - s.snapshotState.Lock() - s.snapshotState.isSnapshotting = true - s.snapshotState.Unlock() -} - -func (s *dbShard) markDoneSnapshotting(success bool, completionTime time.Time) { - s.snapshotState.Lock() - s.snapshotState.isSnapshotting = false - if success { - s.snapshotState.lastSuccessfulSnapshot = completionTime - } - s.snapshotState.Unlock() -} - func (s *dbShard) CleanupExpiredFileSets(earliestToRetain time.Time) error { filePathPrefix := s.opts.CommitLogOptions().FilesystemOptions().FilePathPrefix() multiErr := xerrors.NewMultiError() diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index a9139072b2..02e97b2973 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -134,7 +134,7 @@ func TestShardFlushStateNotStarted(t *testing.T) { s := testDatabaseShard(t, opts) defer s.Close() - notStarted := fileOpState{Status: fileOpNotStarted} + notStarted := fileOpState{WarmStatus: fileOpNotStarted} for st := earliest; !st.After(latest); st = st.Add(ropts.BlockSize()) { assert.Equal(t, notStarted, s.FlushState(earliest)) } @@ -184,7 +184,7 @@ func TestShardFlushDuringBootstrap(t *testing.T) { s := testDatabaseShard(t, DefaultTestOptions()) defer s.Close() s.bootstrapState = Bootstrapping - err := s.Flush(time.Now(), nil, namespace.Context{}) + err := s.WarmFlush(time.Now(), nil, namespace.Context{}) require.Equal(t, err, errShardNotBootstrappedToFlush) } @@ -198,7 +198,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { defer s.Close() s.bootstrapState = Bootstrapped s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ - Status: fileOpFailed, + WarmStatus: fileOpFailed, NumFailures: 1, } @@ -226,15 +226,15 @@ func TestShardFlushSeriesFlushError(t *testing.T) { curr.EXPECT().ID().Return(ident.StringID("foo" + strconv.Itoa(i))).AnyTimes() curr.EXPECT().IsEmpty().Return(false).AnyTimes() curr.EXPECT(). - Flush(gomock.Any(), blockStart, gomock.Any(), 1, gomock.Any()). - Do(func(context.Context, time.Time, persist.DataFn, int, namespace.Context) { + WarmFlush(gomock.Any(), blockStart, gomock.Any(), gomock.Any()). + Do(func(context.Context, time.Time, persist.DataFn, namespace.Context) { flushed[i] = struct{}{} }). Return(series.FlushOutcomeErr, expectedErr) s.list.PushBack(lookup.NewEntry(curr, 0)) } - err := s.Flush(blockStart, flush, namespace.Context{}) + err := s.WarmFlush(blockStart, flush, namespace.Context{}) require.Equal(t, len(flushed), 2) for i := 0; i < 2; i++ { @@ -248,7 +248,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { flushState := s.FlushState(blockStart) require.Equal(t, fileOpState{ - Status: fileOpFailed, + WarmStatus: fileOpFailed, NumFailures: 2, }, flushState) } @@ -268,7 +268,7 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { defer s.Close() s.bootstrapState = Bootstrapped s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ - Status: fileOpFailed, + WarmStatus: fileOpFailed, NumFailures: 1, } @@ -293,15 +293,15 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { curr.EXPECT().ID().Return(ident.StringID("foo" + strconv.Itoa(i))).AnyTimes() curr.EXPECT().IsEmpty().Return(false).AnyTimes() curr.EXPECT(). - Flush(gomock.Any(), blockStart, gomock.Any(), 1, gomock.Any()). - Do(func(context.Context, time.Time, persist.DataFn, int, namespace.Context) { + WarmFlush(gomock.Any(), blockStart, gomock.Any(), gomock.Any()). + Do(func(context.Context, time.Time, persist.DataFn, namespace.Context) { flushed[i] = struct{}{} }). Return(series.FlushOutcomeFlushedToDisk, nil) s.list.PushBack(lookup.NewEntry(curr, 0)) } - err := s.Flush(blockStart, flush, namespace.Context{}) + err := s.WarmFlush(blockStart, flush, namespace.Context{}) require.Equal(t, len(flushed), 2) for i := 0; i < 2; i++ { @@ -314,8 +314,8 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { flushState := s.FlushState(blockStart) require.Equal(t, fileOpState{ - Status: fileOpSuccess, - Version: 1, + WarmStatus: fileOpSuccess, + ColdVersion: 0, NumFailures: 0, }, flushState) } @@ -478,10 +478,10 @@ func TestShardTick(t *testing.T) { // Also check that it expires flush states by time shard.flushState.statesByTime[xtime.ToUnixNano(earliestFlush)] = fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, } shard.flushState.statesByTime[xtime.ToUnixNano(beforeEarliestFlush)] = fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, } assert.Equal(t, 2, len(shard.flushState.statesByTime)) @@ -639,10 +639,10 @@ func testShardWriteAsync(t *testing.T, writes []testWrite) { // Also check that it expires flush states by time shard.flushState.statesByTime[xtime.ToUnixNano(earliestFlush)] = fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, } shard.flushState.statesByTime[xtime.ToUnixNano(beforeEarliestFlush)] = fileOpState{ - Status: fileOpSuccess, + WarmStatus: fileOpSuccess, } assert.Equal(t, 2, len(shard.flushState.statesByTime)) @@ -1065,8 +1065,8 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { ropts := shard.seriesOpts.RetentionOptions() end := opts.ClockOptions().NowFn()().Truncate(ropts.BlockSize()) start := end.Add(-2 * ropts.BlockSize()) - shard.markFlushStateSuccess(start, 1) - shard.markFlushStateSuccess(start.Add(ropts.BlockSize()), 1) + shard.markWarmFlushStateSuccess(start) + shard.markWarmFlushStateSuccess(start.Add(ropts.BlockSize())) retriever := block.NewMockDatabaseBlockRetriever(ctrl) shard.setBlockRetriever(retriever) diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 6ff4c36dd2..1eadf329b1 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -331,8 +331,8 @@ type databaseNamespace interface { // Bootstrap performs bootstrapping. Bootstrap(start time.Time, process bootstrap.Process) error - // Flush flushes in-memory data. - Flush( + // WarmFlush flushes in-memory WarmWrites. + WarmFlush( blockStart time.Time, ShardBootstrapStates ShardBootstrapStates, flush persist.FlushPreparer, @@ -343,7 +343,12 @@ type databaseNamespace interface { flush persist.IndexFlush, ) error - // Snapshot snapshots unflushed in-memory data + // ColdFlush flushes unflushed in-memory ColdWrites. + ColdFlush( + flush persist.FlushPreparer, + ) error + + // Snapshot snapshots unflushed in-memory WarmWrites. Snapshot(blockStart, snapshotTime time.Time, flush persist.SnapshotPreparer) error // NeedsFlush returns true if the namespace needs a flush for the @@ -453,22 +458,31 @@ type databaseShard interface { bootstrappedSeries *result.Map, ) error - // Flush flushes the series' in this shard. - Flush( + // WarmFlush flushes the WarmWrites in this shard. + WarmFlush( blockStart time.Time, flush persist.FlushPreparer, nsCtx namespace.Context, ) error - // Snapshot snapshot's the unflushed series' in this shard. - Snapshot(blockStart, snapshotStart time.Time, flush persist.SnapshotPreparer, nsCtx namespace.Context) error + // ColdFlush flushes the unflushed ColdWrites in this shard. + ColdFlush( + flush persist.FlushPreparer, + resources coldFlushReuseableResources, + nsCtx namespace.Context, + ) error + + // Snapshot snapshot's the unflushed WarmWrites in this shard. + Snapshot( + blockStart time.Time, + snapshotStart time.Time, + flush persist.SnapshotPreparer, + nsCtx namespace.Context, + ) error // FlushState returns the flush state for this shard at block start. FlushState(blockStart time.Time) fileOpState - // SnapshotState returns the snapshot state for this shard. - SnapshotState() (isSnapshotting bool, lastSuccessfulSnapshot time.Time) - // CleanupExpiredFileSets removes expired fileset files. CleanupExpiredFileSets(earliestToRetain time.Time) error diff --git a/src/dbnode/x/xio/block_reader_slice_from_segment_reader_slice.go b/src/dbnode/x/xio/block_reader_slice_from_segment_reader_slice.go new file mode 100644 index 0000000000..604924cfc9 --- /dev/null +++ b/src/dbnode/x/xio/block_reader_slice_from_segment_reader_slice.go @@ -0,0 +1,35 @@ +// 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 xio + +// NewSegmentReaderSliceFromBlockReaderSlice creates a new SegmentReader slice +// from a BlockReader slice. +func NewSegmentReaderSliceFromBlockReaderSlice( + brs []BlockReader, +) []SegmentReader { + srs := make([]SegmentReader, 0, len(brs)) + + for _, br := range brs { + srs = append(srs, br.SegmentReader) + } + + return srs +} diff --git a/src/x/generated-source-files.mk b/src/x/generated-source-files.mk index 522533e5d6..efc6b52b16 100644 --- a/src/x/generated-source-files.mk +++ b/src/x/generated-source-files.mk @@ -26,7 +26,7 @@ byteshashmap-update: cd $(m3x_package_path)/generics/hashmap && cat ./map.go | grep -v nolint | genny -pkg byteskey -ast gen "KeyType=[]byte ValueType=MapValue" > ./byteskey/map_gen.go # NB(prateek): `target_package` should not have a trailing slash -# generic targets meant to be re-used by other users +# Generic targets meant to be re-used by other users .PHONY: hashmap-gen hashmap-gen: $(eval out_dir=$(gopath_prefix)/$(target_package)) @@ -89,11 +89,10 @@ ifeq ($(rename_nogen_key),) [ "$(key_type_alias)" == "" ] || echo "type $(key_type_alias) interface{}" >> $(temp_outdir)/key.go endif ifeq ($(rename_nogen_value),) - # allow users to short circuit the generation of key.go if they don't need it. + # Allow users to short circuit the generation of value.go if they don't need it. echo 'package $(pkg)' > $(temp_outdir)/value.go echo '' >> $(temp_outdir)/value.go [[ $(value_type_alias) = struct* ]] || echo 'type $(value_type_alias) interface{}' >> $(temp_outdir)/value.go - # allow users to short circuit the generation of value.go if they don't need it. endif mv $(out_dir)/map_gen.go $(temp_outdir)/map_gen.go make hashmap-gen-rename-helper @@ -133,7 +132,7 @@ genny-arraypool-ident-tags: rename_gen_types=true # NB(prateek): `target_package` should not have a trailing slash -# generic arraypool generation rule +# Generic arraypool generation rule .PHONY: genny-arraypool genny-arraypool: $(eval out_dir=$(gopath_prefix)/$(target_package)) @@ -153,7 +152,7 @@ elem_type_alias ?= $(elem_type) arraypool-gen-rename: install-gorename $(eval temp_outdir=$(out_dir)$(temp_suffix)) ifneq ($(rename_gen_types),) - # allow users to short circuit the generation of types.go if they don't need it. + # Allow users to short circuit the generation of types.go if they don't need it. echo 'package $(pkg)' > $(temp_outdir)/types.go echo '' >> $(temp_outdir)/types.go echo "type $(elem_type_alias) interface{}" >> $(temp_outdir)/types.go @@ -195,7 +194,7 @@ elem_type_alias ?= $(elem_type) list-gen-rename: install-gorename $(eval temp_outdir=$(out_dir)$(temp_suffix)) ifneq ($(rename_gen_types),) - # allow users to short circuit the generation of types.go if they don't need it. + # Allow users to short circuit the generation of types.go if they don't need it. echo 'package $(pkg)' > $(temp_outdir)/types.go echo '' >> $(temp_outdir)/types.go echo "type $(elem_type_alias) interface{}" >> $(temp_outdir)/types.go From a21322710735a3c15a0d7d01d21e909fe910d7f2 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Mon, 13 May 2019 15:19:43 -0400 Subject: [PATCH 02/18] Only consider dirty cold buckets in considering if blocks need to be cold flushed --- src/dbnode/storage/series/buffer.go | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 4e4a8df2a6..7e23dd315b 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -314,7 +314,7 @@ func (b *dbBuffer) NeedsColdFlushBlockStarts() OptimizedTimes { for t, bucketVersions := range b.bucketsMap { for _, bucket := range bucketVersions.buckets { - if bucket.writeType == ColdWrite { + if bucket.version == writableBucketVer && bucket.writeType == ColdWrite { times.Add(t) break } @@ -596,14 +596,13 @@ func (b *dbBuffer) FetchBlocksForColdFlush( ) ([]xio.BlockReader, error) { res := b.fetchBlocks(ctx, []time.Time{start}, streamsOptions{filterWriteType: true, writeType: ColdWrite, nsCtx: nsCtx}) - if len(res) == 0 { - return nil, nil - } - // We should have at most one BlockReader slice since we are only looking - // at one block. - if len(res) > 1 { - return nil, errors.New("fetchBlocks returned more than one result for one block") + if len(res) != 1 { + // The lifecycle of calling this function is preceded by first checking + // which blocks have cold data that have not yet been flushed, so + // something is wrong if we don't get any data here. + return nil, fmt.Errorf("fetchBlocks did not return just one block for block start %s", start) } + blocks := res[0].Blocks buckets, exists := b.bucketVersionsAt(start) From f14581c88ae4f70e763f1bf2173850f6ac2d4d58 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Wed, 15 May 2019 16:07:04 -0400 Subject: [PATCH 03/18] Cold flush PR comments --- .../server/tchannelthrift/node/service.go | 3 +- src/dbnode/retention/times.go | 3 +- src/dbnode/storage/flush.go | 6 +- src/dbnode/storage/namespace.go | 16 +- src/dbnode/storage/series/buffer_test.go | 252 ++++++++++++++++-- src/dbnode/storage/series/series_all_test.go | 6 + src/dbnode/storage/series/series_mock.go | 12 +- src/dbnode/storage/series/types.go | 5 +- src/dbnode/storage/shard.go | 45 ++-- 9 files changed, 285 insertions(+), 63 deletions(-) diff --git a/src/dbnode/network/server/tchannelthrift/node/service.go b/src/dbnode/network/server/tchannelthrift/node/service.go index b18e75653d..23bb094f95 100644 --- a/src/dbnode/network/server/tchannelthrift/node/service.go +++ b/src/dbnode/network/server/tchannelthrift/node/service.go @@ -770,7 +770,8 @@ func (s *service) FetchBlocksRaw(tctx thrift.Context, req *rpc.FetchBlocksRawReq // Preallocate starts to maximum size since at least one element will likely // be fetching most blocks for peer bootstrapping ropts := nsMetadata.Options().RetentionOptions() - blockStarts := make([]time.Time, 0, ropts.RetentionPeriod()/ropts.BlockSize()) + blockStarts := make([]time.Time, 0, + (ropts.RetentionPeriod()+ropts.FutureRetentionPeriod())/ropts.BlockSize()) for i, request := range req.Elements { blockStarts = blockStarts[:0] diff --git a/src/dbnode/retention/times.go b/src/dbnode/retention/times.go index 855286ac1e..8e6c733668 100644 --- a/src/dbnode/retention/times.go +++ b/src/dbnode/retention/times.go @@ -34,7 +34,8 @@ func FlushTimeStartForRetentionPeriod(retentionPeriod time.Duration, blockSize t // FlushTimeEnd is the latest flushable time func FlushTimeEnd(opts Options, t time.Time) time.Time { - return FlushTimeEndForBlockSize(opts.BlockSize(), t.Add(-opts.BufferPast())) + return FlushTimeEndForBlockSize(opts.BlockSize(), + t.Add(opts.FutureRetentionPeriod()).Add(-opts.BufferPast())) } // FlushTimeEndForBlockSize is the latest flushable time diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index cee0ae8f6a..15ed8dedbf 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -117,9 +117,9 @@ func (m *flushManager) Flush( // over cold flushes and snapshotting. // // In addition, we need to make sure that for any given shard/blockStart - // combination, we attempt a flush before a snapshot as the snapshotting - // process will attempt to snapshot any unflushed blocks which would be - // wasteful if the block is already flushable. + // combination, we attempt a flush and then a cold flush before a snapshot + // as the snapshotting process will attempt to snapshot any unflushed blocks + // which would be wasteful if the block is already flushable. multiErr := xerrors.NewMultiError() if err = m.dataWarmFlush(namespaces, tickStart, dbBootstrapStateAtTickStart); err != nil { multiErr = multiErr.Add(err) diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 163be858c6..f53b622f2d 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -994,6 +994,18 @@ type coldFlushReuseableResources struct { fsReader fs.DataFileSetReader } +func (r *coldFlushReuseableResources) reset() { + for _, seriesList := range r.dirtySeriesToWrite { + if seriesList != nil { + seriesList.Reset() + } + // Don't delete the empty list from the map so that other shards don't + // need to reinitialize the list for these blocks. + } + + r.dirtySeries.Reset() +} + func (n *dbNamespace) ColdFlush( flushPersist persist.FlushPreparer, ) error { @@ -1025,7 +1037,7 @@ func (n *dbNamespace) ColdFlush( resources := coldFlushReuseableResources{ dirtySeries: newDirtySeriesMap(dirtySeriesMapOptions{}), dirtySeriesToWrite: make(map[xtime.UnixNano]*idList), - // TODO(juchan) set pool options + // TODO(juchan): set pool options. idElementPool: newIDElementPool(nil), fsReader: fsReader, } @@ -1034,7 +1046,7 @@ func (n *dbNamespace) ColdFlush( if err != nil { detailedErr := fmt.Errorf("shard %d failed to compact: %v", shard.ID(), err) multiErr = multiErr.Add(detailedErr) - // Continue with remaining shards + // Continue with remaining shards. } } diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index c3ed8cab17..0cf0daab75 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -267,29 +267,44 @@ func TestBufferWriteOutOfOrder(t *testing.T) { func newTestBufferBucketWithData(t *testing.T, opts Options, setAnn setAnnotation) (*BufferBucket, []value) { rops := opts.RetentionOptions() curr := time.Now().Truncate(rops.BlockSize()) - b := &BufferBucket{opts: opts} - b.resetTo(curr, WarmWrite, opts) - data := [][]value{ - { - {curr, 1, xtime.Second, nil}, - {curr.Add(secs(10)), 2, xtime.Second, nil}, - {curr.Add(secs(50)), 3, xtime.Second, nil}, - }, - { - {curr.Add(secs(20)), 4, xtime.Second, nil}, - {curr.Add(secs(40)), 5, xtime.Second, nil}, - {curr.Add(secs(60)), 6, xtime.Second, nil}, - }, - { - {curr.Add(secs(30)), 4, xtime.Second, nil}, - {curr.Add(secs(70)), 5, xtime.Second, nil}, - }, - { - {curr.Add(secs(35)), 6, xtime.Second, nil}, + bd := blockData{ + start: curr, + writeType: WarmWrite, + data: [][]value{ + { + {curr, 1, xtime.Second, nil}, + {curr.Add(secs(10)), 2, xtime.Second, nil}, + {curr.Add(secs(50)), 3, xtime.Second, nil}, + }, + { + {curr.Add(secs(20)), 4, xtime.Second, nil}, + {curr.Add(secs(40)), 5, xtime.Second, nil}, + {curr.Add(secs(60)), 6, xtime.Second, nil}, + }, + { + {curr.Add(secs(30)), 4, xtime.Second, nil}, + {curr.Add(secs(70)), 5, xtime.Second, nil}, + }, + { + {curr.Add(secs(35)), 6, xtime.Second, nil}, + }, }, } + return newTestBufferBucketWithCustomData(t, bd, opts, setAnn) +} + +func newTestBufferBucketWithCustomData( + t *testing.T, + bd blockData, + opts Options, + setAnn setAnnotation, +) (*BufferBucket, []value) { + b := &BufferBucket{opts: opts} + b.resetTo(bd.start, bd.writeType, opts) + data := bd.data + // Empty all existing encoders. b.encoders = nil @@ -305,7 +320,7 @@ func newTestBufferBucketWithData(t *testing.T, opts Options, setAnn setAnnotatio encoded := 0 encoder := opts.EncoderPool().Get() - encoder.Reset(curr, 0, nsCtx.Schema) + encoder.Reset(bd.start, 0, nsCtx.Schema) for _, v := range data[i] { dp := ts.Datapoint{ Timestamp: v.timestamp, @@ -331,6 +346,39 @@ func newTestBufferBucketsWithData(t *testing.T, opts Options, setAnn setAnnotati }, vals } +func newTestBufferBucketVersionsWithCustomData( + t *testing.T, + bd blockData, + opts Options, + setAnn setAnnotation, +) (*BufferBucketVersions, []value) { + newBucket, vals := newTestBufferBucketWithCustomData(t, bd, opts, setAnn) + return &BufferBucketVersions{ + buckets: []*BufferBucket{newBucket}, + start: newBucket.start, + opts: opts, + }, vals +} + +func newTestBufferWithCustomData( + t *testing.T, + blockDatas []blockData, + opts Options, + setAnn setAnnotation, +) (*dbBuffer, map[xtime.UnixNano][]value) { + buffer := newDatabaseBuffer().(*dbBuffer) + buffer.Reset(opts) + expectedMap := make(map[xtime.UnixNano][]value) + + for _, bd := range blockDatas { + bucketVersions, expected := newTestBufferBucketVersionsWithCustomData(t, bd, opts, setAnn) + buffer.bucketsMap[xtime.ToUnixNano(bd.start)] = bucketVersions + expectedMap[xtime.ToUnixNano(bd.start)] = expected + } + + return buffer, expectedMap +} + func TestBufferBucketMerge(t *testing.T) { opts := newBufferTestOptions() @@ -1152,3 +1200,167 @@ func TestOptimizedTimes(t *testing.T) { assert.Equal(t, expectedTimes[i], forEachTimes[i]) } } + +func TestNeedsColdFlushBlockStarts(t *testing.T) { + opts := newBufferTestOptions() + rops := opts.RetentionOptions() + blockSize := rops.BlockSize() + blockStart4 := time.Now().Truncate(blockSize) + blockStart3 := blockStart4.Add(-2 * blockSize) + blockStart2 := blockStart4.Add(-3 * blockSize) + blockStart1 := blockStart4.Add(-4 * blockSize) + + bds := []blockData{ + blockData{ + start: blockStart1, + writeType: ColdWrite, + data: [][]value{ + { + {blockStart1, 1, xtime.Second, nil}, + {blockStart1.Add(secs(5)), 2, xtime.Second, nil}, + {blockStart1.Add(secs(10)), 3, xtime.Second, nil}, + }, + }, + }, + blockData{ + start: blockStart2, + writeType: ColdWrite, + data: [][]value{ + { + {blockStart2.Add(secs(2)), 4, xtime.Second, nil}, + {blockStart2.Add(secs(5)), 5, xtime.Second, nil}, + {blockStart2.Add(secs(11)), 6, xtime.Second, nil}, + {blockStart2.Add(secs(15)), 7, xtime.Second, nil}, + {blockStart2.Add(secs(40)), 8, xtime.Second, nil}, + }, + }, + }, + blockData{ + start: blockStart3, + writeType: ColdWrite, + data: [][]value{ + { + {blockStart3.Add(secs(71)), 9, xtime.Second, nil}, + }, + }, + }, + blockData{ + start: blockStart4, + writeType: WarmWrite, + data: [][]value{ + { + {blockStart4.Add(secs(57)), 10, xtime.Second, nil}, + {blockStart4.Add(secs(66)), 11, xtime.Second, nil}, + {blockStart4.Add(secs(80)), 12, xtime.Second, nil}, + {blockStart4.Add(secs(81)), 13, xtime.Second, nil}, + {blockStart4.Add(secs(82)), 14, xtime.Second, nil}, + {blockStart4.Add(secs(96)), 15, xtime.Second, nil}, + }, + }, + }, + } + + buffer, _ := newTestBufferWithCustomData(t, bds, opts, nil) + flushStarts := buffer.NeedsColdFlushBlockStarts() + + // All three cold blocks should report that they are dirty. + assert.Equal(t, 3, flushStarts.Len()) + assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart1))) + assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart2))) + assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart3))) + + // Simulate that block2 and block3 are flushed (but not yet evicted from + // memory), so only block1 should report as dirty. + buffer.bucketsMap[xtime.ToUnixNano(blockStart2)].buckets[0].version = 1 + buffer.bucketsMap[xtime.ToUnixNano(blockStart3)].buckets[0].version = 1 + + flushStarts = buffer.NeedsColdFlushBlockStarts() + assert.Equal(t, 1, flushStarts.Len()) + assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart1))) +} + +func TestFetchBlocksForColdFlush(t *testing.T) { + opts := newBufferTestOptions() + rops := opts.RetentionOptions() + blockSize := rops.BlockSize() + blockStart4 := time.Now().Truncate(blockSize) + blockStart3 := blockStart4.Add(-2 * blockSize) + blockStartNano3 := xtime.ToUnixNano(blockStart3) + blockStart2 := blockStart4.Add(-3 * blockSize) + blockStart1 := blockStart4.Add(-4 * blockSize) + blockStartNano1 := xtime.ToUnixNano(blockStart1) + + bds := []blockData{ + blockData{ + start: blockStart1, + writeType: ColdWrite, + data: [][]value{ + { + {blockStart1, 1, xtime.Second, nil}, + {blockStart1.Add(secs(5)), 2, xtime.Second, nil}, + {blockStart1.Add(secs(10)), 3, xtime.Second, nil}, + }, + }, + }, + blockData{ + start: blockStart2, + writeType: ColdWrite, + data: [][]value{ + { + {blockStart2.Add(secs(2)), 4, xtime.Second, nil}, + {blockStart2.Add(secs(5)), 5, xtime.Second, nil}, + {blockStart2.Add(secs(11)), 6, xtime.Second, nil}, + {blockStart2.Add(secs(15)), 7, xtime.Second, nil}, + {blockStart2.Add(secs(40)), 8, xtime.Second, nil}, + }, + }, + }, + blockData{ + start: blockStart3, + writeType: ColdWrite, + data: [][]value{ + { + {blockStart3.Add(secs(71)), 9, xtime.Second, nil}, + }, + }, + }, + blockData{ + start: blockStart4, + writeType: WarmWrite, + data: [][]value{ + { + {blockStart4.Add(secs(57)), 10, xtime.Second, nil}, + {blockStart4.Add(secs(66)), 11, xtime.Second, nil}, + {blockStart4.Add(secs(80)), 12, xtime.Second, nil}, + {blockStart4.Add(secs(81)), 13, xtime.Second, nil}, + {blockStart4.Add(secs(82)), 14, xtime.Second, nil}, + {blockStart4.Add(secs(96)), 15, xtime.Second, nil}, + }, + }, + }, + } + + buffer, expected := newTestBufferWithCustomData(t, bds, opts, nil) + ctx := context.NewContext() + defer ctx.Close() + nsCtx := namespace.Context{Schema: testSchemaDesc} + reader, err := buffer.FetchBlocksForColdFlush(ctx, blockStart1, 4, nsCtx) + assert.NoError(t, err) + // Verify that we got the correct data and that version is correct set. + requireReaderValuesEqual(t, expected[blockStartNano1], [][]xio.BlockReader{reader}, opts, nsCtx) + assert.Equal(t, 4, buffer.bucketsMap[blockStartNano1].buckets[0].version) + + // Try to fetch from block1 again, which should result in error since we + // just fetched, which would mark those buckets as not dirty. + _, err = buffer.FetchBlocksForColdFlush(ctx, blockStart1, 9, nsCtx) + assert.Error(t, err) + + reader, err = buffer.FetchBlocksForColdFlush(ctx, blockStart3, 1, nsCtx) + assert.NoError(t, err) + requireReaderValuesEqual(t, expected[blockStartNano3], [][]xio.BlockReader{reader}, opts, nsCtx) + assert.Equal(t, 1, buffer.bucketsMap[blockStartNano3].buckets[0].version) + + // Try to fetch from a block that only has warm buckets, which should error. + _, err = buffer.FetchBlocksForColdFlush(ctx, blockStart4, 1, nsCtx) + assert.Error(t, err) +} diff --git a/src/dbnode/storage/series/series_all_test.go b/src/dbnode/storage/series/series_all_test.go index 0fc36e51e9..a35a22066a 100644 --- a/src/dbnode/storage/series/series_all_test.go +++ b/src/dbnode/storage/series/series_all_test.go @@ -41,6 +41,12 @@ func mins(x float64) time.Duration { return time.Duration(x * float64(time.Minute)) } +type blockData struct { + start time.Time + writeType WriteType + data [][]value +} + type value struct { timestamp time.Time value float64 diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 1b30071240..ea20a3d9da 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -360,18 +360,18 @@ func (mr *MockQueryableBlockRetrieverMockRecorder) IsBlockRetrievable(arg0 inter return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsBlockRetrievable", reflect.TypeOf((*MockQueryableBlockRetriever)(nil).IsBlockRetrievable), arg0) } -// RetrievableBlockVersion mocks base method -func (m *MockQueryableBlockRetriever) RetrievableBlockVersion(arg0 time.Time) int { +// RetrievableBlockColdVersion mocks base method +func (m *MockQueryableBlockRetriever) RetrievableBlockColdVersion(arg0 time.Time) int { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RetrievableBlockVersion", arg0) + ret := m.ctrl.Call(m, "RetrievableBlockColdVersion", arg0) ret0, _ := ret[0].(int) return ret0 } -// RetrievableBlockVersion indicates an expected call of RetrievableBlockVersion -func (mr *MockQueryableBlockRetrieverMockRecorder) RetrievableBlockVersion(arg0 interface{}) *gomock.Call { +// RetrievableBlockColdVersion indicates an expected call of RetrievableBlockColdVersion +func (mr *MockQueryableBlockRetrieverMockRecorder) RetrievableBlockColdVersion(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RetrievableBlockVersion", reflect.TypeOf((*MockQueryableBlockRetriever)(nil).RetrievableBlockVersion), arg0) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RetrievableBlockColdVersion", reflect.TypeOf((*MockQueryableBlockRetriever)(nil).RetrievableBlockColdVersion), arg0) } // Stream mocks base method diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index efbbbd5a84..8dd4834b43 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -158,8 +158,9 @@ type QueryableBlockRetriever interface { // for a given block start time. IsBlockRetrievable(blockStart time.Time) bool - // RetrievableBlockVersion returns the last time a block was marked success - RetrievableBlockVersion(blockStart time.Time) int + // RetrievableBlockColdVersion returns the cold version that was + // successfully persisted. + RetrievableBlockColdVersion(blockStart time.Time) int // BlockStatesSnapshot returns a snapshot of the whether blocks are // retrievable and their flush versions for each block start. This is used diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index d1f8214ce6..a2b8ca4ead 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -356,8 +356,8 @@ func statusIsRetrievable(status fileOpStatus) bool { status)) } -// RetrievableBlockVersion implements series.QueryableBlockRetriever -func (s *dbShard) RetrievableBlockVersion(blockStart time.Time) int { +// RetrievableBlockColdVersion implements series.QueryableBlockRetriever +func (s *dbShard) RetrievableBlockColdVersion(blockStart time.Time) int { flushState := s.FlushState(blockStart) return flushState.ColdVersion } @@ -1921,24 +1921,11 @@ func (s *dbShard) ColdFlush( } s.RUnlock() + resources.reset() dirtySeriesToWrite := resources.dirtySeriesToWrite dirtySeries := resources.dirtySeries idElementPool := resources.idElementPool fsReader := resources.fsReader - // Reset resources. - for _, series := range dirtySeriesToWrite { - if series != nil { - series.Reset() - } - // Don't delete the empty list from the map so that other shards don't - // need to reinitialize the list for these blocks. - } - dirtySeries.Reset() - if fsReader.Status().Open { - if err := fsReader.Close(); err != nil { - return err - } - } var multiErr xerrors.MultiError // First, loop through all series to capture data on which blocks have dirty @@ -1986,9 +1973,7 @@ BlockLoop: } defer fsReader.Close() - s.flushState.RLock() - nextVersion := s.flushState.statesByTime[blockStart].ColdVersion + 1 - s.flushState.RUnlock() + nextVersion := s.RetrievableBlockColdVersion(startTime) + 1 prepareOpts := persist.DataPrepareOptions{ NamespaceMetadata: s.namespace, @@ -2003,13 +1988,13 @@ BlockLoop: continue BlockLoop } - // Persistence is done in two stages. The first stage is to loop through - // series on disk and merge it with what's in memory. The second stage - // is to persist the rest of the series in memory that was not - // persisted in the first stage. + // Writing data for a block is done in two stages. The first stage is + // to loop through series on disk and merge it with what's in memory. + // The second stage is to persist the rest of the series in memory that + // was not persisted in the first stage. // First stage: loop through series on disk. - for id, tagsIter, data, _, err := fsReader.Read(); err != io.EOF; id, _, data, _, err = fsReader.Read() { + for id, tagsIter, data, _, err := fsReader.Read(); err != io.EOF; id, tagsIter, data, _, err = fsReader.Read() { if err != nil { s.incrementFlushStateFailures(startTime) multiErr = multiErr.Add(err) @@ -2062,12 +2047,13 @@ BlockLoop: defer mergedIter.Close() tags, err := convert.TagsFromTagsIter(id, tagsIter, s.opts.IdentifierPool()) + tagsIter.Close() if err != nil { s.incrementFlushStateFailures(startTime) multiErr = multiErr.Add(err) continue BlockLoop } - if err := persistIter(prepared.Persist, id, tags, mergedIter, encoderPool); err != nil { + if err := persistIter(prepared.Persist, mergedIter, id, tags, encoderPool); err != nil { s.incrementFlushStateFailures(startTime) multiErr = multiErr.Add(err) continue BlockLoop @@ -2099,7 +2085,7 @@ BlockLoop: iter := multiIterPool.Get() iter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize, nsCtx.Schema) defer iter.Close() - if err := persistIter(prepared.Persist, series.ID(), series.Tags(), iter, encoderPool); err != nil { + if err := persistIter(prepared.Persist, iter, series.ID(), series.Tags(), encoderPool); err != nil { s.incrementFlushStateFailures(startTime) multiErr = multiErr.Add(err) continue BlockLoop @@ -2107,7 +2093,8 @@ BlockLoop: } } - // Close the flush preparer, which writes the rest of the filesets. + // Close the flush preparer, which writes the rest of the files in the + // fileset. if err := prepared.Close(); err != nil { multiErr = multiErr.Add(err) } @@ -2122,9 +2109,9 @@ BlockLoop: func persistIter( persistFn persist.DataFn, + it encoding.Iterator, id ident.ID, tags ident.Tags, - it encoding.Iterator, encoderPool encoding.EncoderPool, ) error { encoder := encoderPool.Get() @@ -2152,6 +2139,8 @@ func persistIter( checksum := digest.SegmentChecksum(segment) err = persistFn(id, tags, segment, checksum) + id.Finalize() + tags.Finalize() if err != nil { return err } From 3aa8c79a7eb744d69b70da85fac43da15e006394 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Fri, 17 May 2019 19:33:04 -0400 Subject: [PATCH 04/18] Fileset merge interface --- src/dbnode/storage/fs_merge_with_mem.go | 96 +++++++++++ src/dbnode/storage/fs_merger.go | 220 ++++++++++++++++++++++++ src/dbnode/storage/shard.go | 218 +++-------------------- src/dbnode/storage/types.go | 15 ++ 4 files changed, 357 insertions(+), 192 deletions(-) create mode 100644 src/dbnode/storage/fs_merge_with_mem.go create mode 100644 src/dbnode/storage/fs_merger.go diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go new file mode 100644 index 0000000000..67406e7f95 --- /dev/null +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -0,0 +1,96 @@ +// 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 storage + +import ( + "errors" + + "github.com/m3db/m3/src/dbnode/x/xio" + "github.com/m3db/m3/src/x/context" + "github.com/m3db/m3/src/x/ident" + xtime "github.com/m3db/m3/src/x/time" +) + +// fsMergeWithMem implements FsMergeWith +type fsMergeWithMem struct { + shard *dbShard + dirtySeries *dirtySeriesMap + dirtySeriesToWrite map[xtime.UnixNano]*idList +} + +func (m *fsMergeWithMem) Read( + blockStart xtime.UnixNano, + seriesID ident.ID, +) ([]xio.BlockReader, bool, error) { + // Check if this series is in memory (and thus requires merging). + element, exists := m.dirtySeries.Get(idAndBlockStart{blockStart: blockStart, id: seriesID}) + if !exists { + return nil, false, nil + } + + startTime := blockStart.ToTime() + + // Series is in memory, so it will get merged with disk and + // written in this loop. Therefore, we need to remove it from + // the "to write" list so that the later loop does not rewrite + // it. + m.dirtySeriesToWrite[blockStart].Remove(element) + + m.shard.RLock() + entry, _, err := m.shard.lookupEntryWithLock(element.Value) + m.shard.RUnlock() + + nextVersion := m.shard.RetrievableBlockColdVersion(startTime) + 1 + + tmpCtx := context.NewContext() + blocks, err := entry.Series.FetchBlocksForColdFlush(tmpCtx, startTime, nextVersion) + tmpCtx.BlockingClose() + if err != nil { + return nil, false, err + } + + if len(blocks) > 0 { + return blocks, true, nil + } + + return nil, false, nil +} + +func (m *fsMergeWithMem) ForEachRemaining(blockStart xtime.UnixNano, fn forEachRemainingFn) error { + seriesList := m.dirtySeriesToWrite[blockStart] + + for seriesElement := seriesList.Front(); seriesElement != nil; seriesElement = seriesElement.Next() { + m.shard.RLock() + entry, _, err := m.shard.lookupEntryWithLock(seriesElement.Value) + m.shard.RUnlock() + if err != nil { + return err + } + + series := entry.Series + success := fn(series.ID(), series.Tags()) + if !success { + return errors.New("foreach iteration unsuccessful") + } + } + + return nil +} diff --git a/src/dbnode/storage/fs_merger.go b/src/dbnode/storage/fs_merger.go new file mode 100644 index 0000000000..d112a291e7 --- /dev/null +++ b/src/dbnode/storage/fs_merger.go @@ -0,0 +1,220 @@ +// 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 storage + +import ( + "io" + "time" + + "github.com/m3db/m3/src/dbnode/digest" + "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/persist" + "github.com/m3db/m3/src/dbnode/persist/fs" + "github.com/m3db/m3/src/dbnode/storage/index/convert" + "github.com/m3db/m3/src/dbnode/storage/namespace" + "github.com/m3db/m3/src/dbnode/ts" + "github.com/m3db/m3/src/dbnode/x/xio" + xerrors "github.com/m3db/m3/src/x/errors" + "github.com/m3db/m3/src/x/ident" + xtime "github.com/m3db/m3/src/x/time" +) + +type fsMergerReusableResources struct { + fsReader fs.DataFileSetReader + srPool xio.SegmentReaderPool + multiIterPool encoding.MultiReaderIteratorPool + identPool ident.Pool + encoderPool encoding.EncoderPool +} + +type fsMerger struct { + res fsMergerReusableResources +} + +func (m *fsMerger) Merge( + ns namespace.Metadata, + shard uint32, + blockStart xtime.UnixNano, + blockSize time.Duration, + flushPreparer persist.FlushPreparer, + mergeWith FsMergeWith, +) error { + var multiErr xerrors.MultiError + + fsReader := m.res.fsReader + srPool := m.res.srPool + multiIterPool := m.res.multiIterPool + identPool := m.res.identPool + encoderPool := m.res.encoderPool + + startTime := blockStart.ToTime() + openOpts := fs.DataReaderOpenOptions{ + Identifier: fs.FileSetFileIdentifier{ + Namespace: ns.ID(), + Shard: shard, + BlockStart: startTime, + }, + } + if err := fsReader.Open(openOpts); err != nil { + multiErr = multiErr.Add(err) + return multiErr.FinalError() + } + defer fsReader.Close() + + prepareOpts := persist.DataPrepareOptions{ + NamespaceMetadata: ns, + Shard: shard, + BlockStart: startTime, + DeleteIfExists: false, + } + prepared, err := flushPreparer.PrepareData(prepareOpts) + if err != nil { + multiErr = multiErr.Add(err) + return multiErr.FinalError() + } + + // Writing data for a block is done in two stages. The first stage is + // to loop through series on disk and merge it with what's in the merge + // target. The second stage is to persist the rest of the series in the + // merge target that was not persisted in the first stage. + + // First stage: loop through series on disk. + for id, tagsIter, data, _, err := fsReader.Read(); err != io.EOF; id, tagsIter, data, _, err = fsReader.Read() { + if err != nil { + multiErr = multiErr.Add(err) + return multiErr.FinalError() + } + + // There will be at most two BlockReader slices: one for disk data + // and one for in memory data. + brs := make([][]xio.BlockReader, 0, 2) + + // Create BlockReader slice out of disk data. + seg := ts.NewSegment(data, nil, ts.FinalizeHead) + sr := srPool.Get() + sr.Reset(seg) + br := xio.BlockReader{ + SegmentReader: sr, + Start: startTime, + BlockSize: blockSize, + } + brs = append(brs, []xio.BlockReader{br}) + + // Check if this series is in memory (and thus requires merging). + encoded, hasData, err := mergeWith.Read(blockStart, id) + if err != nil { + multiErr = multiErr.Add(err) + return multiErr.FinalError() + } + + if hasData { + brs = append(brs, encoded) + } + + mergedIter := multiIterPool.Get() + mergedIter.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(brs)) + defer mergedIter.Close() + + tags, err := convert.TagsFromTagsIter(id, tagsIter, identPool) + tagsIter.Close() + if err != nil { + multiErr = multiErr.Add(err) + return multiErr.FinalError() + } + if err := persistIter(prepared.Persist, mergedIter, id, tags, encoderPool); err != nil { + multiErr = multiErr.Add(err) + return multiErr.FinalError() + } + } + + // Second stage: loop through rest of the merge target that was not captured + // in the first stage. + err = mergeWith.ForEachRemaining(blockStart, func(seriesID ident.ID, tags ident.Tags) bool { + encoded, hasData, err := mergeWith.Read(blockStart, seriesID) + if err != nil { + multiErr = multiErr.Add(err) + return false + } + + if hasData { + iter := multiIterPool.Get() + iter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize) + defer iter.Close() + if err := persistIter(prepared.Persist, iter, seriesID, tags, encoderPool); err != nil { + multiErr = multiErr.Add(err) + return false + } + } + + return true + }) + if err != nil { + multiErr = multiErr.Add(err) + return multiErr.FinalError() + } + + // Close the flush preparer, which writes the rest of the files in the + // fileset. + if err := prepared.Close(); err != nil { + multiErr = multiErr.Add(err) + return multiErr.FinalError() + } + + return multiErr.FinalError() +} + +func persistIter( + persistFn persist.DataFn, + it encoding.Iterator, + id ident.ID, + tags ident.Tags, + encoderPool encoding.EncoderPool, +) error { + encoder := encoderPool.Get() + for it.Next() { + if err := encoder.Encode(it.Current()); err != nil { + encoder.Close() + return err + } + } + if err := it.Err(); err != nil { + return err + } + + stream := encoder.Stream() + encoder.Close() + + segment, err := stream.Segment() + if err != nil { + return err + } + + checksum := digest.SegmentChecksum(segment) + + err = persistFn(id, tags, segment, checksum) + id.Finalize() + tags.Finalize() + if err != nil { + return err + } + + return nil +} diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index a2b8ca4ead..b16d2fc1ac 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -30,8 +30,6 @@ import ( "time" "github.com/m3db/m3/src/dbnode/clock" - "github.com/m3db/m3/src/dbnode/digest" - "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/generated/proto/pagetoken" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" @@ -1922,12 +1920,15 @@ func (s *dbShard) ColdFlush( s.RUnlock() resources.reset() - dirtySeriesToWrite := resources.dirtySeriesToWrite - dirtySeries := resources.dirtySeries - idElementPool := resources.idElementPool - fsReader := resources.fsReader - var multiErr xerrors.MultiError + var ( + multiErr xerrors.MultiError + dirtySeries = resources.dirtySeries + dirtySeriesToWrite = resources.dirtySeriesToWrite + idElementPool = resources.idElementPool + blockSize = s.namespace.Options().RetentionOptions().BlockSize() + ) + // First, loop through all series to capture data on which blocks have dirty // series and add them to the resources for further processing. s.forEachShardEntry(func(entry *lookup.Entry) bool { @@ -1948,206 +1949,39 @@ func (s *dbShard) ColdFlush( return true }) - blockSize := s.namespace.Options().RetentionOptions().BlockSize() - srPool := s.opts.SegmentReaderPool() - multiIterPool := s.opts.MultiReaderIteratorPool() - encoderPool := s.opts.EncoderPool() - tmpCtx := context.NewContext() - + mergerResources := fsMergerReusableResources{ + fsReader: resources.fsReader, + srPool: s.opts.SegmentReaderPool(), + multiIterPool: s.opts.MultiReaderIteratorPool(), + identPool: s.opts.IdentifierPool(), + encoderPool: s.opts.EncoderPool(), + } + merger := &fsMerger{res: mergerResources} + mergeWithMem := &fsMergeWithMem{ + shard: s, + dirtySeries: dirtySeries, + dirtySeriesToWrite: dirtySeriesToWrite, + } // Loop through each block that we know has ColdWrites. Since each block // has its own fileset, if we encounter an error while trying to persist // a block, we continue to try persisting other blocks. -BlockLoop: - for blockStart, seriesList := range dirtySeriesToWrite { - startTime := blockStart.ToTime() - openOpts := fs.DataReaderOpenOptions{ - Identifier: fs.FileSetFileIdentifier{ - Namespace: s.namespace.ID(), - Shard: s.ID(), - BlockStart: startTime, - }, - } - if err := fsReader.Open(openOpts); err != nil { - multiErr = multiErr.Add(err) - continue BlockLoop - } - defer fsReader.Close() - - nextVersion := s.RetrievableBlockColdVersion(startTime) + 1 - - prepareOpts := persist.DataPrepareOptions{ - NamespaceMetadata: s.namespace, - Shard: s.ID(), - BlockStart: startTime, - DeleteIfExists: false, - } - prepared, err := flushPreparer.PrepareData(prepareOpts) + for blockStart := range dirtySeriesToWrite { + err := merger.Merge(s.namespace, s.ID(), blockStart, blockSize, flushPreparer, mergeWithMem) if err != nil { - s.incrementFlushStateFailures(startTime) - multiErr = multiErr.Add(err) - continue BlockLoop - } - - // Writing data for a block is done in two stages. The first stage is - // to loop through series on disk and merge it with what's in memory. - // The second stage is to persist the rest of the series in memory that - // was not persisted in the first stage. - - // First stage: loop through series on disk. - for id, tagsIter, data, _, err := fsReader.Read(); err != io.EOF; id, tagsIter, data, _, err = fsReader.Read() { - if err != nil { - s.incrementFlushStateFailures(startTime) - multiErr = multiErr.Add(err) - continue BlockLoop - } - - // There will be at most two BlockReader slices: one for disk data - // and one for in memory data. - brs := make([][]xio.BlockReader, 0, 2) - - // Create BlockReader slice out of disk data. - seg := ts.NewSegment(data, nil, ts.FinalizeHead) - sr := srPool.Get() - sr.Reset(seg) - br := xio.BlockReader{ - SegmentReader: sr, - Start: startTime, - BlockSize: blockSize, - } - brs = append(brs, []xio.BlockReader{br}) - - // Check if this series is in memory (and thus requires merging). - if element, ok := dirtySeries.Get(idAndBlockStart{blockStart: blockStart, id: id}); ok { - // Series is in memory, so it will get merged with disk and - // written in this loop. Therefore, we need to remove it from - // the "to write" list so that the later loop does not rewrite - // it. - dirtySeriesToWrite[blockStart].Remove(element) - - s.RLock() - entry, _, err := s.lookupEntryWithLock(element.Value) - s.RUnlock() - - tmpCtx.Reset() - encoded, err := entry.Series.FetchBlocksForColdFlush(tmpCtx, startTime, nextVersion, nsCtx) - tmpCtx.BlockingClose() - if err != nil { - s.incrementFlushStateFailures(startTime) - multiErr = multiErr.Add(err) - continue BlockLoop - } - - if len(encoded) > 0 { - brs = append(brs, encoded) - } - } - - mergedIter := multiIterPool.Get() - mergedIter.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(brs), nsCtx.Schema) - defer mergedIter.Close() - - tags, err := convert.TagsFromTagsIter(id, tagsIter, s.opts.IdentifierPool()) - tagsIter.Close() - if err != nil { - s.incrementFlushStateFailures(startTime) - multiErr = multiErr.Add(err) - continue BlockLoop - } - if err := persistIter(prepared.Persist, mergedIter, id, tags, encoderPool); err != nil { - s.incrementFlushStateFailures(startTime) - multiErr = multiErr.Add(err) - continue BlockLoop - } - } - - // Second stage: loop through rest of series in memory that was not on - // disk. - for seriesElement := seriesList.Front(); seriesElement != nil; seriesElement = seriesElement.Next() { - s.RLock() - entry, _, err := s.lookupEntryWithLock(seriesElement.Value) - s.RUnlock() - if err != nil { - s.incrementFlushStateFailures(startTime) - multiErr = multiErr.Add(err) - continue BlockLoop - } - series := entry.Series - tmpCtx.Reset() - encoded, err := series.FetchBlocksForColdFlush(tmpCtx, startTime, nextVersion, nsCtx) - tmpCtx.BlockingClose() - if err != nil { - s.incrementFlushStateFailures(startTime) - multiErr = multiErr.Add(err) - continue BlockLoop - } - - if len(encoded) > 0 { - iter := multiIterPool.Get() - iter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize, nsCtx.Schema) - defer iter.Close() - if err := persistIter(prepared.Persist, iter, series.ID(), series.Tags(), encoderPool); err != nil { - s.incrementFlushStateFailures(startTime) - multiErr = multiErr.Add(err) - continue BlockLoop - } - } - } - - // Close the flush preparer, which writes the rest of the files in the - // fileset. - if err := prepared.Close(); err != nil { multiErr = multiErr.Add(err) + continue } // After writing the full block successfully, update the cold version // in the flush state. + startTime := blockStart.ToTime() + nextVersion := s.RetrievableBlockColdVersion(startTime) + 1 s.setFlushStateColdVersion(startTime, nextVersion) } return multiErr.FinalError() } -func persistIter( - persistFn persist.DataFn, - it encoding.Iterator, - id ident.ID, - tags ident.Tags, - encoderPool encoding.EncoderPool, -) error { - encoder := encoderPool.Get() - for it.Next() { - if err := encoder.Encode(it.Current()); err != nil { - encoder.Close() - return err - } - } - if err := it.Err(); err != nil { - return err - } - - stream, ok := encoder.Stream(encoding.StreamOptions{}) - encoder.Close() - if !ok { - return nil - } - - segment, err := stream.Segment() - if err != nil { - return err - } - - checksum := digest.SegmentChecksum(segment) - - err = persistFn(id, tags, segment, checksum) - id.Finalize() - tags.Finalize() - if err != nil { - return err - } - - return nil -} - func (s *dbShard) Snapshot( blockStart time.Time, snapshotTime time.Time, diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 1eadf329b1..2952a0b02a 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -980,3 +980,18 @@ const ( // Bootstrapped indicates a bootstrap process has completed. Bootstrapped ) + +type forEachRemainingFn func(seriesID ident.ID, tags ident.Tags) bool + +// FsMergeWith is an interface that the fsMerger uses to merge data with. +type FsMergeWith interface { + // Read returns the data for the given block start and series ID, whether + // any data was found, and the error encountered (if any). + Read(blockStart xtime.UnixNano, seriesID ident.ID) ([]xio.BlockReader, bool, error) + + // ForEachRemaining is the loop for the second stage of merging. The + // fsMerger first loops through the fileset series, merging them with data + // in the merge target. The second stage is the go through the merge target + // data and write the remaining series that were not merged. + ForEachRemaining(blockStart xtime.UnixNano, fn forEachRemainingFn) error +} From af036ee16f02901aad741112109898318ac89101 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Sat, 18 May 2019 02:04:41 -0400 Subject: [PATCH 05/18] Rebase fixes --- src/dbnode/storage/database.go | 34 +----- src/dbnode/storage/fs_merge_with_mem.go | 4 +- src/dbnode/storage/fs_merger.go | 19 ++-- src/dbnode/storage/index_block_test.go | 6 +- src/dbnode/storage/series/buffer_mock.go | 41 ++++++- src/dbnode/storage/series/series_mock.go | 20 ++++ src/dbnode/storage/shard.go | 2 +- src/dbnode/storage/storage_mock.go | 136 +++++++++++++++++------ src/dbnode/storage/types.go | 6 +- 9 files changed, 182 insertions(+), 86 deletions(-) diff --git a/src/dbnode/storage/database.go b/src/dbnode/storage/database.go index 4e17d62f6b..34617d34c9 100644 --- a/src/dbnode/storage/database.go +++ b/src/dbnode/storage/database.go @@ -232,10 +232,10 @@ func (d *db) updateSchemaRegistry(newNamespaces namespace.Map) error { schemaUpdates := newNamespaces.Metadatas() merr := xerrors.NewMultiError() for _, metadata := range schemaUpdates { - curSchemaId := "none" + curSchemaID := "none" curSchema, err := schemaReg.GetLatestSchema(metadata.ID()) if curSchema != nil { - curSchemaId = curSchema.DeployId() + curSchemaID = curSchema.DeployId() } // Log schema update. latestSchema, found := metadata.Options().SchemaHistory().GetLatest() @@ -244,7 +244,7 @@ func (d *db) updateSchemaRegistry(newNamespaces namespace.Map) error { continue } else { d.log.Info("updating database namespace schema", zap.Stringer("namespace", metadata.ID()), - zap.String("current schema", curSchemaId), zap.String("latest schema", latestSchema.DeployId())) + zap.String("current schema", curSchemaID), zap.String("latest schema", latestSchema.DeployId())) } err = schemaReg.SetSchemaHistory(metadata.ID(), metadata.Options().SchemaHistory()) if err != nil { @@ -392,34 +392,6 @@ func (d *db) addNamespacesWithLock(namespaces []namespace.Metadata) error { return nil } -func (d *db) updateNamespaceSchemasWithLock(schemaUpdates []namespace.Metadata) error { - for _, n := range schemaUpdates { - // Ensure namespace exists. - curNamepsace, ok := d.namespaces.Get(n.ID()) - if !ok { - // Should never happen. - return fmt.Errorf("non-existent namespace marked for schema update: %v", n.ID().String()) - } - curSchemaID := "none" - curSchema, found := curNamepsace.SchemaRegistry().GetLatest() - if found { - curSchemaID = curSchema.DeployId() - } - // Log schema update. - latestSchema, found := n.Options().SchemaRegistry().GetLatest() - if !found { - return fmt.Errorf("can not update namespace (%s) schema from %s to empty", n.ID().String(), curSchemaID) - } - d.log.Info("updating database namespace schema", zap.Stringer("namespace", n.ID()), - zap.String("current schema", curSchemaID), zap.String("latest schema", latestSchema.DeployId())) - err := curNamepsace.SetSchemaRegistry(n.Options().SchemaRegistry()) - if err != nil { - return xerrors.Wrapf(err, "failed to update latest schema for namespace %s", n.ID().String()) - } - } - return nil -} - func (d *db) newDatabaseNamespaceWithLock( md namespace.Metadata, ) (databaseNamespace, error) { diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index 67406e7f95..7b3ee399ca 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -23,6 +23,7 @@ package storage import ( "errors" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" @@ -39,6 +40,7 @@ type fsMergeWithMem struct { func (m *fsMergeWithMem) Read( blockStart xtime.UnixNano, seriesID ident.ID, + nsCtx namespace.Context, ) ([]xio.BlockReader, bool, error) { // Check if this series is in memory (and thus requires merging). element, exists := m.dirtySeries.Get(idAndBlockStart{blockStart: blockStart, id: seriesID}) @@ -61,7 +63,7 @@ func (m *fsMergeWithMem) Read( nextVersion := m.shard.RetrievableBlockColdVersion(startTime) + 1 tmpCtx := context.NewContext() - blocks, err := entry.Series.FetchBlocksForColdFlush(tmpCtx, startTime, nextVersion) + blocks, err := entry.Series.FetchBlocksForColdFlush(tmpCtx, startTime, nextVersion, nsCtx) tmpCtx.BlockingClose() if err != nil { return nil, false, err diff --git a/src/dbnode/storage/fs_merger.go b/src/dbnode/storage/fs_merger.go index d112a291e7..1fc36860c5 100644 --- a/src/dbnode/storage/fs_merger.go +++ b/src/dbnode/storage/fs_merger.go @@ -26,10 +26,10 @@ import ( "github.com/m3db/m3/src/dbnode/digest" "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/storage/index/convert" - "github.com/m3db/m3/src/dbnode/storage/namespace" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" xerrors "github.com/m3db/m3/src/x/errors" @@ -50,12 +50,13 @@ type fsMerger struct { } func (m *fsMerger) Merge( + mergeWith FsMergeWith, ns namespace.Metadata, shard uint32, blockStart xtime.UnixNano, blockSize time.Duration, flushPreparer persist.FlushPreparer, - mergeWith FsMergeWith, + nsCtx namespace.Context, ) error { var multiErr xerrors.MultiError @@ -119,7 +120,7 @@ func (m *fsMerger) Merge( brs = append(brs, []xio.BlockReader{br}) // Check if this series is in memory (and thus requires merging). - encoded, hasData, err := mergeWith.Read(blockStart, id) + encoded, hasData, err := mergeWith.Read(blockStart, id, nsCtx) if err != nil { multiErr = multiErr.Add(err) return multiErr.FinalError() @@ -130,7 +131,7 @@ func (m *fsMerger) Merge( } mergedIter := multiIterPool.Get() - mergedIter.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(brs)) + mergedIter.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(brs), nsCtx.Schema) defer mergedIter.Close() tags, err := convert.TagsFromTagsIter(id, tagsIter, identPool) @@ -148,7 +149,7 @@ func (m *fsMerger) Merge( // Second stage: loop through rest of the merge target that was not captured // in the first stage. err = mergeWith.ForEachRemaining(blockStart, func(seriesID ident.ID, tags ident.Tags) bool { - encoded, hasData, err := mergeWith.Read(blockStart, seriesID) + encoded, hasData, err := mergeWith.Read(blockStart, seriesID, nsCtx) if err != nil { multiErr = multiErr.Add(err) return false @@ -156,7 +157,7 @@ func (m *fsMerger) Merge( if hasData { iter := multiIterPool.Get() - iter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize) + iter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize, nsCtx.Schema) defer iter.Close() if err := persistIter(prepared.Persist, iter, seriesID, tags, encoderPool); err != nil { multiErr = multiErr.Add(err) @@ -199,8 +200,12 @@ func persistIter( return err } - stream := encoder.Stream() + stream, ok := encoder.Stream(encoding.StreamOptions{}) encoder.Close() + if !ok { + // Don't write out series with no data. + return nil + } segment, err := stream.Segment() if err != nil { diff --git a/src/dbnode/storage/index_block_test.go b/src/dbnode/storage/index_block_test.go index 9720625e62..5a74972e4d 100644 --- a/src/dbnode/storage/index_block_test.go +++ b/src/dbnode/storage/index_block_test.go @@ -133,8 +133,7 @@ func TestNamespaceIndexNewBlockFn(t *testing.T) { require.NoError(t, index.Close()) }() - indexState := index.(*nsIndex).state - blocksSlice := indexState.blockStartsDescOrder + blocksSlice := index.(*nsIndex).state.blockStartsDescOrder require.Equal(t, 1, len(blocksSlice)) require.Equal(t, xtime.ToUnixNano(now.Truncate(blockSize)), blocksSlice[0]) @@ -948,7 +947,6 @@ func TestNamespaceIndexBlockAggregateQueryAggPath(t *testing.T) { // only queries as much as is needed (wrt to time) ctx := context.NewContext() - q := index.Query{Query: query} qOpts := index.QueryOptions{ StartInclusive: t0, EndExclusive: now.Add(time.Minute), @@ -956,7 +954,7 @@ func TestNamespaceIndexBlockAggregateQueryAggPath(t *testing.T) { aggOpts := index.AggregationOptions{QueryOptions: qOpts} for _, query := range queries { - q := index.Query{query} + q := index.Query{Query: query} b0.EXPECT().Aggregate(gomock.Any(), qOpts, gomock.Any()).Return(true, nil) _, err = idx.AggregateQuery(ctx, q, aggOpts) require.NoError(t, err) diff --git a/src/dbnode/storage/series/buffer_mock.go b/src/dbnode/storage/series/buffer_mock.go index 5c7513d21e..42b3894bfd 100644 --- a/src/dbnode/storage/series/buffer_mock.go +++ b/src/dbnode/storage/series/buffer_mock.go @@ -91,19 +91,19 @@ func (mr *MockdatabaseBufferMockRecorder) Snapshot(ctx, blockStart, id, tags, pe return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Snapshot", reflect.TypeOf((*MockdatabaseBuffer)(nil).Snapshot), ctx, blockStart, id, tags, persistFn, nsCtx) } -// Flush mocks base method -func (m *MockdatabaseBuffer) Flush(ctx context.Context, blockStart time.Time, id ident.ID, tags ident.Tags, persistFn persist.DataFn, version int, nsCtx namespace.Context) (FlushOutcome, error) { +// WarmFlush mocks base method +func (m *MockdatabaseBuffer) WarmFlush(ctx context.Context, blockStart time.Time, id ident.ID, tags ident.Tags, persistFn persist.DataFn, nsCtx namespace.Context) (FlushOutcome, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Flush", ctx, blockStart, id, tags, persistFn, version, nsCtx) + ret := m.ctrl.Call(m, "WarmFlush", ctx, blockStart, id, tags, persistFn, nsCtx) ret0, _ := ret[0].(FlushOutcome) ret1, _ := ret[1].(error) return ret0, ret1 } -// Flush indicates an expected call of Flush -func (mr *MockdatabaseBufferMockRecorder) Flush(ctx, blockStart, id, tags, persistFn, version, nsCtx interface{}) *gomock.Call { +// WarmFlush indicates an expected call of WarmFlush +func (mr *MockdatabaseBufferMockRecorder) WarmFlush(ctx, blockStart, id, tags, persistFn, nsCtx interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Flush", reflect.TypeOf((*MockdatabaseBuffer)(nil).Flush), ctx, blockStart, id, tags, persistFn, version, nsCtx) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WarmFlush", reflect.TypeOf((*MockdatabaseBuffer)(nil).WarmFlush), ctx, blockStart, id, tags, persistFn, nsCtx) } // ReadEncoded mocks base method @@ -121,6 +121,21 @@ func (mr *MockdatabaseBufferMockRecorder) ReadEncoded(ctx, start, end, nsCtx int return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadEncoded", reflect.TypeOf((*MockdatabaseBuffer)(nil).ReadEncoded), ctx, start, end, nsCtx) } +// FetchBlocksForColdFlush mocks base method +func (m *MockdatabaseBuffer) FetchBlocksForColdFlush(ctx context.Context, start time.Time, version int, nsCtx namespace.Context) ([]xio.BlockReader, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FetchBlocksForColdFlush", ctx, start, version, nsCtx) + ret0, _ := ret[0].([]xio.BlockReader) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// FetchBlocksForColdFlush indicates an expected call of FetchBlocksForColdFlush +func (mr *MockdatabaseBufferMockRecorder) FetchBlocksForColdFlush(ctx, start, version, nsCtx interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBlocksForColdFlush", reflect.TypeOf((*MockdatabaseBuffer)(nil).FetchBlocksForColdFlush), ctx, start, version, nsCtx) +} + // FetchBlocks mocks base method func (m *MockdatabaseBuffer) FetchBlocks(ctx context.Context, starts []time.Time, nsCtx namespace.Context) []block.FetchBlockResult { m.ctrl.T.Helper() @@ -164,6 +179,20 @@ func (mr *MockdatabaseBufferMockRecorder) IsEmpty() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsEmpty", reflect.TypeOf((*MockdatabaseBuffer)(nil).IsEmpty)) } +// NeedsColdFlushBlockStarts mocks base method +func (m *MockdatabaseBuffer) NeedsColdFlushBlockStarts() OptimizedTimes { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "NeedsColdFlushBlockStarts") + ret0, _ := ret[0].(OptimizedTimes) + return ret0 +} + +// NeedsColdFlushBlockStarts indicates an expected call of NeedsColdFlushBlockStarts +func (mr *MockdatabaseBufferMockRecorder) NeedsColdFlushBlockStarts() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NeedsColdFlushBlockStarts", reflect.TypeOf((*MockdatabaseBuffer)(nil).NeedsColdFlushBlockStarts)) +} + // Stats mocks base method func (m *MockdatabaseBuffer) Stats() bufferStats { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index ea20a3d9da..4649eb4dce 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -1,6 +1,26 @@ // Code generated by MockGen. DO NOT EDIT. // Source: github.com/m3db/m3/src/dbnode/storage/series (interfaces: DatabaseSeries,QueryableBlockRetriever) +// 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 series is a generated GoMock package. package series diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index b16d2fc1ac..cbb61cae5b 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1966,7 +1966,7 @@ func (s *dbShard) ColdFlush( // has its own fileset, if we encounter an error while trying to persist // a block, we continue to try persisting other blocks. for blockStart := range dirtySeriesToWrite { - err := merger.Merge(s.namespace, s.ID(), blockStart, blockSize, flushPreparer, mergeWithMem) + err := merger.Merge(mergeWithMem, s.namespace, s.ID(), blockStart, blockSize, flushPreparer, nsCtx) if err != nil { multiErr = multiErr.Add(err) continue diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 270b9cbe18..1062860eb9 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1255,18 +1255,18 @@ func (mr *MockdatabaseNamespaceMockRecorder) Bootstrap(start, process interface{ return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseNamespace)(nil).Bootstrap), start, process) } -// Flush mocks base method -func (m *MockdatabaseNamespace) Flush(blockStart time.Time, ShardBootstrapStates ShardBootstrapStates, flush persist.FlushPreparer) error { +// WarmFlush mocks base method +func (m *MockdatabaseNamespace) WarmFlush(blockStart time.Time, ShardBootstrapStates ShardBootstrapStates, flush persist.FlushPreparer) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Flush", blockStart, ShardBootstrapStates, flush) + ret := m.ctrl.Call(m, "WarmFlush", blockStart, ShardBootstrapStates, flush) ret0, _ := ret[0].(error) return ret0 } -// Flush indicates an expected call of Flush -func (mr *MockdatabaseNamespaceMockRecorder) Flush(blockStart, ShardBootstrapStates, flush interface{}) *gomock.Call { +// WarmFlush indicates an expected call of WarmFlush +func (mr *MockdatabaseNamespaceMockRecorder) WarmFlush(blockStart, ShardBootstrapStates, flush interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Flush", reflect.TypeOf((*MockdatabaseNamespace)(nil).Flush), blockStart, ShardBootstrapStates, flush) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WarmFlush", reflect.TypeOf((*MockdatabaseNamespace)(nil).WarmFlush), blockStart, ShardBootstrapStates, flush) } // FlushIndex mocks base method @@ -1283,6 +1283,20 @@ func (mr *MockdatabaseNamespaceMockRecorder) FlushIndex(flush interface{}) *gomo return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FlushIndex", reflect.TypeOf((*MockdatabaseNamespace)(nil).FlushIndex), flush) } +// ColdFlush mocks base method +func (m *MockdatabaseNamespace) ColdFlush(flush persist.FlushPreparer) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ColdFlush", flush) + ret0, _ := ret[0].(error) + return ret0 +} + +// ColdFlush indicates an expected call of ColdFlush +func (mr *MockdatabaseNamespaceMockRecorder) ColdFlush(flush interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ColdFlush", reflect.TypeOf((*MockdatabaseNamespace)(nil).ColdFlush), flush) +} + // Snapshot mocks base method func (m *MockdatabaseNamespace) Snapshot(blockStart, snapshotTime time.Time, flush persist.SnapshotPreparer) error { m.ctrl.T.Helper() @@ -1660,18 +1674,32 @@ func (mr *MockdatabaseShardMockRecorder) Bootstrap(bootstrappedSeries interface{ return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseShard)(nil).Bootstrap), bootstrappedSeries) } -// Flush mocks base method -func (m *MockdatabaseShard) Flush(blockStart time.Time, flush persist.FlushPreparer, nsCtx namespace.Context) error { +// WarmFlush mocks base method +func (m *MockdatabaseShard) WarmFlush(blockStart time.Time, flush persist.FlushPreparer, nsCtx namespace.Context) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Flush", blockStart, flush, nsCtx) + ret := m.ctrl.Call(m, "WarmFlush", blockStart, flush, nsCtx) ret0, _ := ret[0].(error) return ret0 } -// Flush indicates an expected call of Flush -func (mr *MockdatabaseShardMockRecorder) Flush(blockStart, flush, nsCtx interface{}) *gomock.Call { +// WarmFlush indicates an expected call of WarmFlush +func (mr *MockdatabaseShardMockRecorder) WarmFlush(blockStart, flush, nsCtx interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Flush", reflect.TypeOf((*MockdatabaseShard)(nil).Flush), blockStart, flush, nsCtx) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WarmFlush", reflect.TypeOf((*MockdatabaseShard)(nil).WarmFlush), blockStart, flush, nsCtx) +} + +// ColdFlush mocks base method +func (m *MockdatabaseShard) ColdFlush(flush persist.FlushPreparer, resources coldFlushReuseableResources, nsCtx namespace.Context) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ColdFlush", flush, resources, nsCtx) + ret0, _ := ret[0].(error) + return ret0 +} + +// ColdFlush indicates an expected call of ColdFlush +func (mr *MockdatabaseShardMockRecorder) ColdFlush(flush, resources, nsCtx interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ColdFlush", reflect.TypeOf((*MockdatabaseShard)(nil).ColdFlush), flush, resources, nsCtx) } // Snapshot mocks base method @@ -1702,21 +1730,6 @@ func (mr *MockdatabaseShardMockRecorder) FlushState(blockStart interface{}) *gom return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FlushState", reflect.TypeOf((*MockdatabaseShard)(nil).FlushState), blockStart) } -// SnapshotState mocks base method -func (m *MockdatabaseShard) SnapshotState() (bool, time.Time) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SnapshotState") - ret0, _ := ret[0].(bool) - ret1, _ := ret[1].(time.Time) - return ret0, ret1 -} - -// SnapshotState indicates an expected call of SnapshotState -func (mr *MockdatabaseShardMockRecorder) SnapshotState() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SnapshotState", reflect.TypeOf((*MockdatabaseShard)(nil).SnapshotState)) -} - // CleanupExpiredFileSets mocks base method func (m *MockdatabaseShard) CleanupExpiredFileSets(earliestToRetain time.Time) error { m.ctrl.T.Helper() @@ -1732,18 +1745,18 @@ func (mr *MockdatabaseShardMockRecorder) CleanupExpiredFileSets(earliestToRetain } // Repair mocks base method -func (m *MockdatabaseShard) Repair(ctx context.Context, nsCtx namespace.Context, tr time0.Range, repairer databaseShardRepairer) (repair.MetadataComparisonResult, error) { +func (m *MockdatabaseShard) Repair(ctx context.Context, tr time0.Range, repairer databaseShardRepairer) (repair.MetadataComparisonResult, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Repair", ctx, nsCtx, tr, repairer) + ret := m.ctrl.Call(m, "Repair", ctx, tr, repairer) ret0, _ := ret[0].(repair.MetadataComparisonResult) ret1, _ := ret[1].(error) return ret0, ret1 } // Repair indicates an expected call of Repair -func (mr *MockdatabaseShardMockRecorder) Repair(ctx, nsCtx, tr, repairer interface{}) *gomock.Call { +func (mr *MockdatabaseShardMockRecorder) Repair(ctx, tr, repairer interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShard)(nil).Repair), ctx, nsCtx, tr, repairer) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShard)(nil).Repair), ctx, tr, repairer) } // MocknamespaceIndex is a mock of namespaceIndex interface @@ -2341,18 +2354,18 @@ func (mr *MockdatabaseShardRepairerMockRecorder) Options() *gomock.Call { } // Repair mocks base method -func (m *MockdatabaseShardRepairer) Repair(ctx context.Context, nsCtx namespace.Context, tr time0.Range, shard databaseShard) (repair.MetadataComparisonResult, error) { +func (m *MockdatabaseShardRepairer) Repair(ctx context.Context, namespace ident.ID, tr time0.Range, shard databaseShard) (repair.MetadataComparisonResult, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Repair", ctx, nsCtx, tr, shard) + ret := m.ctrl.Call(m, "Repair", ctx, namespace, tr, shard) ret0, _ := ret[0].(repair.MetadataComparisonResult) ret1, _ := ret[1].(error) return ret0, ret1 } // Repair indicates an expected call of Repair -func (mr *MockdatabaseShardRepairerMockRecorder) Repair(ctx, nsCtx, tr, shard interface{}) *gomock.Call { +func (mr *MockdatabaseShardRepairerMockRecorder) Repair(ctx, namespace, tr, shard interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShardRepairer)(nil).Repair), ctx, nsCtx, tr, shard) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShardRepairer)(nil).Repair), ctx, namespace, tr, shard) } // MockdatabaseRepairer is a mock of databaseRepairer interface @@ -3677,3 +3690,56 @@ func (mr *MockOptionsMockRecorder) SchemaRegistry() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SchemaRegistry", reflect.TypeOf((*MockOptions)(nil).SchemaRegistry)) } + +// MockFsMergeWith is a mock of FsMergeWith interface +type MockFsMergeWith struct { + ctrl *gomock.Controller + recorder *MockFsMergeWithMockRecorder +} + +// MockFsMergeWithMockRecorder is the mock recorder for MockFsMergeWith +type MockFsMergeWithMockRecorder struct { + mock *MockFsMergeWith +} + +// NewMockFsMergeWith creates a new mock instance +func NewMockFsMergeWith(ctrl *gomock.Controller) *MockFsMergeWith { + mock := &MockFsMergeWith{ctrl: ctrl} + mock.recorder = &MockFsMergeWithMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockFsMergeWith) EXPECT() *MockFsMergeWithMockRecorder { + return m.recorder +} + +// Read mocks base method +func (m *MockFsMergeWith) Read(blockStart time0.UnixNano, seriesID ident.ID, nsCtx namespace.Context) ([]xio.BlockReader, bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Read", blockStart, seriesID, nsCtx) + ret0, _ := ret[0].([]xio.BlockReader) + ret1, _ := ret[1].(bool) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// Read indicates an expected call of Read +func (mr *MockFsMergeWithMockRecorder) Read(blockStart, seriesID, nsCtx interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Read", reflect.TypeOf((*MockFsMergeWith)(nil).Read), blockStart, seriesID, nsCtx) +} + +// ForEachRemaining mocks base method +func (m *MockFsMergeWith) ForEachRemaining(blockStart time0.UnixNano, fn forEachRemainingFn) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ForEachRemaining", blockStart, fn) + ret0, _ := ret[0].(error) + return ret0 +} + +// ForEachRemaining indicates an expected call of ForEachRemaining +func (mr *MockFsMergeWithMockRecorder) ForEachRemaining(blockStart, fn interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ForEachRemaining", reflect.TypeOf((*MockFsMergeWith)(nil).ForEachRemaining), blockStart, fn) +} diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 2952a0b02a..6303346a2b 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -987,7 +987,11 @@ type forEachRemainingFn func(seriesID ident.ID, tags ident.Tags) bool type FsMergeWith interface { // Read returns the data for the given block start and series ID, whether // any data was found, and the error encountered (if any). - Read(blockStart xtime.UnixNano, seriesID ident.ID) ([]xio.BlockReader, bool, error) + Read( + blockStart xtime.UnixNano, + seriesID ident.ID, + nsCtx namespace.Context, + ) ([]xio.BlockReader, bool, error) // ForEachRemaining is the loop for the second stage of merging. The // fsMerger first loops through the fileset series, merging them with data From e0383dec924eec3b4c41805accc06b1e4054841c Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Wed, 22 May 2019 10:39:23 -0400 Subject: [PATCH 06/18] Change interfaces to make code more testable; add tests --- .../fs_merger.go => persist/fs/merger.go} | 77 +++--- src/dbnode/persist/fs/types.go | 23 +- src/dbnode/storage/fs_merge_with_mem.go | 41 ++-- src/dbnode/storage/fs_merge_with_mem_test.go | 225 ++++++++++++++++++ src/dbnode/storage/namespace.go | 1 + src/dbnode/storage/shard.go | 44 +++- src/dbnode/storage/storage_mock.go | 83 +++---- src/dbnode/storage/types.go | 33 ++- 8 files changed, 397 insertions(+), 130 deletions(-) rename src/dbnode/{storage/fs_merger.go => persist/fs/merger.go} (79%) create mode 100644 src/dbnode/storage/fs_merge_with_mem_test.go diff --git a/src/dbnode/storage/fs_merger.go b/src/dbnode/persist/fs/merger.go similarity index 79% rename from src/dbnode/storage/fs_merger.go rename to src/dbnode/persist/fs/merger.go index 1fc36860c5..8dd008a542 100644 --- a/src/dbnode/storage/fs_merger.go +++ b/src/dbnode/persist/fs/merger.go @@ -18,7 +18,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package storage +package fs import ( "io" @@ -28,29 +28,41 @@ import ( "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" - "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/storage/index/convert" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" + "github.com/m3db/m3/src/x/checked" xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" ) -type fsMergerReusableResources struct { - fsReader fs.DataFileSetReader +type Merger struct { + reader DataFileSetReader srPool xio.SegmentReaderPool multiIterPool encoding.MultiReaderIteratorPool identPool ident.Pool encoderPool encoding.EncoderPool } -type fsMerger struct { - res fsMergerReusableResources +func NewMerger( + reader DataFileSetReader, + srPool xio.SegmentReaderPool, + multiIterPool encoding.MultiReaderIteratorPool, + identPool ident.Pool, + encoderPool encoding.EncoderPool, +) *Merger { + return &Merger{ + reader: reader, + srPool: srPool, + multiIterPool: multiIterPool, + identPool: identPool, + encoderPool: encoderPool, + } } -func (m *fsMerger) Merge( - mergeWith FsMergeWith, +func (m *Merger) Merge( + mergeWith MergeWith, ns namespace.Metadata, shard uint32, blockStart xtime.UnixNano, @@ -60,25 +72,25 @@ func (m *fsMerger) Merge( ) error { var multiErr xerrors.MultiError - fsReader := m.res.fsReader - srPool := m.res.srPool - multiIterPool := m.res.multiIterPool - identPool := m.res.identPool - encoderPool := m.res.encoderPool + reader := m.reader + srPool := m.srPool + multiIterPool := m.multiIterPool + identPool := m.identPool + encoderPool := m.encoderPool startTime := blockStart.ToTime() - openOpts := fs.DataReaderOpenOptions{ - Identifier: fs.FileSetFileIdentifier{ + openOpts := DataReaderOpenOptions{ + Identifier: FileSetFileIdentifier{ Namespace: ns.ID(), Shard: shard, BlockStart: startTime, }, } - if err := fsReader.Open(openOpts); err != nil { + if err := reader.Open(openOpts); err != nil { multiErr = multiErr.Add(err) return multiErr.FinalError() } - defer fsReader.Close() + defer reader.Close() prepareOpts := persist.DataPrepareOptions{ NamespaceMetadata: ns, @@ -98,7 +110,7 @@ func (m *fsMerger) Merge( // merge target that was not persisted in the first stage. // First stage: loop through series on disk. - for id, tagsIter, data, _, err := fsReader.Read(); err != io.EOF; id, tagsIter, data, _, err = fsReader.Read() { + for id, tagsIter, data, _, err := reader.Read(); err != io.EOF; id, tagsIter, data, _, err = reader.Read() { if err != nil { multiErr = multiErr.Add(err) return multiErr.FinalError() @@ -109,18 +121,11 @@ func (m *fsMerger) Merge( brs := make([][]xio.BlockReader, 0, 2) // Create BlockReader slice out of disk data. - seg := ts.NewSegment(data, nil, ts.FinalizeHead) - sr := srPool.Get() - sr.Reset(seg) - br := xio.BlockReader{ - SegmentReader: sr, - Start: startTime, - BlockSize: blockSize, - } + br := blockReaderFromData(data, srPool, startTime, blockSize) brs = append(brs, []xio.BlockReader{br}) // Check if this series is in memory (and thus requires merging). - encoded, hasData, err := mergeWith.Read(blockStart, id, nsCtx) + encoded, hasData, err := mergeWith.Read(id, blockStart, nsCtx) if err != nil { multiErr = multiErr.Add(err) return multiErr.FinalError() @@ -149,7 +154,7 @@ func (m *fsMerger) Merge( // Second stage: loop through rest of the merge target that was not captured // in the first stage. err = mergeWith.ForEachRemaining(blockStart, func(seriesID ident.ID, tags ident.Tags) bool { - encoded, hasData, err := mergeWith.Read(blockStart, seriesID, nsCtx) + encoded, hasData, err := mergeWith.Read(seriesID, blockStart, nsCtx) if err != nil { multiErr = multiErr.Add(err) return false @@ -182,6 +187,22 @@ func (m *fsMerger) Merge( return multiErr.FinalError() } +func blockReaderFromData( + data checked.Bytes, + srPool xio.SegmentReaderPool, + startTime time.Time, + blockSize time.Duration, +) xio.BlockReader { + seg := ts.NewSegment(data, nil, ts.FinalizeHead) + sr := srPool.Get() + sr.Reset(seg) + return xio.BlockReader{ + SegmentReader: sr, + Start: startTime, + BlockSize: blockSize, + } +} + func persistIter( persistFn persist.DataFn, it encoding.Iterator, diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index cf0bc03069..07adbf8928 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -26,19 +26,19 @@ import ( "time" "github.com/m3db/m3/src/dbnode/clock" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/block" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/m3ninx/index/segment/fst" idxpersist "github.com/m3db/m3/src/m3ninx/persist" - "github.com/m3db/m3/src/x/serialize" "github.com/m3db/m3/src/x/checked" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/instrument" "github.com/m3db/m3/src/x/pool" + "github.com/m3db/m3/src/x/serialize" xtime "github.com/m3db/m3/src/x/time" ) @@ -489,3 +489,22 @@ type BlockRetrieverOptions interface { // IdentifierPool returns the identifierPool IdentifierPool() ident.Pool } + +type ForEachRemainingFn func(seriesID ident.ID, tags ident.Tags) bool + +// MergeWith is an interface that the fs merger uses to merge data with. +type MergeWith interface { + // Read returns the data for the given block start and series ID, whether + // any data was found, and the error encountered (if any). + Read( + seriesID ident.ID, + blockStart xtime.UnixNano, + nsCtx namespace.Context, + ) ([]xio.BlockReader, bool, error) + + // ForEachRemaining is the loop for the second stage of merging. The + // fsMerger first loops through the fileset series, merging them with data + // in the merge target. The second stage is the go through the merge target + // data and write the remaining series that were not merged. + ForEachRemaining(blockStart xtime.UnixNano, fn ForEachRemainingFn) error +} diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index 7b3ee399ca..7ca1f1be8d 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -24,22 +24,39 @@ import ( "errors" "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/persist/fs" + "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" ) -// fsMergeWithMem implements FsMergeWith +// fsMergeWithMem implements fs.MergeWith type fsMergeWithMem struct { - shard *dbShard + shard databaseShard + retriever series.QueryableBlockRetriever dirtySeries *dirtySeriesMap dirtySeriesToWrite map[xtime.UnixNano]*idList } +func newFSMergeWithMem( + shard databaseShard, + retriever series.QueryableBlockRetriever, + dirtySeries *dirtySeriesMap, + dirtySeriesToWrite map[xtime.UnixNano]*idList, +) *fsMergeWithMem { + return &fsMergeWithMem{ + shard: shard, + retriever: retriever, + dirtySeries: dirtySeries, + dirtySeriesToWrite: dirtySeriesToWrite, + } +} + func (m *fsMergeWithMem) Read( - blockStart xtime.UnixNano, seriesID ident.ID, + blockStart xtime.UnixNano, nsCtx namespace.Context, ) ([]xio.BlockReader, bool, error) { // Check if this series is in memory (and thus requires merging). @@ -56,14 +73,10 @@ func (m *fsMergeWithMem) Read( // it. m.dirtySeriesToWrite[blockStart].Remove(element) - m.shard.RLock() - entry, _, err := m.shard.lookupEntryWithLock(element.Value) - m.shard.RUnlock() - - nextVersion := m.shard.RetrievableBlockColdVersion(startTime) + 1 + nextVersion := m.retriever.RetrievableBlockColdVersion(startTime) + 1 tmpCtx := context.NewContext() - blocks, err := entry.Series.FetchBlocksForColdFlush(tmpCtx, startTime, nextVersion, nsCtx) + blocks, err := m.shard.FetchBlocksForColdFlush(tmpCtx, element.Value, startTime, nextVersion, nsCtx) tmpCtx.BlockingClose() if err != nil { return nil, false, err @@ -76,19 +89,17 @@ func (m *fsMergeWithMem) Read( return nil, false, nil } -func (m *fsMergeWithMem) ForEachRemaining(blockStart xtime.UnixNano, fn forEachRemainingFn) error { +func (m *fsMergeWithMem) ForEachRemaining(blockStart xtime.UnixNano, fn fs.ForEachRemainingFn) error { seriesList := m.dirtySeriesToWrite[blockStart] for seriesElement := seriesList.Front(); seriesElement != nil; seriesElement = seriesElement.Next() { - m.shard.RLock() - entry, _, err := m.shard.lookupEntryWithLock(seriesElement.Value) - m.shard.RUnlock() + seriesID := seriesElement.Value + tags, err := m.shard.TagsFromSeriesID(seriesID) if err != nil { return err } - series := entry.Series - success := fn(series.ID(), series.Tags()) + success := fn(seriesID, tags) if !success { return errors.New("foreach iteration unsuccessful") } diff --git a/src/dbnode/storage/fs_merge_with_mem_test.go b/src/dbnode/storage/fs_merge_with_mem_test.go new file mode 100644 index 0000000000..f425d6d145 --- /dev/null +++ b/src/dbnode/storage/fs_merge_with_mem_test.go @@ -0,0 +1,225 @@ +// 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 storage + +import ( + "errors" + "testing" + + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/storage/series" + "github.com/m3db/m3/src/dbnode/x/xio" + "github.com/m3db/m3/src/x/ident" + xtime "github.com/m3db/m3/src/x/time" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +type dirtyData struct { + id ident.ID + start xtime.UnixNano +} + +func TestRead(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + shard := NewMockdatabaseShard(ctrl) + retriever := series.NewMockQueryableBlockRetriever(ctrl) + version := 0 + nsCtx := namespace.Context{} + fetchedBlocks := []xio.BlockReader{xio.BlockReader{}} + retriever.EXPECT().RetrievableBlockColdVersion(gomock.Any()).Return(version).AnyTimes() + + dirtySeries := newDirtySeriesMap(dirtySeriesMapOptions{}) + dirtySeriesToWrite := make(map[xtime.UnixNano]*idList) + + data := []dirtyData{ + dirtyData{start: 0, id: ident.StringID("id0")}, + dirtyData{start: 0, id: ident.StringID("id1")}, + dirtyData{start: 1, id: ident.StringID("id2")}, + dirtyData{start: 1, id: ident.StringID("id3")}, + dirtyData{start: 1, id: ident.StringID("id4")}, + dirtyData{start: 2, id: ident.StringID("id5")}, + dirtyData{start: 3, id: ident.StringID("id6")}, + dirtyData{start: 3, id: ident.StringID("id7")}, + dirtyData{start: 4, id: ident.StringID("id8")}, + } + + // Populate bookkeeping data structures with above test data. + for _, d := range data { + addDirtySeries(dirtySeries, dirtySeriesToWrite, d.id, d.start) + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), d.id, d.start.ToTime(), version+1, nsCtx). + Return(fetchedBlocks, nil) + } + + mergeWith := newFSMergeWithMem(shard, retriever, dirtySeries, dirtySeriesToWrite) + + for _, d := range data { + require.True(t, dirtySeries.Contains(idAndBlockStart{blockStart: d.start, id: d.id})) + beforeLen := dirtySeriesToWrite[d.start].Len() + res, exists, err := mergeWith.Read(d.id, d.start, nsCtx) + require.NoError(t, err) + assert.True(t, exists) + assert.Equal(t, fetchedBlocks, res) + // Assert that the Read call removes the element from the "to write" + // list. + assert.Equal(t, beforeLen-1, dirtySeriesToWrite[d.start].Len()) + } + + // Test Read with non-existent dirty block/series. + res, exists, err := mergeWith.Read(ident.StringID("not-present"), 10, nsCtx) + assert.Nil(t, res) + assert.False(t, exists) + assert.NoError(t, err) + + // Test Read with error on fetch. + badFetchID := ident.StringID("bad-fetch") + addDirtySeries(dirtySeries, dirtySeriesToWrite, badFetchID, 11) + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), badFetchID, gomock.Any(), version+1, nsCtx). + Return(nil, errors.New("fetch error")) + res, exists, err = mergeWith.Read(badFetchID, 11, nsCtx) + assert.Nil(t, res) + assert.False(t, exists) + assert.Error(t, err) + + // Test Read with no data on fetch. + emptyDataID := ident.StringID("empty-data") + addDirtySeries(dirtySeries, dirtySeriesToWrite, emptyDataID, 12) + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), emptyDataID, gomock.Any(), version+1, nsCtx). + Return(nil, nil) + res, exists, err = mergeWith.Read(emptyDataID, 12, nsCtx) + assert.Nil(t, res) + assert.False(t, exists) + assert.NoError(t, err) +} + +func TestForEachRemaining(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + shard := NewMockdatabaseShard(ctrl) + retriever := series.NewMockQueryableBlockRetriever(ctrl) + version := 0 + nsCtx := namespace.Context{} + fetchedBlocks := []xio.BlockReader{xio.BlockReader{}} + retriever.EXPECT().RetrievableBlockColdVersion(gomock.Any()).Return(version).AnyTimes() + + dirtySeries := newDirtySeriesMap(dirtySeriesMapOptions{}) + dirtySeriesToWrite := make(map[xtime.UnixNano]*idList) + + id0 := ident.StringID("id0") + id1 := ident.StringID("id1") + id2 := ident.StringID("id2") + id3 := ident.StringID("id3") + id4 := ident.StringID("id4") + id5 := ident.StringID("id5") + id6 := ident.StringID("id6") + id7 := ident.StringID("id7") + id8 := ident.StringID("id8") + data := []dirtyData{ + dirtyData{start: 0, id: id0}, + dirtyData{start: 0, id: id1}, + dirtyData{start: 1, id: id2}, + dirtyData{start: 1, id: id3}, + dirtyData{start: 1, id: id4}, + dirtyData{start: 2, id: id5}, + dirtyData{start: 3, id: id6}, + dirtyData{start: 3, id: id7}, + dirtyData{start: 4, id: id8}, + } + + // Populate bookkeeping data structures with above test data. + for _, d := range data { + addDirtySeries(dirtySeries, dirtySeriesToWrite, d.id, d.start) + } + + mergeWith := newFSMergeWithMem(shard, retriever, dirtySeries, dirtySeriesToWrite) + + var forEachCalls []ident.ID + shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) + mergeWith.ForEachRemaining(0, func(seriesID ident.ID, tags ident.Tags) bool { + forEachCalls = append(forEachCalls, seriesID) + return true + }) + require.Len(t, forEachCalls, 2) + assert.Equal(t, id0, forEachCalls[0]) + assert.Equal(t, id1, forEachCalls[1]) + + // Reset expected calls. + forEachCalls = forEachCalls[:0] + // Read id3 at block start 1, so id2 and id4 should be remaining for block + // start 1. + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), id3, xtime.UnixNano(1).ToTime(), version+1, nsCtx). + Return(fetchedBlocks, nil) + res, exists, err := mergeWith.Read(id3, 1, nsCtx) + require.NoError(t, err) + assert.True(t, exists) + assert.Equal(t, fetchedBlocks, res) + shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) + err = mergeWith.ForEachRemaining(1, func(seriesID ident.ID, tags ident.Tags) bool { + forEachCalls = append(forEachCalls, seriesID) + return true + }) + require.NoError(t, err) + require.Len(t, forEachCalls, 2) + assert.Equal(t, id2, forEachCalls[0]) + assert.Equal(t, id4, forEachCalls[1]) + + // Test call with error getting tags. + shard.EXPECT(). + TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, errors.New("bad-tags")) + err = mergeWith.ForEachRemaining(4, func(seriesID ident.ID, tags ident.Tags) bool { + // This function won't be called with the above error. + return true + }) + assert.Error(t, err) + + // Test call with bad function execution. + shard.EXPECT(). + TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil) + err = mergeWith.ForEachRemaining(4, func(seriesID ident.ID, tags ident.Tags) bool { + return false + }) + assert.Error(t, err) +} + +func addDirtySeries( + dirtySeries *dirtySeriesMap, + dirtySeriesToWrite map[xtime.UnixNano]*idList, + id ident.ID, + start xtime.UnixNano, +) { + seriesList := dirtySeriesToWrite[start] + if seriesList == nil { + seriesList = newIDList(nil) + dirtySeriesToWrite[start] = seriesList + } + element := seriesList.PushBack(id) + + dirtySeries.Set(idAndBlockStart{blockStart: start, id: id}, element) +} diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index f53b622f2d..4280fea267 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1035,6 +1035,7 @@ func (n *dbNamespace) ColdFlush( return err } resources := coldFlushReuseableResources{ + // TODO(juchan): consider setting these options. dirtySeries: newDirtySeriesMap(dirtySeriesMapOptions{}), dirtySeriesToWrite: make(map[xtime.UnixNano]*idList), // TODO(juchan): set pool options. diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index cbb61cae5b..5da3cfd6a7 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1446,6 +1446,23 @@ func (s *dbShard) FetchBlocks( return reader.FetchBlocks(ctx, starts, nsCtx) } +func (s *dbShard) FetchBlocksForColdFlush( + ctx context.Context, + seriesID ident.ID, + start time.Time, + version int, + nsCtx namespace.Context, +) ([]xio.BlockReader, error) { + s.RLock() + entry, _, err := s.lookupEntryWithLock(seriesID) + s.RUnlock() + if err != nil { + return nil, err + } + + return entry.Series.FetchBlocksForColdFlush(ctx, start, version, nsCtx) +} + func (s *dbShard) fetchActiveBlocksMetadata( ctx context.Context, start, end time.Time, @@ -1949,19 +1966,9 @@ func (s *dbShard) ColdFlush( return true }) - mergerResources := fsMergerReusableResources{ - fsReader: resources.fsReader, - srPool: s.opts.SegmentReaderPool(), - multiIterPool: s.opts.MultiReaderIteratorPool(), - identPool: s.opts.IdentifierPool(), - encoderPool: s.opts.EncoderPool(), - } - merger := &fsMerger{res: mergerResources} - mergeWithMem := &fsMergeWithMem{ - shard: s, - dirtySeries: dirtySeries, - dirtySeriesToWrite: dirtySeriesToWrite, - } + merger := fs.NewMerger(resources.fsReader, s.opts.SegmentReaderPool(), + s.opts.MultiReaderIteratorPool(), s.opts.IdentifierPool(), s.opts.EncoderPool()) + mergeWithMem := newFSMergeWithMem(s, s, dirtySeries, dirtySeriesToWrite) // Loop through each block that we know has ColdWrites. Since each block // has its own fileset, if we encounter an error while trying to persist // a block, we continue to try persisting other blocks. @@ -2136,6 +2143,17 @@ func (s *dbShard) Repair( return repairer.Repair(ctx, nsCtx, tr, s) } +func (s *dbShard) TagsFromSeriesID(seriesID ident.ID) (ident.Tags, error) { + s.RLock() + entry, _, err := s.lookupEntryWithLock(seriesID) + s.RUnlock() + if err != nil { + return ident.Tags{}, err + } + + return entry.Series.Tags(), nil +} + func (s *dbShard) BootstrapState() BootstrapState { s.RLock() bs := s.bootstrapState diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 1062860eb9..1c20e73107 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1644,6 +1644,21 @@ func (mr *MockdatabaseShardMockRecorder) FetchBlocks(ctx, id, starts, nsCtx inte return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBlocks", reflect.TypeOf((*MockdatabaseShard)(nil).FetchBlocks), ctx, id, starts, nsCtx) } +// FetchBlocksForColdFlush mocks base method +func (m *MockdatabaseShard) FetchBlocksForColdFlush(ctx context.Context, seriesID ident.ID, start time.Time, version int, nsCtx namespace.Context) ([]xio.BlockReader, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FetchBlocksForColdFlush", ctx, seriesID, start, version, nsCtx) + ret0, _ := ret[0].([]xio.BlockReader) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// FetchBlocksForColdFlush indicates an expected call of FetchBlocksForColdFlush +func (mr *MockdatabaseShardMockRecorder) FetchBlocksForColdFlush(ctx, seriesID, start, version, nsCtx interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBlocksForColdFlush", reflect.TypeOf((*MockdatabaseShard)(nil).FetchBlocksForColdFlush), ctx, seriesID, start, version, nsCtx) +} + // FetchBlocksMetadataV2 mocks base method func (m *MockdatabaseShard) FetchBlocksMetadataV2(ctx context.Context, start, end time.Time, limit int64, pageToken PageToken, opts block.FetchBlocksMetadataOptions) (block.FetchBlocksMetadataResults, PageToken, error) { m.ctrl.T.Helper() @@ -1759,6 +1774,21 @@ func (mr *MockdatabaseShardMockRecorder) Repair(ctx, tr, repairer interface{}) * return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShard)(nil).Repair), ctx, tr, repairer) } +// TagsFromSeriesID mocks base method +func (m *MockdatabaseShard) TagsFromSeriesID(seriesID ident.ID) (ident.Tags, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TagsFromSeriesID", seriesID) + ret0, _ := ret[0].(ident.Tags) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// TagsFromSeriesID indicates an expected call of TagsFromSeriesID +func (mr *MockdatabaseShardMockRecorder) TagsFromSeriesID(seriesID interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TagsFromSeriesID", reflect.TypeOf((*MockdatabaseShard)(nil).TagsFromSeriesID), seriesID) +} + // MocknamespaceIndex is a mock of namespaceIndex interface type MocknamespaceIndex struct { ctrl *gomock.Controller @@ -3690,56 +3720,3 @@ func (mr *MockOptionsMockRecorder) SchemaRegistry() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SchemaRegistry", reflect.TypeOf((*MockOptions)(nil).SchemaRegistry)) } - -// MockFsMergeWith is a mock of FsMergeWith interface -type MockFsMergeWith struct { - ctrl *gomock.Controller - recorder *MockFsMergeWithMockRecorder -} - -// MockFsMergeWithMockRecorder is the mock recorder for MockFsMergeWith -type MockFsMergeWithMockRecorder struct { - mock *MockFsMergeWith -} - -// NewMockFsMergeWith creates a new mock instance -func NewMockFsMergeWith(ctrl *gomock.Controller) *MockFsMergeWith { - mock := &MockFsMergeWith{ctrl: ctrl} - mock.recorder = &MockFsMergeWithMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockFsMergeWith) EXPECT() *MockFsMergeWithMockRecorder { - return m.recorder -} - -// Read mocks base method -func (m *MockFsMergeWith) Read(blockStart time0.UnixNano, seriesID ident.ID, nsCtx namespace.Context) ([]xio.BlockReader, bool, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Read", blockStart, seriesID, nsCtx) - ret0, _ := ret[0].([]xio.BlockReader) - ret1, _ := ret[1].(bool) - ret2, _ := ret[2].(error) - return ret0, ret1, ret2 -} - -// Read indicates an expected call of Read -func (mr *MockFsMergeWithMockRecorder) Read(blockStart, seriesID, nsCtx interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Read", reflect.TypeOf((*MockFsMergeWith)(nil).Read), blockStart, seriesID, nsCtx) -} - -// ForEachRemaining mocks base method -func (m *MockFsMergeWith) ForEachRemaining(blockStart time0.UnixNano, fn forEachRemainingFn) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ForEachRemaining", blockStart, fn) - ret0, _ := ret[0].(error) - return ret0 -} - -// ForEachRemaining indicates an expected call of ForEachRemaining -func (mr *MockFsMergeWithMockRecorder) ForEachRemaining(blockStart, fn interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ForEachRemaining", reflect.TypeOf((*MockFsMergeWith)(nil).ForEachRemaining), blockStart, fn) -} diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 6303346a2b..7082e48bb4 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -444,6 +444,17 @@ type databaseShard interface { nsCtx namespace.Context, ) ([]block.FetchBlockResult, error) + // FetchBlocksForColdFlush fetches blocks for a cold flush. This function + // informs the series and the buffer that a cold flush for the specified + // block start is occurring so that it knows to update bucket versions. + FetchBlocksForColdFlush( + ctx context.Context, + seriesID ident.ID, + start time.Time, + version int, + nsCtx namespace.Context, + ) ([]xio.BlockReader, error) + // FetchBlocksMetadataV2 retrieves blocks metadata. FetchBlocksMetadataV2( ctx context.Context, @@ -493,6 +504,9 @@ type databaseShard interface { tr xtime.Range, repairer databaseShardRepairer, ) (repair.MetadataComparisonResult, error) + + // TagsFromSeriesID returns the series tags from a series ID. + TagsFromSeriesID(seriesID ident.ID) (ident.Tags, error) } // namespaceIndex indexes namespace writes. @@ -980,22 +994,3 @@ const ( // Bootstrapped indicates a bootstrap process has completed. Bootstrapped ) - -type forEachRemainingFn func(seriesID ident.ID, tags ident.Tags) bool - -// FsMergeWith is an interface that the fsMerger uses to merge data with. -type FsMergeWith interface { - // Read returns the data for the given block start and series ID, whether - // any data was found, and the error encountered (if any). - Read( - blockStart xtime.UnixNano, - seriesID ident.ID, - nsCtx namespace.Context, - ) ([]xio.BlockReader, bool, error) - - // ForEachRemaining is the loop for the second stage of merging. The - // fsMerger first loops through the fileset series, merging them with data - // in the merge target. The second stage is the go through the merge target - // data and write the remaining series that were not merged. - ForEachRemaining(blockStart xtime.UnixNano, fn forEachRemainingFn) error -} From 47bd9b67644f45065088abdcfa85d3465e0c1668 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Fri, 24 May 2019 14:36:01 -0400 Subject: [PATCH 07/18] Add merger test --- src/dbnode/generated-source-files.mk | 15 + src/dbnode/generated/mocks/generate.go | 2 +- .../persist/fs/checked_bytes_by_id_map_gen.go | 300 ++++++++++++++ .../fs/checked_bytes_by_id_new_map_gen.go | 95 +++++ src/dbnode/persist/fs/fs_mock.go | 56 ++- src/dbnode/persist/fs/merger.go | 125 +++--- src/dbnode/persist/fs/merger_test.go | 367 ++++++++++++++++++ src/dbnode/persist/fs/types.go | 8 +- src/dbnode/storage/series/series.go | 2 + src/dbnode/storage/shard.go | 16 +- 10 files changed, 916 insertions(+), 70 deletions(-) create mode 100644 src/dbnode/persist/fs/checked_bytes_by_id_map_gen.go create mode 100644 src/dbnode/persist/fs/checked_bytes_by_id_new_map_gen.go create mode 100644 src/dbnode/persist/fs/merger_test.go diff --git a/src/dbnode/generated-source-files.mk b/src/dbnode/generated-source-files.mk index 4eeae4a722..eccd647f0d 100644 --- a/src/dbnode/generated-source-files.mk +++ b/src/dbnode/generated-source-files.mk @@ -130,6 +130,21 @@ genny-map-storage-bootstrap-bootstrapper-commitlog: mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/new_map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go +# Map generation rule for persist/fs +.PHONY: genny-map-persist-fs +genny-map-persist-fs: + cd $(m3x_package_path) && make idhashmap-gen \ + pkg=fs \ + value_type=checked.Bytes \ + target_package=$(m3db_package)/src/dbnode/persist/fs \ + rename_constructor=newCheckedBytesByIDMap \ + rename_constructor_options=newCheckedBytesByIDMapOptions \ + rename_type_prefix=checkedBytes \ + rename_nogen_value=true + # Rename both generated map and constructor files + mv -f $(m3db_package_path)/src/dbnode/persist/fs/map_gen.go $(m3db_package_path)/src/dbnode/persist/fs/checked_bytes_by_id_map_gen.go + mv -f $(m3db_package_path)/src/dbnode/persist/fs/new_map_gen.go $(m3db_package_path)/src/dbnode/persist/fs/checked_bytes_by_id_new_map_gen.go + # Map generation rule for storage/index/ResultsMap .PHONY: genny-map-storage-index-results genny-map-storage-index-results: diff --git a/src/dbnode/generated/mocks/generate.go b/src/dbnode/generated/mocks/generate.go index a7e6974d7d..c36595f5c3 100644 --- a/src/dbnode/generated/mocks/generate.go +++ b/src/dbnode/generated/mocks/generate.go @@ -20,7 +20,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 | genclean -pkg $PACKAGE/src/dbnode/persist/fs -out $GOPATH/src/$PACKAGE/src/dbnode/persist/fs/fs_mock.go" +//go:generate sh -c "mockgen -package=fs $PACKAGE/src/dbnode/persist/fs DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith | genclean -pkg $PACKAGE/src/dbnode/persist/fs -out $GOPATH/src/$PACKAGE/src/dbnode/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 $GOPATH/src/$PACKAGE/src/dbnode/x/xio/io_mock.go" //go:generate sh -c "mockgen -package=digest -destination=$GOPATH/src/$PACKAGE/src/dbnode/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 $GOPATH/src/$PACKAGE/src/dbnode/storage/series/series_mock.go" diff --git a/src/dbnode/persist/fs/checked_bytes_by_id_map_gen.go b/src/dbnode/persist/fs/checked_bytes_by_id_map_gen.go new file mode 100644 index 0000000000..b6ad53390c --- /dev/null +++ b/src/dbnode/persist/fs/checked_bytes_by_id_map_gen.go @@ -0,0 +1,300 @@ +// 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. + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +package fs + +import ( + "github.com/m3db/m3/src/x/checked" + "github.com/m3db/m3/src/x/ident" +) + +// 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. + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +// 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. + +// checkedBytesMapHash is the hash for a given map entry, this is public to support +// iterating over the map using a native Go for loop. +type checkedBytesMapHash uint64 + +// checkedBytesMapHashFn is the hash function to execute when hashing a key. +type checkedBytesMapHashFn func(ident.ID) checkedBytesMapHash + +// checkedBytesMapEqualsFn is the equals key function to execute when detecting equality of a key. +type checkedBytesMapEqualsFn func(ident.ID, ident.ID) bool + +// checkedBytesMapCopyFn is the copy key function to execute when copying the key. +type checkedBytesMapCopyFn func(ident.ID) ident.ID + +// checkedBytesMapFinalizeFn is the finalize key function to execute when finished with a key. +type checkedBytesMapFinalizeFn func(ident.ID) + +// checkedBytesMap uses the genny package to provide a generic hash map that can be specialized +// by running the following command from this root of the repository: +// ``` +// make hashmap-gen pkg=outpkg key_type=Type value_type=Type out_dir=/tmp +// ``` +// Or if you would like to use bytes or ident.ID as keys you can use the +// partially specialized maps to generate your own maps as well: +// ``` +// make byteshashmap-gen pkg=outpkg value_type=Type out_dir=/tmp +// make idhashmap-gen pkg=outpkg value_type=Type out_dir=/tmp +// ``` +// This will output to stdout the generated source file to use for your map. +// It uses linear probing by incrementing the number of the hash created when +// hashing the identifier if there is a collision. +// checkedBytesMap is a value type and not an interface to allow for less painful +// upgrades when adding/removing methods, it is not likely to need mocking so +// an interface would not be super useful either. +type checkedBytesMap struct { + _checkedBytesMapOptions + + // lookup uses hash of the identifier for the key and the MapEntry value + // wraps the value type and the key (used to ensure lookup is correct + // when dealing with collisions), we use uint64 for the hash partially + // because lookups of maps with uint64 keys has a fast path for Go. + lookup map[checkedBytesMapHash]checkedBytesMapEntry +} + +// _checkedBytesMapOptions is a set of options used when creating an identifier map, it is kept +// private so that implementers of the generated map can specify their own options +// that partially fulfill these options. +type _checkedBytesMapOptions struct { + // hash is the hash function to execute when hashing a key. + hash checkedBytesMapHashFn + // equals is the equals key function to execute when detecting equality. + equals checkedBytesMapEqualsFn + // copy is the copy key function to execute when copying the key. + copy checkedBytesMapCopyFn + // finalize is the finalize key function to execute when finished with a + // key, this is optional to specify. + finalize checkedBytesMapFinalizeFn + // initialSize is the initial size for the map, use zero to use Go's std map + // initial size and consequently is optional to specify. + initialSize int +} + +// checkedBytesMapEntry is an entry in the map, this is public to support iterating +// over the map using a native Go for loop. +type checkedBytesMapEntry struct { + // key is used to check equality on lookups to resolve collisions + key _checkedBytesMapKey + // value type stored + value checked.Bytes +} + +type _checkedBytesMapKey struct { + key ident.ID + finalize bool +} + +// Key returns the map entry key. +func (e checkedBytesMapEntry) Key() ident.ID { + return e.key.key +} + +// Value returns the map entry value. +func (e checkedBytesMapEntry) Value() checked.Bytes { + return e.value +} + +// _checkedBytesMapAlloc is a non-exported function so that when generating the source code +// for the map you can supply a public constructor that sets the correct +// hash, equals, copy, finalize options without users of the map needing to +// implement them themselves. +func _checkedBytesMapAlloc(opts _checkedBytesMapOptions) *checkedBytesMap { + m := &checkedBytesMap{_checkedBytesMapOptions: opts} + m.Reallocate() + return m +} + +func (m *checkedBytesMap) newMapKey(k ident.ID, opts _checkedBytesMapKeyOptions) _checkedBytesMapKey { + key := _checkedBytesMapKey{key: k, finalize: opts.finalizeKey} + if !opts.copyKey { + return key + } + + key.key = m.copy(k) + return key +} + +func (m *checkedBytesMap) removeMapKey(hash checkedBytesMapHash, key _checkedBytesMapKey) { + delete(m.lookup, hash) + if key.finalize { + m.finalize(key.key) + } +} + +// Get returns a value in the map for an identifier if found. +func (m *checkedBytesMap) Get(k ident.ID) (checked.Bytes, bool) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + return entry.value, true + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } + var empty checked.Bytes + return empty, false +} + +// Set will set the value for an identifier. +func (m *checkedBytesMap) Set(k ident.ID, v checked.Bytes) { + m.set(k, v, _checkedBytesMapKeyOptions{ + copyKey: true, + finalizeKey: m.finalize != nil, + }) +} + +// checkedBytesMapSetUnsafeOptions is a set of options to use when setting a value with +// the SetUnsafe method. +type checkedBytesMapSetUnsafeOptions struct { + NoCopyKey bool + NoFinalizeKey bool +} + +// SetUnsafe will set the value for an identifier with unsafe options for how +// the map treats the key. +func (m *checkedBytesMap) SetUnsafe(k ident.ID, v checked.Bytes, opts checkedBytesMapSetUnsafeOptions) { + m.set(k, v, _checkedBytesMapKeyOptions{ + copyKey: !opts.NoCopyKey, + finalizeKey: !opts.NoFinalizeKey, + }) +} + +type _checkedBytesMapKeyOptions struct { + copyKey bool + finalizeKey bool +} + +func (m *checkedBytesMap) set(k ident.ID, v checked.Bytes, opts _checkedBytesMapKeyOptions) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + m.lookup[hash] = checkedBytesMapEntry{ + key: entry.key, + value: v, + } + return + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } + + m.lookup[hash] = checkedBytesMapEntry{ + key: m.newMapKey(k, opts), + value: v, + } +} + +// Iter provides the underlying map to allow for using a native Go for loop +// to iterate the map, however callers should only ever read and not write +// the map. +func (m *checkedBytesMap) Iter() map[checkedBytesMapHash]checkedBytesMapEntry { + return m.lookup +} + +// Len returns the number of map entries in the map. +func (m *checkedBytesMap) Len() int { + return len(m.lookup) +} + +// Contains returns true if value exists for key, false otherwise, it is +// shorthand for a call to Get that doesn't return the value. +func (m *checkedBytesMap) Contains(k ident.ID) bool { + _, ok := m.Get(k) + return ok +} + +// Delete will remove a value set in the map for the specified key. +func (m *checkedBytesMap) Delete(k ident.ID) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + m.removeMapKey(hash, entry.key) + return + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } +} + +// Reset will reset the map by simply deleting all keys to avoid +// allocating a new map. +func (m *checkedBytesMap) Reset() { + for hash, entry := range m.lookup { + m.removeMapKey(hash, entry.key) + } +} + +// Reallocate will avoid deleting all keys and reallocate a new +// map, this is useful if you believe you have a large map and +// will not need to grow back to a similar size. +func (m *checkedBytesMap) Reallocate() { + if m.initialSize > 0 { + m.lookup = make(map[checkedBytesMapHash]checkedBytesMapEntry, m.initialSize) + } else { + m.lookup = make(map[checkedBytesMapHash]checkedBytesMapEntry) + } +} diff --git a/src/dbnode/persist/fs/checked_bytes_by_id_new_map_gen.go b/src/dbnode/persist/fs/checked_bytes_by_id_new_map_gen.go new file mode 100644 index 0000000000..977a475856 --- /dev/null +++ b/src/dbnode/persist/fs/checked_bytes_by_id_new_map_gen.go @@ -0,0 +1,95 @@ +// 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. + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +package fs + +import ( + "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/pool" + + "github.com/cespare/xxhash" +) + +// 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. + +// newCheckedBytesByIDMapOptions provides options used when created the map. +type newCheckedBytesByIDMapOptions struct { + InitialSize int + KeyCopyPool pool.BytesPool +} + +// newCheckedBytesByIDMap returns a new byte keyed map. +func newCheckedBytesByIDMap(opts newCheckedBytesByIDMapOptions) *checkedBytesMap { + var ( + copyFn checkedBytesMapCopyFn + finalizeFn checkedBytesMapFinalizeFn + ) + if pool := opts.KeyCopyPool; pool == nil { + copyFn = func(k ident.ID) ident.ID { + return ident.BytesID(append([]byte(nil), k.Bytes()...)) + } + } else { + copyFn = func(k ident.ID) ident.ID { + bytes := k.Bytes() + keyLen := len(bytes) + pooled := pool.Get(keyLen)[:keyLen] + copy(pooled, bytes) + return ident.BytesID(pooled) + } + finalizeFn = func(k ident.ID) { + if slice, ok := k.(ident.BytesID); ok { + pool.Put(slice) + } + } + } + return _checkedBytesMapAlloc(_checkedBytesMapOptions{ + hash: func(id ident.ID) checkedBytesMapHash { + return checkedBytesMapHash(xxhash.Sum64(id.Bytes())) + }, + equals: func(x, y ident.ID) bool { + return x.Equal(y) + }, + copy: copyFn, + finalize: finalizeFn, + initialSize: opts.InitialSize, + }) +} diff --git a/src/dbnode/persist/fs/fs_mock.go b/src/dbnode/persist/fs/fs_mock.go index b7260058d7..ad4ae80a72 100644 --- a/src/dbnode/persist/fs/fs_mock.go +++ b/src/dbnode/persist/fs/fs_mock.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/dbnode/persist/fs (interfaces: DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker) +// Source: github.com/m3db/m3/src/dbnode/persist/fs (interfaces: DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith) // Copyright (c) 2019 Uber Technologies, Inc. // @@ -30,6 +30,7 @@ import ( "time" "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/m3ninx/persist" "github.com/m3db/m3/src/x/checked" "github.com/m3db/m3/src/x/ident" @@ -1164,3 +1165,56 @@ func (mr *MockConcurrentDataFileSetSeekerMockRecorder) SeekIndexEntry(arg0, arg1 mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekIndexEntry", reflect.TypeOf((*MockConcurrentDataFileSetSeeker)(nil).SeekIndexEntry), arg0, arg1) } + +// MockMergeWith is a mock of MergeWith interface +type MockMergeWith struct { + ctrl *gomock.Controller + recorder *MockMergeWithMockRecorder +} + +// MockMergeWithMockRecorder is the mock recorder for MockMergeWith +type MockMergeWithMockRecorder struct { + mock *MockMergeWith +} + +// NewMockMergeWith creates a new mock instance +func NewMockMergeWith(ctrl *gomock.Controller) *MockMergeWith { + mock := &MockMergeWith{ctrl: ctrl} + mock.recorder = &MockMergeWithMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockMergeWith) EXPECT() *MockMergeWithMockRecorder { + return m.recorder +} + +// ForEachRemaining mocks base method +func (m *MockMergeWith) ForEachRemaining(arg0 time0.UnixNano, arg1 ForEachRemainingFn) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ForEachRemaining", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// ForEachRemaining indicates an expected call of ForEachRemaining +func (mr *MockMergeWithMockRecorder) ForEachRemaining(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ForEachRemaining", reflect.TypeOf((*MockMergeWith)(nil).ForEachRemaining), arg0, arg1) +} + +// Read mocks base method +func (m *MockMergeWith) Read(arg0 ident.ID, arg1 time0.UnixNano, arg2 namespace.Context) ([]xio.BlockReader, bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Read", arg0, arg1, arg2) + ret0, _ := ret[0].([]xio.BlockReader) + ret1, _ := ret[1].(bool) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// Read indicates an expected call of Read +func (mr *MockMergeWithMockRecorder) Read(arg0, arg1, arg2 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Read", reflect.TypeOf((*MockMergeWith)(nil).Read), arg0, arg1, arg2) +} diff --git a/src/dbnode/persist/fs/merger.go b/src/dbnode/persist/fs/merger.go index 8dd008a542..f7630e5c25 100644 --- a/src/dbnode/persist/fs/merger.go +++ b/src/dbnode/persist/fs/merger.go @@ -37,6 +37,7 @@ import ( xtime "github.com/m3db/m3/src/x/time" ) +// Merger is in charge of merging filesets with some target MergeWith interface. type Merger struct { reader DataFileSetReader srPool xio.SegmentReaderPool @@ -45,6 +46,7 @@ type Merger struct { encoderPool encoding.EncoderPool } +// NewMerger returns a new Merger. func NewMerger( reader DataFileSetReader, srPool xio.SegmentReaderPool, @@ -61,39 +63,48 @@ func NewMerger( } } +// Merge merges data from a fileset with a merge target and persists it. func (m *Merger) Merge( + fileID FileSetFileIdentifier, mergeWith MergeWith, - ns namespace.Metadata, - shard uint32, - blockStart xtime.UnixNano, - blockSize time.Duration, flushPreparer persist.FlushPreparer, + nsOpts namespace.Options, nsCtx namespace.Context, ) error { - var multiErr xerrors.MultiError - - reader := m.reader - srPool := m.srPool - multiIterPool := m.multiIterPool - identPool := m.identPool - encoderPool := m.encoderPool - - startTime := blockStart.ToTime() - openOpts := DataReaderOpenOptions{ - Identifier: FileSetFileIdentifier{ - Namespace: ns.ID(), - Shard: shard, - BlockStart: startTime, - }, - } + var ( + multiErr xerrors.MultiError + reader = m.reader + srPool = m.srPool + multiIterPool = m.multiIterPool + identPool = m.identPool + encoderPool = m.encoderPool + + nsID = fileID.Namespace + shard = fileID.Shard + startTime = fileID.BlockStart + blockSize = nsOpts.RetentionOptions().BlockSize() + blockStart = xtime.ToUnixNano(startTime) + openOpts = DataReaderOpenOptions{ + Identifier: FileSetFileIdentifier{ + Namespace: nsID, + Shard: shard, + BlockStart: startTime, + }, + } + ) + if err := reader.Open(openOpts); err != nil { multiErr = multiErr.Add(err) return multiErr.FinalError() } defer reader.Close() + nsMd, err := namespace.NewMetadata(nsID, nsOpts) + if err != nil { + return err + } prepareOpts := persist.DataPrepareOptions{ - NamespaceMetadata: ns, + NamespaceMetadata: nsMd, Shard: shard, BlockStart: startTime, DeleteIfExists: false, @@ -106,8 +117,25 @@ func (m *Merger) Merge( // Writing data for a block is done in two stages. The first stage is // to loop through series on disk and merge it with what's in the merge - // target. The second stage is to persist the rest of the series in the - // merge target that was not persisted in the first stage. + // target. Looping through disk in the first stage is done intentionally to + // read disk sequentially to optimize for spinning disk access. + // The second stage is to persist the rest of the series in the + // merge target that were not persisted in the first stage. + + // There will only be one BlockReader slice since we're working within one + // block here. + brs := make([][]xio.BlockReader, 0, 1) + // There will likely be at least two BlockReaders - one for disk data and + // one for data from the merge target. + br := make([]xio.BlockReader, 0, 2) + + // It's safe to share these between iterations and just reset them each time + // because the series gets persisted each loop, so previous iterations' + // reader and iterator will never be needed. + segReader := srPool.Get() + defer segReader.Finalize() + multiIter := multiIterPool.Get() + defer multiIter.Close() // First stage: loop through series on disk. for id, tagsIter, data, _, err := reader.Read(); err != io.EOF; id, tagsIter, data, _, err = reader.Read() { @@ -116,13 +144,10 @@ func (m *Merger) Merge( return multiErr.FinalError() } - // There will be at most two BlockReader slices: one for disk data - // and one for in memory data. - brs := make([][]xio.BlockReader, 0, 2) - - // Create BlockReader slice out of disk data. - br := blockReaderFromData(data, srPool, startTime, blockSize) - brs = append(brs, []xio.BlockReader{br}) + // Reset BlockReaders. + brs = brs[:0] + br = br[:0] + br = append(br, blockReaderFromData(data, segReader, startTime, blockSize)) // Check if this series is in memory (and thus requires merging). encoded, hasData, err := mergeWith.Read(id, blockStart, nsCtx) @@ -130,27 +155,25 @@ func (m *Merger) Merge( multiErr = multiErr.Add(err) return multiErr.FinalError() } - if hasData { - brs = append(brs, encoded) + br = append(br, encoded...) } + brs = append(brs, br) - mergedIter := multiIterPool.Get() - mergedIter.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(brs), nsCtx.Schema) - defer mergedIter.Close() + multiIter.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(brs), nsCtx.Schema) + // tagsIter is never nil. tags, err := convert.TagsFromTagsIter(id, tagsIter, identPool) tagsIter.Close() if err != nil { multiErr = multiErr.Add(err) return multiErr.FinalError() } - if err := persistIter(prepared.Persist, mergedIter, id, tags, encoderPool); err != nil { + if err := persistIter(prepared.Persist, multiIter, id, tags, encoderPool); err != nil { multiErr = multiErr.Add(err) return multiErr.FinalError() } } - // Second stage: loop through rest of the merge target that was not captured // in the first stage. err = mergeWith.ForEachRemaining(blockStart, func(seriesID ident.ID, tags ident.Tags) bool { @@ -161,10 +184,8 @@ func (m *Merger) Merge( } if hasData { - iter := multiIterPool.Get() - iter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize, nsCtx.Schema) - defer iter.Close() - if err := persistIter(prepared.Persist, iter, seriesID, tags, encoderPool); err != nil { + multiIter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize, nsCtx.Schema) + if err := persistIter(prepared.Persist, multiIter, seriesID, tags, encoderPool); err != nil { multiErr = multiErr.Add(err) return false } @@ -189,15 +210,14 @@ func (m *Merger) Merge( func blockReaderFromData( data checked.Bytes, - srPool xio.SegmentReaderPool, + segReader xio.SegmentReader, startTime time.Time, blockSize time.Duration, ) xio.BlockReader { seg := ts.NewSegment(data, nil, ts.FinalizeHead) - sr := srPool.Get() - sr.Reset(seg) + segReader.Reset(seg) return xio.BlockReader{ - SegmentReader: sr, + SegmentReader: segReader, Start: startTime, BlockSize: blockSize, } @@ -221,21 +241,10 @@ func persistIter( return err } - stream, ok := encoder.Stream(encoding.StreamOptions{}) - encoder.Close() - if !ok { - // Don't write out series with no data. - return nil - } - - segment, err := stream.Segment() - if err != nil { - return err - } - + segment := encoder.Discard() checksum := digest.SegmentChecksum(segment) - err = persistFn(id, tags, segment, checksum) + err := persistFn(id, tags, segment, checksum) id.Finalize() tags.Finalize() if err != nil { diff --git a/src/dbnode/persist/fs/merger_test.go b/src/dbnode/persist/fs/merger_test.go new file mode 100644 index 0000000000..b8ac0b3c0e --- /dev/null +++ b/src/dbnode/persist/fs/merger_test.go @@ -0,0 +1,367 @@ +// 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 fs + +import ( + "io" + "testing" + "time" + + "github.com/golang/mock/gomock" + "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/encoding/m3tsz" + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/persist" + "github.com/m3db/m3/src/dbnode/ts" + "github.com/m3db/m3/src/dbnode/x/xio" + "github.com/m3db/m3/src/x/checked" + "github.com/m3db/m3/src/x/ident" + "github.com/m3db/m3/src/x/pool" + xtime "github.com/m3db/m3/src/x/time" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +const ( + blockSize = time.Hour +) + +var ( + srPool xio.SegmentReaderPool + multiIterPool encoding.MultiReaderIteratorPool + identPool ident.Pool + encoderPool encoding.EncoderPool + + startTime = time.Now().Truncate(blockSize) + + id0 = ident.StringID("id0") + id1 = ident.StringID("id1") + id2 = ident.StringID("id2") + id3 = ident.StringID("id3") + id4 = ident.StringID("id4") + id5 = ident.StringID("id5") +) + +// init resources _except_ the fsReader, which should be configured on a +// per-test basis with NewMockDataFileSetReader. +func init() { + srPool = xio.NewSegmentReaderPool(nil) + srPool.Init() + multiIterPool = encoding.NewMultiReaderIteratorPool(nil) + multiIterPool.Init(func(r io.Reader, _ namespace.SchemaDescr) encoding.ReaderIterator { + return m3tsz.NewReaderIterator(r, m3tsz.DefaultIntOptimizationEnabled, encoding.NewOptions()) + }) + bytesPool := pool.NewCheckedBytesPool(nil, nil, func(s []pool.Bucket) pool.BytesPool { + return pool.NewBytesPool(s, nil) + }) + bytesPool.Init() + identPool = ident.NewPool(bytesPool, ident.PoolOptions{}) + encoderPool = encoding.NewEncoderPool(nil) + encoderPool.Init(func() encoding.Encoder { + return m3tsz.NewEncoder(startTime, nil, true, encoding.NewOptions()) + }) +} + +func TestMergeWithIntersection(t *testing.T) { + // This test uses id0-id5. + // id0-id3 is on disk, while the merge target has id1-id5. + // Both have id1, but they don't have datapoints with overlapping + // timestamps. + // Both have id2, and some datapoints have overlapping timestamps. + // Both have id3, and all datapoints have overlapping timestamps. + diskData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + diskData.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 0}, + {Timestamp: startTime.Add(1 * time.Second), Value: 1}, + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + })) + diskData.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + {Timestamp: startTime.Add(3 * time.Second), Value: 3}, + {Timestamp: startTime.Add(6 * time.Second), Value: 4}, + {Timestamp: startTime.Add(7 * time.Second), Value: 5}, + {Timestamp: startTime.Add(9 * time.Second), Value: 6}, + })) + diskData.Set(id2, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(1 * time.Second), Value: 7}, + {Timestamp: startTime.Add(3 * time.Second), Value: 8}, + {Timestamp: startTime.Add(5 * time.Second), Value: 9}, + {Timestamp: startTime.Add(6 * time.Second), Value: 10}, + {Timestamp: startTime.Add(7 * time.Second), Value: 11}, + {Timestamp: startTime.Add(10 * time.Second), Value: 12}, + })) + diskData.Set(id3, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 13}, + {Timestamp: startTime.Add(4 * time.Second), Value: 14}, + {Timestamp: startTime.Add(8 * time.Second), Value: 15}, + })) + + mergeTargetData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + mergeTargetData.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(4 * time.Second), Value: 16}, + {Timestamp: startTime.Add(5 * time.Second), Value: 17}, + {Timestamp: startTime.Add(8 * time.Second), Value: 18}, + })) + mergeTargetData.Set(id2, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(3 * time.Second), Value: 19}, + {Timestamp: startTime.Add(6 * time.Second), Value: 20}, + {Timestamp: startTime.Add(7 * time.Second), Value: 21}, + {Timestamp: startTime.Add(9 * time.Second), Value: 22}, + {Timestamp: startTime.Add(10 * time.Second), Value: 23}, + {Timestamp: startTime.Add(13 * time.Second), Value: 24}, + {Timestamp: startTime.Add(16 * time.Second), Value: 25}, + })) + mergeTargetData.Set(id3, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 26}, + {Timestamp: startTime.Add(4 * time.Second), Value: 27}, + {Timestamp: startTime.Add(8 * time.Second), Value: 28}, + })) + mergeTargetData.Set(id4, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(8 * time.Second), Value: 29}, + })) + mergeTargetData.Set(id5, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(3 * time.Second), Value: 30}, + {Timestamp: startTime.Add(7 * time.Second), Value: 31}, + {Timestamp: startTime.Add(12 * time.Second), Value: 32}, + {Timestamp: startTime.Add(15 * time.Second), Value: 34}, + })) + + expected := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + expected.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 0}, + {Timestamp: startTime.Add(1 * time.Second), Value: 1}, + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + })) + expected.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + {Timestamp: startTime.Add(3 * time.Second), Value: 3}, + {Timestamp: startTime.Add(4 * time.Second), Value: 16}, + {Timestamp: startTime.Add(5 * time.Second), Value: 17}, + {Timestamp: startTime.Add(6 * time.Second), Value: 4}, + {Timestamp: startTime.Add(7 * time.Second), Value: 5}, + {Timestamp: startTime.Add(8 * time.Second), Value: 18}, + {Timestamp: startTime.Add(9 * time.Second), Value: 6}, + })) + expected.Set(id2, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(1 * time.Second), Value: 7}, + {Timestamp: startTime.Add(3 * time.Second), Value: 19}, + {Timestamp: startTime.Add(5 * time.Second), Value: 9}, + {Timestamp: startTime.Add(6 * time.Second), Value: 20}, + {Timestamp: startTime.Add(7 * time.Second), Value: 21}, + {Timestamp: startTime.Add(9 * time.Second), Value: 22}, + {Timestamp: startTime.Add(10 * time.Second), Value: 23}, + {Timestamp: startTime.Add(13 * time.Second), Value: 24}, + {Timestamp: startTime.Add(16 * time.Second), Value: 25}, + })) + expected.Set(id3, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 26}, + {Timestamp: startTime.Add(4 * time.Second), Value: 27}, + {Timestamp: startTime.Add(8 * time.Second), Value: 28}, + })) + expected.Set(id4, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(8 * time.Second), Value: 29}, + })) + expected.Set(id5, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(3 * time.Second), Value: 30}, + {Timestamp: startTime.Add(7 * time.Second), Value: 31}, + {Timestamp: startTime.Add(12 * time.Second), Value: 32}, + {Timestamp: startTime.Add(15 * time.Second), Value: 34}, + })) + + persisted := testMergeWith(t, diskData, mergeTargetData) + // Assert same number of expected series IDs. + require.Equal(t, expected.Len(), len(persisted)) + + for _, actualData := range persisted { + id := actualData.id + expectedData, exists := expected.Get(id) + require.True(t, exists) + seg := ts.NewSegment(expectedData, nil, ts.FinalizeHead) + + expectedDPs := datapointsFromSegment(t, seg) + actualDPs := datapointsFromSegment(t, actualData.segment) + // Assert same number of datapoints for this series. + require.Equal(t, len(expectedDPs), len(actualDPs)) + for i := range expectedDPs { + // Check each datapoint matches what's expected. + assert.Equal(t, expectedDPs[i], actualDPs[i]) + } + } +} + +func testMergeWith( + t *testing.T, + diskData *checkedBytesMap, + mergeTargetData *checkedBytesMap, +) []persistedData { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + reader := mockReaderFromData(ctrl, diskData) + + var persisted []persistedData + preparer := persist.NewMockFlushPreparer(ctrl) + preparer.EXPECT().PrepareData(gomock.Any()).Return( + persist.PreparedDataPersist{ + Persist: func(id ident.ID, tags ident.Tags, segment ts.Segment, checksum uint32) error { + persisted = append(persisted, persistedData{ + id: id, + segment: segment, + }) + return nil + }, + Close: func() error { return nil }, + }, nil) + nsCtx := namespace.Context{} + + merger := NewMerger(reader, srPool, multiIterPool, identPool, encoderPool) + fsID := FileSetFileIdentifier{ + Namespace: ident.StringID("test-ns"), + Shard: uint32(8), + BlockStart: startTime, + } + nsOpts := namespace.NewOptions() + mergeWith := mockMergeWithFromData(t, ctrl, diskData, mergeTargetData) + err := merger.Merge(fsID, mergeWith, preparer, nsOpts, nsCtx) + require.NoError(t, err) + + return persisted +} + +func datapointsToCheckedBytes(t *testing.T, dps []ts.Datapoint) checked.Bytes { + encoder := encoderPool.Get() + for _, dp := range dps { + encoder.Encode(dp, xtime.Second, nil) + } + + r, ok := encoder.Stream(encoding.StreamOptions{}) + require.True(t, ok) + var b [1000]byte + n, err := r.Read(b[:]) + require.NoError(t, err) + + cb := checked.NewBytes(b[:n], nil) + cb.IncRef() + return cb +} + +func mockReaderFromData( + ctrl *gomock.Controller, + diskData *checkedBytesMap, +) *MockDataFileSetReader { + reader := NewMockDataFileSetReader(ctrl) + reader.EXPECT().Open(gomock.Any()).Return(nil) + reader.EXPECT().Close().Return(nil) + tagIter := ident.NewTagsIterator(ident.NewTags(ident.StringTag("tag-key0", "tag-val0"))) + fakeChecksum := uint32(42) + + var inOrderCalls []*gomock.Call + for _, val := range diskData.Iter() { + id := val.Key() + data := val.Value() + inOrderCalls = append(inOrderCalls, + reader.EXPECT().Read().Return(id, tagIter, data, fakeChecksum, nil)) + } + // Make sure to return io.EOF at the end. + inOrderCalls = append(inOrderCalls, + reader.EXPECT().Read().Return(nil, nil, nil, uint32(0), io.EOF)) + gomock.InOrder(inOrderCalls...) + + return reader +} + +func mockMergeWithFromData( + t *testing.T, + ctrl *gomock.Controller, + diskData *checkedBytesMap, + mergeTargetData *checkedBytesMap, +) *MockMergeWith { + mergeWith := NewMockMergeWith(ctrl) + + // Get the series IDs in the merge target that does not exist in disk data. + // This logic is not tested here because it should be part of tests of the + // mergeWith implementation. + var remaining []ident.ID + + // Expect mergeWith.Read for all data points once. Go through all data on + // disk, then go through remaining items from merge target. + for _, val := range diskData.Iter() { + id := val.Key() + + if mergeTargetData.Contains(id) { + data, ok := mergeTargetData.Get(id) + require.True(t, ok) + segReader := srPool.Get() + br := []xio.BlockReader{blockReaderFromData(data, segReader, startTime, blockSize)} + mergeWith.EXPECT().Read(id, gomock.Any(), gomock.Any()). + Return(br, true, nil) + } else { + mergeWith.EXPECT().Read(id, gomock.Any(), gomock.Any()). + Return(nil, false, nil) + } + } + for _, val := range mergeTargetData.Iter() { + id := val.Key() + if !diskData.Contains(id) { + data := val.Value() + segReader := srPool.Get() + br := []xio.BlockReader{blockReaderFromData(data, segReader, startTime, blockSize)} + mergeWith.EXPECT().Read(id, gomock.Any(), gomock.Any()). + Return(br, true, nil) + + // Capture remaining items so that we can call the ForEachRemaining + // fn on them later. + remaining = append(remaining, id) + } + } + + mergeWith.EXPECT(). + ForEachRemaining(xtime.ToUnixNano(startTime), gomock.Any()). + Return(nil). + Do(func(blockStart xtime.UnixNano, fn ForEachRemainingFn) { + for _, id := range remaining { + fn(id, ident.Tags{}) + } + }) + + return mergeWith +} + +type persistedData struct { + id ident.ID + segment ts.Segment +} + +func datapointsFromSegment(t *testing.T, seg ts.Segment) []ts.Datapoint { + segReader := srPool.Get() + segReader.Reset(seg) + iter := multiIterPool.Get() + iter.Reset([]xio.SegmentReader{segReader}, startTime, blockSize, nil) + defer iter.Close() + + var dps []ts.Datapoint + for iter.Next() { + dp, _, _ := iter.Current() + dps = append(dps, dp) + } + require.NoError(t, iter.Err()) + + return dps +} diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index 07adbf8928..2ed4fbda17 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -490,6 +490,8 @@ type BlockRetrieverOptions interface { IdentifierPool() ident.Pool } +// ForEachRemainingFn is the function that is run on each of the remaining +// series of the merge target that did not intersect with the fileset. type ForEachRemainingFn func(seriesID ident.ID, tags ident.Tags) bool // MergeWith is an interface that the fs merger uses to merge data with. @@ -502,9 +504,7 @@ type MergeWith interface { nsCtx namespace.Context, ) ([]xio.BlockReader, bool, error) - // ForEachRemaining is the loop for the second stage of merging. The - // fsMerger first loops through the fileset series, merging them with data - // in the merge target. The second stage is the go through the merge target - // data and write the remaining series that were not merged. + // ForEachRemaining loops through each seriesID/blockStart combination that + // was not already handled by a call to Read(). ForEachRemaining(blockStart xtime.UnixNano, fn ForEachRemainingFn) error } diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 9092b15412..9dd7c86ac3 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -307,6 +307,8 @@ func (s *dbSeries) FetchBlocksForColdFlush( version int, nsCtx namespace.Context, ) ([]xio.BlockReader, error) { + // This needs a write lock because the version on underlying buckets need + // to be modified. s.Lock() defer s.Unlock() diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 5da3cfd6a7..5022fc171a 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1456,7 +1456,7 @@ func (s *dbShard) FetchBlocksForColdFlush( s.RLock() entry, _, err := s.lookupEntryWithLock(seriesID) s.RUnlock() - if err != nil { + if entry == nil || err != nil { return nil, err } @@ -1937,13 +1937,11 @@ func (s *dbShard) ColdFlush( s.RUnlock() resources.reset() - var ( multiErr xerrors.MultiError dirtySeries = resources.dirtySeries dirtySeriesToWrite = resources.dirtySeriesToWrite idElementPool = resources.idElementPool - blockSize = s.namespace.Options().RetentionOptions().BlockSize() ) // First, loop through all series to capture data on which blocks have dirty @@ -1973,7 +1971,14 @@ func (s *dbShard) ColdFlush( // has its own fileset, if we encounter an error while trying to persist // a block, we continue to try persisting other blocks. for blockStart := range dirtySeriesToWrite { - err := merger.Merge(mergeWithMem, s.namespace, s.ID(), blockStart, blockSize, flushPreparer, nsCtx) + startTime := blockStart.ToTime() + fsID := fs.FileSetFileIdentifier{ + Namespace: s.namespace.ID(), + Shard: s.ID(), + BlockStart: startTime, + } + + err := merger.Merge(fsID, mergeWithMem, flushPreparer, s.namespace.Options(), nsCtx) if err != nil { multiErr = multiErr.Add(err) continue @@ -1981,7 +1986,6 @@ func (s *dbShard) ColdFlush( // After writing the full block successfully, update the cold version // in the flush state. - startTime := blockStart.ToTime() nextVersion := s.RetrievableBlockColdVersion(startTime) + 1 s.setFlushStateColdVersion(startTime, nextVersion) } @@ -2147,7 +2151,7 @@ func (s *dbShard) TagsFromSeriesID(seriesID ident.ID) (ident.Tags, error) { s.RLock() entry, _, err := s.lookupEntryWithLock(seriesID) s.RUnlock() - if err != nil { + if entry == nil || err != nil { return ident.Tags{}, err } From 3748a7b18b0d9d99c61b1dc23ea9bcdb476e9658 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Fri, 24 May 2019 15:42:52 -0400 Subject: [PATCH 08/18] FetchBlocksForColdFlush returning no data is okay --- src/dbnode/storage/series/buffer.go | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 7e23dd315b..59f0ca8c91 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -596,10 +596,16 @@ func (b *dbBuffer) FetchBlocksForColdFlush( ) ([]xio.BlockReader, error) { res := b.fetchBlocks(ctx, []time.Time{start}, streamsOptions{filterWriteType: true, writeType: ColdWrite, nsCtx: nsCtx}) - if len(res) != 1 { + if len(res) == 0 { // The lifecycle of calling this function is preceded by first checking - // which blocks have cold data that have not yet been flushed, so - // something is wrong if we don't get any data here. + // which blocks have cold data that have not yet been flushed. + // If we don't get data here, it means that it has since fallen out of + // retention and has evicted. + return nil, nil + } + if len(res) != 1 { + // Must be only one result if anything at all, since fetchBlocks returns + // one result per block start. return nil, fmt.Errorf("fetchBlocks did not return just one block for block start %s", start) } From fe17ef5d886db7de0b64c17766f9f5e4a3377f4b Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Mon, 27 May 2019 19:17:31 -0400 Subject: [PATCH 09/18] Shard ColdFlush test --- src/dbnode/persist/fs/merger.go | 10 +- src/dbnode/persist/fs/types.go | 23 +++++ src/dbnode/storage/fs_merge_with_mem.go | 2 +- src/dbnode/storage/namespace_test.go | 2 + src/dbnode/storage/series/buffer_test.go | 8 +- src/dbnode/storage/shard.go | 52 +++++----- src/dbnode/storage/shard_test.go | 122 +++++++++++++++++++++++ src/dbnode/storage/types.go | 8 ++ 8 files changed, 194 insertions(+), 33 deletions(-) diff --git a/src/dbnode/persist/fs/merger.go b/src/dbnode/persist/fs/merger.go index f7630e5c25..04e77ce43c 100644 --- a/src/dbnode/persist/fs/merger.go +++ b/src/dbnode/persist/fs/merger.go @@ -37,8 +37,7 @@ import ( xtime "github.com/m3db/m3/src/x/time" ) -// Merger is in charge of merging filesets with some target MergeWith interface. -type Merger struct { +type merger struct { reader DataFileSetReader srPool xio.SegmentReaderPool multiIterPool encoding.MultiReaderIteratorPool @@ -53,8 +52,8 @@ func NewMerger( multiIterPool encoding.MultiReaderIteratorPool, identPool ident.Pool, encoderPool encoding.EncoderPool, -) *Merger { - return &Merger{ +) Merger { + return &merger{ reader: reader, srPool: srPool, multiIterPool: multiIterPool, @@ -64,7 +63,7 @@ func NewMerger( } // Merge merges data from a fileset with a merge target and persists it. -func (m *Merger) Merge( +func (m *merger) Merge( fileID FileSetFileIdentifier, mergeWith MergeWith, flushPreparer persist.FlushPreparer, @@ -90,6 +89,7 @@ func (m *Merger) Merge( Shard: shard, BlockStart: startTime, }, + FileSetType: persist.FileSetFlushType, } ) diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index 2ed4fbda17..ff96b7641c 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -26,6 +26,7 @@ import ( "time" "github.com/m3db/m3/src/dbnode/clock" + "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" @@ -508,3 +509,25 @@ type MergeWith interface { // was not already handled by a call to Read(). ForEachRemaining(blockStart xtime.UnixNano, fn ForEachRemainingFn) error } + +// Merger is in charge of merging filesets with some target MergeWith interface. +type Merger interface { + // Merge merges the specified fileset file with a merge target. + Merge( + fileID FileSetFileIdentifier, + mergeWith MergeWith, + flushPreparer persist.FlushPreparer, + nsOpts namespace.Options, + nsCtx namespace.Context, + ) error +} + +// NewMergerFn is the function to call to get a new Merger. Mostly used as a +// convenience for testing. +type NewMergerFn func( + reader DataFileSetReader, + srPool xio.SegmentReaderPool, + multiIterPool encoding.MultiReaderIteratorPool, + identPool ident.Pool, + encoderPool encoding.EncoderPool, +) Merger diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index 7ca1f1be8d..708bcc2566 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -45,7 +45,7 @@ func newFSMergeWithMem( retriever series.QueryableBlockRetriever, dirtySeries *dirtySeriesMap, dirtySeriesToWrite map[xtime.UnixNano]*idList, -) *fsMergeWithMem { +) fs.MergeWith { return &fsMergeWithMem{ shard: shard, retriever: retriever, diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index f0022ecd9c..2841bcc66b 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -415,6 +415,7 @@ func TestNamespaceFlushNotBootstrapped(t *testing.T) { ns, closer := newTestNamespace(t) defer closer() require.Equal(t, errNamespaceNotBootstrapped, ns.WarmFlush(time.Now(), nil, nil)) + require.Equal(t, errNamespaceNotBootstrapped, ns.ColdFlush(nil)) } func TestNamespaceFlushDontNeedFlush(t *testing.T) { @@ -424,6 +425,7 @@ func TestNamespaceFlushDontNeedFlush(t *testing.T) { ns.bootstrapState = Bootstrapped require.NoError(t, ns.WarmFlush(time.Now(), nil, nil)) + require.NoError(t, ns.ColdFlush(nil)) } func TestNamespaceFlushSkipFlushed(t *testing.T) { diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index 0cf0daab75..5592d8a4b4 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -1360,7 +1360,9 @@ func TestFetchBlocksForColdFlush(t *testing.T) { requireReaderValuesEqual(t, expected[blockStartNano3], [][]xio.BlockReader{reader}, opts, nsCtx) assert.Equal(t, 1, buffer.bucketsMap[blockStartNano3].buckets[0].version) - // Try to fetch from a block that only has warm buckets, which should error. - _, err = buffer.FetchBlocksForColdFlush(ctx, blockStart4, 1, nsCtx) - assert.Error(t, err) + // Try to fetch from a block that only has warm buckets. It has no data + // but is not an error. + reader, err = buffer.FetchBlocksForColdFlush(ctx, blockStart4, 1, nsCtx) + assert.NoError(t, err) + requireReaderValuesEqual(t, []value{}, [][]xio.BlockReader{reader}, opts, nsCtx) } diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 5022fc171a..7436c07e3d 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -148,6 +148,8 @@ type dbShard struct { lookup *shardMap list *list.List bootstrapState BootstrapState + newMergerFn fs.NewMergerFn + newFSMergeWithMemFn newFSMergeWithMemFn filesetBeforeFn filesetBeforeFn deleteFilesFn deleteFilesFn snapshotFilesFn snapshotFilesFn @@ -245,28 +247,30 @@ func newDatabaseShard( SubScope("dbshard") s := &dbShard{ - opts: opts, - seriesOpts: seriesOpts, - nowFn: opts.ClockOptions().NowFn(), - state: dbShardStateOpen, - namespace: namespaceMetadata, - shard: shard, - namespaceReaderMgr: namespaceReaderMgr, - increasingIndex: increasingIndex, - seriesPool: opts.DatabaseSeriesPool(), - reverseIndex: reverseIndex, - lookup: newShardMap(shardMapOptions{}), - list: list.New(), - filesetBeforeFn: fs.DataFileSetsBefore, - deleteFilesFn: fs.DeleteFiles, - snapshotFilesFn: fs.SnapshotFiles, - sleepFn: time.Sleep, - identifierPool: opts.IdentifierPool(), - contextPool: opts.ContextPool(), - flushState: newShardFlushState(), - tickWg: &sync.WaitGroup{}, - logger: opts.InstrumentOptions().Logger(), - metrics: newDatabaseShardMetrics(shard, scope), + opts: opts, + seriesOpts: seriesOpts, + nowFn: opts.ClockOptions().NowFn(), + state: dbShardStateOpen, + namespace: namespaceMetadata, + shard: shard, + namespaceReaderMgr: namespaceReaderMgr, + increasingIndex: increasingIndex, + seriesPool: opts.DatabaseSeriesPool(), + reverseIndex: reverseIndex, + lookup: newShardMap(shardMapOptions{}), + list: list.New(), + newMergerFn: fs.NewMerger, + newFSMergeWithMemFn: newFSMergeWithMem, + filesetBeforeFn: fs.DataFileSetsBefore, + deleteFilesFn: fs.DeleteFiles, + snapshotFilesFn: fs.SnapshotFiles, + sleepFn: time.Sleep, + identifierPool: opts.IdentifierPool(), + contextPool: opts.ContextPool(), + flushState: newShardFlushState(), + tickWg: &sync.WaitGroup{}, + logger: opts.InstrumentOptions().Logger(), + metrics: newDatabaseShardMetrics(shard, scope), } s.insertQueue = newDatabaseShardInsertQueue(s.insertSeriesBatch, s.nowFn, scope) @@ -1964,9 +1968,9 @@ func (s *dbShard) ColdFlush( return true }) - merger := fs.NewMerger(resources.fsReader, s.opts.SegmentReaderPool(), + merger := s.newMergerFn(resources.fsReader, s.opts.SegmentReaderPool(), s.opts.MultiReaderIteratorPool(), s.opts.IdentifierPool(), s.opts.EncoderPool()) - mergeWithMem := newFSMergeWithMem(s, s, dirtySeries, dirtySeriesToWrite) + mergeWithMem := s.newFSMergeWithMemFn(s, s, dirtySeries, dirtySeriesToWrite) // Loop through each block that we know has ColdWrites. Since each block // has its own fileset, if we encounter an error while trying to persist // a block, we continue to try persisting other blocks. diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 02e97b2973..6d4daaabec 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -30,8 +30,10 @@ import ( "time" "unsafe" + "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" + "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/block" @@ -320,6 +322,126 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { }, flushState) } +type testDirtySeries struct { + id ident.ID + dirtyTimes []time.Time +} + +func optimizedTimesFromTimes(times []time.Time) series.OptimizedTimes { + var ret series.OptimizedTimes + for _, t := range times { + ret.Add(xtime.ToUnixNano(t)) + } + return ret +} + +func TestShardColdFlush(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + now := time.Now() + nowFn := func() time.Time { + return now + } + opts := DefaultTestOptions() + opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) + blockSize := opts.SeriesOptions().RetentionOptions().BlockSize() + shard := testDatabaseShard(t, opts) + shard.bootstrapState = Bootstrapped + shard.newMergerFn = newMergerTestFn + shard.newFSMergeWithMemFn = newFSMergeWithMemTestFn + + t0 := now.Truncate(blockSize).Add(-10 * blockSize) + t1 := t0.Add(1 * blockSize) + t2 := t0.Add(2 * blockSize) + t3 := t0.Add(3 * blockSize) + t4 := t0.Add(4 * blockSize) + t5 := t0.Add(5 * blockSize) + t6 := t0.Add(6 * blockSize) + t7 := t0.Add(7 * blockSize) + dirtyData := []testDirtySeries{ + {id: ident.StringID("id0"), dirtyTimes: []time.Time{t0, t2, t3, t4}}, + {id: ident.StringID("id1"), dirtyTimes: []time.Time{t1}}, + {id: ident.StringID("id2"), dirtyTimes: []time.Time{t3, t4, t5}}, + {id: ident.StringID("id3"), dirtyTimes: []time.Time{t6, t7}}, + } + for _, ds := range dirtyData { + curr := series.NewMockDatabaseSeries(ctrl) + curr.EXPECT().ID().Return(ds.id) + curr.EXPECT().NeedsColdFlushBlockStarts(). + Return(optimizedTimesFromTimes(ds.dirtyTimes)) + shard.list.PushBack(lookup.NewEntry(curr, 0)) + } + + preparer := persist.NewMockFlushPreparer(ctrl) + fsReader := fs.NewMockDataFileSetReader(ctrl) + resources := coldFlushReuseableResources{ + dirtySeries: newDirtySeriesMap(dirtySeriesMapOptions{}), + dirtySeriesToWrite: make(map[xtime.UnixNano]*idList), + idElementPool: newIDElementPool(nil), + fsReader: fsReader, + } + nsCtx := namespace.Context{} + + // Assert that flush state cold versions all start at 0. + for i := t0; i.Before(t7.Add(blockSize)); i = i.Add(blockSize) { + assert.Equal(t, 0, shard.RetrievableBlockColdVersion(i)) + } + shard.ColdFlush(preparer, resources, nsCtx) + // After a cold flush, all previously dirty block starts should be updated + // to version 1. + for i := t0; i.Before(t7.Add(blockSize)); i = i.Add(blockSize) { + assert.Equal(t, 1, shard.RetrievableBlockColdVersion(i)) + } +} + +func newMergerTestFn( + reader fs.DataFileSetReader, + srPool xio.SegmentReaderPool, + multiIterPool encoding.MultiReaderIteratorPool, + identPool ident.Pool, + encoderPool encoding.EncoderPool, +) fs.Merger { + return &noopMerger{} +} + +type noopMerger struct{} + +func (m *noopMerger) Merge( + fileID fs.FileSetFileIdentifier, + mergeWith fs.MergeWith, + flushPreparer persist.FlushPreparer, + nsOpts namespace.Options, + nsCtx namespace.Context, +) error { + return nil +} + +func newFSMergeWithMemTestFn( + shard databaseShard, + retriever series.QueryableBlockRetriever, + dirtySeries *dirtySeriesMap, + dirtySeriesToWrite map[xtime.UnixNano]*idList, +) fs.MergeWith { + return &noopMergeWith{} +} + +type noopMergeWith struct{} + +func (m *noopMergeWith) Read( + seriesID ident.ID, + blockStart xtime.UnixNano, + nsCtx namespace.Context, +) ([]xio.BlockReader, bool, error) { + return nil, false, nil +} + +func (m *noopMergeWith) ForEachRemaining( + blockStart xtime.UnixNano, + fn fs.ForEachRemainingFn, +) error { + return nil +} + func TestShardSnapshotShardNotBootstrapped(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 7082e48bb4..f457540c02 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -29,6 +29,7 @@ import ( "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" + "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/sharding" @@ -994,3 +995,10 @@ const ( // Bootstrapped indicates a bootstrap process has completed. Bootstrapped ) + +type newFSMergeWithMemFn func( + shard databaseShard, + retriever series.QueryableBlockRetriever, + dirtySeries *dirtySeriesMap, + dirtySeriesToWrite map[xtime.UnixNano]*idList, +) fs.MergeWith From 2943d5b8bbdf018aee94e76fe1ad6657abe4a1a5 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Wed, 29 May 2019 16:27:31 -0400 Subject: [PATCH 10/18] More tests, comments, PR feedback --- src/dbnode/persist/fs/merger.go | 180 ++++++----- src/dbnode/persist/fs/merger_test.go | 284 ++++++++++++++++-- src/dbnode/persist/fs/types.go | 5 +- src/dbnode/storage/flush.go | 5 + src/dbnode/storage/fs_merge_with_mem.go | 15 +- src/dbnode/storage/fs_merge_with_mem_test.go | 16 +- src/dbnode/storage/namespace.go | 13 +- src/dbnode/storage/series/buffer.go | 2 +- src/dbnode/storage/series/series.go | 5 +- src/dbnode/storage/shard.go | 7 +- src/dbnode/storage/shard_test.go | 3 +- ..._reader_slice_from_segment_reader_slice.go | 35 --- 12 files changed, 412 insertions(+), 158 deletions(-) delete mode 100644 src/dbnode/x/xio/block_reader_slice_from_segment_reader_slice.go diff --git a/src/dbnode/persist/fs/merger.go b/src/dbnode/persist/fs/merger.go index 04e77ce43c..20b0376a31 100644 --- a/src/dbnode/persist/fs/merger.go +++ b/src/dbnode/persist/fs/merger.go @@ -32,33 +32,45 @@ import ( "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/x/checked" - xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" ) type merger struct { - reader DataFileSetReader - srPool xio.SegmentReaderPool - multiIterPool encoding.MultiReaderIteratorPool - identPool ident.Pool - encoderPool encoding.EncoderPool + reader DataFileSetReader + blockAllocSize int + srPool xio.SegmentReaderPool + multiIterPool encoding.MultiReaderIteratorPool + identPool ident.Pool + encoderPool encoding.EncoderPool + nsOpts namespace.Options } -// NewMerger returns a new Merger. +// NewMerger returns a new Merger. This implementation is in charge of merging +// the data from an existing fileset with a merge target, giving precendence +// to the data from the merge target. This merged data is then persisted. +// +// Note that the merger it does not know how or where this merged data is +// persisted since it just uses the flushPreparer that is passed in. Further, +// it does not signal to the database of the existence of the newly persisted +// data, nor does it clean up the original fileset. func NewMerger( reader DataFileSetReader, + blockAllocSize int, srPool xio.SegmentReaderPool, multiIterPool encoding.MultiReaderIteratorPool, identPool ident.Pool, encoderPool encoding.EncoderPool, + nsOpts namespace.Options, ) Merger { return &merger{ - reader: reader, - srPool: srPool, - multiIterPool: multiIterPool, - identPool: identPool, - encoderPool: encoderPool, + reader: reader, + blockAllocSize: blockAllocSize, + srPool: srPool, + multiIterPool: multiIterPool, + identPool: identPool, + encoderPool: encoderPool, + nsOpts: nsOpts, } } @@ -67,16 +79,16 @@ func (m *merger) Merge( fileID FileSetFileIdentifier, mergeWith MergeWith, flushPreparer persist.FlushPreparer, - nsOpts namespace.Options, nsCtx namespace.Context, ) error { var ( - multiErr xerrors.MultiError - reader = m.reader - srPool = m.srPool - multiIterPool = m.multiIterPool - identPool = m.identPool - encoderPool = m.encoderPool + reader = m.reader + blockAllocSize = m.blockAllocSize + srPool = m.srPool + multiIterPool = m.multiIterPool + identPool = m.identPool + encoderPool = m.encoderPool + nsOpts = m.nsOpts nsID = fileID.Namespace shard = fileID.Shard @@ -94,8 +106,7 @@ func (m *merger) Merge( ) if err := reader.Open(openOpts); err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() + return err } defer reader.Close() @@ -111,37 +122,62 @@ func (m *merger) Merge( } prepared, err := flushPreparer.PrepareData(prepareOpts) if err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() + return err } - // Writing data for a block is done in two stages. The first stage is - // to loop through series on disk and merge it with what's in the merge - // target. Looping through disk in the first stage is done intentionally to - // read disk sequentially to optimize for spinning disk access. - // The second stage is to persist the rest of the series in the - // merge target that were not persisted in the first stage. - - // There will only be one BlockReader slice since we're working within one - // block here. - brs := make([][]xio.BlockReader, 0, 1) - // There will likely be at least two BlockReaders - one for disk data and - // one for data from the merge target. - br := make([]xio.BlockReader, 0, 2) - - // It's safe to share these between iterations and just reset them each time - // because the series gets persisted each loop, so previous iterations' - // reader and iterator will never be needed. - segReader := srPool.Get() + var ( + // There will only be one BlockReader slice since we're working within one + // block here. + brs = make([][]xio.BlockReader, 0, 1) + // There will likely be at least two BlockReaders - one for disk data and + // one for data from the merge target. + br = make([]xio.BlockReader, 0, 2) + + // It's safe to share these between iterations and just reset them each time + // because the series gets persisted each loop, so previous iterations' + // reader and iterator will never be needed. + segReader = srPool.Get() + multiIter = multiIterPool.Get() + // Initialize this here with nil to be reset before each iteration's + // use. + sliceOfSlices = xio.NewReaderSliceOfSlicesFromBlockReadersIterator(nil) + + // We keep track of IDs/tags to finalize at the end of merging. This + // only applies to those that come from disk Reads, since the whole + // lifecycle of those IDs/tags are contained to this function. We don't + // want finalize this series ID since it is from memory and other + // components may have ownership over it. + // + // We must only finalize these at the end of this function, since the + // flush preparer's underlying writer holds on to those references + // until it is closed (closing the PreparedDataPersist at the end of + // this merge closes the underlying writer). + idsToFinalize []ident.ID + tagsToFinalize []ident.Tags + ) defer segReader.Finalize() - multiIter := multiIterPool.Get() defer multiIter.Close() + defer func() { + for _, res := range idsToFinalize { + res.Finalize() + } + for _, res := range tagsToFinalize { + res.Finalize() + } + }() + + // The merge is performed in two stages. The first stage is to loop through + // series on disk and merge it with what's in the merge target. Looping + // through disk in the first stage is done intentionally to read disk + // sequentially to optimize for spinning disk access. The second stage is to + // persist the rest of the series in the merge target that were not + // persisted in the first stage. // First stage: loop through series on disk. for id, tagsIter, data, _, err := reader.Read(); err != io.EOF; id, tagsIter, data, _, err = reader.Read() { + idsToFinalize = append(idsToFinalize, id) if err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() + return err } // Reset BlockReaders. @@ -150,62 +186,56 @@ func (m *merger) Merge( br = append(br, blockReaderFromData(data, segReader, startTime, blockSize)) // Check if this series is in memory (and thus requires merging). - encoded, hasData, err := mergeWith.Read(id, blockStart, nsCtx) + mergeWithData, hasData, err := mergeWith.Read(id, blockStart, nsCtx) if err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() + return err } if hasData { - br = append(br, encoded...) + br = append(br, mergeWithData...) } brs = append(brs, br) - multiIter.ResetSliceOfSlices(xio.NewReaderSliceOfSlicesFromBlockReadersIterator(brs), nsCtx.Schema) + sliceOfSlices.Reset(brs) + multiIter.ResetSliceOfSlices(sliceOfSlices, nsCtx.Schema) // tagsIter is never nil. tags, err := convert.TagsFromTagsIter(id, tagsIter, identPool) tagsIter.Close() + tagsToFinalize = append(tagsToFinalize, tags) if err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() + return err } - if err := persistIter(prepared.Persist, multiIter, id, tags, encoderPool); err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() + if err := persistIter(prepared.Persist, multiIter, startTime, + id, tags, blockAllocSize, nsCtx.Schema, encoderPool); err != nil { + return err } } // Second stage: loop through rest of the merge target that was not captured // in the first stage. - err = mergeWith.ForEachRemaining(blockStart, func(seriesID ident.ID, tags ident.Tags) bool { - encoded, hasData, err := mergeWith.Read(seriesID, blockStart, nsCtx) + err = mergeWith.ForEachRemaining(blockStart, func(seriesID ident.ID, tags ident.Tags) error { + mergeWithData, hasData, err := mergeWith.Read(seriesID, blockStart, nsCtx) if err != nil { - multiErr = multiErr.Add(err) - return false + return err } if hasData { - multiIter.Reset(xio.NewSegmentReaderSliceFromBlockReaderSlice(encoded), startTime, blockSize, nsCtx.Schema) - if err := persistIter(prepared.Persist, multiIter, seriesID, tags, encoderPool); err != nil { - multiErr = multiErr.Add(err) - return false - } + brs = brs[:0] + brs = append(brs, mergeWithData) + sliceOfSlices.Reset(brs) + multiIter.ResetSliceOfSlices(sliceOfSlices, nsCtx.Schema) + return persistIter(prepared.Persist, multiIter, startTime, + seriesID, tags, blockAllocSize, nsCtx.Schema, encoderPool) } - return true + return nil }) if err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() + return err } // Close the flush preparer, which writes the rest of the files in the // fileset. - if err := prepared.Close(); err != nil { - multiErr = multiErr.Add(err) - return multiErr.FinalError() - } - - return multiErr.FinalError() + return prepared.Close() } func blockReaderFromData( @@ -226,11 +256,15 @@ func blockReaderFromData( func persistIter( persistFn persist.DataFn, it encoding.Iterator, + blockStart time.Time, id ident.ID, tags ident.Tags, + blockAllocSize int, + schema namespace.SchemaDescr, encoderPool encoding.EncoderPool, ) error { encoder := encoderPool.Get() + encoder.Reset(blockStart, blockAllocSize, schema) for it.Next() { if err := encoder.Encode(it.Current()); err != nil { encoder.Close() @@ -245,8 +279,6 @@ func persistIter( checksum := digest.SegmentChecksum(segment) err := persistFn(id, tags, segment, checksum) - id.Finalize() - tags.Finalize() if err != nil { return err } diff --git a/src/dbnode/persist/fs/merger_test.go b/src/dbnode/persist/fs/merger_test.go index b8ac0b3c0e..6ea60a85d2 100644 --- a/src/dbnode/persist/fs/merger_test.go +++ b/src/dbnode/persist/fs/merger_test.go @@ -81,7 +81,8 @@ func init() { } func TestMergeWithIntersection(t *testing.T) { - // This test uses id0-id5. + // This test scenario is when there is an overlap in series data between + // disk and the merge target. // id0-id3 is on disk, while the merge target has id1-id5. // Both have id1, but they don't have datapoints with overlapping // timestamps. @@ -186,32 +187,244 @@ func TestMergeWithIntersection(t *testing.T) { {Timestamp: startTime.Add(15 * time.Second), Value: 34}, })) - persisted := testMergeWith(t, diskData, mergeTargetData) - // Assert same number of expected series IDs. - require.Equal(t, expected.Len(), len(persisted)) + testMergeWith(t, diskData, mergeTargetData, expected) +} - for _, actualData := range persisted { - id := actualData.id - expectedData, exists := expected.Get(id) - require.True(t, exists) - seg := ts.NewSegment(expectedData, nil, ts.FinalizeHead) +func TestMergeWithFullIntersection(t *testing.T) { + // This test scenario is when the merge target contains only and all data + // from disk. + diskData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + diskData.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 0}, + {Timestamp: startTime.Add(1 * time.Second), Value: 1}, + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + })) + diskData.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + {Timestamp: startTime.Add(3 * time.Second), Value: 3}, + {Timestamp: startTime.Add(6 * time.Second), Value: 4}, + {Timestamp: startTime.Add(7 * time.Second), Value: 5}, + {Timestamp: startTime.Add(9 * time.Second), Value: 6}, + })) - expectedDPs := datapointsFromSegment(t, seg) - actualDPs := datapointsFromSegment(t, actualData.segment) - // Assert same number of datapoints for this series. - require.Equal(t, len(expectedDPs), len(actualDPs)) - for i := range expectedDPs { - // Check each datapoint matches what's expected. - assert.Equal(t, expectedDPs[i], actualDPs[i]) - } - } + mergeTargetData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + mergeTargetData.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 7}, + {Timestamp: startTime.Add(1 * time.Second), Value: 8}, + {Timestamp: startTime.Add(2 * time.Second), Value: 9}, + })) + mergeTargetData.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 10}, + {Timestamp: startTime.Add(3 * time.Second), Value: 11}, + {Timestamp: startTime.Add(6 * time.Second), Value: 12}, + {Timestamp: startTime.Add(7 * time.Second), Value: 13}, + {Timestamp: startTime.Add(9 * time.Second), Value: 14}, + })) + + expected := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + expected.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 7}, + {Timestamp: startTime.Add(1 * time.Second), Value: 8}, + {Timestamp: startTime.Add(2 * time.Second), Value: 9}, + })) + expected.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 10}, + {Timestamp: startTime.Add(3 * time.Second), Value: 11}, + {Timestamp: startTime.Add(6 * time.Second), Value: 12}, + {Timestamp: startTime.Add(7 * time.Second), Value: 13}, + {Timestamp: startTime.Add(9 * time.Second), Value: 14}, + })) + + testMergeWith(t, diskData, mergeTargetData, expected) +} + +func TestMergeWithNoIntersection(t *testing.T) { + // This test scenario is when there is no overlap between disk data and + // merge target data (series from one source does not exist in the other). + diskData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + diskData.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 0}, + {Timestamp: startTime.Add(1 * time.Second), Value: 1}, + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + })) + diskData.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + {Timestamp: startTime.Add(3 * time.Second), Value: 3}, + {Timestamp: startTime.Add(6 * time.Second), Value: 4}, + {Timestamp: startTime.Add(7 * time.Second), Value: 5}, + {Timestamp: startTime.Add(9 * time.Second), Value: 6}, + })) + diskData.Set(id2, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(1 * time.Second), Value: 7}, + {Timestamp: startTime.Add(3 * time.Second), Value: 8}, + {Timestamp: startTime.Add(5 * time.Second), Value: 9}, + {Timestamp: startTime.Add(6 * time.Second), Value: 10}, + {Timestamp: startTime.Add(7 * time.Second), Value: 11}, + {Timestamp: startTime.Add(10 * time.Second), Value: 12}, + })) + + mergeTargetData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + mergeTargetData.Set(id3, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 26}, + {Timestamp: startTime.Add(4 * time.Second), Value: 27}, + {Timestamp: startTime.Add(8 * time.Second), Value: 28}, + })) + mergeTargetData.Set(id4, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(8 * time.Second), Value: 29}, + })) + mergeTargetData.Set(id5, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(3 * time.Second), Value: 30}, + {Timestamp: startTime.Add(7 * time.Second), Value: 31}, + {Timestamp: startTime.Add(12 * time.Second), Value: 32}, + {Timestamp: startTime.Add(15 * time.Second), Value: 34}, + })) + + expected := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + expected.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 0}, + {Timestamp: startTime.Add(1 * time.Second), Value: 1}, + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + })) + expected.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + {Timestamp: startTime.Add(3 * time.Second), Value: 3}, + {Timestamp: startTime.Add(6 * time.Second), Value: 4}, + {Timestamp: startTime.Add(7 * time.Second), Value: 5}, + {Timestamp: startTime.Add(9 * time.Second), Value: 6}, + })) + expected.Set(id2, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(1 * time.Second), Value: 7}, + {Timestamp: startTime.Add(3 * time.Second), Value: 8}, + {Timestamp: startTime.Add(5 * time.Second), Value: 9}, + {Timestamp: startTime.Add(6 * time.Second), Value: 10}, + {Timestamp: startTime.Add(7 * time.Second), Value: 11}, + {Timestamp: startTime.Add(10 * time.Second), Value: 12}, + })) + expected.Set(id3, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 26}, + {Timestamp: startTime.Add(4 * time.Second), Value: 27}, + {Timestamp: startTime.Add(8 * time.Second), Value: 28}, + })) + expected.Set(id4, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(8 * time.Second), Value: 29}, + })) + expected.Set(id5, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(3 * time.Second), Value: 30}, + {Timestamp: startTime.Add(7 * time.Second), Value: 31}, + {Timestamp: startTime.Add(12 * time.Second), Value: 32}, + {Timestamp: startTime.Add(15 * time.Second), Value: 34}, + })) + + testMergeWith(t, diskData, mergeTargetData, expected) +} + +func TestMergeWithNoMergeTargetData(t *testing.T) { + // This test scenario is when there is no data in the merge target. + diskData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + diskData.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 0}, + {Timestamp: startTime.Add(1 * time.Second), Value: 1}, + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + })) + diskData.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + {Timestamp: startTime.Add(3 * time.Second), Value: 3}, + {Timestamp: startTime.Add(6 * time.Second), Value: 4}, + {Timestamp: startTime.Add(7 * time.Second), Value: 5}, + {Timestamp: startTime.Add(9 * time.Second), Value: 6}, + })) + diskData.Set(id2, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(1 * time.Second), Value: 7}, + {Timestamp: startTime.Add(3 * time.Second), Value: 8}, + {Timestamp: startTime.Add(5 * time.Second), Value: 9}, + {Timestamp: startTime.Add(6 * time.Second), Value: 10}, + {Timestamp: startTime.Add(7 * time.Second), Value: 11}, + {Timestamp: startTime.Add(10 * time.Second), Value: 12}, + })) + + mergeTargetData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + + expected := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + expected.Set(id0, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(0 * time.Second), Value: 0}, + {Timestamp: startTime.Add(1 * time.Second), Value: 1}, + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + })) + expected.Set(id1, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 2}, + {Timestamp: startTime.Add(3 * time.Second), Value: 3}, + {Timestamp: startTime.Add(6 * time.Second), Value: 4}, + {Timestamp: startTime.Add(7 * time.Second), Value: 5}, + {Timestamp: startTime.Add(9 * time.Second), Value: 6}, + })) + expected.Set(id2, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(1 * time.Second), Value: 7}, + {Timestamp: startTime.Add(3 * time.Second), Value: 8}, + {Timestamp: startTime.Add(5 * time.Second), Value: 9}, + {Timestamp: startTime.Add(6 * time.Second), Value: 10}, + {Timestamp: startTime.Add(7 * time.Second), Value: 11}, + {Timestamp: startTime.Add(10 * time.Second), Value: 12}, + })) + + testMergeWith(t, diskData, mergeTargetData, expected) +} + +func TestMergeWithNoDiskData(t *testing.T) { + // This test scenario is there is no data on disk. + diskData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + + mergeTargetData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + mergeTargetData.Set(id3, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 26}, + {Timestamp: startTime.Add(4 * time.Second), Value: 27}, + {Timestamp: startTime.Add(8 * time.Second), Value: 28}, + })) + mergeTargetData.Set(id4, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(8 * time.Second), Value: 29}, + })) + mergeTargetData.Set(id5, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(3 * time.Second), Value: 30}, + {Timestamp: startTime.Add(7 * time.Second), Value: 31}, + {Timestamp: startTime.Add(12 * time.Second), Value: 32}, + {Timestamp: startTime.Add(15 * time.Second), Value: 34}, + })) + + expected := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + expected.Set(id3, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(2 * time.Second), Value: 26}, + {Timestamp: startTime.Add(4 * time.Second), Value: 27}, + {Timestamp: startTime.Add(8 * time.Second), Value: 28}, + })) + expected.Set(id4, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(8 * time.Second), Value: 29}, + })) + expected.Set(id5, datapointsToCheckedBytes(t, []ts.Datapoint{ + {Timestamp: startTime.Add(3 * time.Second), Value: 30}, + {Timestamp: startTime.Add(7 * time.Second), Value: 31}, + {Timestamp: startTime.Add(12 * time.Second), Value: 32}, + {Timestamp: startTime.Add(15 * time.Second), Value: 34}, + })) + + testMergeWith(t, diskData, mergeTargetData, expected) +} + +func TestMergeWithNoData(t *testing.T) { + // This test scenario is there is no data on disk or the merge target. + diskData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + + mergeTargetData := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + + expected := newCheckedBytesByIDMap(newCheckedBytesByIDMapOptions{}) + + testMergeWith(t, diskData, mergeTargetData, expected) } func testMergeWith( t *testing.T, diskData *checkedBytesMap, mergeTargetData *checkedBytesMap, -) []persistedData { + expectedData *checkedBytesMap, +) { ctrl := gomock.NewController(t) defer ctrl.Finish() reader := mockReaderFromData(ctrl, diskData) @@ -231,18 +444,43 @@ func testMergeWith( }, nil) nsCtx := namespace.Context{} - merger := NewMerger(reader, srPool, multiIterPool, identPool, encoderPool) + nsOpts := namespace.NewOptions() + merger := NewMerger(reader, 0, srPool, multiIterPool, identPool, encoderPool, nsOpts) fsID := FileSetFileIdentifier{ Namespace: ident.StringID("test-ns"), Shard: uint32(8), BlockStart: startTime, } - nsOpts := namespace.NewOptions() mergeWith := mockMergeWithFromData(t, ctrl, diskData, mergeTargetData) - err := merger.Merge(fsID, mergeWith, preparer, nsOpts, nsCtx) + err := merger.Merge(fsID, mergeWith, preparer, nsCtx) require.NoError(t, err) - return persisted + assertPersistedAsExpected(t, persisted, expectedData) +} + +func assertPersistedAsExpected( + t *testing.T, + persisted []persistedData, + expectedData *checkedBytesMap, +) { + // Assert same number of expected series IDs. + require.Equal(t, expectedData.Len(), len(persisted)) + + for _, actualData := range persisted { + id := actualData.id + data, exists := expectedData.Get(id) + require.True(t, exists) + seg := ts.NewSegment(data, nil, ts.FinalizeHead) + + expectedDPs := datapointsFromSegment(t, seg) + actualDPs := datapointsFromSegment(t, actualData.segment) + // Assert same number of datapoints for this series. + require.Equal(t, len(expectedDPs), len(actualDPs)) + for i := range expectedDPs { + // Check each datapoint matches what's expected. + assert.Equal(t, expectedDPs[i], actualDPs[i]) + } + } } func datapointsToCheckedBytes(t *testing.T, dps []ts.Datapoint) checked.Bytes { diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index ff96b7641c..fc87b3bea5 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -493,7 +493,7 @@ type BlockRetrieverOptions interface { // ForEachRemainingFn is the function that is run on each of the remaining // series of the merge target that did not intersect with the fileset. -type ForEachRemainingFn func(seriesID ident.ID, tags ident.Tags) bool +type ForEachRemainingFn func(seriesID ident.ID, tags ident.Tags) error // MergeWith is an interface that the fs merger uses to merge data with. type MergeWith interface { @@ -517,7 +517,6 @@ type Merger interface { fileID FileSetFileIdentifier, mergeWith MergeWith, flushPreparer persist.FlushPreparer, - nsOpts namespace.Options, nsCtx namespace.Context, ) error } @@ -526,8 +525,10 @@ type Merger interface { // convenience for testing. type NewMergerFn func( reader DataFileSetReader, + blockAllocSize int, srPool xio.SegmentReaderPool, multiIterPool encoding.MultiReaderIteratorPool, identPool ident.Pool, encoderPool encoding.EncoderPool, + nsOpts namespace.Options, ) Merger diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 15ed8dedbf..8545e29a23 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -131,6 +131,11 @@ func (m *flushManager) Flush( multiErr = multiErr.Add(err) } + // Snapshots must be done after a cold flush because cleanup logic just + // checks for a successful snapshot checkpoint file. If snapshotting + // went first and succeeded and then a cold flush fails, the old + // commitlogs would still get cleaned up. If the node then restarted, + // cold writes would be lost. if err = m.dataSnapshot(namespaces, tickStart, rotatedCommitlogID); err != nil { multiErr = multiErr.Add(err) } diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index 708bcc2566..47b542a3b9 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -21,8 +21,6 @@ package storage import ( - "errors" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/storage/series" @@ -32,7 +30,12 @@ import ( xtime "github.com/m3db/m3/src/x/time" ) -// fsMergeWithMem implements fs.MergeWith +// fsMergeWithMem implements fs.MergeWith, where the merge target is data in +// memory. It relies on data structures being passed in that are created +// within the shard informing fsMergeWithMem which series require merging. +// These data structures enable efficient reading of data as well as keeping +// track of which series were read so that the remaining series can be looped +// through. type fsMergeWithMem struct { shard databaseShard retriever series.QueryableBlockRetriever @@ -99,9 +102,9 @@ func (m *fsMergeWithMem) ForEachRemaining(blockStart xtime.UnixNano, fn fs.ForEa return err } - success := fn(seriesID, tags) - if !success { - return errors.New("foreach iteration unsuccessful") + err = fn(seriesID, tags) + if err != nil { + return err } } diff --git a/src/dbnode/storage/fs_merge_with_mem_test.go b/src/dbnode/storage/fs_merge_with_mem_test.go index f425d6d145..99ff346563 100644 --- a/src/dbnode/storage/fs_merge_with_mem_test.go +++ b/src/dbnode/storage/fs_merge_with_mem_test.go @@ -161,9 +161,9 @@ func TestForEachRemaining(t *testing.T) { var forEachCalls []ident.ID shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) - mergeWith.ForEachRemaining(0, func(seriesID ident.ID, tags ident.Tags) bool { + mergeWith.ForEachRemaining(0, func(seriesID ident.ID, tags ident.Tags) error { forEachCalls = append(forEachCalls, seriesID) - return true + return nil }) require.Len(t, forEachCalls, 2) assert.Equal(t, id0, forEachCalls[0]) @@ -181,9 +181,9 @@ func TestForEachRemaining(t *testing.T) { assert.True(t, exists) assert.Equal(t, fetchedBlocks, res) shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) - err = mergeWith.ForEachRemaining(1, func(seriesID ident.ID, tags ident.Tags) bool { + err = mergeWith.ForEachRemaining(1, func(seriesID ident.ID, tags ident.Tags) error { forEachCalls = append(forEachCalls, seriesID) - return true + return nil }) require.NoError(t, err) require.Len(t, forEachCalls, 2) @@ -193,17 +193,17 @@ func TestForEachRemaining(t *testing.T) { // Test call with error getting tags. shard.EXPECT(). TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, errors.New("bad-tags")) - err = mergeWith.ForEachRemaining(4, func(seriesID ident.ID, tags ident.Tags) bool { + err = mergeWith.ForEachRemaining(4, func(seriesID ident.ID, tags ident.Tags) error { // This function won't be called with the above error. - return true + return errors.New("unreachable") }) assert.Error(t, err) // Test call with bad function execution. shard.EXPECT(). TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil) - err = mergeWith.ForEachRemaining(4, func(seriesID ident.ID, tags ident.Tags) bool { - return false + err = mergeWith.ForEachRemaining(4, func(seriesID ident.ID, tags ident.Tags) error { + return errors.New("bad") }) assert.Error(t, err) } diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 4280fea267..67190c661a 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -981,9 +981,16 @@ type idAndBlockStart struct { type coldFlushReuseableResources struct { // dirtySeries is a map from a composite key of - // to an element in a list in the dirtySeriesToWrite map. This composite key - // is deliberately made so that this map stays one level deep, making it - // easier to share between shard loops, minimizing the need for allocations. + // to an element in a list in the dirtySeriesToWrite map. This map is used + // to quickly test whether a series is dirty for a particular block start. + // + // The composite key is deliberately made so that this map stays one level + // deep, making it easier to share between shard loops, minimizing the need + // for allocations. + // + // Having a reference to the element in the dirtySeriesToWrite list enables + // efficient removal of the series that have been read and subsequent + // iterating through remaining series to be read. dirtySeries *dirtySeriesMap // dirtySeriesToWrite is a map from block start to a list of dirty series // that have yet to be written to disk. diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 59f0ca8c91..ed2030b972 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -600,7 +600,7 @@ func (b *dbBuffer) FetchBlocksForColdFlush( // The lifecycle of calling this function is preceded by first checking // which blocks have cold data that have not yet been flushed. // If we don't get data here, it means that it has since fallen out of - // retention and has evicted. + // retention and has been evicted. return nil, nil } if len(res) != 1 { diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 9dd7c86ac3..698f3c0a23 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -310,9 +310,10 @@ func (s *dbSeries) FetchBlocksForColdFlush( // This needs a write lock because the version on underlying buckets need // to be modified. s.Lock() - defer s.Unlock() + br, err := s.buffer.FetchBlocksForColdFlush(ctx, start, version, nsCtx) + s.Unlock() - return s.buffer.FetchBlocksForColdFlush(ctx, start, version, nsCtx) + return br, err } func (s *dbSeries) FetchBlocks( diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 7436c07e3d..e4cb1dd9ab 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1968,8 +1968,9 @@ func (s *dbShard) ColdFlush( return true }) - merger := s.newMergerFn(resources.fsReader, s.opts.SegmentReaderPool(), - s.opts.MultiReaderIteratorPool(), s.opts.IdentifierPool(), s.opts.EncoderPool()) + merger := s.newMergerFn(resources.fsReader, s.opts.DatabaseBlockOptions().DatabaseBlockAllocSize(), + s.opts.SegmentReaderPool(), s.opts.MultiReaderIteratorPool(), + s.opts.IdentifierPool(), s.opts.EncoderPool(), s.namespace.Options()) mergeWithMem := s.newFSMergeWithMemFn(s, s, dirtySeries, dirtySeriesToWrite) // Loop through each block that we know has ColdWrites. Since each block // has its own fileset, if we encounter an error while trying to persist @@ -1982,7 +1983,7 @@ func (s *dbShard) ColdFlush( BlockStart: startTime, } - err := merger.Merge(fsID, mergeWithMem, flushPreparer, s.namespace.Options(), nsCtx) + err := merger.Merge(fsID, mergeWithMem, flushPreparer, nsCtx) if err != nil { multiErr = multiErr.Add(err) continue diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 6d4daaabec..ef2e043655 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -396,10 +396,12 @@ func TestShardColdFlush(t *testing.T) { func newMergerTestFn( reader fs.DataFileSetReader, + blockAllocSize int, srPool xio.SegmentReaderPool, multiIterPool encoding.MultiReaderIteratorPool, identPool ident.Pool, encoderPool encoding.EncoderPool, + nsOpts namespace.Options, ) fs.Merger { return &noopMerger{} } @@ -410,7 +412,6 @@ func (m *noopMerger) Merge( fileID fs.FileSetFileIdentifier, mergeWith fs.MergeWith, flushPreparer persist.FlushPreparer, - nsOpts namespace.Options, nsCtx namespace.Context, ) error { return nil diff --git a/src/dbnode/x/xio/block_reader_slice_from_segment_reader_slice.go b/src/dbnode/x/xio/block_reader_slice_from_segment_reader_slice.go deleted file mode 100644 index 604924cfc9..0000000000 --- a/src/dbnode/x/xio/block_reader_slice_from_segment_reader_slice.go +++ /dev/null @@ -1,35 +0,0 @@ -// 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 xio - -// NewSegmentReaderSliceFromBlockReaderSlice creates a new SegmentReader slice -// from a BlockReader slice. -func NewSegmentReaderSliceFromBlockReaderSlice( - brs []BlockReader, -) []SegmentReader { - srs := make([]SegmentReader, 0, len(brs)) - - for _, br := range brs { - srs = append(srs, br.SegmentReader) - } - - return srs -} From acc1fb53f8017a781fb9085f4d0a83d2cc499032 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Sun, 2 Jun 2019 22:07:35 -0400 Subject: [PATCH 11/18] Don't snapshot if cold flush fails --- src/dbnode/persist/fs/fs_mock.go | 9 +++-- src/dbnode/persist/fs/merger.go | 40 +++++++++++--------- src/dbnode/persist/fs/merger_test.go | 7 ++-- src/dbnode/persist/fs/types.go | 5 ++- src/dbnode/storage/flush.go | 6 +++ src/dbnode/storage/flush_test.go | 11 +++--- src/dbnode/storage/fs_merge_with_mem.go | 6 +-- src/dbnode/storage/fs_merge_with_mem_test.go | 13 ++++--- src/dbnode/storage/series/buffer.go | 20 +++++----- src/dbnode/storage/series/buffer_mock.go | 12 +++--- src/dbnode/storage/series/buffer_test.go | 6 +-- src/dbnode/storage/series/series.go | 4 +- src/dbnode/storage/series/series_mock.go | 28 +++++++------- src/dbnode/storage/series/types.go | 4 +- src/dbnode/storage/shard.go | 2 +- src/dbnode/storage/shard_test.go | 3 +- 16 files changed, 98 insertions(+), 78 deletions(-) diff --git a/src/dbnode/persist/fs/fs_mock.go b/src/dbnode/persist/fs/fs_mock.go index ad4ae80a72..2e01ba034c 100644 --- a/src/dbnode/persist/fs/fs_mock.go +++ b/src/dbnode/persist/fs/fs_mock.go @@ -33,6 +33,7 @@ import ( "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/m3ninx/persist" "github.com/m3db/m3/src/x/checked" + "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" time0 "github.com/m3db/m3/src/x/time" @@ -1204,9 +1205,9 @@ func (mr *MockMergeWithMockRecorder) ForEachRemaining(arg0, arg1 interface{}) *g } // Read mocks base method -func (m *MockMergeWith) Read(arg0 ident.ID, arg1 time0.UnixNano, arg2 namespace.Context) ([]xio.BlockReader, bool, error) { +func (m *MockMergeWith) Read(arg0 context.Context, arg1 ident.ID, arg2 time0.UnixNano, arg3 namespace.Context) ([]xio.BlockReader, bool, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Read", arg0, arg1, arg2) + ret := m.ctrl.Call(m, "Read", arg0, arg1, arg2, arg3) ret0, _ := ret[0].([]xio.BlockReader) ret1, _ := ret[1].(bool) ret2, _ := ret[2].(error) @@ -1214,7 +1215,7 @@ func (m *MockMergeWith) Read(arg0 ident.ID, arg1 time0.UnixNano, arg2 namespace. } // Read indicates an expected call of Read -func (mr *MockMergeWithMockRecorder) Read(arg0, arg1, arg2 interface{}) *gomock.Call { +func (mr *MockMergeWithMockRecorder) Read(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Read", reflect.TypeOf((*MockMergeWith)(nil).Read), arg0, arg1, arg2) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Read", reflect.TypeOf((*MockMergeWith)(nil).Read), arg0, arg1, arg2, arg3) } diff --git a/src/dbnode/persist/fs/merger.go b/src/dbnode/persist/fs/merger.go index 20b0376a31..a5f88290e9 100644 --- a/src/dbnode/persist/fs/merger.go +++ b/src/dbnode/persist/fs/merger.go @@ -32,6 +32,7 @@ import ( "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/x/checked" + "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" ) @@ -80,7 +81,7 @@ func (m *merger) Merge( mergeWith MergeWith, flushPreparer persist.FlushPreparer, nsCtx namespace.Context, -) error { +) (err error) { var ( reader = m.reader blockAllocSize = m.blockAllocSize @@ -108,7 +109,13 @@ func (m *merger) Merge( if err := reader.Open(openOpts); err != nil { return err } - defer reader.Close() + defer func() { + // Only set the error here if not set by the end of the function, since + // all other errors take precedence. + if err == nil { + err = reader.Close() + } + }() nsMd, err := namespace.NewMetadata(nsID, nsOpts) if err != nil { @@ -141,23 +148,26 @@ func (m *merger) Merge( // Initialize this here with nil to be reset before each iteration's // use. sliceOfSlices = xio.NewReaderSliceOfSlicesFromBlockReadersIterator(nil) + // Reused context for use in mergeWith.Read, since they all do a + // BlockingClose after usage. + ctx = context.NewContext() // We keep track of IDs/tags to finalize at the end of merging. This // only applies to those that come from disk Reads, since the whole - // lifecycle of those IDs/tags are contained to this function. We don't - // want finalize this series ID since it is from memory and other - // components may have ownership over it. + // lifecycle of those IDs/tags are contained to this function. We don't + // want finalize the IDs from memory since other components may have + // ownership over it. // // We must only finalize these at the end of this function, since the // flush preparer's underlying writer holds on to those references // until it is closed (closing the PreparedDataPersist at the end of // this merge closes the underlying writer). - idsToFinalize []ident.ID - tagsToFinalize []ident.Tags + idsToFinalize = make([]ident.ID, 0, reader.Entries()) + tagsToFinalize = make([]ident.Tags, 0, reader.Entries()) ) - defer segReader.Finalize() - defer multiIter.Close() defer func() { + segReader.Finalize() + multiIter.Close() for _, res := range idsToFinalize { res.Finalize() } @@ -186,7 +196,7 @@ func (m *merger) Merge( br = append(br, blockReaderFromData(data, segReader, startTime, blockSize)) // Check if this series is in memory (and thus requires merging). - mergeWithData, hasData, err := mergeWith.Read(id, blockStart, nsCtx) + mergeWithData, hasData, err := mergeWith.Read(ctx, id, blockStart, nsCtx) if err != nil { return err } @@ -213,7 +223,7 @@ func (m *merger) Merge( // Second stage: loop through rest of the merge target that was not captured // in the first stage. err = mergeWith.ForEachRemaining(blockStart, func(seriesID ident.ID, tags ident.Tags) error { - mergeWithData, hasData, err := mergeWith.Read(seriesID, blockStart, nsCtx) + mergeWithData, hasData, err := mergeWith.Read(ctx, seriesID, blockStart, nsCtx) if err != nil { return err } @@ -272,16 +282,12 @@ func persistIter( } } if err := it.Err(); err != nil { + encoder.Close() return err } segment := encoder.Discard() checksum := digest.SegmentChecksum(segment) - err := persistFn(id, tags, segment, checksum) - if err != nil { - return err - } - - return nil + return persistFn(id, tags, segment, checksum) } diff --git a/src/dbnode/persist/fs/merger_test.go b/src/dbnode/persist/fs/merger_test.go index 6ea60a85d2..821e7a5eee 100644 --- a/src/dbnode/persist/fs/merger_test.go +++ b/src/dbnode/persist/fs/merger_test.go @@ -506,6 +506,7 @@ func mockReaderFromData( ) *MockDataFileSetReader { reader := NewMockDataFileSetReader(ctrl) reader.EXPECT().Open(gomock.Any()).Return(nil) + reader.EXPECT().Entries().Return(diskData.Len()).Times(2) reader.EXPECT().Close().Return(nil) tagIter := ident.NewTagsIterator(ident.NewTags(ident.StringTag("tag-key0", "tag-val0"))) fakeChecksum := uint32(42) @@ -548,10 +549,10 @@ func mockMergeWithFromData( require.True(t, ok) segReader := srPool.Get() br := []xio.BlockReader{blockReaderFromData(data, segReader, startTime, blockSize)} - mergeWith.EXPECT().Read(id, gomock.Any(), gomock.Any()). + mergeWith.EXPECT().Read(gomock.Any(), id, gomock.Any(), gomock.Any()). Return(br, true, nil) } else { - mergeWith.EXPECT().Read(id, gomock.Any(), gomock.Any()). + mergeWith.EXPECT().Read(gomock.Any(), id, gomock.Any(), gomock.Any()). Return(nil, false, nil) } } @@ -561,7 +562,7 @@ func mockMergeWithFromData( data := val.Value() segReader := srPool.Get() br := []xio.BlockReader{blockReaderFromData(data, segReader, startTime, blockSize)} - mergeWith.EXPECT().Read(id, gomock.Any(), gomock.Any()). + mergeWith.EXPECT().Read(gomock.Any(), id, gomock.Any(), gomock.Any()). Return(br, true, nil) // Capture remaining items so that we can call the ForEachRemaining diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index fc87b3bea5..c477800149 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -36,6 +36,7 @@ import ( "github.com/m3db/m3/src/m3ninx/index/segment/fst" idxpersist "github.com/m3db/m3/src/m3ninx/persist" "github.com/m3db/m3/src/x/checked" + "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/instrument" "github.com/m3db/m3/src/x/pool" @@ -500,6 +501,7 @@ type MergeWith interface { // Read returns the data for the given block start and series ID, whether // any data was found, and the error encountered (if any). Read( + ctx context.Context, seriesID ident.ID, blockStart xtime.UnixNano, nsCtx namespace.Context, @@ -521,8 +523,7 @@ type Merger interface { ) error } -// NewMergerFn is the function to call to get a new Merger. Mostly used as a -// convenience for testing. +// NewMergerFn is the function to call to get a new Merger. type NewMergerFn func( reader DataFileSetReader, blockAllocSize int, diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 8545e29a23..4729895d5b 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -129,6 +129,12 @@ func (m *flushManager) Flush( if err == nil { if err = m.dataColdFlush(namespaces); err != nil { multiErr = multiErr.Add(err) + // If cold flush fails, we can't proceed to snapshotting because + // commit log logic just checks for a successful snapshot checkpoint + // file. Therefore if a cold flush fails and a snapshot succeeds, + // the writes from the failed cold flush might be lost when commit + // logs get cleaned up. + return multiErr.FinalError() } // Snapshots must be done after a cold flush because cleanup logic just diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index a62fe67931..ba61e144af 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -31,7 +31,6 @@ import ( "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3/src/dbnode/retention" - xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" xtest "github.com/m3db/m3/src/x/test" @@ -171,7 +170,10 @@ func TestFlushManagerFlushDoneFlushError(t *testing.T) { mockSnapshotPersist = persist.NewMockSnapshotPreparer(ctrl) ) - mockFlushPersist.EXPECT().DoneFlush().Return(fakeErr).Times(2) + gomock.InOrder( + mockFlushPersist.EXPECT().DoneFlush().Return(fakeErr), + mockFlushPersist.EXPECT().DoneFlush().Return(nil), + ) mockPersistManager.EXPECT().StartFlushPersist().Return(mockFlushPersist, nil).Times(2) mockSnapshotPersist.EXPECT().DoneSnapshot(gomock.Any(), testCommitlogFile).Return(nil) @@ -193,10 +195,7 @@ func TestFlushManagerFlushDoneFlushError(t *testing.T) { fm.pm = mockPersistManager now := time.Unix(0, 0) - multiErr := xerrors.NewMultiError() - multiErr = multiErr.Add(fakeErr) - multiErr = multiErr.Add(fakeErr) - require.EqualError(t, multiErr.FinalError(), fm.Flush(now, DatabaseBootstrapState{}).Error()) + require.EqualError(t, fakeErr, fm.Flush(now, DatabaseBootstrapState{}).Error()) } // TestFlushManagerFlushDoneSnapshotError makes sure that snapshot errors do not diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index 47b542a3b9..c0a130b129 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -58,6 +58,7 @@ func newFSMergeWithMem( } func (m *fsMergeWithMem) Read( + ctx context.Context, seriesID ident.ID, blockStart xtime.UnixNano, nsCtx namespace.Context, @@ -78,9 +79,8 @@ func (m *fsMergeWithMem) Read( nextVersion := m.retriever.RetrievableBlockColdVersion(startTime) + 1 - tmpCtx := context.NewContext() - blocks, err := m.shard.FetchBlocksForColdFlush(tmpCtx, element.Value, startTime, nextVersion, nsCtx) - tmpCtx.BlockingClose() + blocks, err := m.shard.FetchBlocksForColdFlush(ctx, element.Value, startTime, nextVersion, nsCtx) + ctx.BlockingClose() if err != nil { return nil, false, err } diff --git a/src/dbnode/storage/fs_merge_with_mem_test.go b/src/dbnode/storage/fs_merge_with_mem_test.go index 99ff346563..1e45718cbc 100644 --- a/src/dbnode/storage/fs_merge_with_mem_test.go +++ b/src/dbnode/storage/fs_merge_with_mem_test.go @@ -27,6 +27,7 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/x/xio" + "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" @@ -47,6 +48,7 @@ func TestRead(t *testing.T) { shard := NewMockdatabaseShard(ctrl) retriever := series.NewMockQueryableBlockRetriever(ctrl) version := 0 + ctx := context.NewContext() nsCtx := namespace.Context{} fetchedBlocks := []xio.BlockReader{xio.BlockReader{}} retriever.EXPECT().RetrievableBlockColdVersion(gomock.Any()).Return(version).AnyTimes() @@ -79,7 +81,7 @@ func TestRead(t *testing.T) { for _, d := range data { require.True(t, dirtySeries.Contains(idAndBlockStart{blockStart: d.start, id: d.id})) beforeLen := dirtySeriesToWrite[d.start].Len() - res, exists, err := mergeWith.Read(d.id, d.start, nsCtx) + res, exists, err := mergeWith.Read(ctx, d.id, d.start, nsCtx) require.NoError(t, err) assert.True(t, exists) assert.Equal(t, fetchedBlocks, res) @@ -89,7 +91,7 @@ func TestRead(t *testing.T) { } // Test Read with non-existent dirty block/series. - res, exists, err := mergeWith.Read(ident.StringID("not-present"), 10, nsCtx) + res, exists, err := mergeWith.Read(ctx, ident.StringID("not-present"), 10, nsCtx) assert.Nil(t, res) assert.False(t, exists) assert.NoError(t, err) @@ -100,7 +102,7 @@ func TestRead(t *testing.T) { shard.EXPECT(). FetchBlocksForColdFlush(gomock.Any(), badFetchID, gomock.Any(), version+1, nsCtx). Return(nil, errors.New("fetch error")) - res, exists, err = mergeWith.Read(badFetchID, 11, nsCtx) + res, exists, err = mergeWith.Read(ctx, badFetchID, 11, nsCtx) assert.Nil(t, res) assert.False(t, exists) assert.Error(t, err) @@ -111,7 +113,7 @@ func TestRead(t *testing.T) { shard.EXPECT(). FetchBlocksForColdFlush(gomock.Any(), emptyDataID, gomock.Any(), version+1, nsCtx). Return(nil, nil) - res, exists, err = mergeWith.Read(emptyDataID, 12, nsCtx) + res, exists, err = mergeWith.Read(ctx, emptyDataID, 12, nsCtx) assert.Nil(t, res) assert.False(t, exists) assert.NoError(t, err) @@ -124,6 +126,7 @@ func TestForEachRemaining(t *testing.T) { shard := NewMockdatabaseShard(ctrl) retriever := series.NewMockQueryableBlockRetriever(ctrl) version := 0 + ctx := context.NewContext() nsCtx := namespace.Context{} fetchedBlocks := []xio.BlockReader{xio.BlockReader{}} retriever.EXPECT().RetrievableBlockColdVersion(gomock.Any()).Return(version).AnyTimes() @@ -176,7 +179,7 @@ func TestForEachRemaining(t *testing.T) { shard.EXPECT(). FetchBlocksForColdFlush(gomock.Any(), id3, xtime.UnixNano(1).ToTime(), version+1, nsCtx). Return(fetchedBlocks, nil) - res, exists, err := mergeWith.Read(id3, 1, nsCtx) + res, exists, err := mergeWith.Read(ctx, id3, 1, nsCtx) require.NoError(t, err) assert.True(t, exists) assert.Equal(t, fetchedBlocks, res) diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index ed2030b972..6855e230e5 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -66,8 +66,10 @@ const ( // block being recently flushed // 2) The number of buckets that contain ColdWrites within a cold flush // cycle + // TODO(juchan): revisit this after ColdWrites usage to see if this number + // is sane. optimizedTimesArraySize = 8 - writableBucketVer = 0 + writableBucketVersion = 0 ) type databaseBuffer interface { @@ -125,7 +127,7 @@ type databaseBuffer interface { IsEmpty() bool - NeedsColdFlushBlockStarts() OptimizedTimes + ColdFlushBlockStarts() OptimizedTimes Stats() bufferStats @@ -309,12 +311,12 @@ func (b *dbBuffer) IsEmpty() bool { return len(b.bucketsMap) == 0 } -func (b *dbBuffer) NeedsColdFlushBlockStarts() OptimizedTimes { +func (b *dbBuffer) ColdFlushBlockStarts() OptimizedTimes { var times OptimizedTimes for t, bucketVersions := range b.bucketsMap { for _, bucket := range bucketVersions.buckets { - if bucket.version == writableBucketVer && bucket.writeType == ColdWrite { + if bucket.version == writableBucketVersion && bucket.writeType == ColdWrite { times.Add(t) break } @@ -816,7 +818,7 @@ func (b *dbBuffer) inOrderBlockStartsRemove(removeTime time.Time) { // BufferBucketVersions is a container for different versions of buffer buckets. // Bucket versions are how the buffer separates writes that have been written // to disk as a fileset and writes that have not. The bucket with a version of -// `writableBucketVer` is the bucket that all writes go into (as thus is the +// `writableBucketVersion` is the bucket that all writes go into (as thus is the // bucket version that have not yet been persisted). After a bucket gets // persisted, its version gets set to a version that the shard passes down to it // (since the shard knows what has been fully persisted to disk). @@ -879,7 +881,7 @@ func (b *BufferBucketVersions) merge(writeType WriteType, nsCtx namespace.Contex res := 0 for _, bucket := range b.buckets { // Only makes sense to merge buckets that are writable. - if bucket.version == writableBucketVer && writeType == bucket.writeType { + if bucket.version == writableBucketVersion && writeType == bucket.writeType { merges, err := bucket.merge(nsCtx) if err != nil { return 0, err @@ -900,7 +902,7 @@ func (b *BufferBucketVersions) removeBucketsUpToVersion( for _, bucket := range b.buckets { bVersion := bucket.version - if bucket.writeType == writeType && bVersion != writableBucketVer && + if bucket.writeType == writeType && bVersion != writableBucketVersion && bVersion <= version { // We no longer need to keep any version which is equal to // or less than the retrievable version, since that means @@ -931,7 +933,7 @@ func (b *BufferBucketVersions) bootstrap(bl block.DatabaseBlock) { func (b *BufferBucketVersions) writableBucket(writeType WriteType) (*BufferBucket, bool) { for _, bucket := range b.buckets { - if bucket.version == writableBucketVer && bucket.writeType == writeType { + if bucket.version == writableBucketVersion && bucket.writeType == writeType { return bucket, true } } @@ -1014,7 +1016,7 @@ func (b *BufferBucket) resetTo( }) b.bootstrapped = nil // We would only ever create a bucket for it to be writable. - b.version = writableBucketVer + b.version = writableBucketVersion b.writeType = writeType } diff --git a/src/dbnode/storage/series/buffer_mock.go b/src/dbnode/storage/series/buffer_mock.go index 42b3894bfd..ec1a5eb2c6 100644 --- a/src/dbnode/storage/series/buffer_mock.go +++ b/src/dbnode/storage/series/buffer_mock.go @@ -179,18 +179,18 @@ func (mr *MockdatabaseBufferMockRecorder) IsEmpty() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsEmpty", reflect.TypeOf((*MockdatabaseBuffer)(nil).IsEmpty)) } -// NeedsColdFlushBlockStarts mocks base method -func (m *MockdatabaseBuffer) NeedsColdFlushBlockStarts() OptimizedTimes { +// ColdFlushBlockStarts mocks base method +func (m *MockdatabaseBuffer) ColdFlushBlockStarts() OptimizedTimes { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "NeedsColdFlushBlockStarts") + ret := m.ctrl.Call(m, "ColdFlushBlockStarts") ret0, _ := ret[0].(OptimizedTimes) return ret0 } -// NeedsColdFlushBlockStarts indicates an expected call of NeedsColdFlushBlockStarts -func (mr *MockdatabaseBufferMockRecorder) NeedsColdFlushBlockStarts() *gomock.Call { +// ColdFlushBlockStarts indicates an expected call of ColdFlushBlockStarts +func (mr *MockdatabaseBufferMockRecorder) ColdFlushBlockStarts() *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NeedsColdFlushBlockStarts", reflect.TypeOf((*MockdatabaseBuffer)(nil).NeedsColdFlushBlockStarts)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ColdFlushBlockStarts", reflect.TypeOf((*MockdatabaseBuffer)(nil).ColdFlushBlockStarts)) } // Stats mocks base method diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index 5592d8a4b4..54f2db34a6 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -1201,7 +1201,7 @@ func TestOptimizedTimes(t *testing.T) { } } -func TestNeedsColdFlushBlockStarts(t *testing.T) { +func TestColdFlushBlockStarts(t *testing.T) { opts := newBufferTestOptions() rops := opts.RetentionOptions() blockSize := rops.BlockSize() @@ -1261,7 +1261,7 @@ func TestNeedsColdFlushBlockStarts(t *testing.T) { } buffer, _ := newTestBufferWithCustomData(t, bds, opts, nil) - flushStarts := buffer.NeedsColdFlushBlockStarts() + flushStarts := buffer.ColdFlushBlockStarts() // All three cold blocks should report that they are dirty. assert.Equal(t, 3, flushStarts.Len()) @@ -1274,7 +1274,7 @@ func TestNeedsColdFlushBlockStarts(t *testing.T) { buffer.bucketsMap[xtime.ToUnixNano(blockStart2)].buckets[0].version = 1 buffer.bucketsMap[xtime.ToUnixNano(blockStart3)].buckets[0].version = 1 - flushStarts = buffer.NeedsColdFlushBlockStarts() + flushStarts = buffer.ColdFlushBlockStarts() assert.Equal(t, 1, flushStarts.Len()) assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart1))) } diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 698f3c0a23..9b880f7506 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -566,11 +566,11 @@ func (s *dbSeries) Snapshot( return s.buffer.Snapshot(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) } -func (s *dbSeries) NeedsColdFlushBlockStarts() OptimizedTimes { +func (s *dbSeries) ColdFlushBlockStarts() OptimizedTimes { s.RLock() defer s.RUnlock() - return s.buffer.NeedsColdFlushBlockStarts() + return s.buffer.ColdFlushBlockStarts() } func (s *dbSeries) Close() { diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 4649eb4dce..480b675708 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -90,6 +90,20 @@ func (mr *MockDatabaseSeriesMockRecorder) Close() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockDatabaseSeries)(nil).Close)) } +// ColdFlushBlockStarts mocks base method +func (m *MockDatabaseSeries) ColdFlushBlockStarts() OptimizedTimes { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ColdFlushBlockStarts") + ret0, _ := ret[0].(OptimizedTimes) + return ret0 +} + +// ColdFlushBlockStarts indicates an expected call of ColdFlushBlockStarts +func (mr *MockDatabaseSeriesMockRecorder) ColdFlushBlockStarts() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ColdFlushBlockStarts", reflect.TypeOf((*MockDatabaseSeries)(nil).ColdFlushBlockStarts)) +} + // FetchBlocks mocks base method func (m *MockDatabaseSeries) FetchBlocks(arg0 context.Context, arg1 []time.Time, arg2 namespace.Context) ([]block.FetchBlockResult, error) { m.ctrl.T.Helper() @@ -177,20 +191,6 @@ func (mr *MockDatabaseSeriesMockRecorder) IsEmpty() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsEmpty", reflect.TypeOf((*MockDatabaseSeries)(nil).IsEmpty)) } -// NeedsColdFlushBlockStarts mocks base method -func (m *MockDatabaseSeries) NeedsColdFlushBlockStarts() OptimizedTimes { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "NeedsColdFlushBlockStarts") - ret0, _ := ret[0].(OptimizedTimes) - return ret0 -} - -// NeedsColdFlushBlockStarts indicates an expected call of NeedsColdFlushBlockStarts -func (mr *MockDatabaseSeriesMockRecorder) NeedsColdFlushBlockStarts() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NeedsColdFlushBlockStarts", reflect.TypeOf((*MockDatabaseSeries)(nil).NeedsColdFlushBlockStarts)) -} - // NumActiveBlocks mocks base method func (m *MockDatabaseSeries) NumActiveBlocks() int { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index 8dd4834b43..1b8e27a356 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -122,8 +122,8 @@ type DatabaseSeries interface { nsCtx namespace.Context, ) error - // NeedsColdFlushBlockStarts returns the block starts that need cold flushes. - NeedsColdFlushBlockStarts() OptimizedTimes + // ColdFlushBlockStarts returns the block starts that need cold flushes. + ColdFlushBlockStarts() OptimizedTimes // Close will close the series and if pooled returned to the pool. Close() diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index e4cb1dd9ab..0fd662ff1d 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1953,7 +1953,7 @@ func (s *dbShard) ColdFlush( s.forEachShardEntry(func(entry *lookup.Entry) bool { curr := entry.Series seriesID := curr.ID() - blockStarts := curr.NeedsColdFlushBlockStarts() + blockStarts := curr.ColdFlushBlockStarts() blockStarts.ForEach(func(t xtime.UnixNano) { seriesList := dirtySeriesToWrite[t] if seriesList == nil { diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index ef2e043655..f201e8493b 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -367,7 +367,7 @@ func TestShardColdFlush(t *testing.T) { for _, ds := range dirtyData { curr := series.NewMockDatabaseSeries(ctrl) curr.EXPECT().ID().Return(ds.id) - curr.EXPECT().NeedsColdFlushBlockStarts(). + curr.EXPECT().ColdFlushBlockStarts(). Return(optimizedTimesFromTimes(ds.dirtyTimes)) shard.list.PushBack(lookup.NewEntry(curr, 0)) } @@ -429,6 +429,7 @@ func newFSMergeWithMemTestFn( type noopMergeWith struct{} func (m *noopMergeWith) Read( + ctx context.Context, seriesID ident.ID, blockStart xtime.UnixNano, nsCtx namespace.Context, From 945e9cde3e395c31520eb421b04fff5ab37d3f5c Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Tue, 4 Jun 2019 19:05:46 -0400 Subject: [PATCH 12/18] Pass block states down to get correct blocks to cold flush --- src/dbnode/persist/fs/fs_mock.go | 8 ++-- src/dbnode/persist/fs/merger.go | 45 +++++++++--------- src/dbnode/persist/fs/merger_test.go | 18 +++---- src/dbnode/persist/fs/types.go | 9 +++- src/dbnode/storage/flush.go | 16 +++---- src/dbnode/storage/fs_merge_with_mem.go | 16 +++++-- src/dbnode/storage/fs_merge_with_mem_test.go | 16 +++---- src/dbnode/storage/namespace.go | 26 ++++++---- src/dbnode/storage/series/buffer.go | 16 +++++-- src/dbnode/storage/series/buffer_mock.go | 8 ++-- src/dbnode/storage/series/buffer_test.go | 50 +++++++++++++++++--- src/dbnode/storage/series/series.go | 4 +- src/dbnode/storage/series/series_mock.go | 8 ++-- src/dbnode/storage/series/types.go | 2 +- src/dbnode/storage/shard.go | 15 +++++- src/dbnode/storage/shard_test.go | 22 +++++++-- 16 files changed, 188 insertions(+), 91 deletions(-) diff --git a/src/dbnode/persist/fs/fs_mock.go b/src/dbnode/persist/fs/fs_mock.go index 2e01ba034c..e69dd50653 100644 --- a/src/dbnode/persist/fs/fs_mock.go +++ b/src/dbnode/persist/fs/fs_mock.go @@ -1191,17 +1191,17 @@ func (m *MockMergeWith) EXPECT() *MockMergeWithMockRecorder { } // ForEachRemaining mocks base method -func (m *MockMergeWith) ForEachRemaining(arg0 time0.UnixNano, arg1 ForEachRemainingFn) error { +func (m *MockMergeWith) ForEachRemaining(arg0 context.Context, arg1 time0.UnixNano, arg2 ForEachRemainingFn, arg3 namespace.Context) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ForEachRemaining", arg0, arg1) + ret := m.ctrl.Call(m, "ForEachRemaining", arg0, arg1, arg2, arg3) ret0, _ := ret[0].(error) return ret0 } // ForEachRemaining indicates an expected call of ForEachRemaining -func (mr *MockMergeWithMockRecorder) ForEachRemaining(arg0, arg1 interface{}) *gomock.Call { +func (mr *MockMergeWithMockRecorder) ForEachRemaining(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ForEachRemaining", reflect.TypeOf((*MockMergeWith)(nil).ForEachRemaining), arg0, arg1) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ForEachRemaining", reflect.TypeOf((*MockMergeWith)(nil).ForEachRemaining), arg0, arg1, arg2, arg3) } // Read mocks base method diff --git a/src/dbnode/persist/fs/merger.go b/src/dbnode/persist/fs/merger.go index a5f88290e9..a60573a952 100644 --- a/src/dbnode/persist/fs/merger.go +++ b/src/dbnode/persist/fs/merger.go @@ -48,10 +48,11 @@ type merger struct { } // NewMerger returns a new Merger. This implementation is in charge of merging -// the data from an existing fileset with a merge target, giving precendence -// to the data from the merge target. This merged data is then persisted. +// the data from an existing fileset with a merge target. If data for a series +// at a timestamp exists both on disk and the merge target, data from the merge +// target will be used. This merged data is then persisted. // -// Note that the merger it does not know how or where this merged data is +// Note that the merger does not know how or where this merged data is // persisted since it just uses the flushPreparer that is passed in. Further, // it does not signal to the database of the existence of the newly persisted // data, nor does it clean up the original fileset. @@ -76,6 +77,8 @@ func NewMerger( } // Merge merges data from a fileset with a merge target and persists it. +// The caller is responsible for finalizing all resources used for the +// MergeWith passed here. func (m *merger) Merge( fileID FileSetFileIdentifier, mergeWith MergeWith, @@ -140,9 +143,9 @@ func (m *merger) Merge( // one for data from the merge target. br = make([]xio.BlockReader, 0, 2) - // It's safe to share these between iterations and just reset them each time - // because the series gets persisted each loop, so previous iterations' - // reader and iterator will never be needed. + // It's safe to share these between iterations and just reset them each + // time because the series gets persisted each loop, so the previous + // iterations' reader and iterator will never be needed. segReader = srPool.Get() multiIter = multiIterPool.Get() // Initialize this here with nil to be reset before each iteration's @@ -150,7 +153,7 @@ func (m *merger) Merge( sliceOfSlices = xio.NewReaderSliceOfSlicesFromBlockReadersIterator(nil) // Reused context for use in mergeWith.Read, since they all do a // BlockingClose after usage. - ctx = context.NewContext() + tmpCtx = context.NewContext() // We keep track of IDs/tags to finalize at the end of merging. This // only applies to those that come from disk Reads, since the whole @@ -196,7 +199,8 @@ func (m *merger) Merge( br = append(br, blockReaderFromData(data, segReader, startTime, blockSize)) // Check if this series is in memory (and thus requires merging). - mergeWithData, hasData, err := mergeWith.Read(ctx, id, blockStart, nsCtx) + tmpCtx.Reset() + mergeWithData, hasData, err := mergeWith.Read(tmpCtx, id, blockStart, nsCtx) if err != nil { return err } @@ -208,7 +212,8 @@ func (m *merger) Merge( sliceOfSlices.Reset(brs) multiIter.ResetSliceOfSlices(sliceOfSlices, nsCtx.Schema) - // tagsIter is never nil. + // tagsIter is never nil. These tags will be valid as long as the IDs + // are valid, and the IDs are valid for the duration of the file writing. tags, err := convert.TagsFromTagsIter(id, tagsIter, identPool) tagsIter.Close() tagsToFinalize = append(tagsToFinalize, tags) @@ -219,26 +224,22 @@ func (m *merger) Merge( id, tags, blockAllocSize, nsCtx.Schema, encoderPool); err != nil { return err } + tmpCtx.BlockingClose() } - // Second stage: loop through rest of the merge target that was not captured - // in the first stage. - err = mergeWith.ForEachRemaining(blockStart, func(seriesID ident.ID, tags ident.Tags) error { - mergeWithData, hasData, err := mergeWith.Read(ctx, seriesID, blockStart, nsCtx) - if err != nil { - return err - } - - if hasData { + // Second stage: loop through any series in the merge target that were not + // captured in the first stage. + tmpCtx.Reset() + err = mergeWith.ForEachRemaining( + tmpCtx, blockStart, + func(seriesID ident.ID, tags ident.Tags, mergeWithData []xio.BlockReader) error { brs = brs[:0] brs = append(brs, mergeWithData) sliceOfSlices.Reset(brs) multiIter.ResetSliceOfSlices(sliceOfSlices, nsCtx.Schema) return persistIter(prepared.Persist, multiIter, startTime, seriesID, tags, blockAllocSize, nsCtx.Schema, encoderPool) - } - - return nil - }) + }, nsCtx) + tmpCtx.BlockingClose() if err != nil { return err } diff --git a/src/dbnode/persist/fs/merger_test.go b/src/dbnode/persist/fs/merger_test.go index 821e7a5eee..48f6636c30 100644 --- a/src/dbnode/persist/fs/merger_test.go +++ b/src/dbnode/persist/fs/merger_test.go @@ -33,6 +33,7 @@ import ( "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/x/checked" + "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/pool" xtime "github.com/m3db/m3/src/x/time" @@ -559,12 +560,6 @@ func mockMergeWithFromData( for _, val := range mergeTargetData.Iter() { id := val.Key() if !diskData.Contains(id) { - data := val.Value() - segReader := srPool.Get() - br := []xio.BlockReader{blockReaderFromData(data, segReader, startTime, blockSize)} - mergeWith.EXPECT().Read(gomock.Any(), id, gomock.Any(), gomock.Any()). - Return(br, true, nil) - // Capture remaining items so that we can call the ForEachRemaining // fn on them later. remaining = append(remaining, id) @@ -572,11 +567,16 @@ func mockMergeWithFromData( } mergeWith.EXPECT(). - ForEachRemaining(xtime.ToUnixNano(startTime), gomock.Any()). + ForEachRemaining(gomock.Any(), xtime.ToUnixNano(startTime), gomock.Any(), gomock.Any()). Return(nil). - Do(func(blockStart xtime.UnixNano, fn ForEachRemainingFn) { + Do(func(ctx context.Context, blockStart xtime.UnixNano, fn ForEachRemainingFn, nsCtx namespace.Context) { for _, id := range remaining { - fn(id, ident.Tags{}) + data, ok := mergeTargetData.Get(id) + if ok { + segReader := srPool.Get() + br := []xio.BlockReader{blockReaderFromData(data, segReader, startTime, blockSize)} + fn(id, ident.Tags{}, br) + } } }) diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index c477800149..b36f4020b4 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -494,7 +494,7 @@ type BlockRetrieverOptions interface { // ForEachRemainingFn is the function that is run on each of the remaining // series of the merge target that did not intersect with the fileset. -type ForEachRemainingFn func(seriesID ident.ID, tags ident.Tags) error +type ForEachRemainingFn func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error // MergeWith is an interface that the fs merger uses to merge data with. type MergeWith interface { @@ -509,7 +509,12 @@ type MergeWith interface { // ForEachRemaining loops through each seriesID/blockStart combination that // was not already handled by a call to Read(). - ForEachRemaining(blockStart xtime.UnixNano, fn ForEachRemainingFn) error + ForEachRemaining( + ctx context.Context, + blockStart xtime.UnixNano, + fn ForEachRemainingFn, + nsCtx namespace.Context, + ) error } // Merger is in charge of merging filesets with some target MergeWith interface. diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 4729895d5b..f43f2ec4f2 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -130,18 +130,16 @@ func (m *flushManager) Flush( if err = m.dataColdFlush(namespaces); err != nil { multiErr = multiErr.Add(err) // If cold flush fails, we can't proceed to snapshotting because - // commit log logic just checks for a successful snapshot checkpoint - // file. Therefore if a cold flush fails and a snapshot succeeds, - // the writes from the failed cold flush might be lost when commit - // logs get cleaned up. + // commit log cleanup logic uses the presence of a successful + // snapshot checkpoint file to determine which commit log files are + // safe to delete. Therefore if a cold flush fails and a snapshot + // succeeds, the writes from the failed cold flush might be lost + // when commit logs get cleaned up, leaving the node in an undurable + // state such that if it restarted, it would not be able to recover + // the cold writes from its commit log. return multiErr.FinalError() } - // Snapshots must be done after a cold flush because cleanup logic just - // checks for a successful snapshot checkpoint file. If snapshotting - // went first and succeeded and then a cold flush fails, the old - // commitlogs would still get cleaned up. If the node then restarted, - // cold writes would be lost. if err = m.dataSnapshot(namespaces, tickStart, rotatedCommitlogID); err != nil { multiErr = multiErr.Add(err) } diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index c0a130b129..219456a71d 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -80,7 +80,6 @@ func (m *fsMergeWithMem) Read( nextVersion := m.retriever.RetrievableBlockColdVersion(startTime) + 1 blocks, err := m.shard.FetchBlocksForColdFlush(ctx, element.Value, startTime, nextVersion, nsCtx) - ctx.BlockingClose() if err != nil { return nil, false, err } @@ -92,7 +91,12 @@ func (m *fsMergeWithMem) Read( return nil, false, nil } -func (m *fsMergeWithMem) ForEachRemaining(blockStart xtime.UnixNano, fn fs.ForEachRemainingFn) error { +func (m *fsMergeWithMem) ForEachRemaining( + ctx context.Context, + blockStart xtime.UnixNano, + fn fs.ForEachRemainingFn, + nsCtx namespace.Context, +) error { seriesList := m.dirtySeriesToWrite[blockStart] for seriesElement := seriesList.Front(); seriesElement != nil; seriesElement = seriesElement.Next() { @@ -102,10 +106,16 @@ func (m *fsMergeWithMem) ForEachRemaining(blockStart xtime.UnixNano, fn fs.ForEa return err } - err = fn(seriesID, tags) + mergeWithData, hasData, err := m.Read(ctx, seriesID, blockStart, nsCtx) if err != nil { return err } + if hasData { + err = fn(seriesID, tags, mergeWithData) + if err != nil { + return err + } + } } return nil diff --git a/src/dbnode/storage/fs_merge_with_mem_test.go b/src/dbnode/storage/fs_merge_with_mem_test.go index 1e45718cbc..ddd3664f94 100644 --- a/src/dbnode/storage/fs_merge_with_mem_test.go +++ b/src/dbnode/storage/fs_merge_with_mem_test.go @@ -164,10 +164,10 @@ func TestForEachRemaining(t *testing.T) { var forEachCalls []ident.ID shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) - mergeWith.ForEachRemaining(0, func(seriesID ident.ID, tags ident.Tags) error { + mergeWith.ForEachRemaining(ctx, 0, func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error { forEachCalls = append(forEachCalls, seriesID) return nil - }) + }, nsCtx) require.Len(t, forEachCalls, 2) assert.Equal(t, id0, forEachCalls[0]) assert.Equal(t, id1, forEachCalls[1]) @@ -184,10 +184,10 @@ func TestForEachRemaining(t *testing.T) { assert.True(t, exists) assert.Equal(t, fetchedBlocks, res) shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) - err = mergeWith.ForEachRemaining(1, func(seriesID ident.ID, tags ident.Tags) error { + err = mergeWith.ForEachRemaining(ctx, 1, func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error { forEachCalls = append(forEachCalls, seriesID) return nil - }) + }, nsCtx) require.NoError(t, err) require.Len(t, forEachCalls, 2) assert.Equal(t, id2, forEachCalls[0]) @@ -196,18 +196,18 @@ func TestForEachRemaining(t *testing.T) { // Test call with error getting tags. shard.EXPECT(). TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, errors.New("bad-tags")) - err = mergeWith.ForEachRemaining(4, func(seriesID ident.ID, tags ident.Tags) error { + err = mergeWith.ForEachRemaining(ctx, 4, func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error { // This function won't be called with the above error. return errors.New("unreachable") - }) + }, nsCtx) assert.Error(t, err) // Test call with bad function execution. shard.EXPECT(). TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil) - err = mergeWith.ForEachRemaining(4, func(seriesID ident.ID, tags ident.Tags) error { + err = mergeWith.ForEachRemaining(ctx, 4, func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error { return errors.New("bad") - }) + }, nsCtx) assert.Error(t, err) } diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 67190c661a..b663d7ad78 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1001,6 +1001,22 @@ type coldFlushReuseableResources struct { fsReader fs.DataFileSetReader } +func newColdFlushReuseableResources(opts Options) (coldFlushReuseableResources, error) { + fsReader, err := fs.NewReader(opts.BytesPool(), opts.CommitLogOptions().FilesystemOptions()) + if err != nil { + return coldFlushReuseableResources{}, nil + } + + return coldFlushReuseableResources{ + // TODO(juchan): consider setting these options. + dirtySeries: newDirtySeriesMap(dirtySeriesMapOptions{}), + dirtySeriesToWrite: make(map[xtime.UnixNano]*idList), + // TODO(juchan): set pool options. + idElementPool: newIDElementPool(nil), + fsReader: fsReader, + }, nil +} + func (r *coldFlushReuseableResources) reset() { for _, seriesList := range r.dirtySeriesToWrite { if seriesList != nil { @@ -1037,18 +1053,10 @@ func (n *dbNamespace) ColdFlush( multiErr := xerrors.NewMultiError() shards := n.GetOwnedShards() - fsReader, err := fs.NewReader(n.opts.BytesPool(), n.opts.CommitLogOptions().FilesystemOptions()) + resources, err := newColdFlushReuseableResources(n.opts) if err != nil { return err } - resources := coldFlushReuseableResources{ - // TODO(juchan): consider setting these options. - dirtySeries: newDirtySeriesMap(dirtySeriesMapOptions{}), - dirtySeriesToWrite: make(map[xtime.UnixNano]*idList), - // TODO(juchan): set pool options. - idElementPool: newIDElementPool(nil), - fsReader: fsReader, - } for _, shard := range shards { err := shard.ColdFlush(flushPersist, resources, nsCtx) if err != nil { diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 6855e230e5..4c252c58ab 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -127,7 +127,7 @@ type databaseBuffer interface { IsEmpty() bool - ColdFlushBlockStarts() OptimizedTimes + ColdFlushBlockStarts(blockStates map[xtime.UnixNano]BlockState) OptimizedTimes Stats() bufferStats @@ -311,12 +311,22 @@ func (b *dbBuffer) IsEmpty() bool { return len(b.bucketsMap) == 0 } -func (b *dbBuffer) ColdFlushBlockStarts() OptimizedTimes { +func (b *dbBuffer) ColdFlushBlockStarts(blockStates map[xtime.UnixNano]BlockState) OptimizedTimes { var times OptimizedTimes for t, bucketVersions := range b.bucketsMap { for _, bucket := range bucketVersions.buckets { - if bucket.version == writableBucketVersion && bucket.writeType == ColdWrite { + if bucket.writeType == ColdWrite && + // We need to cold flush this bucket if it either: + // 1) Has new cold writes that need to be flushed, or + // 2) This bucket version is higher than what has been + // successfully flushed. This can happen if a cold flush was + // attempted, changing this bucket version, but fails to + // completely finish (which is what the shard block state + // signifies). In this case, we need to try to flush this + // bucket again. + (bucket.version == writableBucketVersion || + blockStates[xtime.ToUnixNano(bucket.start)].ColdVersion < bucket.version) { times.Add(t) break } diff --git a/src/dbnode/storage/series/buffer_mock.go b/src/dbnode/storage/series/buffer_mock.go index ec1a5eb2c6..026c7ca8f9 100644 --- a/src/dbnode/storage/series/buffer_mock.go +++ b/src/dbnode/storage/series/buffer_mock.go @@ -180,17 +180,17 @@ func (mr *MockdatabaseBufferMockRecorder) IsEmpty() *gomock.Call { } // ColdFlushBlockStarts mocks base method -func (m *MockdatabaseBuffer) ColdFlushBlockStarts() OptimizedTimes { +func (m *MockdatabaseBuffer) ColdFlushBlockStarts(blockStates map[time0.UnixNano]BlockState) OptimizedTimes { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ColdFlushBlockStarts") + ret := m.ctrl.Call(m, "ColdFlushBlockStarts", blockStates) ret0, _ := ret[0].(OptimizedTimes) return ret0 } // ColdFlushBlockStarts indicates an expected call of ColdFlushBlockStarts -func (mr *MockdatabaseBufferMockRecorder) ColdFlushBlockStarts() *gomock.Call { +func (mr *MockdatabaseBufferMockRecorder) ColdFlushBlockStarts(blockStates interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ColdFlushBlockStarts", reflect.TypeOf((*MockdatabaseBuffer)(nil).ColdFlushBlockStarts)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ColdFlushBlockStarts", reflect.TypeOf((*MockdatabaseBuffer)(nil).ColdFlushBlockStarts), blockStates) } // Stats mocks base method diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index 54f2db34a6..cf09a64c55 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -1260,23 +1260,59 @@ func TestColdFlushBlockStarts(t *testing.T) { }, } + blockStartNano1 := xtime.ToUnixNano(blockStart1) + blockStartNano2 := xtime.ToUnixNano(blockStart2) + blockStartNano3 := xtime.ToUnixNano(blockStart3) + buffer, _ := newTestBufferWithCustomData(t, bds, opts, nil) - flushStarts := buffer.ColdFlushBlockStarts() + blockStates := make(map[xtime.UnixNano]BlockState) + blockStates[blockStartNano1] = BlockState{ + WarmRetrievable: true, + ColdVersion: 0, + } + blockStates[blockStartNano2] = BlockState{ + WarmRetrievable: true, + ColdVersion: 0, + } + blockStates[blockStartNano3] = BlockState{ + WarmRetrievable: true, + ColdVersion: 0, + } + flushStarts := buffer.ColdFlushBlockStarts(blockStates) // All three cold blocks should report that they are dirty. assert.Equal(t, 3, flushStarts.Len()) - assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart1))) - assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart2))) - assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart3))) + assert.True(t, flushStarts.Contains(blockStartNano1)) + assert.True(t, flushStarts.Contains(blockStartNano2)) + assert.True(t, flushStarts.Contains(blockStartNano3)) // Simulate that block2 and block3 are flushed (but not yet evicted from // memory), so only block1 should report as dirty. - buffer.bucketsMap[xtime.ToUnixNano(blockStart2)].buckets[0].version = 1 - buffer.bucketsMap[xtime.ToUnixNano(blockStart3)].buckets[0].version = 1 + buffer.bucketsMap[blockStartNano2].buckets[0].version = 1 + buffer.bucketsMap[blockStartNano3].buckets[0].version = 1 + blockStates[blockStartNano2] = BlockState{ + WarmRetrievable: true, + ColdVersion: 1, + } + blockStates[blockStartNano3] = BlockState{ + WarmRetrievable: true, + ColdVersion: 1, + } - flushStarts = buffer.ColdFlushBlockStarts() + flushStarts = buffer.ColdFlushBlockStarts(blockStates) assert.Equal(t, 1, flushStarts.Len()) assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart1))) + + // Simulate blockStart3 didn't get fully flushed, so it should be flushed + // again. + blockStates[blockStartNano3] = BlockState{ + WarmRetrievable: true, + ColdVersion: 0, + } + flushStarts = buffer.ColdFlushBlockStarts(blockStates) + assert.Equal(t, 2, flushStarts.Len()) + assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart1))) + assert.True(t, flushStarts.Contains(xtime.ToUnixNano(blockStart3))) } func TestFetchBlocksForColdFlush(t *testing.T) { diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 9b880f7506..571a2b39af 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -566,11 +566,11 @@ func (s *dbSeries) Snapshot( return s.buffer.Snapshot(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) } -func (s *dbSeries) ColdFlushBlockStarts() OptimizedTimes { +func (s *dbSeries) ColdFlushBlockStarts(blockStates map[xtime.UnixNano]BlockState) OptimizedTimes { s.RLock() defer s.RUnlock() - return s.buffer.ColdFlushBlockStarts() + return s.buffer.ColdFlushBlockStarts(blockStates) } func (s *dbSeries) Close() { diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 480b675708..c625c949a9 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -91,17 +91,17 @@ func (mr *MockDatabaseSeriesMockRecorder) Close() *gomock.Call { } // ColdFlushBlockStarts mocks base method -func (m *MockDatabaseSeries) ColdFlushBlockStarts() OptimizedTimes { +func (m *MockDatabaseSeries) ColdFlushBlockStarts(arg0 map[time0.UnixNano]BlockState) OptimizedTimes { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ColdFlushBlockStarts") + ret := m.ctrl.Call(m, "ColdFlushBlockStarts", arg0) ret0, _ := ret[0].(OptimizedTimes) return ret0 } // ColdFlushBlockStarts indicates an expected call of ColdFlushBlockStarts -func (mr *MockDatabaseSeriesMockRecorder) ColdFlushBlockStarts() *gomock.Call { +func (mr *MockDatabaseSeriesMockRecorder) ColdFlushBlockStarts(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ColdFlushBlockStarts", reflect.TypeOf((*MockDatabaseSeries)(nil).ColdFlushBlockStarts)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ColdFlushBlockStarts", reflect.TypeOf((*MockDatabaseSeries)(nil).ColdFlushBlockStarts), arg0) } // FetchBlocks mocks base method diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index 1b8e27a356..f910453437 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -123,7 +123,7 @@ type DatabaseSeries interface { ) error // ColdFlushBlockStarts returns the block starts that need cold flushes. - ColdFlushBlockStarts() OptimizedTimes + ColdFlushBlockStarts(blockStates map[xtime.UnixNano]BlockState) OptimizedTimes // Close will close the series and if pooled returned to the pool. Close() diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 0fd662ff1d..44b66a1edc 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -358,6 +358,11 @@ func statusIsRetrievable(status fileOpStatus) bool { status)) } +func (s *dbShard) hasWarmFlushed(blockStart time.Time) bool { + flushState := s.FlushState(blockStart) + return statusIsRetrievable(flushState.WarmStatus) +} + // RetrievableBlockColdVersion implements series.QueryableBlockRetriever func (s *dbShard) RetrievableBlockColdVersion(blockStart time.Time) int { flushState := s.FlushState(blockStart) @@ -1948,13 +1953,21 @@ func (s *dbShard) ColdFlush( idElementPool = resources.idElementPool ) + blockStates := s.BlockStatesSnapshot() // First, loop through all series to capture data on which blocks have dirty // series and add them to the resources for further processing. s.forEachShardEntry(func(entry *lookup.Entry) bool { curr := entry.Series seriesID := curr.ID() - blockStarts := curr.ColdFlushBlockStarts() + blockStarts := curr.ColdFlushBlockStarts(blockStates) blockStarts.ForEach(func(t xtime.UnixNano) { + // Cold flushes can only happen on blockStarts that have been + // warm flushed, because warm flush logic does not currently + // perform any merging logic. + if !s.hasWarmFlushed(t.ToTime()) { + return + } + seriesList := dirtySeriesToWrite[t] if seriesList == nil { seriesList = newIDList(idElementPool) diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index f201e8493b..00e037def4 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -358,6 +358,18 @@ func TestShardColdFlush(t *testing.T) { t5 := t0.Add(5 * blockSize) t6 := t0.Add(6 * blockSize) t7 := t0.Add(7 * blockSize) + // Mark t0-t6 (not t7) as having been warm flushed. Cold flushes can only + // happen after a successful warm flush because warm flushes currently don't + // have merging logic. This means that all blocks except t7 should + // successfully cold flush. + shard.markWarmFlushStateSuccess(t0) + shard.markWarmFlushStateSuccess(t1) + shard.markWarmFlushStateSuccess(t2) + shard.markWarmFlushStateSuccess(t3) + shard.markWarmFlushStateSuccess(t4) + shard.markWarmFlushStateSuccess(t5) + shard.markWarmFlushStateSuccess(t6) + dirtyData := []testDirtySeries{ {id: ident.StringID("id0"), dirtyTimes: []time.Time{t0, t2, t3, t4}}, {id: ident.StringID("id1"), dirtyTimes: []time.Time{t1}}, @@ -367,7 +379,7 @@ func TestShardColdFlush(t *testing.T) { for _, ds := range dirtyData { curr := series.NewMockDatabaseSeries(ctrl) curr.EXPECT().ID().Return(ds.id) - curr.EXPECT().ColdFlushBlockStarts(). + curr.EXPECT().ColdFlushBlockStarts(gomock.Any()). Return(optimizedTimesFromTimes(ds.dirtyTimes)) shard.list.PushBack(lookup.NewEntry(curr, 0)) } @@ -387,11 +399,13 @@ func TestShardColdFlush(t *testing.T) { assert.Equal(t, 0, shard.RetrievableBlockColdVersion(i)) } shard.ColdFlush(preparer, resources, nsCtx) - // After a cold flush, all previously dirty block starts should be updated + // After a cold flush, t0-t6 previously dirty block starts should be updated // to version 1. - for i := t0; i.Before(t7.Add(blockSize)); i = i.Add(blockSize) { + for i := t0; i.Before(t6.Add(blockSize)); i = i.Add(blockSize) { assert.Equal(t, 1, shard.RetrievableBlockColdVersion(i)) } + // t7 shouldn't be cold flushed because it hasn't been warm flushed. + assert.Equal(t, 0, shard.RetrievableBlockColdVersion(t7)) } func newMergerTestFn( @@ -438,8 +452,10 @@ func (m *noopMergeWith) Read( } func (m *noopMergeWith) ForEachRemaining( + ctx context.Context, blockStart xtime.UnixNano, fn fs.ForEachRemainingFn, + nsCtx namespace.Context, ) error { return nil } From 81f02b13baa9e26cc0bc53ca42567fa681453b3d Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Tue, 4 Jun 2019 23:26:34 -0400 Subject: [PATCH 13/18] Fix mergeWithMem test --- src/dbnode/storage/fs_merge_with_mem.go | 16 ++++++++++++---- src/dbnode/storage/fs_merge_with_mem_test.go | 15 +++++++++++++++ 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index 219456a71d..0996cb65ce 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -69,17 +69,25 @@ func (m *fsMergeWithMem) Read( return nil, false, nil } - startTime := blockStart.ToTime() - // Series is in memory, so it will get merged with disk and // written in this loop. Therefore, we need to remove it from // the "to write" list so that the later loop does not rewrite // it. m.dirtySeriesToWrite[blockStart].Remove(element) + return m.fetchBlocks(ctx, element.Value, blockStart, nsCtx) +} + +func (m *fsMergeWithMem) fetchBlocks( + ctx context.Context, + id ident.ID, + blockStart xtime.UnixNano, + nsCtx namespace.Context, +) ([]xio.BlockReader, bool, error) { + startTime := blockStart.ToTime() nextVersion := m.retriever.RetrievableBlockColdVersion(startTime) + 1 - blocks, err := m.shard.FetchBlocksForColdFlush(ctx, element.Value, startTime, nextVersion, nsCtx) + blocks, err := m.shard.FetchBlocksForColdFlush(ctx, id, startTime, nextVersion, nsCtx) if err != nil { return nil, false, err } @@ -106,7 +114,7 @@ func (m *fsMergeWithMem) ForEachRemaining( return err } - mergeWithData, hasData, err := m.Read(ctx, seriesID, blockStart, nsCtx) + mergeWithData, hasData, err := m.fetchBlocks(ctx, seriesID, blockStart, nsCtx) if err != nil { return err } diff --git a/src/dbnode/storage/fs_merge_with_mem_test.go b/src/dbnode/storage/fs_merge_with_mem_test.go index ddd3664f94..abe2fce056 100644 --- a/src/dbnode/storage/fs_merge_with_mem_test.go +++ b/src/dbnode/storage/fs_merge_with_mem_test.go @@ -164,6 +164,12 @@ func TestForEachRemaining(t *testing.T) { var forEachCalls []ident.ID shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), id0, xtime.UnixNano(0).ToTime(), version+1, gomock.Any()). + Return(fetchedBlocks, nil) + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), id1, xtime.UnixNano(0).ToTime(), version+1, gomock.Any()). + Return(fetchedBlocks, nil) mergeWith.ForEachRemaining(ctx, 0, func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error { forEachCalls = append(forEachCalls, seriesID) return nil @@ -184,6 +190,12 @@ func TestForEachRemaining(t *testing.T) { assert.True(t, exists) assert.Equal(t, fetchedBlocks, res) shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), id2, xtime.UnixNano(1).ToTime(), version+1, gomock.Any()). + Return(fetchedBlocks, nil) + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), id4, xtime.UnixNano(1).ToTime(), version+1, gomock.Any()). + Return(fetchedBlocks, nil) err = mergeWith.ForEachRemaining(ctx, 1, func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error { forEachCalls = append(forEachCalls, seriesID) return nil @@ -196,6 +208,9 @@ func TestForEachRemaining(t *testing.T) { // Test call with error getting tags. shard.EXPECT(). TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, errors.New("bad-tags")) + shard.EXPECT(). + FetchBlocksForColdFlush(gomock.Any(), id8, xtime.UnixNano(4).ToTime(), version+1, gomock.Any()). + Return(fetchedBlocks, nil) err = mergeWith.ForEachRemaining(ctx, 4, func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error { // This function won't be called with the above error. return errors.New("unreachable") From 94fd0a54e66e1909775e483b81ab54ea456ae4bc Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Thu, 6 Jun 2019 12:27:22 -0400 Subject: [PATCH 14/18] Snapshots to take into account cold writes --- src/dbnode/persist/fs/merger.go | 13 +++- src/dbnode/storage/series/buffer.go | 20 +++-- src/dbnode/storage/series/buffer_test.go | 97 +++++++++++++++++++----- src/dbnode/storage/shard.go | 9 +-- 4 files changed, 102 insertions(+), 37 deletions(-) diff --git a/src/dbnode/persist/fs/merger.go b/src/dbnode/persist/fs/merger.go index a60573a952..e6b473ae26 100644 --- a/src/dbnode/persist/fs/merger.go +++ b/src/dbnode/persist/fs/merger.go @@ -224,6 +224,9 @@ func (m *merger) Merge( id, tags, blockAllocSize, nsCtx.Schema, encoderPool); err != nil { return err } + // Closing the context will finalize the data returned from + // mergeWith.Read(), but is safe because it has already been persisted + // to disk. tmpCtx.BlockingClose() } // Second stage: loop through any series in the merge target that were not @@ -236,10 +239,16 @@ func (m *merger) Merge( brs = append(brs, mergeWithData) sliceOfSlices.Reset(brs) multiIter.ResetSliceOfSlices(sliceOfSlices, nsCtx.Schema) - return persistIter(prepared.Persist, multiIter, startTime, + err := persistIter(prepared.Persist, multiIter, startTime, seriesID, tags, blockAllocSize, nsCtx.Schema, encoderPool) + // Context is safe to close after persisting data to disk. + tmpCtx.BlockingClose() + // Reset context here within the passed in function so that the + // context gets reset for each remaining series instead of getting + // finalized at the end of the ForEachRemaining call. + tmpCtx.Reset() + return err }, nsCtx) - tmpCtx.BlockingClose() if err != nil { return err } diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 4c252c58ab..a8c4f16a13 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -422,20 +422,18 @@ func (b *dbBuffer) Snapshot( return nil } - // We only snapshot warm writes since cold writes get force merged every - // tick anyway. - _, err := buckets.merge(WarmWrite, nsCtx) + // Snapshot must take both cold and warm writes because cold flushes don't + // happen for the current block (since cold flushes can't happen before a + // warm flush has happened). + streams, err := buckets.mergeToStreams(ctx, streamsOptions{filterWriteType: false}) if err != nil { return err } + numStreams := len(streams) - var ( - stream xio.SegmentReader - streams = buckets.streams(ctx, streamsOptions{filterWriteType: true, writeType: WarmWrite}) - numStreams = len(streams) - ) + var mergedStream xio.SegmentReader if numStreams == 1 { - stream = streams[0] + mergedStream = streams[0] } else { // We may need to merge again here because the regular merge method does // not merge buckets that have different versions. @@ -465,14 +463,14 @@ func (b *dbBuffer) Snapshot( } var ok bool - stream, ok = encoder.Stream(encoding.StreamOptions{}) + mergedStream, ok = encoder.Stream(encoding.StreamOptions{}) if !ok { // Don't write out series with no data. return nil } } - segment, err := stream.Segment() + segment, err := mergedStream.Segment() if err != nil { return err } diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index cf09a64c55..7e0086d8d8 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -1006,12 +1006,11 @@ func testBufferWithEmptyEncoder(t *testing.T, testSnapshot bool) { } func TestBufferSnapshot(t *testing.T) { - opts := newBufferTestOptions() + opts := newBufferTestOptions().SetColdWritesEnabled(true) testBufferSnapshot(t, opts, nil) } func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { - // Setup var ( rops = opts.RetentionOptions() blockSize = rops.BlockSize() @@ -1025,9 +1024,9 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { })) buffer.Reset(opts) - // Create test data to perform out of order writes that will create two in-order + // Create test data to perform warm writes that will create two in-order // encoders so we can verify that Snapshot will perform a merge. - data := []value{ + warmData := []value{ {curr, 1, xtime.Second, nil}, {curr.Add(mins(0.5)), 2, xtime.Second, nil}, {curr.Add(mins(0.5)).Add(-5 * time.Second), 3, xtime.Second, nil}, @@ -1040,38 +1039,83 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { {curr.Add(blockSize), 6, xtime.Second, nil}, } if setAnn != nil { - data = setAnn(data) + warmData = setAnn(warmData) nsCtx = namespace.Context{Schema: testSchemaDesc} } - - // Perform the writes. - for _, v := range data { + // Perform warm writes. + for _, v := range warmData { + // Set curr so that every write is a warm write. curr = v.timestamp verifyWriteToBuffer(t, buffer, v, nsCtx.Schema) } + // Also add cold writes to the buffer to verify that Snapshot will capture + // cold writes as well and perform a merge across both warm and cold data. + // The cold data itself is not in order, so we expect to have two in-order + // encoders for these. + curr = start.Add(mins(1.5)) + // In order for these writes to actually be cold, they all need to have + // timestamps before `curr.Add(-rops.BufferPast())`. Take care to not use + // the same timestamps used in the warm writes above, otherwise these will + // overwrite them. + // Buffer past/future in this test case is 10 seconds. + coldData := []value{ + {start.Add(secs(2)), 11, xtime.Second, nil}, + {start.Add(secs(4)), 12, xtime.Second, nil}, + {start.Add(secs(6)), 13, xtime.Second, nil}, + {start.Add(secs(3)), 14, xtime.Second, nil}, + {start.Add(secs(5)), 15, xtime.Second, nil}, + } + if setAnn != nil { + coldData = setAnn(coldData) + nsCtx = namespace.Context{Schema: testSchemaDesc} + } + // Perform cold writes. + for _, v := range coldData { + verifyWriteToBuffer(t, buffer, v, nsCtx.Schema) + } + // Verify internal state. - var encoders []encoding.Encoder + var ( + warmEncoders []encoding.Encoder + coldEncoders []encoding.Encoder + ) buckets, ok := buffer.bucketVersionsAt(start) require.True(t, ok) + bucket, ok := buckets.writableBucket(WarmWrite) require.True(t, ok) - // Current bucket encoders should all have data in them. + // Warm bucket encoders should all have data in them. for j := range bucket.encoders { encoder := bucket.encoders[j].encoder _, ok := encoder.Stream(encoding.StreamOptions{}) require.True(t, ok) - encoders = append(encoders, encoder) + warmEncoders = append(warmEncoders, encoder) } + assert.Equal(t, 2, len(warmEncoders)) - assert.Equal(t, 2, len(encoders)) + bucket, ok = buckets.writableBucket(ColdWrite) + require.True(t, ok) + // Cold bucket encoders should all have data in them. + for j := range bucket.encoders { + encoder := bucket.encoders[j].encoder + + _, ok := encoder.Stream(encoding.StreamOptions{}) + require.True(t, ok) + + coldEncoders = append(coldEncoders, encoder) + } + assert.Equal(t, 2, len(coldEncoders)) assertPersistDataFn := func(id ident.ID, tags ident.Tags, segment ts.Segment, checlsum uint32) error { // Check we got the right results. - expectedData := data[:len(data)-1] // -1 because we don't expect the last datapoint. + // `len(warmData)-1` because we don't expect the last warm datapoint + // since it's for a different block. + expectedData := warmData[:len(warmData)-1] + expectedData = append(expectedData, coldData...) expectedCopy := make([]value, len(expectedData)) copy(expectedCopy, expectedData) sort.Sort(valuesByTime(expectedCopy)) @@ -1091,8 +1135,9 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { err := buffer.Snapshot(ctx, start, ident.StringID("some-id"), ident.Tags{}, assertPersistDataFn, nsCtx) assert.NoError(t, err) - // Check internal state to make sure the merge happened and was persisted. - encoders = encoders[:0] + // Check internal state of warm bucket to make sure the merge happened and + // was persisted. + warmEncoders = warmEncoders[:0] buckets, ok = buffer.bucketVersionsAt(start) require.True(t, ok) bucket, ok = buckets.writableBucket(WarmWrite) @@ -1104,11 +1149,29 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { _, ok := encoder.Stream(encoding.StreamOptions{}) require.True(t, ok) - encoders = append(encoders, encoder) + warmEncoders = append(warmEncoders, encoder) } + // Ensure single encoder again. + assert.Equal(t, 1, len(warmEncoders)) + // Check internal state of cold bucket to make sure the merge happened and + // was persisted. + coldEncoders = coldEncoders[:0] + buckets, ok = buffer.bucketVersionsAt(start) + require.True(t, ok) + bucket, ok = buckets.writableBucket(ColdWrite) + require.True(t, ok) + // Current bucket encoders should all have data in them. + for i := range bucket.encoders { + encoder := bucket.encoders[i].encoder + + _, ok := encoder.Stream(encoding.StreamOptions{}) + require.True(t, ok) + + coldEncoders = append(coldEncoders, encoder) + } // Ensure single encoder again. - assert.Equal(t, 1, len(encoders)) + assert.Equal(t, 1, len(coldEncoders)) } func mustGetLastEncoded(t *testing.T, entry inOrderEncoder) ts.Datapoint { diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 44b66a1edc..b2f0f86335 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -344,7 +344,7 @@ func (s *dbShard) Stream( // IsBlockRetrievable implements series.QueryableBlockRetriever func (s *dbShard) IsBlockRetrievable(blockStart time.Time) bool { flushState := s.FlushState(blockStart) - return statusIsRetrievable(flushState.WarmStatus) || flushState.ColdVersion > 0 + return statusIsRetrievable(flushState.WarmStatus) } func statusIsRetrievable(status fileOpStatus) bool { @@ -358,11 +358,6 @@ func statusIsRetrievable(status fileOpStatus) bool { status)) } -func (s *dbShard) hasWarmFlushed(blockStart time.Time) bool { - flushState := s.FlushState(blockStart) - return statusIsRetrievable(flushState.WarmStatus) -} - // RetrievableBlockColdVersion implements series.QueryableBlockRetriever func (s *dbShard) RetrievableBlockColdVersion(blockStart time.Time) int { flushState := s.FlushState(blockStart) @@ -1964,7 +1959,7 @@ func (s *dbShard) ColdFlush( // Cold flushes can only happen on blockStarts that have been // warm flushed, because warm flush logic does not currently // perform any merging logic. - if !s.hasWarmFlushed(t.ToTime()) { + if !s.IsBlockRetrievable(t.ToTime()) { return } From d318ad915316a37b0dca2f24a50baadd4edac1b7 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Thu, 6 Jun 2019 14:03:00 -0400 Subject: [PATCH 15/18] Fix test --- src/dbnode/storage/series/buffer.go | 2 +- src/dbnode/storage/series/buffer_test.go | 115 +++++++++++++++++++++-- 2 files changed, 107 insertions(+), 10 deletions(-) diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index a8c4f16a13..cb07f999d6 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -425,7 +425,7 @@ func (b *dbBuffer) Snapshot( // Snapshot must take both cold and warm writes because cold flushes don't // happen for the current block (since cold flushes can't happen before a // warm flush has happened). - streams, err := buckets.mergeToStreams(ctx, streamsOptions{filterWriteType: false}) + streams, err := buckets.mergeToStreams(ctx, streamsOptions{filterWriteType: false, nsCtx: nsCtx}) if err != nil { return err } diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index 7e0086d8d8..255f7d217b 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -1006,11 +1006,12 @@ func testBufferWithEmptyEncoder(t *testing.T, testSnapshot bool) { } func TestBufferSnapshot(t *testing.T) { - opts := newBufferTestOptions().SetColdWritesEnabled(true) + opts := newBufferTestOptions() testBufferSnapshot(t, opts, nil) } func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { + // Setup var ( rops = opts.RetentionOptions() blockSize = rops.BlockSize() @@ -1024,9 +1025,9 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { })) buffer.Reset(opts) - // Create test data to perform warm writes that will create two in-order + // Create test data to perform out of order writes that will create two in-order // encoders so we can verify that Snapshot will perform a merge. - warmData := []value{ + data := []value{ {curr, 1, xtime.Second, nil}, {curr.Add(mins(0.5)), 2, xtime.Second, nil}, {curr.Add(mins(0.5)).Add(-5 * time.Second), 3, xtime.Second, nil}, @@ -1039,9 +1040,108 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { {curr.Add(blockSize), 6, xtime.Second, nil}, } if setAnn != nil { - warmData = setAnn(warmData) + data = setAnn(data) nsCtx = namespace.Context{Schema: testSchemaDesc} } + + // Perform the writes. + for _, v := range data { + curr = v.timestamp + verifyWriteToBuffer(t, buffer, v, nsCtx.Schema) + } + + // Verify internal state. + var encoders []encoding.Encoder + + buckets, ok := buffer.bucketVersionsAt(start) + require.True(t, ok) + bucket, ok := buckets.writableBucket(WarmWrite) + require.True(t, ok) + // Current bucket encoders should all have data in them. + for j := range bucket.encoders { + encoder := bucket.encoders[j].encoder + + _, ok := encoder.Stream(encoding.StreamOptions{}) + require.True(t, ok) + + encoders = append(encoders, encoder) + } + + assert.Equal(t, 2, len(encoders)) + + assertPersistDataFn := func(id ident.ID, tags ident.Tags, segment ts.Segment, checlsum uint32) error { + // Check we got the right results. + expectedData := data[:len(data)-1] // -1 because we don't expect the last datapoint. + expectedCopy := make([]value, len(expectedData)) + copy(expectedCopy, expectedData) + sort.Sort(valuesByTime(expectedCopy)) + actual := [][]xio.BlockReader{{ + xio.BlockReader{ + SegmentReader: xio.NewSegmentReader(segment), + }, + }} + requireReaderValuesEqual(t, expectedCopy, actual, opts, nsCtx) + + return nil + } + + // Perform a snapshot. + ctx := context.NewContext() + defer ctx.Close() + err := buffer.Snapshot(ctx, start, ident.StringID("some-id"), ident.Tags{}, assertPersistDataFn, nsCtx) + assert.NoError(t, err) + + // Check internal state to make sure the merge happened and was persisted. + encoders = encoders[:0] + buckets, ok = buffer.bucketVersionsAt(start) + require.True(t, ok) + bucket, ok = buckets.writableBucket(WarmWrite) + require.True(t, ok) + // Current bucket encoders should all have data in them. + for i := range bucket.encoders { + encoder := bucket.encoders[i].encoder + + _, ok := encoder.Stream(encoding.StreamOptions{}) + require.True(t, ok) + + encoders = append(encoders, encoder) + } + + // Ensure single encoder again. + assert.Equal(t, 1, len(encoders)) +} + +func TestBufferSnapshotWithColdWrites(t *testing.T) { + opts := newBufferTestOptions().SetColdWritesEnabled(true) + + var ( + rops = opts.RetentionOptions() + blockSize = rops.BlockSize() + curr = time.Now().Truncate(blockSize) + start = curr + buffer = newDatabaseBuffer().(*dbBuffer) + nsCtx namespace.Context + ) + opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(func() time.Time { + return curr + })) + buffer.Reset(opts) + + // Create test data to perform warm writes that will create two in-order + // encoders so we can verify that Snapshot will perform a merge. + warmData := []value{ + {curr, 1, xtime.Second, nil}, + {curr.Add(mins(0.5)), 2, xtime.Second, nil}, + {curr.Add(mins(0.5)).Add(-5 * time.Second), 3, xtime.Second, nil}, + {curr.Add(mins(1.0)), 4, xtime.Second, nil}, + {curr.Add(mins(1.5)), 5, xtime.Second, nil}, + {curr.Add(mins(1.5)).Add(-5 * time.Second), 6, xtime.Second, nil}, + + // Add one write for a different block to make sure Snapshot only returns + // date for the requested block. + {curr.Add(blockSize), 6, xtime.Second, nil}, + } + // Perform warm writes. for _, v := range warmData { // Set curr so that every write is a warm write. @@ -1066,10 +1166,7 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { {start.Add(secs(3)), 14, xtime.Second, nil}, {start.Add(secs(5)), 15, xtime.Second, nil}, } - if setAnn != nil { - coldData = setAnn(coldData) - nsCtx = namespace.Context{Schema: testSchemaDesc} - } + // Perform cold writes. for _, v := range coldData { verifyWriteToBuffer(t, buffer, v, nsCtx.Schema) @@ -1133,7 +1230,7 @@ func testBufferSnapshot(t *testing.T, opts Options, setAnn setAnnotation) { ctx := context.NewContext() defer ctx.Close() err := buffer.Snapshot(ctx, start, ident.StringID("some-id"), ident.Tags{}, assertPersistDataFn, nsCtx) - assert.NoError(t, err) + require.NoError(t, err) // Check internal state of warm bucket to make sure the merge happened and // was persisted. From a1153313a75297433e879df08c39e8df5d2fc78f Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Fri, 7 Jun 2019 13:44:41 -0400 Subject: [PATCH 16/18] Tags NoFinalize; additional comments --- src/dbnode/storage/fs_merge_with_mem.go | 12 +++++++++++- src/dbnode/storage/series/buffer.go | 2 +- src/dbnode/storage/shard.go | 15 +++++++++++---- src/dbnode/storage/storage_mock.go | 7 ++++--- src/dbnode/storage/types.go | 2 +- 5 files changed, 28 insertions(+), 10 deletions(-) diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index 0996cb65ce..a384b71d50 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -99,6 +99,10 @@ func (m *fsMergeWithMem) fetchBlocks( return nil, false, nil } +// The data passed to ForEachRemaining (through the fs.ForEachRemainingFn) is +// basically a copy that will be finalized when the context is closed, but the +// ID and tags are expected to live for as long as the caller of the MergeWith +// requires them, so they should either be NoFinalize() or passed as copies. func (m *fsMergeWithMem) ForEachRemaining( ctx context.Context, blockStart xtime.UnixNano, @@ -109,10 +113,16 @@ func (m *fsMergeWithMem) ForEachRemaining( for seriesElement := seriesList.Front(); seriesElement != nil; seriesElement = seriesElement.Next() { seriesID := seriesElement.Value - tags, err := m.shard.TagsFromSeriesID(seriesID) + tags, ok, err := m.shard.TagsFromSeriesID(seriesID) if err != nil { return err } + if !ok { + // Receiving not ok means that the series was not found, for some + // reason like it falling out of retention, therefore we skip this + // series and continue. + continue + } mergeWithData, hasData, err := m.fetchBlocks(ctx, seriesID, blockStart, nsCtx) if err != nil { diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index cb07f999d6..02b64e64d7 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -436,7 +436,7 @@ func (b *dbBuffer) Snapshot( mergedStream = streams[0] } else { // We may need to merge again here because the regular merge method does - // not merge buckets that have different versions. + // not merge warm and cold buckets or buckets that have different versions. sr := make([]xio.SegmentReader, 0, numStreams) for _, stream := range streams { sr = append(sr, stream) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index b2f0f86335..4231b9261a 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -343,6 +343,10 @@ func (s *dbShard) Stream( // IsBlockRetrievable implements series.QueryableBlockRetriever func (s *dbShard) IsBlockRetrievable(blockStart time.Time) bool { + return s.hasWarmFlushed(blockStart) +} + +func (s *dbShard) hasWarmFlushed(blockStart time.Time) bool { flushState := s.FlushState(blockStart) return statusIsRetrievable(flushState.WarmStatus) } @@ -1091,6 +1095,9 @@ func (s *dbShard) newShardEntry( default: return nil, errNewShardEntryTagsTypeInvalid } + // Don't put tags back in a pool since the merge logic may still have a + // handle on these. + seriesTags.NoFinalize() series := s.seriesPool.Get() series.Reset(seriesID, seriesTags, s.seriesBlockRetriever, @@ -1959,7 +1966,7 @@ func (s *dbShard) ColdFlush( // Cold flushes can only happen on blockStarts that have been // warm flushed, because warm flush logic does not currently // perform any merging logic. - if !s.IsBlockRetrievable(t.ToTime()) { + if !s.hasWarmFlushed(t.ToTime()) { return } @@ -2160,15 +2167,15 @@ func (s *dbShard) Repair( return repairer.Repair(ctx, nsCtx, tr, s) } -func (s *dbShard) TagsFromSeriesID(seriesID ident.ID) (ident.Tags, error) { +func (s *dbShard) TagsFromSeriesID(seriesID ident.ID) (ident.Tags, bool, error) { s.RLock() entry, _, err := s.lookupEntryWithLock(seriesID) s.RUnlock() if entry == nil || err != nil { - return ident.Tags{}, err + return ident.Tags{}, false, err } - return entry.Series.Tags(), nil + return entry.Series.Tags(), true, nil } func (s *dbShard) BootstrapState() BootstrapState { diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 1c20e73107..4e8c55ae31 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1775,12 +1775,13 @@ func (mr *MockdatabaseShardMockRecorder) Repair(ctx, tr, repairer interface{}) * } // TagsFromSeriesID mocks base method -func (m *MockdatabaseShard) TagsFromSeriesID(seriesID ident.ID) (ident.Tags, error) { +func (m *MockdatabaseShard) TagsFromSeriesID(seriesID ident.ID) (ident.Tags, bool, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "TagsFromSeriesID", seriesID) ret0, _ := ret[0].(ident.Tags) - ret1, _ := ret[1].(error) - return ret0, ret1 + ret1, _ := ret[1].(bool) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 } // TagsFromSeriesID indicates an expected call of TagsFromSeriesID diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index f457540c02..1c4e97cff2 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -507,7 +507,7 @@ type databaseShard interface { ) (repair.MetadataComparisonResult, error) // TagsFromSeriesID returns the series tags from a series ID. - TagsFromSeriesID(seriesID ident.ID) (ident.Tags, error) + TagsFromSeriesID(seriesID ident.ID) (ident.Tags, bool, error) } // namespaceIndex indexes namespace writes. From 4f27aefa9c0dcf06bb460028ad9d389272578ca8 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Fri, 7 Jun 2019 13:48:32 -0400 Subject: [PATCH 17/18] Regen mocks --- src/dbnode/storage/storage_mock.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 4e8c55ae31..fdd976c018 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1760,18 +1760,18 @@ func (mr *MockdatabaseShardMockRecorder) CleanupExpiredFileSets(earliestToRetain } // Repair mocks base method -func (m *MockdatabaseShard) Repair(ctx context.Context, tr time0.Range, repairer databaseShardRepairer) (repair.MetadataComparisonResult, error) { +func (m *MockdatabaseShard) Repair(ctx context.Context, nsCtx namespace.Context, tr time0.Range, repairer databaseShardRepairer) (repair.MetadataComparisonResult, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Repair", ctx, tr, repairer) + ret := m.ctrl.Call(m, "Repair", ctx, nsCtx, tr, repairer) ret0, _ := ret[0].(repair.MetadataComparisonResult) ret1, _ := ret[1].(error) return ret0, ret1 } // Repair indicates an expected call of Repair -func (mr *MockdatabaseShardMockRecorder) Repair(ctx, tr, repairer interface{}) *gomock.Call { +func (mr *MockdatabaseShardMockRecorder) Repair(ctx, nsCtx, tr, repairer interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShard)(nil).Repair), ctx, tr, repairer) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShard)(nil).Repair), ctx, nsCtx, tr, repairer) } // TagsFromSeriesID mocks base method @@ -2385,18 +2385,18 @@ func (mr *MockdatabaseShardRepairerMockRecorder) Options() *gomock.Call { } // Repair mocks base method -func (m *MockdatabaseShardRepairer) Repair(ctx context.Context, namespace ident.ID, tr time0.Range, shard databaseShard) (repair.MetadataComparisonResult, error) { +func (m *MockdatabaseShardRepairer) Repair(ctx context.Context, nsCtx namespace.Context, tr time0.Range, shard databaseShard) (repair.MetadataComparisonResult, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Repair", ctx, namespace, tr, shard) + ret := m.ctrl.Call(m, "Repair", ctx, nsCtx, tr, shard) ret0, _ := ret[0].(repair.MetadataComparisonResult) ret1, _ := ret[1].(error) return ret0, ret1 } // Repair indicates an expected call of Repair -func (mr *MockdatabaseShardRepairerMockRecorder) Repair(ctx, namespace, tr, shard interface{}) *gomock.Call { +func (mr *MockdatabaseShardRepairerMockRecorder) Repair(ctx, nsCtx, tr, shard interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShardRepairer)(nil).Repair), ctx, namespace, tr, shard) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Repair", reflect.TypeOf((*MockdatabaseShardRepairer)(nil).Repair), ctx, nsCtx, tr, shard) } // MockdatabaseRepairer is a mock of databaseRepairer interface From 117c6ff1b71df2a263752dc7967f90958a2116c9 Mon Sep 17 00:00:00 2001 From: Justin Chan Date: Fri, 7 Jun 2019 14:26:35 -0400 Subject: [PATCH 18/18] Fix TestForEachRemaining --- src/dbnode/storage/fs_merge_with_mem_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/dbnode/storage/fs_merge_with_mem_test.go b/src/dbnode/storage/fs_merge_with_mem_test.go index abe2fce056..7deab20778 100644 --- a/src/dbnode/storage/fs_merge_with_mem_test.go +++ b/src/dbnode/storage/fs_merge_with_mem_test.go @@ -163,7 +163,7 @@ func TestForEachRemaining(t *testing.T) { mergeWith := newFSMergeWithMem(shard, retriever, dirtySeries, dirtySeriesToWrite) var forEachCalls []ident.ID - shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) + shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, true, nil).Times(2) shard.EXPECT(). FetchBlocksForColdFlush(gomock.Any(), id0, xtime.UnixNano(0).ToTime(), version+1, gomock.Any()). Return(fetchedBlocks, nil) @@ -189,7 +189,7 @@ func TestForEachRemaining(t *testing.T) { require.NoError(t, err) assert.True(t, exists) assert.Equal(t, fetchedBlocks, res) - shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil).Times(2) + shard.EXPECT().TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, true, nil).Times(2) shard.EXPECT(). FetchBlocksForColdFlush(gomock.Any(), id2, xtime.UnixNano(1).ToTime(), version+1, gomock.Any()). Return(fetchedBlocks, nil) @@ -207,7 +207,7 @@ func TestForEachRemaining(t *testing.T) { // Test call with error getting tags. shard.EXPECT(). - TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, errors.New("bad-tags")) + TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, false, errors.New("bad-tags")) shard.EXPECT(). FetchBlocksForColdFlush(gomock.Any(), id8, xtime.UnixNano(4).ToTime(), version+1, gomock.Any()). Return(fetchedBlocks, nil) @@ -219,7 +219,7 @@ func TestForEachRemaining(t *testing.T) { // Test call with bad function execution. shard.EXPECT(). - TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, nil) + TagsFromSeriesID(gomock.Any()).Return(ident.Tags{}, true, nil) err = mergeWith.ForEachRemaining(ctx, 4, func(seriesID ident.ID, tags ident.Tags, data []xio.BlockReader) error { return errors.New("bad") }, nsCtx)