Skip to content

Commit

Permalink
Merge branch 'same-version' of https://github.com/yujuncen/tidb into …
Browse files Browse the repository at this point in the history
…same-version
  • Loading branch information
YuJuncen committed Aug 11, 2021
2 parents 6e0175d + c210f99 commit 815b774
Show file tree
Hide file tree
Showing 14 changed files with 674 additions and 382 deletions.
13 changes: 12 additions & 1 deletion ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -326,6 +326,15 @@ func (s *testIntegrationSuite7) TestIndexLength(c *C) {
tk.MustExec("create index idx5 on idx_len(g)")
tk.MustExec("alter table idx_len add index idxg(g)")
tk.MustExec("create table idx_len1(a int(0), b timestamp(0), c datetime(0), d time(0), f float(0), g decimal(0), index(a), index(b), index(c), index(d), index(f), index(g))")

tk.MustExec("drop table idx_len;")
tk.MustExec("create table idx_len(a text, b text charset ascii, c blob, index(a(768)), index (b(3072)), index (c(3072)));")
tk.MustExec("drop table idx_len;")
tk.MustExec("create table idx_len(a text, b text charset ascii, c blob);")
tk.MustExec("alter table idx_len add index (a(768))")
tk.MustExec("alter table idx_len add index (b(3072))")
tk.MustExec("alter table idx_len add index (c(3072))")
tk.MustExec("drop table idx_len;")
}

