Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Handle commit log files with corrupt info headers during cleanup and bootstrap #1066

Merged
merged 47 commits into from
Oct 16, 2018

Conversation

richardartoul
Copy link
Contributor

@richardartoul richardartoul commented Oct 10, 2018

  • Automatically delete commit log files with corrupt info headers during cleanup instead of erroring out forever and requiring manual intervention.
  • Add static configuration for whether or not commitlog bootstrapping corruption should return unfulfilled or not for the whole thing (default to return unfulfilled)
  • Always read all the commit log files we intended to read anyways, regardless of that config.
  • Auto-detect in commitlog bootstrapper if it is possible for the peers bootstrapper to satisfy requests. If not, return fulfilled instead of unfulfilled because the peers bootstrapper cant perform a repair anyways.
  • Don't return errors from the commitlog bootstrapper for corrupt files
  • Emit logs for all encountered corrupted files
  • Emit metrics for all encountered corrupt files

@richardartoul richardartoul changed the title Handle commit log files with corrupt info headers during cleanup Handle commit log files with corrupt info headers during cleanup and bootstrap Oct 10, 2018
@codecov
Copy link

codecov bot commented Oct 10, 2018

Codecov Report

Merging #1066 into master will increase coverage by 2.03%.
The diff coverage is 82.44%.

Impacted file tree graph

@@            Coverage Diff             @@
##           master    #1066      +/-   ##
==========================================
+ Coverage    75.3%   77.34%   +2.03%     
==========================================
  Files         569      578       +9     
  Lines       48213    48590     +377     
==========================================
+ Hits        36307    37580    +1273     
+ Misses       9640     8642     -998     
- Partials     2266     2368     +102
Flag Coverage Δ
#aggregator 81.59% <ø> (ø) ⬆️
#collector 59.23% <ø> (ø) ⬆️
#dbnode 81.39% <82.44%> (+3.89%) ⬆️
#m3em 73.21% <ø> (ø) ⬆️
#m3ninx 75.25% <ø> (+4.02%) ⬆️
#m3nsch 51.19% <ø> (ø) ⬆️
#msg 74.98% <ø> (ø) ⬆️
#query 63.67% <ø> (-1.65%) ⬇️
#x 75.1% <ø> (+5.74%) ⬆️

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 2eb71f7...b8020fe. Read the comment docs.

// 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
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

super nit: mind making a union type instead of this just to be clear to users. something like:

type File struct {
  FilePath string
  Start time.Time
  Duration time.Duration
  Index int64
}

