Skip to content

Commit

Permalink
decode with value size (pingcap#1641)
Browse files Browse the repository at this point in the history
Add size parameter for Decode to reduce slice growth
BenchmarkDecodeWithSize-8 300000 6153 ns/op
BenchmarkDecodeWithOutSize-8 200000 11934 ns/op
  • Loading branch information
shenli authored Aug 26, 2016
1 parent 2b5d2b1 commit 558ba06
Show file tree
Hide file tree
Showing 10 changed files with 77 additions and 14 deletions.
13 changes: 13 additions & 0 deletions bench_test.go
Original file line number Diff line number Diff line change
@@ -1,3 +1,16 @@
// Copyright 2016 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package tidb

import (
Expand Down
2 changes: 1 addition & 1 deletion plan/statistics/statistics.go
Original file line number Diff line number Diff line change
Expand Up @@ -346,7 +346,7 @@ func TableFromPB(ti *model.TableInfo, tpb *TablePB) (*Table, error) {
t.Columns = make([]*Column, len(tpb.GetColumns()))
for i, cInfo := range t.info.Columns {
cpb := tpb.Columns[i]
values, err := codec.Decode(cpb.GetValue())
values, err := codec.Decode(cpb.GetValue(), 1)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion store/localstore/xapi_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ func (s *testXAPISuite) TestSelect(c *C) {
handles := make([]int, 0, 10)
for _, row := range idxResp.Rows {
var err error
datums, err := codec.Decode(row.Handle)
datums, err := codec.Decode(row.Handle, 1)
c.Check(err, IsNil)
c.Check(datums, HasLen, 1)
handles = append(handles, int(datums[0].GetInt64()))
Expand Down
2 changes: 1 addition & 1 deletion table/tables/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ func (c *indexIter) Next() (val []types.Datum, h int64, err error) {
}
// get indexedValues
buf := c.it.Key()[len(c.prefix):]
vv, err := codec.Decode(buf)
vv, err := codec.Decode(buf, len(c.idx.idxInfo.Columns))
if err != nil {
return nil, 0, errors.Trace(err)
}
Expand Down
4 changes: 2 additions & 2 deletions tablecodec/tablecodec.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,7 @@ func DecodeValues(data []byte, fts []*types.FieldType, inIndex bool) ([]types.Da
if data == nil {
return nil, nil
}
values, err := codec.Decode(data)
values, err := codec.Decode(data, len(fts))
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -348,7 +348,7 @@ func EncodeIndexSeekKey(tableID int64, idxID int64, encodedValue []byte) kv.Key
// DecodeIndexKey decodes datums from an index key.
func DecodeIndexKey(key kv.Key) ([]types.Datum, error) {
b := key[prefixLen+idLen:]
return codec.Decode(b)
return codec.Decode(b, 1)
}

// CutIndexKey cuts encoded index key into colIDs to bytes slices map.
Expand Down
49 changes: 49 additions & 0 deletions util/codec/bench_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
// Copyright 2016 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package codec

import (
"testing"

"github.com/pingcap/tidb/util/types"
)

var valueCnt = 100

func composeEncodedData(size int) []byte {
values := make([]types.Datum, 0, size)
for i := 0; i < size; i++ {
values = append(values, types.NewDatum(i))
}
bs, _ := EncodeValue(nil, values...)
return bs
}

func BenchmarkDecodeWithSize(b *testing.B) {
b.StopTimer()
bs := composeEncodedData(valueCnt)
b.StartTimer()
for i := 0; i < b.N; i++ {
Decode(bs, valueCnt)
}
}

func BenchmarkDecodeWithOutSize(b *testing.B) {
b.StopTimer()
bs := composeEncodedData(valueCnt)
b.StartTimer()
for i := 0; i < b.N; i++ {
Decode(bs, 1)
}
}
5 changes: 3 additions & 2 deletions util/codec/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,14 +129,15 @@ func EncodeValue(b []byte, v ...types.Datum) ([]byte, error) {

// Decode decodes values from a byte slice generated with EncodeKey or EncodeValue
// before.
func Decode(b []byte) ([]types.Datum, error) {
// size is the size of decoded datum slice.
func Decode(b []byte, size int) ([]types.Datum, error) {
if len(b) < 1 {
return nil, errors.New("invalid encoded key")
}

var (
err error
values = make([]types.Datum, 0, 1)
values = make([]types.Datum, 0, size)
)

for len(b) > 0 {
Expand Down
10 changes: 5 additions & 5 deletions util/codec/codec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,13 +78,13 @@ func (s *testCodecSuite) TestCodecKey(c *C) {
comment := Commentf("%d %v", i, t)
b, err := EncodeKey(nil, t.Input...)
c.Assert(err, IsNil, comment)
args, err := Decode(b)
args, err := Decode(b, 1)
c.Assert(err, IsNil)
c.Assert(args, DeepEquals, t.Expect)

b, err = EncodeValue(nil, t.Input...)
c.Assert(err, IsNil)
args, err = Decode(b)
args, err = Decode(b, 1)
c.Assert(err, IsNil)
c.Assert(args, DeepEquals, t.Expect)
}
Expand Down Expand Up @@ -492,7 +492,7 @@ func (s *testCodecSuite) TestTime(c *C) {

b, err := EncodeKey(nil, m)
c.Assert(err, IsNil)
v, err := Decode(b)
v, err := Decode(b, 1)
c.Assert(err, IsNil)
var t mysql.Time
t.Type = mysql.TypeDatetime
Expand Down Expand Up @@ -537,7 +537,7 @@ func (s *testCodecSuite) TestDuration(c *C) {

b, err := EncodeKey(nil, types.NewDatum(m))
c.Assert(err, IsNil)
v, err := Decode(b)
v, err := Decode(b, 1)
c.Assert(err, IsNil)
m.Fsp = mysql.MaxFsp
c.Assert(v, DeepEquals, types.MakeDatums(m))
Expand Down Expand Up @@ -592,7 +592,7 @@ func (s *testCodecSuite) TestDecimal(c *C) {
c.Assert(err, IsNil)
b, err := EncodeKey(nil, types.NewDatum(dec))
c.Assert(err, IsNil)
v, err := Decode(b)
v, err := Decode(b, 1)
c.Assert(err, IsNil)
c.Assert(v, HasLen, 1)
vv := v[0].GetMysqlDecimal()
Expand Down
2 changes: 1 addition & 1 deletion xapi/xapi.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ func (pr *partialResult) Next() (handle int64, data []types.Datum, err error) {
}
if !pr.aggregate {
handleBytes := row.GetHandle()
datums, err := codec.Decode(handleBytes)
datums, err := codec.Decode(handleBytes, 1)
if err != nil {
return 0, nil, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion xapi/xeval/eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -530,7 +530,7 @@ func (e *Evaluator) decodeValueList(valueListExpr *tipb.Expr) (*decodedValueList
if decoded != nil {
return decoded, nil
}
list, err := codec.Decode(valueListExpr.Val)
list, err := codec.Decode(valueListExpr.Val, 1)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down

0 comments on commit 558ba06

Please sign in to comment.