func (s *testIntegrationSuite3) TestIssue3833(c *C) {
Expand Down Expand Up @@ -521,7 +530,9 @@ func (s *testIntegrationSuite5) TestErrnoErrorCode(c *C) {
tk.MustGetErrCode(sql, errno.ErrMultiplePriKey)
sql = "create table test_error_code_3(pt blob ,primary key (pt));"
tk.MustGetErrCode(sql, errno.ErrBlobKeyWithoutLength)
sql = "create table test_error_code_3(a text, unique (a(3073)));"
sql = "create table test_error_code_3(a text, unique (a(769)));"
tk.MustGetErrCode(sql, errno.ErrTooLongKey)
sql = "create table test_error_code_3(a text charset ascii, unique (a(3073)));"
tk.MustGetErrCode(sql, errno.ErrTooLongKey)
sql = "create table test_error_code_3(`id` int, key `primary`(`id`));"
tk.MustGetErrCode(sql, errno.ErrWrongNameForIndex)
Expand Down
11 changes: 8 additions & 3 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,13 @@ func checkIndexColumn(col *model.ColumnInfo, indexColumnLen int) error {
}
}

if types.IsString(col.FieldType.Tp) {
desc, err := charset.GetCharsetDesc(col.Charset)
if err != nil {
return err
}
indexColumnLen *= desc.Maxlen
}
// Specified length must be shorter than the max length for prefix.
if indexColumnLen > config.GetGlobalConfig().MaxIndexLength {
return errTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength)
Expand All @@ -182,15 +189,13 @@ func getIndexColumnLength(col *model.ColumnInfo, colLen int) (int, error) {
switch col.Tp {
case mysql.TypeBit:
return (length + 7) >> 3, nil
case mysql.TypeVarchar, mysql.TypeString:
case mysql.TypeVarchar, mysql.TypeString, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeBlob, mysql.TypeLongBlob:
// Different charsets occupy different numbers of bytes on each character.
desc, err := charset.GetCharsetDesc(col.Charset)
if err != nil {
return 0, errUnsupportedCharset.GenWithStackByArgs(col.Charset, col.Collate)
}
return desc.Maxlen * length, nil
case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeBlob, mysql.TypeLongBlob:
return length, nil
case mysql.TypeTiny, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeShort:
return mysql.DefaultLengthOfMysqlTypes[col.Tp], nil
case mysql.TypeFloat:
Expand Down
3 changes: 2 additions & 1 deletion executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -559,7 +559,6 @@ func (s *testInfoschemaTableSuite) TestTableSessionVar(c *C) {
}

func (s *testInfoschemaTableSuite) TestForAnalyzeStatus(c *C) {
c.Skip("Skip this unstable test(#25896) and bring it back before 2021-07-29.")
tk := testkit.NewTestKit(c, s.store)
statistics.ClearHistoryJobs()
tk.MustExec("use test")
Expand All @@ -586,6 +585,8 @@ func (s *testInfoschemaTableSuite) TestForAnalyzeStatus(c *C) {
tk.MustExec("create table t1 (a int, b int, index idx(a))")
tk.MustExec("insert into t1 values (1,2),(3,4)")
tk.MustExec("analyze table t1")
tk.MustQuery("show warnings").Check(testkit.Rows()) // no warning
c.Assert(s.dom.StatsHandle().LoadNeededHistograms(), IsNil)
tk.MustExec("CREATE ROLE r_t1 ;")
tk.MustExec("GRANT ALL PRIVILEGES ON test.t1 TO r_t1;")
tk.MustExec("GRANT r_t1 TO analyze_tester;")
Expand Down
2 changes: 1 addition & 1 deletion session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,7 @@ const (
charset TEXT NOT NULL,
collation TEXT NOT NULL,
source VARCHAR(10) NOT NULL DEFAULT 'unknown',
INDEX sql_index(original_sql(1024),default_db(1024)) COMMENT "accelerate the speed when add global binding query",
INDEX sql_index(original_sql(700),default_db(68)) COMMENT "accelerate the speed when add global binding query",
INDEX time_index(update_time) COMMENT "accelerate the speed when querying with last update time"
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;`

Expand Down
13 changes: 13 additions & 0 deletions testkit/result.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,3 +84,16 @@ func RowsWithSep(sep string, args ...string) [][]interface{} {
}
return rows
}

// Rows returns the result data.
func (res *Result) Rows() [][]interface{} {
ifacesSlice := make([][]interface{}, len(res.rows))
for i := range res.rows {
ifaces := make([]interface{}, len(res.rows[i]))
for j := range res.rows[i] {
ifaces[j] = res.rows[i][j]
}
ifacesSlice[i] = ifaces
}
return ifacesSlice
}
195 changes: 195 additions & 0 deletions testkit/testdata/testdata.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,195 @@
// Copyright 2021 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.

// +build !codes

package testdata

import (
"bytes"
"encoding/json"
"flag"
"fmt"
"io"
"os"
"path/filepath"
"reflect"
"regexp"
"runtime"
"strings"
"testing"

"github.com/pingcap/errors"
"github.com/stretchr/testify/require"
)

// record is a flag used for generate test result.
var record bool

func init() {
flag.BoolVar(&record, "record", false, "to generate test result")
}

type testCases struct {
Name string
Cases *json.RawMessage // For delayed parse.
decodedOut interface{} // For generate output.
}

// TestData stores all the data of a test suite.
type TestData struct {
input []testCases
output []testCases
filePathPrefix string
funcMap map[string]int
}

// LoadTestSuiteData loads test suite data from file.
func LoadTestSuiteData(dir, suiteName string) (res TestData, err error) {
res.filePathPrefix = filepath.Join(dir, suiteName)
res.input, err = loadTestSuiteCases(fmt.Sprintf("%s_in.json", res.filePathPrefix))
if err != nil {
return res, err
}
if record {
res.output = make([]testCases, len(res.input))
for i := range res.input {
res.output[i].Name = res.input[i].Name
}
} else {
res.output, err = loadTestSuiteCases(fmt.Sprintf("%s_out.json", res.filePathPrefix))
if err != nil {
return res, err
}
if len(res.input) != len(res.output) {
return res, errors.New(fmt.Sprintf("Number of test input cases %d does not match test output cases %d", len(res.input), len(res.output)))
}
}
res.funcMap = make(map[string]int, len(res.input))
for i, test := range res.input {
res.funcMap[test.Name] = i
if test.Name != res.output[i].Name {
return res, errors.New(fmt.Sprintf("Input name of the %d-case %s does not match output %s", i, test.Name, res.output[i].Name))
}
}
return res, nil
}

func loadTestSuiteCases(filePath string) (res []testCases, err error) {
jsonFile, err := os.Open(filePath)
if err != nil {
return res, err
}
defer func() {
if err1 := jsonFile.Close(); err == nil && err1 != nil {
err = err1
}
}()
byteValue, err := io.ReadAll(jsonFile)
if err != nil {
return res, err
}
// Remove comments, since they are not allowed in json.
re := regexp.MustCompile("(?s)//.*?\n")
err = json.Unmarshal(re.ReplaceAll(byteValue, nil), &res)
return res, err
}

// GetTestCases gets the test cases for a test function.
func (td *TestData) GetTestCases(t *testing.T, in interface{}, out interface{}) {
// Extract caller's name.
pc, _, _, ok := runtime.Caller(1)
require.True(t, ok)
details := runtime.FuncForPC(pc)
funcNameIdx := strings.LastIndex(details.Name(), ".")
funcName := details.Name()[funcNameIdx+1:]

casesIdx, ok := td.funcMap[funcName]
require.Truef(t, ok, "Must get test %s", funcName)
err := json.Unmarshal(*td.input[casesIdx].Cases, in)
require.NoError(t, err)
if !record {
err = json.Unmarshal(*td.output[casesIdx].Cases, out)
require.NoError(t, err)
} else {
// Init for generate output file.
inputLen := reflect.ValueOf(in).Elem().Len()
v := reflect.ValueOf(out).Elem()
if v.Kind() == reflect.Slice {
v.Set(reflect.MakeSlice(v.Type(), inputLen, inputLen))
}
}
td.output[casesIdx].decodedOut = out
}

// OnRecord execute the function to update result.
func (td *TestData) OnRecord(updateFunc func()) {
if record {
updateFunc()
}
}

// ConvertRowsToStrings converts [][]interface{} to []string.
func (td *TestData) ConvertRowsToStrings(rows [][]interface{}) (rs []string) {
for _, row := range rows {
s := fmt.Sprintf("%v", row)
// Trim the leftmost `[` and rightmost `]`.
s = s[1 : len(s)-1]
rs = append(rs, s)
}
return rs
}

// GenerateOutputIfNeeded generate the output file.
func (td *TestData) GenerateOutputIfNeeded() error {
if !record {
return nil
}

buf := new(bytes.Buffer)
enc := json.NewEncoder(buf)
enc.SetEscapeHTML(false)
enc.SetIndent("", " ")
for i, test := range td.output {
err := enc.Encode(test.decodedOut)
if err != nil {
return err
}
res := make([]byte, len(buf.Bytes()))
copy(res, buf.Bytes())
buf.Reset()
rm := json.RawMessage(res)
td.output[i].Cases = &rm
}
err := enc.Encode(td.output)
if err != nil {
return err
}
file, err := os.Create(fmt.Sprintf("%s_out.json", td.filePathPrefix))
if err != nil {
return err
}
defer func() {
if err1 := file.Close(); err == nil && err1 != nil {
err = err1
}
}()
_, err = file.Write(buf.Bytes())
return err
}

// Record is a temporary method for testutil to avoid "flag redefined: record" error,
// After we migrate all tests based on former testdata, we should remove testutil and this method.
func Record() bool {
return record
}
26 changes: 26 additions & 0 deletions util/math/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
// Copyright 2021 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 math

import (
"testing"

"github.com/pingcap/tidb/util/testbridge"
"go.uber.org/goleak"
)

func TestMain(m *testing.M) {
testbridge.WorkaroundGoCheckFlags()
goleak.VerifyTestMain(m)
}
61 changes: 28 additions & 33 deletions util/math/math_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,39 +18,34 @@ import (
"strconv"
"testing"

. "github.com/pingcap/check"
"github.com/stretchr/testify/require"
)

func TestT(t *testing.T) {
TestingT(t)
}

var _ = Suite(&testMath{})

type testMath struct{}

func (s *testMath) TestStrLenOfUint64Fast_RandomTestCases(c *C) {
for i := 0; i < 1000000; i++ {
num := rand.Uint64()
expected := len(strconv.FormatUint(num, 10))
actual := StrLenOfUint64Fast(num)
c.Assert(actual, Equals, expected)
}
}

func (s *testMath) TestStrLenOfUint64Fast_ManualTestCases(c *C) {
nums := [22]uint64{0,
1, 12, 123, 1234, 12345,
123456, 1234567, 12345678, 123456789, 1234567890,
1234567891, 12345678912, 123456789123, 1234567891234, 12345678912345,
123456789123456, 1234567891234567, 12345678912345678, 123456789123456789,
123456789123457890,
^uint64(0),
}

for _, num := range nums {
expected := len(strconv.FormatUint(num, 10))
actual := StrLenOfUint64Fast(num)
c.Assert(actual, Equals, expected)
}
func TestStrLenOfUint64Fast(t *testing.T) {
t.Run("RandomInput", func(t *testing.T) {
t.Parallel()
for i := 0; i < 1000000; i++ {
num := rand.Uint64()
expected := len(strconv.FormatUint(num, 10))
actual := StrLenOfUint64Fast(num)
require.Equal(t, expected, actual)
}
})

t.Run("ManualInput", func(t *testing.T) {
t.Parallel()
nums := [22]uint64{0,
1, 12, 123, 1234, 12345,
123456, 1234567, 12345678, 123456789, 1234567890,
1234567891, 12345678912, 123456789123, 1234567891234, 12345678912345,
123456789123456, 1234567891234567, 12345678912345678, 123456789123456789,
123456789123457890,
^uint64(0),
}
for _, num := range nums {
expected := len(strconv.FormatUint(num, 10))
actual := StrLenOfUint64Fast(num)
require.Equal(t, expected, actual)
}
})
}
Loading

0 comments on commit 815b774

Please sign in to comment.