Skip to content

Commit

Permalink
*: Clean up tablecodec and tables (pingcap#1338)
Browse files Browse the repository at this point in the history
  • Loading branch information
shenli authored Jun 22, 2016
1 parent ee1bdd6 commit 633b5b9
Show file tree
Hide file tree
Showing 7 changed files with 62 additions and 75 deletions.
3 changes: 2 additions & 1 deletion ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testleak"
"github.com/pingcap/tidb/util/types"
Expand Down Expand Up @@ -266,7 +267,7 @@ func (s *testColumnSuite) checkColumnKVExist(c *C, ctx context.Context, t table.

if isExist {
c.Assert(err, IsNil)
v, err1 := tables.DecodeValue(data, &col.FieldType)
v, err1 := tablecodec.DecodeColumnValue(data, &col.FieldType)
c.Assert(err1, IsNil)
value, err1 := v.ConvertTo(&col.FieldType)
c.Assert(err1, IsNil)
Expand Down
2 changes: 1 addition & 1 deletion ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -321,7 +321,7 @@ func fetchRowColVals(txn kv.Transaction, t table.Table, handle int64, indexInfo
}
return nil, errors.Trace(err)
}
val, err := tables.DecodeValue(data, &col.FieldType)
val, err := tablecodec.DecodeColumnValue(data, &col.FieldType)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down
3 changes: 1 addition & 2 deletions inspectkv/inspectkv.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util"
Expand Down Expand Up @@ -400,7 +399,7 @@ func rowWithCols(txn kv.Retriever, t table.Table, h int64, cols []*table.Column)
return nil, errors.Trace(err)
}

val, err := tables.DecodeValue(data, &col.FieldType)
val, err := tablecodec.DecodeColumnValue(data, &col.FieldType)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion table/tables/bounded_tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ func (t *BoundedTable) RecordKey(h int64, col *table.Column) kv.Key {
if col != nil {
colID = col.ID
}
return encodeRecordKey(t.recordPrefix, h, colID)
return tablecodec.EncodeRecordKey(t.recordPrefix, h, colID)
}

// FirstKey implements table.Table FirstKey interface.
Expand Down
2 changes: 1 addition & 1 deletion table/tables/memory_tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ func (t *MemoryTable) RecordKey(h int64, col *table.Column) kv.Key {
if col != nil {
colID = col.ID
}
return encodeRecordKey(t.recordPrefix, h, colID)
return tablecodec.EncodeRecordKey(t.recordPrefix, h, colID)
}

// FirstKey implements table.Table FirstKey interface.
Expand Down
71 changes: 3 additions & 68 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/types"
)

Expand Down Expand Up @@ -162,7 +161,7 @@ func (t *Table) RecordKey(h int64, col *table.Column) kv.Key {
if col != nil {
colID = col.ID
}
return encodeRecordKey(t.recordPrefix, h, colID)
return tablecodec.EncodeRecordKey(t.recordPrefix, h, colID)
}

// FirstKey implements table.Table FirstKey interface.
Expand Down Expand Up @@ -461,7 +460,7 @@ func (t *Table) RowWithCols(ctx context.Context, h int64, cols []*table.Column)
return nil, errors.Trace(err)
}

