Merge pull request #9770 from influxdata/jmw-inspect-series-file

Add verify-seriesfile to influx_inspect
pull/9803/head
Jeff Wendling 2018-05-02 11:27:53 -06:00 committed by GitHub
commit cf81a3e66e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 727 additions and 6 deletions

View File

@ -143,7 +143,7 @@ func (cmd *Command) process(path string) error {
}
func (cmd *Command) printUsage() {
fmt.Println(`Deletes a measurement from a raw tsm file.
fmt.Print(`Deletes a measurement from a raw tsm file.
Usage: influx_inspect deletetsm [flags] path...

View File

@ -39,6 +39,7 @@ The commands are:
help display this help message
report displays a shard level report
verify verifies integrity of TSM files
verify-seriesfile verifies integrity of the Series file
"help" is the default command.

View File

@ -15,7 +15,8 @@ import (
"github.com/influxdata/influxdb/cmd/influx_inspect/export"
"github.com/influxdata/influxdb/cmd/influx_inspect/help"
"github.com/influxdata/influxdb/cmd/influx_inspect/report"
"github.com/influxdata/influxdb/cmd/influx_inspect/verify"
"github.com/influxdata/influxdb/cmd/influx_inspect/verify/seriesfile"
"github.com/influxdata/influxdb/cmd/influx_inspect/verify/tsm"
_ "github.com/influxdata/influxdb/tsdb/engine"
)
@ -90,10 +91,15 @@ func (m *Main) Run(args ...string) error {
return fmt.Errorf("report: %s", err)
}
case "verify":
name := verify.NewCommand()
name := tsm.NewCommand()
if err := name.Run(args...); err != nil {
return fmt.Errorf("verify: %s", err)
}
case "verify-seriesfile":
name := seriesfile.NewCommand()
if err := name.Run(args...); err != nil {
return fmt.Errorf("verify-seriesfile: %s", err)
}
default:
return fmt.Errorf(`unknown command "%s"`+"\n"+`Run 'influx_inspect help' for usage`+"\n\n", name)
}

View File

@ -0,0 +1,120 @@
// Package seriesfile verifies integrity of series files.
package seriesfile
import (
"flag"
"fmt"
"io"
"io/ioutil"
"os"
"path/filepath"
"runtime"
"github.com/influxdata/influxdb/logger"
"go.uber.org/zap/zapcore"
)
// Command represents the program execution for "influx_inspect verify-seriesfile".
type Command struct {
Stdout io.Writer
Stderr io.Writer
dir string
db string
seriesFile string
verbose bool
concurrent int
}
// NewCommand returns a new instance of Command.
func NewCommand() *Command {
return &Command{
Stdout: os.Stdout,
Stderr: os.Stderr,
}
}
// Run executes the command.
func (cmd *Command) Run(args ...string) error {
fs := flag.NewFlagSet("verify-seriesfile", flag.ExitOnError)
fs.StringVar(&cmd.dir, "dir", filepath.Join(os.Getenv("HOME"), ".influxdb", "data"),
"Data directory.")
fs.StringVar(&cmd.db, "db", "",
"Only use this database inside of the data directory.")
fs.StringVar(&cmd.seriesFile, "series-file", "",
"Path to a series file. This overrides -db and -dir.")
fs.BoolVar(&cmd.verbose, "v", false,
"Verbose output.")
fs.IntVar(&cmd.concurrent, "c", runtime.GOMAXPROCS(0),
"How many concurrent workers to run.")
fs.SetOutput(cmd.Stdout)
fs.Usage = cmd.printUsage
if err := fs.Parse(args); err != nil {
return err
}
config := logger.NewConfig()
config.Level = zapcore.WarnLevel
if cmd.verbose {
config.Level = zapcore.InfoLevel
}
logger, err := config.New(cmd.Stderr)
if err != nil {
return err
}
v := NewVerify()
v.Logger = logger
v.Concurrent = cmd.concurrent
if cmd.seriesFile != "" {
_, err := v.VerifySeriesFile(cmd.seriesFile)
return err
}
if cmd.db != "" {
_, err := v.VerifySeriesFile(filepath.Join(cmd.dir, cmd.db, "_series"))
return err
}
dbs, err := ioutil.ReadDir(cmd.dir)
if err != nil {
return err
}
for _, db := range dbs {
if !db.IsDir() {
continue
}
_, err := v.VerifySeriesFile(filepath.Join(cmd.dir, db.Name(), "_series"))
if err != nil {
return err
}
}
return nil
}
func (cmd *Command) printUsage() {
usage := `Verifies the integrity of Series files.
Usage: influx_inspect verify-seriesfile [flags]
-dir <path>
Root data path.
Defaults to "%[1]s/.influxdb/data".
-db <name>
Only verify this database inside of the data directory.
-series-file <path>
Path to a series file. This overrides -db and -dir.
-v
Enable verbose logging.
-c
How many concurrent workers to run.
Defaults to "%[2]d" on this machine.
`
fmt.Printf(usage, os.Getenv("HOME"), runtime.GOMAXPROCS(0))
}

View File

@ -0,0 +1,403 @@
package seriesfile
import (
"fmt"
"io/ioutil"
"os"
"path/filepath"
"runtime"
"sort"
"sync"
"github.com/influxdata/influxdb/tsdb"
"go.uber.org/zap"
)
// verifyResult contains the result of a Verify... call
type verifyResult struct {
valid bool
err error
}
// Verify contains configuration for running verification of series files.
type Verify struct {
Concurrent int
Logger *zap.Logger
done chan struct{}
}
// NewVerify constructs a Verify with good defaults.
func NewVerify() Verify {
return Verify{
Concurrent: runtime.GOMAXPROCS(0),
Logger: zap.NewNop(),
}
}
// VerifySeriesFile performs verifications on a series file. The error is only returned
// if there was some fatal problem with operating, not if there was a problem with the series file.
func (v Verify) VerifySeriesFile(filePath string) (valid bool, err error) {
v.Logger = v.Logger.With(zap.String("path", filePath))
v.Logger.Info("Verifying series file")
defer func() {
if rec := recover(); rec != nil {
v.Logger.Error("Panic verifying file", zap.String("recovered", fmt.Sprint(rec)))
valid = false
}
}()
partitionInfos, err := ioutil.ReadDir(filePath)
if os.IsNotExist(err) {
v.Logger.Error("Series file does not exist")
return false, nil
}
if err != nil {
return false, err
}
// Check every partition in concurrently.
concurrent := v.Concurrent
if concurrent <= 0 {
concurrent = 1
}
in := make(chan string, len(partitionInfos))
out := make(chan verifyResult, len(partitionInfos))
// Make sure all the workers are cleaned up when we return.
var wg sync.WaitGroup
defer wg.Wait()
// Set up cancellation. Any return will cause the workers to be cancelled.
v.done = make(chan struct{})
defer close(v.done)
for i := 0; i < concurrent; i++ {
wg.Add(1)
go func() {
defer wg.Done()
for partitionPath := range in {
valid, err := v.VerifyPartition(partitionPath)
select {
case out <- verifyResult{valid: valid, err: err}:
case <-v.done:
return
}
}
}()
}
// send off the work and read the results.
for _, partitionInfo := range partitionInfos {
in <- filepath.Join(filePath, partitionInfo.Name())
}
close(in)
for range partitionInfos {
result := <-out
if result.err != nil {
return false, err
} else if !result.valid {
return false, nil
}
}
return true, nil
}
// VerifyPartition performs verifications on a partition of a series file. The error is only returned
// if there was some fatal problem with operating, not if there was a problem with the partition.
func (v Verify) VerifyPartition(partitionPath string) (valid bool, err error) {
v.Logger = v.Logger.With(zap.String("partition", filepath.Base(partitionPath)))
v.Logger.Info("Verifying partition")
defer func() {
if rec := recover(); rec != nil {
v.Logger.Error("Panic verifying partition", zap.String("recovered", fmt.Sprint(rec)))
valid = false
}
}()
segmentInfos, err := ioutil.ReadDir(partitionPath)
if err != nil {
return false, err
}
segments := make([]*tsdb.SeriesSegment, 0, len(segmentInfos))
ids := make(map[uint64]IDData)
// check every segment
for _, segmentInfo := range segmentInfos {
select {
default:
case <-v.done:
return false, nil
}
segmentPath := filepath.Join(partitionPath, segmentInfo.Name())
segmentID, err := tsdb.ParseSeriesSegmentFilename(segmentInfo.Name())
if err != nil {
continue
}
if valid, err := v.VerifySegment(segmentPath, ids); err != nil {
return false, err
} else if !valid {
return false, nil
}
// open the segment for verifying the index. we want it to be open outside
// the for loop as well, so the defer is ok.
segment := tsdb.NewSeriesSegment(segmentID, segmentPath)
if err := segment.Open(); err != nil {
return false, err
}
defer segment.Close()
segments = append(segments, segment)
}
// check the index
indexPath := filepath.Join(partitionPath, "index")
if valid, err := v.VerifyIndex(indexPath, segments, ids); err != nil {
return false, err
} else if !valid {
return false, nil
}
return true, nil
}
// IDData keeps track of data about a series ID.
type IDData struct {
Offset int64
Key []byte
Deleted bool
}
// VerifySegment performs verifications on a segment of a series file. The error is only returned
// if there was some fatal problem with operating, not if there was a problem with the partition.
// The ids map is populated with information about the ids stored in the segment.
func (v Verify) VerifySegment(segmentPath string, ids map[uint64]IDData) (valid bool, err error) {
segmentName := filepath.Base(segmentPath)
v.Logger = v.Logger.With(zap.String("segment", segmentName))
v.Logger.Info("Verifying segment")
// Open up the segment and grab it's data.
segmentID, err := tsdb.ParseSeriesSegmentFilename(segmentName)
if err != nil {
return false, err
}
segment := tsdb.NewSeriesSegment(segmentID, segmentPath)
if err := segment.Open(); err != nil {
v.Logger.Error("Error opening segment", zap.Error(err))
return false, nil
}
defer segment.Close()
buf := newBuffer(segment.Data())
defer func() {
if rec := recover(); rec != nil {
v.Logger.Error("Panic verifying segment", zap.String("recovered", fmt.Sprint(rec)),
zap.Int64("offset", buf.offset))
valid = false
}
}()
// Skip the header: it has already been verified by the Open call.
if err := buf.advance(tsdb.SeriesSegmentHeaderSize); err != nil {
v.Logger.Error("Unable to advance buffer",
zap.Int64("offset", buf.offset),
zap.Error(err))
return false, nil
}
prevID, firstID := uint64(0), true
entries:
for len(buf.data) > 0 {
select {
default:
case <-v.done:
return false, nil
}
flag, id, key, sz := tsdb.ReadSeriesEntry(buf.data)
// Check the flag is valid and for id monotonicity.
switch flag {
case tsdb.SeriesEntryInsertFlag:
if !firstID && prevID > id {
v.Logger.Error("ID is not monotonically increasing",
zap.Uint64("prev_id", prevID),
zap.Uint64("id", id),
zap.Int64("offset", buf.offset))
return false, nil
}
firstID = false
prevID = id
if ids != nil {
keyCopy := make([]byte, len(key))
copy(keyCopy, key)
ids[id] = IDData{
Offset: tsdb.JoinSeriesOffset(segment.ID(), uint32(buf.offset)),
Key: keyCopy,
}
}
case tsdb.SeriesEntryTombstoneFlag:
if ids != nil {
data := ids[id]
data.Deleted = true
ids[id] = data
}
case 0: // if zero, there are no more entries
if err := buf.advance(sz); err != nil {
v.Logger.Error("Unable to advance buffer",
zap.Int64("offset", buf.offset),
zap.Error(err))
return false, nil
}
break entries
default:
v.Logger.Error("Invalid flag",
zap.Uint8("flag", flag),
zap.Int64("offset", buf.offset))
return false, nil
}
// Ensure the key parses. This may panic, but our defer handler should
// make the error message more usable by providing the key.
parsed := false
func() {
defer func() {
if rec := recover(); rec != nil {
v.Logger.Error("Panic parsing key",
zap.String("key", fmt.Sprintf("%x", key)),
zap.Int64("offset", buf.offset),
zap.String("recovered", fmt.Sprint(rec)))
}
}()
tsdb.ParseSeriesKey(key)
parsed = true
}()
if !parsed {
return false, nil
}
// Advance past the entry.
if err := buf.advance(sz); err != nil {
v.Logger.Error("Unable to advance buffer",
zap.Int64("offset", buf.offset),
zap.Error(err))
return false, nil
}
}
return true, nil
}
// VerifyIndex performs verification on an index in a series file. The error is only returned
// if there was some fatal problem with operating, not if there was a problem with the partition.
// The ids map must be built from verifying the passed in segments.
func (v Verify) VerifyIndex(indexPath string, segments []*tsdb.SeriesSegment,
ids map[uint64]IDData) (valid bool, err error) {
v.Logger.Info("Verifying index")
defer func() {
if rec := recover(); rec != nil {
v.Logger.Error("Panic verifying index", zap.String("recovered", fmt.Sprint(rec)))
valid = false
}
}()
index := tsdb.NewSeriesIndex(indexPath)
if err := index.Open(); err != nil {
v.Logger.Error("Error opening index", zap.Error(err))
return false, nil
}
defer index.Close()
if err := index.Recover(segments); err != nil {
v.Logger.Error("Error recovering index", zap.Error(err))
return false, nil
}
// we check all the ids in a consistent order to get the same errors if
// there is a problem
idsList := make([]uint64, 0, len(ids))
for id := range ids {
idsList = append(idsList, id)
}
sort.Slice(idsList, func(i, j int) bool {
return idsList[i] < idsList[j]
})
for _, id := range idsList {
select {
default:
case <-v.done:
return false, nil
}
IDData := ids[id]
expectedOffset, expectedID := IDData.Offset, id
if IDData.Deleted {
expectedOffset, expectedID = 0, 0
}
// check both that the offset is right and that we get the right
// id for the key
if gotOffset := index.FindOffsetByID(id); gotOffset != expectedOffset {
v.Logger.Error("Index inconsistency",
zap.Uint64("id", id),
zap.Int64("got_offset", gotOffset),
zap.Int64("expected_offset", expectedOffset))
return false, nil
}
if gotID := index.FindIDBySeriesKey(segments, IDData.Key); gotID != expectedID {
v.Logger.Error("Index inconsistency",
zap.Uint64("id", id),
zap.Uint64("got_id", gotID),
zap.Uint64("expected_id", expectedID))
return false, nil
}
}
return true, nil
}
// buffer allows one to safely advance a byte slice and keep track of how many bytes were advanced.
type buffer struct {
offset int64
data []byte
}
// newBuffer constructs a buffer with the provided data.
func newBuffer(data []byte) *buffer {
return &buffer{
offset: 0,
data: data,
}
}
// advance will consume n bytes from the data slice and return an error if there is not enough
// data to do so.
func (b *buffer) advance(n int64) error {
if int64(len(b.data)) < n {
return fmt.Errorf("unable to advance %d bytes: %d remaining", n, len(b.data))
}
b.data = b.data[n:]
b.offset += n
return nil
}

View File

@ -0,0 +1,163 @@
package seriesfile_test
import (
"fmt"
"io"
"io/ioutil"
"os"
"path/filepath"
"testing"
"time"
"github.com/influxdata/influxdb/cmd/influx_inspect/verify/seriesfile"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb"
)
func TestVerifies_Valid(t *testing.T) {
test := NewTest(t)
defer test.Close()
passed, err := seriesfile.NewVerify().VerifySeriesFile(test.Path)
test.AssertNoError(err)
test.Assert(passed)
}
func TestVerifies_Invalid(t *testing.T) {
test := NewTest(t)
defer test.Close()
test.AssertNoError(filepath.Walk(test.Path, func(path string, info os.FileInfo, err error) error {
if err != nil {
return err
}
if info.IsDir() {
return nil
}
test.Backup(path)
defer test.Restore(path)
fh, err := os.OpenFile(path, os.O_RDWR, 0)
test.AssertNoError(err)
defer fh.Close()
_, err = fh.WriteAt([]byte("BOGUS"), 0)
test.AssertNoError(err)
test.AssertNoError(fh.Close())
passed, err := seriesfile.NewVerify().VerifySeriesFile(test.Path)
test.AssertNoError(err)
test.Assert(!passed)
return nil
}))
}
//
// helpers
//
type Test struct {
*testing.T
Path string
}
func NewTest(t *testing.T) *Test {
t.Helper()
dir, err := ioutil.TempDir("", "verify-seriesfile-")
if err != nil {
t.Fatal(err)
}
// create a series file in the directory
err = func() error {
seriesFile := tsdb.NewSeriesFile(dir)
if err := seriesFile.Open(); err != nil {
return err
}
defer seriesFile.Close()
seriesFile.EnableCompactions()
const (
compactionThreshold = 100
numSeries = 2 * tsdb.SeriesFilePartitionN * compactionThreshold
)
for _, partition := range seriesFile.Partitions() {
partition.CompactThreshold = compactionThreshold
}
var names [][]byte
var tagsSlice []models.Tags
for i := 0; i < numSeries; i++ {
names = append(names, []byte(fmt.Sprintf("series%d", i)))
tagsSlice = append(tagsSlice, nil)
}
_, err := seriesFile.CreateSeriesListIfNotExists(names, tagsSlice, nil)
if err != nil {
return err
}
// wait for compaction to make sure we detect issues with the index
partitions := seriesFile.Partitions()
wait:
for _, partition := range partitions {
if partition.Compacting() {
time.Sleep(100 * time.Millisecond)
goto wait
}
}
return seriesFile.Close()
}()
if err != nil {
os.RemoveAll(dir)
t.Fatal(err)
}
return &Test{
T: t,
Path: dir,
}
}
func (t *Test) Close() {
os.RemoveAll(t.Path)
}
func (t *Test) AssertNoError(err error) {
t.Helper()
if err != nil {
t.Fatal("unexpected error:", err)
}
}
func (t *Test) Assert(x bool) {
t.Helper()
if !x {
t.Fatal("unexpected condition")
}
}
// Backup makes a copy of the path for a later Restore.
func (t *Test) Backup(path string) {
in, err := os.Open(path)
t.AssertNoError(err)
defer in.Close()
out, err := os.Create(path + ".backup")
t.AssertNoError(err)
defer out.Close()
_, err = io.Copy(out, in)
t.AssertNoError(err)
}
// Restore restores the file at the path to the time when Backup was called last.
func (t *Test) Restore(path string) {
t.AssertNoError(os.Rename(path+".backup", path))
}

View File

@ -1,5 +1,5 @@
// Package verify verifies integrity of TSM files.
package verify
// Package tsm verifies integrity of TSM files.
package tsm
import (
"flag"

View File

@ -1,3 +1,3 @@
package verify_test
package tsm_test
// TODO: write some tests

View File

@ -12,6 +12,7 @@ SYNPOSIS
'influx_inspect export' [options]
'influx_inspect report' [options]
'influx_inspect verify' [options]
'influx_inspect verify-seriesfile' [options]
DESCRIPTION
-----------
@ -83,4 +84,21 @@ VERIFY OPTIONS
-dir <path>::
Root storage path. Defaults to '~/.influxdb'.
VERIFY-SERIESFILE OPTIONS
--------------
-dir <path>::
Root data storage path. Defaults to '~/.influxdb/data'.
-db <name>::
Specific db to check. Optional.
-file <path>::
Path to a specific series file to check. Overrides '-dir' and '-db'. Optional.
-v::
Verbose output. Optional.
-c <amount::
Number of concurrent workers to run. Defaults to the number of cores on the machine. Optional.
include:footer.txt[]

View File

@ -282,6 +282,13 @@ func (p *SeriesPartition) CreateSeriesListIfNotExists(keys [][]byte, keyPartitio
return nil
}
// Compacting returns if the SeriesPartition is currently compacting.
func (p *SeriesPartition) Compacting() bool {
p.mu.RLock()
defer p.mu.RUnlock()
return p.compacting
}
// DeleteSeriesID flags a series as permanently deleted.
// If the series is reintroduced later then it must create a new id.
func (p *SeriesPartition) DeleteSeriesID(id uint64) error {

View File

@ -168,6 +168,9 @@ func (s *SeriesSegment) CloseForWrite() (err error) {
return err
}
// Data returns the raw data.
func (s *SeriesSegment) Data() []byte { return s.data }
// ID returns the id the segment was initialized with.
func (s *SeriesSegment) ID() uint16 { return s.id }