Skip to content

Commit

Permalink
*: replace interface{} with Datum in kv.Index
Browse files Browse the repository at this point in the history
  • Loading branch information
coocood committed Mar 3, 2016
1 parent 70b15ac commit 3d06fb9
Show file tree
Hide file tree
Showing 9 changed files with 80 additions and 76 deletions.
11 changes: 5 additions & 6 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/types"
)

func buildIndexInfo(tblInfo *model.TableInfo, unique bool, indexName model.CIStr, indexID int64, idxColNames []*coldef.IndexColName) (*model.IndexInfo, error) {
Expand Down Expand Up @@ -307,20 +308,18 @@ func checkRowExist(txn kv.Transaction, t table.Table, handle int64) (bool, error
return true, nil
}

func fetchRowColVals(txn kv.Transaction, t table.Table, handle int64, indexInfo *model.IndexInfo) ([]interface{}, error) {
func fetchRowColVals(txn kv.Transaction, t table.Table, handle int64, indexInfo *model.IndexInfo) ([]types.Datum, error) {
// fetch datas
cols := t.Cols()
var vals []interface{}
vals := make([]types.Datum, 0, len(indexInfo.Columns))
for _, v := range indexInfo.Columns {
var val interface{}

col := cols[v.Offset]
k := t.RecordKey(handle, col)
data, err := txn.Get(k)
if err != nil {
return nil, errors.Trace(err)
}
val, err = tables.DecodeValue(data, &col.FieldType)
val, err := tables.DecodeValue(data, &col.FieldType)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -435,7 +434,7 @@ func (d *ddl) backfillTableIndex(t table.Table, indexInfo *model.IndexInfo, hand
return nil
}

var vals []interface{}
var vals []types.Datum
vals, err = fetchRowColVals(txn, t, handle, indexInfo)
if err != nil {
return errors.Trace(err)
Expand Down
40 changes: 20 additions & 20 deletions ddl/index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ func (s *testIndexSuite) TestIndex(c *C) {
txn, err = ctx.GetTxn(true)
c.Assert(err, IsNil)

exist, _, err := index.X.Exist(txn, []interface{}{1}, h)
exist, _, err := index.X.Exist(txn, types.MakeDatums(1), h)
c.Assert(err, IsNil)
c.Assert(exist, IsTrue)

Expand All @@ -149,7 +149,7 @@ func (s *testIndexSuite) TestIndex(c *C) {
txn, err = ctx.GetTxn(true)
c.Assert(err, IsNil)

exist, _, err = index.X.Exist(txn, []interface{}{1}, h)
exist, _, err = index.X.Exist(txn, types.MakeDatums(1), h)
c.Assert(err, IsNil)
c.Assert(exist, IsFalse)

Expand Down Expand Up @@ -177,7 +177,7 @@ func (s *testIndexSuite) testGetIndex(c *C, t table.Table, name string, isExist
}
}

func (s *testIndexSuite) checkIndexKVExist(c *C, ctx context.Context, t table.Table, handle int64, indexCol *column.IndexedCol, columnValues []interface{}, isExist bool) {
func (s *testIndexSuite) checkIndexKVExist(c *C, ctx context.Context, t table.Table, handle int64, indexCol *column.IndexedCol, columnValues []types.Datum, isExist bool) {
c.Assert(len(indexCol.Columns), Equals, len(columnValues))

txn, err := ctx.GetTxn(true)
Expand All @@ -194,9 +194,9 @@ func (s *testIndexSuite) checkIndexKVExist(c *C, ctx context.Context, t table.Ta
func (s *testIndexSuite) checkNoneIndex(c *C, ctx context.Context, d *ddl, tblInfo *model.TableInfo, handle int64, indexCol *column.IndexedCol, row []types.Datum) {
t := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID)

columnValues := make([]interface{}, len(indexCol.Columns))
columnValues := make([]types.Datum, len(indexCol.Columns))
for i, column := range indexCol.Columns {
columnValues[i] = row[column.Offset].GetValue()
columnValues[i] = row[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, false)
Expand All @@ -218,9 +218,9 @@ func (s *testIndexSuite) checkDeleteOnlyIndex(c *C, ctx context.Context, d *ddl,
c.Assert(err, IsNil)
c.Assert(i, Equals, int64(1))

columnValues := make([]interface{}, len(indexCol.Columns))
columnValues := make([]types.Datum, len(indexCol.Columns))
for i, column := range indexCol.Columns {
columnValues[i] = row[column.Offset].GetValue()
columnValues[i] = row[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, isDropped)
Expand All @@ -247,7 +247,7 @@ func (s *testIndexSuite) checkDeleteOnlyIndex(c *C, ctx context.Context, d *ddl,
c.Assert(i, Equals, int64(2))

for i, column := range indexCol.Columns {
columnValues[i] = newRow[column.Offset].GetValue()
columnValues[i] = newRow[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, false)
Expand All @@ -264,7 +264,7 @@ func (s *testIndexSuite) checkDeleteOnlyIndex(c *C, ctx context.Context, d *ddl,
s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, false)

for i, column := range indexCol.Columns {
columnValues[i] = newUpdateRow[column.Offset].GetValue()
columnValues[i] = newUpdateRow[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, false)
Expand Down Expand Up @@ -305,9 +305,9 @@ func (s *testIndexSuite) checkWriteOnlyIndex(c *C, ctx context.Context, d *ddl,
c.Assert(err, IsNil)
c.Assert(i, Equals, int64(1))

columnValues := make([]interface{}, len(indexCol.Columns))
columnValues := make([]types.Datum, len(indexCol.Columns))
for i, column := range indexCol.Columns {
columnValues[i] = row[column.Offset].GetValue()
columnValues[i] = row[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, isDropped)
Expand All @@ -334,7 +334,7 @@ func (s *testIndexSuite) checkWriteOnlyIndex(c *C, ctx context.Context, d *ddl,
c.Assert(i, Equals, int64(2))

for i, column := range indexCol.Columns {
columnValues[i] = newRow[column.Offset].GetValue()
columnValues[i] = newRow[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, true)
Expand All @@ -351,7 +351,7 @@ func (s *testIndexSuite) checkWriteOnlyIndex(c *C, ctx context.Context, d *ddl,
s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, false)

for i, column := range indexCol.Columns {
columnValues[i] = newUpdateRow[column.Offset].GetValue()
columnValues[i] = newUpdateRow[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, true)
Expand Down Expand Up @@ -413,9 +413,9 @@ func (s *testIndexSuite) checkReorganizationIndex(c *C, ctx context.Context, d *
})
c.Assert(i, Equals, int64(2))

columnValues := make([]interface{}, len(indexCol.Columns))
columnValues := make([]types.Datum, len(indexCol.Columns))
for i, column := range indexCol.Columns {
columnValues[i] = newRow[column.Offset].GetValue()
columnValues[i] = newRow[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, !isDropped)
Expand All @@ -432,7 +432,7 @@ func (s *testIndexSuite) checkReorganizationIndex(c *C, ctx context.Context, d *
s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, false)

for i, column := range indexCol.Columns {
columnValues[i] = newUpdateRow[column.Offset].GetValue()
columnValues[i] = newUpdateRow[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, !isDropped)
Expand Down Expand Up @@ -472,9 +472,9 @@ func (s *testIndexSuite) checkPublicIndex(c *C, ctx context.Context, d *ddl, tbl
c.Assert(err, IsNil)
c.Assert(i, Equals, int64(1))

columnValues := make([]interface{}, len(indexCol.Columns))
columnValues := make([]types.Datum, len(indexCol.Columns))
for i, column := range indexCol.Columns {
columnValues[i] = row[column.Offset].GetValue()
columnValues[i] = row[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, true)
Expand All @@ -501,7 +501,7 @@ func (s *testIndexSuite) checkPublicIndex(c *C, ctx context.Context, d *ddl, tbl
c.Assert(i, Equals, int64(2))

for i, column := range indexCol.Columns {
columnValues[i] = newRow[column.Offset].GetValue()
columnValues[i] = newRow[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, true)
Expand All @@ -518,7 +518,7 @@ func (s *testIndexSuite) checkPublicIndex(c *C, ctx context.Context, d *ddl, tbl
s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, false)

for i, column := range indexCol.Columns {
columnValues[i] = newUpdateRow[column.Offset].GetValue()
columnValues[i] = newUpdateRow[column.Offset]
}

s.checkIndexKVExist(c, ctx, t, handle, indexCol, columnValues, true)
Expand Down
12 changes: 7 additions & 5 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -338,13 +338,15 @@ func (e *IndexRangeExec) Fields() []*ast.ResultField {
// Next implements Executor Next interface.
func (e *IndexRangeExec) Next() (*Row, error) {
if e.iter == nil {
seekVals := make([]interface{}, len(e.scan.idx.Columns))
seekVals := make([]types.Datum, len(e.scan.idx.Columns))
for i := 0; i < len(e.lowVals); i++ {
var err error
if e.lowVals[i] == plan.MinNotNullVal {
seekVals[i] = []byte{}
seekVals[i].SetBytes([]byte{})
} else {
seekVals[i], err = types.Convert(e.lowVals[i], e.scan.valueTypes[i])
var val interface{}
val, err = types.Convert(e.lowVals[i], e.scan.valueTypes[i])
seekVals[i].SetValue(val)
if err != nil {
return nil, errors.Trace(err)
}
Expand All @@ -370,7 +372,7 @@ func (e *IndexRangeExec) Next() (*Row, error) {
}
if !e.skipLowCmp {
var cmp int
cmp, err = indexCompare(idxKey, e.lowVals)
cmp, err = indexCompare(types.DatumsToInterfaces(idxKey), e.lowVals)
if err != nil {
return nil, errors.Trace(err)
}
Expand All @@ -379,7 +381,7 @@ func (e *IndexRangeExec) Next() (*Row, error) {
}
e.skipLowCmp = true
}
cmp, err := indexCompare(idxKey, e.highVals)
cmp, err := indexCompare(types.DatumsToInterfaces(idxKey), e.highVals)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down
3 changes: 2 additions & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/types"
)

func TestT(t *testing.T) {
Expand Down Expand Up @@ -97,7 +98,7 @@ func (s *testSuite) TestAdmin(c *C) {
tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("admin_test"))
c.Assert(err, IsNil)
c.Assert(tb.Indices(), HasLen, 1)
err = tb.Indices()[0].X.Create(txn, []interface{}{int64(10)}, 1)
err = tb.Indices()[0].X.Create(txn, types.MakeDatums(int64(10)), 1)
c.Assert(err, IsNil)
err = txn.Commit()
c.Assert(err, IsNil)
Expand Down
22 changes: 11 additions & 11 deletions inspectkv/inspectkv.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,9 +90,9 @@ func GetBgDDLInfo(txn kv.Transaction) (*DDLInfo, error) {
return info, nil
}

func nextIndexVals(data []interface{}) []interface{} {
func nextIndexVals(data []types.Datum) []types.Datum {
// Add 0x0 to the end of data.
return append(data, nil)
return append(data, types.Datum{})
}

// RecordData is the record data composed of a handle and values.
Expand All @@ -103,7 +103,7 @@ type RecordData struct {

// GetIndexRecordsCount returns the total number of the index records from startVals.
// If startVals = nil, returns the total number of the index records.
func GetIndexRecordsCount(txn kv.Transaction, kvIndex kv.Index, startVals []interface{}) (int64, error) {
func GetIndexRecordsCount(txn kv.Transaction, kvIndex kv.Index, startVals []types.Datum) (int64, error) {
it, _, err := kvIndex.Seek(txn, startVals)
if err != nil {
return 0, errors.Trace(err)
Expand All @@ -128,24 +128,24 @@ func GetIndexRecordsCount(txn kv.Transaction, kvIndex kv.Index, startVals []inte
// It returns data and the next startVals until it doesn't have data, then returns data is nil and
// the next startVals is the values which can't get data. If startVals = nil and limit = -1,
// it returns the index data of the whole.
func ScanIndexData(txn kv.Transaction, kvIndex kv.Index, startVals []interface{}, limit int64) (
[]*RecordData, []interface{}, error) {
func ScanIndexData(txn kv.Transaction, kvIndex kv.Index, startVals []types.Datum, limit int64) (
[]*RecordData, []types.Datum, error) {
it, _, err := kvIndex.Seek(txn, startVals)
if err != nil {
return nil, nil, errors.Trace(err)
}
defer it.Close()

var idxRows []*RecordData
var curVals []interface{}
var curVals []types.Datum
for limit != 0 {
val, h, err1 := it.Next()
if terror.ErrorEqual(err1, io.EOF) {
return idxRows, nextIndexVals(curVals), nil
} else if err1 != nil {
return nil, nil, errors.Trace(err1)
}
idxRows = append(idxRows, &RecordData{Handle: h, Values: types.MakeDatums(val...)})
idxRows = append(idxRows, &RecordData{Handle: h, Values: val})
limit--
curVals = val
}
Expand Down Expand Up @@ -195,14 +195,14 @@ func checkIndexAndRecord(txn kv.Transaction, t table.Table, idx *column.IndexedC

vals2, err := rowWithCols(txn, t, h, cols)
if terror.ErrorEqual(err, kv.ErrNotExist) {
record := &RecordData{Handle: h, Values: types.MakeDatums(vals1...)}
record := &RecordData{Handle: h, Values: vals1}
err = errors.Errorf("index:%v != record:%v", record, nil)
}
if err != nil {
return errors.Trace(err)
}
if !reflect.DeepEqual(types.MakeDatums(vals1...), vals2) {
record1 := &RecordData{Handle: h, Values: types.MakeDatums(vals1...)}
if !reflect.DeepEqual(vals1, vals2) {
record1 := &RecordData{Handle: h, Values: vals1}
record2 := &RecordData{Handle: h, Values: vals2}
return errors.Errorf("index:%v != record:%v", record1, record2)
}
Expand All @@ -220,7 +220,7 @@ func checkRecordAndIndex(txn kv.Transaction, t table.Table, idx *column.IndexedC
startKey := t.RecordKey(0, nil)
kvIndex := kv.NewKVIndex(t.IndexPrefix(), idx.Name.L, idx.ID, idx.Unique)
filterFunc := func(h1 int64, vals1 []types.Datum, cols []*column.Col) (bool, error) {
isExist, h2, err := kvIndex.Exist(txn, types.DatumsToInterfaces(vals1), h1)
isExist, h2, err := kvIndex.Exist(txn, vals1, h1)
if terror.ErrorEqual(err, kv.ErrKeyExists) {
record1 := &RecordData{Handle: h1, Values: vals1}
record2 := &RecordData{Handle: h2, Values: vals1}
Expand Down
12 changes: 6 additions & 6 deletions inspectkv/inspectkv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -213,18 +213,18 @@ func (s *testSuite) TestScan(c *C) {
idxRow1 := &RecordData{Handle: int64(1), Values: types.MakeDatums(int64(10))}
idxRow2 := &RecordData{Handle: int64(2), Values: types.MakeDatums(int64(20))}
kvIndex := kv.NewKVIndex(tb.IndexPrefix(), indices[0].Name.L, indices[0].ID, indices[0].Unique)
idxRows, nextVals, err := ScanIndexData(txn, kvIndex, types.DatumsToInterfaces(idxRow1.Values), 2)
idxRows, nextVals, err := ScanIndexData(txn, kvIndex, idxRow1.Values, 2)
c.Assert(err, IsNil)
c.Assert(idxRows, DeepEquals, []*RecordData{idxRow1, idxRow2})
idxRows, nextVals, err = ScanIndexData(txn, kvIndex, types.DatumsToInterfaces(idxRow1.Values), 1)
idxRows, nextVals, err = ScanIndexData(txn, kvIndex, idxRow1.Values, 1)
c.Assert(err, IsNil)
c.Assert(idxRows, DeepEquals, []*RecordData{idxRow1})
idxRows, nextVals, err = ScanIndexData(txn, kvIndex, nextVals, 1)
c.Assert(err, IsNil)
c.Assert(idxRows, DeepEquals, []*RecordData{idxRow2})
idxRows, nextVals, err = ScanIndexData(txn, kvIndex, nextVals, 1)
c.Assert(idxRows, IsNil)
c.Assert(nextVals, DeepEquals, []interface{}{nil})
c.Assert(nextVals, DeepEquals, types.MakeDatums(nil))
c.Assert(err, IsNil)

s.testTableData(c, tb, []*RecordData{record1, record2})
Expand Down Expand Up @@ -305,7 +305,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx *column.IndexedCol) {
// current index data:
// index data (handle, data): (1, 10), (2, 20), (3, 30)
// index col data (handle, data): (1, 10), (2, 20), (4, 40)
err = idx.X.Create(txn, []interface{}{int64(30)}, 3)
err = idx.X.Create(txn, types.MakeDatums(int64(30)), 3)
c.Assert(err, IsNil)
col := tb.Cols()[idx.Columns[0].Offset]
key := tb.RecordKey(4, col)
Expand All @@ -325,7 +325,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx *column.IndexedCol) {
// current index data:
// index data (handle, data): (1, 10), (2, 20), (3, 30), (4, 40)
// index col data (handle, data): (1, 10), (2, 20), (4, 40), (3, 31)
err = idx.X.Create(txn, []interface{}{int64(40)}, 4)
err = idx.X.Create(txn, types.MakeDatums(int64(40)), 4)
c.Assert(err, IsNil)
key = tb.RecordKey(3, col)
err = tables.SetColValue(txn, key, types.NewDatum(int64(31)))
Expand Down Expand Up @@ -382,7 +382,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx *column.IndexedCol) {
// current index data:
// index data (handle, data): (1, 10), (2, 20), (3, 30)
// index col data (handle, data): (1, 10), (2, 20), (3, 30), (4, 40)
err = idx.X.Delete(txn, []interface{}{int64(40)}, 4)
err = idx.X.Delete(txn, types.MakeDatums(int64(40)), 4)
c.Assert(err, IsNil)
key = tb.RecordKey(4, col)
err = tables.SetColValue(txn, key, types.NewDatum(int64(40)))
Expand Down
Loading

0 comments on commit 3d06fb9

Please sign in to comment.