mirror of https://github.com/etcd-io/bbolt.git
parent
ee27a544ca
commit
f16e2522ce
cmd/bbolt
internal
btesting
tests
|
@ -207,7 +207,7 @@ func (cmd *CheckCommand) Run(args ...string) error {
|
|||
// Perform consistency check.
|
||||
return db.View(func(tx *bolt.Tx) error {
|
||||
var count int
|
||||
for err := range tx.Check(CmdKeyValueStringer()) {
|
||||
for err := range tx.Check(CmdKvStringer()) {
|
||||
fmt.Fprintln(cmd.Stdout, err)
|
||||
count++
|
||||
}
|
||||
|
@ -540,11 +540,7 @@ func formatBytes(b []byte, format string) (string, error) {
|
|||
case "bytes":
|
||||
return string(b), nil
|
||||
case "auto":
|
||||
if isPrintable(string(b)) {
|
||||
return string(b), nil
|
||||
} else {
|
||||
return fmt.Sprintf("%x", b), nil
|
||||
}
|
||||
return bytesToAsciiOrHex(b), nil
|
||||
case "redacted":
|
||||
return fmt.Sprintf("<redacted len:%d>", len(b)), nil
|
||||
default:
|
||||
|
@ -1573,6 +1569,15 @@ func isPrintable(s string) bool {
|
|||
return true
|
||||
}
|
||||
|
||||
func bytesToAsciiOrHex(b []byte) string {
|
||||
sb := string(b)
|
||||
if isPrintable(sb) {
|
||||
return sb
|
||||
} else {
|
||||
return hex.EncodeToString(b)
|
||||
}
|
||||
}
|
||||
|
||||
func stringToPage(str string) (uint64, error) {
|
||||
return strconv.ParseUint(str, 10, 64)
|
||||
}
|
||||
|
@ -1690,24 +1695,16 @@ Additional options include:
|
|||
`, "\n")
|
||||
}
|
||||
|
||||
type cmdKeyValueStringer struct{}
|
||||
type cmdKvStringer struct{}
|
||||
|
||||
func (_ cmdKeyValueStringer) KeyToString(key []byte) string {
|
||||
if isPrintable(string(key)) {
|
||||
return string(key)
|
||||
} else {
|
||||
return hex.EncodeToString(key)
|
||||
}
|
||||
func (_ cmdKvStringer) KeyToString(key []byte) string {
|
||||
return bytesToAsciiOrHex(key)
|
||||
}
|
||||
|
||||
func (_ cmdKeyValueStringer) ValueToString(value []byte) string {
|
||||
if isPrintable(string(value)) {
|
||||
return string(value)
|
||||
} else {
|
||||
return hex.EncodeToString(value)
|
||||
}
|
||||
func (_ cmdKvStringer) ValueToString(value []byte) string {
|
||||
return bytesToAsciiOrHex(value)
|
||||
}
|
||||
|
||||
func CmdKeyValueStringer() bolt.KeyValueStringer {
|
||||
return cmdKeyValueStringer{}
|
||||
func CmdKvStringer() bolt.KVStringer {
|
||||
return cmdKvStringer{}
|
||||
}
|
||||
|
|
2
db.go
2
db.go
|
@ -1148,7 +1148,7 @@ func (db *DB) freepages() []pgid {
|
|||
panic(fmt.Sprintf("freepages: failed to get all reachable pages (%v)", e))
|
||||
}
|
||||
}()
|
||||
tx.checkBucket(&tx.root, reachable, nofreed, HexKeyValueStringer(), ech)
|
||||
tx.checkBucket(&tx.root, reachable, nofreed, HexKVStringer(), ech)
|
||||
close(ech)
|
||||
|
||||
// TODO: If check bucket reported any corruptions (ech) we shouldn't proceed to freeing the pages.
|
||||
|
|
|
@ -396,7 +396,7 @@ func TestOpen_Check(t *testing.T) {
|
|||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = db.View(func(tx *bolt.Tx) error { return <-tx.Check(bolt.HexKeyValueStringer()) }); err != nil {
|
||||
if err = db.View(func(tx *bolt.Tx) error { return <-tx.Check(bolt.HexKVStringer()) }); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = db.Close(); err != nil {
|
||||
|
@ -407,7 +407,7 @@ func TestOpen_Check(t *testing.T) {
|
|||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := db.View(func(tx *bolt.Tx) error { return <-tx.Check(bolt.HexKeyValueStringer()) }); err != nil {
|
||||
if err := db.View(func(tx *bolt.Tx) error { return <-tx.Check(bolt.HexKVStringer()) }); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := db.Close(); err != nil {
|
||||
|
|
|
@ -119,7 +119,7 @@ func (db *DB) MustCheck() {
|
|||
err := db.Update(func(tx *bolt.Tx) error {
|
||||
// Collect all the errors.
|
||||
var errors []error
|
||||
for err := range tx.Check(bolt.HexKeyValueStringer()) {
|
||||
for err := range tx.Check(bolt.HexKVStringer()) {
|
||||
errors = append(errors, err)
|
||||
if len(errors) > 10 {
|
||||
break
|
||||
|
|
|
@ -2,10 +2,9 @@ package tests_test
|
|||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/stretchr/testify/require"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
bolt "go.etcd.io/bbolt"
|
||||
"go.etcd.io/bbolt/internal/btesting"
|
||||
"go.etcd.io/bbolt/internal/guts_cli"
|
||||
|
@ -14,74 +13,74 @@ import (
|
|||
|
||||
func TestTx_RecursivelyCheckPages_MisplacedPage(t *testing.T) {
|
||||
db := btesting.MustCreateDB(t)
|
||||
assert.NoError(t,
|
||||
require.NoError(t,
|
||||
db.Fill([]byte("data"), 1, 10000,
|
||||
func(tx int, k int) []byte { return []byte(fmt.Sprintf("%04d", k)) },
|
||||
func(tx int, k int) []byte { return make([]byte, 100) },
|
||||
))
|
||||
assert.NoError(t, db.Close())
|
||||
require.NoError(t, db.Close())
|
||||
|
||||
xRay := surgeon.NewXRay(db.Path())
|
||||
|
||||
path1, err := xRay.FindPathsToKey([]byte("0451"))
|
||||
assert.NoError(t, err, "Cannot find page that contains key:'0451'")
|
||||
assert.Len(t, path1, 1, "Expected only one page that contains key:'0451'")
|
||||
require.NoError(t, err, "Cannot find page that contains key:'0451'")
|
||||
require.Len(t, path1, 1, "Expected only one page that contains key:'0451'")
|
||||
|
||||
path2, err := xRay.FindPathsToKey([]byte("7563"))
|
||||
assert.NoError(t, err, "Cannot find page that contains key:'7563'")
|
||||
assert.Len(t, path2, 1, "Expected only one page that contains key:'7563'")
|
||||
require.NoError(t, err, "Cannot find page that contains key:'7563'")
|
||||
require.Len(t, path2, 1, "Expected only one page that contains key:'7563'")
|
||||
|
||||
srcPage := path1[0][len(path1[0])-1]
|
||||
targetPage := path2[0][len(path2[0])-1]
|
||||
assert.NoError(t, surgeon.CopyPage(db.Path(), srcPage, targetPage))
|
||||
require.NoError(t, surgeon.CopyPage(db.Path(), srcPage, targetPage))
|
||||
|
||||
db.MustReopen()
|
||||
assert.NoError(t, db.Update(func(tx *bolt.Tx) error {
|
||||
require.NoError(t, db.Update(func(tx *bolt.Tx) error {
|
||||
// Collect all the errors.
|
||||
var errors []error
|
||||
for err := range tx.Check(bolt.HexKeyValueStringer()) {
|
||||
for err := range tx.Check(bolt.HexKVStringer()) {
|
||||
errors = append(errors, err)
|
||||
}
|
||||
assert.Len(t, errors, 1)
|
||||
assert.ErrorContains(t, errors[0], fmt.Sprintf("leaf page(%v) needs to be >= to the key in the ancestor", targetPage))
|
||||
require.Len(t, errors, 1)
|
||||
require.ErrorContains(t, errors[0], fmt.Sprintf("leaf page(%v) needs to be >= the key in the ancestor", targetPage))
|
||||
return nil
|
||||
}))
|
||||
assert.NoError(t, db.Close())
|
||||
require.NoError(t, db.Close())
|
||||
}
|
||||
|
||||
func TestTx_RecursivelyCheckPages_CorruptedLeaf(t *testing.T) {
|
||||
db := btesting.MustCreateDB(t)
|
||||
assert.NoError(t,
|
||||
require.NoError(t,
|
||||
db.Fill([]byte("data"), 1, 10000,
|
||||
func(tx int, k int) []byte { return []byte(fmt.Sprintf("%04d", k)) },
|
||||
func(tx int, k int) []byte { return make([]byte, 100) },
|
||||
))
|
||||
assert.NoError(t, db.Close())
|
||||
require.NoError(t, db.Close())
|
||||
|
||||
xray := surgeon.NewXRay(db.Path())
|
||||
|
||||
path1, err := xray.FindPathsToKey([]byte("0451"))
|
||||
assert.NoError(t, err, "Cannot find page that contains key:'0451'")
|
||||
assert.Len(t, path1, 1, "Expected only one page that contains key:'0451'")
|
||||
require.NoError(t, err, "Cannot find page that contains key:'0451'")
|
||||
require.Len(t, path1, 1, "Expected only one page that contains key:'0451'")
|
||||
|
||||
srcPage := path1[0][len(path1[0])-1]
|
||||
p, pbuf, err := guts_cli.ReadPage(db.Path(), uint64(srcPage))
|
||||
assert.NoError(t, err)
|
||||
assert.Positive(t, p.Count(), "page must be not empty")
|
||||
require.NoError(t, err)
|
||||
require.Positive(t, p.Count(), "page must be not empty")
|
||||
p.LeafPageElement(p.Count() / 2).Key()[0] = 'z'
|
||||
assert.NoError(t, surgeon.WritePage(db.Path(), pbuf))
|
||||
require.NoError(t, surgeon.WritePage(db.Path(), pbuf))
|
||||
|
||||
db.MustReopen()
|
||||
assert.NoError(t, db.Update(func(tx *bolt.Tx) error {
|
||||
require.NoError(t, db.Update(func(tx *bolt.Tx) error {
|
||||
// Collect all the errors.
|
||||
var errors []error
|
||||
for err := range tx.Check(bolt.HexKeyValueStringer()) {
|
||||
for err := range tx.Check(bolt.HexKVStringer()) {
|
||||
errors = append(errors, err)
|
||||
}
|
||||
assert.Len(t, errors, 2)
|
||||
assert.ErrorContains(t, errors[0], fmt.Sprintf("leaf page(%v) needs to be < than key of the next element in ancestor", srcPage))
|
||||
assert.ErrorContains(t, errors[1], fmt.Sprintf("leaf page(%v) needs to be > (found <) than previous element", srcPage))
|
||||
require.Len(t, errors, 2)
|
||||
require.ErrorContains(t, errors[0], fmt.Sprintf("leaf page(%v) needs to be < than key of the next element in ancestor", srcPage))
|
||||
require.ErrorContains(t, errors[1], fmt.Sprintf("leaf page(%v) needs to be > (found <) than previous element", srcPage))
|
||||
return nil
|
||||
}))
|
||||
assert.NoError(t, db.Close())
|
||||
require.NoError(t, db.Close())
|
||||
}
|
||||
|
|
2
tx.go
2
tx.go
|
@ -190,7 +190,7 @@ func (tx *Tx) Commit() error {
|
|||
|
||||
// If strict mode is enabled then perform a consistency check.
|
||||
if tx.db.StrictMode {
|
||||
ch := tx.Check(HexKeyValueStringer())
|
||||
ch := tx.Check(HexKVStringer())
|
||||
var errs []string
|
||||
for {
|
||||
err, ok := <-ch
|
||||
|
|
57
tx_check.go
57
tx_check.go
|
@ -13,13 +13,13 @@ import (
|
|||
// because of caching. This overhead can be removed if running on a read-only
|
||||
// transaction, however, it is not safe to execute other writer transactions at
|
||||
// the same time.
|
||||
func (tx *Tx) Check(keyValueStringer KeyValueStringer) <-chan error {
|
||||
func (tx *Tx) Check(kvStringer KVStringer) <-chan error {
|
||||
ch := make(chan error)
|
||||
go tx.check(keyValueStringer, ch)
|
||||
go tx.check(kvStringer, ch)
|
||||
return ch
|
||||
}
|
||||
|
||||
func (tx *Tx) check(keyValueStringer KeyValueStringer, ch chan error) {
|
||||
func (tx *Tx) check(kvStringer KVStringer, ch chan error) {
|
||||
// Force loading free list if opened in ReadOnly mode.
|
||||
tx.db.loadFreelist()
|
||||
|
||||
|
@ -45,7 +45,7 @@ func (tx *Tx) check(keyValueStringer KeyValueStringer, ch chan error) {
|
|||
}
|
||||
|
||||
// Recursively check buckets.
|
||||
tx.checkBucket(&tx.root, reachable, freed, keyValueStringer, ch)
|
||||
tx.checkBucket(&tx.root, reachable, freed, kvStringer, ch)
|
||||
|
||||
// Ensure all pages below high water mark are either reachable or freed.
|
||||
for i := pgid(0); i < tx.meta.pgid; i++ {
|
||||
|
@ -60,7 +60,7 @@ func (tx *Tx) check(keyValueStringer KeyValueStringer, ch chan error) {
|
|||
}
|
||||
|
||||
func (tx *Tx) checkBucket(b *Bucket, reachable map[pgid]*page, freed map[pgid]bool,
|
||||
keyValueStringer KeyValueStringer, ch chan error) {
|
||||
kvStringer KVStringer, ch chan error) {
|
||||
// Ignore inline buckets.
|
||||
if b.root == 0 {
|
||||
return
|
||||
|
@ -89,12 +89,12 @@ func (tx *Tx) checkBucket(b *Bucket, reachable map[pgid]*page, freed map[pgid]bo
|
|||
}
|
||||
})
|
||||
|
||||
tx.recursivelyCheckPages(b.root, keyValueStringer.KeyToString, ch)
|
||||
tx.recursivelyCheckPages(b.root, kvStringer.KeyToString, ch)
|
||||
|
||||
// Check each bucket within this bucket.
|
||||
_ = b.ForEachBucket(func(k []byte) error {
|
||||
if child := b.Bucket(k); child != nil {
|
||||
tx.checkBucket(child, reachable, freed, keyValueStringer, ch)
|
||||
tx.checkBucket(child, reachable, freed, kvStringer, ch)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
|
@ -127,8 +127,8 @@ func (tx *Tx) recursivelyCheckPagesInternal(
|
|||
elem := p.branchPageElement(uint16(i))
|
||||
if i == 0 && runningMin != nil && compareKeys(runningMin, elem.key()) > 0 {
|
||||
ch <- fmt.Errorf("key (%d, %s) on the branch page(%d) needs to be >="+
|
||||
" to the index in the ancestor. Pages stack: %v",
|
||||
i, keyToString(elem.key()), pgid, pagesStack)
|
||||
" to the key(%s) in the ancestor. Pages stack: %v",
|
||||
i, keyToString(elem.key()), pgid, keyToString(runningMin), pagesStack)
|
||||
}
|
||||
|
||||
if maxKeyOpen != nil && compareKeys(elem.key(), maxKeyOpen) >= 0 {
|
||||
|
@ -146,22 +146,25 @@ func (tx *Tx) recursivelyCheckPagesInternal(
|
|||
maxKeyInSubtree = tx.recursivelyCheckPagesInternal(elem.pgid, elem.key(), maxKey, pagesStack, keyToString, ch)
|
||||
runningMin = maxKeyInSubtree
|
||||
}
|
||||
return
|
||||
return maxKeyInSubtree
|
||||
case p.flags&leafPageFlag != 0:
|
||||
runningMin := minKeyClosed
|
||||
for i := range p.leafPageElements() {
|
||||
elem := p.leafPageElement(uint16(i))
|
||||
if i == 0 && runningMin != nil && compareKeys(runningMin, elem.key()) > 0 {
|
||||
ch <- fmt.Errorf("key[%d]=(hex)%s on leaf page(%d) needs to be >= to the key in the ancestor. Stack: %v",
|
||||
i, keyToString(elem.key()), pgid, pagesStack)
|
||||
}
|
||||
if i > 0 && compareKeys(runningMin, elem.key()) > 0 {
|
||||
ch <- fmt.Errorf("key[%d]=(hex)%s on leaf page(%d) needs to be > (found <) than previous element (hex)%s. Stack: %v",
|
||||
ch <- fmt.Errorf("The first key[%d]=(hex)%s on leaf page(%d) needs to be >= the key in the ancestor (%s). Stack: %v",
|
||||
i, keyToString(elem.key()), pgid, keyToString(runningMin), pagesStack)
|
||||
}
|
||||
if i > 0 && compareKeys(runningMin, elem.key()) == 0 {
|
||||
ch <- fmt.Errorf("key[%d]=(hex)%s on leaf page(%d) needs to be > (found =) than previous element (hex)%s. Stack: %v",
|
||||
i, keyToString(elem.key()), pgid, keyToString(runningMin), pagesStack)
|
||||
if i > 0 {
|
||||
cmpRet := compareKeys(runningMin, elem.key())
|
||||
if cmpRet > 0 {
|
||||
ch <- fmt.Errorf("key[%d]=(hex)%s on leaf page(%d) needs to be > (found <) than previous element (hex)%s. Stack: %v",
|
||||
i, keyToString(elem.key()), pgid, keyToString(runningMin), pagesStack)
|
||||
}
|
||||
if cmpRet == 0 {
|
||||
ch <- fmt.Errorf("key[%d]=(hex)%s on leaf page(%d) needs to be > (found =) than previous element (hex)%s. Stack: %v",
|
||||
i, keyToString(elem.key()), pgid, keyToString(runningMin), pagesStack)
|
||||
}
|
||||
}
|
||||
if maxKeyOpen != nil && compareKeys(elem.key(), maxKeyOpen) >= 0 {
|
||||
ch <- fmt.Errorf("key[%d]=(hex)%s on leaf page(%d) needs to be < than key of the next element in ancestor (hex)%s. Pages stack: %v",
|
||||
|
@ -175,28 +178,28 @@ func (tx *Tx) recursivelyCheckPagesInternal(
|
|||
default:
|
||||
ch <- fmt.Errorf("unexpected page type for pgid:%d", pgid)
|
||||
}
|
||||
return nil
|
||||
return maxKeyInSubtree
|
||||
}
|
||||
|
||||
// ===========================================================================================
|
||||
|
||||
// KeyValueStringer allows to prepare human-readable diagnostic messages.
|
||||
type KeyValueStringer interface {
|
||||
// KVStringer allows to prepare human-readable diagnostic messages.
|
||||
type KVStringer interface {
|
||||
KeyToString([]byte) string
|
||||
ValueToString([]byte) string
|
||||
}
|
||||
|
||||
// HexKeyValueStringer serializes both key & value to hex representation.
|
||||
func HexKeyValueStringer() KeyValueStringer {
|
||||
return hexKeyValueStringer{}
|
||||
// HexKVStringer serializes both key & value to hex representation.
|
||||
func HexKVStringer() KVStringer {
|
||||
return hexKvStringer{}
|
||||
}
|
||||
|
||||
type hexKeyValueStringer struct{}
|
||||
type hexKvStringer struct{}
|
||||
|
||||
func (_ hexKeyValueStringer) KeyToString(key []byte) string {
|
||||
func (_ hexKvStringer) KeyToString(key []byte) string {
|
||||
return hex.EncodeToString(key)
|
||||
}
|
||||
|
||||
func (_ hexKeyValueStringer) ValueToString(value []byte) string {
|
||||
func (_ hexKvStringer) ValueToString(value []byte) string {
|
||||
return hex.EncodeToString(value)
|
||||
}
|
||||
|
|
|
@ -48,7 +48,7 @@ func TestTx_Check_ReadOnly(t *testing.T) {
|
|||
numChecks := 2
|
||||
errc := make(chan error, numChecks)
|
||||
check := func() {
|
||||
errc <- <-tx.Check(bolt.HexKeyValueStringer())
|
||||
errc <- <-tx.Check(bolt.HexKVStringer())
|
||||
}
|
||||
// Ensure the freelist is not reloaded and does not race.
|
||||
for i := 0; i < numChecks; i++ {
|
||||
|
|
Loading…
Reference in New Issue