Skip to content

Commit

Permalink
server, tablecodec: show region key's record id or index values. (pin…
Browse files Browse the repository at this point in the history
  • Loading branch information
disksing authored Mar 20, 2018
1 parent 4886dcb commit 8fa4980
Show file tree
Hide file tree
Showing 4 changed files with 101 additions and 10 deletions.
30 changes: 22 additions & 8 deletions server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -445,12 +445,14 @@ func (rt *RegionDetail) addTableInRange(dbName string, curTable *model.TableInfo

// FrameItem includes a index's or record's meta data with table's info.
type FrameItem struct {
DBName string `json:"db_name"`
TableName string `json:"table_name"`
TableID int64 `json:"table_id"`
IsRecord bool `json:"is_record"`
IndexName string `json:"index_name,omitempty"`
IndexID int64 `json:"index_id,omitempty"`
DBName string `json:"db_name"`
TableName string `json:"table_name"`
TableID int64 `json:"table_id"`
IsRecord bool `json:"is_record"`
RecordID int64 `json:"record_id,omitempty"`
IndexName string `json:"index_name,omitempty"`
IndexID int64 `json:"index_id,omitempty"`
IndexValues []string `json:"index_values,omitempty"`
}

// RegionFrameRange contains a frame range info which the region covered.
Expand Down Expand Up @@ -744,6 +746,14 @@ func NewFrameItemFromRegionKey(key []byte) (frame *FrameItem, err error) {
frame = &FrameItem{}
frame.TableID, frame.IndexID, frame.IsRecord, err = tablecodec.DecodeKeyHead(key)
if err == nil {
if frame.IsRecord {
_, frame.RecordID, err = tablecodec.DecodeRecordKey(key)
} else {
_, _, frame.IndexValues, err = tablecodec.DecodeIndexKey(key)
}
log.Warnf("decode region key %q fail: %v", key, err)
// Ignore decode errors.
err = nil
return
}
if bytes.HasPrefix(key, tablecodec.TablePrefix()) {
Expand Down Expand Up @@ -817,9 +827,11 @@ func NewRegionFrameRange(region *tikv.KeyLocation) (idxRange *RegionFrameRange,
// are not covered by this frame range, it returns nil.
func (r *RegionFrameRange) getRecordFrame(tableID int64, dbName, tableName string) *FrameItem {
if tableID == r.first.TableID && r.first.IsRecord {
r.first.DBName, r.first.TableName = dbName, tableName
return r.first
}
if tableID == r.last.TableID && r.last.IsRecord {
r.last.DBName, r.last.TableName = dbName, tableName
return r.last
}

Expand All @@ -836,11 +848,13 @@ func (r *RegionFrameRange) getRecordFrame(tableID int64, dbName, tableName strin

// getIndexFrame returns the indnex frame of a table. If the table's indices are
// not covered by this frame range, it returns nil.
func (r *RegionFrameRange) getIndexFrame(tableID, indexID int64, dbName, tableName, indexname string) *FrameItem {
func (r *RegionFrameRange) getIndexFrame(tableID, indexID int64, dbName, tableName, indexName string) *FrameItem {
if tableID == r.first.TableID && !r.first.IsRecord && indexID == r.first.IndexID {
r.first.DBName, r.first.TableName, r.first.IndexName = dbName, tableName, indexName
return r.first
}
if tableID == r.last.TableID && indexID == r.last.IndexID {
r.last.DBName, r.last.TableName, r.last.IndexName = dbName, tableName, indexName
return r.last
}

Expand All @@ -852,7 +866,7 @@ func (r *RegionFrameRange) getIndexFrame(tableID, indexID int64, dbName, tableNa
TableName: tableName,
TableID: tableID,
IsRecord: false,
IndexName: indexname,
IndexName: indexName,
IndexID: indexID,
}
}
Expand Down
21 changes: 19 additions & 2 deletions server/http_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,9 +50,22 @@ func (ts *HTTPHandlerTestSuite) TestRegionIndexRange(c *C) {
sTableID := int64(3)
sIndex := int64(11)
eTableID := int64(9)
recordID := int64(133)
indexValues := []types.Datum{
types.NewIntDatum(100),
types.NewBytesDatum([]byte("foobar")),
types.NewFloat64Datum(-100.25),
}
var expectIndexValues []string
for _, v := range indexValues {
expectIndexValues = append(expectIndexValues, fmt.Sprintf("%d-%v", v.Kind(), v.GetValue()))
}
encodedValue, err := codec.EncodeKey(&stmtctx.StatementContext{TimeZone: time.Local}, nil, indexValues...)
c.Assert(err, IsNil)

startKey := tablecodec.EncodeTableIndexPrefix(sTableID, sIndex)
endKey := tablecodec.GenTableRecordPrefix(eTableID)
startKey := tablecodec.EncodeIndexSeekKey(sTableID, sIndex, encodedValue)
recordPrefix := tablecodec.GenTableRecordPrefix(eTableID)
endKey := tablecodec.EncodeRecordKey(recordPrefix, recordID)

region := &tikv.KeyLocation{
Region: tikv.RegionVerID{},
Expand All @@ -63,7 +76,11 @@ func (ts *HTTPHandlerTestSuite) TestRegionIndexRange(c *C) {
c.Assert(err, IsNil)
c.Assert(r.first.IndexID, Equals, sIndex)
c.Assert(r.first.IsRecord, IsFalse)
c.Assert(r.first.RecordID, Equals, int64(0))
c.Assert(r.first.IndexValues, DeepEquals, expectIndexValues)
c.Assert(r.last.IsRecord, IsTrue)
c.Assert(r.last.RecordID, Equals, recordID)
c.Assert(r.last.IndexValues, IsNil)

testCases := []struct {
tableID int64
Expand Down
29 changes: 29 additions & 0 deletions tablecodec/tablecodec.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package tablecodec
import (
"bytes"
"encoding/binary"
"fmt"
"math"
"time"

Expand All @@ -31,6 +32,7 @@ import (
var (
errInvalidKey = terror.ClassXEval.New(codeInvalidKey, "invalid key")
errInvalidRecordKey = terror.ClassXEval.New(codeInvalidRecordKey, "invalid record key")
errInvalidIndexKey = terror.ClassXEval.New(codeInvalidIndexKey, "invalid index key")
errInvalidColumnCount = terror.ClassXEval.New(codeInvalidColumnCount, "invalid column count")
)

Expand Down Expand Up @@ -109,6 +111,32 @@ func DecodeRecordKey(key kv.Key) (tableID int64, handle int64, err error) {
return
}

// DecodeIndexKey decodes the key and gets the tableID, indexID, indexValues.
func DecodeIndexKey(key kv.Key) (tableID int64, indexID int64, indexValues []string, err error) {
k := key

tableID, indexID, isRecord, err := DecodeKeyHead(key)
if err != nil {
return 0, 0, nil, errors.Trace(err)
}
if isRecord {
return 0, 0, nil, errInvalidIndexKey.Gen("invalid index key - %q", k)
}
key = key[prefixLen+idLen:]

for len(key) > 0 {
// FIXME: Without the schema information, we can only decode the raw kind of
// the column. For instance, MysqlTime is internally saved as uint64.
remain, d, e := codec.DecodeOne(key)
if e != nil {
return 0, 0, nil, errInvalidIndexKey.Gen("invalid index key - %q %v", k, e)
}
indexValues = append(indexValues, fmt.Sprintf("%d-%v", d.Kind(), d.GetValue()))
key = remain
}
return
}

// DecodeKeyHead decodes the key's head and gets the tableID, indexID. isRecordKey is true when is a record key.
func DecodeKeyHead(key kv.Key) (tableID int64, indexID int64, isRecordKey bool, err error) {
isRecordKey = false
Expand Down Expand Up @@ -566,4 +594,5 @@ const (
codeInvalidRecordKey = 4
codeInvalidColumnCount = 5
codeInvalidKey = 6
codeInvalidIndexKey = 7
)
31 changes: 31 additions & 0 deletions tablecodec/tablecodec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package tablecodec

import (
"fmt"
"math"
"testing"
"time"
Expand Down Expand Up @@ -323,3 +324,33 @@ func (s *testTableCodecSuite) TestReplaceRecordKeyTableID(c *C) {
c.Assert(err, IsNil)
c.Assert(tTableID, Equals, tableID)
}

func (s *testTableCodecSuite) TestDecodeIndexKey(c *C) {
tableID := int64(4)
indexID := int64(5)
values := []types.Datum{
types.NewIntDatum(1),
types.NewBytesDatum([]byte("abc")),
types.NewFloat64Datum(123.45),
// MysqlTime is not supported.
// types.NewTimeDatum(types.Time{
// Time: types.FromGoTime(time.Now()),
// Fsp: 6,
// Type: mysql.TypeTimestamp,
// }),
}
var valueStrs []string
for _, v := range values {
valueStrs = append(valueStrs, fmt.Sprintf("%d-%v", v.Kind(), v.GetValue()))
}
sc := &stmtctx.StatementContext{TimeZone: time.Local}
encodedValue, err := codec.EncodeKey(sc, nil, values...)
c.Assert(err, IsNil)
indexKey := EncodeIndexSeekKey(tableID, indexID, encodedValue)

decodeTableID, decodeIndexID, decodeValues, err := DecodeIndexKey(indexKey)
c.Assert(err, IsNil)
c.Assert(decodeTableID, Equals, tableID)
c.Assert(decodeIndexID, Equals, indexID)
c.Assert(decodeValues, DeepEquals, valueStrs)
}

0 comments on commit 8fa4980

Please sign in to comment.