v[i], err = DecodeValue(data, &col.FieldType)
v[i], err = tablecodec.DecodeColumnValue(data, &col.FieldType)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -666,75 +665,11 @@ func (t *Table) Seek(ctx context.Context, h int64) (int64, bool, error) {

var (
recordPrefixSep = []byte("_r")
indexPrefixSep = []byte("_i")
)

func encodeRecordKey(recordPrefix kv.Key, h int64, columnID int64) kv.Key {
buf := make([]byte, 0, len(recordPrefix)+16)
buf = append(buf, recordPrefix...)
buf = codec.EncodeInt(buf, h)

if columnID != 0 {
buf = codec.EncodeInt(buf, columnID)
}
return buf
}

// EncodeValue encodes a go value to bytes.
func EncodeValue(raw types.Datum) ([]byte, error) {
v, err := flatten(raw)
if err != nil {
return nil, errors.Trace(err)
}
b, err := codec.EncodeValue(nil, v)
return b, errors.Trace(err)
}

// DecodeValue implements table.Table DecodeValue interface.
func DecodeValue(data []byte, tp *types.FieldType) (types.Datum, error) {
values, err := codec.Decode(data)
if err != nil {
return types.Datum{}, errors.Trace(err)
}
return tablecodec.Unflatten(values[0], tp)
}

func flatten(data types.Datum) (types.Datum, error) {
switch data.Kind() {
case types.KindMysqlTime:
// for mysql datetime, timestamp and date type
b, err := data.GetMysqlTime().Marshal()
if err != nil {
return types.NewDatum(nil), errors.Trace(err)
}
return types.NewDatum(b), nil
case types.KindMysqlDuration:
// for mysql time type
data.SetInt64(int64(data.GetMysqlDuration().Duration))
return data, nil
case types.KindMysqlDecimal:
data.SetString(data.GetMysqlDecimal().String())
return data, nil
case types.KindMysqlEnum:
data.SetUint64(data.GetMysqlEnum().Value)
return data, nil
case types.KindMysqlSet:
data.SetUint64(data.GetMysqlSet().Value)
return data, nil
case types.KindMysqlBit:
data.SetUint64(data.GetMysqlBit().Value)
return data, nil
case types.KindMysqlHex:
data.SetInt64(data.GetMysqlHex().Value)
return data, nil
default:
return data, nil
}
}

// SetColValue implements table.Table SetColValue interface.
func SetColValue(rm kv.RetrieverMutator, key []byte, data types.Datum) error {
v, err := EncodeValue(data)
v, err := tablecodec.EncodeValue(data)
if err != nil {
return errors.Trace(err)
}
Expand Down
54 changes: 53 additions & 1 deletion tablecodec/tablecodec.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,16 @@ func EncodeRowKey(tableID int64, encodedHandle []byte) kv.Key {
func EncodeColumnKey(tableID int64, handle int64, columnID int64) kv.Key {
buf := make([]byte, 0, recordRowKeyLen+idLen)
buf = appendTableRecordPrefix(buf, tableID)
buf = codec.EncodeInt(buf, handle)
buf = EncodeRecordKey(buf, handle, columnID)
return buf
}

// EncodeRecordKey encodes the recordPrefix, row handle and columnID into a kv.Key.
// TODO: Remove this function
func EncodeRecordKey(recordPrefix kv.Key, h int64, columnID int64) kv.Key {
buf := make([]byte, 0, len(recordPrefix)+16)
buf = append(buf, recordPrefix...)
buf = codec.EncodeInt(buf, h)
if columnID != 0 {
buf = codec.EncodeInt(buf, columnID)
}
Expand Down Expand Up @@ -102,6 +111,49 @@ func DecodeRowKey(key kv.Key) (int64, error) {
return handle, errors.Trace(err)
}

// EncodeValue encodes a go value to bytes.
func EncodeValue(raw types.Datum) ([]byte, error) {
v, err := flatten(raw)
if err != nil {
return nil, errors.Trace(err)
}
b, err := codec.EncodeValue(nil, v)
return b, errors.Trace(err)
}

func flatten(data types.Datum) (types.Datum, error) {
switch data.Kind() {
case types.KindMysqlTime:
// for mysql datetime, timestamp and date type
b, err := data.GetMysqlTime().Marshal()
if err != nil {
return types.NewDatum(nil), errors.Trace(err)
}
return types.NewDatum(b), nil
case types.KindMysqlDuration:
// for mysql time type
data.SetInt64(int64(data.GetMysqlDuration().Duration))
return data, nil
case types.KindMysqlDecimal:
data.SetString(data.GetMysqlDecimal().String())
return data, nil
case types.KindMysqlEnum:
data.SetUint64(data.GetMysqlEnum().Value)
return data, nil
case types.KindMysqlSet:
data.SetUint64(data.GetMysqlSet().Value)
return data, nil
case types.KindMysqlBit:
data.SetUint64(data.GetMysqlBit().Value)
return data, nil
case types.KindMysqlHex:
data.SetInt64(data.GetMysqlHex().Value)
return data, nil
default:
return data, nil
}
}

// DecodeValues decodes a byte slice into datums with column types.
func DecodeValues(data []byte, fts []*types.FieldType, inIndex bool) ([]types.Datum, error) {
if data == nil {
Expand Down

0 comments on commit 633b5b9

Please sign in to comment.