Skip to content

Commit

Permalink
inspectkv: address comments
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala committed Dec 18, 2015
1 parent b12a616 commit 82addd4
Show file tree
Hide file tree
Showing 2 changed files with 270 additions and 174 deletions.
265 changes: 170 additions & 95 deletions inspectkv/inspectkv.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,17 @@ package inspectkv
import (
"io"
"reflect"
"strings"

"github.com/juju/errors"
"github.com/pingcap/tidb/column"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util"
)

// DDLInfo is for DDL information.
Expand Down Expand Up @@ -75,16 +78,26 @@ type RecordData struct {
Values []interface{}
}

// Equal returns ture if r is equal to rd.
func (rd *RecordData) Equal(r *RecordData) bool {
if r == nil {
return false
// GetIndexRecordsCount returns the total number of the index records.
func GetIndexRecordsCount(txn kv.Transaction, kvIndex kv.Index, startVals []interface{}) (int64, error) {
it, _, err := kvIndex.Seek(txn, startVals)
if err != nil {
return 0, errors.Trace(err)
}
if rd.Handle != r.Handle || !reflect.DeepEqual(rd.Values, r.Values) {
return false
defer it.Close()

var cnt int64
for {
_, _, err := it.Next()
if terror.ErrorEqual(err, io.EOF) {
break
} else if err != nil {
return 0, errors.Trace(err)
}
cnt++
}

return true
return cnt, nil
}

// ScanIndexData scans the index handles and values in a limited number, according to the index information.
Expand Down Expand Up @@ -127,110 +140,121 @@ func ScanIndexData(txn kv.Transaction, kvIndex kv.Index, startVals []interface{}
// It returns data and the next startHandle until it doesn't have data, then returns data is nil and
// the next startHandle is the handle which can't get data.
// If limit = -1, it returns the index data of the whole.
func ScanIndexColData(retriever kv.Retriever, t table.Table, idx *column.IndexedCol, startHandle, limit int64) (
func ScanIndexColData(txn kv.Transaction, t table.Table, idx *column.IndexedCol, startHandle, limit int64) (
[]*RecordData, int64, error) {
cols := make([]*column.Col, len(idx.Columns))
for i, col := range idx.Columns {
cols[i] = t.Cols()[col.Offset]
}

return scanTableData(retriever, t, cols, startHandle, limit)
return scanTableData(txn, t, cols, startHandle, limit)
}

// EqualRecords returns ture if a is equal to b.
func EqualRecords(a, b []*RecordData) bool {
if len(a) != len(b) {
return false
// CompareIndexData compares index data one by one.
// It returns ture if the data from the index is equal to the data from the table columns,
// otherwise it returns false and a different set of records.
func CompareIndexData(txn kv.Transaction, t table.Table, idx *column.IndexedCol) (
*RecordData, *RecordData, bool, error) {
ret1, ret2, isEqual, err := checkIndexAndCols(txn, t, idx)
if err != nil {
return nil, nil, false, errors.Trace(err)
}

for i, r := range a {
if !r.Equal(b[i]) {
return false
}
if isEqual {
ret1, ret2, isEqual, err = checkColsAndIndex(txn, t, idx)
}

return true
return ret1, ret2, isEqual, errors.Trace(err)
}

// DiffRecords compares records one by one.
// It returns the difference between a and b.
// e.g,
// a {{1, "val0"}, {2, "val2"}}
// b {{1, "val1"}, {3, "val3"}}
// returns {{1, "val0"}, {2, "val2"}, nil}, {{1, "val1"}, nil, {3, "val3"}}, nil
func DiffRecords(a, b []*RecordData) ([]*RecordData, []*RecordData, error) {
if len(a) == 0 {
return make([]*RecordData, len(b)), b, nil
func checkIndexAndCols(txn kv.Transaction, t table.Table, idx *column.IndexedCol) (
*RecordData, *RecordData, bool, error) {
kvIndex := kv.NewKVIndex(t.IndexPrefix(), idx.Name.L, idx.ID, idx.Unique)
it, err := kvIndex.SeekFirst(txn)
if err != nil {
return nil, nil, false, errors.Trace(err)
}
defer it.Close()

cols := make([]*column.Col, len(idx.Columns))
for i, col := range idx.Columns {
cols[i] = t.Cols()[col.Offset]
}

var ret1, ret2 []*RecordData
k := 0
for i := 0; i < len(a); i++ {
if len(b[k:]) == 0 {
ret1 = append(ret1, a[i:]...)
ret2 = append(ret2, make([]*RecordData, len(a[i:]))...)
var isEqual bool
var ret1, ret2 *RecordData
for {
vals1, h, err := it.Next()
if terror.ErrorEqual(err, io.EOF) {
isEqual = true
break
} else if err != nil {
return nil, nil, false, errors.Trace(err)
}

for j := k; j < len(b); j++ {
if a[i].Handle > b[j].Handle {
ret1 = append(ret1, nil)
ret2 = append(ret2, b[j])
continue
}
if a[i].Handle < b[j].Handle {
ret1 = append(ret1, a[i])
ret2 = append(ret2, nil)
if len(a[i+1:]) == 0 {
ret1 = append(ret1, make([]*RecordData, len(b[j:]))...)
ret2 = append(ret2, b[j:]...)
}
break
}
if !reflect.DeepEqual(a[i].Values, b[j].Values) {
ret1 = append(ret1, a[i])
ret2 = append(ret2, b[j])
}
k = j + 1
val2, err := t.RowWithCols(txn, h, cols)
if terror.ErrorEqual(err, kv.ErrNotExist) {
ret1 = &RecordData{Handle: h, Values: vals1}
break
}
if err != nil {
return nil, nil, false, errors.Trace(err)
}
if !reflect.DeepEqual(vals1, val2) {
ret1 = &RecordData{Handle: h, Values: vals1}
ret2 = &RecordData{Handle: h, Values: val2}
break
}
}

return ret1, ret2, nil
return ret1, ret2, isEqual, nil
}

// EqualIndexData returns ture if the data from the index is equal to the data from the table columns.
func EqualIndexData(txn kv.Transaction, t table.Table, idx *column.IndexedCol) (bool, error) {
kvIndex := kv.NewKVIndex(t.IndexPrefix(), idx.Name.L, idx.ID, idx.Unique)

rs1, _, err := ScanIndexData(txn, kvIndex, nil, -1)
if err != nil {
return false, errors.Trace(err)
}
rs2, _, err := ScanIndexColData(txn, t, idx, 0, -1)
if err != nil {
return false, errors.Trace(err)
func checkColsAndIndex(txn kv.Transaction, t table.Table, idx *column.IndexedCol) (
*RecordData, *RecordData, bool, error) {
cols := make([]*column.Col, len(idx.Columns))
for i, col := range idx.Columns {
cols[i] = t.Cols()[col.Offset]
}

return EqualRecords(rs1, rs2), nil
}

// DiffIndexData compares index data one by one.
// It returns the difference between data from the index and data from the table columns.
func DiffIndexData(txn kv.Transaction, t table.Table, idx *column.IndexedCol) (
[]*RecordData, []*RecordData, error) {
isEqual := true
var ret1, ret2 *RecordData
startKey := t.RecordKey(0, nil)
kvIndex := kv.NewKVIndex(t.IndexPrefix(), idx.Name.L, idx.ID, idx.Unique)
err := t.IterRecords(txn, string(startKey), cols,
func(h1 int64, vals1 []interface{}, cols []*column.Col) (bool, error) {
it, hit, err := kvIndex.Seek(txn, vals1)
if err != nil {
isEqual = false
return false, errors.Trace(err)
}
defer it.Close()

if !hit {
ret1 = &RecordData{Handle: h1, Values: vals1}
isEqual = false
return false, nil
}
_, h2, err := it.Next()
if err != nil {
isEqual = false
return false, errors.Trace(err)
}
if h1 != h2 {
isEqual = false
ret1 = &RecordData{Handle: h1, Values: vals1}
ret2 = &RecordData{Handle: h2, Values: vals1}
return false, nil
}

return true, nil
})

rs1, _, err := ScanIndexData(txn, kvIndex, nil, -1)
if err != nil {
return nil, nil, errors.Trace(err)
}
rs2, _, err := ScanIndexColData(txn, t, idx, 0, -1)
if err != nil {
return nil, nil, errors.Trace(err)
return nil, nil, false, errors.Trace(err)
}

return DiffRecords(rs1, rs2)
return ret1, ret2, isEqual, nil
}

func scanTableData(retriever kv.Retriever, t table.Table, cols []*column.Col, startHandle, limit int64) (
Expand Down Expand Up @@ -291,27 +315,78 @@ func ScanSnapshotTableData(store kv.Storage, ver kv.Version, t table.Table, star
return records, nextHandle, errors.Trace(err)
}

// EqualTableData returns ture if records is equal to the data that scans from the the start handle.
// If limit = -1, it compares the table data of the whole.
func EqualTableData(retriever kv.Retriever, t table.Table, records []*RecordData, startHandle, limit int64) (
bool, error) {
rs, _, err := ScanTableData(retriever, t, startHandle, limit)
if err != nil {
return false, errors.Trace(err)
// CompareTableData compares records and the corresponding table data one by one.
// It returns ture if records is equal to the data that scans from table, otherwise
// it returns false and a different set of records.
func CompareTableData(txn kv.Transaction, t table.Table, records []*RecordData) (
*RecordData, *RecordData, bool, error) {
isEqual := true
var ret1, ret2 *RecordData

for _, r := range records {
vals, err := t.RowWithCols(txn, r.Handle, t.Cols())
if terror.ErrorEqual(err, kv.ErrNotExist) {
ret2 = &RecordData{Handle: r.Handle, Values: r.Values}
isEqual = false
break
}
if err != nil {
return nil, nil, false, errors.Trace(err)
}

if !reflect.DeepEqual(r.Values, vals) {
ret1 = &RecordData{Handle: r.Handle, Values: vals}
ret2 = &RecordData{Handle: r.Handle, Values: r.Values}
isEqual = false
break
}
}
if !isEqual {
return ret1, ret2, isEqual, nil
}

startKey := t.RecordKey(0, nil)
err := t.IterRecords(txn, string(startKey), t.Cols(),
func(h int64, vals []interface{}, cols []*column.Col) (bool, error) {
for _, r := range records {
if r.Handle == h && reflect.DeepEqual(r.Values, vals) {
return true, nil
}
}
isEqual = false
ret1 = &RecordData{Handle: h, Values: vals}

return EqualRecords(rs, records), nil
return false, nil
})

return ret1, ret2, isEqual, errors.Trace(err)
}

// DiffTableData compares records and the corresponding table data one by one.
// It returns the difference between the table data that scans from the start handle and records .
// If limit = -1, it compares the table data of the whole.
func DiffTableData(retriever kv.Retriever, t table.Table, records []*RecordData, startHandle, limit int64) (
[]*RecordData, []*RecordData, error) {
rs, _, err := ScanTableData(retriever, t, startHandle, limit)
// GetTableRecordsCount returns the total number of table records.
func GetTableRecordsCount(txn kv.Transaction, t table.Table, startHandle int64) (int64, error) {
startKey := t.RecordKey(startHandle, nil)
it, err := txn.Seek(startKey)
if err != nil {
return nil, nil, errors.Trace(err)
return 0, errors.Trace(err)
}
defer it.Close()

var cnt int64
prefix := t.KeyPrefix()
for it.Valid() && strings.HasPrefix(it.Key(), prefix) {
handle, err := tables.DecodeRecordKeyHandle(it.Key())
if err != nil {
return 0, errors.Trace(err)
}

rk := t.RecordKey(handle, nil)
err = kv.NextUntil(it, util.RowKeyPrefixFilter(rk))
if err != nil {
return 0, errors.Trace(err)
}

cnt++
}

return DiffRecords(rs, records)
return cnt, nil
}
Loading

0 comments on commit 82addd4

Please sign in to comment.