Skip to content

Commit

Permalink
Merge branch 'master' into hanfei/stats
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 committed Mar 22, 2017
2 parents 6fefdd7 + 2c3e731 commit 67e1a81
Show file tree
Hide file tree
Showing 30 changed files with 331 additions and 172 deletions.
3 changes: 2 additions & 1 deletion ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -912,6 +912,7 @@ func (s *testColumnSuite) TestModifyColumn(c *C) {
{"text", "blob", false},
{"varchar(10)", "varchar(8)", false},
{"varchar(10)", "varchar(11)", true},
{"varchar(10) character set utf8 collate utf8_bin", "varchar(10) character set utf8", true},
}
for _, ca := range cases {
ftA := s.colDefStrToFieldType(c, ca.origin)
Expand All @@ -925,7 +926,7 @@ func (s *testColumnSuite) colDefStrToFieldType(c *C, str string) *types.FieldTyp
stmt, err := parser.New().ParseOneStmt(sqlA, "", "")
c.Assert(err, IsNil)
colDef := stmt.(*ast.AlterTableStmt).Specs[0].NewColumn
col, _, err := columnDefToCol(nil, 0, colDef)
col, _, err := buildColumnAndConstraint(nil, 0, colDef)
c.Assert(err, IsNil)
return &col.FieldType
}
14 changes: 14 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,18 @@ func getDefaultCharsetAndCollate() (string, string) {
return "utf8", "utf8_bin"
}

func getDefaultCollateForCharset(str string) string {
switch str {
case charset.CharsetBin:
return charset.CollationBin
case charset.CharsetUTF8MB4:
return charset.CollationUTF8MB4
case charset.CharsetUTF8:
return charset.CollationUTF8
}
return ""
}