type FileOrError struct {
  Path string
  File File
  Error error
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I could also do something like:

type FileOrError struct {
    file File
    err Error
}


func (f *FileOrError) FIle() (File, Error) {
    return f.file, f.err
}

And then its basically impossible to misuse it

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i don't like that people could use the Path regardless of error in that scheme – (e.g. the cleanup manager still has to delete the file regardless of err). In addition to what you suggested, could you expose a typed error with the path available to get around that?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just to clarify, are you saying you don’t like the idea of someone receiving an error but then still needing to access the File object to get at the path, so we should export an error struct that include the file path, correct? Yeah that seems better to me too

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep, exactly.

@@ -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 {
Copy link
Collaborator

@prateek prateek Oct 11, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can't skip corrupt commit log files like they didn't exist w/o risking data loss. It'd be much safer to see if any corrupt commit logs exist and if so fall back to the peers bootstrapper (in the case there are peers), and if there are no peers allow users to opt in to skip corrupt commit log files but don't do it by default.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm ok with making this opt-outable for users who really care about data integrity, but I think by default it should skip them. If you can't even read the info part of the commitlog, then (barring true corruption of the filesystem unrelated to M3DB shutdown) your commitlog isn't missing any data anyways other than the standard amount you would be missing due to the writebehind strategy. I don't think this is a situation where we need to involve operators in the general case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

by putting the commitlog bootstrapper before peers bootstrapper you're basically accepting losing the last few writes that occurred right before shutdown due to the writebehind strategy (as well as any writes that were issued while you were down) anyways.

In my mind we should emit a log, skip over bad commitlog files, and then eventually address this issue with the corruption APIs / a background repair process.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The other reason I think it sucks to push this burden down to the operators is that basically by the time this happen and you realize whats going on, there are only two options:

  1. Disable commitlog bootstrapper and replace it with peers bootstrapper, which is fine, but only viable in non-catastrophic scenarios where all the other nodes are up.

  2. Go and delete the bad commitlog file and then restart with the commitlog bootstrapper, at which point you've probably written out enough commit log files that what should have been 6->8 minutes of downtime turns into an hour-long ordeal.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For the peers case: how about just returning returning all requested shard ranges as unfullfilled in the commit log bootstrapper (won't need any interface changes from the way things are now) when the commit log runs into an error and that way we'll fall back to peers automatically.

For the single node case: we can have a new RunOption to control the behaviour of whether to skip or not and the bootstrap process can be configured from the top level to specify what to pass it. That way the single node config we put out can default to come up but we can put a WARNING next to it and in docs.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Spoke offline and settled on:

1. Add static configuration for whether or not commitlog bootstrapping corruption should return unfulfilled or not for the whole thing (default to return unfulfilled)
2. Always read all the commit log files we intended to read anyways, regardless of that config.
3. Add exception in commitlog bootstrap for single node deployment which will detect that this is the only node in the placement and default to returning fulfilled because there is nothing else you could possibly do to recover data
4. Don't return errors from the commitlog bootstrapper for corrupt files
5. Emit metrics / logs for all encountered corrupt files.

This gives us a good mix between maintaining data integrity as much as we can, but also not imposing undue operational burden.

In single node failures, the issue will just resolve itself.
In a single node failure within a multi-node cluster, the issue will just resolve itself.
In a multi-node failure within a multi-node cluster, some nodes with corrupt commitlog files may get stuck in the peer bootstrapping phase, but they can be "released" with a K.V change to the peer bootstrapper consistency level and come online without requiring an additional costly restart.

@richardartoul richardartoul changed the title Handle commit log files with corrupt info headers during cleanup and bootstrap [WIP] - Handle commit log files with corrupt info headers during cleanup and bootstrap Oct 11, 2018
@richardartoul richardartoul changed the title [WIP] - Handle commit log files with corrupt info headers during cleanup and bootstrap Handle commit log files with corrupt info headers during cleanup and bootstrap Oct 12, 2018
@@ -30,6 +30,61 @@ import (
"github.com/m3db/m3/src/dbnode/persist/fs/msgpack"
)

type fsError struct {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit move these error types to the bottom of the file

@@ -38,6 +93,13 @@ type File struct {
Index int64
}

func newErrorWithPath(err error, path string) ErrorWithPath {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same for this ctor, move it next to the associated type at the bottom of the file

}

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.
Copy link
Collaborator

@prateek prateek Oct 13, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why sort at all? and if you're going to sort - you can sort errors first (or last) and then guarantee order too.

or better yet change the return type for this function: ([]File, []ErrorWithPath, error) and then won't need the FileOrError type.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorting is important because you want to try and read the commit log files in order in the bootstrapper

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Your interface change suggestion is good though, just made it

@@ -38,27 +38,30 @@ import (
)

func TestFiles(t *testing.T) {
// TODO(r): Find some time/people to help investigate this flakey test.
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lol is this no longer flaky?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah I fixed it

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

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)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah is this the fix?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yep

func filterFiles(opts Options, files []File, predicate FileFilterPredicate) []File {
filteredFiles := make([]File, 0, len(files))
for _, f := range files {
func filterFiles(opts Options, files []FileOrError, predicate FileFilterPredicate) ([]File, []ErrorWithPath) {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

as above, can move this to be used within the Files() method.

}
}

type commitLogSourceMetrics struct {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

huge +1 to this.

@@ -275,6 +323,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)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could you do same thing we normally do - use an atomic and have a background routine emitting the guage regularly. that way you'd get more than a single datapoint indicating what's going on.

@@ -682,7 +751,7 @@ func (s *commitLogSource) newReadCommitLogPredBasedOnAvailableSnapshotFiles(
shardsTimeRanges result.ShardTimeRanges,
snapshotFilesByShard map[uint32]fs.FileSetFilesSlice,
) (
func(f commitlog.File) bool,
commitlog.FileFilterPredicate,
Copy link
Collaborator

@prateek prateek Oct 13, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 for type safety

@@ -1355,6 +1433,17 @@ 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
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lol what i wouldn't give for macros.

return false, nil
}

couldObtainDataFromPeers, err := s.couldObtainDataFromPeers(
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hm - i thought you were going to check if it's a single node for this case. why use the more extended peer condition?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seemed like it was slightly better to not hand off to the peers bootstrapper if it can't help anyways. Think I should remove it?

"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
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@@ -38,27 +38,30 @@ import (
)

func TestFiles(t *testing.T) {
// TODO(r): Find some time/people to help investigate this flakey test.
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

func newCommitLogSourceDataAndIndexMetrics(scope tally.Scope) commitLogSourceDataAndIndexMetrics {
return commitLogSourceDataAndIndexMetrics{
data: newCommitLogSourceMetrics(scope),
index: newCommitLogSourceMetrics(scope),
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a different tag for each one? Otherwise they'll overwrite each other's metrics.

e.g.

data:  newCommitLogSourceMetrics(scope.Tagged(map[string]string{"source_type": "data"})),
index:  newCommitLogSourceMetrics(scope.Tagged(map[string]string{"source_type": "index"})),

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh duh, thanks

return m.gaugeLoop(m.mergingSnapshotsAndCommitlogs)
}

func (m commitLogSourceMetrics) gaugeLoop(g tally.Gauge) func() {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you type-ify this function so it's a little cleaner, e.g.:

type closerFn func()

func (m commitLogSourceMetrics) emitReadingSnapshots() closerFn {
	return m.gaugeLoop(m.readingSnapshots)
}

@@ -300,9 +379,17 @@ func (s *commitLogSource) ReadData(
blockStart: dp.Timestamp.Truncate(blockSize),
}
}
doneReadingCommitlogs()
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it a waste to just emit the gauges the whole time this function is running? Then we could do it all at the start and defer all these cleanups. It would be better mainly because it would defend against an early return not stopping the loop which would cause a memory leak (for loop continuing to hold ref to the commit log source forever.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

see comment below

@@ -1319,11 +1421,13 @@ func (s *commitLogSource) ReadIndex(
)

// Start by reading any available snapshot files.
doneReadingSnapshots := s.metrics.index.emitReadingSnapshots()
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we could do this with a defer somehow that would also be nicer, is there some state it can just read for the whole execution of the function and emit one or zero instead of needing to explicitly emit based on control flow?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah there isn't any obvious state we could read to make a decision (any state we could read would have to be added based on control flow.)

I think we either leave it as is, or we can simplify it by not distinguishing between reading commitlogs and snapshotting and just having a gauge for "commitlog bootstrapping data" and "commitlog bootstrapping index"

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

honestly, making the distinction between commitlogs and snapshots is not worth it because once I land the perf improvement reading snapshots will take like 20-30 seconds so not worth separating the two. I'll make the change

}

if shouldReturnUnfulfilled {
indexResult.SetUnfulfilled(shardsTimeRanges)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We still return our data right? (I know this is likely true, just want to verify). That way we can join whatever we had at least.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah this should have no effect on what data we return (pretty sure my prop test still verifies the data is there) so as long as the caller doesn't ignore the data it will get merged in


func TestCleanupManagerDeletesCorruptCommitLogFiles(t *testing.T) {
// TODO(rartoul): Re-enable this once https://github.com/m3db/m3/issues/1078
// is resolved.
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a note to the issue to "be sure to re-enable test TestCleanupManagerDeletesCorruptCommitLogFiles with the same PR that fixes this issue".

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure

fsOpts = s.opts.CommitLogOptions().FilesystemOptions()
filePathPrefix = fsOpts.FilePathPrefix()
// Emit bootstrapping gauge for duration of ReadData
doneBootstrapping = s.metrics.data.emitBootstrapping
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, did you mean this?:

		doneBootstrapping      = s.metrics.data.emitBootstrapping()

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Re: why didn't this break, as a meta point if we wanted to be more "type safe" about it we could make it return an io.Closer than calls the method.

import "github.com/m3db/m3x/close"

var _ close.Closer = closerFn(nil)

// Implement close.Closer
func (f closerFn) Close() {
	f()
}

// Now to call it

func myThing() close.Closer {
	return closerFn(func() { /* code */})
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed the function call. I get what you're saying, but seems like it really hurts readability for something that would have been caught by a test if it was really important (we don't currently test whether metrics are being emitted properly.) I think I'm just gonna make the fix and leave as is for now

Copy link
Collaborator

@robskillington robskillington left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Other than final comment, LGTM

@richardartoul richardartoul merged commit f42a475 into master Oct 16, 2018
@justinjc justinjc deleted the ra/fix-commitlog-cleanup branch October 30, 2018 20:48
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants