Skip to content

Commit

Permalink
stats: limit the length of sample values (#7931) (#7982)
Browse files Browse the repository at this point in the history
  • Loading branch information
alivxxx authored and zz-jason committed Oct 23, 2018
1 parent 65f77f7 commit c2c7d3d
Show file tree
Hide file tree
Showing 3 changed files with 34 additions and 7 deletions.
21 changes: 21 additions & 0 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,12 @@ package executor_test

import (
"fmt"
"strings"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/testkit"
Expand Down Expand Up @@ -84,3 +87,21 @@ func (s *testSuite) TestAnalyzeParameters(c *C) {
tbl = s.domain.StatsHandle().GetTableStats(tableInfo)
c.Assert(tbl.Columns[1].Len(), Equals, 4)
}

func (s *testSuite) TestAnalyzeTooLongColumns(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a json)")
value := fmt.Sprintf(`{"x":"%s"}`, strings.Repeat("x", mysql.MaxFieldVarCharLength))
tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value))

tk.MustExec("analyze table t")
is := executor.GetInfoSchema(tk.Se.(sessionctx.Context))
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
tbl := s.domain.StatsHandle().GetTableStats(tableInfo)
c.Assert(tbl.Columns[1].Len(), Equals, 0)
c.Assert(tbl.Columns[1].TotColSize, Equals, int64(65559))
}
12 changes: 6 additions & 6 deletions statistics/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,19 +96,19 @@ func (b *SortedBuilder) Iterate(data types.Datum) error {
// BuildColumn builds histogram from samples for column.
func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType) (*Histogram, error) {
count := collector.Count
if count == 0 {
return &Histogram{ID: id, NullCount: collector.NullCount}, nil
ndv := collector.FMSketch.NDV()
if ndv > count {
ndv = count
}
if count == 0 || len(collector.Samples) == 0 {
return NewHistogram(id, ndv, collector.NullCount, 0, tp, 0, collector.TotalSize), nil
}
sc := ctx.GetSessionVars().StmtCtx
samples := collector.Samples
err := types.SortDatums(sc, samples)
if err != nil {
return nil, errors.Trace(err)
}
ndv := collector.FMSketch.NDV()
if ndv > count {
ndv = count
}
hg := NewHistogram(id, ndv, collector.NullCount, 0, tp, int(numBuckets), collector.TotalSize)

sampleNum := int64(len(samples))
Expand Down
8 changes: 7 additions & 1 deletion statistics/sample.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"math/rand"

"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -73,6 +74,8 @@ func SampleCollectorToProto(c *SampleCollector) *tipb.SampleCollector {
return collector
}

const maxSampleValueLength = mysql.MaxFieldVarCharLength / 2

// SampleCollectorFromProto converts SampleCollector from its protobuf representation.
func SampleCollectorFromProto(collector *tipb.SampleCollector) *SampleCollector {
s := &SampleCollector{
Expand All @@ -85,7 +88,10 @@ func SampleCollectorFromProto(collector *tipb.SampleCollector) *SampleCollector
}
s.CMSketch = CMSketchFromProto(collector.CmSketch)
for _, val := range collector.Samples {
s.Samples = append(s.Samples, types.NewBytesDatum(val))
// When store the histogram bucket boundaries to kv, we need to limit the length of the value.
if len(val) <= maxSampleValueLength {
s.Samples = append(s.Samples, types.NewBytesDatum(val))
}
}
return s
}
Expand Down

0 comments on commit c2c7d3d

Please sign in to comment.