func setColumnFlagWithConstraint(colMap map[string]*table.Column, v *ast.Constraint) {
switch v.Tp {
case ast.ConstraintPrimaryKey:
Expand Down Expand Up @@ -186,6 +198,8 @@ func setCharsetCollationFlenDecimal(tp *types.FieldType) {
tp.Charset = charset.CharsetBin
tp.Collate = charset.CharsetBin
}
} else if len(tp.Collate) == 0 {
tp.Collate = getDefaultCollateForCharset(tp.Charset)
}
// If flen is not assigned, assigned it by type.
if tp.Flen == types.UnspecifiedLength {
Expand Down
54 changes: 9 additions & 45 deletions distsql/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,8 +14,6 @@
package distsql

import (
"io"
"io/ioutil"
"time"

"github.com/juju/errors"
Expand All @@ -24,7 +22,6 @@ import (
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/bytespool"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/types"
"github.com/pingcap/tipb/go-tipb"
Expand Down Expand Up @@ -101,23 +98,21 @@ func (r *selectResult) fetch(ctx goctx.Context) {
queryHistgram.WithLabelValues(label).Observe(duration.Seconds())
}()
for {
reader, err := r.resp.Next()
resultSubset, err := r.resp.Next()
if err != nil {
r.results <- resultWithErr{err: errors.Trace(err)}
return
}
if reader == nil {
if resultSubset == nil {
return
}
pr := &partialResult{
index: r.index,
fields: r.fields,
reader: reader,
aggregate: r.aggregate,
ignoreData: r.ignoreData,
done: make(chan error, 1),
}
go pr.fetch()
pr.unmarshal(resultSubset)

select {
case r.results <- resultWithErr{result: pr}:
Expand Down Expand Up @@ -157,58 +152,32 @@ type partialResult struct {
index bool
aggregate bool
fields []*types.FieldType
reader io.ReadCloser
resp *tipb.SelectResponse
chunkIdx int
cursor int
dataOffset int64
ignoreData bool

done chan error
fetched bool
}

func (pr *partialResult) fetch() {
defer close(pr.done)
func (pr *partialResult) unmarshal(resultSubset []byte) error {
pr.resp = new(tipb.SelectResponse)
var b []byte
var err error
if rc, ok := pr.reader.(*bytespool.ReadCloser); ok {
b = rc.SharedBytes()
} else {
b, err = ioutil.ReadAll(pr.reader)
if err != nil {
pr.done <- errors.Trace(err)
return
}
}

err = pr.resp.Unmarshal(b)
err := pr.resp.Unmarshal(resultSubset)
if err != nil {
pr.done <- errors.Trace(err)
return
return errors.Trace(err)
}

if pr.resp.Error != nil {
pr.done <- errInvalidResp.Gen("[%d %s]", pr.resp.Error.GetCode(), pr.resp.Error.GetMsg())
return
return errInvalidResp.Gen("[%d %s]", pr.resp.Error.GetCode(), pr.resp.Error.GetMsg())
}

pr.done <- nil
return nil
}

var dummyData = make([]types.Datum, 0)

// Next returns the next row of the sub result.
// If no more row to return, data would be nil.
func (pr *partialResult) Next() (handle int64, data []types.Datum, err error) {
if !pr.fetched {
err = <-pr.done
pr.fetched = true
if err != nil {
return 0, nil, err
}
}
if len(pr.resp.Chunks) > 0 {
// For new resp rows structure.
chunk := pr.getChunk()
Expand Down Expand Up @@ -279,7 +248,7 @@ func (pr *partialResult) getChunk() *tipb.Chunk {

// Close closes the sub result.
func (pr *partialResult) Close() error {
return pr.reader.Close()
return nil
}

// Select do a select request, returns SelectResult.
Expand Down Expand Up @@ -356,11 +325,6 @@ func composeRequest(req *tipb.SelectRequest, keyRanges []kv.KeyRange, concurrenc
return kvReq, nil
}

// SupportExpression checks if the expression is supported by the client.
func SupportExpression(client kv.Client, expr *tipb.Expr) bool {
return false
}

// XAPI error codes.
const (
codeInvalidResp = 1
Expand Down
11 changes: 4 additions & 7 deletions distsql/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,7 @@
package distsql

import (
"bytes"
"errors"
"io"
"io/ioutil"
"runtime"
"testing"
"time"
Expand Down Expand Up @@ -93,23 +90,23 @@ type mockResponse struct {
count int
}

func (resp *mockResponse) Next() (io.ReadCloser, error) {
func (resp *mockResponse) Next() ([]byte, error) {
resp.count++
if resp.count == 100 {
return nil, errors.New("error happend")
}
return mockReaderCloser(), nil
return mockSubresult(), nil
}

func (resp *mockResponse) Close() error {
return nil
}

func mockReaderCloser() io.ReadCloser {
func mockSubresult() []byte {
resp := new(tipb.SelectResponse)
b, err := resp.Marshal()
if err != nil {
panic(err)
}
return ioutil.NopCloser(bytes.NewBuffer(b))
return b
}
5 changes: 5 additions & 0 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -407,6 +407,11 @@ func (do *Domain) PrivilegeHandle() *privileges.Handle {
return do.privHandle
}

// StatsHandle returns the statistic handle.
func (do *Domain) StatsHandle() *statistics.Handle {
return do.statsHandle
}

func (do *Domain) loadTableStats() error {
ver, err := do.store.CurrentVersion()
if err != nil {
Expand Down
11 changes: 6 additions & 5 deletions expression/builtin_miscellaneous.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,12 @@
package expression

import (
"net"
"time"

"github.com/juju/errors"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/util/types"
"github.com/twinj/uuid"
"net"
"time"
)

var (
Expand Down Expand Up @@ -450,8 +450,9 @@ type builtinUUIDSig struct {
}

// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_uuid
func (b *builtinUUIDSig) eval(row []types.Datum) (d types.Datum, err error) {
return d, errFunctionNotExists.GenByArgs("UUID")
func (b *builtinUUIDSig) eval(_ []types.Datum) (d types.Datum, err error) {
d.SetString(uuid.NewV1().String())
return
}

type uuidShortFunctionClass struct {
Expand Down
25 changes: 25 additions & 0 deletions expression/builtin_miscellaneous_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,33 @@ import (
"github.com/pingcap/tidb/util/testleak"
"github.com/pingcap/tidb/util/testutil"
"github.com/pingcap/tidb/util/types"
"strings"
)

func (s *testEvaluatorSuite) TestUUID(c *C) {
defer testleak.AfterTest(c)()
fc := funcs[ast.UUID]
f, err := fc.getFunction(datumsToConstants(types.MakeDatums()), s.ctx)
r, err := f.eval(nil)
c.Assert(err, IsNil)
parts := strings.Split(r.GetString(), "-")
c.Assert(len(parts), Equals, 5)
for i, p := range parts {
switch i {
case 0:
c.Assert(len(p), Equals, 8)
case 1:
fallthrough
case 2:
fallthrough
case 3:
c.Assert(len(p), Equals, 4)
case 4:
c.Assert(len(p), Equals, 12)
}
}
}

func (s *testEvaluatorSuite) TestAnyValue(c *C) {
defer testleak.AfterTest(c)()

Expand Down
42 changes: 41 additions & 1 deletion expression/builtin_string.go
Original file line number Diff line number Diff line change
Expand Up @@ -1695,7 +1695,47 @@ type builtinInstrSig struct {

// See https://dev.mysql.com/doc/refman/5.6/en/string-functions.html#function_instr
func (b *builtinInstrSig) eval(row []types.Datum) (d types.Datum, err error) {
return d, errFunctionNotExists.GenByArgs("instr")
args, err := b.evalArgs(row)
if err != nil {
return d, errors.Trace(err)
}
// INSTR(str, substr)
if args[0].IsNull() || args[1].IsNull() {
return d, nil
}

var str, substr string
if str, err = args[0].ToString(); err != nil {
return d, errors.Trace(err)
}
if substr, err = args[1].ToString(); err != nil {
return d, errors.Trace(err)
}

// INSTR performs case **insensitive** search by default, while at least one argument is binary string
// we do case sensitive search.
var caseSensitive bool
if args[0].Kind() == types.KindBytes || args[1].Kind() == types.KindBytes {
caseSensitive = true
}

var pos, idx int
if caseSensitive {
idx = strings.Index(str, substr)
} else {
idx = strings.Index(strings.ToLower(str), strings.ToLower(substr))
}
if idx == -1 {
pos = 0
} else {
if caseSensitive {
pos = idx + 1
} else {
pos = utf8.RuneCountInString(str[:idx]) + 1
}
}
d.SetInt64(int64(pos))
return d, nil
}

type loadFileFunctionClass struct {
Expand Down
44 changes: 44 additions & 0 deletions expression/builtin_string_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -984,6 +984,50 @@ func (s *testEvaluatorSuite) TestLpad(c *C) {
}
}

func (s *testEvaluatorSuite) TestInstr(c *C) {
defer testleak.AfterTest(c)()
tbl := []struct {
Args []interface{}
Want interface{}
}{
{[]interface{}{"foobarbar", "bar"}, 4},
{[]interface{}{"xbar", "foobar"}, 0},

{[]interface{}{123456234, 234}, 2},
{[]interface{}{123456, 567}, 0},
{[]interface{}{1e10, 1e2}, 1},
{[]interface{}{1.234, ".234"}, 2},
{[]interface{}{1.234, ""}, 1},
{[]interface{}{"", 123}, 0},
{[]interface{}{"", ""}, 1},

{[]interface{}{"中文美好", "美好"}, 3},
{[]interface{}{"中文美好", "世界"}, 0},
{[]interface{}{"中文abc", "a"}, 3},

{[]interface{}{"live LONG and prosper", "long"}, 6},

{[]interface{}{"not BINARY string", "binary"}, 5},
{[]interface{}{[]byte("BINARY string"), []byte("binary")}, 0},
{[]interface{}{[]byte("BINARY string"), []byte("BINARY")}, 1},
{[]interface{}{[]byte("中文abc"), []byte("abc")}, 7},

{[]interface{}{"foobar", nil}, nil},
{[]interface{}{nil, "foobar"}, nil},
{[]interface{}{nil, nil}, nil},
}

Dtbl := tblToDtbl(tbl)
instr := funcs[ast.Instr]
for i, t := range Dtbl {
f, err := instr.getFunction(datumsToConstants(t["Args"]), s.ctx)
c.Assert(err, IsNil)
got, err := f.eval(nil)
c.Assert(err, IsNil)
c.Assert(got, DeepEquals, t["Want"][0], Commentf("[%d]: args: %v", i, t["Args"]))
}
}

func (s *testEvaluatorSuite) TestMakeSet(c *C) {
defer testleak.AfterTest(c)()

Expand Down
3 changes: 3 additions & 0 deletions expression/builtin_time.go
Original file line number Diff line number Diff line change
Expand Up @@ -324,6 +324,9 @@ func builtinDateFormat(args []types.Datum, ctx context.Context) (types.Datum, er
return d, errors.Trace(err)
}

if date.IsNull() {
return d, nil
}
t := date.GetMysqlTime()
str, err := t.DateFormat(args[1].GetString())
if err != nil {
Expand Down
Loading

0 comments on commit 67e1a81

Please sign in to comment.