Skip to content

Commit

Permalink
*: tiny refactor table record/index key codec.
Browse files Browse the repository at this point in the history
  • Loading branch information
qiuyesuifeng committed Dec 8, 2015
1 parent 44819de commit 958ce22
Show file tree
Hide file tree
Showing 9 changed files with 79 additions and 31 deletions.
6 changes: 4 additions & 2 deletions ddl/ddl_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,8 @@ LOOP:
// Make sure there is index with name c3_index
c.Assert(nidx, NotNil)
c.Assert(nidx.ID, Greater, int64(0))
idx := kv.NewKVIndex(t.IndexPrefix(), "c3_index", nidx.ID, false)
idx, err := kv.NewKVIndex(t.IndexPrefix(), "c3_index", nidx.ID, false)
c.Assert(err, IsNil)
txn, err := ctx.GetTxn(true)
c.Assert(err, IsNil)
defer ctx.FinishTxn(true)
Expand Down Expand Up @@ -280,7 +281,8 @@ LOOP:
}
// Make sure there is no index with name c3_index
c.Assert(nidx, IsNil)
idx := kv.NewKVIndex(t.IndexPrefix(), "c3_index", c3idx.ID, false)
idx, err := kv.NewKVIndex(t.IndexPrefix(), "c3_index", c3idx.ID, false)
c.Assert(err, IsNil)
txn, err := ctx.GetTxn(true)
c.Assert(err, IsNil)
defer ctx.FinishTxn(true)
Expand Down
13 changes: 10 additions & 3 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -389,7 +389,7 @@ func (d *ddl) getSnapshotRows(t table.Table, version uint64, seekHandle int64) (
}

var handle int64
handle, err = util.DecodeHandleFromRowKey(string(key))
handle, err = util.DecodeHandleFromRowKey(t.Prefix(), key)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -422,7 +422,10 @@ func lockRow(txn kv.Transaction, t table.Table, h int64) error {
}

func (d *ddl) backfillTableIndex(t table.Table, indexInfo *model.IndexInfo, handles []int64, reorgInfo *reorgInfo) error {
kvX := kv.NewKVIndex(t.IndexPrefix(), indexInfo.Name.L, indexInfo.ID, indexInfo.Unique)
kvX, err := kv.NewKVIndex(t.IndexPrefix(), indexInfo.Name.L, indexInfo.ID, indexInfo.Unique)
if err != nil {
return errors.Trace(err)
}

for _, handle := range handles {
log.Debug("building index...", handle)
Expand Down Expand Up @@ -479,7 +482,11 @@ func (d *ddl) backfillTableIndex(t table.Table, indexInfo *model.IndexInfo, hand
}

func (d *ddl) dropTableIndex(t table.Table, indexInfo *model.IndexInfo) error {
prefix := kv.GenIndexPrefix(t.IndexPrefix(), indexInfo.ID)
prefix, err := kv.GenIndexPrefix(t.IndexPrefix(), indexInfo.ID)
if err != nil {
return errors.Trace(err)
}

prefixBytes := []byte(prefix)

for {
Expand Down
30 changes: 17 additions & 13 deletions kv/index_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,10 @@ package kv
import (
"bytes"
"encoding/binary"
"fmt"
"io"
"strings"

"github.com/juju/errors"
"github.com/pingcap/tidb/util/codec"
)

var (
Expand Down Expand Up @@ -103,24 +101,30 @@ type kvIndex struct {
}

// GenIndexPrefix generates the index prefix.
func GenIndexPrefix(indexPrefix string, indexID int64) string {
// Use EncodeBytes to guarantee generating different index prefix.
// e.g, two indices c1 and c with index prefix p, if no EncodeBytes,
// the index format looks p_c and p_c1, if c has an index value which the first encoded byte is '1',
// we will meet an error, because p_c1 is for index c1.
// If EncodeBytes, c1 -> c1\x00\x01 and c -> c\x00\x01, the prefixs are different.
key := fmt.Sprintf("%s_%d", indexPrefix, indexID)
return string(codec.EncodeBytes(nil, []byte(key)))
func GenIndexPrefix(indexPrefix string, indexID int64) (string, error) {
key, err := EncodeValue(indexID)
if err != nil {
return "", errors.Trace(err)
}

return string(indexPrefix + string(key)), nil
}

// NewKVIndex builds a new kvIndex object.
func NewKVIndex(indexPrefix, indexName string, indexID int64, unique bool) Index {
return &kvIndex{
func NewKVIndex(indexPrefix string, indexName string, indexID int64, unique bool) (Index, error) {
index := &kvIndex{
indexName: indexName,
indexID: indexID,
unique: unique,
prefix: GenIndexPrefix(indexPrefix, indexID),
}

var err error
index.prefix, err = GenIndexPrefix(indexPrefix, indexID)
if err != nil {
return nil, errors.Trace(err)
}

return index, nil
}

// GenIndexKey generates storage key for index values. Returned distinct indicates whether the
Expand Down
6 changes: 4 additions & 2 deletions kv/index_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,8 @@ func (s *testIndexSuite) TearDownSuite(c *C) {
}

func (s *testIndexSuite) TestIndex(c *C) {
index := kv.NewKVIndex("i", "test", 0, false)
index, err := kv.NewKVIndex("i", "test", 0, false)
c.Assert(err, IsNil)

// Test ununiq index.
txn, err := s.s.Begin()
Expand Down Expand Up @@ -118,7 +119,8 @@ func (s *testIndexSuite) TestIndex(c *C) {
err = txn.Commit()
c.Assert(err, IsNil)

index = kv.NewKVIndex("j", "test", 1, true)
index, err = kv.NewKVIndex("j", "test", 1, true)
c.Assert(err, IsNil)

// Test uniq index.
txn, err = s.s.Begin()
Expand Down
5 changes: 4 additions & 1 deletion plan/plans/from_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,8 +176,11 @@ func (p *testFromSuit) TestTableDefaultPlan(c *C) {
Primary: false,
State: model.StatePublic,
},
X: kv.NewKVIndex("i", "id", 0, false),
}

idxCol.X, err = kv.NewKVIndex("i", "id", 0, false)
c.Assert(err, IsNil)

p.tbl.AddIndex(idxCol)

expr4 := &expression.Ident{
Expand Down
5 changes: 4 additions & 1 deletion plan/plans/index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,11 @@ func (p *testIndexSuit) SetUpSuite(c *C) {
Primary: false,
State: model.StatePublic,
},
X: kv.NewKVIndex("i", "id", 0, false),
}

idxCol.X, err = kv.NewKVIndex("i", "id", 0, false)
c.Assert(err, IsNil)

p.tbl.AddIndex(idxCol)
var i int64
for i = 0; i < 10; i++ {
Expand Down
3 changes: 3 additions & 0 deletions table/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,9 @@ type Table interface {
// FindIndexByColName finds the index by column name.
FindIndexByColName(name string) *column.IndexedCol

// Prefix returns the table prefix string.
Prefix() string

// KeyPrefix returns the key prefix string.
KeyPrefix() string

Expand Down
32 changes: 25 additions & 7 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ type Table struct {
publicColumns []*column.Col
writableColumns []*column.Col
indices []*column.IndexedCol
prefix []byte
recordPrefix string
encRecordPrefix []byte
indexPrefix string
Expand All @@ -74,15 +75,21 @@ func TableFromMeta(alloc autoid.Allocator, tblInfo *model.TableInfo) table.Table

t := NewTable(tblInfo.ID, tblInfo.Name.O, columns, alloc)

var err error
for _, idxInfo := range tblInfo.Indices {
if idxInfo.State == model.StateNone {
log.Fatalf("index %s can't be in none state", idxInfo.Name)
}

idx := &column.IndexedCol{
IndexInfo: *idxInfo,
X: kv.NewKVIndex(t.indexPrefix, idxInfo.Name.L, idxInfo.ID, idxInfo.Unique),
}

idx.X, err = kv.NewKVIndex(t.IndexPrefix(), idxInfo.Name.L, idxInfo.ID, idxInfo.Unique)
if err != nil {
log.Fatalf("create new index failed - %s - %v", idxInfo.Name.L, errors.ErrorStack(err))
}

t.AddIndex(idx)
}

Expand All @@ -96,15 +103,18 @@ func NewTable(tableID int64, tableName string, cols []*column.Col, alloc autoid.
t := &Table{
ID: tableID,
Name: name,
recordPrefix: fmt.Sprintf("t%d_r", tableID),
indexPrefix: fmt.Sprintf("t%d_i", tableID),
prefix: []byte{'t'},
recordPrefix: fmt.Sprintf("%d_r", tableID),
indexPrefix: fmt.Sprintf("%d_i", tableID),
alloc: alloc,
Columns: cols,
state: model.StatePublic,
}

t.encRecordPrefix, _ = kv.EncodeValue(t.recordPrefix)
t.encRecordPrefix = append(t.prefix, []byte(t.encRecordPrefix)...)
t.encIndexPrefix, _ = kv.EncodeValue(t.indexPrefix)
t.encIndexPrefix = append(t.prefix, []byte(t.encIndexPrefix)...)

t.publicColumns = t.Cols()
t.writableColumns = t.writableCols()
Expand Down Expand Up @@ -240,6 +250,11 @@ func (t *Table) flatten(data interface{}) (interface{}, error) {
}
}

// Prefix implements table.Table Prefix interface.
func (t *Table) Prefix() string {
return string(t.prefix)
}

// KeyPrefix implements table.Table KeyPrefix interface.
func (t *Table) KeyPrefix() string {
return string(t.encRecordPrefix)
Expand All @@ -252,10 +267,14 @@ func (t *Table) IndexPrefix() string {

// RecordKey implements table.Table RecordKey interface.
func (t *Table) RecordKey(h int64, col *column.Col) []byte {
var key []byte
if col != nil {
return util.EncodeRecordKey(t.recordPrefix, h, col.ID)
key = util.EncodeRecordKey(t.recordPrefix, h, col.ID)
} else {
key = util.EncodeRecordKey(t.recordPrefix, h, 0)
}
return util.EncodeRecordKey(t.recordPrefix, h, 0)

return append(t.prefix, key...)
}

// FirstKey implements table.Table FirstKey interface.
Expand Down Expand Up @@ -452,8 +471,6 @@ func (t *Table) AddRecord(ctx context.Context, r []interface{}, h int64) (record
// Set public and write only column value.
for _, col := range t.writableCols() {
var value interface{}
key := t.RecordKey(recordID, col)

if col.State == model.StateWriteOnly || col.State == model.StateWriteReorganization {
// if col is in write only or write reorganization state, we must add it with its default value.
value, _, err = GetColDefaultValue(ctx, &col.ColumnInfo)
Expand All @@ -468,6 +485,7 @@ func (t *Table) AddRecord(ctx context.Context, r []interface{}, h int64) (record
value = r[col.Offset]
}

key := t.RecordKey(recordID, col)
err = t.SetColValue(txn, key, value)
if err != nil {
return 0, errors.Trace(err)
Expand Down
10 changes: 8 additions & 2 deletions util/prefix_helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,8 +115,14 @@ func EncodeRecordKey(tablePrefix string, h int64, columnID int64) []byte {
}

// DecodeHandleFromRowKey decodes the string form a row key and returns an int64.
func DecodeHandleFromRowKey(rk string) (int64, error) {
vals, err := kv.DecodeValue([]byte(rk))
func DecodeHandleFromRowKey(prefix []byte, rk string) (int64, error) {
key := []byte(rk)
if !bytes.HasPrefix(key, prefix) {
return 0, errors.New("invalid key - %v - %s", prefix, rk)
}

key = key[len(prefix):]
vals, err := kv.DecodeValue(key)
if err != nil {
return 0, errors.Trace(err)
}
Expand Down

0 comments on commit 958ce22

Please sign in to comment.