From 8cf491706b84cd0a1ebc03ccb683d8cb8b167d95 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 10 Oct 2018 17:47:59 -0400 Subject: [PATCH 01/47] Handle commit log files with corrupt info headers during cleanup --- src/dbnode/persist/fs/commitlog/files.go | 28 +++++++++++++++++------- src/dbnode/storage/cleanup.go | 14 +++++++++++- src/dbnode/storage/cleanup_test.go | 20 +++++++++++++++++ src/dbnode/storage/util.go | 4 ++-- 4 files changed, 55 insertions(+), 11 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index cc2009fe1a..cbf79c9812 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -30,12 +30,17 @@ import ( "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" ) +type openError error + // File represents a commit log file and its associated metadata. type File struct { FilePath string Start time.Time Duration time.Duration Index int64 + // Contains any errors encountered (except for filesystem errors) when trying + // to read the files log info. + Error error } // ReadLogInfo reads the commit log info out of a commitlog file @@ -50,7 +55,7 @@ func ReadLogInfo(filePath string, opts Options) (time.Time, time.Duration, int64 fd, err = os.Open(filePath) if err != nil { - return time.Time{}, 0, 0, err + return time.Time{}, 0, 0, openError(err) } chunkReader := newChunkReader(opts.FlushSize()) @@ -90,17 +95,24 @@ func Files(opts Options) ([]File, error) { commitLogFiles := make([]File, 0, len(filePaths)) for _, filePath := range filePaths { + file := File{ + FilePath: filePath, + } + start, duration, index, err := ReadLogInfo(filePath, opts) - if err != nil { + if _, ok := err.(openError); ok { return nil, err } - commitLogFiles = append(commitLogFiles, File{ - FilePath: filePath, - Start: start, - Duration: duration, - Index: index, - }) + if err != nil { + file.Error = err + } else { + file.Start = start + file.Duration = duration + file.Index = index + } + + commitLogFiles = append(commitLogFiles, file) } sort.Slice(commitLogFiles, func(i, j int) bool { diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index 9af4ba23ef..03e00d4d7b 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -289,9 +289,21 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitlog.File, error) { return nil, err } - shouldCleanupFile := func(start time.Time, duration time.Duration) (bool, error) { + shouldCleanupFile := func(f commitlog.File) (bool, error) { + if f.Error != nil { + // If we were unable to read the commit log files info header, then we're forced to assume + // that the file is corrupt and remove it. This can happen in situations where M3DB experiences + // sudden shutdown. + m.opts.InstrumentOptions().Logger().Errorf( + "encountered err: %v reading commit log file: %v info during cleanup, marking file for deletion", + f.Error, f.FilePath) + return true, nil + } + for _, ns := range namespaces { var ( + start = f.Start + duration = f.Duration ropts = ns.Options().RetentionOptions() nsBlocksStart, nsBlocksEnd = commitLogNamespaceBlockTimes(start, duration, ropts) needsFlush = ns.NeedsFlush(nsBlocksStart, nsBlocksEnd) diff --git a/src/dbnode/storage/cleanup_test.go b/src/dbnode/storage/cleanup_test.go index 1ecca1414c..7932b4a84a 100644 --- a/src/dbnode/storage/cleanup_test.go +++ b/src/dbnode/storage/cleanup_test.go @@ -710,3 +710,23 @@ func TestCleanupManagerCommitLogTimesMultiNS(t *testing.T) { require.True(t, contains(filesToCleanup, time10)) require.True(t, contains(filesToCleanup, time20)) } + +func TestCleanupManagerDeletesCorruptCommitLogFiles(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + var ( + _, mgr = newCleanupManagerCommitLogTimesTest(t, ctrl) + err = errors.New("some_error") + ) + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { + return []commitlog.File{ + commitlog.File{Start: time10, Error: err}, + }, nil + } + + filesToCleanup, err := mgr.commitLogTimes(currentTime) + require.NoError(t, err) + + require.True(t, contains(filesToCleanup, time10)) +} diff --git a/src/dbnode/storage/util.go b/src/dbnode/storage/util.go index f9ce8b0780..ba38b0fbf5 100644 --- a/src/dbnode/storage/util.go +++ b/src/dbnode/storage/util.go @@ -59,11 +59,11 @@ func timesInRange(startInclusive, endInclusive time.Time, windowSize time.Durati // satisfy the provided predicate. func filterCommitLogFiles( files []commitlog.File, - predicate func(start time.Time, duration time.Duration) (bool, error), + predicate func(f commitlog.File) (bool, error), ) ([]commitlog.File, error) { filtered := make([]commitlog.File, 0, len(files)) for _, f := range files { - passed, err := predicate(f.Start, f.Duration) + passed, err := predicate(f) if err != nil { return nil, err } From 949025c114cffbe5fb009b30ba78783b706311e5 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 10 Oct 2018 17:50:15 -0400 Subject: [PATCH 02/47] Rename openError to fsError --- src/dbnode/persist/fs/commitlog/files.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index cbf79c9812..660ecfe1f4 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -30,7 +30,7 @@ import ( "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" ) -type openError error +type fsError error // File represents a commit log file and its associated metadata. type File struct { @@ -77,7 +77,7 @@ func ReadLogInfo(filePath string, opts Options) (time.Time, time.Duration, int64 err = fd.Close() fd = nil if err != nil { - return time.Time{}, 0, 0, err + return time.Time{}, 0, 0, fsError(err) } return time.Unix(0, logInfo.Start), time.Duration(logInfo.Duration), logInfo.Index, decoderErr @@ -100,7 +100,7 @@ func Files(opts Options) ([]File, error) { } start, duration, index, err := ReadLogInfo(filePath, opts) - if _, ok := err.(openError); ok { + if _, ok := err.(fsError); ok { return nil, err } From 96743419bf071e23c4262cdebcbc5053ba42d66a Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 10 Oct 2018 17:52:11 -0400 Subject: [PATCH 03/47] Clarify comment --- src/dbnode/persist/fs/commitlog/files.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index 660ecfe1f4..79bcebefa6 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -38,8 +38,11 @@ type File struct { Start time.Time Duration time.Duration Index int64 - // Contains any errors encountered (except for filesystem errors) when trying - // to read the files log info. + // Contains any errors encountered when trying to read the commitlogs file info. We + // attempt to not include filesystem errors in this field, but that is accomplished + // on a best-effort basis and it is possible for this field to contain an error that + // is the result of a filesystem / O.S / hardware issue as opposed to an actually + // corrupt file. Error error } From f0ad534d7897a48422809bc38cb41ae88e5bc46e Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 10 Oct 2018 17:53:04 -0400 Subject: [PATCH 04/47] Fix typo --- src/dbnode/persist/fs/commitlog/files.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index 79bcebefa6..2b4e8a1e26 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -58,7 +58,7 @@ func ReadLogInfo(filePath string, opts Options) (time.Time, time.Duration, int64 fd, err = os.Open(filePath) if err != nil { - return time.Time{}, 0, 0, openError(err) + return time.Time{}, 0, 0, fsError(err) } chunkReader := newChunkReader(opts.FlushSize()) From 16827fe08932318e3853a8de4ac7bc00becee7a8 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 10 Oct 2018 17:58:56 -0400 Subject: [PATCH 05/47] Skip corrupt commit log files in the commitlog bootstrapper --- .../storage/bootstrap/bootstrapper/commitlog/source.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index dc6d0fde92..ed773f1fe4 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -782,6 +782,14 @@ func (s *commitLogSource) newReadCommitLogPred( // that has a snapshot more recent than the global minimum. If we use an array for fast-access this could // be a small win in terms of memory utilization. return func(f commitlog.File) bool { + if f.Error != nil { + s.log. + Errorf( + "opting to skip commit log: %s due to corruption, err: %v", + f.FilePath, f.Error) + return false + } + _, ok := commitlogFilesPresentBeforeStart[f.FilePath] if !ok { // If the file wasn't on disk before the node started then it only contains From a53a56dfdb4bf7b0c5599b803a8a8403313f0d6c Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 10 Oct 2018 18:22:31 -0400 Subject: [PATCH 06/47] modify commit log source prop test to test that it handle commit log files with corrupt info headers --- src/dbnode/persist/fs/commitlog/files.go | 12 +++- .../commitlog/source_prop_test.go | 60 ++++++++++++++----- 2 files changed, 54 insertions(+), 18 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index 2b4e8a1e26..09d9fc2564 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -30,7 +30,13 @@ import ( "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" ) -type fsError error +type fsError struct { + err error +} + +func (e fsError) Error() string { + return e.err.Error() +} // File represents a commit log file and its associated metadata. type File struct { @@ -58,7 +64,7 @@ func ReadLogInfo(filePath string, opts Options) (time.Time, time.Duration, int64 fd, err = os.Open(filePath) if err != nil { - return time.Time{}, 0, 0, fsError(err) + return time.Time{}, 0, 0, fsError{err} } chunkReader := newChunkReader(opts.FlushSize()) @@ -80,7 +86,7 @@ func ReadLogInfo(filePath string, opts Options) (time.Time, time.Duration, int64 err = fd.Close() fd = nil if err != nil { - return time.Time{}, 0, 0, fsError(err) + return time.Time{}, 0, 0, fsError{err} } return time.Unix(0, logInfo.Start), time.Duration(logInfo.Duration), logInfo.Index, decoderErr diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 6f369521dd..715819d327 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -282,6 +282,29 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { if err != nil { return false, err } + + if input.includeCorruptedCommitlogFile { + // Write out an additional commit log file with a corrupt info header to + // make sure that the commitlog source skips it. + commitLogFiles, err := commitlog.Files(commitLogOpts) + if err != nil { + return false, err + } + + if len(commitLogFiles) > 0 { + lastCommitLogFile := commitLogFiles[len(commitLogFiles)-1] + nextCommitLogFile, _, err := fs.NextCommitLogsFile( + commitLogOpts.FilesystemOptions().FilePathPrefix(), lastCommitLogFile.Start) + if err != nil { + return false, err + } + + err = ioutil.WriteFile(nextCommitLogFile, []byte("corruption"), 0644) + if err != nil { + return false, err + } + } + } } // Instantiate a commitlog source @@ -385,13 +408,14 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { } type propTestInput struct { - currentTime time.Time - snapshotTime time.Time - snapshotExists bool - commitLogExists bool - bufferPast time.Duration - bufferFuture time.Duration - writes []generatedWrite + currentTime time.Time + snapshotTime time.Time + snapshotExists bool + commitLogExists bool + bufferPast time.Duration + bufferFuture time.Duration + writes []generatedWrite + includeCorruptedCommitlogFile bool } type generatedWrite struct { @@ -411,19 +435,20 @@ func (w generatedWrite) String() string { func genPropTestInputs(nsMeta namespace.Metadata, blockStart time.Time) gopter.Gen { curriedGenPropTestInput := func(input interface{}) gopter.Gen { var ( - inputs = input.([]interface{}) - snapshotTime = inputs[0].(time.Time) - snapshotExists = inputs[1].(bool) - commitLogExists = inputs[2].(bool) - bufferPast = time.Duration(inputs[3].(int64)) - bufferFuture = time.Duration(inputs[4].(int64)) - numDatapoints = inputs[5].(int) + inputs = input.([]interface{}) + snapshotTime = inputs[0].(time.Time) + snapshotExists = inputs[1].(bool) + commitLogExists = inputs[2].(bool) + bufferPast = time.Duration(inputs[3].(int64)) + bufferFuture = time.Duration(inputs[4].(int64)) + numDatapoints = inputs[5].(int) + includeCorruptedCommitlogFile = inputs[6].(bool) ) return genPropTestInput( blockStart, bufferPast, bufferFuture, snapshotTime, snapshotExists, commitLogExists, - numDatapoints, nsMeta.ID().String()) + numDatapoints, nsMeta.ID().String(), includeCorruptedCommitlogFile) } return gopter.CombineGens( @@ -440,6 +465,9 @@ func genPropTestInputs(nsMeta namespace.Metadata, blockStart time.Time) gopter.G gen.Int64Range(0, int64(blockSize)), // Run iterations of the test with between 0 and 100 datapoints gen.IntRange(0, 100), + // Whether the test should generate an additional corrupt commitlog file + // to ensure the commit log bootstrapper skips it correctly. + gen.Bool(), ).FlatMap(curriedGenPropTestInput, reflect.TypeOf(propTestInput{})) } @@ -452,6 +480,7 @@ func genPropTestInput( commitLogExists bool, numDatapoints int, ns string, + includeCorruptedCommitlogFile bool, ) gopter.Gen { return gen.SliceOfN(numDatapoints, genWrite(start, bufferPast, bufferFuture, ns)). Map(func(val []generatedWrite) propTestInput { @@ -463,6 +492,7 @@ func genPropTestInput( snapshotExists: snapshotExists, commitLogExists: commitLogExists, writes: val, + includeCorruptedCommitlogFile: includeCorruptedCommitlogFile, } }) } From cecfa02cd49dfb2de9a411bc339c6a03aca030af Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Thu, 11 Oct 2018 15:25:53 -0400 Subject: [PATCH 07/47] wip --- .../bootstrap/bootstrapper/commitlog/source.go | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index ed773f1fe4..055c57570f 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -1497,6 +1497,23 @@ func (s *commitLogSource) availability( return availableShardTimeRanges, nil } +func (s *commitLogSource) isOnlyNodeInPlacement( + ns namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, + runOpts bootstrap.RunOptions, +) (bool, error) { + var ( + // topoState = runOpts.InitialTopologyState() + // availableShardTimeRanges = result.ShardTimeRanges{} + ) + + for shardIDUint := range shardsTimeRanges { + + } + + return true, nil +} + func newReadSeriesPredicate(ns namespace.Metadata) commitlog.SeriesFilterPredicate { nsID := ns.ID() return func(id ident.ID, namespace ident.ID) bool { From 4eea9bda31ce47cc9f53124b82c3a79235037737 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Thu, 11 Oct 2018 15:47:39 -0400 Subject: [PATCH 08/47] wip --- src/dbnode/persist/fs/commitlog/iterator.go | 8 +++---- src/dbnode/persist/fs/commitlog/types.go | 2 +- .../bootstrapper/commitlog/source.go | 21 ++++++++----------- 3 files changed, 14 insertions(+), 17 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/iterator.go b/src/dbnode/persist/fs/commitlog/iterator.go index 7ef48e2b5e..6c76f30f41 100644 --- a/src/dbnode/persist/fs/commitlog/iterator.go +++ b/src/dbnode/persist/fs/commitlog/iterator.go @@ -65,18 +65,18 @@ type iteratorRead struct { // ReadAllPredicate can be passed as the ReadCommitLogPredicate for callers // that want a convenient way to read all the commitlogs func ReadAllPredicate() FileFilterPredicate { - return func(_ File) bool { return true } + return func(_ File) (bool, bool) { return true, false } } // NewIterator creates a new commit log iterator -func NewIterator(iterOpts IteratorOpts) (Iterator, error) { +func NewIterator(iterOpts IteratorOpts) (iter Iterator, corruptFiles []string, err error) { opts := iterOpts.CommitLogOptions iops := opts.InstrumentOptions() iops = iops.SetMetricsScope(iops.MetricsScope().SubScope("iterator")) files, err := Files(opts) if err != nil { - return nil, err + return nil, nil, err } filteredFiles := filterFiles(opts, files, iterOpts.FileFilterPredicate) @@ -90,7 +90,7 @@ func NewIterator(iterOpts IteratorOpts) (Iterator, error) { log: iops.Logger(), files: filteredFiles, seriesPred: iterOpts.SeriesFilterPredicate, - }, nil + }, nil, nil } func (i *iterator) Next() bool { diff --git a/src/dbnode/persist/fs/commitlog/types.go b/src/dbnode/persist/fs/commitlog/types.go index f5fddc7c18..ce7be4fedd 100644 --- a/src/dbnode/persist/fs/commitlog/types.go +++ b/src/dbnode/persist/fs/commitlog/types.go @@ -180,7 +180,7 @@ type Options interface { // FileFilterPredicate is a predicate that allows the caller to determine // which commitlogs the iterator should read from -type FileFilterPredicate func(f File) bool +type FileFilterPredicate func(f File) (shouldRead bool, isCorrupt bool) // SeriesFilterPredicate is a predicate that determines whether datapoints for a given series // should be returned from the Commit log reader. The predicate is pushed down to the diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 055c57570f..1e960b6757 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -55,7 +55,8 @@ var ( const encoderChanBufSize = 1000 -type newIteratorFn func(opts commitlog.IteratorOpts) (commitlog.Iterator, error) +type newIteratorFn func(opts commitlog.IteratorOpts) ( + iter commitlog.Iterator, corruptFiles []string, err error) type snapshotFilesFn func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error) type newReaderFn func(bytesPool pool.CheckedBytesPool, opts fs.Options) (fs.DataFileSetReader, error) @@ -1497,21 +1498,17 @@ func (s *commitLogSource) availability( return availableShardTimeRanges, nil } -func (s *commitLogSource) isOnlyNodeInPlacement( +func (s *commitLogSource) couldObtainDataFromPeers( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, runOpts bootstrap.RunOptions, -) (bool, error) { - var ( - // topoState = runOpts.InitialTopologyState() - // availableShardTimeRanges = result.ShardTimeRanges{} - ) - - for shardIDUint := range shardsTimeRanges { - +) bool { + // TODO: Refactor InitialTopologyState to store Replicas along with MajorityReplicas + initialTopologyState := runOpts.InitialTopologyState() + if initialTopologyState.MajorityReplicas > 1 { + return true } - - return true, nil + return false } func newReadSeriesPredicate(ns namespace.Metadata) commitlog.SeriesFilterPredicate { From 60ca533e59c380d253459c9870c093764f0cae1b Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Thu, 11 Oct 2018 16:11:49 -0400 Subject: [PATCH 09/47] wip --- src/dbnode/persist/fs/commitlog/iterator.go | 22 ++++++++---- .../bootstrapper/commitlog/source.go | 35 ++++++++++++------- 2 files changed, 37 insertions(+), 20 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/iterator.go b/src/dbnode/persist/fs/commitlog/iterator.go index 6c76f30f41..b03224ff2c 100644 --- a/src/dbnode/persist/fs/commitlog/iterator.go +++ b/src/dbnode/persist/fs/commitlog/iterator.go @@ -78,7 +78,7 @@ func NewIterator(iterOpts IteratorOpts) (iter Iterator, corruptFiles []string, e if err != nil { return nil, nil, err } - filteredFiles := filterFiles(opts, files, iterOpts.FileFilterPredicate) + filteredFiles, corruptFiles := filterFiles(opts, files, iterOpts.FileFilterPredicate) scope := iops.MetricsScope() return &iterator{ @@ -90,7 +90,7 @@ func NewIterator(iterOpts IteratorOpts) (iter Iterator, corruptFiles []string, e log: iops.Logger(), files: filteredFiles, seriesPred: iterOpts.SeriesFilterPredicate, - }, nil, nil + }, corruptFiles, nil } func (i *iterator) Next() bool { @@ -190,14 +190,22 @@ func (i *iterator) nextReader() bool { return true } -func filterFiles(opts Options, files []File, predicate FileFilterPredicate) []File { - filteredFiles := make([]File, 0, len(files)) +func filterFiles(opts Options, files []File, predicate FileFilterPredicate) ([]File, []string) { + var ( + filtered = make([]File, 0, len(files)) + corrupt = make([]string, 0, len(files)) + ) + for _, f := range files { - if predicate(f) { - filteredFiles = append(filteredFiles, f) + shouldRead, isCorrupt := predicate(f) + if shouldRead { + filtered = append(filtered, f) + } + if isCorrupt { + corrupt = append(corrupt, f.FilePath) } } - return filteredFiles + return filtered, corrupt } func (i *iterator) closeAndResetReader() error { diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 1e960b6757..569fbc427c 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -185,8 +185,9 @@ func (s *commitLogSource) ReadData( } var ( - fsOpts = s.opts.CommitLogOptions().FilesystemOptions() - filePathPrefix = fsOpts.FilePathPrefix() + encounteredCorruptData = false + fsOpts = s.opts.CommitLogOptions().FilesystemOptions() + filePathPrefix = fsOpts.FilePathPrefix() ) // Determine which snapshot files are available. @@ -242,10 +243,13 @@ func (s *commitLogSource) ReadData( s.log.Infof("datapointsRead: %d", datapointsRead) }() - iter, err := s.newIteratorFn(iterOpts) + iter, corruptFiles, err := s.newIteratorFn(iterOpts) if err != nil { return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } + if len(corruptFiles) > 0 { + encounteredCorruptData = true + } defer iter.Close() @@ -683,7 +687,7 @@ func (s *commitLogSource) newReadCommitLogPredBasedOnAvailableSnapshotFiles( shardsTimeRanges result.ShardTimeRanges, snapshotFilesByShard map[uint32]fs.FileSetFilesSlice, ) ( - func(f commitlog.File) bool, + commitlog.FileFilterPredicate, map[xtime.UnixNano]map[uint32]fs.FileSetFile, error, ) { @@ -740,7 +744,7 @@ func (s *commitLogSource) newReadCommitLogPredBasedOnAvailableSnapshotFiles( func (s *commitLogSource) newReadCommitLogPred( ns namespace.Metadata, minimumMostRecentSnapshotTimeByBlock map[xtime.UnixNano]time.Time, -) func(f commitlog.File) bool { +) commitlog.FileFilterPredicate { var ( rOpts = ns.Options().RetentionOptions() blockSize = rOpts.BlockSize() @@ -782,13 +786,13 @@ func (s *commitLogSource) newReadCommitLogPred( // we need to read, but we can still skip datapoints from the commitlog itself that belong to a shard // that has a snapshot more recent than the global minimum. If we use an array for fast-access this could // be a small win in terms of memory utilization. - return func(f commitlog.File) bool { + return func(f commitlog.File) (bool, bool) { if f.Error != nil { s.log. Errorf( "opting to skip commit log: %s due to corruption, err: %v", f.FilePath, f.Error) - return false + return false, true } _, ok := commitlogFilesPresentBeforeStart[f.FilePath] @@ -796,7 +800,7 @@ func (s *commitLogSource) newReadCommitLogPred( // If the file wasn't on disk before the node started then it only contains // writes that are already in memory (and in-fact the file may be actively // being written to.) - return false + return false, false } for _, rangeToCheck := range rangesToCheck { @@ -810,7 +814,7 @@ func (s *commitLogSource) newReadCommitLogPred( Infof( "opting to read commit log: %s with start: %s and duration: %s", f.FilePath, f.Start.String(), f.Duration.String()) - return true + return true, false } } @@ -818,7 +822,7 @@ func (s *commitLogSource) newReadCommitLogPred( Infof( "opting to skip commit log: %s with start: %s and duration: %s", f.FilePath, f.Start.String(), f.Duration.String()) - return false + return false, false } } @@ -1277,8 +1281,9 @@ func (s *commitLogSource) ReadIndex( } var ( - fsOpts = s.opts.CommitLogOptions().FilesystemOptions() - filePathPrefix = fsOpts.FilePathPrefix() + encounteredCorruptData = false + fsOpts = s.opts.CommitLogOptions().FilesystemOptions() + filePathPrefix = fsOpts.FilePathPrefix() ) // Determine which snapshot files are available. @@ -1350,10 +1355,14 @@ func (s *commitLogSource) ReadIndex( // Next, read all of the data from the commit log files that wasn't covered // by the snapshot files. - iter, err := s.newIteratorFn(iterOpts) + iter, corruptFiles, err := s.newIteratorFn(iterOpts) if err != nil { return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } + if len(corruptFiles) > 0 { + encounteredCorruptData = true + } + defer iter.Close() for iter.Next() { From a2741936174a89ea6c90798678ce164028fcdb11 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Thu, 11 Oct 2018 16:20:33 -0400 Subject: [PATCH 10/47] wi[ --- .../bootstrap/bootstrapper/commitlog/source.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 569fbc427c..ec536edbc0 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -337,6 +337,17 @@ func (s *commitLogSource) ReadData( } s.log.Infof("done merging..., took: %s", time.Since(mergeStart).String()) + couldObtainDataFromPeers := s.couldObtainDataFromPeers( + shardsTimeRanges, runOpts) + if encounteredCorruptData && couldObtainDataFromPeers { + // If we encountered any corrupt data and there is a possibility of the + // peers bootstrapper being able to correct it, mark the entire range + // as unfulfilled so Peers bootstrapper can attempt a repair, but keep + // the data we read from the commit log as well in case the peers + // bootstrapper is unable to satisfy the bootstrap because all peers are + // down or if the commitlog contained data that the peers do not have. + bootstrapResult.SetUnfulfilled(shardsTimeRanges) + } return bootstrapResult, nil } @@ -1513,6 +1524,8 @@ func (s *commitLogSource) couldObtainDataFromPeers( runOpts bootstrap.RunOptions, ) bool { // TODO: Refactor InitialTopologyState to store Replicas along with MajorityReplicas + // TODO: Actually also need to check the shard state of the peers too because if they're + // not available we can't get data from them. initialTopologyState := runOpts.InitialTopologyState() if initialTopologyState.MajorityReplicas > 1 { return true From d870758172b5e1b797ea23bad2700433d403980e Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Thu, 11 Oct 2018 17:21:38 -0400 Subject: [PATCH 11/47] Refactor to return unfulfilled on error --- .../bootstrapper/commitlog/source.go | 33 +++++++++++++++++-- .../commitlog/source_data_test.go | 30 +++++++++-------- .../commitlog/source_index_test.go | 16 ++++----- 3 files changed, 55 insertions(+), 24 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index ec536edbc0..604a58eb05 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -307,7 +307,13 @@ func (s *commitLogSource) ReadData( } if iterErr := iter.Err(); iterErr != nil { - return nil, iterErr + // Log the error and mark that we encountered corrupt data, but don't + // return the error because we want to give the peers bootstrapper the + // opportunity to repair the data instead of failing the bootstrap + // altogether. + s.log.Errorf( + "error in commitlog iterator: %v", iterErr) + encounteredCorruptData = true } for _, encoderChan := range encoderChans { @@ -333,12 +339,13 @@ func (s *commitLogSource) ReadData( shardDataByShard, ) if err != nil { + // TODO: Should probably not return this error? return nil, err } s.log.Infof("done merging..., took: %s", time.Since(mergeStart).String()) couldObtainDataFromPeers := s.couldObtainDataFromPeers( - shardsTimeRanges, runOpts) + ns, shardsTimeRanges, runOpts) if encounteredCorruptData && couldObtainDataFromPeers { // If we encountered any corrupt data and there is a possibility of the // peers bootstrapper being able to correct it, mark the entire range @@ -1349,6 +1356,7 @@ func (s *commitLogSource) ReadIndex( ns.ID(), shard, true, tr, blockSize, snapshotFilesByShard[shard], mostRecentCompleteSnapshotByBlockShard) if err != nil { + // TODO: Probably should not return an error here return nil, err } @@ -1384,6 +1392,16 @@ func (s *commitLogSource) ReadIndex( indexResults, indexOptions, indexBlockSize, resultOptions) } + if iterErr := iter.Err(); iterErr != nil { + // Log the error and mark that we encountered corrupt data, but don't + // return the error because we want to give the peers bootstrapper the + // opportunity to repair the data instead of failing the bootstrap + // altogether. + s.log.Errorf( + "error in commitlog iterator: %v", iterErr) + encounteredCorruptData = true + } + // If all successful then we mark each index block as fulfilled for _, block := range indexResult.IndexResults() { blockRange := xtime.Range{ @@ -1409,6 +1427,17 @@ func (s *commitLogSource) ReadIndex( } } + couldObtainDataFromPeers := s.couldObtainDataFromPeers( + ns, shardsTimeRanges, opts) + if encounteredCorruptData && couldObtainDataFromPeers { + // If we encountered any corrupt data and there is a possibility of the + // peers bootstrapper being able to correct it, mark the entire range + // as unfulfilled so Peers bootstrapper can attempt a repair, but keep + // the data we read from the commit log as well in case the peers + // bootstrapper is unable to satisfy the bootstrap because all peers are + // down or if the commitlog contained data that the peers do not have. + indexResult.SetUnfulfilled(shardsTimeRanges) + } return indexResult, nil } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go index 996626059c..bab10bfe6e 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go @@ -38,6 +38,7 @@ import ( "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3/src/dbnode/storage/namespace" + "github.com/m3db/m3/src/dbnode/topology" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" @@ -49,8 +50,9 @@ import ( ) var ( - testNamespaceID = ident.StringID("testnamespace") - testDefaultRunOpts = bootstrap.NewRunOptions() + testNamespaceID = ident.StringID("testnamespace") + testDefaultRunOpts = bootstrap.NewRunOptions(). + SetInitialTopologyState(&topology.StateSnapshot{}) minCommitLogRetention = 10 * time.Minute ) @@ -111,8 +113,8 @@ func TestReadErrorOnNewIteratorError(t *testing.T) { opts := testDefaultOpts src := newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return nil, fmt.Errorf("an error") + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return nil, nil, fmt.Errorf("an error") } ranges := xtime.Ranges{} @@ -157,8 +159,8 @@ func TestReadOrderedValues(t *testing.T) { // "baz" is in shard 2 and should not be returned {baz, start.Add(4 * time.Minute), 1.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator(values, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator(values, nil), nil, nil } targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} @@ -199,8 +201,8 @@ func TestReadUnorderedValues(t *testing.T) { {foo, start.Add(3 * time.Minute), 4.0, xtime.Second, nil}, {foo, start, 5.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator(values, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator(values, nil), nil, nil } targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} @@ -245,8 +247,8 @@ func TestReadHandlesDifferentSeriesWithIdenticalUniqueIndex(t *testing.T) { {foo, start, 1.0, xtime.Second, nil}, {bar, start, 2.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator(values, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator(values, nil), nil, nil } targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} @@ -286,8 +288,8 @@ func TestReadTrimsToRanges(t *testing.T) { {foo, start.Add(1 * time.Minute), 3.0, xtime.Nanosecond, nil}, {foo, end.Truncate(blockSize).Add(blockSize).Add(time.Nanosecond), 4.0, xtime.Nanosecond, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator(values, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator(values, nil), nil, nil } targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} @@ -334,8 +336,8 @@ func TestItMergesSnapshotsAndCommitLogs(t *testing.T) { End: end, }) - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator(commitLogValues, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator(commitLogValues, nil), nil, nil } src.snapshotFilesFn = func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error) { return fs.FileSetFilesSlice{ diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go index b1cefa3a4e..9a1599bb14 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go @@ -117,8 +117,8 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { {someOtherNamespace, start.Add(dataBlockSize), 1.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator(values, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator(values, nil), nil, nil } ranges := xtime.Ranges{} @@ -195,8 +195,8 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { otherNamespaceValues = append(otherNamespaceValues, value) } } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator(otherNamespaceValues, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator(otherNamespaceValues, nil), nil, nil } res, err = src.ReadIndex(md2, targetRanges, testDefaultRunOpts) @@ -212,8 +212,8 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { // Update the iterator function to return no values (since this namespace has no data) // because the real commit log reader does this (via the ReadSeries predicate). - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator([]testValue{}, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator([]testValue{}, nil), nil, nil } res, err = src.ReadIndex(md3, targetRanges, testDefaultRunOpts) @@ -253,8 +253,8 @@ func TestBootstrapIndexEmptyShardTimeRanges(t *testing.T) { require.NoError(t, err) values := []testValue{} - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { - return newTestCommitLogIterator(values, nil), nil + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + return newTestCommitLogIterator(values, nil), nil, nil } res, err := src.ReadIndex(md, result.ShardTimeRanges{}, testDefaultRunOpts) From 3883221bd5cb7c5ecba4a27f3ea7fff36aa4a428 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Thu, 11 Oct 2018 17:38:01 -0400 Subject: [PATCH 12/47] Update test --- .../bootstrap/bootstrapper/commitlog/source_prop_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 715819d327..24d805961b 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -1,4 +1,3 @@ -// +build big // // Copyright (c) 2017 Uber Technologies, Inc. // @@ -285,7 +284,7 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { if input.includeCorruptedCommitlogFile { // Write out an additional commit log file with a corrupt info header to - // make sure that the commitlog source skips it. + // make sure that the commitlog source skips it in the single node scenario. commitLogFiles, err := commitlog.Files(commitLogOpts) if err != nil { return false, err From f13822708032902f3f18a1e38afb0676c16d2664 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Thu, 11 Oct 2018 18:04:21 -0400 Subject: [PATCH 13/47] update prop test --- .../bootstrapper/commitlog/source.go | 3 + .../commitlog/source_prop_test.go | 67 ++++++++++++++----- 2 files changed, 55 insertions(+), 15 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 604a58eb05..9df8b92c31 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -248,6 +248,7 @@ func (s *commitLogSource) ReadData( return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } if len(corruptFiles) > 0 { + fmt.Println("yolo") encounteredCorruptData = true } @@ -1557,8 +1558,10 @@ func (s *commitLogSource) couldObtainDataFromPeers( // not available we can't get data from them. initialTopologyState := runOpts.InitialTopologyState() if initialTopologyState.MajorityReplicas > 1 { + fmt.Println("could get from peers: ", true) return true } + fmt.Println("could get from peers: ", false) return false } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 24d805961b..8879c83121 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -39,6 +39,7 @@ import ( "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3/src/dbnode/storage/namespace" + "github.com/m3db/m3/src/dbnode/topology" tu "github.com/m3db/m3/src/dbnode/topology/testutil" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3cluster/shard" @@ -293,7 +294,7 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { if len(commitLogFiles) > 0 { lastCommitLogFile := commitLogFiles[len(commitLogFiles)-1] nextCommitLogFile, _, err := fs.NextCommitLogsFile( - commitLogOpts.FilesystemOptions().FilePathPrefix(), lastCommitLogFile.Start) + fsOpts.FilePathPrefix(), lastCommitLogFile.Start) if err != nil { return false, err } @@ -349,12 +350,19 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { } // Perform the bootstrap - var ( + var initialTopoState *topology.StateSnapshot + if input.multiNodeCluster { + initialTopoState = tu.NewStateSnapshot(3, tu.HostShardStates{ + tu.SelfID: tu.Shards(allShardsSlice, shard.Available), + "not-self1": tu.Shards(allShardsSlice, shard.Available), + "not-self2": tu.Shards(allShardsSlice, shard.Available), + }) + } else { initialTopoState = tu.NewStateSnapshot(1, tu.HostShardStates{ tu.SelfID: tu.Shards(allShardsSlice, shard.Available), }) - runOpts = testDefaultRunOpts.SetInitialTopologyState(initialTopoState) - ) + } + runOpts := testDefaultRunOpts.SetInitialTopologyState(initialTopoState) dataResult, err := source.BootstrapData(nsMeta, shardTimeRanges, runOpts) if err != nil { return false, err @@ -366,11 +374,25 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { values = append(values, testValue{write.series, write.datapoint.Timestamp, write.datapoint.Value, write.unit, write.annotation}) } - if !dataResult.Unfulfilled().IsEmpty() { - return false, fmt.Errorf( - "data result unfulfilled should be empty but was: %s", - dataResult.Unfulfilled().String(), - ) + commitLogFiles, err := commitlog.Files(commitLogOpts) + if err != nil { + return false, err + } + + commitLogFilesExist := len(commitLogFiles) > 0 + // In the multi-node setup we want to return unfulfilled if there are any corrupt files, but + // we always want to return fulfilled in the single node setup. + if input.multiNodeCluster && input.includeCorruptedCommitlogFile && commitLogFilesExist { + if dataResult.Unfulfilled().IsEmpty() { + return false, fmt.Errorf( + "data result unfulfilled should not be empty in multi node cluster but was") + } + } else { + if !dataResult.Unfulfilled().IsEmpty() { + return false, fmt.Errorf( + "data result unfulfilled in single node cluster should be empty but was: %s", + dataResult.Unfulfilled().String()) + } } err = verifyShardResultsAreCorrect(values, blockSize, dataResult.ShardResults(), bootstrapOpts) if err != nil { @@ -389,11 +411,19 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { return false, err } - if !indexResult.Unfulfilled().IsEmpty() { - return false, fmt.Errorf( - "index result unfulfilled should be empty but was: %s", - indexResult.Unfulfilled().String(), - ) + // In the multi-node setup we want to return unfulfilled if there are any corrupt files, but + // we always want to return fulfilled in the single node setup. + if input.multiNodeCluster && input.includeCorruptedCommitlogFile && commitLogFilesExist { + if indexResult.Unfulfilled().IsEmpty() { + return false, fmt.Errorf( + "index result unfulfilled should not be empty in multi node cluster but was") + } + } else { + if !indexResult.Unfulfilled().IsEmpty() { + return false, fmt.Errorf( + "index result unfulfilled in single node cluster should be empty but was: %s", + indexResult.Unfulfilled().String()) + } } return true, nil @@ -415,6 +445,7 @@ type propTestInput struct { bufferFuture time.Duration writes []generatedWrite includeCorruptedCommitlogFile bool + multiNodeCluster bool } type generatedWrite struct { @@ -442,12 +473,13 @@ func genPropTestInputs(nsMeta namespace.Metadata, blockStart time.Time) gopter.G bufferFuture = time.Duration(inputs[4].(int64)) numDatapoints = inputs[5].(int) includeCorruptedCommitlogFile = inputs[6].(bool) + multiNodeCluster = inputs[7].(bool) ) return genPropTestInput( blockStart, bufferPast, bufferFuture, snapshotTime, snapshotExists, commitLogExists, - numDatapoints, nsMeta.ID().String(), includeCorruptedCommitlogFile) + numDatapoints, nsMeta.ID().String(), includeCorruptedCommitlogFile, multiNodeCluster) } return gopter.CombineGens( @@ -467,6 +499,9 @@ func genPropTestInputs(nsMeta namespace.Metadata, blockStart time.Time) gopter.G // Whether the test should generate an additional corrupt commitlog file // to ensure the commit log bootstrapper skips it correctly. gen.Bool(), + // Whether the test should simulate the InitialTopologyState to mimic a + // multi node cluster or not. + gen.Bool(), ).FlatMap(curriedGenPropTestInput, reflect.TypeOf(propTestInput{})) } @@ -480,6 +515,7 @@ func genPropTestInput( numDatapoints int, ns string, includeCorruptedCommitlogFile bool, + multiNodeCluster bool, ) gopter.Gen { return gen.SliceOfN(numDatapoints, genWrite(start, bufferPast, bufferFuture, ns)). Map(func(val []generatedWrite) propTestInput { @@ -492,6 +528,7 @@ func genPropTestInput( commitLogExists: commitLogExists, writes: val, includeCorruptedCommitlogFile: includeCorruptedCommitlogFile, + multiNodeCluster: multiNodeCluster, } }) } From ad5d08bea843bc51be1a8b3e3edecf2978b32d72 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Thu, 11 Oct 2018 18:04:51 -0400 Subject: [PATCH 14/47] remove prints --- src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 9df8b92c31..604a58eb05 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -248,7 +248,6 @@ func (s *commitLogSource) ReadData( return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } if len(corruptFiles) > 0 { - fmt.Println("yolo") encounteredCorruptData = true } @@ -1558,10 +1557,8 @@ func (s *commitLogSource) couldObtainDataFromPeers( // not available we can't get data from them. initialTopologyState := runOpts.InitialTopologyState() if initialTopologyState.MajorityReplicas > 1 { - fmt.Println("could get from peers: ", true) return true } - fmt.Println("could get from peers: ", false) return false } From c14f51c604153f6c281998e28308d2c7e5b9206b Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 12:30:45 -0400 Subject: [PATCH 15/47] cleanup handling of errors --- .../persist/fs/commitlog/commit_log_test.go | 11 ++- src/dbnode/persist/fs/commitlog/files.go | 73 +++++++++++++++---- src/dbnode/persist/fs/commitlog/files_test.go | 7 +- src/dbnode/persist/fs/commitlog/iterator.go | 26 ++++--- src/dbnode/persist/fs/commitlog/types.go | 2 +- .../bootstrapper/commitlog/source.go | 31 ++++---- 6 files changed, 108 insertions(+), 42 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/commit_log_test.go b/src/dbnode/persist/fs/commitlog/commit_log_test.go index eb390f74b3..b3a35467eb 100644 --- a/src/dbnode/persist/fs/commitlog/commit_log_test.go +++ b/src/dbnode/persist/fs/commitlog/commit_log_test.go @@ -302,8 +302,9 @@ func assertCommitLogWritesByIterating(t *testing.T, l *commitLog, writes []testW FileFilterPredicate: ReadAllPredicate(), SeriesFilterPredicate: ReadAllSeriesPredicate(), } - iter, err := NewIterator(iterOpts) + iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) + require.Equal(t, 0, len(corruptFiles)) defer iter.Close() // Convert the writes to be in-order, but keyed by series ID because the @@ -425,8 +426,10 @@ func TestReadCommitLogMissingMetadata(t *testing.T) { FileFilterPredicate: ReadAllPredicate(), SeriesFilterPredicate: ReadAllSeriesPredicate(), } - iter, err := NewIterator(iterOpts) + iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) + require.Equal(t, 0, len(corruptFiles)) + for iter.Next() { require.NoError(t, iter.Err()) } @@ -526,8 +529,10 @@ func TestCommitLogIteratorUsesPredicateFilter(t *testing.T) { FileFilterPredicate: commitLogPredicate, SeriesFilterPredicate: ReadAllSeriesPredicate(), } - iter, err := NewIterator(iterOpts) + iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) + require.Equal(t, 0, len(corruptFiles)) + iterStruct := iter.(*iterator) require.True(t, len(iterStruct.files) == 2) } diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index 09d9fc2564..f02877b15d 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -38,18 +38,65 @@ func (e fsError) Error() string { return e.err.Error() } +// ErrorWithPath is an error that includes the path of the file that +// had the error. +type ErrorWithPath struct { + err error + path string +} + +// Error returns the error. +func (e ErrorWithPath) Error() string { + return e.err.Error() +} + +// Path returns the path of hte file that the error is associated with. +func (e ErrorWithPath) Path() string { + return e.path +} + +// FileOrError is a union/option type that returns an error if there was +// any issue reading the commitlog info, or a File if there was not. Its +// purpose is to force callers to handle the error. +type FileOrError struct { + f File + // Contains any errors encountered when trying to read the commitlogs file info. We + // attempt to not include filesystem errors in this field, but that is accomplished + // on a best-effort basis and it is possible for this field to contain an error that + // is the result of a filesystem / O.S / hardware issue as opposed to an actually + // corrupt file. + e error +} + +// File returns a File if the commitlog info was read, or an error otherwise. +func (f FileOrError) File() (File, error) { + return f.f, f.e +} + +func newFileOrError(f File, e error, path string) FileOrError { + if e != nil { + e = newErrorWithPath(e, path) + } + + return FileOrError{ + f: f, + e: e, + } +} + // File represents a commit log file and its associated metadata. type File struct { FilePath string Start time.Time Duration time.Duration Index int64 - // Contains any errors encountered when trying to read the commitlogs file info. We - // attempt to not include filesystem errors in this field, but that is accomplished - // on a best-effort basis and it is possible for this field to contain an error that - // is the result of a filesystem / O.S / hardware issue as opposed to an actually - // corrupt file. - Error error +} + +func newErrorWithPath(err error, path string) ErrorWithPath { + return ErrorWithPath{ + err: err, + path: path, + } } // ReadLogInfo reads the commit log info out of a commitlog file @@ -94,7 +141,7 @@ func ReadLogInfo(filePath string, opts Options) (time.Time, time.Duration, int64 // Files returns a slice of all available commit log files on disk along with // their associated metadata. -func Files(opts Options) ([]File, error) { +func Files(opts Options) ([]FileOrError, error) { commitLogsDir := fs.CommitLogsDirPath( opts.FilesystemOptions().FilePathPrefix()) filePaths, err := fs.SortedCommitLogFiles(commitLogsDir) @@ -102,7 +149,7 @@ func Files(opts Options) ([]File, error) { return nil, err } - commitLogFiles := make([]File, 0, len(filePaths)) + commitLogFiles := make([]FileOrError, 0, len(filePaths)) for _, filePath := range filePaths { file := File{ FilePath: filePath, @@ -113,19 +160,19 @@ func Files(opts Options) ([]File, error) { return nil, err } - if err != nil { - file.Error = err - } else { + if err == nil { file.Start = start file.Duration = duration file.Index = index } - commitLogFiles = append(commitLogFiles, file) + commitLogFiles = append(commitLogFiles, newFileOrError( + file, err, filePath)) } sort.Slice(commitLogFiles, func(i, j int) bool { - return commitLogFiles[i].Start.Before(commitLogFiles[j].Start) + // Sorting is best effort here since we may not know the start. + return commitLogFiles[i].f.Start.Before(commitLogFiles[j].f.Start) }) return commitLogFiles, nil diff --git a/src/dbnode/persist/fs/commitlog/files_test.go b/src/dbnode/persist/fs/commitlog/files_test.go index d633bae2bc..973147ac44 100644 --- a/src/dbnode/persist/fs/commitlog/files_test.go +++ b/src/dbnode/persist/fs/commitlog/files_test.go @@ -40,7 +40,7 @@ import ( func TestFiles(t *testing.T) { // TODO(r): Find some time/people to help investigate this flakey test. t.Skip() - + dir, err := ioutil.TempDir("", "commitlogs") require.NoError(t, err) defer os.RemoveAll(dir) @@ -58,7 +58,10 @@ func TestFiles(t *testing.T) { // Make sure its sorted var lastFileStart time.Time - for _, file := range files { + for _, fileOrError := range files { + file, err := fileOrError.File() + require.NoError(t, err) + require.Equal(t, 10*time.Minute, file.Duration) require.Equal(t, int64(0), file.Index) require.True(t, strings.Contains(file.FilePath, dir)) diff --git a/src/dbnode/persist/fs/commitlog/iterator.go b/src/dbnode/persist/fs/commitlog/iterator.go index b03224ff2c..e2b533e54f 100644 --- a/src/dbnode/persist/fs/commitlog/iterator.go +++ b/src/dbnode/persist/fs/commitlog/iterator.go @@ -65,11 +65,11 @@ type iteratorRead struct { // ReadAllPredicate can be passed as the ReadCommitLogPredicate for callers // that want a convenient way to read all the commitlogs func ReadAllPredicate() FileFilterPredicate { - return func(_ File) (bool, bool) { return true, false } + return func(_ File) bool { return true } } // NewIterator creates a new commit log iterator -func NewIterator(iterOpts IteratorOpts) (iter Iterator, corruptFiles []string, err error) { +func NewIterator(iterOpts IteratorOpts) (iter Iterator, corruptFiles []ErrorWithPath, err error) { opts := iterOpts.CommitLogOptions iops := opts.InstrumentOptions() iops = iops.SetMetricsScope(iops.MetricsScope().SubScope("iterator")) @@ -190,19 +190,25 @@ func (i *iterator) nextReader() bool { return true } -func filterFiles(opts Options, files []File, predicate FileFilterPredicate) ([]File, []string) { +func filterFiles(opts Options, files []FileOrError, predicate FileFilterPredicate) ([]File, []ErrorWithPath) { var ( filtered = make([]File, 0, len(files)) - corrupt = make([]string, 0, len(files)) + corrupt = make([]ErrorWithPath, 0, len(files)) ) - for _, f := range files { - shouldRead, isCorrupt := predicate(f) - if shouldRead { - filtered = append(filtered, f) + for _, file := range files { + f, err := file.File() + if err != nil { + errorWithpath, ok := err.(ErrorWithPath) + if ok { + corrupt = append(corrupt, errorWithpath) + } + + continue } - if isCorrupt { - corrupt = append(corrupt, f.FilePath) + + if predicate(f) { + filtered = append(filtered, f) } } return filtered, corrupt diff --git a/src/dbnode/persist/fs/commitlog/types.go b/src/dbnode/persist/fs/commitlog/types.go index ce7be4fedd..f5fddc7c18 100644 --- a/src/dbnode/persist/fs/commitlog/types.go +++ b/src/dbnode/persist/fs/commitlog/types.go @@ -180,7 +180,7 @@ type Options interface { // FileFilterPredicate is a predicate that allows the caller to determine // which commitlogs the iterator should read from -type FileFilterPredicate func(f File) (shouldRead bool, isCorrupt bool) +type FileFilterPredicate func(f File) bool // SeriesFilterPredicate is a predicate that determines whether datapoints for a given series // should be returned from the Commit log reader. The predicate is pushed down to the diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 604a58eb05..69fb4c05a9 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -56,7 +56,7 @@ var ( const encoderChanBufSize = 1000 type newIteratorFn func(opts commitlog.IteratorOpts) ( - iter commitlog.Iterator, corruptFiles []string, err error) + iter commitlog.Iterator, corruptFiles []commitlog.ErrorWithPath, err error) type snapshotFilesFn func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error) type newReaderFn func(bytesPool pool.CheckedBytesPool, opts fs.Options) (fs.DataFileSetReader, error) @@ -248,6 +248,12 @@ func (s *commitLogSource) ReadData( return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } if len(corruptFiles) > 0 { + for _, f := range corruptFiles { + s.log. + Errorf( + "opting to skip commit log: %s due to corruption, err: %v", + f.Path, f.Error) + } encounteredCorruptData = true } @@ -804,21 +810,13 @@ func (s *commitLogSource) newReadCommitLogPred( // we need to read, but we can still skip datapoints from the commitlog itself that belong to a shard // that has a snapshot more recent than the global minimum. If we use an array for fast-access this could // be a small win in terms of memory utilization. - return func(f commitlog.File) (bool, bool) { - if f.Error != nil { - s.log. - Errorf( - "opting to skip commit log: %s due to corruption, err: %v", - f.FilePath, f.Error) - return false, true - } - + return func(f commitlog.File) bool { _, ok := commitlogFilesPresentBeforeStart[f.FilePath] if !ok { // If the file wasn't on disk before the node started then it only contains // writes that are already in memory (and in-fact the file may be actively // being written to.) - return false, false + return false } for _, rangeToCheck := range rangesToCheck { @@ -832,7 +830,7 @@ func (s *commitLogSource) newReadCommitLogPred( Infof( "opting to read commit log: %s with start: %s and duration: %s", f.FilePath, f.Start.String(), f.Duration.String()) - return true, false + return true } } @@ -840,7 +838,7 @@ func (s *commitLogSource) newReadCommitLogPred( Infof( "opting to skip commit log: %s with start: %s and duration: %s", f.FilePath, f.Start.String(), f.Duration.String()) - return false, false + return false } } @@ -1379,6 +1377,13 @@ func (s *commitLogSource) ReadIndex( return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } if len(corruptFiles) > 0 { + for _, f := range corruptFiles { + s.log. + Errorf( + "opting to skip commit log: %s due to corruption, err: %v", + f.Path, f.Error) + } + encounteredCorruptData = true } From 6515237b52be608e38b739ab627cd7c2b0067088 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:27:42 -0400 Subject: [PATCH 16/47] Refactor cleanup code and tests --- src/dbnode/persist/fs/commitlog/files.go | 5 +- src/dbnode/storage/cleanup.go | 69 ++++++++++--- src/dbnode/storage/cleanup_prop_test.go | 36 +++++-- src/dbnode/storage/cleanup_test.go | 124 +++++++++++++++-------- src/dbnode/storage/util.go | 8 +- 5 files changed, 169 insertions(+), 73 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index f02877b15d..8ae1890416 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -73,7 +73,8 @@ func (f FileOrError) File() (File, error) { return f.f, f.e } -func newFileOrError(f File, e error, path string) FileOrError { +// NewFileOrError creates a new FileOrError. +func NewFileOrError(f File, e error, path string) FileOrError { if e != nil { e = newErrorWithPath(e, path) } @@ -166,7 +167,7 @@ func Files(opts Options) ([]FileOrError, error) { file.Index = index } - commitLogFiles = append(commitLogFiles, newFileOrError( + commitLogFiles = append(commitLogFiles, NewFileOrError( file, err, filePath)) } diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index 03e00d4d7b..7ae729d36e 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -35,7 +35,7 @@ import ( "github.com/uber-go/tally" ) -type commitLogFilesFn func(commitlog.Options) ([]commitlog.File, error) +type commitLogFilesFn func(commitlog.Options) ([]commitlog.FileOrError, error) type deleteFilesFn func(files []string) error @@ -269,7 +269,7 @@ func (m *cleanupManager) cleanupNamespaceSnapshotFiles(earliestToRetain time.Tim // commitLogTimes returns the earliest time before which the commit logs are expired, // as well as a list of times we need to clean up commit log files for. -func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitlog.File, error) { +func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAndPath, error) { // NB(prateek): this logic of polling the namespaces across the commit log's entire // retention history could get expensive if commit logs are retained for long periods. // e.g. if we retain them for 40 days, with a block 2 hours; then every time @@ -290,16 +290,6 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitlog.File, error) { } shouldCleanupFile := func(f commitlog.File) (bool, error) { - if f.Error != nil { - // If we were unable to read the commit log files info header, then we're forced to assume - // that the file is corrupt and remove it. This can happen in situations where M3DB experiences - // sudden shutdown. - m.opts.InstrumentOptions().Logger().Errorf( - "encountered err: %v reading commit log file: %v info during cleanup, marking file for deletion", - f.Error, f.FilePath) - return true, nil - } - for _, ns := range namespaces { var ( start = f.Start @@ -328,6 +318,8 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitlog.File, error) { isCapturedBySnapshot, err := ns.IsCapturedBySnapshot( nsBlocksStart, nsBlocksEnd, start.Add(duration)) if err != nil { + // Return error because we don't want to proceed since this is not a commitlog + // file specific issue. return false, err } @@ -344,7 +336,54 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitlog.File, error) { return true, nil } - return filterCommitLogFiles(files, shouldCleanupFile) + filesToCleanup := make([]commitLogFileWithErrorAndPath, 0, len(files)) + for _, fileOrErr := range files { + f, err := fileOrErr.File() + + if err != nil { + // If we were unable to read the commit log files info header, then we're forced to assume + // that the file is corrupt and remove it. This can happen in situations where M3DB experiences + // sudden shutdown. + errorWithPath, ok := err.(commitlog.ErrorWithPath) + if !ok { + m.opts.InstrumentOptions().Logger().Errorf( + "commitlog file error did not contain path: %v", err) + // Continue because we want to try and clean up the remining files instead of erroring out. + continue + } + + m.opts.InstrumentOptions().Logger().Errorf( + "encountered err: %v reading commit log file: %v info during cleanup, marking file for deletion", + errorWithPath.Error(), errorWithPath.Path()) + + filesToCleanup = append(filesToCleanup, commitLogFileWithErrorAndPath{ + f: f, + path: errorWithPath.Path(), + err: err, + }) + continue + } + + shouldDelete, err := shouldCleanupFile(f) + if err != nil { + return nil, err + } + + if shouldDelete { + filesToCleanup = append(filesToCleanup, commitLogFileWithErrorAndPath{ + f: f, + path: f.FilePath, + }) + } + } + + return filesToCleanup, nil +} + +type commitLogFileWithErrorAndPath struct { + f commitlog.File + path string + err error } // commitLogNamespaceBlockTimes returns the range of namespace block starts for which the @@ -377,10 +416,10 @@ func commitLogNamespaceBlockTimes( return earliest, latest } -func (m *cleanupManager) cleanupCommitLogs(filesToCleanup []commitlog.File) error { +func (m *cleanupManager) cleanupCommitLogs(filesToCleanup []commitLogFileWithErrorAndPath) error { filesToDelete := make([]string, 0, len(filesToCleanup)) for _, f := range filesToCleanup { - filesToDelete = append(filesToDelete, f.FilePath) + filesToDelete = append(filesToDelete, f.path) } return m.deleteFilesFn(filesToDelete) } diff --git a/src/dbnode/storage/cleanup_prop_test.go b/src/dbnode/storage/cleanup_prop_test.go index 84ef0071c6..10927111b5 100644 --- a/src/dbnode/storage/cleanup_prop_test.go +++ b/src/dbnode/storage/cleanup_prop_test.go @@ -69,13 +69,14 @@ func newPropTestCleanupMgr( n = numIntervals(oldest, newest, blockSize) currStart = oldest ) - cm.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - files := make([]commitlog.File, 0, n) + cm.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + files := make([]commitlog.FileOrError, 0, n) for i := 0; i < n; i++ { - files = append(files, commitlog.File{ - Start: currStart, - Duration: blockSize, - }) + files = append(files, commitlog.NewFileOrError( + commitlog.File{ + Start: currStart, + Duration: blockSize, + }, nil, "path")) } return files, nil } @@ -105,11 +106,19 @@ func TestPropertyCommitLogNotCleanedForUnflushedData(t *testing.T) { if err != nil { return false, err } - for _, f := range filesToCleanup { - s, e := commitLogNamespaceBlockTimes(f.Start, f.Duration, ns.ropts) - needsFlush := ns.NeedsFlush(s, e) - isCapturedBySnapshot, err := ns.IsCapturedBySnapshot(s, e, f.Start.Add(f.Duration)) + for _, file := range filesToCleanup { + if file.err != nil { + continue + } + + var ( + f = file.f + s, e = commitLogNamespaceBlockTimes(f.Start, f.Duration, ns.ropts) + needsFlush = ns.NeedsFlush(s, e) + isCapturedBySnapshot, err = ns.IsCapturedBySnapshot(s, e, f.Start.Add(f.Duration)) + ) require.NoError(t, err) + if needsFlush && !isCapturedBySnapshot { return false, fmt.Errorf("trying to cleanup commit log at %v, but ns needsFlush; (range: %v, %v)", f.Start.String(), s.String(), e.String()) @@ -141,7 +150,12 @@ func TestPropertyCommitLogNotCleanedForUnflushedDataMultipleNs(t *testing.T) { if err != nil { return false, err } - for _, f := range filesToCleanup { + for _, file := range filesToCleanup { + if file.err != nil { + continue + } + + f := file.f for _, ns := range nses { s, e := commitLogNamespaceBlockTimes(f.Start, f.Duration, ns.ropts) needsFlush := ns.NeedsFlush(s, e) diff --git a/src/dbnode/storage/cleanup_test.go b/src/dbnode/storage/cleanup_test.go index 7932b4a84a..78fb606894 100644 --- a/src/dbnode/storage/cleanup_test.go +++ b/src/dbnode/storage/cleanup_test.go @@ -73,9 +73,11 @@ func TestCleanupManagerCleanup(t *testing.T) { mgr.opts.CommitLogOptions(). SetBlockSize(rOpts.BlockSize())) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{FilePath: "foo", Start: timeFor(14400)}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{FilePath: "foo", Start: timeFor(14400)}, + nil, "foo"), }, nil } var deletedFiles []string @@ -478,11 +480,17 @@ func TestCleanupManagerCommitLogTimesAllFlushed(t *testing.T) { defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - commitlog.File{Start: time30, Duration: commitLogBlockSize}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + nil, ""), }, nil } @@ -505,11 +513,17 @@ func TestCleanupManagerCommitLogTimesMiddlePendingFlush(t *testing.T) { defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - commitlog.File{Start: time30, Duration: commitLogBlockSize}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + nil, ""), }, nil } @@ -533,11 +547,17 @@ func TestCleanupManagerCommitLogTimesStartPendingFlush(t *testing.T) { defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - commitlog.File{Start: time30, Duration: commitLogBlockSize}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + nil, ""), }, nil } @@ -562,11 +582,17 @@ func TestCleanupManagerCommitLogTimesAllPendingFlush(t *testing.T) { defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - commitlog.File{Start: time30, Duration: commitLogBlockSize}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + nil, ""), }, nil } @@ -587,9 +613,9 @@ func timeFor(s int64) time.Time { return time.Unix(s, 0) } -func contains(arr []commitlog.File, t time.Time) bool { +func contains(arr []commitLogFileWithErrorAndPath, t time.Time) bool { for _, at := range arr { - if at.Start.Equal(t) { + if at.f.Start.Equal(t) { return true } } @@ -605,11 +631,17 @@ func TestCleanupManagerCommitLogTimesAllPendingFlushButHaveSnapshot(t *testing.T currentTime = timeFor(50) commitLogBlockSize = 10 * time.Second ) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - commitlog.File{Start: time30, Duration: commitLogBlockSize}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + nil, ""), }, nil } @@ -646,9 +678,11 @@ func TestCleanupManagerCommitLogTimesHandlesIsCapturedBySnapshotError(t *testing defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{Start: time30, Duration: commitLogBlockSize}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + nil, ""), }, nil } @@ -667,11 +701,17 @@ func TestCleanupManagerCommitLogTimesMultiNS(t *testing.T) { defer ctrl.Finish() ns1, ns2, mgr := newCleanupManagerCommitLogTimesTestMultiNS(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - commitlog.File{Start: time30, Duration: commitLogBlockSize}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + nil, ""), + commitlog.NewFileOrError( + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + nil, ""), }, nil } @@ -719,9 +759,11 @@ func TestCleanupManagerDeletesCorruptCommitLogFiles(t *testing.T) { _, mgr = newCleanupManagerCommitLogTimesTest(t, ctrl) err = errors.New("some_error") ) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, error) { - return []commitlog.File{ - commitlog.File{Start: time10, Error: err}, + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { + return []commitlog.FileOrError{ + commitlog.NewFileOrError( + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + err, ""), }, nil } diff --git a/src/dbnode/storage/util.go b/src/dbnode/storage/util.go index ba38b0fbf5..127fac0c20 100644 --- a/src/dbnode/storage/util.go +++ b/src/dbnode/storage/util.go @@ -58,10 +58,10 @@ func timesInRange(startInclusive, endInclusive time.Time, windowSize time.Durati // filterCommitLogFiles returns the values in the slice `files` which // satisfy the provided predicate. func filterCommitLogFiles( - files []commitlog.File, - predicate func(f commitlog.File) (bool, error), -) ([]commitlog.File, error) { - filtered := make([]commitlog.File, 0, len(files)) + files []commitlog.FileOrError, + predicate func(f commitlog.FileOrError) (bool, error), +) ([]commitlog.FileOrError, error) { + filtered := make([]commitlog.FileOrError, 0, len(files)) for _, f := range files { passed, err := predicate(f) if err != nil { From e60e7e27b83fe13183653c87c8ccdb4078ac0106 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:38:34 -0400 Subject: [PATCH 17/47] Fix flaky test --- src/dbnode/persist/fs/commitlog/files_test.go | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files_test.go b/src/dbnode/persist/fs/commitlog/files_test.go index 973147ac44..3eba481a6c 100644 --- a/src/dbnode/persist/fs/commitlog/files_test.go +++ b/src/dbnode/persist/fs/commitlog/files_test.go @@ -38,23 +38,23 @@ import ( ) func TestFiles(t *testing.T) { - // TODO(r): Find some time/people to help investigate this flakey test. - t.Skip() - dir, err := ioutil.TempDir("", "commitlogs") require.NoError(t, err) defer os.RemoveAll(dir) createTestCommitLogFiles(t, dir, 10*time.Minute, 5) - opts := NewOptions() + var ( + minNumBlocks = 5 + opts = NewOptions() + ) opts = opts.SetFilesystemOptions( opts.FilesystemOptions(). SetFilePathPrefix(dir), ) files, err := Files(opts) require.NoError(t, err) - require.Equal(t, 5, len(files)) + require.True(t, len(files) >= minNumBlocks) // Make sure its sorted var lastFileStart time.Time @@ -74,12 +74,12 @@ func TestFiles(t *testing.T) { } } -// createTestCommitLogFiles creates the specified number of commit log files -// on disk with the appropriate block size. Commit log files will be valid -// and contain readable metadata. +// createTestCommitLogFiles creates at least the specified number of commit log files +// on disk with the appropriate block size. Commit log files will be valid and contain +// readable metadata. func createTestCommitLogFiles( - t *testing.T, filePathPrefix string, blockSize time.Duration, numBlocks int) { - require.True(t, numBlocks >= 2) + t *testing.T, filePathPrefix string, blockSize time.Duration, minNumBlocks int) { + require.True(t, minNumBlocks >= 2) var ( nowLock = sync.RWMutex{} @@ -112,13 +112,13 @@ func createTestCommitLogFiles( } // Commit log writer is asynchronous and performs batching so getting the exact number // of files that we want is tricky. The implementation below loops infinitely, writing - // a single datapoint and increasing the time after each iteration until numBlocks -1 + // a single datapoint and increasing the time after each iteration until numBlocks // files are on disk. After that, it terminates, and the final batch flush from calling // commitlog.Close() will generate the last file. for { files, err := fs.SortedCommitLogFiles(commitLogsDir) require.NoError(t, err) - if len(files) == numBlocks-1 { + if len(files) == minNumBlocks { break } err = commitLog.Write(context.NewContext(), series, ts.Datapoint{}, xtime.Second, nil) @@ -129,5 +129,5 @@ func createTestCommitLogFiles( require.NoError(t, commitLog.Close()) files, err := fs.SortedCommitLogFiles(commitLogsDir) require.NoError(t, err) - require.Equal(t, numBlocks, len(files)) + require.True(t, len(files) >= minNumBlocks) } From acf3b341a35617a072d3f52a25bfc568797123cd Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:41:02 -0400 Subject: [PATCH 18/47] share logging code --- .../bootstrapper/commitlog/source.go | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 69fb4c05a9..c6f103c802 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -248,12 +248,7 @@ func (s *commitLogSource) ReadData( return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } if len(corruptFiles) > 0 { - for _, f := range corruptFiles { - s.log. - Errorf( - "opting to skip commit log: %s due to corruption, err: %v", - f.Path, f.Error) - } + s.logCorruptFiles(corruptFiles) encounteredCorruptData = true } @@ -1377,13 +1372,7 @@ func (s *commitLogSource) ReadIndex( return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } if len(corruptFiles) > 0 { - for _, f := range corruptFiles { - s.log. - Errorf( - "opting to skip commit log: %s due to corruption, err: %v", - f.Path, f.Error) - } - + s.logCorruptFiles(corruptFiles) encounteredCorruptData = true } @@ -1487,6 +1476,15 @@ func (s commitLogSource) maybeAddToIndex( return err } +func (s *commitLogSource) logCorruptFiles(corruptFiles []commitlog.ErrorWithPath) { + for _, f := range corruptFiles { + s.log. + Errorf( + "opting to skip commit log: %s due to corruption, err: %v", + f.Path, f.Error) + } +} + // The commitlog bootstrapper determines availability primarily by checking if the // origin host has ever reached the "Available" state for the shard that is being // bootstrapped. If not, then it can't provide data for that shard because it doesn't From cbf359ee1782fa2186a4e9e64f46459e5c5c31ef Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:48:55 -0400 Subject: [PATCH 19/47] more refactoring --- .../bootstrapper/commitlog/source.go | 5 ++- src/dbnode/storage/cleanup.go | 34 +++++++++++-------- src/dbnode/storage/util.go | 21 ------------ 3 files changed, 21 insertions(+), 39 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index c6f103c802..886e2a2a08 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -350,7 +350,7 @@ func (s *commitLogSource) ReadData( if encounteredCorruptData && couldObtainDataFromPeers { // If we encountered any corrupt data and there is a possibility of the // peers bootstrapper being able to correct it, mark the entire range - // as unfulfilled so Peers bootstrapper can attempt a repair, but keep + // as unfulfilled so the peers bootstrapper can attempt a repair, but keep // the data we read from the commit log as well in case the peers // bootstrapper is unable to satisfy the bootstrap because all peers are // down or if the commitlog contained data that the peers do not have. @@ -1349,7 +1349,6 @@ func (s *commitLogSource) ReadIndex( ns.ID(), shard, true, tr, blockSize, snapshotFilesByShard[shard], mostRecentCompleteSnapshotByBlockShard) if err != nil { - // TODO: Probably should not return an error here return nil, err } @@ -1426,7 +1425,7 @@ func (s *commitLogSource) ReadIndex( if encounteredCorruptData && couldObtainDataFromPeers { // If we encountered any corrupt data and there is a possibility of the // peers bootstrapper being able to correct it, mark the entire range - // as unfulfilled so Peers bootstrapper can attempt a repair, but keep + // as unfulfilled so the peers bootstrapper can attempt a repair, but keep // the data we read from the commit log as well in case the peers // bootstrapper is unable to satisfy the bootstrap because all peers are // down or if the commitlog contained data that the peers do not have. diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index 7ae729d36e..fde9b7c1ea 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -356,11 +356,8 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAn "encountered err: %v reading commit log file: %v info during cleanup, marking file for deletion", errorWithPath.Error(), errorWithPath.Path()) - filesToCleanup = append(filesToCleanup, commitLogFileWithErrorAndPath{ - f: f, - path: errorWithPath.Path(), - err: err, - }) + filesToCleanup = append(filesToCleanup, newCommitLogFileWithErrorAndPath( + f, errorWithPath.Path(), err)) continue } @@ -370,22 +367,14 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAn } if shouldDelete { - filesToCleanup = append(filesToCleanup, commitLogFileWithErrorAndPath{ - f: f, - path: f.FilePath, - }) + filesToCleanup = append(filesToCleanup, newCommitLogFileWithErrorAndPath( + f, f.FilePath, nil)) } } return filesToCleanup, nil } -type commitLogFileWithErrorAndPath struct { - f commitlog.File - path string - err error -} - // commitLogNamespaceBlockTimes returns the range of namespace block starts for which the // given commit log block may contain data for. // @@ -423,3 +412,18 @@ func (m *cleanupManager) cleanupCommitLogs(filesToCleanup []commitLogFileWithErr } return m.deleteFilesFn(filesToDelete) } + +type commitLogFileWithErrorAndPath struct { + f commitlog.File + path string + err error +} + +func newCommitLogFileWithErrorAndPath( + f commitlog.File, path string, err error) commitLogFileWithErrorAndPath { + return commitLogFileWithErrorAndPath{ + f: f, + path: path, + err: err, + } +} diff --git a/src/dbnode/storage/util.go b/src/dbnode/storage/util.go index 127fac0c20..87faa3a8a2 100644 --- a/src/dbnode/storage/util.go +++ b/src/dbnode/storage/util.go @@ -22,8 +22,6 @@ package storage import ( "time" - - "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" ) // numIntervals returns the number of intervals between [start, end] for a given @@ -55,25 +53,6 @@ func timesInRange(startInclusive, endInclusive time.Time, windowSize time.Durati return times } -// filterCommitLogFiles returns the values in the slice `files` which -// satisfy the provided predicate. -func filterCommitLogFiles( - files []commitlog.FileOrError, - predicate func(f commitlog.FileOrError) (bool, error), -) ([]commitlog.FileOrError, error) { - filtered := make([]commitlog.FileOrError, 0, len(files)) - for _, f := range files { - passed, err := predicate(f) - if err != nil { - return nil, err - } - if passed { - filtered = append(filtered, f) - } - } - return filtered, nil -} - // filterTimes returns the values in the slice `times` which satisfy // the provided predicate. func filterTimes(times []time.Time, predicate func(t time.Time) bool) []time.Time { From 422c423eb812f85f33c996cd1281cbb867653a53 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 11:15:25 -0700 Subject: [PATCH 20/47] Improve logic for determining if peer bootstrapper could provide any data in commitlog source --- src/cmd/services/m3dbnode/config/bootstrap.go | 3 +- .../bootstrapper/commitlog/options.go | 16 ++++ .../bootstrapper/commitlog/source.go | 73 ++++++++++++++++--- .../bootstrap/bootstrapper/commitlog/types.go | 7 ++ .../bootstrap/bootstrapper/peers/options.go | 4 + 5 files changed, 92 insertions(+), 11 deletions(-) diff --git a/src/cmd/services/m3dbnode/config/bootstrap.go b/src/cmd/services/m3dbnode/config/bootstrap.go index eced3b6ef5..1609e3d997 100644 --- a/src/cmd/services/m3dbnode/config/bootstrap.go +++ b/src/cmd/services/m3dbnode/config/bootstrap.go @@ -140,7 +140,8 @@ func (bsc BootstrapConfiguration) New( case commitlog.CommitLogBootstrapperName: cOpts := commitlog.NewOptions(). SetResultOptions(rsOpts). - SetCommitLogOptions(opts.CommitLogOptions()) + SetCommitLogOptions(opts.CommitLogOptions()). + SetRuntimeOptionsManager(opts.RuntimeOptionsManager()) inspection, err := fs.InspectFilesystem(fsOpts) if err != nil { diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go index bbca8221e4..7ed2c4865d 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go @@ -24,6 +24,7 @@ import ( "errors" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" + "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" ) @@ -35,6 +36,7 @@ const ( var ( errEncodingConcurrencyPositive = errors.New("encoding concurrency must be positive") errMergeShardConcurrencyPositive = errors.New("merge shard concurrency must be positive") + errRuntimeOptsMgrNotSet = errors.New("runtime options manager is not set") ) type options struct { @@ -42,6 +44,7 @@ type options struct { commitLogOpts commitlog.Options encodingConcurrency int mergeShardConcurrency int + runtimeOptsMgr runtime.OptionsManager } // NewOptions creates new bootstrap options @@ -61,6 +64,9 @@ func (o *options) Validate() error { if o.mergeShardConcurrency <= 0 { return errMergeShardConcurrencyPositive } + if o.runtimeOptsMgr == nil { + return errRuntimeOptsMgrNotSet + } return o.commitLogOpts.Validate() } @@ -103,3 +109,13 @@ func (o *options) SetMergeShardsConcurrency(value int) Options { func (o *options) MergeShardsConcurrency() int { return o.mergeShardConcurrency } + +func (o *options) SetRuntimeOptionsManager(value runtime.OptionsManager) Options { + opts := *o + opts.runtimeOptsMgr = value + return &opts +} + +func (o *options) RuntimeOptionsManager() runtime.OptionsManager { + return o.runtimeOptsMgr +} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 886e2a2a08..77bc47bf1a 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -345,8 +345,12 @@ func (s *commitLogSource) ReadData( } s.log.Infof("done merging..., took: %s", time.Since(mergeStart).String()) - couldObtainDataFromPeers := s.couldObtainDataFromPeers( + couldObtainDataFromPeers, err := s.couldObtainDataFromPeers( ns, shardsTimeRanges, runOpts) + if err != nil { + return nil, err + } + if encounteredCorruptData && couldObtainDataFromPeers { // If we encountered any corrupt data and there is a possibility of the // peers bootstrapper being able to correct it, mark the entire range @@ -1420,8 +1424,12 @@ func (s *commitLogSource) ReadIndex( } } - couldObtainDataFromPeers := s.couldObtainDataFromPeers( + couldObtainDataFromPeers, err := s.couldObtainDataFromPeers( ns, shardsTimeRanges, opts) + if err != nil { + return nil, err + } + if encounteredCorruptData && couldObtainDataFromPeers { // If we encountered any corrupt data and there is a possibility of the // peers bootstrapper being able to correct it, mark the entire range @@ -1553,15 +1561,60 @@ func (s *commitLogSource) couldObtainDataFromPeers( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, runOpts bootstrap.RunOptions, -) bool { - // TODO: Refactor InitialTopologyState to store Replicas along with MajorityReplicas - // TODO: Actually also need to check the shard state of the peers too because if they're - // not available we can't get data from them. - initialTopologyState := runOpts.InitialTopologyState() - if initialTopologyState.MajorityReplicas > 1 { - return true +) (bool, error) { + var ( + initialTopologyState = runOpts.InitialTopologyState() + majorityReplicas = initialTopologyState.MajorityReplicas + runtimeOpts = s.opts.RuntimeOptionsManager().Get() + bootstrapConsistencyLevel = runtimeOpts.ClientBootstrapConsistencyLevel() + ) + + for shardIDUint := range shardsTimeRanges { + shardID := topology.ShardID(shardIDUint) + hostShardStates, ok := initialTopologyState.ShardStates[shardID] + if !ok { + // This shard was not part of the topology when the bootstrapping process began. + continue + } + + var ( + numPeers = len(hostShardStates) + numAvailablePeers = 0 + ) + for _, hostShardState := range hostShardStates { + if hostShardState.Host.ID() == initialTopologyState.Origin.ID() { + // Don't take self into account + continue + } + + shardState := hostShardState.ShardState + switch shardState { + // Don't want to peer bootstrap from a node that has not yet completely + // taken ownership of the shard. + case shard.Initializing: + // Success cases - We can bootstrap from this host, which is enough to + // mark this shard as bootstrappable. + case shard.Leaving: + fallthrough + case shard.Available: + numAvailablePeers++ + case shard.Unknown: + fallthrough + default: + return false, fmt.Errorf("unknown shard state: %v", shardState) + } + } + + if topology.ReadConsistencyAchieved( + bootstrapConsistencyLevel, majorityReplicas, numPeers, numAvailablePeers) { + // If we can achieve read consistency for any shard than we return true because + // we can't make any shard-by-shard distinction due to the fact that any given + // commitlog can contain writes for any shard. + return true, nil + } } - return false + + return false, nil } func newReadSeriesPredicate(ns namespace.Metadata) commitlog.SeriesFilterPredicate { diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go index 047a100369..bb827982ef 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go @@ -22,6 +22,7 @@ package commitlog import ( "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" + "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" ) @@ -53,4 +54,10 @@ type Options interface { // MergeShardConcurrency returns the concurrency for merging shards MergeShardsConcurrency() int + + // SetRuntimeOptionsManagers sets the RuntimeOptionsManager. + SetRuntimeOptionsManager(value runtime.OptionsManager) Options + + // RuntimeOptionsManagers returns the RuntimeOptionsManager. + RuntimeOptionsManager() runtime.OptionsManager } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/peers/options.go b/src/dbnode/storage/bootstrap/bootstrapper/peers/options.go index 43b83a2b93..aa3935f6d4 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/peers/options.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/peers/options.go @@ -43,6 +43,7 @@ var ( errAdminClientNotSet = errors.New("admin client not set") errInvalidFetchBlocksMetadataVersion = errors.New("invalid fetch blocks metadata endpoint version") errPersistManagerNotSet = errors.New("persist manager not set") + errRuntimeOptionsManagerNotSet = errors.New("runtime options manager not set") ) type options struct { @@ -78,6 +79,9 @@ func (o *options) Validate() error { if o.persistManager == nil { return errPersistManagerNotSet } + if o.runtimeOptionsManager == nil { + return errRuntimeOptionsManagerNotSet + } return nil } From e82986420e71d15364f85faf11a0c38f93800792 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 11:35:06 -0700 Subject: [PATCH 21/47] Fix tests --- .../bootstrap/bootstrapper/commitlog/source.go | 3 +++ .../bootstrapper/commitlog/source_data_test.go | 16 +++++++++------- .../bootstrapper/commitlog/source_index_test.go | 8 ++++---- .../bootstrapper/commitlog/source_prop_test.go | 10 ++++++++-- 4 files changed, 24 insertions(+), 13 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 77bc47bf1a..2e437a93a1 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -1605,6 +1605,9 @@ func (s *commitLogSource) couldObtainDataFromPeers( } } + fmt.Println("majority: ", majorityReplicas) + fmt.Println("numPeers: ", numPeers) + fmt.Println("numAvailablePeers: ", numAvailablePeers) if topology.ReadConsistencyAchieved( bootstrapConsistencyLevel, majorityReplicas, numPeers, numAvailablePeers) { // If we can achieve read consistency for any shard than we return true because diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go index bab10bfe6e..8947cc8e1d 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go @@ -34,6 +34,7 @@ import ( "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/storage/block" "github.com/m3db/m3/src/dbnode/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" @@ -84,7 +85,8 @@ func testOptions() Options { return opts.SetResultOptions(ropts.SetDatabaseBlockOptions(rlopts. SetEncoderPool(encoderPool). SetReaderIteratorPool(readerIteratorPool). - SetMultiReaderIteratorPool(multiReaderIteratorPool))) + SetMultiReaderIteratorPool(multiReaderIteratorPool))). + SetRuntimeOptionsManager(runtime.NewOptionsManager()) } func TestAvailableEmptyRangeError(t *testing.T) { @@ -113,7 +115,7 @@ func TestReadErrorOnNewIteratorError(t *testing.T) { opts := testDefaultOpts src := newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return nil, nil, fmt.Errorf("an error") } @@ -159,7 +161,7 @@ func TestReadOrderedValues(t *testing.T) { // "baz" is in shard 2 and should not be returned {baz, start.Add(4 * time.Minute), 1.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } @@ -201,7 +203,7 @@ func TestReadUnorderedValues(t *testing.T) { {foo, start.Add(3 * time.Minute), 4.0, xtime.Second, nil}, {foo, start, 5.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } @@ -247,7 +249,7 @@ func TestReadHandlesDifferentSeriesWithIdenticalUniqueIndex(t *testing.T) { {foo, start, 1.0, xtime.Second, nil}, {bar, start, 2.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } @@ -288,7 +290,7 @@ func TestReadTrimsToRanges(t *testing.T) { {foo, start.Add(1 * time.Minute), 3.0, xtime.Nanosecond, nil}, {foo, end.Truncate(blockSize).Add(blockSize).Add(time.Nanosecond), 4.0, xtime.Nanosecond, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } @@ -336,7 +338,7 @@ func TestItMergesSnapshotsAndCommitLogs(t *testing.T) { End: end, }) - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(commitLogValues, nil), nil, nil } src.snapshotFilesFn = func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error) { diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go index 9a1599bb14..0e712487cc 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go @@ -117,7 +117,7 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { {someOtherNamespace, start.Add(dataBlockSize), 1.0, xtime.Second, nil}, } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } @@ -195,7 +195,7 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { otherNamespaceValues = append(otherNamespaceValues, value) } } - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(otherNamespaceValues, nil), nil, nil } @@ -212,7 +212,7 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { // Update the iterator function to return no values (since this namespace has no data) // because the real commit log reader does this (via the ReadSeries predicate). - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator([]testValue{}, nil), nil, nil } @@ -253,7 +253,7 @@ func TestBootstrapIndexEmptyShardTimeRanges(t *testing.T) { require.NoError(t, err) values := []testValue{} - src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []string, error) { + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, []commitlog.ErrorWithPath, error) { return newTestCommitLogIterator(values, nil), nil, nil } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 8879c83121..2da7dec518 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -1,3 +1,4 @@ +// +build big // // Copyright (c) 2017 Uber Technologies, Inc. // @@ -292,7 +293,12 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { } if len(commitLogFiles) > 0 { - lastCommitLogFile := commitLogFiles[len(commitLogFiles)-1] + lastCommitLogFileOrErr := commitLogFiles[len(commitLogFiles)-1] + lastCommitLogFile, err := lastCommitLogFileOrErr.File() + if err != nil { + return false, err + } + nextCommitLogFile, _, err := fs.NextCommitLogsFile( fsOpts.FilePathPrefix(), lastCommitLogFile.Start) if err != nil { @@ -352,7 +358,7 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { // Perform the bootstrap var initialTopoState *topology.StateSnapshot if input.multiNodeCluster { - initialTopoState = tu.NewStateSnapshot(3, tu.HostShardStates{ + initialTopoState = tu.NewStateSnapshot(2, tu.HostShardStates{ tu.SelfID: tu.Shards(allShardsSlice, shard.Available), "not-self1": tu.Shards(allShardsSlice, shard.Available), "not-self2": tu.Shards(allShardsSlice, shard.Available), From be684ff80a9471cad3dc20d10633153a7793bfde Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 12:00:03 -0700 Subject: [PATCH 22/47] More refactoring --- src/cmd/services/m3dbnode/config/bootstrap.go | 14 +++++ .../bootstrapper/commitlog/options.go | 35 ++++++++---- .../bootstrapper/commitlog/source.go | 57 ++++++++++++------- .../bootstrap/bootstrapper/commitlog/types.go | 8 +++ 4 files changed, 82 insertions(+), 32 deletions(-) diff --git a/src/cmd/services/m3dbnode/config/bootstrap.go b/src/cmd/services/m3dbnode/config/bootstrap.go index 1609e3d997..354791c3d9 100644 --- a/src/cmd/services/m3dbnode/config/bootstrap.go +++ b/src/cmd/services/m3dbnode/config/bootstrap.go @@ -56,6 +56,9 @@ type BootstrapConfiguration struct { // Peers bootstrapper configuration. Peers *BootstrapPeersConfiguration `yaml:"peers"` + // Commitlog bootstrapper configuration. + Commitlog *BootstrapCommitlogConfiguration `yaml:"commitlog"` + // CacheSeriesMetadata determines whether individual bootstrappers cache // series metadata across all calls (namespaces / shards / blocks). CacheSeriesMetadata *bool `yaml:"cacheSeriesMetadata"` @@ -91,6 +94,17 @@ type BootstrapPeersConfiguration struct { FetchBlocksMetadataEndpointVersion client.FetchBlocksMetadataEndpointVersion `yaml:"fetchBlocksMetadataEndpointVersion"` } +// BootstrapCommitlogConfiguration specifies config for the commitlog bootstrapper. +type BootstrapCommitlogConfiguration struct { + // ReturnUnfulfilledForCorruptCommitlogFiles controls whether the commitlog bootstrapper + // will return unfulfilled for all shard time ranges when it encounters a corrupt commit + // file. Note that regardless of this value, the commitlog bootstrapper will still try and + // read all the uncorrupted commitlog files and return as much data as it can, but setting + // this to true allows the node to attempt a repair if the peers bootstrapper is configured + // after the commitlog bootstrapper. + ReturnUnfulfilledForCorruptCommitlogFiles bool `yaml:"returnUnfulfilledForCorruptCommitlogFiles"` +} + // New creates a bootstrap process based on the bootstrap configuration. func (bsc BootstrapConfiguration) New( opts storage.Options, diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go index 7ed2c4865d..ae585620bf 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/options.go @@ -29,8 +29,9 @@ import ( ) const ( - defaultEncodingConcurrency = 4 - defaultMergeShardConcurrency = 4 + defaultEncodingConcurrency = 4 + defaultMergeShardConcurrency = 4 + defaultReturnUnfulfilledForCorruptCommitlogFiles = true ) var ( @@ -40,20 +41,22 @@ var ( ) type options struct { - resultOpts result.Options - commitLogOpts commitlog.Options - encodingConcurrency int - mergeShardConcurrency int - runtimeOptsMgr runtime.OptionsManager + resultOpts result.Options + commitLogOpts commitlog.Options + encodingConcurrency int + mergeShardConcurrency int + runtimeOptsMgr runtime.OptionsManager + returnUnfulfilledForCorruptCommitlogFiles bool } // NewOptions creates new bootstrap options func NewOptions() Options { return &options{ - resultOpts: result.NewOptions(), - commitLogOpts: commitlog.NewOptions(), - encodingConcurrency: defaultEncodingConcurrency, - mergeShardConcurrency: defaultMergeShardConcurrency, + resultOpts: result.NewOptions(), + commitLogOpts: commitlog.NewOptions(), + encodingConcurrency: defaultEncodingConcurrency, + mergeShardConcurrency: defaultMergeShardConcurrency, + returnUnfulfilledForCorruptCommitlogFiles: defaultReturnUnfulfilledForCorruptCommitlogFiles, } } @@ -119,3 +122,13 @@ func (o *options) SetRuntimeOptionsManager(value runtime.OptionsManager) Options func (o *options) RuntimeOptionsManager() runtime.OptionsManager { return o.runtimeOptsMgr } + +func (o *options) SetReturnUnfulfilledForCorruptCommitlogFiles(value bool) Options { + opts := *o + opts.returnUnfulfilledForCorruptCommitlogFiles = value + return &opts +} + +func (o *options) ReturnUnfulfilledForCorruptCommitlogFiles() bool { + return o.returnUnfulfilledForCorruptCommitlogFiles +} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 2e437a93a1..cef3837dfb 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -345,21 +345,16 @@ func (s *commitLogSource) ReadData( } s.log.Infof("done merging..., took: %s", time.Since(mergeStart).String()) - couldObtainDataFromPeers, err := s.couldObtainDataFromPeers( - ns, shardsTimeRanges, runOpts) + shouldReturnUnfulfilled, err := s.shouldReturnUnfulfilled( + encounteredCorruptData, ns, shardsTimeRanges, runOpts) if err != nil { return nil, err } - if encounteredCorruptData && couldObtainDataFromPeers { - // If we encountered any corrupt data and there is a possibility of the - // peers bootstrapper being able to correct it, mark the entire range - // as unfulfilled so the peers bootstrapper can attempt a repair, but keep - // the data we read from the commit log as well in case the peers - // bootstrapper is unable to satisfy the bootstrap because all peers are - // down or if the commitlog contained data that the peers do not have. + if shouldReturnUnfulfilled { bootstrapResult.SetUnfulfilled(shardsTimeRanges) } + return bootstrapResult, nil } @@ -1424,24 +1419,47 @@ func (s *commitLogSource) ReadIndex( } } - couldObtainDataFromPeers, err := s.couldObtainDataFromPeers( - ns, shardsTimeRanges, opts) + shouldReturnUnfulfilled, err := s.shouldReturnUnfulfilled( + encounteredCorruptData, ns, shardsTimeRanges, opts) if err != nil { return nil, err } - if encounteredCorruptData && couldObtainDataFromPeers { - // If we encountered any corrupt data and there is a possibility of the - // peers bootstrapper being able to correct it, mark the entire range - // as unfulfilled so the peers bootstrapper can attempt a repair, but keep - // the data we read from the commit log as well in case the peers - // bootstrapper is unable to satisfy the bootstrap because all peers are - // down or if the commitlog contained data that the peers do not have. + if shouldReturnUnfulfilled { indexResult.SetUnfulfilled(shardsTimeRanges) } return indexResult, nil } +// If we encountered any corrupt data and there is a possibility of the +// peers bootstrapper being able to correct it, we want to mark the entire range +// as unfulfilled so the peers bootstrapper can attempt a repair, but keep +// the data we read from the commit log as well in case the peers +// bootstrapper is unable to satisfy the bootstrap because all peers are +// down or if the commitlog contained data that the peers do not have. +func (s commitLogSource) shouldReturnUnfulfilled( + encounteredCorruptData bool, + ns namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, + opts bootstrap.RunOptions, +) (bool, error) { + if !s.opts.ReturnUnfulfilledForCorruptCommitlogFiles() { + return false, nil + } + + if !encounteredCorruptData { + return false, nil + } + + couldObtainDataFromPeers, err := s.couldObtainDataFromPeers( + ns, shardsTimeRanges, opts) + if err != nil { + return false, err + } + + return couldObtainDataFromPeers, nil +} + func (s commitLogSource) maybeAddToIndex( id ident.ID, tags ident.Tags, @@ -1605,9 +1623,6 @@ func (s *commitLogSource) couldObtainDataFromPeers( } } - fmt.Println("majority: ", majorityReplicas) - fmt.Println("numPeers: ", numPeers) - fmt.Println("numAvailablePeers: ", numAvailablePeers) if topology.ReadConsistencyAchieved( bootstrapConsistencyLevel, majorityReplicas, numPeers, numAvailablePeers) { // If we can achieve read consistency for any shard than we return true because diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go index bb827982ef..59f89c72d0 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/types.go @@ -55,6 +55,14 @@ type Options interface { // MergeShardConcurrency returns the concurrency for merging shards MergeShardsConcurrency() int + // SetReturnUnfulfilledForCorruptCommitlogFiles sets whether the bootstrapper + // should return unfulfilled if it encounters corrupt commitlog files. + SetReturnUnfulfilledForCorruptCommitlogFiles(value bool) Options + + // ReturnUnfulfilledForCorruptCommitlogFiles returns whether the bootstrapper + // should return unfulfilled if it encounters corrupt commitlog files. + ReturnUnfulfilledForCorruptCommitlogFiles() bool + // SetRuntimeOptionsManagers sets the RuntimeOptionsManager. SetRuntimeOptionsManager(value runtime.OptionsManager) Options From 0fd6c9d48edc94a338d6f15eb963e8237ff6c635 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:08:08 -0700 Subject: [PATCH 23/47] Fix broken tests --- .../bootstrap_after_buffer_rotation_regression_test.go | 3 ++- ...ootstrap_before_buffer_rotation_no_tick_regression_test.go | 4 +++- src/dbnode/integration/bootstrap_helpers.go | 3 ++- .../integration/fs_commitlog_mixed_mode_read_write_test.go | 3 ++- 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go index aa0895ed82..b95fd3068d 100644 --- a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go +++ b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go @@ -108,7 +108,8 @@ func TestBootstrapAfterBufferRotation(t *testing.T) { bootstrapOpts := newDefaulTestResultOptions(setup.storageOpts) bootstrapCommitlogOpts := bcl.NewOptions(). SetResultOptions(bootstrapOpts). - SetCommitLogOptions(commitLogOpts) + SetCommitLogOptions(commitLogOpts). + SetRuntimeManager(runtime.runtime.NewOptionsManager()) fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() commitlogBootstrapperProvider, err := bcl.NewCommitLogBootstrapperProvider( bootstrapCommitlogOpts, mustInspectFilesystem(fsOpts), nil) diff --git a/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go b/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go index 1fb9db87f1..64db69060a 100644 --- a/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go +++ b/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go @@ -28,6 +28,7 @@ import ( "github.com/m3db/m3/src/dbnode/integration/generate" "github.com/m3db/m3/src/dbnode/retention" + "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/bootstrap" bcl "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" @@ -122,7 +123,8 @@ func TestBootstrapBeforeBufferRotationNoTick(t *testing.T) { bootstrapOpts := newDefaulTestResultOptions(setup.storageOpts) bootstrapCommitlogOpts := bcl.NewOptions(). SetResultOptions(bootstrapOpts). - SetCommitLogOptions(commitLogOpts) + SetCommitLogOptions(commitLogOpts). + SetRuntimeManager(runtime.runtime.NewOptionsManager()) fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() commitlogBootstrapperProvider, err := bcl.NewCommitLogBootstrapperProvider( bootstrapCommitlogOpts, mustInspectFilesystem(fsOpts), nil) diff --git a/src/dbnode/integration/bootstrap_helpers.go b/src/dbnode/integration/bootstrap_helpers.go index bf014d58b4..2d703ef607 100644 --- a/src/dbnode/integration/bootstrap_helpers.go +++ b/src/dbnode/integration/bootstrap_helpers.go @@ -173,7 +173,8 @@ func setupCommitLogBootstrapperWithFSInspection( bsOpts := newDefaulTestResultOptions(setup.storageOpts) bclOpts := bcl.NewOptions(). SetResultOptions(bsOpts). - SetCommitLogOptions(commitLogOpts) + SetCommitLogOptions(commitLogOpts). + SetRuntimeManager(runtime.runtime.NewOptionsManager()) fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() bs, err := bcl.NewCommitLogBootstrapperProvider( bclOpts, mustInspectFilesystem(fsOpts), noOpAll) diff --git a/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go b/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go index 67ded77cf0..e770c69e6d 100644 --- a/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go +++ b/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go @@ -225,7 +225,8 @@ func setCommitLogAndFilesystemBootstrapper(t *testing.T, opts testOptions, setup bsOpts := newDefaulTestResultOptions(setup.storageOpts) bclOpts := bcl.NewOptions(). SetResultOptions(bsOpts). - SetCommitLogOptions(commitLogOpts) + SetCommitLogOptions(commitLogOpts). + SetRuntimeManager(runtime.runtime.NewOptionsManager()) commitLogBootstrapper, err := bcl.NewCommitLogBootstrapperProvider( bclOpts, mustInspectFilesystem(fsOpts), noOpAll) From f57e71b0acaed4831cd805a61c7cd857d7ac8620 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:13:18 -0700 Subject: [PATCH 24/47] fix test --- .../bootstrap_after_buffer_rotation_regression_test.go | 3 ++- ...bootstrap_before_buffer_rotation_no_tick_regression_test.go | 2 +- src/dbnode/integration/bootstrap_helpers.go | 3 ++- .../integration/fs_commitlog_mixed_mode_read_write_test.go | 3 ++- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go index b95fd3068d..a7c80a57c3 100644 --- a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go +++ b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go @@ -28,6 +28,7 @@ import ( "github.com/m3db/m3/src/dbnode/integration/generate" "github.com/m3db/m3/src/dbnode/retention" + "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/bootstrap" bcl "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" @@ -109,7 +110,7 @@ func TestBootstrapAfterBufferRotation(t *testing.T) { bootstrapCommitlogOpts := bcl.NewOptions(). SetResultOptions(bootstrapOpts). SetCommitLogOptions(commitLogOpts). - SetRuntimeManager(runtime.runtime.NewOptionsManager()) + SetRuntimeOptionsManager(runtime.NewOptionsManager()) fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() commitlogBootstrapperProvider, err := bcl.NewCommitLogBootstrapperProvider( bootstrapCommitlogOpts, mustInspectFilesystem(fsOpts), nil) diff --git a/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go b/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go index 64db69060a..4d5b6efce1 100644 --- a/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go +++ b/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go @@ -124,7 +124,7 @@ func TestBootstrapBeforeBufferRotationNoTick(t *testing.T) { bootstrapCommitlogOpts := bcl.NewOptions(). SetResultOptions(bootstrapOpts). SetCommitLogOptions(commitLogOpts). - SetRuntimeManager(runtime.runtime.NewOptionsManager()) + SetRuntimeOptionsManager(runtime.NewOptionsManager()) fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() commitlogBootstrapperProvider, err := bcl.NewCommitLogBootstrapperProvider( bootstrapCommitlogOpts, mustInspectFilesystem(fsOpts), nil) diff --git a/src/dbnode/integration/bootstrap_helpers.go b/src/dbnode/integration/bootstrap_helpers.go index 2d703ef607..0a1dfb15f6 100644 --- a/src/dbnode/integration/bootstrap_helpers.go +++ b/src/dbnode/integration/bootstrap_helpers.go @@ -27,6 +27,7 @@ import ( "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/storage/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" bcl "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" @@ -174,7 +175,7 @@ func setupCommitLogBootstrapperWithFSInspection( bclOpts := bcl.NewOptions(). SetResultOptions(bsOpts). SetCommitLogOptions(commitLogOpts). - SetRuntimeManager(runtime.runtime.NewOptionsManager()) + SetRuntimeOptionsManager(runtime.NewOptionsManager()) fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() bs, err := bcl.NewCommitLogBootstrapperProvider( bclOpts, mustInspectFilesystem(fsOpts), noOpAll) diff --git a/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go b/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go index e770c69e6d..29cb03c4c4 100644 --- a/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go +++ b/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go @@ -30,6 +30,7 @@ import ( "github.com/m3db/m3/src/dbnode/integration/generate" persistfs "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/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" bcl "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" @@ -226,7 +227,7 @@ func setCommitLogAndFilesystemBootstrapper(t *testing.T, opts testOptions, setup bclOpts := bcl.NewOptions(). SetResultOptions(bsOpts). SetCommitLogOptions(commitLogOpts). - SetRuntimeManager(runtime.runtime.NewOptionsManager()) + SetRuntimeOptionsManager(runtime.NewOptionsManager()) commitLogBootstrapper, err := bcl.NewCommitLogBootstrapperProvider( bclOpts, mustInspectFilesystem(fsOpts), noOpAll) From 9883df1d62cba7b592f4aca6dafd52d8f853001a Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:17:50 -0700 Subject: [PATCH 25/47] Fix broken test --- src/dbnode/persist/fs/commitlog/read_write_prop_test.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/read_write_prop_test.go b/src/dbnode/persist/fs/commitlog/read_write_prop_test.go index 5870995c1b..92ae6039c8 100644 --- a/src/dbnode/persist/fs/commitlog/read_write_prop_test.go +++ b/src/dbnode/persist/fs/commitlog/read_write_prop_test.go @@ -87,8 +87,9 @@ func TestCommitLogReadWrite(t *testing.T) { FileFilterPredicate: ReadAllPredicate(), SeriesFilterPredicate: ReadAllSeriesPredicate(), } - iter, err := NewIterator(iterOpts) + iter, corruptFiles, err := NewIterator(iterOpts) require.NoError(t, err) + require.True(t, len(corruptFiles) == 0) defer iter.Close() // Convert the writes to be in-order, but keyed by series ID because the @@ -381,7 +382,9 @@ func (s *clState) writesArePresent(writes ...generatedWrite) error { FileFilterPredicate: ReadAllPredicate(), SeriesFilterPredicate: ReadAllSeriesPredicate(), } - iter, err := NewIterator(iterOpts) + // Based on the corruption type this could return some corrupt files + // or it could not, so we don't check it. + iter, _, err := NewIterator(iterOpts) if err != nil { if s.shouldCorrupt { return nil From d4aacfebb47a55a9ff5eac15c83a2f26436da25d Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:29:37 -0700 Subject: [PATCH 26/47] Add metrics for cleanup commitlogs --- src/dbnode/storage/cleanup.go | 25 +++++++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index fde9b7c1ea..14bdcfc85c 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -53,7 +53,22 @@ type cleanupManager struct { deleteFilesFn deleteFilesFn deleteInactiveDirectoriesFn deleteInactiveDirectoriesFn cleanupInProgress bool - status tally.Gauge + metrics cleanupManagerMetrics +} + +type cleanupManagerMetrics struct { + status tally.Gauge + corruptCommitlogFile tally.Counter + deletedCommitlogFile tally.Counter +} + +func newCleanupManagerMetrics(scope tally.Scope) cleanupManagerMetrics { + clScope := scope.SubScope("commitlog") + return cleanupManagerMetrics{ + status: scope.Gauge("cleanup"), + corruptCommitlogFile: clScope.Counter("corrupt"), + deletedCommitlogFile: clScope.Counter("deleted"), + } } func newCleanupManager(database database, scope tally.Scope) databaseCleanupManager { @@ -70,7 +85,7 @@ func newCleanupManager(database database, scope tally.Scope) databaseCleanupMana commitLogFilesFn: commitlog.Files, deleteFilesFn: fs.DeleteFiles, deleteInactiveDirectoriesFn: fs.DeleteInactiveDirectories, - status: scope.Gauge("cleanup"), + metrics: newCleanupManagerMetrics(scope), } } @@ -128,6 +143,7 @@ func (m *cleanupManager) Cleanup(t time.Time) error { "encountered errors when cleaning up commit logs for commitLogFiles %v: %v", filesToCleanup, err)) } + m.metrics.deletedCommitlogFile.Inc(len(filesToCleanup)) return multiErr.FinalError() } @@ -138,9 +154,9 @@ func (m *cleanupManager) Report() { m.RUnlock() if cleanupInProgress { - m.status.Update(1) + m.metrics.status.Update(1) } else { - m.status.Update(0) + m.metrics.status.Update(0) } } @@ -341,6 +357,7 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAn f, err := fileOrErr.File() if err != nil { + m.metrics.corruptCommitlogFile.Inc(1) // If we were unable to read the commit log files info header, then we're forced to assume // that the file is corrupt and remove it. This can happen in situations where M3DB experiences // sudden shutdown. From c3de970c933c0098e5241d285333e135a2aa2e1a Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:29:56 -0700 Subject: [PATCH 27/47] Fix compulation error --- src/dbnode/storage/cleanup.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index 14bdcfc85c..5407c3d2b2 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -143,7 +143,7 @@ func (m *cleanupManager) Cleanup(t time.Time) error { "encountered errors when cleaning up commit logs for commitLogFiles %v: %v", filesToCleanup, err)) } - m.metrics.deletedCommitlogFile.Inc(len(filesToCleanup)) + m.metrics.deletedCommitlogFile.Inc(int64(len(filesToCleanup))) return multiErr.FinalError() } From c976906a4baa8953d2c2dda62e9a26071eb4f90b Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:44:08 -0700 Subject: [PATCH 28/47] Add metrics for commitlog bootstrapper --- .../bootstrapper/commitlog/source.go | 63 +++++++++++++++++-- 1 file changed, 59 insertions(+), 4 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index cef3837dfb..ec72a31237 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -47,6 +47,7 @@ import ( "github.com/m3db/m3x/pool" xsync "github.com/m3db/m3x/sync" xtime "github.com/m3db/m3x/time" + "github.com/uber-go/tally" ) var ( @@ -70,6 +71,38 @@ type commitLogSource struct { newIteratorFn newIteratorFn snapshotFilesFn snapshotFilesFn newReaderFn newReaderFn + + metrics commitLogSourceDataAndIndexMetrics +} + +type commitLogSourceDataAndIndexMetrics struct { + data commitLogSourceMetrics + index commitLogSourceMetrics +} + +func newCommitLogSourceDataAndIndexMetrics(scope tally.Scope) commitLogSourceDataAndIndexMetrics { + return commitLogSourceDataAndIndexMetrics{ + data: newCommitLogSourceMetrics(scope), + index: newCommitLogSourceMetrics(scope), + } +} + +type commitLogSourceMetrics struct { + corruptCommitlogFile tally.Counter + readingSnapshots tally.Gauge + readingCommitlogs tally.Gauge + mergingSnapshotsAndCommitlogs tally.Gauge +} + +func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics { + statusScope := scope.SubScope("status") + clScope := scope.SubScope("commitlog") + return commitLogSourceMetrics{ + corruptCommitlogFile: clScope.Counter("corrupt"), + readingSnapshots: statusScope.Gauge("reading-snapshots"), + readingCommitlogs: statusScope.Gauge("reading-commitlogs"), + mergingSnapshotsAndCommitlogs: statusScope.Gauge("merging-snapshots-and-commitlogs"), + } } type encoder struct { @@ -78,6 +111,12 @@ type encoder struct { } func newCommitLogSource(opts Options, inspection fs.Inspection) bootstrap.Source { + scope := opts. + ResultOptions(). + InstrumentOptions(). + MetricsScope(). + SubScope("bootstrapper-commitlog") + return &commitLogSource{ opts: opts, log: opts. @@ -91,6 +130,8 @@ func newCommitLogSource(opts Options, inspection fs.Inspection) bootstrap.Source newIteratorFn: commitlog.NewIterator, snapshotFilesFn: fs.SnapshotFiles, newReaderFn: fs.NewReader, + + metrics: newCommitLogSourceDataAndIndexMetrics(scope), } } @@ -248,7 +289,7 @@ func (s *commitLogSource) ReadData( return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } if len(corruptFiles) > 0 { - s.logCorruptFiles(corruptFiles) + s.logAndEmitCorruptFiles(corruptFiles, true) encounteredCorruptData = true } @@ -281,6 +322,7 @@ func (s *commitLogSource) ReadData( } // Read / M3TSZ encode all the datapoints in the commit log that we need to read. + s.metrics.data.readingCommitlogs.Update(1) for iter.Next() { series, dp, unit, annotation := iter.Current() if !s.shouldEncodeForData(shardDataByShard, blockSize, series, dp.Timestamp) { @@ -306,6 +348,7 @@ func (s *commitLogSource) ReadData( blockStart: dp.Timestamp.Truncate(blockSize), } } + s.metrics.data.readingCommitlogs.Update(0) if iterErr := iter.Err(); iterErr != nil { // Log the error and mark that we encountered corrupt data, but don't @@ -314,6 +357,7 @@ func (s *commitLogSource) ReadData( // altogether. s.log.Errorf( "error in commitlog iterator: %v", iterErr) + s.metrics.data.corruptCommitlogFile.Inc(1) encounteredCorruptData = true } @@ -330,6 +374,7 @@ func (s *commitLogSource) ReadData( // the data that is available in the snapshot files. mergeStart := time.Now() s.log.Infof("starting merge...") + s.metrics.data.mergingSnapshotsAndCommitlogs.Update(1) bootstrapResult, err := s.mergeAllShardsCommitLogEncodersAndSnapshots( ns, shardsTimeRanges, @@ -339,8 +384,8 @@ func (s *commitLogSource) ReadData( blockSize, shardDataByShard, ) + s.metrics.data.mergingSnapshotsAndCommitlogs.Update(0) if err != nil { - // TODO: Should probably not return this error? return nil, err } s.log.Infof("done merging..., took: %s", time.Since(mergeStart).String()) @@ -1343,11 +1388,13 @@ func (s *commitLogSource) ReadIndex( ) // Start by reading any available snapshot files. + s.metrics.index.readingSnapshots.Update(1) for shard, tr := range shardsTimeRanges { shardResult, err := s.bootstrapShardSnapshots( ns.ID(), shard, true, tr, blockSize, snapshotFilesByShard[shard], mostRecentCompleteSnapshotByBlockShard) if err != nil { + s.metrics.index.readingSnapshots.Update(0) return nil, err } @@ -1362,6 +1409,7 @@ func (s *commitLogSource) ReadIndex( } } } + s.metrics.index.readingSnapshots.Update(0) // Next, read all of the data from the commit log files that wasn't covered // by the snapshot files. @@ -1370,7 +1418,7 @@ func (s *commitLogSource) ReadIndex( return nil, fmt.Errorf("unable to create commit log iterator: %v", err) } if len(corruptFiles) > 0 { - s.logCorruptFiles(corruptFiles) + s.logAndEmitCorruptFiles(corruptFiles, false) encounteredCorruptData = true } @@ -1392,6 +1440,7 @@ func (s *commitLogSource) ReadIndex( s.log.Errorf( "error in commitlog iterator: %v", iterErr) encounteredCorruptData = true + s.metrics.index.corruptCommitlogFile.Inc(1) } // If all successful then we mark each index block as fulfilled @@ -1501,12 +1550,18 @@ func (s commitLogSource) maybeAddToIndex( return err } -func (s *commitLogSource) logCorruptFiles(corruptFiles []commitlog.ErrorWithPath) { +func (s *commitLogSource) logAndEmitCorruptFiles( + corruptFiles []commitlog.ErrorWithPath, isData bool) { for _, f := range corruptFiles { s.log. Errorf( "opting to skip commit log: %s due to corruption, err: %v", f.Path, f.Error) + if isData { + s.metrics.data.corruptCommitlogFile.Inc(1) + } else { + s.metrics.index.corruptCommitlogFile.Inc(1) + } } } From ce9beca2539ad261703f5602322e38e76f49a245 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:52:11 -0700 Subject: [PATCH 29/47] Fix imports --- src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go | 1 + 1 file changed, 1 insertion(+) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index ec72a31237..4012ab8d96 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -47,6 +47,7 @@ import ( "github.com/m3db/m3x/pool" xsync "github.com/m3db/m3x/sync" xtime "github.com/m3db/m3x/time" + "github.com/uber-go/tally" ) From 02ce8e4dccd8783da9c8a784d3f9af5ab32c7ee6 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 13:53:18 -0700 Subject: [PATCH 30/47] More tweaks --- src/dbnode/integration/commitlog_bootstrap_merge_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/dbnode/integration/commitlog_bootstrap_merge_test.go b/src/dbnode/integration/commitlog_bootstrap_merge_test.go index a3049c7c46..c410e80928 100644 --- a/src/dbnode/integration/commitlog_bootstrap_merge_test.go +++ b/src/dbnode/integration/commitlog_bootstrap_merge_test.go @@ -29,6 +29,7 @@ import ( "github.com/m3db/m3/src/dbnode/integration/generate" persistfs "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/bootstrap" "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" bcl "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper/commitlog" @@ -124,7 +125,8 @@ func TestCommitLogAndFSMergeBootstrap(t *testing.T) { bsOpts := newDefaulTestResultOptions(setup.storageOpts) bclOpts := bcl.NewOptions(). SetResultOptions(bsOpts). - SetCommitLogOptions(commitLogOpts) + SetCommitLogOptions(commitLogOpts). + SetRuntimeOptionsManager(runtime.NewOptionsManager()) fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() commitLogBootstrapper, err := bcl.NewCommitLogBootstrapperProvider( From 07284d589452d42746c40f31d1d38e2465dcb1ec Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 14:12:21 -0700 Subject: [PATCH 31/47] Fix broken test --- src/dbnode/storage/bootstrap/bootstrapper/peers/peers_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/peers/peers_test.go b/src/dbnode/storage/bootstrap/bootstrapper/peers/peers_test.go index 8da2deefe6..6b198a3edd 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/peers/peers_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/peers/peers_test.go @@ -25,6 +25,7 @@ import ( "github.com/m3db/m3/src/dbnode/client" "github.com/m3db/m3/src/dbnode/persist" + "github.com/m3db/m3/src/dbnode/runtime" "github.com/golang/mock/gomock" "github.com/stretchr/testify/assert" @@ -41,7 +42,8 @@ func TestNewPeersBootstrapper(t *testing.T) { opts := NewOptions(). SetAdminClient(client.NewMockAdminClient(ctrl)). - SetPersistManager(persist.NewMockManager(ctrl)) + SetPersistManager(persist.NewMockManager(ctrl)). + SetRuntimeOptionsManager(runtime.NewMockOptionsManager(ctrl)) b, err := NewPeersBootstrapperProvider(opts, nil) assert.NoError(t, err) From 979ecf7abf23eae98292441be72754054a095deb Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 14:14:23 -0700 Subject: [PATCH 32/47] Fix broken test --- src/cmd/services/m3dbnode/config/config_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/src/cmd/services/m3dbnode/config/config_test.go b/src/cmd/services/m3dbnode/config/config_test.go index c27fa02569..81fead9293 100644 --- a/src/cmd/services/m3dbnode/config/config_test.go +++ b/src/cmd/services/m3dbnode/config/config_test.go @@ -368,6 +368,7 @@ db: fs: numProcessorsPerCPU: 0.125 peers: null + commitlog: null cacheSeriesMetadata: null blockRetrieve: null cache: From d5a159b1617b168ba79aa260f80ea225f875000e Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Fri, 12 Oct 2018 15:36:33 -0700 Subject: [PATCH 33/47] Disable deletion of corrupt commit log files temporarily --- src/dbnode/storage/cleanup.go | 7 +++++-- src/dbnode/storage/cleanup_test.go | 4 ++++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index 5407c3d2b2..0c9a411236 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -373,8 +373,11 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAn "encountered err: %v reading commit log file: %v info during cleanup, marking file for deletion", errorWithPath.Error(), errorWithPath.Path()) - filesToCleanup = append(filesToCleanup, newCommitLogFileWithErrorAndPath( - f, errorWithPath.Path(), err)) + // TODO(rartoul): Leave this out until we have a way of distinguishing between a corrupt commit + // log file and the commit log file that is actively being written to (which may still be missing + // the header): https://github.com/m3db/m3/issues/1078 + // filesToCleanup = append(filesToCleanup, newCommitLogFileWithErrorAndPath( + // f, errorWithPath.Path(), err)) continue } diff --git a/src/dbnode/storage/cleanup_test.go b/src/dbnode/storage/cleanup_test.go index 78fb606894..86a343ea92 100644 --- a/src/dbnode/storage/cleanup_test.go +++ b/src/dbnode/storage/cleanup_test.go @@ -752,6 +752,10 @@ func TestCleanupManagerCommitLogTimesMultiNS(t *testing.T) { } func TestCleanupManagerDeletesCorruptCommitLogFiles(t *testing.T) { + // TODO(rartoul): Re-enable this once https://github.com/m3db/m3/issues/1078 + // is resolved. + t.Skip() + ctrl := gomock.NewController(t) defer ctrl.Finish() From 18e405edbd8ef520d745ccc113215ee9d88414c2 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 11:23:25 -0400 Subject: [PATCH 34/47] Move stuff to bottom of file --- src/dbnode/persist/fs/commitlog/files.go | 64 ++++++++++++------------ 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index 8ae1890416..6451e00df6 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -30,31 +30,6 @@ import ( "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" ) -type fsError struct { - err error -} - -func (e fsError) Error() string { - return e.err.Error() -} - -// ErrorWithPath is an error that includes the path of the file that -// had the error. -type ErrorWithPath struct { - err error - path string -} - -// Error returns the error. -func (e ErrorWithPath) Error() string { - return e.err.Error() -} - -// Path returns the path of hte file that the error is associated with. -func (e ErrorWithPath) Path() string { - return e.path -} - // FileOrError is a union/option type that returns an error if there was // any issue reading the commitlog info, or a File if there was not. Its // purpose is to force callers to handle the error. @@ -93,13 +68,6 @@ type File struct { Index int64 } -func newErrorWithPath(err error, path string) ErrorWithPath { - return ErrorWithPath{ - err: err, - path: path, - } -} - // ReadLogInfo reads the commit log info out of a commitlog file func ReadLogInfo(filePath string, opts Options) (time.Time, time.Duration, int64, error) { var fd *os.File @@ -178,3 +146,35 @@ func Files(opts Options) ([]FileOrError, error) { return commitLogFiles, nil } + +// ErrorWithPath is an error that includes the path of the file that +// had the error. +type ErrorWithPath struct { + err error + path string +} + +// Error returns the error. +func (e ErrorWithPath) Error() string { + return e.err.Error() +} + +// Path returns the path of hte file that the error is associated with. +func (e ErrorWithPath) Path() string { + return e.path +} + +func newErrorWithPath(err error, path string) ErrorWithPath { + return ErrorWithPath{ + err: err, + path: path, + } +} + +type fsError struct { + err error +} + +func (e fsError) Error() string { + return e.err.Error() +} From 7a94eaf25a7c02b92fe7a14c2fbdaf5d99a4e851 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 11:39:45 -0400 Subject: [PATCH 35/47] Refactor files interface --- src/dbnode/persist/fs/commitlog/files.go | 57 +++++++------------ src/dbnode/persist/fs/commitlog/files_test.go | 8 +-- src/dbnode/persist/fs/commitlog/iterator.go | 26 ++------- 3 files changed, 28 insertions(+), 63 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index 6451e00df6..6212095d8f 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -30,36 +30,6 @@ import ( "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" ) -// FileOrError is a union/option type that returns an error if there was -// any issue reading the commitlog info, or a File if there was not. Its -// purpose is to force callers to handle the error. -type FileOrError struct { - f File - // Contains any errors encountered when trying to read the commitlogs file info. We - // attempt to not include filesystem errors in this field, but that is accomplished - // on a best-effort basis and it is possible for this field to contain an error that - // is the result of a filesystem / O.S / hardware issue as opposed to an actually - // corrupt file. - e error -} - -// File returns a File if the commitlog info was read, or an error otherwise. -func (f FileOrError) File() (File, error) { - return f.f, f.e -} - -// NewFileOrError creates a new FileOrError. -func NewFileOrError(f File, e error, path string) FileOrError { - if e != nil { - e = newErrorWithPath(e, path) - } - - return FileOrError{ - f: f, - e: e, - } -} - // File represents a commit log file and its associated metadata. type File struct { FilePath string @@ -110,15 +80,16 @@ func ReadLogInfo(filePath string, opts Options) (time.Time, time.Duration, int64 // Files returns a slice of all available commit log files on disk along with // their associated metadata. -func Files(opts Options) ([]FileOrError, error) { +func Files(opts Options) ([]File, []ErrorWithPath, error) { commitLogsDir := fs.CommitLogsDirPath( opts.FilesystemOptions().FilePathPrefix()) filePaths, err := fs.SortedCommitLogFiles(commitLogsDir) if err != nil { - return nil, err + return nil, nil, err } - commitLogFiles := make([]FileOrError, 0, len(filePaths)) + commitLogFiles := make([]File, 0, len(filePaths)) + errorsWithPath := make([]ErrorWithPath, 0) for _, filePath := range filePaths { file := File{ FilePath: filePath, @@ -126,7 +97,13 @@ func Files(opts Options) ([]FileOrError, error) { start, duration, index, err := ReadLogInfo(filePath, opts) if _, ok := err.(fsError); ok { - return nil, err + return nil, nil, err + } + + if err != nil { + errorsWithPath = append(errorsWithPath, newErrorWithPath( + err, filePath)) + continue } if err == nil { @@ -135,16 +112,20 @@ func Files(opts Options) ([]FileOrError, error) { file.Index = index } - commitLogFiles = append(commitLogFiles, NewFileOrError( - file, err, filePath)) + commitLogFiles = append(commitLogFiles, File{ + FilePath: filePath, + Start: start, + Duration: duration, + Index: index, + }) } sort.Slice(commitLogFiles, func(i, j int) bool { // Sorting is best effort here since we may not know the start. - return commitLogFiles[i].f.Start.Before(commitLogFiles[j].f.Start) + return commitLogFiles[i].Start.Before(commitLogFiles[j].Start) }) - return commitLogFiles, nil + return commitLogFiles, errorsWithPath, nil } // ErrorWithPath is an error that includes the path of the file that diff --git a/src/dbnode/persist/fs/commitlog/files_test.go b/src/dbnode/persist/fs/commitlog/files_test.go index 3eba481a6c..599fe5772d 100644 --- a/src/dbnode/persist/fs/commitlog/files_test.go +++ b/src/dbnode/persist/fs/commitlog/files_test.go @@ -52,16 +52,14 @@ func TestFiles(t *testing.T) { opts.FilesystemOptions(). SetFilePathPrefix(dir), ) - files, err := Files(opts) + files, corruptFiles, err := Files(opts) require.NoError(t, err) + require.True(t, len(corruptFiles) == 0) require.True(t, len(files) >= minNumBlocks) // Make sure its sorted var lastFileStart time.Time - for _, fileOrError := range files { - file, err := fileOrError.File() - require.NoError(t, err) - + for _, file := range files { require.Equal(t, 10*time.Minute, file.Duration) require.Equal(t, int64(0), file.Index) require.True(t, strings.Contains(file.FilePath, dir)) diff --git a/src/dbnode/persist/fs/commitlog/iterator.go b/src/dbnode/persist/fs/commitlog/iterator.go index e2b533e54f..606dcfa01d 100644 --- a/src/dbnode/persist/fs/commitlog/iterator.go +++ b/src/dbnode/persist/fs/commitlog/iterator.go @@ -74,11 +74,11 @@ func NewIterator(iterOpts IteratorOpts) (iter Iterator, corruptFiles []ErrorWith iops := opts.InstrumentOptions() iops = iops.SetMetricsScope(iops.MetricsScope().SubScope("iterator")) - files, err := Files(opts) + files, corruptFiles, err := Files(opts) if err != nil { return nil, nil, err } - filteredFiles, corruptFiles := filterFiles(opts, files, iterOpts.FileFilterPredicate) + filteredFiles := filterFiles(opts, files, iterOpts.FileFilterPredicate) scope := iops.MetricsScope() return &iterator{ @@ -190,28 +190,14 @@ func (i *iterator) nextReader() bool { return true } -func filterFiles(opts Options, files []FileOrError, predicate FileFilterPredicate) ([]File, []ErrorWithPath) { - var ( - filtered = make([]File, 0, len(files)) - corrupt = make([]ErrorWithPath, 0, len(files)) - ) - - for _, file := range files { - f, err := file.File() - if err != nil { - errorWithpath, ok := err.(ErrorWithPath) - if ok { - corrupt = append(corrupt, errorWithpath) - } - - continue - } - +func filterFiles(opts Options, files []File, predicate FileFilterPredicate) []File { + filtered := make([]File, 0, len(files)) + for _, f := range files { if predicate(f) { filtered = append(filtered, f) } } - return filtered, corrupt + return filtered } func (i *iterator) closeAndResetReader() error { From d5e10e681de1b4c4f212b586fd247acba473d990 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 11:53:10 -0400 Subject: [PATCH 36/47] Refactor code to use new interface --- .../commitlog/source_prop_test.go | 17 ++- src/dbnode/storage/cleanup.go | 48 +++--- src/dbnode/storage/cleanup_prop_test.go | 15 +- src/dbnode/storage/cleanup_test.go | 138 ++++++------------ 4 files changed, 83 insertions(+), 135 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 2da7dec518..161b24ecdd 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -1,5 +1,3 @@ -// +build big -// // Copyright (c) 2017 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy @@ -71,7 +69,7 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { namespace.NewOptions().IndexOptions().SetEnabled(true), ) ) - parameters.MinSuccessfulTests = 80 + parameters.MinSuccessfulTests = 40 parameters.Rng.Seed(seed) nsMeta, err := namespace.NewMetadata(testNamespaceID, nsOpts) require.NoError(t, err) @@ -287,14 +285,16 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { if input.includeCorruptedCommitlogFile { // Write out an additional commit log file with a corrupt info header to // make sure that the commitlog source skips it in the single node scenario. - commitLogFiles, err := commitlog.Files(commitLogOpts) + commitLogFiles, corruptFiles, err := commitlog.Files(commitLogOpts) if err != nil { return false, err } + if len(corruptFiles) > 0 { + return false, fmt.Errorf("found corrupt commit log files: %v", corruptFiles) + } if len(commitLogFiles) > 0 { - lastCommitLogFileOrErr := commitLogFiles[len(commitLogFiles)-1] - lastCommitLogFile, err := lastCommitLogFileOrErr.File() + lastCommitLogFile := commitLogFiles[len(commitLogFiles)-1] if err != nil { return false, err } @@ -380,10 +380,13 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { values = append(values, testValue{write.series, write.datapoint.Timestamp, write.datapoint.Value, write.unit, write.annotation}) } - commitLogFiles, err := commitlog.Files(commitLogOpts) + commitLogFiles, corruptFiles, err := commitlog.Files(commitLogOpts) if err != nil { return false, err } + if len(corruptFiles) > 0 && !input.includeCorruptedCommitlogFile { + return false, fmt.Errorf("found corrupt commit log files: %v", corruptFiles) + } commitLogFilesExist := len(commitLogFiles) > 0 // In the multi-node setup we want to return unfulfilled if there are any corrupt files, but diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index 0c9a411236..9fdd90124b 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -35,7 +35,7 @@ import ( "github.com/uber-go/tally" ) -type commitLogFilesFn func(commitlog.Options) ([]commitlog.FileOrError, error) +type commitLogFilesFn func(commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) type deleteFilesFn func(files []string) error @@ -296,7 +296,7 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAn // are only retained for a period of 1-2 days (at most), after we which we'd live we with the // data loss. - files, err := m.commitLogFilesFn(m.opts.CommitLogOptions()) + files, corruptFiles, err := m.commitLogFilesFn(m.opts.CommitLogOptions()) if err != nil { return nil, err } @@ -353,34 +353,7 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAn } filesToCleanup := make([]commitLogFileWithErrorAndPath, 0, len(files)) - for _, fileOrErr := range files { - f, err := fileOrErr.File() - - if err != nil { - m.metrics.corruptCommitlogFile.Inc(1) - // If we were unable to read the commit log files info header, then we're forced to assume - // that the file is corrupt and remove it. This can happen in situations where M3DB experiences - // sudden shutdown. - errorWithPath, ok := err.(commitlog.ErrorWithPath) - if !ok { - m.opts.InstrumentOptions().Logger().Errorf( - "commitlog file error did not contain path: %v", err) - // Continue because we want to try and clean up the remining files instead of erroring out. - continue - } - - m.opts.InstrumentOptions().Logger().Errorf( - "encountered err: %v reading commit log file: %v info during cleanup, marking file for deletion", - errorWithPath.Error(), errorWithPath.Path()) - - // TODO(rartoul): Leave this out until we have a way of distinguishing between a corrupt commit - // log file and the commit log file that is actively being written to (which may still be missing - // the header): https://github.com/m3db/m3/issues/1078 - // filesToCleanup = append(filesToCleanup, newCommitLogFileWithErrorAndPath( - // f, errorWithPath.Path(), err)) - continue - } - + for _, f := range files { shouldDelete, err := shouldCleanupFile(f) if err != nil { return nil, err @@ -392,6 +365,21 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAn } } + for _, errorWithPath := range corruptFiles { + m.metrics.corruptCommitlogFile.Inc(1) + // If we were unable to read the commit log files info header, then we're forced to assume + // that the file is corrupt and remove it. This can happen in situations where M3DB experiences + // sudden shutdown. + m.opts.InstrumentOptions().Logger().Errorf( + "encountered err: %v reading commit log file: %v info during cleanup, marking file for deletion", + errorWithPath.Error(), errorWithPath.Path()) + // TODO(rartoul): Leave this out until we have a way of distinguishing between a corrupt commit + // log file and the commit log file that is actively being written to (which may still be missing + // the header): https://github.com/m3db/m3/issues/1078 + // filesToCleanup = append(filesToCleanup, newCommitLogFileWithErrorAndPath( + // f, errorWithPath.Path(), err)) + } + return filesToCleanup, nil } diff --git a/src/dbnode/storage/cleanup_prop_test.go b/src/dbnode/storage/cleanup_prop_test.go index 10927111b5..f30a91214b 100644 --- a/src/dbnode/storage/cleanup_prop_test.go +++ b/src/dbnode/storage/cleanup_prop_test.go @@ -69,16 +69,15 @@ func newPropTestCleanupMgr( n = numIntervals(oldest, newest, blockSize) currStart = oldest ) - cm.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - files := make([]commitlog.FileOrError, 0, n) + cm.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + files := make([]commitlog.File, 0, n) for i := 0; i < n; i++ { - files = append(files, commitlog.NewFileOrError( - commitlog.File{ - Start: currStart, - Duration: blockSize, - }, nil, "path")) + files = append(files, commitlog.File{ + Start: currStart, + Duration: blockSize, + }) } - return files, nil + return files, nil, nil } return cm diff --git a/src/dbnode/storage/cleanup_test.go b/src/dbnode/storage/cleanup_test.go index 86a343ea92..39455db7ad 100644 --- a/src/dbnode/storage/cleanup_test.go +++ b/src/dbnode/storage/cleanup_test.go @@ -73,12 +73,10 @@ func TestCleanupManagerCleanup(t *testing.T) { mgr.opts.CommitLogOptions(). SetBlockSize(rOpts.BlockSize())) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{FilePath: "foo", Start: timeFor(14400)}, - nil, "foo"), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + {FilePath: "foo", Start: timeFor(14400)}, + }, nil, nil } var deletedFiles []string mgr.deleteFilesFn = func(files []string) error { @@ -480,18 +478,12 @@ func TestCleanupManagerCommitLogTimesAllFlushed(t *testing.T) { defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time30, Duration: commitLogBlockSize}, - nil, ""), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + }, nil, nil } gomock.InOrder( @@ -513,18 +505,12 @@ func TestCleanupManagerCommitLogTimesMiddlePendingFlush(t *testing.T) { defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time30, Duration: commitLogBlockSize}, - nil, ""), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + }, nil, nil } ns.EXPECT().IsCapturedBySnapshot( @@ -547,18 +533,12 @@ func TestCleanupManagerCommitLogTimesStartPendingFlush(t *testing.T) { defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time30, Duration: commitLogBlockSize}, - nil, ""), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + }, nil, nil } ns.EXPECT().IsCapturedBySnapshot( @@ -582,18 +562,12 @@ func TestCleanupManagerCommitLogTimesAllPendingFlush(t *testing.T) { defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time30, Duration: commitLogBlockSize}, - nil, ""), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + }, nil, nil } ns.EXPECT().IsCapturedBySnapshot( @@ -631,18 +605,12 @@ func TestCleanupManagerCommitLogTimesAllPendingFlushButHaveSnapshot(t *testing.T currentTime = timeFor(50) commitLogBlockSize = 10 * time.Second ) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time30, Duration: commitLogBlockSize}, - nil, ""), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + }, nil, nil } gomock.InOrder( @@ -678,12 +646,10 @@ func TestCleanupManagerCommitLogTimesHandlesIsCapturedBySnapshotError(t *testing defer ctrl.Finish() ns, mgr := newCleanupManagerCommitLogTimesTest(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{Start: time30, Duration: commitLogBlockSize}, - nil, ""), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + }, nil, nil } gomock.InOrder( @@ -701,18 +667,12 @@ func TestCleanupManagerCommitLogTimesMultiNS(t *testing.T) { defer ctrl.Finish() ns1, ns2, mgr := newCleanupManagerCommitLogTimesTestMultiNS(t, ctrl) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time20, Duration: commitLogBlockSize}, - nil, ""), - commitlog.NewFileOrError( - commitlog.File{Start: time30, Duration: commitLogBlockSize}, - nil, ""), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + commitlog.File{Start: time20, Duration: commitLogBlockSize}, + commitlog.File{Start: time30, Duration: commitLogBlockSize}, + }, nil, nil } // ns1 is flushed for time10->time20 and time20->time30. @@ -763,12 +723,10 @@ func TestCleanupManagerDeletesCorruptCommitLogFiles(t *testing.T) { _, mgr = newCleanupManagerCommitLogTimesTest(t, ctrl) err = errors.New("some_error") ) - mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.FileOrError, error) { - return []commitlog.FileOrError{ - commitlog.NewFileOrError( - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - err, ""), - }, nil + mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { + return []commitlog.File{ + commitlog.File{Start: time10, Duration: commitLogBlockSize}, + }, nil, nil } filesToCleanup, err := mgr.commitLogTimes(currentTime) From 48b671a8b065ea9bf949cfcfdf226f58b8dc24df Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 12:01:15 -0400 Subject: [PATCH 37/47] Fix tests --- src/dbnode/persist/fs/commitlog/files.go | 4 ++-- src/dbnode/storage/cleanup.go | 2 +- src/dbnode/storage/cleanup_test.go | 20 +++++++++++++++----- 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index 6212095d8f..23e66ca678 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -101,7 +101,7 @@ func Files(opts Options) ([]File, []ErrorWithPath, error) { } if err != nil { - errorsWithPath = append(errorsWithPath, newErrorWithPath( + errorsWithPath = append(errorsWithPath, NewErrorWithPath( err, filePath)) continue } @@ -145,7 +145,7 @@ func (e ErrorWithPath) Path() string { return e.path } -func newErrorWithPath(err error, path string) ErrorWithPath { +func NewErrorWithPath(err error, path string) ErrorWithPath { return ErrorWithPath{ err: err, path: path, diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index 9fdd90124b..6f5917ffd0 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -377,7 +377,7 @@ func (m *cleanupManager) commitLogTimes(t time.Time) ([]commitLogFileWithErrorAn // log file and the commit log file that is actively being written to (which may still be missing // the header): https://github.com/m3db/m3/issues/1078 // filesToCleanup = append(filesToCleanup, newCommitLogFileWithErrorAndPath( - // f, errorWithPath.Path(), err)) + // commitlog.File{}, errorWithPath.Path(), err)) } return filesToCleanup, nil diff --git a/src/dbnode/storage/cleanup_test.go b/src/dbnode/storage/cleanup_test.go index 39455db7ad..166f45f7b1 100644 --- a/src/dbnode/storage/cleanup_test.go +++ b/src/dbnode/storage/cleanup_test.go @@ -596,6 +596,16 @@ func contains(arr []commitLogFileWithErrorAndPath, t time.Time) bool { return false } +func containsCorrupt(arr []commitLogFileWithErrorAndPath, path string) bool { + for _, f := range arr { + if f.path == path { + return true + } + } + + return false +} + func TestCleanupManagerCommitLogTimesAllPendingFlushButHaveSnapshot(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -722,15 +732,15 @@ func TestCleanupManagerDeletesCorruptCommitLogFiles(t *testing.T) { var ( _, mgr = newCleanupManagerCommitLogTimesTest(t, ctrl) err = errors.New("some_error") + path = "path" ) mgr.commitLogFilesFn = func(_ commitlog.Options) ([]commitlog.File, []commitlog.ErrorWithPath, error) { - return []commitlog.File{ - commitlog.File{Start: time10, Duration: commitLogBlockSize}, - }, nil, nil + return []commitlog.File{}, []commitlog.ErrorWithPath{ + commitlog.NewErrorWithPath(err, path), + }, nil } filesToCleanup, err := mgr.commitLogTimes(currentTime) require.NoError(t, err) - - require.True(t, contains(filesToCleanup, time10)) + require.True(t, containsCorrupt(filesToCleanup, path)) } From c300a2009436ff688f49fa930a7951e593e23fd8 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 12:01:34 -0400 Subject: [PATCH 38/47] Add function comment --- src/dbnode/persist/fs/commitlog/files.go | 1 + 1 file changed, 1 insertion(+) diff --git a/src/dbnode/persist/fs/commitlog/files.go b/src/dbnode/persist/fs/commitlog/files.go index 23e66ca678..820fc872ad 100644 --- a/src/dbnode/persist/fs/commitlog/files.go +++ b/src/dbnode/persist/fs/commitlog/files.go @@ -145,6 +145,7 @@ func (e ErrorWithPath) Path() string { return e.path } +// NewErrorWithPath creates a new ErrorWithPath. func NewErrorWithPath(err error, path string) ErrorWithPath { return ErrorWithPath{ err: err, From a3801873da751f9ea4e72bd3157ab2767d72f4b1 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 12:17:09 -0400 Subject: [PATCH 39/47] emit gauges in loop --- .../bootstrapper/commitlog/source.go | 50 ++++++++++++++++--- 1 file changed, 42 insertions(+), 8 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 4012ab8d96..9369055926 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -95,6 +95,36 @@ type commitLogSourceMetrics struct { mergingSnapshotsAndCommitlogs tally.Gauge } +func (m commitLogSourceMetrics) emitReadingSnapshots() func() { + return m.gaugeLoop(m.readingSnapshots) +} + +func (m commitLogSourceMetrics) emitReadingCommitlogs() func() { + return m.gaugeLoop(m.readingCommitlogs) +} + +func (m commitLogSourceMetrics) emitMergingSnapshotsAndCommitlogs() func() { + return m.gaugeLoop(m.mergingSnapshotsAndCommitlogs) +} + +func (m commitLogSourceMetrics) gaugeLoop(g tally.Gauge) func() { + doneCh := make(chan struct{}) + go func() { + for { + select { + case <-doneCh: + g.Update(0) + return + default: + g.Update(1) + time.Sleep(time.Second) + } + } + }() + + return func() { close(doneCh) } +} + func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics { statusScope := scope.SubScope("status") clScope := scope.SubScope("commitlog") @@ -323,7 +353,7 @@ func (s *commitLogSource) ReadData( } // Read / M3TSZ encode all the datapoints in the commit log that we need to read. - s.metrics.data.readingCommitlogs.Update(1) + doneReadingCommitlogs := s.metrics.data.emitReadingCommitlogs() for iter.Next() { series, dp, unit, annotation := iter.Current() if !s.shouldEncodeForData(shardDataByShard, blockSize, series, dp.Timestamp) { @@ -349,7 +379,7 @@ func (s *commitLogSource) ReadData( blockStart: dp.Timestamp.Truncate(blockSize), } } - s.metrics.data.readingCommitlogs.Update(0) + doneReadingCommitlogs() if iterErr := iter.Err(); iterErr != nil { // Log the error and mark that we encountered corrupt data, but don't @@ -373,9 +403,11 @@ func (s *commitLogSource) ReadData( // Merge all the different encoders from the commit log that we created with // the data that is available in the snapshot files. - mergeStart := time.Now() s.log.Infof("starting merge...") - s.metrics.data.mergingSnapshotsAndCommitlogs.Update(1) + var ( + mergeStart = time.Now() + doneMerging = s.metrics.data.emitMergingSnapshotsAndCommitlogs() + ) bootstrapResult, err := s.mergeAllShardsCommitLogEncodersAndSnapshots( ns, shardsTimeRanges, @@ -385,7 +417,7 @@ func (s *commitLogSource) ReadData( blockSize, shardDataByShard, ) - s.metrics.data.mergingSnapshotsAndCommitlogs.Update(0) + doneMerging() if err != nil { return nil, err } @@ -1389,13 +1421,13 @@ func (s *commitLogSource) ReadIndex( ) // Start by reading any available snapshot files. - s.metrics.index.readingSnapshots.Update(1) + doneReadingSnapshots := s.metrics.index.emitReadingSnapshots() for shard, tr := range shardsTimeRanges { shardResult, err := s.bootstrapShardSnapshots( ns.ID(), shard, true, tr, blockSize, snapshotFilesByShard[shard], mostRecentCompleteSnapshotByBlockShard) if err != nil { - s.metrics.index.readingSnapshots.Update(0) + doneReadingSnapshots() return nil, err } @@ -1410,7 +1442,7 @@ func (s *commitLogSource) ReadIndex( } } } - s.metrics.index.readingSnapshots.Update(0) + doneReadingSnapshots() // Next, read all of the data from the commit log files that wasn't covered // by the snapshot files. @@ -1425,6 +1457,7 @@ func (s *commitLogSource) ReadIndex( defer iter.Close() + doneReadingCommitlogs := s.metrics.index.emitReadingCommitlogs() for iter.Next() { series, dp, _, _ := iter.Current() @@ -1432,6 +1465,7 @@ func (s *commitLogSource) ReadIndex( series.ID, series.Tags, series.Shard, highestShard, dp.Timestamp, bootstrapRangesByShard, indexResults, indexOptions, indexBlockSize, resultOptions) } + doneReadingCommitlogs() if iterErr := iter.Err(); iterErr != nil { // Log the error and mark that we encountered corrupt data, but don't From d490cde255da9decb2cf72e566bf598cb6e3977c Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 13:59:39 -0400 Subject: [PATCH 40/47] simplify couldObtainDataFromPeers logic --- .../bootstrapper/commitlog/source.go | 55 ++----------------- 1 file changed, 4 insertions(+), 51 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 9369055926..a41592ce34 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -1669,60 +1669,13 @@ func (s *commitLogSource) couldObtainDataFromPeers( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, runOpts bootstrap.RunOptions, -) (bool, error) { +) bool { var ( - initialTopologyState = runOpts.InitialTopologyState() - majorityReplicas = initialTopologyState.MajorityReplicas - runtimeOpts = s.opts.RuntimeOptionsManager().Get() - bootstrapConsistencyLevel = runtimeOpts.ClientBootstrapConsistencyLevel() + initialTopologyState = runOpts.InitialTopologyState() + majorityReplicas = initialTopologyState.MajorityReplicas ) - for shardIDUint := range shardsTimeRanges { - shardID := topology.ShardID(shardIDUint) - hostShardStates, ok := initialTopologyState.ShardStates[shardID] - if !ok { - // This shard was not part of the topology when the bootstrapping process began. - continue - } - - var ( - numPeers = len(hostShardStates) - numAvailablePeers = 0 - ) - for _, hostShardState := range hostShardStates { - if hostShardState.Host.ID() == initialTopologyState.Origin.ID() { - // Don't take self into account - continue - } - - shardState := hostShardState.ShardState - switch shardState { - // Don't want to peer bootstrap from a node that has not yet completely - // taken ownership of the shard. - case shard.Initializing: - // Success cases - We can bootstrap from this host, which is enough to - // mark this shard as bootstrappable. - case shard.Leaving: - fallthrough - case shard.Available: - numAvailablePeers++ - case shard.Unknown: - fallthrough - default: - return false, fmt.Errorf("unknown shard state: %v", shardState) - } - } - - if topology.ReadConsistencyAchieved( - bootstrapConsistencyLevel, majorityReplicas, numPeers, numAvailablePeers) { - // If we can achieve read consistency for any shard than we return true because - // we can't make any shard-by-shard distinction due to the fact that any given - // commitlog can contain writes for any shard. - return true, nil - } - } - - return false, nil + return majorityReplicas > 1 } func newReadSeriesPredicate(ns namespace.Metadata) commitlog.SeriesFilterPredicate { From 6e167e2bbad3870a659582777b0506d38880fb95 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 14:06:19 -0400 Subject: [PATCH 41/47] improve logic and comment --- .../bootstrap/bootstrapper/commitlog/source.go | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index a41592ce34..8f15a65c85 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -1535,13 +1535,10 @@ func (s commitLogSource) shouldReturnUnfulfilled( return false, nil } - couldObtainDataFromPeers, err := s.couldObtainDataFromPeers( + areShardsReplicated := s.areShardsReplicated( ns, shardsTimeRanges, opts) - if err != nil { - return false, err - } - return couldObtainDataFromPeers, nil + return areShardsReplicated, nil } func (s commitLogSource) maybeAddToIndex( @@ -1665,7 +1662,7 @@ func (s *commitLogSource) availability( return availableShardTimeRanges, nil } -func (s *commitLogSource) couldObtainDataFromPeers( +func (s *commitLogSource) areShardsReplicated( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, runOpts bootstrap.RunOptions, @@ -1675,6 +1672,13 @@ func (s *commitLogSource) couldObtainDataFromPeers( majorityReplicas = initialTopologyState.MajorityReplicas ) + // In any situation where we could actually stream data from our peers + // the replication factor would be 2 or larger which means that the + // value of majorityReplicas would be 2 or larger also. This heuristic can + // only be used to infer whether the replication factor is 1 or larger, but + // cannot be used to determine what the actual replication factor is in all + // situations because it can be ambiguous. For example, both R.F 2 and 3 will + // have majority replica values of 2. return majorityReplicas > 1 } From 9a6849c9deb52151bcce217ed512a98b873022c6 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 15 Oct 2018 14:06:45 -0400 Subject: [PATCH 42/47] mark prop test as large --- .../bootstrap/bootstrapper/commitlog/source_prop_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 161b24ecdd..ce38e2260b 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -1,3 +1,5 @@ +// +build big +// // Copyright (c) 2017 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy From fa4b0c3e5685946910cd783e4469ce982816bd8d Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Tue, 16 Oct 2018 10:21:23 -0400 Subject: [PATCH 43/47] Address feedback --- .../bootstrapper/commitlog/source.go | 127 +++++++++--------- 1 file changed, 66 insertions(+), 61 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 8f15a65c85..daccd07d87 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -47,7 +47,6 @@ import ( "github.com/m3db/m3x/pool" xsync "github.com/m3db/m3x/sync" xtime "github.com/m3db/m3x/time" - "github.com/uber-go/tally" ) @@ -76,66 +75,6 @@ type commitLogSource struct { metrics commitLogSourceDataAndIndexMetrics } -type commitLogSourceDataAndIndexMetrics struct { - data commitLogSourceMetrics - index commitLogSourceMetrics -} - -func newCommitLogSourceDataAndIndexMetrics(scope tally.Scope) commitLogSourceDataAndIndexMetrics { - return commitLogSourceDataAndIndexMetrics{ - data: newCommitLogSourceMetrics(scope), - index: newCommitLogSourceMetrics(scope), - } -} - -type commitLogSourceMetrics struct { - corruptCommitlogFile tally.Counter - readingSnapshots tally.Gauge - readingCommitlogs tally.Gauge - mergingSnapshotsAndCommitlogs tally.Gauge -} - -func (m commitLogSourceMetrics) emitReadingSnapshots() func() { - return m.gaugeLoop(m.readingSnapshots) -} - -func (m commitLogSourceMetrics) emitReadingCommitlogs() func() { - return m.gaugeLoop(m.readingCommitlogs) -} - -func (m commitLogSourceMetrics) emitMergingSnapshotsAndCommitlogs() func() { - return m.gaugeLoop(m.mergingSnapshotsAndCommitlogs) -} - -func (m commitLogSourceMetrics) gaugeLoop(g tally.Gauge) func() { - doneCh := make(chan struct{}) - go func() { - for { - select { - case <-doneCh: - g.Update(0) - return - default: - g.Update(1) - time.Sleep(time.Second) - } - } - }() - - return func() { close(doneCh) } -} - -func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics { - statusScope := scope.SubScope("status") - clScope := scope.SubScope("commitlog") - return commitLogSourceMetrics{ - corruptCommitlogFile: clScope.Counter("corrupt"), - readingSnapshots: statusScope.Gauge("reading-snapshots"), - readingCommitlogs: statusScope.Gauge("reading-commitlogs"), - mergingSnapshotsAndCommitlogs: statusScope.Gauge("merging-snapshots-and-commitlogs"), - } -} - type encoder struct { lastWriteAt time.Time enc encoding.Encoder @@ -1746,3 +1685,69 @@ func (ir ioReaders) close() { r.(xio.SegmentReader).Finalize() } } + +type commitLogSourceDataAndIndexMetrics struct { + data commitLogSourceMetrics + index commitLogSourceMetrics +} + +func newCommitLogSourceDataAndIndexMetrics(scope tally.Scope) commitLogSourceDataAndIndexMetrics { + return commitLogSourceDataAndIndexMetrics{ + data: newCommitLogSourceMetrics(scope.Tagged(map[string]string{ + "source_type": "data", + })), + index: newCommitLogSourceMetrics(scope.Tagged(map[string]string{ + "source_type": "index", + })), + } +} + +type commitLogSourceMetrics struct { + corruptCommitlogFile tally.Counter + readingSnapshots tally.Gauge + readingCommitlogs tally.Gauge + mergingSnapshotsAndCommitlogs tally.Gauge +} + +func (m commitLogSourceMetrics) emitReadingSnapshots() gaugeLoopCloserFn { + return m.gaugeLoop(m.readingSnapshots) +} + +func (m commitLogSourceMetrics) emitReadingCommitlogs() gaugeLoopCloserFn { + return m.gaugeLoop(m.readingCommitlogs) +} + +func (m commitLogSourceMetrics) emitMergingSnapshotsAndCommitlogs() gaugeLoopCloserFn { + return m.gaugeLoop(m.mergingSnapshotsAndCommitlogs) +} + +type gaugeLoopCloserFn func() + +func (m commitLogSourceMetrics) gaugeLoop(g tally.Gauge) gaugeLoopCloserFn { + doneCh := make(chan struct{}) + go func() { + for { + select { + case <-doneCh: + g.Update(0) + return + default: + g.Update(1) + time.Sleep(time.Second) + } + } + }() + + return func() { close(doneCh) } +} + +func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics { + statusScope := scope.SubScope("status") + clScope := scope.SubScope("commitlog") + return commitLogSourceMetrics{ + corruptCommitlogFile: clScope.Counter("corrupt"), + readingSnapshots: statusScope.Gauge("reading-snapshots"), + readingCommitlogs: statusScope.Gauge("reading-commitlogs"), + mergingSnapshotsAndCommitlogs: statusScope.Gauge("merging-snapshots-and-commitlogs"), + } +} From 926308c952af030ca2cab6e02f4de910d42c15d2 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Tue, 16 Oct 2018 10:38:59 -0400 Subject: [PATCH 44/47] Fix subscopes --- .../bootstrapper/commitlog/source.go | 51 +++++-------------- 1 file changed, 14 insertions(+), 37 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index daccd07d87..985f33791d 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -196,10 +196,13 @@ func (s *commitLogSource) ReadData( } var ( + // Emit bootstrapping gauge for duration of ReadData + doneBootstrapping = s.metrics.data.emitBootstrapping encounteredCorruptData = false fsOpts = s.opts.CommitLogOptions().FilesystemOptions() filePathPrefix = fsOpts.FilePathPrefix() ) + defer doneBootstrapping() // Determine which snapshot files are available. snapshotFilesByShard, err := s.snapshotFilesByShard( @@ -292,7 +295,6 @@ func (s *commitLogSource) ReadData( } // Read / M3TSZ encode all the datapoints in the commit log that we need to read. - doneReadingCommitlogs := s.metrics.data.emitReadingCommitlogs() for iter.Next() { series, dp, unit, annotation := iter.Current() if !s.shouldEncodeForData(shardDataByShard, blockSize, series, dp.Timestamp) { @@ -318,7 +320,6 @@ func (s *commitLogSource) ReadData( blockStart: dp.Timestamp.Truncate(blockSize), } } - doneReadingCommitlogs() if iterErr := iter.Err(); iterErr != nil { // Log the error and mark that we encountered corrupt data, but don't @@ -343,10 +344,7 @@ func (s *commitLogSource) ReadData( // Merge all the different encoders from the commit log that we created with // the data that is available in the snapshot files. s.log.Infof("starting merge...") - var ( - mergeStart = time.Now() - doneMerging = s.metrics.data.emitMergingSnapshotsAndCommitlogs() - ) + mergeStart := time.Now() bootstrapResult, err := s.mergeAllShardsCommitLogEncodersAndSnapshots( ns, shardsTimeRanges, @@ -356,7 +354,6 @@ func (s *commitLogSource) ReadData( blockSize, shardDataByShard, ) - doneMerging() if err != nil { return nil, err } @@ -1308,10 +1305,13 @@ func (s *commitLogSource) ReadIndex( } var ( + // Emit bootstrapping gauge for duration of ReadIndex + doneReadingIndex = s.metrics.index.emitBootstrapping() encounteredCorruptData = false fsOpts = s.opts.CommitLogOptions().FilesystemOptions() filePathPrefix = fsOpts.FilePathPrefix() ) + defer doneReadingIndex() // Determine which snapshot files are available. snapshotFilesByShard, err := s.snapshotFilesByShard( @@ -1360,13 +1360,11 @@ func (s *commitLogSource) ReadIndex( ) // Start by reading any available snapshot files. - doneReadingSnapshots := s.metrics.index.emitReadingSnapshots() for shard, tr := range shardsTimeRanges { shardResult, err := s.bootstrapShardSnapshots( ns.ID(), shard, true, tr, blockSize, snapshotFilesByShard[shard], mostRecentCompleteSnapshotByBlockShard) if err != nil { - doneReadingSnapshots() return nil, err } @@ -1381,7 +1379,6 @@ func (s *commitLogSource) ReadIndex( } } } - doneReadingSnapshots() // Next, read all of the data from the commit log files that wasn't covered // by the snapshot files. @@ -1396,7 +1393,6 @@ func (s *commitLogSource) ReadIndex( defer iter.Close() - doneReadingCommitlogs := s.metrics.index.emitReadingCommitlogs() for iter.Next() { series, dp, _, _ := iter.Current() @@ -1404,7 +1400,6 @@ func (s *commitLogSource) ReadIndex( series.ID, series.Tags, series.Shard, highestShard, dp.Timestamp, bootstrapRangesByShard, indexResults, indexOptions, indexBlockSize, resultOptions) } - doneReadingCommitlogs() if iterErr := iter.Err(); iterErr != nil { // Log the error and mark that we encountered corrupt data, but don't @@ -1703,36 +1698,22 @@ func newCommitLogSourceDataAndIndexMetrics(scope tally.Scope) commitLogSourceDat } type commitLogSourceMetrics struct { - corruptCommitlogFile tally.Counter - readingSnapshots tally.Gauge - readingCommitlogs tally.Gauge - mergingSnapshotsAndCommitlogs tally.Gauge -} - -func (m commitLogSourceMetrics) emitReadingSnapshots() gaugeLoopCloserFn { - return m.gaugeLoop(m.readingSnapshots) -} - -func (m commitLogSourceMetrics) emitReadingCommitlogs() gaugeLoopCloserFn { - return m.gaugeLoop(m.readingCommitlogs) -} - -func (m commitLogSourceMetrics) emitMergingSnapshotsAndCommitlogs() gaugeLoopCloserFn { - return m.gaugeLoop(m.mergingSnapshotsAndCommitlogs) + corruptCommitlogFile tally.Counter + bootstrapping tally.Gauge } type gaugeLoopCloserFn func() -func (m commitLogSourceMetrics) gaugeLoop(g tally.Gauge) gaugeLoopCloserFn { +func (m commitLogSourceMetrics) emitBootstrapping() gaugeLoopCloserFn { doneCh := make(chan struct{}) go func() { for { select { case <-doneCh: - g.Update(0) + m.bootstrapping.Update(0) return default: - g.Update(1) + m.bootstrapping.Update(1) time.Sleep(time.Second) } } @@ -1742,12 +1723,8 @@ func (m commitLogSourceMetrics) gaugeLoop(g tally.Gauge) gaugeLoopCloserFn { } func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics { - statusScope := scope.SubScope("status") - clScope := scope.SubScope("commitlog") return commitLogSourceMetrics{ - corruptCommitlogFile: clScope.Counter("corrupt"), - readingSnapshots: statusScope.Gauge("reading-snapshots"), - readingCommitlogs: statusScope.Gauge("reading-commitlogs"), - mergingSnapshotsAndCommitlogs: statusScope.Gauge("merging-snapshots-and-commitlogs"), + corruptCommitlogFile: scope.SubScope("commitlog").Counter("corrupt"), + bootstrapping: scope.SubScope("status").Gauge("bootstrapping"), } } From 656df6ed2839a180cd6f9f017da0d31f92dd036e Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Tue, 16 Oct 2018 10:46:16 -0400 Subject: [PATCH 45/47] Fix import order --- src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go | 1 + 1 file changed, 1 insertion(+) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 985f33791d..00d584318a 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -47,6 +47,7 @@ import ( "github.com/m3db/m3x/pool" xsync "github.com/m3db/m3x/sync" xtime "github.com/m3db/m3x/time" + "github.com/uber-go/tally" ) From da67bf75ba533c0912c30a8326726e576b77f8f0 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Tue, 16 Oct 2018 11:20:57 -0400 Subject: [PATCH 46/47] fix flaky test --- src/dbnode/persist/fs/commitlog/files_test.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/dbnode/persist/fs/commitlog/files_test.go b/src/dbnode/persist/fs/commitlog/files_test.go index 599fe5772d..ef0d94b97c 100644 --- a/src/dbnode/persist/fs/commitlog/files_test.go +++ b/src/dbnode/persist/fs/commitlog/files_test.go @@ -110,13 +110,12 @@ func createTestCommitLogFiles( } // Commit log writer is asynchronous and performs batching so getting the exact number // of files that we want is tricky. The implementation below loops infinitely, writing - // a single datapoint and increasing the time after each iteration until numBlocks - // files are on disk. After that, it terminates, and the final batch flush from calling - // commitlog.Close() will generate the last file. + // a single datapoint and increasing the time after each iteration until minNumBlocks + // files are on disk. for { files, err := fs.SortedCommitLogFiles(commitLogsDir) require.NoError(t, err) - if len(files) == minNumBlocks { + if len(files) >= minNumBlocks { break } err = commitLog.Write(context.NewContext(), series, ts.Datapoint{}, xtime.Second, nil) From b8020fe1fb87e6a20d38e514b427b26bdcee3817 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Tue, 16 Oct 2018 15:13:25 -0400 Subject: [PATCH 47/47] Fix broken metric --- src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 00d584318a..7fd906d4f3 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -198,12 +198,12 @@ func (s *commitLogSource) ReadData( var ( // Emit bootstrapping gauge for duration of ReadData - doneBootstrapping = s.metrics.data.emitBootstrapping + doneReadingData = s.metrics.data.emitBootstrapping() encounteredCorruptData = false fsOpts = s.opts.CommitLogOptions().FilesystemOptions() filePathPrefix = fsOpts.FilePathPrefix() ) - defer doneBootstrapping() + defer doneReadingData() // Determine which snapshot files are available. snapshotFilesByShard, err := s.snapshotFilesByShard(