diff --git a/go/cmd/dolt/cli/flags.go b/go/cmd/dolt/cli/flags.go index b6e6111ca16..e1602e93b3a 100644 --- a/go/cmd/dolt/cli/flags.go +++ b/go/cmd/dolt/cli/flags.go @@ -62,6 +62,7 @@ const ( PasswordFlag = "password" PortFlag = "port" PruneFlag = "prune" + QuietFlag = "quiet" RemoteParam = "remote" SetUpstreamFlag = "set-upstream" ShallowFlag = "shallow" diff --git a/go/cmd/dolt/commands/fsck.go b/go/cmd/dolt/commands/fsck.go new file mode 100644 index 00000000000..b7e4d85268c --- /dev/null +++ b/go/cmd/dolt/commands/fsck.go @@ -0,0 +1,122 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package commands + +import ( + "context" + + "github.com/fatih/color" + + "github.com/dolthub/dolt/go/cmd/dolt/cli" + "github.com/dolthub/dolt/go/libraries/doltcore/doltdb" + "github.com/dolthub/dolt/go/libraries/doltcore/env" + "github.com/dolthub/dolt/go/libraries/utils/argparser" +) + +type FsckCmd struct{} + +var _ cli.Command = FsckCmd{} + +func (cmd FsckCmd) Description() string { + return "Verifies the contents of the database are not corrupted." +} + +var fsckDocs = cli.CommandDocumentationContent{ + ShortDesc: "Verifies the contents of the database are not corrupted.", + LongDesc: "Verifies the contents of the database are not corrupted.", + Synopsis: []string{ + "[--quiet]", + }, +} + +func (cmd FsckCmd) Docs() *cli.CommandDocumentation { + return cli.NewCommandDocumentation(fsckDocs, cmd.ArgParser()) +} + +func (cmd FsckCmd) ArgParser() *argparser.ArgParser { + ap := argparser.NewArgParserWithMaxArgs(cmd.Name(), 0) + ap.SupportsFlag(cli.QuietFlag, "", "Don't show progress. Just print final report.") + + return ap +} + +func (cmd FsckCmd) Name() string { + return "fsck" +} + +func (cmd FsckCmd) Exec(ctx context.Context, commandStr string, args []string, dEnv *env.DoltEnv, _ cli.CliContext) int { + ap := cmd.ArgParser() + apr, _, terminate, status := ParseArgsOrPrintHelp(ap, commandStr, args, fsckDocs) + if terminate { + return status + } + + quiet := apr.Contains(cli.QuietFlag) + + progress := make(chan string, 32) + go fsckHandleProgress(ctx, progress, quiet) + + var report *doltdb.FSCKReport + terminate = func() bool { + defer close(progress) + var err error + report, err = dEnv.DoltDB.FSCK(ctx, progress) + if err != nil { + cli.PrintErrln(err.Error()) + return true + } + // skip printing the report is we were cancelled. Most likely we tripped on the error above first. + select { + case <-ctx.Done(): + cli.PrintErrln(ctx.Err().Error()) + return true + default: + return false + } + }() + if terminate { + return 1 + } + + return printFSCKReport(report) +} + +func printFSCKReport(report *doltdb.FSCKReport) int { + cli.Printf("Chunks Scanned: %d\n", report.ChunkCount) + if len(report.Problems) == 0 { + cli.Println("No problems found.") + return 0 + } else { + for _, e := range report.Problems { + cli.Println(color.RedString("------ Corruption Found ------")) + cli.PrintErrln(e.Error()) + } + + return 1 + } +} + +func fsckHandleProgress(ctx context.Context, progress chan string, quiet bool) { + for item := range progress { + if !quiet { + cli.Println(item) + } + select { + case <-ctx.Done(): + return + default: + } + } +} diff --git a/go/cmd/dolt/commands/utils.go b/go/cmd/dolt/commands/utils.go index 5918babd962..db199f9d847 100644 --- a/go/cmd/dolt/commands/utils.go +++ b/go/cmd/dolt/commands/utils.go @@ -805,7 +805,7 @@ func getFastforward(row sql.Row, index int) bool { } func getHashOf(queryist cli.Queryist, sqlCtx *sql.Context, ref string) (string, error) { - q, err := dbr.InterpolateForDialect("select hashof(?)", []interface{}{ref}, dialect.MySQL) + q, err := dbr.InterpolateForDialect("select dolt_hashof(?)", []interface{}{ref}, dialect.MySQL) if err != nil { return "", fmt.Errorf("error interpolating hashof query: %v", err) } diff --git a/go/cmd/dolt/dolt.go b/go/cmd/dolt/dolt.go index 1390473a4a2..e6df8ea6a89 100644 --- a/go/cmd/dolt/dolt.go +++ b/go/cmd/dolt/dolt.go @@ -109,6 +109,7 @@ var doltSubCommands = []cli.Command{ indexcmds.Commands, commands.ReadTablesCmd{}, commands.GarbageCollectionCmd{}, + commands.FsckCmd{}, commands.FilterBranchCmd{}, commands.MergeBaseCmd{}, commands.RootsCmd{}, @@ -151,6 +152,7 @@ var commandsWithoutCliCtx = []cli.Command{ &commands.Assist{}, commands.ProfileCmd{}, commands.ArchiveCmd{}, + commands.FsckCmd{}, } var commandsWithoutGlobalArgSupport = []cli.Command{ diff --git a/go/libraries/doltcore/doltdb/doltdb.go b/go/libraries/doltcore/doltdb/doltdb.go index 375bb45db91..a3dfff801e6 100644 --- a/go/libraries/doltcore/doltdb/doltdb.go +++ b/go/libraries/doltcore/doltdb/doltdb.go @@ -15,6 +15,7 @@ package doltdb import ( + "bytes" "context" "errors" "fmt" @@ -22,6 +23,8 @@ import ( "os" "path/filepath" "strings" + "sync" + "sync/atomic" "time" "github.com/dolthub/go-mysql-server/sql" @@ -2045,3 +2048,114 @@ func (ddb *DoltDB) GetStashRootAndHeadCommitAtIdx(ctx context.Context, idx int) func (ddb *DoltDB) PersistGhostCommits(ctx context.Context, ghostCommits hash.HashSet) error { return ddb.db.Database.PersistGhostCommitIDs(ctx, ghostCommits) } + +type FSCKReport struct { + ChunkCount uint32 + Problems []error +} + +// FSCK performs a full file system check on the database. This is currently exposed with the CLI as `dolt fsck` +// The success of failure of the scan are returned in the report as a list of errors. The error returned by this function +// indicates a deeper issue such as having database in an old format. +func (ddb *DoltDB) FSCK(ctx context.Context, progress chan string) (*FSCKReport, error) { + cs := datas.ChunkStoreFromDatabase(ddb.db) + + vs := types.NewValueStore(cs) + + gs, ok := cs.(*nbs.GenerationalNBS) + if !ok { + return nil, errors.New("FSCK requires a local database") + } + + chunkCount, err := gs.OldGen().Count() + if err != nil { + return nil, err + } + chunkCount2, err := gs.NewGen().Count() + if err != nil { + return nil, err + } + chunkCount += chunkCount2 + proccessedCnt := int64(0) + + var errs []error + + decodeMsg := func(chk chunks.Chunk) string { + hrs := "" + val, err := types.DecodeValue(chk, vs) + if err == nil { + hrs = val.HumanReadableString() + } else { + hrs = fmt.Sprintf("Unable to decode value: %s", err.Error()) + } + return hrs + } + + // Append safely to the slice of errors with a mutex. + errsLock := &sync.Mutex{} + appendErr := func(err error) { + errsLock.Lock() + defer errsLock.Unlock() + errs = append(errs, err) + } + + // Callback for validating chunks. This code could be called concurrently, though that is not currently the case. + validationCallback := func(chunk chunks.Chunk) { + chunkOk := true + pCnt := atomic.AddInt64(&proccessedCnt, 1) + h := chunk.Hash() + raw := chunk.Data() + calcChkSum := hash.Of(raw) + + if h != calcChkSum { + fuzzyMatch := false + // Special case for the journal chunk source. We may have an address which has 4 null bytes at the end. + if h[hash.ByteLen-1] == 0 && h[hash.ByteLen-2] == 0 && h[hash.ByteLen-3] == 0 && h[hash.ByteLen-4] == 0 { + // Now we'll just verify that the first 16 bytes match. + ln := hash.ByteLen - 4 + fuzzyMatch = bytes.Compare(h[:ln], calcChkSum[:ln]) == 0 + } + if !fuzzyMatch { + hrs := decodeMsg(chunk) + appendErr(errors.New(fmt.Sprintf("Chunk: %s content hash mismatch: %s\n%s", h.String(), calcChkSum.String(), hrs))) + chunkOk = false + } + } + + if chunkOk { + // Round trip validation. Ensure that the top level store returns the same data. + c, err := cs.Get(ctx, h) + if err != nil { + appendErr(errors.New(fmt.Sprintf("Chunk: %s load failed with error: %s", h.String(), err.Error()))) + chunkOk = false + } else if bytes.Compare(raw, c.Data()) != 0 { + hrs := decodeMsg(chunk) + appendErr(errors.New(fmt.Sprintf("Chunk: %s read with incorrect ID: %s\n%s", h.String(), c.Hash().String(), hrs))) + chunkOk = false + } + } + + percentage := (float64(pCnt) * 100) / float64(chunkCount) + result := fmt.Sprintf("(%4.1f%% done)", percentage) + + progStr := "OK: " + h.String() + if !chunkOk { + progStr = "FAIL: " + h.String() + } + progStr = result + " " + progStr + progress <- progStr + } + + err = gs.OldGen().IterateAllChunks(ctx, validationCallback) + if err != nil { + return nil, err + } + err = gs.NewGen().IterateAllChunks(ctx, validationCallback) + if err != nil { + return nil, err + } + + FSCKReport := FSCKReport{Problems: errs, ChunkCount: chunkCount} + + return &FSCKReport, nil +} diff --git a/go/store/chunks/chunk_store.go b/go/store/chunks/chunk_store.go index c07a8ba6ece..e1bed5c74c9 100644 --- a/go/store/chunks/chunk_store.go +++ b/go/store/chunks/chunk_store.go @@ -186,6 +186,13 @@ type ChunkStoreGarbageCollector interface { // interactions with generational stores. See ValueStore and // NomsBlockStore/GenerationalNBS for details. MarkAndSweepChunks(ctx context.Context, hashes <-chan []hash.Hash, dest ChunkStore) error + + // Count returns the number of chunks in the store. + Count() (uint32, error) + + // IterateAllChunks iterates over all chunks in the store, calling the provided callback for each chunk. This is + // a wrapper over the internal chunkSource.iterateAllChunks() method. + IterateAllChunks(context.Context, func(chunk Chunk)) error } type PrefixChunkStore interface { diff --git a/go/store/chunks/memory_store.go b/go/store/chunks/memory_store.go index 2474c4df7bb..485eafea4ba 100644 --- a/go/store/chunks/memory_store.go +++ b/go/store/chunks/memory_store.go @@ -382,6 +382,14 @@ LOOP: return nil } +func (ms *MemoryStoreView) Count() (uint32, error) { + return uint32(len(ms.pending)), nil +} + +func (ms *MemoryStoreView) IterateAllChunks(_ context.Context, _ func(Chunk)) error { + panic("runtime error: GetChunkHashes should never be called on the MemoryStoreView") +} + func (ms *MemoryStoreView) Stats() interface{} { return nil } diff --git a/go/store/chunks/test_utils.go b/go/store/chunks/test_utils.go index 51426680d3e..037210f3c7c 100644 --- a/go/store/chunks/test_utils.go +++ b/go/store/chunks/test_utils.go @@ -99,6 +99,14 @@ func (s *TestStoreView) MarkAndSweepChunks(ctx context.Context, hashes <-chan [] return collector.MarkAndSweepChunks(ctx, hashes, collector) } +func (s *TestStoreView) Count() (uint32, error) { + panic("currently unused") +} + +func (s *TestStoreView) IterateAllChunks(_ context.Context, _ func(Chunk)) error { + panic("currently unused") +} + func (s *TestStoreView) Reads() int { reads := atomic.LoadInt32(&s.reads) return int(reads) diff --git a/go/store/nbs/archive_chunk_source.go b/go/store/nbs/archive_chunk_source.go index ee12bc44b99..41221a1aff2 100644 --- a/go/store/nbs/archive_chunk_source.go +++ b/go/store/nbs/archive_chunk_source.go @@ -16,6 +16,7 @@ package nbs import ( "context" + "encoding/binary" "io" "os" "path/filepath" @@ -152,3 +153,27 @@ func (acs archiveChunkSource) getRecordRanges(_ context.Context, _ []getRecord) func (acs archiveChunkSource) getManyCompressed(ctx context.Context, eg *errgroup.Group, reqs []getRecord, found func(context.Context, CompressedChunk), stats *Stats) (bool, error) { return false, errors.New("Archive chunk source does not support getManyCompressed") } + +func (acs archiveChunkSource) iterateAllChunks(ctx context.Context, cb func(chunks.Chunk)) error { + addrCount := uint32(len(acs.aRdr.prefixes)) + for i := uint32(0); i < addrCount; i++ { + var h hash.Hash + suffix := acs.aRdr.getSuffixByID(i) + + // Reconstruct the hash from the prefix and suffix. + binary.BigEndian.PutUint64(h[:uint64Size], acs.aRdr.prefixes[i]) + copy(h[uint64Size:], suffix[:]) + + if ctx.Err() != nil { + return ctx.Err() + } + + data, err := acs.aRdr.get(h) + if err != nil { + return err + } + + cb(chunks.NewChunkWithHash(h, data)) + } + return nil +} diff --git a/go/store/nbs/archive_test.go b/go/store/nbs/archive_test.go index 3dbc646950a..da3c324cf2a 100644 --- a/go/store/nbs/archive_test.go +++ b/go/store/nbs/archive_test.go @@ -715,3 +715,7 @@ func (tcs *testChunkSource) clone() (chunkSource, error) { func (tcs *testChunkSource) currentSize() uint64 { panic("never used") } + +func (tcs *testChunkSource) iterateAllChunks(_ context.Context, _ func(chunks.Chunk)) error { + panic("never used") +} diff --git a/go/store/nbs/chunk_source_adapter.go b/go/store/nbs/chunk_source_adapter.go index f86d7527b43..db1eb72fd65 100644 --- a/go/store/nbs/chunk_source_adapter.go +++ b/go/store/nbs/chunk_source_adapter.go @@ -17,6 +17,7 @@ package nbs import ( "context" + "github.com/dolthub/dolt/go/store/chunks" "github.com/dolthub/dolt/go/store/hash" ) @@ -53,3 +54,7 @@ func (csa chunkSourceAdapter) clone() (chunkSource, error) { } return &chunkSourceAdapter{tr, csa.h}, nil } + +func (csa chunkSourceAdapter) iterateAllChunks(ctx context.Context, cb func(chunks.Chunk)) error { + panic("unimplemented") +} diff --git a/go/store/nbs/empty_chunk_source.go b/go/store/nbs/empty_chunk_source.go index c8f094f98ac..5df2696c33d 100644 --- a/go/store/nbs/empty_chunk_source.go +++ b/go/store/nbs/empty_chunk_source.go @@ -93,3 +93,7 @@ func (ecs emptyChunkSource) close() error { func (ecs emptyChunkSource) clone() (chunkSource, error) { return ecs, nil } + +func (ecs emptyChunkSource) iterateAllChunks(_ context.Context, _ func(chunks.Chunk)) error { + return nil +} diff --git a/go/store/nbs/file_table_reader.go b/go/store/nbs/file_table_reader.go index a95c2109c32..873450efd2b 100644 --- a/go/store/nbs/file_table_reader.go +++ b/go/store/nbs/file_table_reader.go @@ -30,6 +30,7 @@ import ( "path/filepath" "time" + "github.com/dolthub/dolt/go/store/chunks" "github.com/dolthub/dolt/go/store/hash" ) @@ -169,6 +170,49 @@ func (ftr *fileTableReader) hash() hash.Hash { return ftr.h } +func (ftr *fileTableReader) iterateAllChunks(ctx context.Context, cb func(chunk chunks.Chunk)) error { + count := ftr.idx.chunkCount() + + rdr, err := ftr.tableReader.r.Reader(ctx) + if err != nil { + return err + } + + skr, ok := rdr.(io.ReadSeeker) + if !ok { + return errors.New("runtime error: reader does not support seeking") + } + + for i := uint32(0); i < count; i++ { + if ctx.Err() != nil { + return ctx.Err() + } + + var h hash.Hash + ie, err := ftr.idx.indexEntry(i, &h) + if err != nil { + return err + } + + readNBytes, err := readNFrom(skr, ie.Offset(), ie.Length()) + if err != nil { + return err + } + + cchk, err := NewCompressedChunk(h, readNBytes) + if err != nil { + return err + } + chk, err := cchk.ToChunk() + if err != nil { + return err + } + + cb(chk) + } + return nil +} + func (ftr *fileTableReader) Close() error { return ftr.tableReader.close() } diff --git a/go/store/nbs/journal_chunk_source.go b/go/store/nbs/journal_chunk_source.go index 0b94b13e1b4..c8dd8a4ac02 100644 --- a/go/store/nbs/journal_chunk_source.go +++ b/go/store/nbs/journal_chunk_source.go @@ -209,6 +209,68 @@ func (s journalChunkSource) close() error { return nil } +func (s journalChunkSource) iterateAllChunks(ctx context.Context, cb func(chunks.Chunk)) error { + var finalErr error + + // TODO - a less time consuming lock is possible here. Using s.journal.snapshot and processJournalRecords() + // would allow for no locking. Need to filter out the journal records which are actually chunks, then convert + // those to chunks and pass them to cb. When we support online FSCK this will allow the server to keep running uninterrupted. + s.journal.lock.RLock() + defer s.journal.lock.RUnlock() + + s.journal.ranges.novel.Iter(func(h hash.Hash, r Range) (stop bool) { + if ctx.Err() != nil { + finalErr = ctx.Err() + return true + } + + cchk, err := s.journal.getCompressedChunkAtRange(r, h) + if err != nil { + finalErr = err + return true + } + + chunk, err := cchk.ToChunk() + if err != nil { + finalErr = err + return true + } + cb(chunk) + return false + }) + + if finalErr != nil { + return finalErr + } + + s.journal.ranges.cached.Iter(func(a16 addr16, r Range) (stop bool) { + if ctx.Err() != nil { + finalErr = ctx.Err() + return true + } + + // We only have 16 bytes of the hash. The value returned here will have 4 0x00 bytes at the end. + var h hash.Hash + copy(h[:], a16[:]) + + cchk, err := s.journal.getCompressedChunkAtRange(r, h) + if err != nil { + finalErr = err + return true + } + + chunk, err := cchk.ToChunk() + if err != nil { + finalErr = err + return true + } + cb(chunk) + return false + }) + + return finalErr +} + func equalSpecs(left, right []tableSpec) bool { if len(left) != len(right) { return false diff --git a/go/store/nbs/journal_record.go b/go/store/nbs/journal_record.go index 9f79e79ff59..77afb7f0466 100644 --- a/go/store/nbs/journal_record.go +++ b/go/store/nbs/journal_record.go @@ -255,6 +255,7 @@ func processJournalRecords(ctx context.Context, r io.ReadSeeker, off int64, cb f } if !validateJournalRecord(buf) { + // We read the journal file until we run into an invalid record. break // stop if we can't validate |rec| } diff --git a/go/store/nbs/nbs_metrics_wrapper.go b/go/store/nbs/nbs_metrics_wrapper.go index b8609584499..78b7a75ec78 100644 --- a/go/store/nbs/nbs_metrics_wrapper.go +++ b/go/store/nbs/nbs_metrics_wrapper.go @@ -83,6 +83,14 @@ func (nbsMW *NBSMetricWrapper) MarkAndSweepChunks(ctx context.Context, hashes <- return nbsMW.nbs.MarkAndSweepChunks(ctx, hashes, dest) } +func (nbsMW *NBSMetricWrapper) Count() (uint32, error) { + return nbsMW.nbs.Count() +} + +func (nbsMW *NBSMetricWrapper) IterateAllChunks(ctx context.Context, cb func(chunk chunks.Chunk)) error { + return nbsMW.nbs.IterateAllChunks(ctx, cb) +} + // PruneTableFiles deletes old table files that are no longer referenced in the manifest. func (nbsMW *NBSMetricWrapper) PruneTableFiles(ctx context.Context) error { return nbsMW.nbs.PruneTableFiles(ctx) diff --git a/go/store/nbs/store.go b/go/store/nbs/store.go index 2d377afbd77..e17f1e847a8 100644 --- a/go/store/nbs/store.go +++ b/go/store/nbs/store.go @@ -1667,6 +1667,28 @@ LOOP: return gcc.copyTablesToDir(ctx, tfp) } +func (nbs *NomsBlockStore) IterateAllChunks(ctx context.Context, cb func(chunk chunks.Chunk)) error { + for _, v := range nbs.tables.novel { + err := v.iterateAllChunks(ctx, cb) + if err != nil { + return err + } + if ctx.Err() != nil { + return ctx.Err() + } + } + for _, v := range nbs.tables.upstream { + err := v.iterateAllChunks(ctx, cb) + if err != nil { + return err + } + if ctx.Err() != nil { + return ctx.Err() + } + } + return nil +} + func (nbs *NomsBlockStore) swapTables(ctx context.Context, specs []tableSpec) (err error) { nbs.mu.Lock() defer nbs.mu.Unlock() diff --git a/go/store/nbs/table.go b/go/store/nbs/table.go index b1037dc5485..b487422d079 100644 --- a/go/store/nbs/table.go +++ b/go/store/nbs/table.go @@ -240,6 +240,15 @@ type chunkSource interface { // currentSize returns the current total physical size of the chunkSource. currentSize() uint64 + + // scanAllChunks will call the provided function for each chunk in the chunkSource. This is currently used + // to perform integrity checks, and the chunk passed in will have the address from the index, and the content + // loaded. This iterator doesn't have a way to stop the iteration other than the context being canceled. + // + // If there is a failure reading the chunk, the error will be returned - note that this can happen in the middle of + // the scan, and will likely mean that the scan didn't complete. Note that errors returned by this method are not + // related to the callback - if the callback discovers an error, it must manage that out of band. + iterateAllChunks(context.Context, func(chunk chunks.Chunk)) error } type chunkSources []chunkSource diff --git a/integration-tests/bats/corrupt_dbs/README.md b/integration-tests/bats/corrupt_dbs/README.md new file mode 100644 index 00000000000..0dba6325500 --- /dev/null +++ b/integration-tests/bats/corrupt_dbs/README.md @@ -0,0 +1,14 @@ + +This directory contains a set of corrupt databases that are used to test the behavior of fsck, +and perhaps other tools in the future. Please catalog the contents of each database here. +(.dolt/* -> test-dir) + +## bad_commit + This database contains a commit (rlmgv0komq0oj7qu4osdo759vs4c5pvg) that has contents in the database + that do not have contents which matches the address (Actual data in table file: gpphmuvegiedtjtbfku4ru8jalfdk21u). + This represents updating the author of that commit maliciously. + +## bad_journal_crc + This database contains a journal file which has been altered to have an object (7i48kt4h41hcjniri7scv5m8a69cdn13) + which has a bad CRC. The object in question doesn't trip any problems until it's fully loaded, which indicates + that the CRC for the journal record is correct, but the data is not. diff --git a/integration-tests/bats/corrupt_dbs/bad_commit/noms/9tj851gpv71861ln5k6uutfot66u79r4 b/integration-tests/bats/corrupt_dbs/bad_commit/noms/9tj851gpv71861ln5k6uutfot66u79r4 new file mode 100644 index 00000000000..718fa6ea205 Binary files /dev/null and b/integration-tests/bats/corrupt_dbs/bad_commit/noms/9tj851gpv71861ln5k6uutfot66u79r4 differ diff --git a/integration-tests/bats/corrupt_dbs/bad_commit/noms/LOCK b/integration-tests/bats/corrupt_dbs/bad_commit/noms/LOCK new file mode 100644 index 00000000000..e69de29bb2d diff --git a/integration-tests/bats/corrupt_dbs/bad_commit/noms/manifest b/integration-tests/bats/corrupt_dbs/bad_commit/noms/manifest new file mode 100644 index 00000000000..1c0ec6ca405 --- /dev/null +++ b/integration-tests/bats/corrupt_dbs/bad_commit/noms/manifest @@ -0,0 +1 @@ +5:__DOLT__:pfculecuhmg18r2v4cf52p6lnp9r4rrk:vq4ej4fr5vhmrmbmc7f4bsh7ieg7rro8:pfculecuhmg18r2v4cf52p6lnp9r4rrk:9tj851gpv71861ln5k6uutfot66u79r4:2 \ No newline at end of file diff --git a/integration-tests/bats/corrupt_dbs/bad_commit/noms/oldgen/LOCK b/integration-tests/bats/corrupt_dbs/bad_commit/noms/oldgen/LOCK new file mode 100644 index 00000000000..e69de29bb2d diff --git a/integration-tests/bats/corrupt_dbs/bad_commit/noms/oldgen/aa3a08lf6sbtq9cl58mledifrqvt3fid b/integration-tests/bats/corrupt_dbs/bad_commit/noms/oldgen/aa3a08lf6sbtq9cl58mledifrqvt3fid new file mode 100644 index 00000000000..b8729f1fd31 Binary files /dev/null and b/integration-tests/bats/corrupt_dbs/bad_commit/noms/oldgen/aa3a08lf6sbtq9cl58mledifrqvt3fid differ diff --git a/integration-tests/bats/corrupt_dbs/bad_commit/noms/oldgen/manifest b/integration-tests/bats/corrupt_dbs/bad_commit/noms/oldgen/manifest new file mode 100644 index 00000000000..8abc4506fbe --- /dev/null +++ b/integration-tests/bats/corrupt_dbs/bad_commit/noms/oldgen/manifest @@ -0,0 +1 @@ +5:__DOLT__:qld52734cm7e9efc91v55nb7601pkid5:00000000000000000000000000000000:00000000000000000000000000000000:aa3a08lf6sbtq9cl58mledifrqvt3fid:15 \ No newline at end of file diff --git a/integration-tests/bats/corrupt_dbs/bad_commit/repo_state.json b/integration-tests/bats/corrupt_dbs/bad_commit/repo_state.json new file mode 100755 index 00000000000..32f2d6197d5 --- /dev/null +++ b/integration-tests/bats/corrupt_dbs/bad_commit/repo_state.json @@ -0,0 +1,6 @@ +{ + "head": "refs/heads/main", + "remotes": {}, + "backups": {}, + "branches": {} +} \ No newline at end of file diff --git a/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/LOCK b/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/LOCK new file mode 100644 index 00000000000..e69de29bb2d diff --git a/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/journal.idx b/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/journal.idx new file mode 100644 index 00000000000..5b89738b38c Binary files /dev/null and b/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/journal.idx differ diff --git a/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/manifest b/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/manifest new file mode 100644 index 00000000000..5524e868582 --- /dev/null +++ b/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/manifest @@ -0,0 +1 @@ +5:__DOLT__:stsmuqfqkisdfaktc56rf7m4vu8dju09:ar19erb29csnfn2d3klbk6ic08q421ng:00000000000000000000000000000000:vvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvv:59 \ No newline at end of file diff --git a/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/vvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvv b/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/vvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvv new file mode 100644 index 00000000000..d754c3d414b Binary files /dev/null and b/integration-tests/bats/corrupt_dbs/bad_journal_crc/noms/vvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvv differ diff --git a/integration-tests/bats/corrupt_dbs/bad_journal_crc/repo_state.json b/integration-tests/bats/corrupt_dbs/bad_journal_crc/repo_state.json new file mode 100755 index 00000000000..32f2d6197d5 --- /dev/null +++ b/integration-tests/bats/corrupt_dbs/bad_journal_crc/repo_state.json @@ -0,0 +1,6 @@ +{ + "head": "refs/heads/main", + "remotes": {}, + "backups": {}, + "branches": {} +} \ No newline at end of file diff --git a/integration-tests/bats/fsck.bats b/integration-tests/bats/fsck.bats new file mode 100644 index 00000000000..af90c55edc4 --- /dev/null +++ b/integration-tests/bats/fsck.bats @@ -0,0 +1,82 @@ +#! /usr/bin/env bats +load $BATS_TEST_DIRNAME/helper/common.bash + +setup() { + setup_no_dolt_init +} + +teardown() { + teardown_common +} + +make_inserts() { + for ((i=1; i<=25; i++)) + do + dolt sql -q "INSERT INTO tbl (guid) VALUES (UUID())" + done + dolt commit -a -m "Add 25 values" +} + +# Helper function to create enough chunks to allow archive to be created. Duplicate in archive.bats. +make_updates() { + for ((i=1; i<=10; i++)) + do + dolt sql -q " + SET @max_id = (SELECT MAX(i) FROM tbl); + SET @random_id = FLOOR(1 + RAND() * @max_id); + UPDATE tbl SET guid = UUID() WHERE i >= @random_id LIMIT 1;" + done + dolt commit -a -m "Update 10 values." +} + +@test "fsck: bad commit" { + mkdir .dolt + cp -R $BATS_CWD/corrupt_dbs/bad_commit/* .dolt/ + + # validate that cp worked. + dolt status + + run dolt fsck + [ "$status" -eq 1 ] + [[ "$output" =~ "Chunk: rlmgv0komq0oj7qu4osdo759vs4c5pvg content hash mismatch: gpphmuvegiedtjtbfku4ru8jalfdk21u" ]] || false + [[ "$output" =~ "hacky@hackypants.com" ]] || false +} + +# This test runs over 45 seconds, resulting in a timeout in lambdabats +# bats test_tags=no_lambda +@test "fsck: good archive" { + dolt init + dolt sql -q "create table tbl (i int auto_increment primary key, guid char(36))" + dolt commit -A -m "create tbl" + + for ((j=1; j<=10; j++)) + do + make_inserts + make_updates + done + + dolt gc + dolt archive + + dolt fsck +} + +@test "fsck: good journal" { + dolt init + dolt sql -q "create table tbl (i int auto_increment primary key, guid char(36))" + dolt commit -Am "Create table tbl" + + make_inserts + + # Objects are in the journal. Don't gc. + dolt fsck +} + +@test "fsck: bad journal crc" { + mkdir .dolt + cp -R $BATS_CWD/corrupt_dbs/bad_journal_crc/* .dolt/ + + run dolt fsck + [ "$status" -eq 1 ] + [[ "$output" =~ "Chunk: 7i48kt4h41hcjniri7scv5m8a69cdn13 content hash mismatch: hitg0bb0hsakip96qvu2hts0hkrrla9o" ]] || false +} diff --git a/integration-tests/bats/helper/local-remote.bash b/integration-tests/bats/helper/local-remote.bash index 71eedb97f20..0de595a44f0 100644 --- a/integration-tests/bats/helper/local-remote.bash +++ b/integration-tests/bats/helper/local-remote.bash @@ -139,6 +139,7 @@ SKIP_SERVER_TESTS=$(cat <<-EOM ~rebase.bats~ ~shallow-clone.bats~ ~archive.bats~ +~fsck.bats~ EOM )