Skip to content

Commit

Permalink
util: Fix lock view cannot show key info correctly for partition table (
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-srebot authored Aug 12, 2021
1 parent 30cf15a commit 04d9017
Show file tree
Hide file tree
Showing 2 changed files with 111 additions and 19 deletions.
39 changes: 31 additions & 8 deletions util/keydecoder/keydecoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"fmt"

"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/tablecodec"
Expand All @@ -41,6 +42,8 @@ type DecodedKey struct {
DbName string `json:"db_name,omitempty"`
TableID int64 `json:"table_id"`
TableName string `json:"table_name,omitempty"`
PartitionID int64 `json:"partition_id,omitempty"`
PartitionName string `json:"partition_name,omitempty"`
HandleType HandleType `json:"handle_type,omitempty"`
IsPartitionHandle bool `json:"partition_handle,omitempty"`
HandleValue string `json:"handle_value,omitempty"`
Expand All @@ -60,7 +63,6 @@ func handleType(handle kv.Handle) HandleType {
return handleType(h.Handle)
} else {
logutil.BgLogger().Warn("Unexpected kv.Handle type",
zap.Any("handle", handle),
zap.String("handle Type", fmt.Sprintf("%T", handle)),
)
}
Expand All @@ -73,13 +75,13 @@ func DecodeKey(key []byte, infoschema infoschema.InfoSchema) (DecodedKey, error)
if !tablecodec.IsRecordKey(key) && !tablecodec.IsIndexKey(key) {
return result, errors.Errorf("Unknown key type for key %v", key)
}
tableID, indexID, isRecordKey, err := tablecodec.DecodeKeyHead(key)
tableOrPartitionID, indexID, isRecordKey, err := tablecodec.DecodeKeyHead(key)
if err != nil {
return result, err
}
result.TableID = tableID
result.TableID = tableOrPartitionID

table, tableFound := infoschema.TableByID(tableID)
table, tableFound := infoschema.TableByID(tableOrPartitionID)

// The schema may have changed since when the key is get.
// Then we just omit the table name and show the table ID only.
Expand All @@ -89,21 +91,42 @@ func DecodeKey(key []byte, infoschema infoschema.InfoSchema) (DecodedKey, error)

schema, ok := infoschema.SchemaByTable(table.Meta())
if !ok {
logutil.BgLogger().Warn("no schema associated with table found in infoschema", zap.Int64("tableID", tableID), zap.Error(err))
logutil.BgLogger().Warn("no schema associated with table found in infoschema", zap.Int64("tableOrPartitionID", tableOrPartitionID))
return result, nil
}
result.DbID = schema.ID
result.DbName = schema.Name.O
} else {
logutil.BgLogger().Warn("no table found in infoschema", zap.Int64("tableID", tableID), zap.Error(err))
// If the table of this ID is not found, try to find it as a partition.
var schema *model.DBInfo
var partition *model.PartitionDefinition
table, schema, partition = infoschema.FindTableByPartitionID(tableOrPartitionID)
if table != nil {
tableFound = true
result.TableID = table.Meta().ID
result.TableName = table.Meta().Name.O
}
if schema != nil {
result.DbID = schema.ID
result.DbName = schema.Name.O
}
if partition != nil {
result.PartitionID = partition.ID
result.PartitionName = partition.Name.O
}
if !tableFound {
logutil.BgLogger().Warn("no table found in infoschema", zap.Int64("tableOrPartitionID", tableOrPartitionID))
}
}
if isRecordKey {
_, handle, err := tablecodec.DecodeRecordKey(key)
if err != nil {
logutil.BgLogger().Warn("decode record key failed", zap.Int64("tableID", tableID), zap.Error(err))
return result, errors.Errorf("cannot decode record key of table %d", tableID)
logutil.BgLogger().Warn("decode record key failed", zap.Int64("tableOrPartitionID", tableOrPartitionID), zap.Error(err))
return result, errors.Errorf("cannot decode record key of table %d", tableOrPartitionID)
}
result.HandleType = handleType(handle)
// The PartitionHandle is used by the Global Index feature for partition tables, which is currently an
// unfinished feature. So we don't care about it much for now.
_, result.IsPartitionHandle = handle.(kv.PartitionHandle)
result.HandleValue = handle.String()
} else {
Expand Down
91 changes: 80 additions & 11 deletions util/keydecoder/keydecoder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,27 @@ func TestDecodeKey(t *testing.T) {
},
}
tableInfo2 := &model.TableInfo{ID: 2, Name: model.NewCIStr("table2")}
stubTableInfos := []*model.TableInfo{tableInfo1, tableInfo2}
tableInfo3 := &model.TableInfo{
ID: 3,
Name: model.NewCIStr("table3"),
Columns: []*model.ColumnInfo{
{ID: 10, Name: model.NewCIStr("col"), State: model.StatePublic},
},
Indices: []*model.IndexInfo{
{ID: 4, Name: model.NewCIStr("index4"), State: model.StatePublic},
},
Partition: &model.PartitionInfo{
Type: model.PartitionTypeRange,
Expr: "`col`",
Enable: true,
Definitions: []model.PartitionDefinition{
{ID: 5, Name: model.NewCIStr("p0"), LessThan: []string{"10"}},
{ID: 6, Name: model.NewCIStr("p1"), LessThan: []string{"MAXVALUE"}},
},
},
}

stubTableInfos := []*model.TableInfo{tableInfo1, tableInfo2, tableInfo3}
stubInfoschema := infoschema.MockInfoSchema(stubTableInfos)

decodedKey, err := DecodeKey([]byte{
Expand All @@ -55,10 +75,12 @@ func TestDecodeKey(t *testing.T) {
assert.Equal(t, decodedKey.DbName, "test")
assert.Equal(t, decodedKey.TableID, int64(1))
assert.Equal(t, decodedKey.TableName, "table1")
assert.Equal(t, decodedKey.PartitionID, int64(0))
assert.Equal(t, decodedKey.PartitionName, "")
assert.Equal(t, decodedKey.HandleType, IntHandle)
assert.Equal(t, decodedKey.IsPartitionHandle, false)
assert.Equal(t, decodedKey.HandleValue, "1")
// these are default values, ie. will be omitted when got marshaled into json
// These are default values, ie. will be omitted when got marshaled into json
assert.Equal(t, decodedKey.IndexID, int64(0))
assert.Equal(t, decodedKey.IndexName, "")
assert.Nil(t, decodedKey.IndexValues)
Expand All @@ -80,10 +102,12 @@ func TestDecodeKey(t *testing.T) {
assert.Equal(t, decodedKey.DbName, "test")
assert.Equal(t, decodedKey.TableID, int64(2))
assert.Equal(t, decodedKey.TableName, "table2")
assert.Equal(t, decodedKey.PartitionID, int64(0))
assert.Equal(t, decodedKey.PartitionName, "")
assert.Equal(t, decodedKey.HandleType, CommonHandle)
assert.Equal(t, decodedKey.IsPartitionHandle, false)
assert.Equal(t, decodedKey.HandleValue, "{100, abc}")
// these are default values, ie. will be omitted when got marshaled into json
// These are default values, ie. will be omitted when got marshaled into json
assert.Equal(t, decodedKey.IndexID, int64(0))
assert.Equal(t, decodedKey.IndexName, "")
assert.Nil(t, decodedKey.IndexValues)
Expand All @@ -109,15 +133,58 @@ func TestDecodeKey(t *testing.T) {
assert.Equal(t, decodedKey.DbName, "test")
assert.Equal(t, decodedKey.TableID, int64(1))
assert.Equal(t, decodedKey.TableName, "table1")
assert.Equal(t, decodedKey.PartitionID, int64(0))
assert.Equal(t, decodedKey.PartitionName, "")
assert.Equal(t, decodedKey.IndexID, int64(1))
assert.Equal(t, decodedKey.IndexName, "index1")
assert.Equal(t, decodedKey.IndexValues, []string{"abc", "1"})
// these are default values, ie. will be omitted when got marshaled into json
// These are default values, ie. will be omitted when got marshaled into json
assert.Equal(t, decodedKey.HandleType, HandleType(""))
assert.Equal(t, decodedKey.HandleValue, "")
assert.Equal(t, decodedKey.IsPartitionHandle, false)

// Row key in a partitioned table.
key = []byte("t\x80\x00\x00\x00\x00\x00\x00\x05_r\x80\x00\x00\x00\x00\x00\x00\x0a")
decodedKey, err = DecodeKey(key, stubInfoschema)
assert.Nil(t, err)
assert.Equal(t, decodedKey.DbID, int64(0))
assert.Equal(t, decodedKey.DbName, "test")
assert.Equal(t, decodedKey.TableID, int64(3))
assert.Equal(t, decodedKey.TableName, "table3")
assert.Equal(t, decodedKey.PartitionID, int64(5))
assert.Equal(t, decodedKey.PartitionName, "p0")
assert.Equal(t, decodedKey.HandleType, IntHandle)
assert.Equal(t, decodedKey.HandleValue, "10")
// These are default values, ie. will be omitted when got marshaled into json
assert.Equal(t, decodedKey.IndexID, int64(0))
assert.Equal(t, decodedKey.IndexName, "")
assert.Nil(t, decodedKey.IndexValues)
assert.Equal(t, decodedKey.IsPartitionHandle, false)

// Index key in a partitioned table.
values = types.MakeDatums("abcde", 2)
encodedValue, err = codec.EncodeKey(sc, nil, values...)
assert.Nil(t, err)
key = []byte("t\x80\x00\x00\x00\x00\x00\x00\x06_i\x80\x00\x00\x00\x00\x00\x00\x04")
key = append(key, encodedValue...)

decodedKey, err = DecodeKey(key, stubInfoschema)
assert.Nil(t, err)
assert.Equal(t, decodedKey.DbID, int64(0))
assert.Equal(t, decodedKey.DbName, "test")
assert.Equal(t, decodedKey.TableID, int64(3))
assert.Equal(t, decodedKey.TableName, "table3")
assert.Equal(t, decodedKey.PartitionID, int64(6))
assert.Equal(t, decodedKey.PartitionName, "p1")
assert.Equal(t, decodedKey.IndexID, int64(4))
assert.Equal(t, decodedKey.IndexName, "index4")
assert.Equal(t, decodedKey.IndexValues, []string{"abcde", "2"})
// These are default values, ie. will be omitted when got marshaled into json
assert.Equal(t, decodedKey.HandleType, HandleType(""))
assert.Equal(t, decodedKey.HandleValue, "")
assert.Equal(t, decodedKey.IsPartitionHandle, false)

// totally invalid key
// Totally invalid key
key = []byte("this-is-a-totally-invalidkey")
decodedKey, err = DecodeKey(key, stubInfoschema)
assert.NotNil(t, err)
Expand All @@ -131,27 +198,29 @@ func TestDecodeKey(t *testing.T) {
decodedKey, err = DecodeKey(key, stubInfoschema)
assert.NotNil(t, err)

// table cannot be found in infoschema
// this is possible when the schema have changed since when the key is get.
// Table cannot be found in infoschema
// This is possible when the schema have changed since when the key is get.
decodedKey, err = DecodeKey([]byte{
't',
// table id = 3
0x80, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x03,
0x80, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x04,
'_',
'r',
// int handle, value = 1
0x80, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01,
}, stubInfoschema)
// we should get as much information as we can
// We should get as much information as we can
assert.Nil(t, err)
assert.Equal(t, decodedKey.TableID, int64(3))
assert.Equal(t, decodedKey.TableID, int64(4))
assert.Equal(t, decodedKey.HandleType, IntHandle)
assert.Equal(t, decodedKey.HandleValue, "1")

// rest information are all default value, ie. omitted when got marshaled into json
// Rest information are all default value, ie. omitted when got marshaled into json
assert.Equal(t, decodedKey.DbID, int64(0))
assert.Equal(t, decodedKey.DbName, "")
assert.Equal(t, decodedKey.TableName, "")
assert.Equal(t, decodedKey.PartitionID, int64(0))
assert.Equal(t, decodedKey.PartitionName, "")
assert.Equal(t, decodedKey.IndexID, int64(0))
assert.Equal(t, decodedKey.IndexName, "")
assert.Equal(t, decodedKey.IsPartitionHandle, false)
Expand Down

0 comments on commit 04d9017

Please sign in to comment.