Skip to content

Commit

Permalink
Merge branch 'master' into refine_isexist
Browse files Browse the repository at this point in the history
  • Loading branch information
Yisaer authored Dec 6, 2021
2 parents a653167 + 9c62e96 commit 6637609
Show file tree
Hide file tree
Showing 4 changed files with 47 additions and 36 deletions.
16 changes: 16 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3621,6 +3621,22 @@ func (s *testIntegrationSuite3) TestIssue29282(c *C) {
}
}

// See https://github.com/pingcap/tidb/issues/29327
func (s *testIntegrationSuite3) TestEnumDefaultValue(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1;")
tk.MustExec("CREATE TABLE `t1` ( `a` enum('','a','b') NOT NULL DEFAULT 'b' ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci;")
tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" +
" `a` enum('','a','b') COLLATE utf8mb4_0900_ai_ci NOT NULL DEFAULT 'b'\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci"))
tk.MustExec("drop table if exists t1;")
tk.MustExec("CREATE TABLE `t1` ( `a` enum('','a','b') NOT NULL DEFAULT 'b ' ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci;")
tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" +
" `a` enum('','a','b') COLLATE utf8mb4_0900_ai_ci NOT NULL DEFAULT 'b'\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci"))
}

func (s *testIntegrationSuite3) TestIssue29326(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
4 changes: 3 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -998,11 +998,13 @@ func getEnumDefaultValue(v types.Datum, col *table.Column) (string, error) {
v.SetMysqlEnum(enumVal, col.Collate)
return v.ToString()
}

str, err := v.ToString()
if err != nil {
return "", errors.Trace(err)
}
// Ref: https://dev.mysql.com/doc/refman/8.0/en/enum.html
// Trailing spaces are automatically deleted from ENUM member values in the table definition when a table is created.
str = strings.TrimRight(str, " ")
enumVal, err := types.ParseEnumName(col.Elems, str, col.Collate)
if err != nil {
return "", ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O)
Expand Down
48 changes: 14 additions & 34 deletions expression/flag_simplify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,41 +15,20 @@
package expression_test

import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testutil"
)

var _ = Suite(&testFlagSimplifySuite{})
"testing"

type testFlagSimplifySuite struct {
store kv.Storage
dom *domain.Domain
ctx sessionctx.Context
testData testutil.TestData
}
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/testkit/testdata"
)

func (s *testFlagSimplifySuite) SetUpSuite(c *C) {
var err error
s.store, s.dom, err = newStoreWithBootstrap()
c.Assert(err, IsNil)
s.ctx = mock.NewContext()
s.testData, err = testutil.LoadTestSuiteData("testdata", "flag_simplify")
c.Assert(err, IsNil)
}
func TestSimplifyExpressionByFlag(t *testing.T) {
t.Parallel()

func (s *testFlagSimplifySuite) TearDownSuite(c *C) {
c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil)
s.dom.Close()
s.store.Close()
}
store, clean := testkit.CreateMockStore(t)
defer clean()

func (s *testFlagSimplifySuite) TestSimplifyExpressionByFlag(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(id int primary key, a bigint unsigned not null, b bigint unsigned)")
Expand All @@ -59,11 +38,12 @@ func (s *testFlagSimplifySuite) TestSimplifyExpressionByFlag(c *C) {
SQL string
Plan []string
}
s.testData.GetTestCases(c, &input, &output)
flagSimplifyData := expression.GetFlagSimplifyData()
flagSimplifyData.GetTestCases(t, &input, &output)
for i, tt := range input {
s.testData.OnRecord(func() {
testdata.OnRecord(func() {
output[i].SQL = tt
output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows())
output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows())
})
tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...))
}
Expand Down
15 changes: 14 additions & 1 deletion expression/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"time"

"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/testkit/testdata"
"github.com/pingcap/tidb/testkit/testmain"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testbridge"
Expand All @@ -28,6 +29,8 @@ import (
"go.uber.org/goleak"
)

var testDataMap = make(testdata.BookKeeper)

func TestMain(m *testing.M) {
testbridge.WorkaroundGoCheckFlags()
testmain.ShortCircuitForBench(m)
Expand All @@ -45,13 +48,19 @@ func TestMain(m *testing.M) {
// Note, SetSystemTZ() is a sync.Once operation.
timeutil.SetSystemTZ("system")

testDataMap.LoadTestSuiteData("testdata", "flag_simplify")

opts := []goleak.Option{
goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"),
goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"),
goleak.IgnoreTopFunction("github.com/pingcap/tidb/table/tables.mockRemoteService"),
}

goleak.VerifyTestMain(m, opts...)
callback := func(i int) int {
testDataMap.GenerateOutputIfNeeded()
return i
}
goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...)
}

func createContext(t *testing.T) *mock.Context {
Expand All @@ -63,3 +72,7 @@ func createContext(t *testing.T) *mock.Context {
ctx.GetSessionVars().PlanColumnID = 0
return ctx
}

func GetFlagSimplifyData() testdata.TestData {
return testDataMap["flag_simplify"]
}

0 comments on commit 6637609

Please sign in to comment.