diff --git a/pkg/executor/insert_test.go b/pkg/executor/insert_test.go index aef0018a08660..6b0733ed5426f 100644 --- a/pkg/executor/insert_test.go +++ b/pkg/executor/insert_test.go @@ -30,12 +30,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestInsertOnDuplicateKey(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - testInsertOnDuplicateKey(t, tk) -} - func TestInsertOnDuplicateKeyWithBinlog(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/executor/internal/BUILD.bazel b/pkg/executor/internal/BUILD.bazel index 240510f070017..e69de29bb2d1d 100644 --- a/pkg/executor/internal/BUILD.bazel +++ b/pkg/executor/internal/BUILD.bazel @@ -1,9 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "internal", - srcs = ["testkit.go"], - importpath = "github.com/pingcap/tidb/pkg/executor/internal", - visibility = ["//pkg/executor:__subpackages__"], - deps = ["//pkg/testkit"], -) diff --git a/pkg/executor/internal/testkit.go b/pkg/executor/internal/testkit.go deleted file mode 100644 index 96c1ecdbfa8bc..0000000000000 --- a/pkg/executor/internal/testkit.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2023 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package internal - -import ( - "fmt" - - "github.com/pingcap/tidb/pkg/testkit" -) - -// FillData fill data into table -func FillData(tk *testkit.TestKit, table string) { - tk.MustExec("use test") - tk.MustExec(fmt.Sprintf("create table %s(id int not null default 1, name varchar(255), PRIMARY KEY(id));", table)) - - // insert data - tk.MustExec(fmt.Sprintf("insert INTO %s VALUES (1, \"hello\");", table)) - tk.MustExec(fmt.Sprintf("insert into %s values (2, \"hello\");", table)) -} diff --git a/pkg/executor/stale_txn_test.go b/pkg/executor/stale_txn_test.go index 4b30125453be9..8a880e31febb6 100644 --- a/pkg/executor/stale_txn_test.go +++ b/pkg/executor/stale_txn_test.go @@ -1315,21 +1315,6 @@ func TestPlanCacheWithStaleReadByBinaryProto(t *testing.T) { tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) } -func TestIssue33728(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (id int primary key, v int)") - err := tk.ExecToErr("select * from t1 as of timestamp NULL") - require.Error(t, err) - require.Equal(t, "[planner:8135]invalid as of timestamp: as of timestamp cannot be NULL", err.Error()) - - err = tk.ExecToErr("start transaction read only as of timestamp NULL") - require.Error(t, err) - require.Equal(t, "[planner:8135]invalid as of timestamp: as of timestamp cannot be NULL", err.Error()) -} - func TestStalePrepare(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/executor/test/executor/BUILD.bazel b/pkg/executor/test/executor/BUILD.bazel index 8046b5d761d57..ba040bee380cf 100644 --- a/pkg/executor/test/executor/BUILD.bazel +++ b/pkg/executor/test/executor/BUILD.bazel @@ -8,7 +8,7 @@ go_test( "main_test.go", ], flaky = True, - shard_count = 50, + shard_count = 46, deps = [ "//pkg/config", "//pkg/ddl", @@ -32,7 +32,6 @@ go_test( "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", "//pkg/sessiontxn", - "//pkg/store/driver/error", "//pkg/store/mockstore", "//pkg/table/tables", "//pkg/tablecodec", diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index 54a3c0a1b79e7..15eec21cdcbf8 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -52,7 +52,6 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" - error2 "github.com/pingcap/tidb/pkg/store/driver/error" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -633,117 +632,6 @@ func TestTiDBLastQueryInfo(t *testing.T) { tk.MustExec("rollback") } -func TestSelectForUpdate(t *testing.T) { - store := testkit.CreateMockStore(t) - - setTxnTk := testkit.NewTestKit(t, store) - setTxnTk.MustExec("set global tidb_txn_mode=''") - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("use test") - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("use test") - - tk.MustExec("drop table if exists t, t1") - - txn, err := tk.Session().Txn(true) - require.True(t, kv.ErrInvalidTxn.Equal(err)) - require.False(t, txn.Valid()) - tk.MustExec("create table t (c1 int, c2 int, c3 int)") - tk.MustExec("insert t values (11, 2, 3)") - tk.MustExec("insert t values (12, 2, 3)") - tk.MustExec("insert t values (13, 2, 3)") - - tk.MustExec("create table t1 (c1 int)") - tk.MustExec("insert t1 values (11)") - - // conflict - tk1.MustExec("begin") - tk1.MustQuery("select * from t where c1=11 for update") - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=211 where c1=11") - tk2.MustExec("commit") - - err = tk1.ExecToErr("commit") - require.Error(t, err) - - // no conflict for subquery. - tk1.MustExec("begin") - tk1.MustQuery("select * from t where exists(select null from t1 where t1.c1=t.c1) for update") - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=211 where c1=12") - tk2.MustExec("commit") - - tk1.MustExec("commit") - - // not conflict - tk1.MustExec("begin") - tk1.MustQuery("select * from t where c1=11 for update") - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=22 where c1=12") - tk2.MustExec("commit") - - tk1.MustExec("commit") - - // not conflict, auto commit - tk1.MustExec("set @@autocommit=1;") - tk1.MustQuery("select * from t where c1=11 for update") - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=211 where c1=11") - tk2.MustExec("commit") - - tk1.MustExec("commit") - - // conflict - tk1.MustExec("begin") - tk1.MustQuery("select * from (select * from t for update) t join t1 for update") - - tk2.MustExec("begin") - tk2.MustExec("update t1 set c1 = 13") - tk2.MustExec("commit") - - err = tk1.ExecToErr("commit") - require.Error(t, err) -} - -func TestSelectForUpdateOf(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("use test") - - tk.MustExec("drop table if exists t, t1") - tk.MustExec("create table t (i int)") - tk.MustExec("create table t1 (i int)") - tk.MustExec("insert t values (1)") - tk.MustExec("insert t1 values (1)") - - tk.MustExec("begin pessimistic") - tk.MustQuery("select * from t, t1 where t.i = t1.i for update of t").Check(testkit.Rows("1 1")) - - tk1.MustExec("begin pessimistic") - - // no lock for t - tk1.MustQuery("select * from t1 for update").Check(testkit.Rows("1")) - - // meet lock for t1 - err := tk1.ExecToErr("select * from t for update nowait") - require.True(t, terror.ErrorEqual(err, error2.ErrLockAcquireFailAndNoWaitSet), fmt.Sprintf("err: %v", err)) - - // t1 rolled back, tk1 acquire the lock - tk.MustExec("rollback") - tk1.MustQuery("select * from t for update nowait").Check(testkit.Rows("1")) - - tk1.MustExec("rollback") -} - func TestPartitionHashCode(t *testing.T) { store := testkit.CreateMockStore(t) @@ -763,37 +651,6 @@ func TestPartitionHashCode(t *testing.T) { wg.Wait() } -func TestIndexLookupRuntimeStats(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (a int, b int, index(a))") - tk.MustExec("insert into t1 values (1,2),(2,3),(3,4)") - rows := tk.MustQuery("explain analyze select * from t1 use index(a) where a > 1").Rows() - require.Len(t, rows, 3) - explain := fmt.Sprintf("%v", rows[0]) - require.Regexp(t, ".*time:.*loops:.*index_task:.*table_task: {total_time.*num.*concurrency.*}.*", explain) - indexExplain := fmt.Sprintf("%v", rows[1]) - tableExplain := fmt.Sprintf("%v", rows[2]) - require.Regexp(t, ".*time:.*loops:.*cop_task:.*", indexExplain) - require.Regexp(t, ".*time:.*loops:.*cop_task:.*", tableExplain) -} - -func TestHashAggRuntimeStats(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (a int, b int)") - tk.MustExec("insert into t1 values (1,2),(2,3),(3,4)") - rows := tk.MustQuery("explain analyze SELECT /*+ HASH_AGG() */ count(*) FROM t1 WHERE a < 10;").Rows() - require.Len(t, rows, 5) - explain := fmt.Sprintf("%v", rows[0]) - pattern := ".*time:.*loops:.*partial_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*final_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*" - require.Regexp(t, pattern, explain) -} - func TestPrevStmtDesensitization(t *testing.T) { store := testkit.CreateMockStore(t) @@ -1681,24 +1538,6 @@ func TestAdminShowDDLJobs(t *testing.T) { require.Equal(t, t2.In(time.UTC), tt.In(time.UTC)) } -func TestAdminShowDDLJobsRowCount(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // Test for issue: https://github.com/pingcap/tidb/issues/25968 - tk.MustExec("use test") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (id bigint key,b int);") - tk.MustExec("split table t by (10),(20),(30);") - tk.MustExec("insert into t values (0,0),(10,10),(20,20),(30,30);") - tk.MustExec("alter table t add index idx1(b);") - require.Equal(t, "4", tk.MustQuery("admin show ddl jobs 1").Rows()[0][7]) - - tk.MustExec("insert into t values (1,0),(2,10),(3,20),(4,30);") - tk.MustExec("alter table t add index idx2(b);") - require.Equal(t, "8", tk.MustQuery("admin show ddl jobs 1").Rows()[0][7]) -} - func TestAdminShowDDLJobsInfo(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2015,16 +1854,6 @@ func TestLowResolutionTSORead(t *testing.T) { tk.MustQuery("select * from low_resolution_tso").Check(testkit.Rows("2")) } -func TestStaleReadAtFutureTime(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - // Setting tx_read_ts to a time in the future will fail. (One day before the 2038 problem) - tk.MustGetErrMsg("set @@tx_read_ts = '2038-01-18 03:14:07'", "cannot set read timestamp to a future time") - // TxnReadTS Is not updated if check failed. - require.Zero(t, tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) -} - func TestAdapterStatement(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2692,40 +2521,6 @@ func TestAdmin(t *testing.T) { require.Equal(t, historyJobs2, historyJobs) } -func TestForSelectScopeInUnion(t *testing.T) { - store := testkit.CreateMockStore(t) - setTxnTk := testkit.NewTestKit(t, store) - setTxnTk.MustExec("set global tidb_txn_mode=''") - // A union B for update, the "for update" option belongs to union statement, so - // it should works on both A and B. - tk1 := testkit.NewTestKit(t, store) - tk2 := testkit.NewTestKit(t, store) - tk1.MustExec("use test") - tk1.MustExec("drop table if exists t") - tk1.MustExec("create table t(a int)") - tk1.MustExec("insert into t values (1)") - - tk1.MustExec("begin") - // 'For update' would act on the second select. - tk1.MustQuery("select 1 as a union select a from t for update") - - tk2.MustExec("use test") - tk2.MustExec("update t set a = a + 1") - - // As tk1 use select 'for update', it should detect conflict and fail. - _, err := tk1.Exec("commit") - require.Error(t, err) - - tk1.MustExec("begin") - tk1.MustQuery("select 1 as a union select a from t limit 5 for update") - tk1.MustQuery("select 1 as a union select a from t order by a for update") - - tk2.MustExec("update t set a = a + 1") - - _, err = tk1.Exec("commit") - require.Error(t, err) -} - func TestMaxOneRow(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2746,29 +2541,6 @@ func TestMaxOneRow(t *testing.T) { require.NoError(t, rs.Close()) } -func TestSummaryFailedUpdate(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int as(-a))") - tk.MustExec("insert into t(a) values(1), (3), (7)") - sm := &testkit.MockSessionManager{ - PS: make([]*util.ProcessInfo, 0), - } - tk.Session().SetSessionManager(sm) - dom.ExpensiveQueryHandle().SetSessionManager(sm) - defer tk.MustExec("SET GLOBAL tidb_mem_oom_action = DEFAULT") - tk.MustQuery("select variable_value from mysql.GLOBAL_VARIABLES where variable_name = 'tidb_mem_oom_action'").Check(testkit.Rows("LOG")) - - tk.MustExec("SET GLOBAL tidb_mem_oom_action='CANCEL'") - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil, nil)) - tk.MustExec("set @@tidb_mem_quota_query=1") - require.True(t, exeerrors.ErrMemoryExceedForQuery.Equal(tk.ExecToErr("update t set t.a = t.a - 1 where t.a in (select a from t where a < 4)"))) - tk.MustExec("set @@tidb_mem_quota_query=1000000000") - tk.MustQuery("select stmt_type from information_schema.statements_summary where digest_text = 'update `t` set `t` . `a` = `t` . `a` - ? where `t` . `a` in ( select `a` from `t` where `a` < ? )'").Check(testkit.Rows("Update")) -} - func TestIsFastPlan(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/executor/test/partitiontest/BUILD.bazel b/pkg/executor/test/partitiontest/BUILD.bazel deleted file mode 100644 index eba32c340a02d..0000000000000 --- a/pkg/executor/test/partitiontest/BUILD.bazel +++ /dev/null @@ -1,18 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_test") - -go_test( - name = "partitiontest_test", - timeout = "short", - srcs = [ - "main_test.go", - "partition_test.go", - ], - flaky = True, - race = "on", - shard_count = 4, - deps = [ - "//pkg/testkit", - "@com_github_pingcap_failpoint//:failpoint", - "@com_github_stretchr_testify//require", - ], -) diff --git a/pkg/executor/test/partitiontest/main_test.go b/pkg/executor/test/partitiontest/main_test.go deleted file mode 100644 index 139a1418e142b..0000000000000 --- a/pkg/executor/test/partitiontest/main_test.go +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright 2023 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package partitiontest diff --git a/pkg/executor/test/partitiontest/partition_test.go b/pkg/executor/test/partitiontest/partition_test.go deleted file mode 100644 index cbffd97769b9e..0000000000000 --- a/pkg/executor/test/partitiontest/partition_test.go +++ /dev/null @@ -1,449 +0,0 @@ -// Copyright 2023 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package partitiontest - -import ( - "fmt" - "testing" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/testkit" - "github.com/stretchr/testify/require" -) - -func TestPartitionedTableReplace(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - testSQL := `drop table if exists replace_test; - create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) - partition by range (id) ( - PARTITION p0 VALUES LESS THAN (3), - PARTITION p1 VALUES LESS THAN (5), - PARTITION p2 VALUES LESS THAN (7), - PARTITION p3 VALUES LESS THAN (9));` - tk.MustExec(testSQL) - testSQL = `replace replace_test (c1) values (1),(2),(NULL);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") - - errReplaceSQL := `replace replace_test (c1) values ();` - tk.MustExec("begin") - err := tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (xxx) values (3);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test_xxx (c1) values ();` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - replaceSetSQL := `replace replace_test set c1 = 3;` - tk.MustExec(replaceSetSQL) - require.Empty(t, tk.Session().LastMessage()) - - errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSetSQL = `replace replace_test set xxx = 6;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - tk.MustExec(`drop table if exists replace_test_1`) - tk.MustExec(`create table replace_test_1 (id int, c1 int) partition by range (id) ( - PARTITION p0 VALUES LESS THAN (4), - PARTITION p1 VALUES LESS THAN (6), - PARTITION p2 VALUES LESS THAN (8), - PARTITION p3 VALUES LESS THAN (10), - PARTITION p4 VALUES LESS THAN (100))`) - tk.MustExec(`replace replace_test_1 select id, c1 from replace_test;`) - require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") - - tk.MustExec(`drop table if exists replace_test_2`) - tk.MustExec(`create table replace_test_2 (id int, c1 int) partition by range (id) ( - PARTITION p0 VALUES LESS THAN (10), - PARTITION p1 VALUES LESS THAN (50), - PARTITION p2 VALUES LESS THAN (100), - PARTITION p3 VALUES LESS THAN (300))`) - tk.MustExec(`replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;`) - require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") - - errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - tk.MustExec(`drop table if exists replace_test_3`) - replaceUniqueIndexSQL := `create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by range (c2) ( - PARTITION p0 VALUES LESS THAN (4), - PARTITION p1 VALUES LESS THAN (7), - PARTITION p2 VALUES LESS THAN (11))` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=8;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=8;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - replaceUniqueIndexSQL = `replace into replace_test_3 set c1=8, c2=8;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by range (c1) ( - PARTITION p0 VALUES LESS THAN (4), - PARTITION p1 VALUES LESS THAN (7), - PARTITION p2 VALUES LESS THAN (11));` - tk.MustExec(`drop table if exists replace_test_4`) - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by range (c2) ( - PARTITION p0 VALUES LESS THAN (4), - PARTITION p1 VALUES LESS THAN (7), - PARTITION p2 VALUES LESS THAN (11));` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - issue989SQL := `CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by range (b) ( - PARTITION p1 VALUES LESS THAN (100), - PARTITION p2 VALUES LESS THAN (200))` - tk.MustExec(issue989SQL) - issue989SQL = `insert into tIssue989 (a, b) values (1, 2);` - tk.MustExec(issue989SQL) - issue989SQL = `replace into tIssue989(a, b) values (111, 2);` - tk.MustExec(issue989SQL) - r := tk.MustQuery("select * from tIssue989;") - r.Check(testkit.Rows("111 2")) -} - -func TestHashPartitionedTableReplace(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@session.tidb_enable_table_partition = '1';") - tk.MustExec("drop table if exists replace_test;") - testSQL := `create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) - partition by hash(id) partitions 4;` - tk.MustExec(testSQL) - - testSQL = `replace replace_test (c1) values (1),(2),(NULL);` - tk.MustExec(testSQL) - - errReplaceSQL := `replace replace_test (c1) values ();` - tk.MustExec("begin") - err := tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (xxx) values (3);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test_xxx (c1) values ();` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSetSQL = `replace replace_test set xxx = 6;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - tk.MustExec(`replace replace_test set c1 = 3;`) - tk.MustExec(`replace replace_test set c1 = 4;`) - tk.MustExec(`replace replace_test set c1 = 5;`) - tk.MustExec(`replace replace_test set c1 = 6;`) - tk.MustExec(`replace replace_test set c1 = 7;`) - - tk.MustExec(`drop table if exists replace_test_1`) - tk.MustExec(`create table replace_test_1 (id int, c1 int) partition by hash(id) partitions 5;`) - tk.MustExec(`replace replace_test_1 select id, c1 from replace_test;`) - - tk.MustExec(`drop table if exists replace_test_2`) - tk.MustExec(`create table replace_test_2 (id int, c1 int) partition by hash(id) partitions 6;`) - - tk.MustExec(`replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;`) - - errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - tk.MustExec(`drop table if exists replace_test_3`) - replaceUniqueIndexSQL := `create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by hash(c2) partitions 7;` - tk.MustExec(replaceUniqueIndexSQL) - - tk.MustExec(`replace into replace_test_3 set c2=8;`) - tk.MustExec(`replace into replace_test_3 set c2=8;`) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - tk.MustExec(`replace into replace_test_3 set c1=8, c2=8;`) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - - tk.MustExec(`replace into replace_test_3 set c2=NULL;`) - tk.MustExec(`replace into replace_test_3 set c2=NULL;`) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - for i := 0; i < 100; i++ { - sql := fmt.Sprintf("replace into replace_test_3 set c2=%d;", i) - tk.MustExec(sql) - } - result := tk.MustQuery("select count(*) from replace_test_3") - result.Check(testkit.Rows("102")) - - replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by hash(c1) partitions 8;` - tk.MustExec(`drop table if exists replace_test_4`) - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by hash (c2) partitions 9;` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - issue989SQL := `CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by hash (b) partitions 10;` - tk.MustExec(issue989SQL) - issue989SQL = `insert into tIssue989 (a, b) values (1, 2);` - tk.MustExec(issue989SQL) - issue989SQL = `replace into tIssue989(a, b) values (111, 2);` - tk.MustExec(issue989SQL) - r := tk.MustQuery("select * from tIssue989;") - r.Check(testkit.Rows("111 2")) -} - -func TestPartitionedTableUpdate(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec(`create table t (id int not null default 1, name varchar(255)) - PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) - - tk.MustExec(`insert INTO t VALUES (1, "hello");`) - tk.CheckExecResult(1, 0) - tk.MustExec(`insert INTO t VALUES (7, "hello");`) - tk.CheckExecResult(1, 0) - - // update non partition column - tk.MustExec(`UPDATE t SET name = "abc" where id > 0;`) - tk.CheckExecResult(2, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - r := tk.MustQuery(`SELECT * from t order by id limit 2;`) - r.Check(testkit.Rows("1 abc", "7 abc")) - - // update partition column - tk.MustExec(`update t set id = id + 1`) - tk.CheckExecResult(2, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - r = tk.MustQuery(`SELECT * from t order by id limit 2;`) - r.Check(testkit.Rows("2 abc", "8 abc")) - - // update partition column, old and new record locates on different partitions - tk.MustExec(`update t set id = 20 where id = 8`) - tk.CheckExecResult(1, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery(`SELECT * from t order by id limit 2;`) - r.Check(testkit.Rows("2 abc", "20 abc")) - - // table option is auto-increment - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t (id int not null auto_increment, name varchar(255), primary key(id)) - PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) - - tk.MustExec("insert into t(name) values ('aa')") - tk.MustExec("update t set id = 8 where name = 'aa'") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("insert into t(name) values ('bb')") - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows("8 aa", "9 bb")) - - err := tk.ExecToErr("update t set id = null where name = 'aa'") - require.EqualError(t, err, "[table:1048]Column 'id' cannot be null") - - // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t (id int, name int unique) - PARTITION BY RANGE ( name ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) - tk.MustExec("insert t values (1, 1), (2, 2);") - err = tk.ExecToErr("update t set name = 1 where id = 2") - require.Error(t, err) - tk.MustQuery("select * from t").Check(testkit.Rows("1 1", "2 2")) - - // test update ignore for pimary key - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t(a bigint, primary key (a)) - PARTITION BY RANGE (a) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11))`) - tk.MustExec("insert into t values (5)") - tk.MustExec("insert into t values (7)") - err = tk.ExecToErr("update ignore t set a = 5 where a = 7;") - require.NoError(t, err) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '5' for key 't.PRIMARY'")) - tk.MustQuery("select * from t order by a").Check(testkit.Rows("5", "7")) - - // test update ignore for truncate as warning - err = tk.ExecToErr("update ignore t set a = 1 where a = (select '2a')") - require.NoError(t, err) - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) - - // test update ignore for unique key - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t(a bigint, unique key I_uniq (a)) - PARTITION BY RANGE (a) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11))`) - tk.MustExec("insert into t values (5)") - tk.MustExec("insert into t values (7)") - err = tk.ExecToErr("update ignore t set a = 5 where a = 7;") - require.NoError(t, err) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '5' for key 't.I_uniq'")) - tk.MustQuery("select * from t order by a").Check(testkit.Rows("5", "7")) -} - -func TestPartitionedTableDelete(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - createTable := `CREATE TABLE test.t (id int not null default 1, name varchar(255), index(id)) - PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))` - - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec(createTable) - for i := 1; i < 21; i++ { - tk.MustExec(fmt.Sprintf(`insert into t values (%d, "hello")`, i)) - } - - tk.MustExec(`delete from t where id = 2 limit 1;`) - tk.CheckExecResult(1, 0) - - // Test delete with false condition - tk.MustExec(`delete from t where 0;`) - tk.CheckExecResult(0, 0) - - tk.MustExec("insert into t values (2, 'abc')") - tk.MustExec(`delete from t where t.id = 2 limit 1`) - tk.CheckExecResult(1, 0) - - // Test delete ignore - tk.MustExec("insert into t values (2, 'abc')") - err := tk.ExecToErr("delete from t where id = (select '2a')") - require.Error(t, err) - err = tk.ExecToErr("delete ignore from t where id = (select '2a')") - require.NoError(t, err) - tk.CheckExecResult(1, 0) - r := tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) - - // Test delete without using index, involve multiple partitions. - tk.MustExec("delete from t ignore index(id) where id >= 13 and id <= 17") - tk.CheckExecResult(5, 0) - - tk.MustExec("admin check table t") - tk.MustExec(`delete from t;`) - tk.CheckExecResult(14, 0) - - // Fix that partitioned table should not use PointGetPlan. - tk.MustExec(`create table t1 (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range (c1) (partition p0 values less than (3440))`) - tk.MustExec("insert into t1 values (379, 379, 379)") - tk.MustExec("delete from t1 where c1 = 379") - tk.CheckExecResult(1, 0) - tk.MustExec(`drop table t1;`) -} diff --git a/pkg/executor/test/writetest/BUILD.bazel b/pkg/executor/test/writetest/BUILD.bazel index 1080b392de576..59e3023cb91fe 100644 --- a/pkg/executor/test/writetest/BUILD.bazel +++ b/pkg/executor/test/writetest/BUILD.bazel @@ -8,17 +8,15 @@ go_test( "write_test.go", ], flaky = True, - shard_count = 27, + shard_count = 10, deps = [ "//br/pkg/lightning/mydump", "//pkg/config", "//pkg/executor", - "//pkg/executor/internal", "//pkg/kv", "//pkg/meta/autoid", "//pkg/parser/model", "//pkg/parser/mysql", - "//pkg/planner/core", "//pkg/session", "//pkg/sessionctx", "//pkg/sessiontxn", diff --git a/pkg/executor/test/writetest/write_test.go b/pkg/executor/test/writetest/write_test.go index c8bb54ef2cabb..d279259fc0636 100644 --- a/pkg/executor/test/writetest/write_test.go +++ b/pkg/executor/test/writetest/write_test.go @@ -24,11 +24,9 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/executor" - "github.com/pingcap/tidb/pkg/executor/internal" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -41,422 +39,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestInsert(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - testSQL := `drop table if exists insert_test;create table insert_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` - tk.MustExec(testSQL) - testSQL = `insert insert_test (c1) values (1),(2),(NULL);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") - - errInsertSelectSQL := `insert insert_test (c1) values ();` - tk.MustExec("begin") - err := tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test (c1, c2) values (1,2),(1);` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test (xxx) values (3);` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test_xxx (c1) values ();` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - insertSetSQL := `insert insert_test set c1 = 3;` - tk.MustExec(insertSetSQL) - require.Empty(t, tk.Session().LastMessage()) - - errInsertSelectSQL = `insert insert_test set c1 = 4, c1 = 5;` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test set xxx = 6;` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - insertSelectSQL := `create table insert_test_1 (id int, c1 int);` - tk.MustExec(insertSelectSQL) - insertSelectSQL = `insert insert_test_1 select id, c1 from insert_test;` - tk.MustExec(insertSelectSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") - - insertSelectSQL = `create table insert_test_2 (id int, c1 int);` - tk.MustExec(insertSelectSQL) - insertSelectSQL = `insert insert_test_1 select id, c1 from insert_test union select id * 10, c1 * 10 from insert_test;` - tk.MustExec(insertSelectSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") - - errInsertSelectSQL = `insert insert_test_1 select c1 from insert_test;` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test_1 values(default, default, default, default, default)` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - // Updating column is PK handle. - // Make sure the record is "1, 1, nil, 1". - r := tk.MustQuery("select * from insert_test where id = 1;") - rowStr := fmt.Sprintf("%v %v %v %v", "1", "1", nil, "1") - r.Check(testkit.Rows(rowStr)) - insertSQL := `insert into insert_test (id, c3) values (1, 2) on duplicate key update id=values(id), c2=10;` - tk.MustExec(insertSQL) - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from insert_test where id = 1;") - rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "1") - r.Check(testkit.Rows(rowStr)) - - insertSQL = `insert into insert_test (id, c2) values (1, 1) on duplicate key update insert_test.c2=10;` - tk.MustExec(insertSQL) - require.Empty(t, tk.Session().LastMessage()) - - err = tk.ExecToErr(`insert into insert_test (id, c2) values(1, 1) on duplicate key update t.c2 = 10`) - require.Error(t, err) - - // for on duplicate key - insertSQL = `INSERT INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3;` - tk.MustExec(insertSQL) - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from insert_test where id = 1;") - rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "6") - r.Check(testkit.Rows(rowStr)) - - // for on duplicate key with ignore - insertSQL = `INSERT IGNORE INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3;` - tk.MustExec(insertSQL) - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from insert_test where id = 1;") - rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "11") - r.Check(testkit.Rows(rowStr)) - - tk.MustExec("create table insert_err (id int, c1 varchar(8))") - err = tk.ExecToErr("insert insert_err values (1, 'abcdabcdabcd')") - require.True(t, types.ErrDataTooLong.Equal(err)) - err = tk.ExecToErr("insert insert_err values (1, '你好,世界')") - require.NoError(t, err) - - tk.MustExec("create table TEST1 (ID INT NOT NULL, VALUE INT DEFAULT NULL, PRIMARY KEY (ID))") - err = tk.ExecToErr("INSERT INTO TEST1(id,value) VALUE(3,3) on DUPLICATE KEY UPDATE VALUE=4") - require.NoError(t, err) - require.Empty(t, tk.Session().LastMessage()) - - tk.MustExec("create table t (id int)") - tk.MustExec("insert into t values(1)") - tk.MustExec("update t t1 set id = (select count(*) + 1 from t t2 where t1.id = t2.id)") - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows("2")) - - // issue 3235 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c decimal(5, 5))") - err = tk.ExecToErr("insert into t value(0)") - require.NoError(t, err) - err = tk.ExecToErr("insert into t value(1)") - require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c binary(255))") - err = tk.ExecToErr("insert into t value(1)") - require.NoError(t, err) - r = tk.MustQuery("select length(c) from t;") - r.Check(testkit.Rows("255")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c varbinary(255))") - err = tk.ExecToErr("insert into t value(1)") - require.NoError(t, err) - r = tk.MustQuery("select length(c) from t;") - r.Check(testkit.Rows("1")) - - // issue 3509 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c int)") - tk.MustExec("set @origin_time_zone = @@time_zone") - tk.MustExec("set @@time_zone = '+08:00'") - err = tk.ExecToErr("insert into t value(Unix_timestamp('2002-10-27 01:00'))") - require.NoError(t, err) - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows("1035651600")) - tk.MustExec("set @@time_zone = @origin_time_zone") - - // issue 3832 - tk.MustExec("create table t1 (b char(0));") - err = tk.ExecToErr(`insert into t1 values ("");`) - require.NoError(t, err) - - // issue 3895 - tk.MustExec("USE test;") - tk.MustExec("DROP TABLE IF EXISTS t;") - tk.MustExec("CREATE TABLE t(a DECIMAL(4,2));") - tk.MustExec("INSERT INTO t VALUES (1.000001);") - r = tk.MustQuery("SHOW WARNINGS;") - // TODO: MySQL8.0 reports Note 1265 Data truncated for column 'a' at row 1 - r.Check(testkit.Rows("Warning 1366 Incorrect decimal value: '1.000001' for column 'a' at row 1")) - tk.MustExec("INSERT INTO t VALUES (1.000000);") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows()) - - // issue 4653 - tk.MustExec("DROP TABLE IF EXISTS t;") - tk.MustExec("CREATE TABLE t(a datetime);") - err = tk.ExecToErr("INSERT INTO t VALUES('2017-00-00')") - require.Error(t, err) - tk.MustExec("set sql_mode = ''") - tk.MustExec("INSERT INTO t VALUES('2017-00-00')") - r = tk.MustQuery("SELECT * FROM t;") - r.Check(testkit.Rows("2017-00-00 00:00:00")) - tk.MustExec("set sql_mode = 'strict_all_tables';") - r = tk.MustQuery("SELECT * FROM t;") - r.Check(testkit.Rows("2017-00-00 00:00:00")) - - // test auto_increment with unsigned. - tk.MustExec("drop table if exists test") - tk.MustExec("CREATE TABLE test(id int(10) UNSIGNED NOT NULL AUTO_INCREMENT, p int(10) UNSIGNED NOT NULL, PRIMARY KEY(p), KEY(id))") - tk.MustExec("insert into test(p) value(1)") - tk.MustQuery("select * from test").Check(testkit.Rows("1 1")) - tk.MustQuery("select * from test use index (id) where id = 1").Check(testkit.Rows("1 1")) - tk.MustExec("insert into test values(NULL, 2)") - tk.MustQuery("select * from test use index (id) where id = 2").Check(testkit.Rows("2 2")) - tk.MustExec("insert into test values(2, 3)") - tk.MustQuery("select * from test use index (id) where id = 2").Check(testkit.Rows("2 2", "2 3")) - - // issue 6360 - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a bigint unsigned);") - tk.MustExec(" set @orig_sql_mode = @@sql_mode; set @@sql_mode = 'strict_all_tables';") - err = tk.ExecToErr("insert into t value (-1);") - require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) - tk.MustExec("set @@sql_mode = '';") - tk.MustExec("insert into t value (-1);") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) - tk.MustExec("insert into t select -1;") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 constant -1 overflows bigint")) - tk.MustExec("insert into t select cast(-1 as unsigned);") - tk.MustExec("insert into t value (-1.111);") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) - tk.MustExec("insert into t value ('-1.111');") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) - tk.MustExec("update t set a = -1 limit 1;") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 constant -1 overflows bigint")) - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows("0", "0", "18446744073709551615", "0", "0")) - tk.MustExec("set @@sql_mode = @orig_sql_mode;") - - // issue 6424 & issue 20207 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a time(6))") - tk.MustExec("insert into t value('20070219173709.055870'), ('20070219173709.055'), ('20070219173709.055870123')") - tk.MustQuery("select * from t").Check(testkit.Rows("17:37:09.055870", "17:37:09.055000", "17:37:09.055870")) - tk.MustExec("truncate table t") - tk.MustExec("insert into t value(20070219173709.055870), (20070219173709.055), (20070219173709.055870123)") - tk.MustQuery("select * from t").Check(testkit.Rows("17:37:09.055870", "17:37:09.055000", "17:37:09.055870")) - err = tk.ExecToErr("insert into t value(-20070219173709.055870)") - require.EqualError(t, err, "[table:1292]Incorrect time value: '-20070219173709.055870' for column 'a' at row 1") - - tk.MustExec("drop table if exists t") - tk.MustExec("set @@sql_mode=''") - tk.MustExec("create table t(a float unsigned, b double unsigned)") - tk.MustExec("insert into t value(-1.1, -1.1), (-2.1, -2.1), (0, 0), (1.1, 1.1)") - tk.MustQuery("show warnings"). - Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1", "Warning 1264 Out of range value for column 'b' at row 1", - "Warning 1264 Out of range value for column 'a' at row 2", "Warning 1264 Out of range value for column 'b' at row 2")) - tk.MustQuery("select * from t").Check(testkit.Rows("0 0", "0 0", "0 0", "1.1 1.1")) - - // issue 7061 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int default 1, b int default 2)") - tk.MustExec("insert into t values(default, default)") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2")) - tk.MustExec("truncate table t") - tk.MustExec("insert into t values(default(b), default(a))") - tk.MustQuery("select * from t").Check(testkit.Rows("2 1")) - tk.MustExec("truncate table t") - tk.MustExec("insert into t (b) values(default)") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2")) - tk.MustExec("truncate table t") - tk.MustExec("insert into t (b) values(default(a))") - tk.MustQuery("select * from t").Check(testkit.Rows("1 1")) - - tk.MustExec("create view v as select * from t") - err = tk.ExecToErr("insert into v values(1,2)") - require.EqualError(t, err, "insert into view v is not supported now") - err = tk.ExecToErr("replace into v values(1,2)") - require.EqualError(t, err, "replace into view v is not supported now") - tk.MustExec("drop view v") - - tk.MustExec("create sequence seq") - err = tk.ExecToErr("insert into seq values()") - require.EqualError(t, err, "insert into sequence seq is not supported now") - err = tk.ExecToErr("replace into seq values()") - require.EqualError(t, err, "replace into sequence seq is not supported now") - tk.MustExec("drop sequence seq") - - // issue 22851 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(name varchar(255), b int, c int, primary key(name(2)))") - tk.MustExec("insert into t(name, b) values(\"cha\", 3)") - err = tk.ExecToErr("insert into t(name, b) values(\"chb\", 3)") - require.EqualError(t, err, "[kv:1062]Duplicate entry 'ch' for key 't.PRIMARY'") - tk.MustExec("insert into t(name, b) values(\"测试\", 3)") - err = tk.ExecToErr("insert into t(name, b) values(\"测试\", 3)") - require.EqualError(t, err, "[kv:1062]Duplicate entry '\xe6\xb5' for key 't.PRIMARY'") -} - -func TestInsertAutoInc(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - createSQL := `drop table if exists insert_autoinc_test; create table insert_autoinc_test (id int primary key auto_increment, c1 int);` - tk.MustExec(createSQL) - - insertSQL := `insert into insert_autoinc_test(c1) values (1), (2)` - tk.MustExec(insertSQL) - tk.MustExec("begin") - r := tk.MustQuery("select * from insert_autoinc_test;") - rowStr1 := fmt.Sprintf("%v %v", "1", "1") - rowStr2 := fmt.Sprintf("%v %v", "2", "2") - r.Check(testkit.Rows(rowStr1, rowStr2)) - tk.MustExec("commit") - - tk.MustExec("begin") - insertSQL = `insert into insert_autoinc_test(id, c1) values (5,5)` - tk.MustExec(insertSQL) - insertSQL = `insert into insert_autoinc_test(c1) values (6)` - tk.MustExec(insertSQL) - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr3 := fmt.Sprintf("%v %v", "5", "5") - rowStr4 := fmt.Sprintf("%v %v", "6", "6") - r.Check(testkit.Rows(rowStr1, rowStr2, rowStr3, rowStr4)) - tk.MustExec("commit") - - tk.MustExec("begin") - insertSQL = `insert into insert_autoinc_test(id, c1) values (3,3)` - tk.MustExec(insertSQL) - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr5 := fmt.Sprintf("%v %v", "3", "3") - r.Check(testkit.Rows(rowStr1, rowStr2, rowStr5, rowStr3, rowStr4)) - tk.MustExec("commit") - - tk.MustExec("begin") - insertSQL = `insert into insert_autoinc_test(c1) values (7)` - tk.MustExec(insertSQL) - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr6 := fmt.Sprintf("%v %v", "7", "7") - r.Check(testkit.Rows(rowStr1, rowStr2, rowStr5, rowStr3, rowStr4, rowStr6)) - tk.MustExec("commit") - - // issue-962 - createSQL = `drop table if exists insert_autoinc_test; create table insert_autoinc_test (id int primary key auto_increment, c1 int);` - tk.MustExec(createSQL) - insertSQL = `insert into insert_autoinc_test(id, c1) values (0.3, 1)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr1 = fmt.Sprintf("%v %v", "1", "1") - r.Check(testkit.Rows(rowStr1)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (-0.3, 2)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr2 = fmt.Sprintf("%v %v", "2", "2") - r.Check(testkit.Rows(rowStr1, rowStr2)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (-3.3, 3)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr3 = fmt.Sprintf("%v %v", "-3", "3") - r.Check(testkit.Rows(rowStr3, rowStr1, rowStr2)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (4.3, 4)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr4 = fmt.Sprintf("%v %v", "4", "4") - r.Check(testkit.Rows(rowStr3, rowStr1, rowStr2, rowStr4)) - insertSQL = `insert into insert_autoinc_test(c1) values (5)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr5 = fmt.Sprintf("%v %v", "5", "5") - r.Check(testkit.Rows(rowStr3, rowStr1, rowStr2, rowStr4, rowStr5)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (null, 6)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr6 = fmt.Sprintf("%v %v", "6", "6") - r.Check(testkit.Rows(rowStr3, rowStr1, rowStr2, rowStr4, rowStr5, rowStr6)) - - // SQL_MODE=NO_AUTO_VALUE_ON_ZERO - createSQL = `drop table if exists insert_autoinc_test; create table insert_autoinc_test (id int primary key auto_increment, c1 int);` - tk.MustExec(createSQL) - insertSQL = `insert into insert_autoinc_test(id, c1) values (5, 1)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr1 = fmt.Sprintf("%v %v", "5", "1") - r.Check(testkit.Rows(rowStr1)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 2)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr2 = fmt.Sprintf("%v %v", "6", "2") - r.Check(testkit.Rows(rowStr1, rowStr2)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 3)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr3 = fmt.Sprintf("%v %v", "7", "3") - r.Check(testkit.Rows(rowStr1, rowStr2, rowStr3)) - tk.MustExec("set SQL_MODE=NO_AUTO_VALUE_ON_ZERO") - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 4)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr4 = fmt.Sprintf("%v %v", "0", "4") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 5)` - err := tk.ExecToErr(insertSQL) - // ERROR 1062 (23000): Duplicate entry '0' for key 'PRIMARY' - require.Error(t, err) - insertSQL = `insert into insert_autoinc_test(c1) values (6)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr5 = fmt.Sprintf("%v %v", "8", "6") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (null, 7)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr6 = fmt.Sprintf("%v %v", "9", "7") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5, rowStr6)) - tk.MustExec("set SQL_MODE='';") - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 8)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr7 := fmt.Sprintf("%v %v", "10", "8") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5, rowStr6, rowStr7)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (null, 9)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr8 := fmt.Sprintf("%v %v", "11", "9") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5, rowStr6, rowStr7, rowStr8)) -} - func TestInsertIgnore(t *testing.T) { store := testkit.CreateMockStore(t) var cfg kv.InjectionConfig @@ -577,656 +159,6 @@ commit;` tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1526 Table has no partition for value 3")) } -func TestIssue38950(t *testing.T) { - store := testkit.CreateMockStore(t) - var cfg kv.InjectionConfig - tk := testkit.NewTestKit(t, kv.NewInjectedStore(store, &cfg)) - tk.MustExec("use test;") - tk.MustExec("drop table if exists t; create table t (id smallint auto_increment primary key);") - tk.MustExec("alter table t add column c1 int default 1;") - tk.MustExec("insert ignore into t(id) values (194626268);") - require.Empty(t, tk.Session().LastMessage()) - - tk.MustQuery("select * from t").Check(testkit.Rows("32767 1")) - - tk.MustExec("insert ignore into t(id) values ('*') on duplicate key update c1 = 2;") - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - tk.MustQuery("select * from t").Check(testkit.Rows("32767 2")) -} - -func TestInsertOnDup(t *testing.T) { - store := testkit.CreateMockStore(t) - var cfg kv.InjectionConfig - tk := testkit.NewTestKit(t, kv.NewInjectedStore(store, &cfg)) - tk.MustExec("use test") - testSQL := `drop table if exists t; - create table t (i int unique key);` - tk.MustExec(testSQL) - testSQL = `insert into t values (1),(2);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") - - r := tk.MustQuery("select * from t;") - rowStr1 := fmt.Sprintf("%v", "1") - rowStr2 := fmt.Sprintf("%v", "2") - r.Check(testkit.Rows(rowStr1, rowStr2)) - - tk.MustExec("insert into t values (1), (2) on duplicate key update i = values(i)") - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows(rowStr1, rowStr2)) - - tk.MustExec("insert into t values (2), (3) on duplicate key update i = 3") - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 1 Warnings: 0") - r = tk.MustQuery("select * from t;") - rowStr3 := fmt.Sprintf("%v", "3") - r.Check(testkit.Rows(rowStr1, rowStr3)) - - testSQL = `drop table if exists t; - create table t (i int primary key, j int unique key);` - tk.MustExec(testSQL) - testSQL = `insert into t values (-1, 1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - - r = tk.MustQuery("select * from t;") - rowStr1 = fmt.Sprintf("%v %v", "-1", "1") - r.Check(testkit.Rows(rowStr1)) - - tk.MustExec("insert into t values (1, 1) on duplicate key update j = values(j)") - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows(rowStr1)) - - testSQL = `drop table if exists test; -create table test (i int primary key, j int unique); -begin; -insert into test values (1,1); -insert into test values (2,1) on duplicate key update i = -i, j = -j; -commit;` - tk.MustExec(testSQL) - testSQL = `select * from test;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("-1 -1")) - - testSQL = `delete from test; -insert into test values (1, 1); -begin; -delete from test where i = 1; -insert into test values (2, 1) on duplicate key update i = -i, j = -j; -commit;` - tk.MustExec(testSQL) - testSQL = `select * from test;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("2 1")) - - testSQL = `delete from test; -insert into test values (1, 1); -begin; -update test set i = 2, j = 2 where i = 1; -insert into test values (1, 3) on duplicate key update i = -i, j = -j; -insert into test values (2, 4) on duplicate key update i = -i, j = -j; -commit;` - tk.MustExec(testSQL) - testSQL = `select * from test order by i;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("-2 -2", "1 3")) - - testSQL = `delete from test; -begin; -insert into test values (1, 3), (1, 3) on duplicate key update i = values(i), j = values(j); -commit;` - tk.MustExec(testSQL) - testSQL = `select * from test order by i;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1 3")) - - testSQL = `create table tmp (id int auto_increment, code int, primary key(id, code)); - create table m (id int primary key auto_increment, code int unique); - insert tmp (code) values (1); - insert tmp (code) values (1); - set tidb_init_chunk_size=1; - insert m (code) select code from tmp on duplicate key update code = values(code);` - tk.MustExec(testSQL) - testSQL = `select * from m;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1 1")) - - // The following two cases are used for guaranteeing the last_insert_id - // to be set as the value of on-duplicate-update assigned. - testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT AUTO_INCREMENT PRIMARY KEY, - f2 VARCHAR(5) NOT NULL UNIQUE); - INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - - testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT AUTO_INCREMENT UNIQUE, - f2 VARCHAR(5) NOT NULL UNIQUE); - INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = 2;` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - - testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT); - INSERT t1 VALUES (1) ON DUPLICATE KEY UPDATE f1 = 1;` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - tk.MustQuery(`SELECT * FROM t1;`).Check(testkit.Rows("1")) - - testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT NOT NULL UNIQUE); - INSERT t1 VALUES (1, 1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - tk.MustExec(`INSERT t1 VALUES (1, 1), (1, 1) ON DUPLICATE KEY UPDATE f1 = 2, f2 = 2;`) - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 1 Warnings: 0") - tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 1", "2 2")) - err := tk.ExecToErr(`INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) - require.Error(t, err) - tk.MustExec(`INSERT IGNORE t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) - require.Empty(t, tk.Session().LastMessage()) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'f2' cannot be null")) - tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) - - tk.MustExec(`SET sql_mode='';`) - tk.MustExec(`INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'f2' cannot be null")) - tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) -} - -func TestInsertIgnoreOnDup(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - testSQL := `drop table if exists t; - create table t (i int not null primary key, j int unique key);` - tk.MustExec(testSQL) - testSQL = `insert into t values (1, 1), (2, 2);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") - testSQL = `insert ignore into t values(1, 1) on duplicate key update i = 2;` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `select * from t;` - r := tk.MustQuery(testSQL) - r.Check(testkit.Rows("1 1", "2 2")) - testSQL = `insert ignore into t values(1, 1) on duplicate key update j = 2;` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `select * from t;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1 1", "2 2")) - - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t2(`col_25` set('Alice','Bob','Charlie','David') NOT NULL,`col_26` date NOT NULL DEFAULT '2016-04-15', PRIMARY KEY (`col_26`) clustered, UNIQUE KEY `idx_9` (`col_25`,`col_26`),UNIQUE KEY `idx_10` (`col_25`))") - tk.MustExec("insert into t2(col_25, col_26) values('Bob', '1989-03-23'),('Alice', '2023-11-24'), ('Charlie', '2023-12-05')") - tk.MustExec("insert ignore into t2 (col_25,col_26) values ( 'Bob','1977-11-23' ) on duplicate key update col_25 = 'Alice', col_26 = '2036-12-13'") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry 'Alice' for key 't2.idx_10'")) - tk.MustQuery("select * from t2").Check(testkit.Rows("Bob 1989-03-23", "Alice 2023-11-24", "Charlie 2023-12-05")) - - tk.MustExec("drop table if exists t4") - tk.MustExec("create table t4(id int primary key clustered, k int, v int, unique key uk1(k))") - tk.MustExec("insert into t4 values (1, 10, 100), (3, 30, 300)") - tk.MustExec("insert ignore into t4 (id, k, v) values(1, 0, 0) on duplicate key update id = 2, k = 30") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '30' for key 't4.uk1'")) - tk.MustQuery("select * from t4").Check(testkit.Rows("1 10 100", "3 30 300")) - - tk.MustExec("drop table if exists t5") - tk.MustExec("create table t5(k1 varchar(100), k2 varchar(100), uk1 int, v int, primary key(k1, k2) clustered, unique key ukk1(uk1), unique key ukk2(v))") - tk.MustExec("insert into t5(k1, k2, uk1, v) values('1', '1', 1, '100'), ('1', '3', 2, '200')") - tk.MustExec("update ignore t5 set k2 = '2', uk1 = 2 where k1 = '1' and k2 = '1'") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '2' for key 't5.ukk1'")) - tk.MustQuery("select * from t5").Check(testkit.Rows("1 1 1 100", "1 3 2 200")) - - tk.MustExec("drop table if exists t6") - tk.MustExec("create table t6 (a int, b int, c int, primary key(a, b) clustered, unique key idx_14(b), unique key idx_15(b), unique key idx_16(a, b))") - tk.MustExec("insert into t6 select 10, 10, 20") - tk.MustExec("insert ignore into t6 set a = 20, b = 10 on duplicate key update a = 100") - tk.MustQuery("select * from t6").Check(testkit.Rows("100 10 20")) - tk.MustExec("insert ignore into t6 set a = 200, b= 10 on duplicate key update c = 1000") - tk.MustQuery("select * from t6").Check(testkit.Rows("100 10 1000")) -} - -func TestReplace(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - testSQL := `drop table if exists replace_test; - create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` - tk.MustExec(testSQL) - testSQL = `replace replace_test (c1) values (1),(2),(NULL);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") - - errReplaceSQL := `replace replace_test (c1) values ();` - tk.MustExec("begin") - err := tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (xxx) values (3);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test_xxx (c1) values ();` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - replaceSetSQL := `replace replace_test set c1 = 3;` - tk.MustExec(replaceSetSQL) - require.Empty(t, tk.Session().LastMessage()) - - errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSetSQL = `replace replace_test set xxx = 6;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - replaceSelectSQL := `create table replace_test_1 (id int, c1 int);` - tk.MustExec(replaceSelectSQL) - replaceSelectSQL = `replace replace_test_1 select id, c1 from replace_test;` - tk.MustExec(replaceSelectSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") - - replaceSelectSQL = `create table replace_test_2 (id int, c1 int);` - tk.MustExec(replaceSelectSQL) - replaceSelectSQL = `replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;` - tk.MustExec(replaceSelectSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") - - errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - replaceUniqueIndexSQL := `create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2));` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=1;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=1;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `replace into replace_test_3 set c1=1, c2=1;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2));` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2));` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - // For Issue989 - issue989SQL := `CREATE TABLE tIssue989 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b));` - tk.MustExec(issue989SQL) - issue989SQL = `insert into tIssue989 (a, b) values (1, 2);` - tk.MustExec(issue989SQL) - require.Empty(t, tk.Session().LastMessage()) - issue989SQL = `replace into tIssue989(a, b) values (111, 2);` - tk.MustExec(issue989SQL) - require.Empty(t, tk.Session().LastMessage()) - r := tk.MustQuery("select * from tIssue989;") - r.Check(testkit.Rows("111 2")) - - // For Issue1012 - issue1012SQL := `CREATE TABLE tIssue1012 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b));` - tk.MustExec(issue1012SQL) - issue1012SQL = `insert into tIssue1012 (a, b) values (1, 2);` - tk.MustExec(issue1012SQL) - issue1012SQL = `insert into tIssue1012 (a, b) values (2, 1);` - tk.MustExec(issue1012SQL) - issue1012SQL = `replace into tIssue1012(a, b) values (1, 1);` - tk.MustExec(issue1012SQL) - require.Equal(t, int64(3), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from tIssue1012;") - r.Check(testkit.Rows("1 1")) - - // Test Replace with info message - tk.MustExec(`drop table if exists t1`) - tk.MustExec(`create table t1(a int primary key, b int);`) - tk.MustExec(`insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5);`) - tk.MustExec(`replace into t1 values(1,1);`) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - tk.MustExec(`replace into t1 values(1,1),(2,2);`) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") - tk.MustExec(`replace into t1 values(4,14),(5,15),(6,16),(7,17),(8,18)`) - require.Equal(t, int64(7), int64(tk.Session().AffectedRows())) - require.Equal(t, tk.Session().LastMessage(), "Records: 5 Duplicates: 2 Warnings: 0") - tk.MustExec(`replace into t1 select * from (select 1, 2) as tmp;`) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Equal(t, tk.Session().LastMessage(), "Records: 1 Duplicates: 1 Warnings: 0") - - // Assign `DEFAULT` in `REPLACE` statement - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1 (a int primary key, b int default 20, c int default 30);") - tk.MustExec("insert into t1 value (1, 2, 3);") - tk.MustExec("replace t1 set a=1, b=default;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 30")) - tk.MustExec("replace t1 set a=2, b=default, c=default") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 30", "2 20 30")) - tk.MustExec("replace t1 set a=2, b=default(c), c=default(b);") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 30", "2 30 20")) - tk.MustExec("replace t1 set a=default(b)+default(c)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 30", "2 30 20", "50 20 30")) - // With generated columns - tk.MustExec("create table t2 (pk int primary key, a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored);") - tk.MustExec("replace t2 set pk=1, b=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1")) - tk.MustExec("replace t2 set pk=2, a=10, b=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1", "2 10 -10 -10")) - tk.MustExec("replace t2 set pk=2, c=default, a=20;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1", "2 20 -20 -20")) - tk.MustExec("replace t2 set pk=2, a=default, b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1", "2 1 -1 -1")) - tk.MustExec("replace t2 set pk=3, a=default(a), b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1", "2 1 -1 -1", "3 1 -1 -1")) - tk.MustGetErrCode("replace t2 set b=default(a);", mysql.ErrBadGeneratedColumn) - tk.MustGetErrCode("replace t2 set a=default(b), b=default(b);", mysql.ErrBadGeneratedColumn) - tk.MustGetErrCode("replace t2 set a=default(a), c=default(c);", mysql.ErrNoDefaultForField) - tk.MustGetErrCode("replace t2 set c=default(a);", mysql.ErrBadGeneratedColumn) - tk.MustExec("drop table t1, t2") -} - -// TestUpdateCastOnlyModifiedValues for issue #4514. -func TestUpdateCastOnlyModifiedValues(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table update_modified (col_1 int, col_2 enum('a', 'b'))") - tk.MustExec("set SQL_MODE=''") - tk.MustExec("insert into update_modified values (0, 3)") - r := tk.MustQuery("SELECT * FROM update_modified") - r.Check(testkit.Rows("0 ")) - tk.MustExec("set SQL_MODE=STRICT_ALL_TABLES") - tk.MustExec("update update_modified set col_1 = 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("SELECT * FROM update_modified") - r.Check(testkit.Rows("1 ")) - err := tk.ExecToErr("update update_modified set col_1 = 2, col_2 = 'c'") - require.Error(t, err) - r = tk.MustQuery("SELECT * FROM update_modified") - r.Check(testkit.Rows("1 ")) - tk.MustExec("update update_modified set col_1 = 3, col_2 = 'a'") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("SELECT * FROM update_modified") - r.Check(testkit.Rows("3 a")) - - // Test update a field with different column type. - tk.MustExec(`CREATE TABLE update_with_diff_type (a int, b JSON)`) - tk.MustExec(`INSERT INTO update_with_diff_type VALUES(3, '{"a": "测试"}')`) - tk.MustExec(`UPDATE update_with_diff_type SET a = '300'`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("SELECT a FROM update_with_diff_type") - r.Check(testkit.Rows("300")) - tk.MustExec(`UPDATE update_with_diff_type SET b = '{"a": "\\u6d4b\\u8bd5"}'`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") - r = tk.MustQuery("SELECT b FROM update_with_diff_type") - r.Check(testkit.Rows(`{"a": "测试"}`)) -} - -func fillMultiTableForUpdate(tk *testkit.TestKit) { - // Create and fill table items - tk.MustExec("CREATE TABLE items (id int, price TEXT);") - tk.MustExec(`insert into items values (11, "items_price_11"), (12, "items_price_12"), (13, "items_price_13");`) - tk.CheckExecResult(3, 0) - // Create and fill table month - tk.MustExec("CREATE TABLE month (mid int, mprice TEXT);") - tk.MustExec(`insert into month values (11, "month_price_11"), (22, "month_price_22"), (13, "month_price_13");`) - tk.CheckExecResult(3, 0) -} - -func TestMultipleTableUpdate(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - fillMultiTableForUpdate(tk) - - tk.MustExec(`UPDATE items, month SET items.price=month.mprice WHERE items.id=month.mid;`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - tk.MustExec("begin") - r := tk.MustQuery("SELECT * FROM items") - r.Check(testkit.Rows("11 month_price_11", "12 items_price_12", "13 month_price_13")) - tk.MustExec("commit") - - // Single-table syntax but with multiple tables - tk.MustExec(`UPDATE items join month on items.id=month.mid SET items.price=month.mid;`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - tk.MustExec("begin") - r = tk.MustQuery("SELECT * FROM items") - r.Check(testkit.Rows("11 11", "12 items_price_12", "13 13")) - tk.MustExec("commit") - - // JoinTable with alias table name. - tk.MustExec(`UPDATE items T0 join month T1 on T0.id=T1.mid SET T0.price=T1.mprice;`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - tk.MustExec("begin") - r = tk.MustQuery("SELECT * FROM items") - r.Check(testkit.Rows("11 month_price_11", "12 items_price_12", "13 month_price_13")) - tk.MustExec("commit") - - // fix https://github.com/pingcap/tidb/issues/369 - testSQL := ` - DROP TABLE IF EXISTS t1, t2; - create table t1 (c int); - create table t2 (c varchar(256)); - insert into t1 values (1), (2); - insert into t2 values ("a"), ("b"); - update t1, t2 set t1.c = 10, t2.c = "abc";` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 4 Changed: 4 Warnings: 0") - - // fix https://github.com/pingcap/tidb/issues/376 - testSQL = `DROP TABLE IF EXISTS t1, t2; - create table t1 (c1 int); - create table t2 (c2 int); - insert into t1 values (1), (2); - insert into t2 values (1), (2); - update t1, t2 set t1.c1 = 10, t2.c2 = 2 where t2.c2 = 1;` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") - - r = tk.MustQuery("select * from t1") - r.Check(testkit.Rows("10", "10")) - - // test https://github.com/pingcap/tidb/issues/3604 - tk.MustExec("drop table if exists t, t") - tk.MustExec("create table t (a int, b int)") - tk.MustExec("insert into t values(1, 1), (2, 2), (3, 3)") - require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") - tk.MustExec("update t m, t n set m.a = m.a + 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") - tk.MustQuery("select * from t").Check(testkit.Rows("2 1", "3 2", "4 3")) - tk.MustExec("update t m, t n set n.a = n.a - 1, n.b = n.b + 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2", "2 3", "3 4")) -} - -func TestDelete(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - internal.FillData(tk, "delete_test") - - tk.MustExec(`update delete_test set name = "abc" where id = 2;`) - tk.CheckExecResult(1, 0) - - tk.MustExec(`delete from delete_test where id = 2 limit 1;`) - tk.CheckExecResult(1, 0) - - // Test delete with false condition - tk.MustExec(`delete from delete_test where 0;`) - tk.CheckExecResult(0, 0) - - tk.MustExec("insert into delete_test values (2, 'abc')") - tk.MustExec(`delete from delete_test where delete_test.id = 2 limit 1`) - tk.CheckExecResult(1, 0) - - // Select data - tk.MustExec("begin") - rows := tk.MustQuery(`SELECT * from delete_test limit 2;`) - rows.Check(testkit.Rows("1 hello")) - tk.MustExec("commit") - - // Test delete ignore - tk.MustExec("insert into delete_test values (2, 'abc')") - err := tk.ExecToErr("delete from delete_test where id = (select '2a')") - require.Error(t, err) - err = tk.ExecToErr("delete ignore from delete_test where id = (select '2a')") - require.NoError(t, err) - tk.CheckExecResult(1, 0) - r := tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) - - tk.MustExec(`delete from delete_test ;`) - tk.CheckExecResult(1, 0) - - tk.MustExec("create view v as select * from delete_test") - err = tk.ExecToErr("delete from v where name = 'aaa'") - require.EqualError(t, err, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) - tk.MustExec("drop view v") - - tk.MustExec("create sequence seq") - err = tk.ExecToErr("delete from seq") - require.EqualError(t, err, "delete sequence seq is not supported now") - tk.MustExec("drop sequence seq") -} - -func fillDataMultiTable(tk *testkit.TestKit) { - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2, t3") - // Create and fill table t1 - tk.MustExec("create table t1 (id int, data int);") - tk.MustExec("insert into t1 values (11, 121), (12, 122), (13, 123);") - tk.CheckExecResult(3, 0) - // Create and fill table t2 - tk.MustExec("create table t2 (id int, data int);") - tk.MustExec("insert into t2 values (11, 221), (22, 222), (23, 223);") - tk.CheckExecResult(3, 0) - // Create and fill table t3 - tk.MustExec("create table t3 (id int, data int);") - tk.MustExec("insert into t3 values (11, 321), (22, 322), (23, 323);") - tk.CheckExecResult(3, 0) -} - -func TestMultiTableDelete(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - fillDataMultiTable(tk) - - tk.MustExec(`delete t1, t2 from t1 inner join t2 inner join t3 where t1.id=t2.id and t2.id=t3.id;`) - tk.CheckExecResult(2, 0) - - // Select data - r := tk.MustQuery("select * from t3") - require.Len(t, r.Rows(), 3) -} - -func TestQualifiedDelete(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t1 (c1 int, c2 int, index (c1))") - tk.MustExec("create table t2 (c1 int, c2 int)") - tk.MustExec("insert into t1 values (1, 1), (2, 2)") - - // delete with index - tk.MustExec("delete from t1 where t1.c1 = 1") - tk.CheckExecResult(1, 0) - - // delete with no index - tk.MustExec("delete from t1 where t1.c2 = 2") - tk.CheckExecResult(1, 0) - - r := tk.MustQuery("select * from t1") - require.Len(t, r.Rows(), 0) - tk.MustExec("insert into t1 values (1, 3)") - tk.MustExec("delete from t1 as a where a.c1 = 1") - tk.CheckExecResult(1, 0) - - tk.MustExec("insert into t1 values (1, 1), (2, 2)") - tk.MustExec("insert into t2 values (2, 1), (3,1)") - tk.MustExec("delete t1, t2 from t1 join t2 where t1.c1 = t2.c2") - tk.CheckExecResult(3, 0) - - tk.MustExec("insert into t2 values (2, 1), (3,1)") - tk.MustExec("delete a, b from t1 as a join t2 as b where a.c2 = b.c1") - tk.CheckExecResult(2, 0) - - err := tk.ExecToErr("delete t1, t2 from t1 as a join t2 as b where a.c2 = b.c1") - require.Error(t, err) -} - type testCase struct { data []byte expected []string @@ -1407,60 +339,6 @@ func TestLatch(t *testing.T) { tk1.MustExec("commit") } -func TestUpdateSelect(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table msg (id varchar(8), b int, status int, primary key (id, b))") - tk.MustExec("insert msg values ('abc', 1, 1)") - tk.MustExec("create table detail (id varchar(8), start varchar(8), status int, index idx_start(start))") - tk.MustExec("insert detail values ('abc', '123', 2)") - tk.MustExec("UPDATE msg SET msg.status = (SELECT detail.status FROM detail WHERE msg.id = detail.id)") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("admin check table msg") -} - -func TestUpdateDelete(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("CREATE TABLE ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host));") - tk.MustExec("insert into ttt values (8,8),(9,9);") - - tk.MustExec("begin") - tk.MustExec("update ttt set id = 0, host='9' where id = 9 limit 1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("delete from ttt where id = 0 limit 1;") - tk.MustQuery("select * from ttt use index (i_host) order by host;").Check(testkit.Rows("8 8")) - tk.MustExec("update ttt set id = 0, host='8' where id = 8 limit 1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("delete from ttt where id = 0 limit 1;") - tk.MustQuery("select * from ttt use index (i_host) order by host;").Check(testkit.Rows()) - tk.MustExec("commit") - tk.MustExec("admin check table ttt;") - tk.MustExec("drop table ttt") -} - -func TestUpdateAffectRowCnt(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table a(id int auto_increment, a int default null, primary key(id))") - tk.MustExec("insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1)") - tk.MustExec("update a set id = id*10 where a = 1001") - ctx := tk.Session().(sessionctx.Context) - require.Equal(t, uint64(2), ctx.GetSessionVars().StmtCtx.AffectedRows()) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - - tk.MustExec("drop table a") - tk.MustExec("create table a ( a bigint, b bigint)") - tk.MustExec("insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1)") - tk.MustExec("update a set a = a*10 where b = 1001") - ctx = tk.Session().(sessionctx.Context) - require.Equal(t, uint64(2), ctx.GetSessionVars().StmtCtx.AffectedRows()) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") -} - func TestReplaceLog(t *testing.T) { store, domain := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1683,524 +561,6 @@ func TestPessimisticDeleteYourWrites(t *testing.T) { session2.MustQuery("select * from x").Check(testkit.Rows("1 2")) } -// TestWriteListPartitionTable2 test for write list partition when the partition expression is complicated and contain generated column. -func TestWriteListPartitionTable2(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec("drop table if exists t") - tk.MustExec(`create table t (id int, name varchar(10),b int generated always as (length(name)+1) virtual) - partition by list (id*2 + b*b + b*b - b*b*2 - abs(id)) ( - partition p0 values in (3,5,6,9,17), - partition p1 values in (1,2,10,11,19,20), - partition p2 values in (4,12,13,14,18), - partition p3 values in (7,8,15,16,null) - );`) - - // Test add unique index failed. - tk.MustExec("insert into t (id,name) values (1, 'a'),(1,'b')") - err := tk.ExecToErr("alter table t add unique index idx (id,b)") - require.EqualError(t, err, "[kv:1062]Duplicate entry '1-2' for key 't.idx'") - // Test add unique index success. - tk.MustExec("delete from t where name='b'") - tk.MustExec("alter table t add unique index idx (id,b)") - - // --------------------------Test insert--------------------------- - // Test insert 1 partition. - tk.MustExec("delete from t") - tk.MustExec("insert into t (id,name) values (1, 'a'),(2,'b'),(10,'c')") - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 a", "2 b", "10 c")) - // Test insert multi-partitions. - tk.MustExec("delete from t") - tk.MustExec("insert into t (id,name) values (1, 'a'),(3,'c'),(4,'e')") - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows("3 c")) - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 a")) - tk.MustQuery("select id,name from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows()) - // Test insert on duplicate. - tk.MustExec("insert into t (id,name) values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'") - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows("3 x", "5 g")) - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 x")) - tk.MustQuery("select id,name from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows()) - // Test insert on duplicate error - err = tk.ExecToErr("insert into t (id,name) values (3, 'a'), (11,'x') on duplicate key update id=id+1") - require.EqualError(t, err, "[kv:1062]Duplicate entry '4-2' for key 't.idx'") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g")) - // Test insert ignore with duplicate - tk.MustExec("insert ignore into t (id,name) values (1, 'b'), (5,'a'),(null,'y')") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '1-2' for key 't.idx'", "Warning 1062 Duplicate entry '5-2' for key 't.idx'")) - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows("3 x", "5 g")) - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 x")) - tk.MustQuery("select id,name from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows(" y")) - // Test insert ignore without duplicate - tk.MustExec("insert ignore into t (id,name) values (15, 'a'),(17,'a')") - tk.MustQuery("select id,name from t partition(p0,p1,p2) order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g", "17 a")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows(" y", "15 a")) - // Test insert meet no partition error. - err = tk.ExecToErr("insert into t (id,name) values (100, 'd')") - require.EqualError(t, err, "[table:1526]Table has no partition for value 100") - - // --------------------------Test update--------------------------- - // Test update 1 partition. - tk.MustExec("delete from t") - tk.MustExec("insert into t (id,name) values (1, 'a'),(2,'b'),(3,'c')") - tk.MustExec("update t set name='b' where id=2;") - tk.MustQuery("select id,name from t partition(p1)").Check(testkit.Rows("1 a", "2 b")) - tk.MustExec("update t set name='x' where id in (1,2)") - tk.MustQuery("select id,name from t partition(p1)").Check(testkit.Rows("1 x", "2 x")) - tk.MustExec("update t set name='y' where id < 3") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) - // Test update meet duplicate error. - tk.MustGetErrMsg("update t set id=2 where id = 1", "[kv:1062]Duplicate entry '2-2' for key 't.idx'") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) - - // Test update multi-partitions - tk.MustExec("update t set name='z' where id in (1,2,3);") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 z", "2 z", "3 z")) - tk.MustExec("update t set name='a' limit 3") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 a", "2 a", "3 a")) - tk.MustExec("update t set id=id*10 where id in (1,2)") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - // Test update meet duplicate error. - tk.MustGetErrMsg("update t set id=id+17 where id in (3,10)", "[kv:1062]Duplicate entry '20-2' for key 't.idx'") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - // Test update meet no partition error. - tk.MustGetErrMsg("update t set id=id*2 where id in (3,20)", "[table:1526]Table has no partition for value 40") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - - // --------------------------Test replace--------------------------- - // Test replace 1 partition. - tk.MustExec("delete from t") - tk.MustExec("replace into t (id,name) values (1, 'a'),(2,'b')") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 a", "2 b")) - // Test replace multi-partitions. - tk.MustExec("replace into t (id,name) values (3, 'c'),(4,'d'),(7,'f')") - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows("3 c")) - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 a", "2 b")) - tk.MustQuery("select id,name from t partition(p2) order by id").Check(testkit.Rows("4 d")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows("7 f")) - // Test replace on duplicate. - tk.MustExec("replace into t (id,name) values (1, 'x'),(7,'x')") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) - // Test replace meet no partition error. - tk.MustGetErrMsg("replace into t (id,name) values (10,'x'),(50,'x')", "[table:1526]Table has no partition for value 50") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) - - // --------------------------Test delete--------------------------- - // Test delete 1 partition. - tk.MustExec("delete from t where id = 3") - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows()) - tk.MustExec("delete from t where id in (1,2)") - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows()) - // Test delete multi-partitions. - tk.MustExec("delete from t where id in (4,7,10,11)") - tk.MustQuery("select id,name from t").Check(testkit.Rows()) - tk.MustExec("insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f')") - tk.MustExec("delete from t where id < 10") - tk.MustQuery("select id,name from t").Check(testkit.Rows()) - tk.MustExec("insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f')") - tk.MustExec("delete from t limit 3") - tk.MustQuery("select id,name from t").Check(testkit.Rows()) -} - -func TestWriteListColumnsPartitionTable1(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - - tk.MustExec("drop table if exists t") - tk.MustExec(`create table t (id int, name varchar(10)) partition by list columns (id) ( - partition p0 values in (3,5,6,9,17), - partition p1 values in (1,2,10,11,19,20), - partition p2 values in (4,12,13,14,18), - partition p3 values in (7,8,15,16,null) - );`) - - // Test add unique index failed. - tk.MustExec("insert into t values (1, 'a'),(1,'b')") - tk.MustGetErrMsg("alter table t add unique index idx (id)", "[kv:1062]Duplicate entry '1' for key 't.idx'") - // Test add unique index success. - tk.MustExec("delete from t where name='b'") - tk.MustExec("alter table t add unique index idx (id)") - - // --------------------------Test insert--------------------------- - // Test insert 1 partition. - tk.MustExec("delete from t") - tk.MustExec("insert into t values (1, 'a'),(2,'b'),(10,'c')") - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 a", "2 b", "10 c")) - // Test insert multi-partitions. - tk.MustExec("delete from t") - tk.MustExec("insert into t values (1, 'a'),(3,'c'),(4,'e')") - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows("3 c")) - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 a")) - tk.MustQuery("select * from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows()) - // Test insert on duplicate. - tk.MustExec("insert into t values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'") - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows("3 x", "5 g")) - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 x")) - tk.MustQuery("select * from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows()) - // Test insert on duplicate error - tk.MustGetErrMsg("insert into t values (3, 'a'), (11,'x') on duplicate key update id=id+1", "[kv:1062]Duplicate entry '4' for key 't.idx'") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g")) - // Test insert ignore with duplicate - tk.MustExec("insert ignore into t values (1, 'b'), (5,'a'),(null,'y')") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 't.idx'", "Warning 1062 Duplicate entry '5' for key 't.idx'")) - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows("3 x", "5 g")) - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 x")) - tk.MustQuery("select * from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows(" y")) - // Test insert ignore without duplicate - tk.MustExec("insert ignore into t values (15, 'a'),(17,'a')") - tk.MustQuery("select * from t partition(p0,p1,p2) order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g", "17 a")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows(" y", "15 a")) - // Test insert meet no partition error. - tk.MustGetErrMsg("insert into t values (100, 'd')", "[table:1526]Table has no partition for value from column_list") - - // --------------------------Test update--------------------------- - // Test update 1 partition. - tk.MustExec("delete from t") - tk.MustExec("insert into t values (1, 'a'),(2,'b'),(3,'c')") - tk.MustExec("update t set name='b' where id=2;") - tk.MustQuery("select * from t partition(p1)").Check(testkit.Rows("1 a", "2 b")) - tk.MustExec("update t set name='x' where id in (1,2)") - tk.MustQuery("select * from t partition(p1)").Check(testkit.Rows("1 x", "2 x")) - tk.MustExec("update t set name='y' where id < 3") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) - // Test update meet duplicate error. - tk.MustGetErrMsg("update t set id=2 where id = 1", "[kv:1062]Duplicate entry '2' for key 't.idx'") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) - - // Test update multi-partitions - tk.MustExec("update t set name='z' where id in (1,2,3);") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 z", "2 z", "3 z")) - tk.MustExec("update t set name='a' limit 3") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 a", "2 a", "3 a")) - tk.MustExec("update t set id=id*10 where id in (1,2)") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - // Test update meet duplicate error. - tk.MustGetErrMsg("update t set id=id+17 where id in (3,10)", "[kv:1062]Duplicate entry '20' for key 't.idx'") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - // Test update meet no partition error. - tk.MustGetErrMsg("update t set id=id*2 where id in (3,20)", "[table:1526]Table has no partition for value from column_list") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - - // --------------------------Test replace--------------------------- - // Test replace 1 partition. - tk.MustExec("delete from t") - tk.MustExec("replace into t values (1, 'a'),(2,'b')") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 a", "2 b")) - // Test replace multi-partitions. - tk.MustExec("replace into t values (3, 'c'),(4,'d'),(7,'f')") - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows("3 c")) - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 a", "2 b")) - tk.MustQuery("select * from t partition(p2) order by id").Check(testkit.Rows("4 d")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows("7 f")) - // Test replace on duplicate. - tk.MustExec("replace into t values (1, 'x'),(7,'x')") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) - // Test replace meet no partition error. - tk.MustGetErrMsg("replace into t values (10,'x'),(100,'x')", "[table:1526]Table has no partition for value from column_list") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) - - // --------------------------Test delete--------------------------- - // Test delete 1 partition. - tk.MustExec("delete from t where id = 3") - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows()) - tk.MustExec("delete from t where id in (1,2)") - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows()) - // Test delete multi-partitions. - tk.MustExec("delete from t where id in (4,7,10,11)") - tk.MustQuery("select * from t").Check(testkit.Rows()) - tk.MustExec("insert into t values (3, 'c'),(4,'d'),(7,'f')") - tk.MustExec("delete from t where id < 10") - tk.MustQuery("select * from t").Check(testkit.Rows()) - tk.MustExec("insert into t values (3, 'c'),(4,'d'),(7,'f')") - tk.MustExec("delete from t limit 3") - tk.MustQuery("select * from t").Check(testkit.Rows()) -} - -func TestUpdate(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - internal.FillData(tk, "update_test") - - updateStr := `UPDATE update_test SET name = "abc" where id > 0;` - tk.MustExec(updateStr) - tk.CheckExecResult(2, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - - // select data - tk.MustExec("begin") - r := tk.MustQuery(`SELECT * from update_test limit 2;`) - r.Check(testkit.Rows("1 abc", "2 abc")) - tk.MustExec("commit") - - tk.MustExec(`UPDATE update_test SET name = "foo"`) - tk.CheckExecResult(2, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - - // table option is auto-increment - tk.MustExec("begin") - tk.MustExec("drop table if exists update_test;") - tk.MustExec("commit") - tk.MustExec("begin") - tk.MustExec("create table update_test(id int not null auto_increment, name varchar(255), primary key(id))") - tk.MustExec("insert into update_test(name) values ('aa')") - tk.MustExec("update update_test set id = 8 where name = 'aa'") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("insert into update_test(name) values ('bb')") - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from update_test;") - r.Check(testkit.Rows("8 aa", "9 bb")) - tk.MustExec("commit") - - tk.MustExec("begin") - tk.MustExec("drop table if exists update_test;") - tk.MustExec("commit") - tk.MustExec("begin") - tk.MustExec("create table update_test(id int not null auto_increment, name varchar(255), index(id))") - tk.MustExec("insert into update_test(name) values ('aa')") - err := tk.ExecToErr("update update_test set id = null where name = 'aa'") - require.EqualError(t, err, "[table:1048]Column 'id' cannot be null") - - tk.MustExec("drop table update_test") - tk.MustExec("create table update_test(id int)") - tk.MustExec("begin") - tk.MustExec("insert into update_test(id) values (1)") - tk.MustExec("update update_test set id = 2 where id = 1 limit 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("select * from update_test;") - r.Check(testkit.Rows("2")) - tk.MustExec("commit") - - // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. - tk.MustExec("create table update_unique (id int primary key, name int unique)") - tk.MustExec("insert update_unique values (1, 1), (2, 2);") - tk.MustExec("begin") - err = tk.ExecToErr("update update_unique set name = 1 where id = 2") - require.Error(t, err) - tk.MustExec("commit") - tk.MustQuery("select * from update_unique").Check(testkit.Rows("1 1", "2 2")) - - // test update ignore for pimary key - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a bigint, primary key (a));") - tk.MustExec("insert into t values (1)") - tk.MustExec("insert into t values (2)") - err = tk.ExecToErr("update ignore t set a = 1 where a = 2;") - require.NoError(t, err) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 't.PRIMARY'")) - tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) - - // test update ignore for truncate as warning - err = tk.ExecToErr("update ignore t set a = 1 where a = (select '2a')") - require.NoError(t, err) - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1062 Duplicate entry '1' for key 't.PRIMARY'")) - - tk.MustExec("update ignore t set a = 42 where a = 2;") - tk.MustQuery("select * from t").Check(testkit.Rows("1", "42")) - - // test update ignore for unique key - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a bigint, unique key I_uniq (a));") - tk.MustExec("insert into t values (1)") - tk.MustExec("insert into t values (2)") - err = tk.ExecToErr("update ignore t set a = 1 where a = 2;") - require.NoError(t, err) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 't.I_uniq'")) - tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) - - // test issue21965 - tk.MustExec("drop table if exists t;") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec("create table t (a int) partition by list (a) (partition p0 values in (0,1));") - tk.MustExec("insert ignore into t values (1);") - tk.MustExec("update ignore t set a=2 where a=1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a int key) partition by list (a) (partition p0 values in (0,1));") - tk.MustExec("insert ignore into t values (1);") - tk.MustExec("update ignore t set a=2 where a=1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(id integer auto_increment, t1 datetime, t2 datetime, primary key (id))") - tk.MustExec("insert into t(t1, t2) values('2000-10-01 01:01:01', '2017-01-01 10:10:10')") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2000-10-01 01:01:01 2017-01-01 10:10:10")) - tk.MustExec("update t set t1 = '2017-10-01 10:10:11', t2 = date_add(t1, INTERVAL 10 MINUTE) where id = 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2017-10-01 10:10:11 2000-10-01 01:11:01")) - - // for issue #5132 - tk.MustExec("CREATE TABLE `tt1` (" + - "`a` int(11) NOT NULL," + - "`b` varchar(32) DEFAULT NULL," + - "`c` varchar(32) DEFAULT NULL," + - "PRIMARY KEY (`a`)," + - "UNIQUE KEY `b_idx` (`b`)" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;") - tk.MustExec("insert into tt1 values(1, 'a', 'a');") - tk.MustExec("insert into tt1 values(2, 'd', 'b');") - r = tk.MustQuery("select * from tt1;") - r.Check(testkit.Rows("1 a a", "2 d b")) - tk.MustExec("update tt1 set a=5 where c='b';") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("select * from tt1;") - r.Check(testkit.Rows("1 a a", "5 d b")) - - // Automatic Updating for TIMESTAMP - tk.MustExec("CREATE TABLE `tsup` (" + - "`a` int," + - "`ts` TIMESTAMP DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP," + - "KEY `idx` (`ts`)" + - ");") - tk.MustExec("set @orig_sql_mode=@@sql_mode; set @@sql_mode='';") - tk.MustExec("insert into tsup values(1, '0000-00-00 00:00:00');") - tk.MustExec("update tsup set a=5;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r1 := tk.MustQuery("select ts from tsup use index (idx);") - r2 := tk.MustQuery("select ts from tsup;") - r1.Check(r2.Rows()) - tk.MustExec("update tsup set ts='2019-01-01';") - tk.MustQuery("select ts from tsup;").Check(testkit.Rows("2019-01-01 00:00:00")) - tk.MustExec("set @@sql_mode=@orig_sql_mode;") - - // issue 5532 - tk.MustExec("create table decimals (a decimal(20, 0) not null)") - tk.MustExec("insert into decimals values (201)") - // A warning rather than data truncated error. - tk.MustExec("update decimals set a = a + 1.23;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 1") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect DECIMAL value: '202.23'")) - r = tk.MustQuery("select * from decimals") - r.Check(testkit.Rows("202")) - - tk.MustExec("drop table t") - tk.MustExec("CREATE TABLE `t` ( `c1` year DEFAULT NULL, `c2` year DEFAULT NULL, `c3` date DEFAULT NULL, `c4` datetime DEFAULT NULL, KEY `idx` (`c1`,`c2`))") - err = tk.ExecToErr("UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2") - require.NoError(t, err) - - tk.MustGetErrCode("update (select * from t) t set c1 = 1111111", mysql.ErrNonUpdatableTable) - - // test update ignore for bad null error - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t (i int not null default 10)`) - tk.MustExec("insert into t values (1)") - tk.MustExec("update ignore t set i = null;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1048 Column 'i' cannot be null")) - tk.MustQuery("select * from t").Check(testkit.Rows("0")) - - // issue 7237, update subquery table should be forbidden - tk.MustExec("drop table t") - tk.MustExec("create table t (k int, v int)") - err = tk.ExecToErr("update t, (select * from t) as b set b.k = t.k") - require.EqualError(t, err, "[planner:1288]The target table b of the UPDATE is not updatable") - tk.MustExec("update t, (select * from t) as b set t.k = b.k") - - // issue 8045 - tk.MustExec("drop table if exists t1") - tk.MustExec(`CREATE TABLE t1 (c1 float)`) - tk.MustExec("INSERT INTO t1 SET c1 = 1") - tk.MustExec("UPDATE t1 SET c1 = 1.2 WHERE c1=1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - - // issue 8119 - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (c1 float(1,1));") - tk.MustExec("insert into t values (0.0);") - err = tk.ExecToErr("update t set c1 = 2.0;") - require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a datetime not null, b datetime)") - tk.MustExec("insert into t value('1999-12-12', '1999-12-13')") - tk.MustExec("set @orig_sql_mode=@@sql_mode; set @@sql_mode='';") - tk.MustQuery("select * from t").Check(testkit.Rows("1999-12-12 00:00:00 1999-12-13 00:00:00")) - tk.MustExec("update t set a = ''") - tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 1999-12-13 00:00:00")) - tk.MustExec("update t set b = ''") - tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 0000-00-00 00:00:00")) - tk.MustExec("set @@sql_mode=@orig_sql_mode;") - - tk.MustExec("create view v as select * from t") - err = tk.ExecToErr("update v set a = '2000-11-11'") - require.EqualError(t, err, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) - tk.MustExec("drop view v") - - tk.MustExec("create sequence seq") - tk.MustGetErrCode("update seq set minvalue=1", mysql.ErrBadField) - tk.MustExec("drop sequence seq") - - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, c int, d int, e int, index idx(a))") - tk.MustExec("create table t2(a int, b int, c int)") - tk.MustExec("update t1 join t2 on t1.a=t2.a set t1.a=1 where t2.b=1 and t2.c=2") - - // Assign `DEFAULT` in `UPDATE` statement - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1 (a int default 1, b int default 2);") - tk.MustExec("insert into t1 values (10, 10), (20, 20);") - tk.MustExec("update t1 set a=default where b=10;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "20 20")) - tk.MustExec("update t1 set a=30, b=default where a=20;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "30 2")) - tk.MustExec("update t1 set a=default, b=default where a=30;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "1 2")) - tk.MustExec("insert into t1 values (40, 40)") - tk.MustExec("update t1 set a=default, b=default") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 2", "1 2", "1 2")) - tk.MustExec("update t1 set a=default(b), b=default(a)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("2 1", "2 1", "2 1")) - // With generated columns - tk.MustExec("create table t2 (a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored);") - tk.MustExec("insert into t2 values (10, default, default), (20, default, default)") - tk.MustExec("update t2 set b=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("10 -10 -10", "20 -20 -20")) - tk.MustExec("update t2 set a=30, b=default where a=10;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("30 -30 -30", "20 -20 -20")) - tk.MustExec("update t2 set c=default, a=40 where c=-20;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("30 -30 -30", "40 -40 -40")) - tk.MustExec("update t2 set a=default, b=default, c=default where b=-30;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "40 -40 -40")) - tk.MustExec("update t2 set a=default(a), b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "1 -1 -1")) - // Same as in MySQL 8.0.27, but still weird behavior: a=default(b) => NULL - tk.MustExec("update t2 set a=default(b), b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows(" ", " ")) - tk.MustGetErrCode("update t2 set b=default(a);", mysql.ErrBadGeneratedColumn) - tk.MustExec("update t2 set a=default(a), c=default(c)") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "1 -1 -1")) - // Same as in MySQL 8.0.27, but still weird behavior: a=default(b) => NULL - tk.MustExec("update t2 set a=default(b), b=default(b)") - tk.MustQuery("select * from t2;").Check(testkit.Rows(" ", " ")) - tk.MustExec("update t2 set a=default(a), c=default(c)") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "1 -1 -1")) - // Allowed in MySQL, but should probably not be allowed. - tk.MustGetErrCode("update t2 set a=default(a), c=default(a)", mysql.ErrBadGeneratedColumn) - tk.MustExec("drop table t1, t2") -} - func TestListColumnsPartitionWithGlobalIndex(t *testing.T) { failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") diff --git a/pkg/executor/update_test.go b/pkg/executor/update_test.go index 03586f8225c84..e6052ca91ea61 100644 --- a/pkg/executor/update_test.go +++ b/pkg/executor/update_test.go @@ -69,32 +69,6 @@ func getPresumeExistsCount(t *testing.T, se session.Session) int { return presumeNotExistsCnt } -func TestIssue21447(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk1, tk2 := testkit.NewTestKit(t, store), testkit.NewTestKit(t, store) - tk1.MustExec("use test") - tk2.MustExec("use test") - - tk1.MustExec("drop table if exists t1") - tk1.MustExec("create table t1(id int primary key, name varchar(40))") - tk1.MustExec("insert into t1 values(1, 'abc')") - - tk1.MustExec("begin pessimistic") - tk2.MustExec("begin pessimistic") - tk2.MustExec("update t1 set name='xyz' where id=1") - tk2.CheckExecResult(1, 0) - tk2.MustQuery("select * from t1 where id = 1").Check(testkit.Rows("1 xyz")) - tk2.MustExec("commit") - tk1.MustExec("update t1 set name='xyz' where id=1") - tk1.CheckExecResult(0, 0) - tk1.MustQuery("select * from t1 where id = 1").Check(testkit.Rows("1 abc")) - tk1.MustQuery("select * from t1 where id = 1 for update").Check(testkit.Rows("1 xyz")) - tk1.MustQuery("select * from t1 where id in (1, 2)").Check(testkit.Rows("1 abc")) - tk1.MustQuery("select * from t1 where id in (1, 2) for update").Check(testkit.Rows("1 xyz")) - tk1.MustExec("commit") -} - func TestLockUnchangedUniqueKeys(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/pkg/planner/core/binary_plan_test.go b/pkg/planner/core/binary_plan_test.go index 55959a7d0f647..268510d156d6c 100644 --- a/pkg/planner/core/binary_plan_test.go +++ b/pkg/planner/core/binary_plan_test.go @@ -377,23 +377,6 @@ func TestDecodeBinaryPlan(t *testing.T) { } } -func TestInvalidDecodeBinaryPlan(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - str1 := "some random bytes" - str2 := base64.StdEncoding.EncodeToString([]byte(str1)) - str3 := base64.StdEncoding.EncodeToString(snappy.Encode(nil, []byte(str1))) - - tk.MustQuery(`select tidb_decode_binary_plan('` + str1 + `')`).Check(testkit.Rows("")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 illegal base64 data at input byte 4")) - tk.MustQuery(`select tidb_decode_binary_plan('` + str2 + `')`).Check(testkit.Rows("")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 snappy: corrupt input")) - tk.MustQuery(`select tidb_decode_binary_plan('` + str3 + `')`).Check(testkit.Rows("")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 proto: illegal wireType 7")) -} - func TestUnnecessaryBinaryPlanInSlowLog(t *testing.T) { originCfg := config.GetGlobalConfig() newCfg := *originCfg diff --git a/pkg/planner/core/plan_cache_test.go b/pkg/planner/core/plan_cache_test.go index f7d10edaf5ea2..a8be4a0413b04 100644 --- a/pkg/planner/core/plan_cache_test.go +++ b/pkg/planner/core/plan_cache_test.go @@ -51,45 +51,6 @@ func TestInitLRUWithSystemVar(t *testing.T) { require.NotNil(t, lru) } -func TestIssue45086(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec(`use test`) - - tk.MustExec(`CREATE TABLE t (a int(11) DEFAULT NULL, b date DEFAULT NULL)`) - tk.MustExec(`INSERT INTO t VALUES (1, current_date())`) - - tk.MustExec(`PREPARE stmt FROM 'SELECT * FROM t WHERE b=current_date()'`) - require.Equal(t, len(tk.MustQuery(`EXECUTE stmt`).Rows()), 1) -} - -func TestPlanCacheSizeSwitch(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // default value = 100 - tk.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("100")) - tk.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("100")) - - // keep the same value when updating any one of them - tk.MustExec(`set @@tidb_prepared_plan_cache_size = 200`) - tk.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("200")) - tk.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("200")) - tk.MustExec(`set @@tidb_session_plan_cache_size = 300`) - tk.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("300")) - tk.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("300")) - - tk.MustExec(`set global tidb_prepared_plan_cache_size = 400`) - tk1 := testkit.NewTestKit(t, store) - tk1.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("400")) - tk1.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("400")) - - tk.MustExec(`set global tidb_session_plan_cache_size = 500`) - tk2 := testkit.NewTestKit(t, store) - tk2.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("500")) - tk2.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("500")) -} - func TestNonPreparedPlanCachePlanString(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -119,34 +80,6 @@ func TestNonPreparedPlanCachePlanString(t *testing.T) { tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) } -func TestNonPreparedPlanCacheWithExplain(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec(`use test`) - tk.MustExec("create table t(a int)") - tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") - tk.MustExec("select * from t where a=1") // cache this plan - - tk.MustQuery("explain select * from t where a=2").Check(testkit.Rows( - `TableReader_7 10.00 root data:Selection_6`, - `└─Selection_6 10.00 cop[tikv] eq(test.t.a, 2)`, - ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - - tk.MustQuery("explain format=verbose select * from t where a=2").Check(testkit.Rows( - `TableReader_7 10.00 168975.57 root data:Selection_6`, - `└─Selection_6 10.00 2534000.00 cop[tikv] eq(test.t.a, 2)`, - ` └─TableFullScan_5 10000.00 2035000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - - tk.MustQuery("explain analyze select * from t where a=2").CheckAt([]int{0, 1, 2, 3}, [][]interface{}{ - {"TableReader_7", "10.00", "0", "root"}, - {"└─Selection_6", "10.00", "0", "cop[tikv]"}, - {" └─TableFullScan_5", "10000.00", "0", "cop[tikv]"}, - }) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) -} - func TestNonPreparedPlanCacheInformationSchema(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -339,95 +272,6 @@ func TestIssue38533(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } -func TestPlanCacheGeneratedCols(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`set @@tidb_opt_fix_control = "45798:on"`) - tk.MustExec(`create table t1 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))))`) - tk.MustExec(`create table t2 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) virtual)`) - tk.MustExec(`create table t3 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) stored)`) - tk.MustExec(`create table t4 (a int, info json, index zips( (CAST(info->'$.zipcode' AS UNSIGNED ARRAY))))`) - - tk.MustExec(`set @a=1`) - tk.MustExec(`set @b=2`) - - tk.MustExec(`prepare s1 from 'select * from t1 where a=?'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustQuery(`execute s1 using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute s1 using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache - - tk.MustExec(`prepare s1 from 'select * from t2 where a=?'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustQuery(`execute s1 using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute s1 using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache - - tk.MustExec(`prepare s1 from 'select * from t3 where a=?'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustQuery(`execute s1 using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute s1 using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache - - tk.MustExec(`prepare s1 from 'select * from t4 where a=?'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustQuery(`execute s1 using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute s1 using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache -} - -func TestPlanCacheGeneratedCols2(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`set @@tidb_opt_fix_control = "45798:on"`) - tk.MustExec(`CREATE TABLE t1 ( - ipk varbinary(255) NOT NULL, - i_id varchar(45) DEFAULT NULL, - i_set_id varchar(45) DEFAULT NULL, - p_id varchar(45) DEFAULT NULL, - p_set_id varchar(45) DEFAULT NULL, - m_id bigint(20) DEFAULT NULL, - m_i_id varchar(127) DEFAULT NULL, - m_i_set_id varchar(127) DEFAULT NULL, - d json DEFAULT NULL, - p_sources json DEFAULT NULL, - nslc json DEFAULT NULL, - cl json DEFAULT NULL, - fii json DEFAULT NULL, - fpi json DEFAULT NULL, - PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */, - UNIQUE KEY i_id (i_id), - KEY d ((cast(d as char(253) array))), - KEY m_i_id (m_i_id), - KEY m_i_set_id (m_i_set_id), - KEY fpi ((cast(fpi as unsigned array))), - KEY nslc ((cast(nslc as char(1000) array))), - KEY cl ((cast(cl as char(3000) array))), - KEY fii ((cast(fii as unsigned array))), - KEY m_id (m_id), - KEY i_set_id (i_set_id), - KEY m_i_and_m_id (m_i_id,m_id))`) - - tk.MustExec(`CREATE TABLE t2 ( - ipk varbinary(255) NOT NULL, - created_time bigint(20) DEFAULT NULL, - arrival_time bigint(20) DEFAULT NULL, - updated_time bigint(20) DEFAULT NULL, - timestamp_data json DEFAULT NULL, - PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */)`) - - tk.MustExec(`prepare stmt from 'select * - from ( t1 left outer join t2 on ( t1 . ipk = t2 . ipk ) ) - where ( t1 . i_id = ? )'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustExec(`set @a='a', @b='b'`) - tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute stmt using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache -} - func TestInvalidRange(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -511,25 +355,6 @@ func TestIssue38205(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } -func TestPlanCacheExprBlacklistCompatibility(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t (a int)") - - tk.MustExec("prepare st from 'select * from t where mod(a, 2)=1'") - tk.MustExec("execute st") - tk.MustExec("execute st") - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) - - tk.MustExec("insert into mysql.expr_pushdown_blacklist(name) values('mod')") - tk.MustExec(`admin reload expr_pushdown_blacklist`) - tk.MustExec("execute st") // no `mod can not be pushed-down` error - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // expr blacklist is updated - tk.MustExec("execute st") - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) -} - func TestIssue40224(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1050,41 +875,6 @@ func TestPlanCacheSubquerySPMEffective(t *testing.T) { } } -func TestNonPreparedPlanCacheFieldNames(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec(`use test`) - tk.MustExec("create table t(a int, index(a))") - tk.MustExec("create table tt(a varchar(10))") - tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") - - checkFieldName := func(sql, hit string, fields ...string) { - rs, err := tk.Exec(sql) - require.NoError(t, err) - for i, f := range rs.Fields() { - require.Equal(t, f.Column.Name.L, fields[i]) - } - require.NoError(t, rs.Close()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(hit)) - } - - checkFieldName(`select a+1 from t where a<10`, `0`, `a+1`) - checkFieldName(`select a+1 from t where a<20`, `1`, `a+1`) - checkFieldName(`select a+2 from t where a<30`, `0`, `a+2`) // can not hit since field names changed - checkFieldName(`select a+2 from t where a<40`, `1`, `a+2`) - checkFieldName(`select a,a+1 from t where a<30`, `0`, `a`, `a+1`) // can not hit since field names changed - checkFieldName(`select a,a+1 from t where a<40`, `1`, `a`, `a+1`) - checkFieldName(`select a+'123' from tt where a='1'`, `0`, `a+'123'`) - checkFieldName(`select a+'123' from tt where a='2'`, `1`, `a+'123'`) - - checkFieldName(`select 1 from t where a<10`, `0`, `1`) - checkFieldName(`select 1 from t where a<20`, `1`, `1`) - checkFieldName(`select 2 from t where a<10`, `0`, `2`) - checkFieldName(`select 2 from t where a<20`, `1`, `2`) - checkFieldName(`select 1,2 from t where a<10`, `0`, `1`, `2`) - checkFieldName(`select 1,2 from t where a<20`, `1`, `1`, `2`) -} - func TestIssue42125(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1402,51 +1192,6 @@ func TestBuiltinFuncFlen(t *testing.T) { } } -func TestNonPreparedPlanCacheBuiltinFuncs(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) - tk.MustExec(`create table t (a int, b varchar(32), c datetime, key(a))`) - - // normal builtin functions can be supported - supportedCases := []string{ - `select * from t where mod(a, 5) < 2`, - `select * from t where c < now()`, - `select date_format(c, '%Y-%m-%d') from t where a < 10`, - `select str_to_date(b, '%Y-%m-%d') from t where a < 10`, - `select * from t where a-2 < 20`, - `select * from t where a+b > 100`, - } - for _, sql := range supportedCases { - tk.MustExec(sql) - tk.MustExec(sql) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) - } - - // unsupported cases - unsupportedCases := []string{ - `select * from t where -a > 10`, // '-' cannot support - `select * from t where a < 1 and b like '%abc%'`, // LIKE - `select database() from t`, - } - for _, sql := range unsupportedCases { - tk.MustExec(sql) - tk.MustExec(sql) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) - } -} - -func TestIssue48165(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`create table t(a int)`) - tk.MustExec(`insert into t values(1)`) - tk.MustExec(`prepare s from "select * from t where tidb_parse_tso(a) > unix_timestamp()"`) - tk.MustQuery(`execute s`).Check(testkit.Rows("1")) -} - func BenchmarkPlanCacheInsert(b *testing.B) { store := testkit.CreateMockStore(b) tk := testkit.NewTestKit(b, store) diff --git a/pkg/planner/core/plan_test.go b/pkg/planner/core/plan_test.go index 894fc966a9f35..1b858bad5c877 100644 --- a/pkg/planner/core/plan_test.go +++ b/pkg/planner/core/plan_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/aggregation" - "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -702,17 +701,6 @@ func TestBuildFinalModeAggregation(t *testing.T) { checkResult(ctx, mixedAggFuncs, groupByItems) } -func TestIssue40857(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.MustExec("drop table if exists t;") - tk.MustExec("CREATE TABLE t (c1 mediumint(9) DEFAULT '-4747160',c2 year(4) NOT NULL DEFAULT '2075',c3 double DEFAULT '1.1559030660251948',c4 enum('wbv4','eli','d8ym','m3gsx','lz7td','o','d1k7l','y1x','xcxq','bj','n7') DEFAULT 'xcxq',c5 int(11) DEFAULT '255080866',c6 tinyint(1) DEFAULT '1',PRIMARY KEY (c2),KEY `c4d86d54-091c-4307-957b-b164c9652b7f` (c6,c4) );") - tk.MustExec("insert into t values (-4747160, 2075, 722.5719203870632, 'xcxq', 1576824797, 1);") - tk.MustExec("select /*+ stream_agg() */ bit_or(t.c5) as r0 from t where t.c3 in (select c6 from t where not(t.c6 <> 1) and not(t.c3 in(9263.749352636818))) group by t.c1;") - require.Empty(t, tk.Session().LastMessage()) -} - func TestCloneFineGrainedShuffleStreamCount(t *testing.T) { window := &core.PhysicalWindow{} newPlan, err := window.Clone() @@ -742,38 +730,3 @@ func TestCloneFineGrainedShuffleStreamCount(t *testing.T) { require.Equal(t, ok, true) require.Equal(t, sort.TiFlashFineGrainedShuffleStreamCount, newSort.TiFlashFineGrainedShuffleStreamCount) } - -func TestIssue40535(t *testing.T) { - store := testkit.CreateMockStore(t) - var cfg kv.InjectionConfig - tk := testkit.NewTestKit(t, kv.NewInjectedStore(store, &cfg)) - tk.MustExec("use test;") - tk.MustExec("drop table if exists t1; drop table if exists t2;") - tk.MustExec("CREATE TABLE `t1`(`c1` bigint(20) NOT NULL DEFAULT '-2312745469307452950', `c2` datetime DEFAULT '5316-02-03 06:54:49', `c3` tinyblob DEFAULT NULL, PRIMARY KEY (`c1`) /*T![clustered_index] CLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;") - tk.MustExec("CREATE TABLE `t2`(`c1` set('kn8pu','7et','vekx6','v3','liwrh','q14','1met','nnd5i','5o0','8cz','l') DEFAULT '7et,vekx6,liwrh,q14,1met', `c2` float DEFAULT '1.683167', KEY `k1` (`c2`,`c1`), KEY `k2` (`c2`)) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci;") - tk.MustExec("(select /*+ agg_to_cop()*/ locate(t1.c3, t1.c3) as r0, t1.c3 as r1 from t1 where not( IsNull(t1.c1)) order by r0,r1) union all (select concat_ws(',', t2.c2, t2.c1) as r0, t2.c1 as r1 from t2 order by r0, r1) order by 1 limit 273;") - require.Empty(t, tk.Session().LastMessage()) -} - -func TestIssue47445(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.MustExec("CREATE TABLE golang1 ( `fcbpdt` CHAR (8) COLLATE utf8_general_ci NOT NULL, `fcbpsq` VARCHAR (20) COLLATE utf8_general_ci NOT NULL, `procst` char (4) COLLATE utf8_general_ci DEFAULT NULL,`cipstx` VARCHAR (105) COLLATE utf8_general_ci DEFAULT NULL, `cipsst` CHAR (4) COLLATE utf8_general_ci DEFAULT NULL, `dyngtg` VARCHAR(4) COLLATE utf8_general_ci DEFAULT NULL, `blncdt` VARCHAR (8) COLLATE utf8_general_ci DEFAULT NULL, PRIMARY KEY ( fcbpdt, fcbpsq ))") - tk.MustExec("insert into golang1 values('20230925','12023092502158016','abc','','','','')") - tk.MustExec("create table golang2 (`sysgrp` varchar(20) NOT NULL,`procst` varchar(8) NOT NULL,`levlid` int(11) NOT NULL,PRIMARY key (procst));") - tk.MustExec("insert into golang2 VALUES('COMMON','ACSC',90)") - tk.MustExec("insert into golang2 VALUES('COMMON','abc',8)") - tk.MustExec("insert into golang2 VALUES('COMMON','CH02',6)") - tk.MustExec("UPDATE golang1 a SET procst =(CASE WHEN ( SELECT levlid FROM golang2 b WHERE b.sysgrp = 'COMMON' AND b.procst = 'ACSC' ) > ( SELECT levlid FROM golang2 c WHERE c.sysgrp = 'COMMON' AND c.procst = a.procst ) THEN 'ACSC' ELSE a.procst END ), cipstx = 'CI010000', cipsst = 'ACSC', dyngtg = 'EAYT', blncdt= '20230925' WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'") - tk.MustQuery("select * from golang1").Check(testkit.Rows("20230925 12023092502158016 ACSC CI010000 ACSC EAYT 20230925")) - tk.MustExec("UPDATE golang1 a SET procst= (SELECT 1 FROM golang2 c WHERE c.procst = a.procst) WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'") - tk.MustQuery("select * from golang1").Check(testkit.Rows("20230925 12023092502158016 1 CI010000 ACSC EAYT 20230925")) -} - -func TestExplainValuesStatement(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustMatchErrMsg("EXPLAIN FORMAT = TRADITIONAL ((VALUES ROW ()) ORDER BY 1)", ".*Unknown table ''.*") -} diff --git a/tests/integrationtest/r/executor/cte.result b/tests/integrationtest/r/executor/cte.result index 2721c2f4b9c10..60f4967359e55 100644 --- a/tests/integrationtest/r/executor/cte.result +++ b/tests/integrationtest/r/executor/cte.result @@ -1,3 +1,4 @@ +set tidb_max_chunk_size=default; with recursive cte1 as (select 1 c1 union all select c1 + 1 c1 from cte1 where c1 < 5) select * from cte1; c1 1 diff --git a/tests/integrationtest/r/executor/delete.result b/tests/integrationtest/r/executor/delete.result index f92a7a503effd..d452dd98a9e21 100644 --- a/tests/integrationtest/r/executor/delete.result +++ b/tests/integrationtest/r/executor/delete.result @@ -14,3 +14,96 @@ execute stmt using @a; select * from t; a 2 +drop table if exists delete_test; +drop view if exists v; +drop sequence if exists seq; +create table delete_test(id int not null default 1, name varchar(255), PRIMARY KEY(id)); +insert INTO delete_test VALUES (1, "hello"); +insert into delete_test values (2, "hello"); +update delete_test set name = "abc" where id = 2; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +delete from delete_test where id = 2 limit 1; +affected rows: 1 +info: +delete from delete_test where 0; +affected rows: 0 +info: +insert into delete_test values (2, 'abc'); +delete from delete_test where delete_test.id = 2 limit 1; +affected rows: 1 +info: +begin; +SELECT * from delete_test limit 2; +id name +1 hello +commit; +insert into delete_test values (2, 'abc'); +delete from delete_test where id = (select '2a'); +Error 1292 (22007): Truncated incorrect DOUBLE value: '2a' +delete ignore from delete_test where id = (select '2a'); +affected rows: 1 +info: +SHOW WARNINGS; +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1292 Truncated incorrect DOUBLE value: '2a' +delete from delete_test; +affected rows: 1 +info: +create view v as select * from delete_test; +delete from v where name = 'aaa'; +Error 1105 (HY000): delete view v is not supported now +drop view v; +create sequence seq; +delete from seq; +Error 1105 (HY000): delete sequence seq is not supported now +drop sequence seq; +drop table if exists t1, t2; +create table t1 (c1 int, c2 int, index (c1)); +create table t2 (c1 int, c2 int); +insert into t1 values (1, 1), (2, 2); +delete from t1 where t1.c1 = 1; +affected rows: 1 +info: +delete from t1 where t1.c2 = 2; +affected rows: 1 +info: +select * from t1; +c1 c2 +insert into t1 values (1, 3); +delete from t1 as a where a.c1 = 1; +affected rows: 1 +info: +insert into t1 values (1, 1), (2, 2); +insert into t2 values (2, 1), (3,1); +delete t1, t2 from t1 join t2 where t1.c1 = t2.c2; +affected rows: 3 +info: +insert into t2 values (2, 1), (3,1); +delete a, b from t1 as a join t2 as b where a.c2 = b.c1; +affected rows: 2 +info: +delete t1, t2 from t1 as a join t2 as b where a.c2 = b.c1; +Error 1109 (42S02): Unknown table 't1' in MULTI DELETE +drop table if exists t1, t2, t3; +create table t1 (id int, data int); +insert into t1 values (11, 121), (12, 122), (13, 123); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +create table t2 (id int, data int); +insert into t2 values (11, 221), (22, 222), (23, 223); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +create table t3 (id int, data int); +insert into t3 values (11, 321), (22, 322), (23, 323); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +delete t1, t2 from t1 inner join t2 inner join t3 where t1.id=t2.id and t2.id=t3.id; +affected rows: 2 +info: +select * from t3; +id data +11 321 +22 322 +23 323 diff --git a/tests/integrationtest/r/executor/executor.result b/tests/integrationtest/r/executor/executor.result index c7f2c7bde73e0..e84cda08a7f47 100644 --- a/tests/integrationtest/r/executor/executor.result +++ b/tests/integrationtest/r/executor/executor.result @@ -4405,3 +4405,147 @@ id a b c d 1 1 1 1 1 5 5 5 5 5 set @@tidb_enable_collect_execution_info=default; +drop table if exists t1; +create table t1 (a int, b int, index(a)); +insert into t1 values (1,2),(2,3),(3,4); +explain analyze select * from t1 use index(a) where a > 1; +id estRows actRows task access object execution info operator info memory disk +IndexLookUp_7 3333.33 2 root NULL .*time:.*loops:.*index_task:.*table_task: {total_time.*num.*concurrency.*}.* NULL KB N/A +├─IndexRangeScan_5(Build) 3333.33 2 cop[tikv] table:t1, index:a(a) .*time:.*loops:.*cop_task:.* range:(1,+inf], keep order:false, stats:pseudo N/A N/A +└─TableRowIDScan_6(Probe) 3333.33 2 cop[tikv] table:t1 .*time:.*loops:.*cop_task:.* keep order:false, stats:pseudo N/A N/A +drop table if exists t1; +create table t1 (a int, b int); +insert into t1 values (1,2),(2,3),(3,4); +explain analyze SELECT /*+ HASH_AGG() */ count(*) FROM t1 WHERE a < 10; +id estRows actRows task access object execution info operator info memory disk +HashAgg_11 1.00 1 root NULL .*time:.*loops:.*partial_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*final_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.* funcs:count(Column#5)->Column#4 KB N/A +└─TableReader_12 1.00 1 root NULL time.*loops.*cop_task.* data:HashAgg_6 Bytes N/A + └─HashAgg_6 1.00 1 cop[tikv] NULL tikv_task:.* funcs:count(1)->Column#5 N/A N/A + └─Selection_10 3323.33 3 cop[tikv] NULL tikv_task:.* lt(executor__executor.t1.a, 10) N/A N/A + └─TableFullScan_9 10000.00 3 cop[tikv] table:t1 tikv_task:.* keep order:false, stats:pseudo N/A N/A +set global tidb_txn_mode=''; +drop table if exists t, t1; +create table t (c1 int, c2 int, c3 int); +insert t values (11, 2, 3); +insert t values (12, 2, 3); +insert t values (13, 2, 3); +create table t1 (c1 int); +insert t1 values (11); +begin; +select * from t where c1=11 for update; +c1 c2 c3 +11 2 3 +begin; +update t set c2=211 where c1=11; +commit; +commit; +Error 9007 (HY000): Write conflict, reason=Optimistic [try again later] +begin; +select * from t where exists(select null from t1 where t1.c1=t.c1) for update; +c1 c2 c3 +11 211 3 +begin; +update t set c2=211 where c1=12; +commit; +commit; +begin; +select * from t where c1=11 for update; +c1 c2 c3 +11 211 3 +begin; +update t set c2=22 where c1=12; +commit; +commit; +set @@autocommit=1; +select * from t where c1=11 for update; +c1 c2 c3 +11 211 3 +begin; +update t set c2=211 where c1=11; +commit; +commit; +begin; +select * from (select * from t for update) t join t1 for update; +c1 c2 c3 c1 +11 211 3 11 +12 22 3 11 +13 2 3 11 +begin; +update t1 set c1 = 13; +commit; +commit; +Error 9007 (HY000): Write conflict, reason=Optimistic [try again later] +set global tidb_txn_mode=pessimistic; +drop table if exists t, t1; +create table t (i int); +create table t1 (i int); +insert t values (1); +insert t1 values (1); +begin pessimistic; +select * from t, t1 where t.i = t1.i for update of t; +i i +1 1 +begin pessimistic; +select * from t1 for update; +i +1 +select * from t for update nowait; +Error 3572 (HY000): Statement aborted because lock(s) could not be acquired immediately and NOWAIT is set. +rollback; +select * from t for update nowait; +i +1 +rollback; +set session tidb_txn_mode=''; +drop table if exists t; +create table t(a int); +insert into t values (1); +begin; +select 1 as a union select a from t for update; +a +1 +set session tidb_txn_mode=''; +update t set a = a + 1; +commit; +Error 9007 (HY000): Write conflict, reason=Optimistic [try again later] +begin; +select 1 as a union select a from t limit 5 for update; +a +1 +2 +select 1 as a union select a from t order by a for update; +a +1 +2 +update t set a = a + 1; +commit; +Error 9007 (HY000): Write conflict, reason=Optimistic [try again later] +set session tidb_txn_mode=pessimistic; +drop table if exists t; +create table t (id bigint key,b int); +split table t by (10),(20),(30); +TOTAL_SPLIT_REGION SCATTER_FINISH_RATIO +3 1 +insert into t values (0,0),(10,10),(20,20),(30,30); +alter table t add index idx1(b); +admin show ddl jobs 1; +JOB_ID DB_NAME TABLE_NAME JOB_TYPE SCHEMA_STATE SCHEMA_ID TABLE_ID ROW_COUNT CREATE_TIME START_TIME END_TIME STATE + executor__executor t public 4 synced +insert into t values (1,0),(2,10),(3,20),(4,30); +alter table t add index idx2(b); +admin show ddl jobs 1; +JOB_ID DB_NAME TABLE_NAME JOB_TYPE SCHEMA_STATE SCHEMA_ID TABLE_ID ROW_COUNT CREATE_TIME START_TIME END_TIME STATE + executor__executor t public 8 synced +drop table if exists t; +create table t(a int, b int as(-a)); +insert into t(a) values(1), (3), (7); +SET GLOBAL tidb_mem_oom_action='CANCEL'; +set @@tidb_mem_quota_query=1; +update t set t.a = t.a - 1 where t.a in (select a from t where a < 4); +Error 8175 (HY000): Your query has been cancelled due to exceeding the allowed memory limit for a single SQL query. Please try narrowing your query scope or increase the tidb_mem_quota_query limit and try again.[conn=] +set @@tidb_mem_quota_query=1000000000; +select stmt_type from information_schema.statements_summary where digest_text = 'update `t` set `t` . `a` = `t` . `a` - ? where `t` . `a` in ( select `a` from `t` where `a` < ? )'; +stmt_type +Update +set @@tidb_mem_quota_query=default; +set global tidb_mem_oom_action=default; diff --git a/tests/integrationtest/r/executor/insert.result b/tests/integrationtest/r/executor/insert.result index 1c9bd502d91bf..977ff1e9ac41e 100644 --- a/tests/integrationtest/r/executor/insert.result +++ b/tests/integrationtest/r/executor/insert.result @@ -1222,3 +1222,912 @@ commit; commit; Error 1062 (23000): Duplicate entry '146576795' for key 't.PRIMARY' set global tidb_disable_txn_auto_retry=default; +drop table if exists t; +create table t (id smallint auto_increment primary key); +alter table t add column c1 int default 1; +insert ignore into t(id) values (194626268); +affected rows: 1 +info: +select * from t; +id c1 +32767 1 +insert ignore into t(id) values ('*') on duplicate key update c1 = 2; +affected rows: 2 +info: +select * from t; +id c1 +32767 2 +drop table if exists t; +create table t (i int not null primary key, j int unique key); +insert into t values (1, 1), (2, 2); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +insert ignore into t values(1, 1) on duplicate key update i = 2; +affected rows: 0 +info: +select * from t; +i j +1 1 +2 2 +insert ignore into t values(1, 1) on duplicate key update j = 2; +affected rows: 0 +info: +select * from t; +i j +1 1 +2 2 +drop table if exists t2; +create table t2(`col_25` set('Alice','Bob','Charlie','David') NOT NULL,`col_26` date NOT NULL DEFAULT '2016-04-15', PRIMARY KEY (`col_26`) clustered, UNIQUE KEY `idx_9` (`col_25`,`col_26`),UNIQUE KEY `idx_10` (`col_25`)); +insert into t2(col_25, col_26) values('Bob', '1989-03-23'),('Alice', '2023-11-24'), ('Charlie', '2023-12-05'); +insert ignore into t2 (col_25,col_26) values ( 'Bob','1977-11-23' ) on duplicate key update col_25 = 'Alice', col_26 = '2036-12-13'; +show warnings; +Level Code Message +Warning 1062 Duplicate entry 'Alice' for key 't2.idx_10' +select * from t2; +col_25 col_26 +Bob 1989-03-23 +Alice 2023-11-24 +Charlie 2023-12-05 +drop table if exists t4; +create table t4(id int primary key clustered, k int, v int, unique key uk1(k)); +insert into t4 values (1, 10, 100), (3, 30, 300); +insert ignore into t4 (id, k, v) values(1, 0, 0) on duplicate key update id = 2, k = 30; +show warnings; +Level Code Message +Warning 1062 Duplicate entry '30' for key 't4.uk1' +select * from t4; +id k v +1 10 100 +3 30 300 +drop table if exists t5; +create table t5(k1 varchar(100), k2 varchar(100), uk1 int, v int, primary key(k1, k2) clustered, unique key ukk1(uk1), unique key ukk2(v)); +insert into t5(k1, k2, uk1, v) values('1', '1', 1, '100'), ('1', '3', 2, '200'); +update ignore t5 set k2 = '2', uk1 = 2 where k1 = '1' and k2 = '1'; +show warnings; +Level Code Message +Warning 1062 Duplicate entry '2' for key 't5.ukk1' +select * from t5; +k1 k2 uk1 v +1 1 1 100 +1 3 2 200 +drop table if exists t6; +create table t6 (a int, b int, c int, primary key(a, b) clustered, unique key idx_14(b), unique key idx_15(b), unique key idx_16(a, b)); +insert into t6 select 10, 10, 20; +insert ignore into t6 set a = 20, b = 10 on duplicate key update a = 100; +select * from t6; +a b c +100 10 20 +insert ignore into t6 set a = 200, b= 10 on duplicate key update c = 1000; +select * from t6; +a b c +100 10 1000 +drop table if exists insert_autoinc_test; +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(c1) values (1), (2); +begin; +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +commit; +begin; +insert into insert_autoinc_test(id, c1) values (5,5); +insert into insert_autoinc_test(c1) values (6); +commit; +begin; +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +5 5 +6 6 +commit; +begin; +insert into insert_autoinc_test(id, c1) values (3,3); +commit; +begin; +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +3 3 +5 5 +6 6 +commit; +begin; +insert into insert_autoinc_test(c1) values (7); +commit; +begin; +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +3 3 +5 5 +6 6 +7 7 +commit; +drop table if exists insert_autoinc_test; +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(id, c1) values (0.3, 1); +select * from insert_autoinc_test; +id c1 +1 1 +insert into insert_autoinc_test(id, c1) values (-0.3, 2); +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +insert into insert_autoinc_test(id, c1) values (-3.3, 3); +select * from insert_autoinc_test; +id c1 +-3 3 +1 1 +2 2 +insert into insert_autoinc_test(id, c1) values (4.3, 4); +select * from insert_autoinc_test; +id c1 +-3 3 +1 1 +2 2 +4 4 +insert into insert_autoinc_test(c1) values (5); +select * from insert_autoinc_test; +id c1 +-3 3 +1 1 +2 2 +4 4 +5 5 +insert into insert_autoinc_test(id, c1) values (null, 6); +select * from insert_autoinc_test; +id c1 +-3 3 +1 1 +2 2 +4 4 +5 5 +6 6 +drop table if exists insert_autoinc_test; +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(id, c1) values (5, 1); +select * from insert_autoinc_test; +id c1 +5 1 +insert into insert_autoinc_test(id, c1) values (0, 2); +select * from insert_autoinc_test; +id c1 +5 1 +6 2 +insert into insert_autoinc_test(id, c1) values (0, 3); +select * from insert_autoinc_test; +id c1 +5 1 +6 2 +7 3 +set SQL_MODE=NO_AUTO_VALUE_ON_ZERO; +insert into insert_autoinc_test(id, c1) values (0, 4); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +insert into insert_autoinc_test(id, c1) values (0, 5); +Error 1062 (23000): Duplicate entry '0' for key 'insert_autoinc_test.PRIMARY' +insert into insert_autoinc_test(c1) values (6); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +8 6 +insert into insert_autoinc_test(id, c1) values (null, 7); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +8 6 +9 7 +set SQL_MODE=''; +insert into insert_autoinc_test(id, c1) values (0, 8); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +8 6 +9 7 +10 8 +insert into insert_autoinc_test(id, c1) values (null, 9); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +8 6 +9 7 +10 8 +11 9 +set sql_mode = default; +drop table if exists insert_test; +create table insert_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1); +insert insert_test (c1) values (1),(2),(NULL); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +begin; +insert insert_test (c1) values (); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +insert insert_test (c1, c2) values (1,2),(1); +Error 1136 (21S01): Column count doesn't match value count at row 2 +rollback; +begin; +insert insert_test (xxx) values (3); +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +begin; +insert insert_test_xxx (c1) values (); +Error 1146 (42S02): Table 'executor__insert.insert_test_xxx' doesn't exist +rollback; +insert insert_test set c1 = 3; +affected rows: 1 +info: +begin; +insert insert_test set c1 = 4, c1 = 5; +Error 1110 (42000): Column 'c1' specified twice +rollback; +begin; +insert insert_test set xxx = 6; +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +drop table if exists insert_test_1, insert_test_2; +create table insert_test_1 (id int, c1 int); +insert insert_test_1 select id, c1 from insert_test; +affected rows: 4 +info: Records: 4 Duplicates: 0 Warnings: 0 +create table insert_test_2 (id int, c1 int); +insert insert_test_1 select id, c1 from insert_test union select id * 10, c1 * 10 from insert_test; +affected rows: 8 +info: Records: 8 Duplicates: 0 Warnings: 0 +begin; +insert insert_test_1 select c1 from insert_test; +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +insert insert_test_1 values(default, default, default, default, default); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +select * from insert_test where id = 1; +id c1 c2 c3 +1 1 NULL 1 +insert into insert_test (id, c3) values (1, 2) on duplicate key update id=values(id), c2=10; +affected rows: 2 +info: +select * from insert_test where id = 1; +id c1 c2 c3 +1 1 10 1 +insert into insert_test (id, c2) values (1, 1) on duplicate key update insert_test.c2=10; +affected rows: 0 +info: +insert into insert_test (id, c2) values(1, 1) on duplicate key update t.c2 = 10; +Error 1054 (42S22): Unknown column 't.c2' in 'field list' +INSERT INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3; +affected rows: 2 +info: +select * from insert_test where id = 1; +id c1 c2 c3 +1 1 10 6 +INSERT IGNORE INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3; +affected rows: 2 +info: +select * from insert_test where id = 1; +id c1 c2 c3 +1 1 10 11 +drop table if exists insert_err; +create table insert_err (id int, c1 varchar(8)); +insert insert_err values (1, 'abcdabcdabcd'); +Error 1406 (22001): Data too long for column 'c1' at row 1 +insert insert_err values (1, '你好,世界'); +create table TEST1 (ID INT NOT NULL, VALUE INT DEFAULT NULL, PRIMARY KEY (ID)); +INSERT INTO TEST1(id,value) VALUE(3,3) on DUPLICATE KEY UPDATE VALUE=4; +affected rows: 1 +info: +drop table if exists t; +create table t (id int); +insert into t values(1); +update t t1 set id = (select count(*) + 1 from t t2 where t1.id = t2.id); +select * from t; +id +2 +drop table if exists t; +create table t(c decimal(5, 5)); +insert into t value(0); +insert into t value(1); +Error 1264 (22003): Out of range value for column 'c' at row 1 +drop table if exists t; +create table t(c binary(255)); +insert into t value(1); +select length(c) from t; +length(c) +255 +drop table if exists t; +create table t(c varbinary(255)); +insert into t value(1); +select length(c) from t; +length(c) +1 +drop table if exists t; +create table t(c int); +set @@time_zone = '+08:00'; +insert into t value(Unix_timestamp('2002-10-27 01:00')); +select * from t; +c +1035651600 +set @@time_zone = default; +drop table if exists t1; +create table t1 (b char(0)); +insert into t1 values (""); +DROP TABLE IF EXISTS t; +CREATE TABLE t(a DECIMAL(4,2)); +INSERT INTO t VALUES (1.000001); +SHOW WARNINGS; +Level Code Message +Warning 1366 Incorrect decimal value: '1.000001' for column 'a' at row 1 +INSERT INTO t VALUES (1.000000); +SHOW WARNINGS; +Level Code Message +DROP TABLE IF EXISTS t; +CREATE TABLE t(a datetime); +INSERT INTO t VALUES('2017-00-00'); +Error 1292 (22007): Incorrect datetime value: '2017-00-00' for column 'a' at row 1 +set sql_mode = ''; +INSERT INTO t VALUES('2017-00-00'); +SELECT * FROM t; +a +2017-00-00 00:00:00 +set sql_mode = 'strict_all_tables'; +SELECT * FROM t; +a +2017-00-00 00:00:00 +set sql_mode = default; +drop table if exists test; +CREATE TABLE test(id int(10) UNSIGNED NOT NULL AUTO_INCREMENT, p int(10) UNSIGNED NOT NULL, PRIMARY KEY(p), KEY(id)); +insert into test(p) value(1); +select * from test; +id p +1 1 +select * from test use index (id) where id = 1; +id p +1 1 +insert into test values(NULL, 2); +select * from test use index (id) where id = 2; +id p +2 2 +insert into test values(2, 3); +select * from test use index (id) where id = 2; +id p +2 2 +2 3 +drop table if exists t; +create table t(a bigint unsigned); +set @@sql_mode = 'strict_all_tables'; +insert into t value (-1); +Error 1264 (22003): Out of range value for column 'a' at row 1 +set @@sql_mode = ''; +insert into t value (-1); +show warnings; +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +insert into t select -1; +show warnings; +Level Code Message +Warning 1690 constant -1 overflows bigint +insert into t select cast(-1 as unsigned); +insert into t value (-1.111); +show warnings; +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +insert into t value ('-1.111'); +show warnings; +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +update t set a = -1 limit 1; +show warnings; +Level Code Message +Warning 1690 constant -1 overflows bigint +select * from t; +a +0 +0 +18446744073709551615 +0 +0 +set @@sql_mode = default; +drop table if exists t; +create table t(a time(6)); +insert into t value('20070219173709.055870'), ('20070219173709.055'), ('20070219173709.055870123'); +select * from t; +a +17:37:09.055870 +17:37:09.055000 +17:37:09.055870 +truncate table t; +insert into t value(20070219173709.055870), (20070219173709.055), (20070219173709.055870123); +select * from t; +a +17:37:09.055870 +17:37:09.055000 +17:37:09.055870 +insert into t value(-20070219173709.055870); +Error 1292 (22007): Incorrect time value: '-20070219173709.055870' for column 'a' at row 1 +drop table if exists t; +set @@sql_mode=''; +create table t(a float unsigned, b double unsigned); +insert into t value(-1.1, -1.1), (-2.1, -2.1), (0, 0), (1.1, 1.1); +show warnings; +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +Warning 1264 Out of range value for column 'b' at row 1 +Warning 1264 Out of range value for column 'a' at row 2 +Warning 1264 Out of range value for column 'b' at row 2 +select * from t; +a b +0 0 +0 0 +0 0 +1.1 1.1 +set @@sql_mode=default; +drop table if exists t; +create table t(a int default 1, b int default 2); +insert into t values(default, default); +select * from t; +a b +1 2 +truncate table t; +insert into t values(default(b), default(a)); +select * from t; +a b +2 1 +truncate table t; +insert into t (b) values(default); +select * from t; +a b +1 2 +truncate table t; +insert into t (b) values(default(a)); +select * from t; +a b +1 1 +drop view if exists v; +create view v as select * from t; +insert into v values(1,2); +Error 1105 (HY000): insert into view v is not supported now +replace into v values(1,2); +Error 1105 (HY000): replace into view v is not supported now +drop view v; +drop sequence if exists seq; +create sequence seq; +insert into seq values(); +Error 1105 (HY000): insert into sequence seq is not supported now +replace into seq values(); +Error 1105 (HY000): replace into sequence seq is not supported now +drop sequence seq; +drop table if exists t; +create table t(name varchar(255), b int, c int, primary key(name(2))); +insert into t(name, b) values("cha", 3); +insert into t(name, b) values("chb", 3); +Error 1062 (23000): Duplicate entry 'ch' for key 't.PRIMARY' +insert into t(name, b) values("测试", 3); +insert into t(name, b) values("测试", 3); +Error 1062 (23000): Duplicate entry 'æµ' for key 't.PRIMARY' +drop table if exists t; +create table t (i int unique key); +insert into t values (1),(2); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +select * from t; +i +1 +2 +insert into t values (1), (2) on duplicate key update i = values(i); +affected rows: 0 +info: Records: 2 Duplicates: 0 Warnings: 0 +select * from t; +i +1 +2 +insert into t values (2), (3) on duplicate key update i = 3; +affected rows: 2 +info: Records: 2 Duplicates: 1 Warnings: 0 +select * from t; +i +1 +3 +drop table if exists t; +create table t (i int primary key, j int unique key); +insert into t values (-1, 1); +affected rows: 1 +info: +select * from t; +i j +-1 1 +insert into t values (1, 1) on duplicate key update j = values(j); +affected rows: 0 +info: +select * from t; +i j +-1 1 +drop table if exists test; +create table test (i int primary key, j int unique); +begin; +insert into test values (1,1); +insert into test values (2,1) on duplicate key update i = -i, j = -j; +commit; +select * from test; +i j +-1 -1 +delete from test; +insert into test values (1, 1); +begin; +delete from test where i = 1; +insert into test values (2, 1) on duplicate key update i = -i, j = -j; +commit; +select * from test; +i j +2 1 +delete from test; +insert into test values (1, 1); +begin; +update test set i = 2, j = 2 where i = 1; +insert into test values (1, 3) on duplicate key update i = -i, j = -j; +insert into test values (2, 4) on duplicate key update i = -i, j = -j; +commit; +select * from test order by i; +i j +-2 -2 +1 3 +delete from test; +begin; +insert into test values (1, 3), (1, 3) on duplicate key update i = values(i), j = values(j); +commit; +select * from test order by i; +i j +1 3 +create table tmp (id int auto_increment, code int, primary key(id, code)); +create table m (id int primary key auto_increment, code int unique); +insert tmp (code) values (1); +insert tmp (code) values (1); +set tidb_init_chunk_size=1; +insert m (code) select code from tmp on duplicate key update code = values(code); +select * from m; +id code +1 1 +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT AUTO_INCREMENT PRIMARY KEY, +f2 VARCHAR(5) NOT NULL UNIQUE); +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +affected rows: 1 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +affected rows: 0 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT AUTO_INCREMENT UNIQUE, +f2 VARCHAR(5) NOT NULL UNIQUE); +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +affected rows: 1 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +affected rows: 0 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = 2; +affected rows: 2 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT); +INSERT t1 VALUES (1) ON DUPLICATE KEY UPDATE f1 = 1; +affected rows: 1 +info: +SELECT * FROM t1; +f1 +1 +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT NOT NULL UNIQUE); +INSERT t1 VALUES (1, 1); +affected rows: 1 +info: +INSERT t1 VALUES (1, 1), (1, 1) ON DUPLICATE KEY UPDATE f1 = 2, f2 = 2; +affected rows: 3 +info: Records: 2 Duplicates: 1 Warnings: 0 +SELECT * FROM t1 order by f1; +f1 f2 +1 1 +2 2 +INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +Error 1048 (23000): Column 'f2' cannot be null +INSERT IGNORE t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +affected rows: 2 +info: +show warnings; +Level Code Message +Warning 1048 Column 'f2' cannot be null +SELECT * FROM t1 order by f1; +f1 f2 +1 0 +2 2 +SET sql_mode=''; +INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +show warnings; +Level Code Message +Warning 1048 Column 'f2' cannot be null +SELECT * FROM t1 order by f1; +f1 f2 +1 0 +2 2 +set sql_mode=default; +set tidb_init_chunk_size=default; +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +insert into t1 values(1, 100); +affected rows: 1 +info: +insert into t2 values(1, 200); +affected rows: 1 +info: +insert into t1 select a2, b2 from t2 on duplicate key update b1 = a2; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t1; +a1 b1 +1 1 +insert into t1 select a2, b2 from t2 on duplicate key update b1 = b2; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t1; +a1 b1 +1 200 +insert into t1 select a2, b2 from t2 on duplicate key update a1 = a2; +affected rows: 0 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t1; +a1 b1 +1 200 +insert into t1 select a2, b2 from t2 on duplicate key update b1 = 300; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t1; +a1 b1 +1 300 +insert into t1 values(1, 1) on duplicate key update b1 = 400; +affected rows: 2 +info: +select * from t1; +a1 b1 +1 400 +insert into t1 select 1, 500 from t2 on duplicate key update b1 = 400; +affected rows: 0 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t1; +a1 b1 +1 400 +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +insert into t1 select * from t2 on duplicate key update c = t2.b; +Error 1054 (42S22): Unknown column 'c' in 'field list' +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +insert into t1 select * from t2 on duplicate key update a = b; +Error 1052 (23000): Column 'b' in field list is ambiguous +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +insert into t1 select * from t2 on duplicate key update c = b; +Error 1054 (42S22): Unknown column 'c' in 'field list' +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +insert into t1 select * from t2 on duplicate key update a1 = values(b2); +Error 1054 (42S22): Unknown column 'b2' in 'field list' +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +insert into t1 values(1, 100); +affected rows: 1 +info: +insert into t2 values(1, 200); +affected rows: 1 +info: +insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t1; +a1 b1 +1 400 +insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2; +affected rows: 0 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t1; +a1 b1 +1 400 +drop table if exists t; +create table t(k1 bigint, k2 bigint, val bigint, primary key(k1, k2)); +insert into t (val, k1, k2) values (3, 1, 2); +affected rows: 1 +info: +select * from t; +k1 k2 val +1 2 3 +insert into t (val, k1, k2) select c, a, b from (select 1 as a, 2 as b, 4 as c) tmp on duplicate key update val = tmp.c; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t; +k1 k2 val +1 2 4 +drop table if exists t; +create table t(k1 double, k2 double, v double, primary key(k1, k2)); +insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c; +affected rows: 1 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t; +k1 k2 v +1 2 3 +insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c; +affected rows: 0 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t; +k1 k2 v +1 2 3 +drop table if exists t1, t2; +create table t1(id int, a int, b int); +insert into t1 values (1, 1, 1); +affected rows: 1 +info: +insert into t1 values (2, 2, 1); +affected rows: 1 +info: +insert into t1 values (3, 3, 1); +affected rows: 1 +info: +create table t2(a int primary key, b int, unique(b)); +insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b; +affected rows: 5 +info: Records: 3 Duplicates: 2 Warnings: 0 +select * from t2 order by a; +a b +3 1 +drop table if exists t1, t2; +create table t1(id int, a int, b int); +insert into t1 values (1, 1, 1); +affected rows: 1 +info: +insert into t1 values (2, 1, 2); +affected rows: 1 +info: +insert into t1 values (3, 3, 1); +affected rows: 1 +info: +create table t2(a int primary key, b int, unique(b)); +insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b; +affected rows: 4 +info: Records: 3 Duplicates: 1 Warnings: 0 +select * from t2 order by a; +a b +1 2 +3 1 +drop table if exists t1, t2; +create table t1(id int, a int, b int, c int); +insert into t1 values (1, 1, 1, 1); +affected rows: 1 +info: +insert into t1 values (2, 2, 1, 2); +affected rows: 1 +info: +insert into t1 values (3, 3, 2, 2); +affected rows: 1 +info: +insert into t1 values (4, 4, 2, 2); +affected rows: 1 +info: +create table t2(a int primary key, b int, c int, unique(b), unique(c)); +insert into t2 select a, b, c from t1 order by id on duplicate key update b=t2.b, c=t2.c; +affected rows: 2 +info: Records: 4 Duplicates: 0 Warnings: 0 +select * from t2 order by a; +a b c +1 1 1 +3 2 2 +drop table if exists t1; +create table t1(a int primary key, b int); +insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5); +affected rows: 5 +info: Records: 5 Duplicates: 0 Warnings: 0 +insert into t1 values(4,14),(5,15),(6,16),(7,17),(8,18) on duplicate key update b=b+10; +affected rows: 7 +info: Records: 5 Duplicates: 2 Warnings: 0 +drop table if exists a, b; +create table a(x int primary key); +create table b(x int, y int); +insert into a values(1); +affected rows: 1 +info: +insert into b values(1, 2); +affected rows: 1 +info: +insert into a select x from b ON DUPLICATE KEY UPDATE a.x=b.y; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from a; +x +2 +## Test issue 28078. +## Use different types of columns so that there's likely to be error if the types mismatches. +drop table if exists a, b; +create table a(id int, a1 timestamp, a2 varchar(10), a3 float, unique(id)); +create table b(id int, b1 time, b2 varchar(10), b3 int); +insert into a values (1, '2022-01-04 07:02:04', 'a', 1.1), (2, '2022-01-04 07:02:05', 'b', 2.2); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +insert into b values (2, '12:34:56', 'c', 10), (3, '01:23:45', 'd', 20); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +insert into a (id) select id from b on duplicate key update a.a2 = b.b2, a.a3 = 3.3; +affected rows: 3 +info: Records: 2 Duplicates: 1 Warnings: 0 +select * from a; +id a1 a2 a3 +1 2022-01-04 07:02:04 a 1.1 +2 2022-01-04 07:02:05 c 3.3 +3 NULL NULL NULL +insert into a (id) select 4 from b where b3 = 20 on duplicate key update a.a3 = b.b3; +affected rows: 1 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from a; +id a1 a2 a3 +1 2022-01-04 07:02:04 a 1.1 +2 2022-01-04 07:02:05 c 3.3 +3 NULL NULL NULL +4 NULL NULL NULL +insert into a (a2, a3) select 'x', 1.2 from b on duplicate key update a.a2 = b.b3; +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +select * from a; +id a1 a2 a3 +1 2022-01-04 07:02:04 a 1.1 +2 2022-01-04 07:02:05 c 3.3 +3 NULL NULL NULL +4 NULL NULL NULL +NULL NULL x 1.2 +NULL NULL x 1.2 +## reproduce insert on duplicate key update bug under new row format. +drop table if exists t1; +create table t1(c1 decimal(6,4), primary key(c1)); +insert into t1 set c1 = 0.1; +insert into t1 set c1 = 0.1 on duplicate key update c1 = 1; +select * from t1 use index(primary); +c1 +1.0000 diff --git a/tests/integrationtest/r/executor/partition/write.result b/tests/integrationtest/r/executor/partition/write.result new file mode 100644 index 0000000000000..06c6839eef19c --- /dev/null +++ b/tests/integrationtest/r/executor/partition/write.result @@ -0,0 +1,814 @@ +# TestWriteListPartitionTable2 +# test for write list partition when the partition expression is complicated and contain generated column. +set @@session.tidb_enable_list_partition = ON; +drop table if exists t; +create table t (id int, name varchar(10),b int generated always as (length(name)+1) virtual) +partition by list (id*2 + b*b + b*b - b*b*2 - abs(id)) ( +partition p0 values in (3,5,6,9,17), +partition p1 values in (1,2,10,11,19,20), +partition p2 values in (4,12,13,14,18), +partition p3 values in (7,8,15,16,null) +); +analyze table t; +## Test add unique index failed. +insert into t (id,name) values (1, 'a'),(1,'b'); +alter table t add unique index idx (id,b); +Error 1062 (23000): Duplicate entry '1-2' for key 't.idx' +## Test add unique index success. +delete from t where name='b'; +alter table t add unique index idx (id,b); +## --------------------------Test insert--------------------------- +## Test insert 1 partition. +delete from t; +insert into t (id,name) values (1, 'a'),(2,'b'),(10,'c'); +select id,name from t partition(p1) order by id; +id name +1 a +2 b +10 c +## Test insert multi-partitions. +delete from t; +insert into t (id,name) values (1, 'a'),(3,'c'),(4,'e'); +select id,name from t partition(p0) order by id; +id name +3 c +select id,name from t partition(p1) order by id; +id name +1 a +select id,name from t partition(p2) order by id; +id name +4 e +select id,name from t partition(p3) order by id; +id name +## Test insert on duplicate. +insert into t (id,name) values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'; +select id,name from t partition(p0) order by id; +id name +3 x +5 g +select id,name from t partition(p1) order by id; +id name +1 x +select id,name from t partition(p2) order by id; +id name +4 e +select id,name from t partition(p3) order by id; +id name +## Test insert on duplicate error +insert into t (id,name) values (3, 'a'), (11,'x') on duplicate key update id=id+1; +Error 1062 (23000): Duplicate entry '4-2' for key 't.idx' +select id,name from t order by id; +id name +1 x +3 x +4 e +5 g +## Test insert ignore with duplicate +insert ignore into t (id,name) values (1, 'b'), (5,'a'),(null,'y'); +show warnings; +Level Code Message +Warning 1062 Duplicate entry '1-2' for key 't.idx' +Warning 1062 Duplicate entry '5-2' for key 't.idx' +select id,name from t partition(p0) order by id; +id name +3 x +5 g +select id,name from t partition(p1) order by id; +id name +1 x +select id,name from t partition(p2) order by id; +id name +4 e +select id,name from t partition(p3) order by id; +id name +NULL y +## Test insert ignore without duplicate +insert ignore into t (id,name) values (15, 'a'),(17,'a'); +select id,name from t partition(p0,p1,p2) order by id; +id name +1 x +3 x +4 e +5 g +17 a +select id,name from t partition(p3) order by id; +id name +NULL y +15 a +## Test insert meet no partition error. +insert into t (id,name) values (100, 'd'); +Error 1526 (HY000): Table has no partition for value 100 +## --------------------------Test update--------------------------- +## Test update 1 partition. +delete from t; +insert into t (id,name) values (1, 'a'),(2,'b'),(3,'c'); +update t set name='b' where id=2;; +select id,name from t partition(p1); +id name +1 a +2 b +update t set name='x' where id in (1,2); +select id,name from t partition(p1); +id name +1 x +2 x +update t set name='y' where id < 3; +select id,name from t order by id; +id name +1 y +2 y +3 c +## Test update meet duplicate error. +update t set id=2 where id = 1; +Error 1062 (23000): Duplicate entry '2-2' for key 't.idx' +select id,name from t order by id; +id name +1 y +2 y +3 c +## Test update multi-partitions +update t set name='z' where id in (1,2,3);; +select id,name from t order by id; +id name +1 z +2 z +3 z +update t set name='a' limit 3; +select id,name from t order by id; +id name +1 a +2 a +3 a +update t set id=id*10 where id in (1,2); +select id,name from t order by id; +id name +3 a +10 a +20 a +## Test update meet duplicate error. +update t set id=id+17 where id in (3,10); +Error 1062 (23000): Duplicate entry '20-2' for key 't.idx' +select id,name from t order by id; +id name +3 a +10 a +20 a +## Test update meet no partition error. +update t set id=id*2 where id in (3,20); +Error 1526 (HY000): Table has no partition for value 40 +select id,name from t order by id; +id name +3 a +10 a +20 a +## --------------------------Test replace--------------------------- +## Test replace 1 partition. +delete from t; +replace into t (id,name) values (1, 'a'),(2,'b'); +select id,name from t order by id; +id name +1 a +2 b +## Test replace multi-partitions. +replace into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +select id,name from t partition(p0) order by id; +id name +3 c +select id,name from t partition(p1) order by id; +id name +1 a +2 b +select id,name from t partition(p2) order by id; +id name +4 d +select id,name from t partition(p3) order by id; +id name +7 f +## Test replace on duplicate. +replace into t (id,name) values (1, 'x'),(7,'x'); +select id,name from t order by id; +id name +1 x +2 b +3 c +4 d +7 x +## Test replace meet no partition error. +replace into t (id,name) values (10,'x'),(50,'x'); +Error 1526 (HY000): Table has no partition for value 50 +select id,name from t order by id; +id name +1 x +2 b +3 c +4 d +7 x +## --------------------------Test delete--------------------------- +## Test delete 1 partition. +delete from t where id = 3; +select id,name from t partition(p0) order by id; +id name +delete from t where id in (1,2); +select id,name from t partition(p1) order by id; +id name +## Test delete multi-partitions. +delete from t where id in (4,7,10,11); +select id,name from t; +id name +insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +delete from t where id < 10; +select id,name from t; +id name +insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +delete from t limit 3; +select id,name from t; +id name +set @@session.tidb_enable_list_partition = default; +# TestWriteListColumnsPartitionTable1 +set @@session.tidb_enable_list_partition = ON; +drop table if exists t; +create table t (id int, name varchar(10)) partition by list columns (id) ( +partition p0 values in (3,5,6,9,17), +partition p1 values in (1,2,10,11,19,20), +partition p2 values in (4,12,13,14,18), +partition p3 values in (7,8,15,16,null) +); +analyze table t; +## Test add unique index failed. +insert into t values (1, 'a'),(1,'b'); +alter table t add unique index idx (id); +Error 1062 (23000): Duplicate entry '1' for key 't.idx' +## Test add unique index success. +delete from t where name='b'; +alter table t add unique index idx (id); +## --------------------------Test insert--------------------------- +## Test insert 1 partition. +delete from t; +insert into t values (1, 'a'),(2,'b'),(10,'c'); +select * from t partition(p1) order by id; +id name +1 a +2 b +10 c +## Test insert multi-partitions. +delete from t; +insert into t values (1, 'a'),(3,'c'),(4,'e'); +select * from t partition(p0) order by id; +id name +3 c +select * from t partition(p1) order by id; +id name +1 a +select * from t partition(p2) order by id; +id name +4 e +select * from t partition(p3) order by id; +id name +## Test insert on duplicate. +insert into t values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'; +select * from t partition(p0) order by id; +id name +3 x +5 g +select * from t partition(p1) order by id; +id name +1 x +select * from t partition(p2) order by id; +id name +4 e +select * from t partition(p3) order by id; +id name +## Test insert on duplicate error +insert into t values (3, 'a'), (11,'x') on duplicate key update id=id+1; +Error 1062 (23000): Duplicate entry '4' for key 't.idx' +select * from t order by id; +id name +1 x +3 x +4 e +5 g +## Test insert ignore with duplicate +insert ignore into t values (1, 'b'), (5,'a'),(null,'y'); +show warnings; +Level Code Message +Warning 1062 Duplicate entry '1' for key 't.idx' +Warning 1062 Duplicate entry '5' for key 't.idx' +select * from t partition(p0) order by id; +id name +3 x +5 g +select * from t partition(p1) order by id; +id name +1 x +select * from t partition(p2) order by id; +id name +4 e +select * from t partition(p3) order by id; +id name +NULL y +## Test insert ignore without duplicate +insert ignore into t values (15, 'a'),(17,'a'); +select * from t partition(p0,p1,p2) order by id; +id name +1 x +3 x +4 e +5 g +17 a +select * from t partition(p3) order by id; +id name +NULL y +15 a +## Test insert meet no partition error. +insert into t values (100, 'd'); +Error 1526 (HY000): Table has no partition for value from column_list +## --------------------------Test update--------------------------- +## Test update 1 partition. +delete from t; +insert into t values (1, 'a'),(2,'b'),(3,'c'); +update t set name='b' where id=2;; +select * from t partition(p1); +id name +1 a +2 b +update t set name='x' where id in (1,2); +select * from t partition(p1); +id name +1 x +2 x +update t set name='y' where id < 3; +select * from t order by id; +id name +1 y +2 y +3 c +## Test update meet duplicate error. +update t set id=2 where id = 1; +Error 1062 (23000): Duplicate entry '2' for key 't.idx' +select * from t order by id; +id name +1 y +2 y +3 c +## Test update multi-partitions +update t set name='z' where id in (1,2,3);; +select * from t order by id; +id name +1 z +2 z +3 z +update t set name='a' limit 3; +select * from t order by id; +id name +1 a +2 a +3 a +update t set id=id*10 where id in (1,2); +select * from t order by id; +id name +3 a +10 a +20 a +## Test update meet duplicate error. +update t set id=id+17 where id in (3,10); +Error 1062 (23000): Duplicate entry '20' for key 't.idx' +select * from t order by id; +id name +3 a +10 a +20 a +## Test update meet no partition error. +update t set id=id*2 where id in (3,20); +Error 1526 (HY000): Table has no partition for value from column_list +select * from t order by id; +id name +3 a +10 a +20 a +## --------------------------Test replace--------------------------- +## Test replace 1 partition. +delete from t; +replace into t values (1, 'a'),(2,'b'); +select * from t order by id; +id name +1 a +2 b +## Test replace multi-partitions. +replace into t values (3, 'c'),(4,'d'),(7,'f'); +select * from t partition(p0) order by id; +id name +3 c +select * from t partition(p1) order by id; +id name +1 a +2 b +select * from t partition(p2) order by id; +id name +4 d +select * from t partition(p3) order by id; +id name +7 f +## Test replace on duplicate. +replace into t values (1, 'x'),(7,'x'); +select * from t order by id; +id name +1 x +2 b +3 c +4 d +7 x +## Test replace meet no partition error. +replace into t values (10,'x'),(100,'x'); +Error 1526 (HY000): Table has no partition for value from column_list +select * from t order by id; +id name +1 x +2 b +3 c +4 d +7 x +## --------------------------Test delete--------------------------- +## Test delete 1 partition. +delete from t where id = 3; +select * from t partition(p0) order by id; +id name +delete from t where id in (1,2); +select * from t partition(p1) order by id; +id name +## Test delete multi-partitions. +delete from t where id in (4,7,10,11); +select * from t; +id name +insert into t values (3, 'c'),(4,'d'),(7,'f'); +delete from t where id < 10; +select * from t; +id name +insert into t values (3, 'c'),(4,'d'),(7,'f'); +delete from t limit 3; +select * from t; +id name +set @@session.tidb_enable_list_partition = default; +set tidb_opt_fix_control = "44262:ON"; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) +partition by range (id) ( +PARTITION p0 VALUES LESS THAN (3), +PARTITION p1 VALUES LESS THAN (5), +PARTITION p2 VALUES LESS THAN (7), +PARTITION p3 VALUES LESS THAN (9)); +replace replace_test (c1) values (1),(2),(NULL); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +begin; +replace replace_test (c1) values (); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +replace replace_test (c1, c2) values (1,2),(1); +Error 1136 (21S01): Column count doesn't match value count at row 2 +rollback; +begin; +replace replace_test (xxx) values (3); +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +begin; +replace replace_test_xxx (c1) values (); +Error 1146 (42S02): Table 'executor__partition__write.replace_test_xxx' doesn't exist +rollback; +replace replace_test set c1 = 3; +affected rows: 1 +info: +begin; +replace replace_test set c1 = 4, c1 = 5; +Error 1110 (42000): Column 'c1' specified twice +rollback; +begin; +replace replace_test set xxx = 6; +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int) partition by range (id) ( +PARTITION p0 VALUES LESS THAN (4), +PARTITION p1 VALUES LESS THAN (6), +PARTITION p2 VALUES LESS THAN (8), +PARTITION p3 VALUES LESS THAN (10), +PARTITION p4 VALUES LESS THAN (100)); +replace replace_test_1 select id, c1 from replace_test; +affected rows: 4 +info: Records: 4 Duplicates: 0 Warnings: 0 +drop table if exists replace_test_2; +create table replace_test_2 (id int, c1 int) partition by range (id) ( +PARTITION p0 VALUES LESS THAN (10), +PARTITION p1 VALUES LESS THAN (50), +PARTITION p2 VALUES LESS THAN (100), +PARTITION p3 VALUES LESS THAN (300)); +replace replace_test_2 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +affected rows: 8 +info: Records: 8 Duplicates: 0 Warnings: 0 +begin; +replace replace_test_2 select c1 from replace_test; +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by range (c2) ( +PARTITION p0 VALUES LESS THAN (4), +PARTITION p1 VALUES LESS THAN (7), +PARTITION p2 VALUES LESS THAN (11)); +replace into replace_test_3 set c2=8; +affected rows: 1 +info: +replace into replace_test_3 set c2=8; +affected rows: 1 +info: +replace into replace_test_3 set c1=8, c2=8; +affected rows: 2 +info: +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by range (c1) ( +PARTITION p0 VALUES LESS THAN (4), +PARTITION p1 VALUES LESS THAN (7), +PARTITION p2 VALUES LESS THAN (11)); +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by range (c2) ( +PARTITION p0 VALUES LESS THAN (4), +PARTITION p1 VALUES LESS THAN (7), +PARTITION p2 VALUES LESS THAN (11)); +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by range (b) ( +PARTITION p1 VALUES LESS THAN (100), +PARTITION p2 VALUES LESS THAN (200)); +insert into tIssue989 (a, b) values (1, 2); +affected rows: 1 +info: +replace into tIssue989(a, b) values (111, 2); +affected rows: 2 +info: +select * from tIssue989; +a b +111 2 +set tidb_opt_fix_control = default; +set tidb_opt_fix_control = "44262:ON"; +drop table if exists t; +create table t (id int not null default 1, name varchar(255)) +PARTITION BY RANGE ( id ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21)); +insert INTO t VALUES (1, "hello"); +insert INTO t VALUES (7, "hello"); +## update non partition column +UPDATE t SET name = "abc" where id > 0; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +SELECT * from t order by id limit 2; +id name +1 abc +7 abc +## update partition column +update t set id = id + 1; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +SELECT * from t order by id limit 2; +id name +2 abc +8 abc +## update partition column, old and new record locates on different partitions +update t set id = 20 where id = 8; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +SELECT * from t order by id limit 2; +id name +2 abc +20 abc +## table option is auto-increment +drop table if exists t; +create table t (id int not null auto_increment, name varchar(255), primary key(id)) +PARTITION BY RANGE ( id ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21)); +insert into t(name) values ('aa'); +update t set id = 8 where name = 'aa'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +insert into t(name) values ('bb'); +select * from t; +id name +8 aa +9 bb +update t set id = null where name = 'aa'; +Error 1048 (23000): Column 'id' cannot be null +## Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. +drop table if exists t; +create table t (id int, name int unique) +PARTITION BY RANGE ( name ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21)); +insert t values (1, 1), (2, 2); +update t set name = 1 where id = 2; +Error 1062 (23000): Duplicate entry '1' for key 't.name' +select * from t; +id name +1 1 +2 2 +## test update ignore for pimary key +drop table if exists t; +create table t(a bigint, primary key (a)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11)); +insert into t values (5); +insert into t values (7); +update ignore t set a = 5 where a = 7; +SHOW WARNINGS; +Level Code Message +Warning 1062 Duplicate entry '5' for key 't.PRIMARY' +select * from t order by a; +a +5 +7 +## test update ignore for truncate as warning +update ignore t set a = 1 where a = (select '2a'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1292 Truncated incorrect DOUBLE value: '2a' +## test update ignore for unique key +drop table if exists t; +create table t(a bigint, unique key I_uniq (a)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11)); +insert into t values (5); +insert into t values (7); +update ignore t set a = 5 where a = 7; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 1 +SHOW WARNINGS; +Level Code Message +Warning 1062 Duplicate entry '5' for key 't.I_uniq' +select * from t order by a; +a +5 +7 +set tidb_opt_fix_control = default; +drop table if exists t; +set tidb_opt_fix_control = "44262:ON"; +CREATE TABLE t (id int not null default 1, name varchar(255), index(id)) +PARTITION BY RANGE ( id ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21)); +insert into t values (1, "hello"),(2, "hello"),(3, "hello"),(4, "hello"),(5, "hello"),(6, "hello"),(7, "hello"),(8, "hello"),(9, "hello"),(10, "hello"),(11, "hello"),(12, "hello"),(13, "hello"),(14, "hello"),(15, "hello"),(16, "hello"),(17, "hello"),(18, "hello"),(19, "hello"),(20, "hello"); +delete from t where id = 2 limit 1; +affected rows: 1 +info: +## Test delete with false condition +delete from t where 0; +affected rows: 0 +info: +insert into t values (2, 'abc'); +delete from t where t.id = 2 limit 1; +affected rows: 1 +info: +## Test delete ignore +insert into t values (2, 'abc'); +delete from t where id = (select '2a'); +Error 1292 (22007): Truncated incorrect DOUBLE value: '2a' +delete ignore from t where id = (select '2a'); +affected rows: 1 +info: +SHOW WARNINGS; +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1292 Truncated incorrect DOUBLE value: '2a' +## Test delete without using index, involve multiple partitions. +delete from t ignore index(id) where id >= 13 and id <= 17; +affected rows: 5 +info: +admin check table t; +delete from t; +affected rows: 14 +info: +## Fix that partitioned table should not use PointGetPlan. +drop table if exists t1; +create table t1 (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range (c1) (partition p0 values less than (3440)); +insert into t1 values (379, 379, 379); +delete from t1 where c1 = 379; +affected rows: 1 +info: +drop table t1; +set tidb_opt_fix_control=default; +set @@session.tidb_enable_table_partition = '1'; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) +partition by hash(id) partitions 4; +replace replace_test (c1) values (1),(2),(NULL); +begin; +replace replace_test (c1) values (); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +replace replace_test (c1, c2) values (1,2),(1); +Error 1136 (21S01): Column count doesn't match value count at row 2 +rollback; +begin; +replace replace_test (xxx) values (3); +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +begin; +replace replace_test_xxx (c1) values (); +Error 1146 (42S02): Table 'executor__partition__write.replace_test_xxx' doesn't exist +rollback; +begin; +replace replace_test set c1 = 4, c1 = 5; +Error 1110 (42000): Column 'c1' specified twice +rollback; +begin; +replace replace_test set xxx = 6; +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +replace replace_test set c1 = 3; +replace replace_test set c1 = 4; +replace replace_test set c1 = 5; +replace replace_test set c1 = 6; +replace replace_test set c1 = 7; +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int) partition by hash(id) partitions 5; +replace replace_test_1 select id, c1 from replace_test; +drop table if exists replace_test_2; +create table replace_test_2 (id int, c1 int) partition by hash(id) partitions 6; +replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +begin; +replace replace_test_1 select c1 from replace_test; +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by hash(c2) partitions 7; +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c2=8; +affected rows: 1 +info: +replace into replace_test_3 set c1=8, c2=8; +affected rows: 2 +info: +replace into replace_test_3 set c2=NULL; +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_3 set c2=0; +replace into replace_test_3 set c2=1; +replace into replace_test_3 set c2=2; +replace into replace_test_3 set c2=3; +replace into replace_test_3 set c2=4; +replace into replace_test_3 set c2=5; +replace into replace_test_3 set c2=6; +replace into replace_test_3 set c2=7; +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c2=9; +select count(*) from replace_test_3; +count(*) +12 +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by hash(c1) partitions 8; +replace into replace_test_4 set c2=NULL; +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by hash (c2) partitions 9; +replace into replace_test_5 set c1=1, c2=2; +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by hash (b) partitions 10; +insert into tIssue989 (a, b) values (1, 2); +replace into tIssue989(a, b) values (111, 2); +select * from tIssue989; +a b +111 2 +set @@session.tidb_enable_table_partition = default; diff --git a/tests/integrationtest/r/executor/stale_txn.result b/tests/integrationtest/r/executor/stale_txn.result index d1e7c85a2829b..5b4498b73834c 100644 --- a/tests/integrationtest/r/executor/stale_txn.result +++ b/tests/integrationtest/r/executor/stale_txn.result @@ -36,3 +36,9 @@ id v set tidb_txn_mode = default; set tx_isolation = default; set autocommit = default; +drop table if exists t1; +create table t1 (id int primary key, v int); +select * from t1 as of timestamp NULL; +Error 8135 (HY000): invalid as of timestamp: as of timestamp cannot be NULL +start transaction read only as of timestamp NULL; +Error 8135 (HY000): invalid as of timestamp: as of timestamp cannot be NULL diff --git a/tests/integrationtest/r/executor/update.result b/tests/integrationtest/r/executor/update.result index 5accbf8f0f70e..5b11c91279039 100644 --- a/tests/integrationtest/r/executor/update.result +++ b/tests/integrationtest/r/executor/update.result @@ -384,3 +384,510 @@ id a 1 0 2 4294967295 set sql_mode=default; +drop table if exists t1; +create table t1(id int primary key, name varchar(40)); +insert into t1 values(1, 'abc'); +begin pessimistic; +begin pessimistic; +update t1 set name='xyz' where id=1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select * from t1 where id = 1; +id name +1 xyz +commit; +update t1 set name='xyz' where id=1; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 0 +select * from t1 where id = 1; +id name +1 abc +select * from t1 where id = 1 for update; +id name +1 xyz +select * from t1 where id in (1, 2); +id name +1 abc +select * from t1 where id in (1, 2) for update; +id name +1 xyz +commit; +drop table if exists update_test; +create table update_test(id int not null default 1, name varchar(255), PRIMARY KEY(id)); +insert INTO update_test VALUES (1, "hello"); +insert into update_test values (2, "hello"); +UPDATE update_test SET name = "abc" where id > 0; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +SELECT * from update_test limit 2; +id name +1 abc +2 abc +commit; +UPDATE update_test SET name = "foo"; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +drop table if exists update_test; +commit; +begin; +create table update_test(id int not null auto_increment, name varchar(255), primary key(id)); +insert into update_test(name) values ('aa'); +update update_test set id = 8 where name = 'aa'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +insert into update_test(name) values ('bb'); +commit; +begin; +select * from update_test; +id name +8 aa +9 bb +commit; +begin; +drop table if exists update_test; +commit; +begin; +create table update_test(id int not null auto_increment, name varchar(255), index(id)); +insert into update_test(name) values ('aa'); +update update_test set id = null where name = 'aa'; +Error 1048 (23000): Column 'id' cannot be null +drop table update_test; +create table update_test(id int); +begin; +insert into update_test(id) values (1); +update update_test set id = 2 where id = 1 limit 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select * from update_test; +id +2 +commit; +drop table if exists update_unique; +create table update_unique (id int primary key, name int unique); +insert update_unique values (1, 1), (2, 2); +begin; +update update_unique set name = 1 where id = 2; +Error 1062 (23000): Duplicate entry '1' for key 'update_unique.name' +commit; +select * from update_unique; +id name +1 1 +2 2 +drop table if exists t; +create table t(a bigint, primary key (a)); +insert into t values (1); +insert into t values (2); +update ignore t set a = 1 where a = 2; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 1 +SHOW WARNINGS; +Level Code Message +Warning 1062 Duplicate entry '1' for key 't.PRIMARY' +select * from t; +a +1 +2 +update ignore t set a = 1 where a = (select '2a'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1062 Duplicate entry '1' for key 't.PRIMARY' +update ignore t set a = 42 where a = 2; +select * from t; +a +1 +42 +drop table if exists t; +create table t(a bigint, unique key I_uniq (a)); +insert into t values (1); +insert into t values (2); +update ignore t set a = 1 where a = 2; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 1 +SHOW WARNINGS; +Level Code Message +Warning 1062 Duplicate entry '1' for key 't.I_uniq' +select * from t; +a +1 +2 +drop table if exists t; +set @@session.tidb_enable_list_partition = ON; +create table t (a int) partition by list (a) (partition p0 values in (0,1)); +analyze table t; +insert ignore into t values (1); +update ignore t set a=2 where a=1; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 0 +drop table if exists t; +create table t (a int key) partition by list (a) (partition p0 values in (0,1)); +insert ignore into t values (1); +update ignore t set a=2 where a=1; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 0 +set @@session.tidb_enable_list_partition = default; +drop table if exists t; +create table t(id integer auto_increment, t1 datetime, t2 datetime, primary key (id)); +insert into t(t1, t2) values('2000-10-01 01:01:01', '2017-01-01 10:10:10'); +select * from t; +id t1 t2 +1 2000-10-01 01:01:01 2017-01-01 10:10:10 +update t set t1 = '2017-10-01 10:10:11', t2 = date_add(t1, INTERVAL 10 MINUTE) where id = 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select * from t; +id t1 t2 +1 2017-10-01 10:10:11 2000-10-01 01:11:01 +drop table if exists tt1; +CREATE TABLE `tt1` (`a` int(11) NOT NULL,`b` varchar(32) DEFAULT NULL,`c` varchar(32) DEFAULT NULL,PRIMARY KEY (`a`),UNIQUE KEY `b_idx` (`b`)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +insert into tt1 values(1, 'a', 'a'); +insert into tt1 values(2, 'd', 'b'); +select * from tt1; +a b c +1 a a +2 d b +update tt1 set a=5 where c='b'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select * from tt1; +a b c +1 a a +5 d b +drop table if exists tsup; +CREATE TABLE `tsup` (`a` int,`ts` TIMESTAMP DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,KEY `idx` (`ts`)); +set @@sql_mode=''; +insert into tsup values(1, '0000-00-00 00:00:00'); +update tsup set a=5; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select t1.ts = t2.ts from (select ts from tsup use index (idx)) as t1, (select ts from tsup use index ()) as t2; +t1.ts = t2.ts +1 +update tsup set ts='2019-01-01'; +select ts from tsup; +ts +2019-01-01 00:00:00 +set @@sql_mode=default; +drop table if exists decimals; +create table decimals (a decimal(20, 0) not null); +insert into decimals values (201); +update decimals set a = a + 1.23; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 1 +show warnings; +Level Code Message +Warning 1292 Truncated incorrect DECIMAL value: '202.23' +select * from decimals; +a +202 +drop table t; +CREATE TABLE `t` ( `c1` year DEFAULT NULL, `c2` year DEFAULT NULL, `c3` date DEFAULT NULL, `c4` datetime DEFAULT NULL, KEY `idx` (`c1`,`c2`)); +UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2; +update (select * from t) t set c1 = 1111111; +Error 1288 (HY000): The target table t of the UPDATE is not updatable +drop table if exists t; +create table t (i int not null default 10); +insert into t values (1); +update ignore t set i = null; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 1 +SHOW WARNINGS; +Level Code Message +Warning 1048 Column 'i' cannot be null +select * from t; +i +0 +drop table t; +create table t (k int, v int); +update t, (select * from t) as b set b.k = t.k; +Error 1288 (HY000): The target table b of the UPDATE is not updatable +update t, (select * from t) as b set t.k = b.k; +drop table if exists t1; +CREATE TABLE t1 (c1 float); +INSERT INTO t1 SET c1 = 1; +UPDATE t1 SET c1 = 1.2 WHERE c1=1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +drop table if exists t; +create table t (c1 float(1,1)); +insert into t values (0.0); +update t set c1 = 2.0; +Error 1264 (22003): Out of range value for column 'c1' at row 1 +drop table if exists t; +create table t(a datetime not null, b datetime); +insert into t value('1999-12-12', '1999-12-13'); +set @@sql_mode=''; +select * from t; +a b +1999-12-12 00:00:00 1999-12-13 00:00:00 +update t set a = ''; +select * from t; +a b +0000-00-00 00:00:00 1999-12-13 00:00:00 +update t set b = ''; +select * from t; +a b +0000-00-00 00:00:00 0000-00-00 00:00:00 +set @@sql_mode=default; +drop view if exists v; +create view v as select * from t; +update v set a = '2000-11-11'; +Error 1288 (HY000): The target table v of the UPDATE is not updatable +drop view v; +drop sequence if exists seq; +create sequence seq; +update seq set minvalue=1; +Error 1054 (42S22): Unknown column 'minvalue' in 'field list' +drop sequence seq; +drop table if exists t1, t2; +create table t1(a int, b int, c int, d int, e int, index idx(a)); +create table t2(a int, b int, c int); +update t1 join t2 on t1.a=t2.a set t1.a=1 where t2.b=1 and t2.c=2; +drop table if exists t1, t2; +create table t1 (a int default 1, b int default 2); +insert into t1 values (10, 10), (20, 20); +update t1 set a=default where b=10; +select * from t1; +a b +1 10 +20 20 +update t1 set a=30, b=default where a=20; +select * from t1; +a b +1 10 +30 2 +update t1 set a=default, b=default where a=30; +select * from t1; +a b +1 10 +1 2 +insert into t1 values (40, 40); +update t1 set a=default, b=default; +select * from t1; +a b +1 2 +1 2 +1 2 +update t1 set a=default(b), b=default(a); +select * from t1; +a b +2 1 +2 1 +2 1 +create table t2 (a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored); +insert into t2 values (10, default, default), (20, default, default); +update t2 set b=default; +select * from t2; +a b c +10 -10 -10 +20 -20 -20 +update t2 set a=30, b=default where a=10; +select * from t2; +a b c +30 -30 -30 +20 -20 -20 +update t2 set c=default, a=40 where c=-20; +select * from t2; +a b c +30 -30 -30 +40 -40 -40 +update t2 set a=default, b=default, c=default where b=-30; +select * from t2; +a b c +1 -1 -1 +40 -40 -40 +update t2 set a=default(a), b=default, c=default; +select * from t2; +a b c +1 -1 -1 +1 -1 -1 +update t2 set a=default(b), b=default, c=default; +select * from t2; +a b c +NULL NULL NULL +NULL NULL NULL +update t2 set b=default(a); +Error 3105 (HY000): The value specified for generated column 'b' in table 't2' is not allowed. +update t2 set a=default(a), c=default(c); +select * from t2; +a b c +1 -1 -1 +1 -1 -1 +update t2 set a=default(b), b=default(b); +select * from t2; +a b c +NULL NULL NULL +NULL NULL NULL +update t2 set a=default(a), c=default(c); +select * from t2; +a b c +1 -1 -1 +1 -1 -1 +update t2 set a=default(a), c=default(a); +Error 3105 (HY000): The value specified for generated column 'c' in table 't2' is not allowed. +drop table t1, t2; +drop table if exists msg, detail; +create table msg (id varchar(8), b int, status int, primary key (id, b)); +insert msg values ('abc', 1, 1); +create table detail (id varchar(8), start varchar(8), status int, index idx_start(start)); +insert detail values ('abc', '123', 2); +UPDATE msg SET msg.status = (SELECT detail.status FROM detail WHERE msg.id = detail.id); +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +admin check table msg; +drop table if exists ttt; +CREATE TABLE ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host)); +insert into ttt values (8,8),(9,9); +begin; +update ttt set id = 0, host='9' where id = 9 limit 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +delete from ttt where id = 0 limit 1; +select * from ttt use index (i_host) order by host; +id host +8 8 +update ttt set id = 0, host='8' where id = 8 limit 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +delete from ttt where id = 0 limit 1; +select * from ttt use index (i_host) order by host; +id host +commit; +admin check table ttt; +drop table ttt; +drop table if exists a; +create table a(id int auto_increment, a int default null, primary key(id)); +insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1); +update a set id = id*10 where a = 1001; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +drop table a; +create table a ( a bigint, b bigint); +insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1); +update a set a = a*10 where b = 1001; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +drop table if exists items, month; +CREATE TABLE items (id int, price TEXT); +insert into items values (11, "items_price_11"), (12, "items_price_12"), (13, "items_price_13"); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +CREATE TABLE month (mid int, mprice TEXT); +insert into month values (11, "month_price_11"), (22, "month_price_22"), (13, "month_price_13"); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +UPDATE items, month SET items.price=month.mprice WHERE items.id=month.mid; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +SELECT * FROM items; +id price +11 month_price_11 +12 items_price_12 +13 month_price_13 +commit; +UPDATE items join month on items.id=month.mid SET items.price=month.mid; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +SELECT * FROM items; +id price +11 11 +12 items_price_12 +13 13 +commit; +UPDATE items T0 join month T1 on T0.id=T1.mid SET T0.price=T1.mprice; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +SELECT * FROM items; +id price +11 month_price_11 +12 items_price_12 +13 month_price_13 +commit; +DROP TABLE IF EXISTS t1, t2; +create table t1 (c int); +create table t2 (c varchar(256)); +insert into t1 values (1), (2); +insert into t2 values ("a"), ("b"); +update t1, t2 set t1.c = 10, t2.c = "abc"; +affected rows: 4 +info: Rows matched: 4 Changed: 4 Warnings: 0 +DROP TABLE IF EXISTS t1, t2; +create table t1 (c1 int); +create table t2 (c2 int); +insert into t1 values (1), (2); +insert into t2 values (1), (2); +update t1, t2 set t1.c1 = 10, t2.c2 = 2 where t2.c2 = 1; +affected rows: 3 +info: Rows matched: 3 Changed: 3 Warnings: 0 +select * from t1; +c1 +10 +10 +drop table if exists t; +create table t (a int, b int); +insert into t values(1, 1), (2, 2), (3, 3); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +update t m, t n set m.a = m.a + 1; +affected rows: 3 +info: Rows matched: 3 Changed: 3 Warnings: 0 +select * from t; +a b +2 1 +3 2 +4 3 +update t m, t n set n.a = n.a - 1, n.b = n.b + 1; +affected rows: 3 +info: Rows matched: 3 Changed: 3 Warnings: 0 +select * from t; +a b +1 2 +2 3 +3 4 +drop table if exists update_modified; +create table update_modified (col_1 int, col_2 enum('a', 'b')); +set SQL_MODE=''; +insert into update_modified values (0, 3); +SELECT * FROM update_modified; +col_1 col_2 +0 +set SQL_MODE=STRICT_ALL_TABLES; +update update_modified set col_1 = 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +SELECT * FROM update_modified; +col_1 col_2 +1 +update update_modified set col_1 = 2, col_2 = 'c'; +Error 1265 (01000): Data truncated for column '%s' at row %d +SELECT * FROM update_modified; +col_1 col_2 +1 +update update_modified set col_1 = 3, col_2 = 'a'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +SELECT * FROM update_modified; +col_1 col_2 +3 a +drop table if exists update_with_diff_type; +CREATE TABLE update_with_diff_type (a int, b JSON); +INSERT INTO update_with_diff_type VALUES(3, '{"a": "测试"}'); +UPDATE update_with_diff_type SET a = '300'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +SELECT a FROM update_with_diff_type; +a +300 +UPDATE update_with_diff_type SET b = '{"a": "\\u6d4b\\u8bd5"}'; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 0 +SELECT b FROM update_with_diff_type; +b +{"a": "测试"} +set SQL_MODE=default; diff --git a/tests/integrationtest/r/executor/write.result b/tests/integrationtest/r/executor/write.result index 1217d60a5b38c..70bfb62329b75 100644 --- a/tests/integrationtest/r/executor/write.result +++ b/tests/integrationtest/r/executor/write.result @@ -1974,3 +1974,175 @@ b 4 set @@session.tidb_enable_list_partition = default; set @@allow_auto_random_explicit_insert = default; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1); +replace replace_test (c1) values (1),(2),(NULL); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +begin; +replace replace_test (c1) values (); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +replace replace_test (c1, c2) values (1,2),(1); +Error 1136 (21S01): Column count doesn't match value count at row 2 +rollback; +begin; +replace replace_test (xxx) values (3); +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +begin; +replace replace_test_xxx (c1) values (); +Error 1146 (42S02): Table 'executor__write.replace_test_xxx' doesn't exist +rollback; +replace replace_test set c1 = 3; +affected rows: 1 +info: +begin; +replace replace_test set c1 = 4, c1 = 5; +Error 1110 (42000): Column 'c1' specified twice +rollback; +begin; +replace replace_test set xxx = 6; +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int); +replace replace_test_1 select id, c1 from replace_test; +affected rows: 4 +info: Records: 4 Duplicates: 0 Warnings: 0 +begin; +replace replace_test_0 select c1 from replace_test; +Error 1146 (42S02): Table 'executor__write.replace_test_0' doesn't exist +rollback; +create table replace_test_2 (id int, c1 int); +replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +affected rows: 8 +info: Records: 8 Duplicates: 0 Warnings: 0 +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)); +replace into replace_test_3 set c2=1; +affected rows: 1 +info: +replace into replace_test_3 set c2=1; +affected rows: 1 +info: +replace into replace_test_3 set c1=1, c2=1; +affected rows: 2 +info: +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)); +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)); +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b)); +insert into tIssue989 (a, b) values (1, 2); +affected rows: 1 +info: +replace into tIssue989(a, b) values (111, 2); +affected rows: 2 +info: +select * from tIssue989; +a b +111 2 +drop table if exists tIssue1012; +CREATE TABLE tIssue1012 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b)); +insert into tIssue1012 (a, b) values (1, 2); +insert into tIssue1012 (a, b) values (2, 1); +replace into tIssue1012(a, b) values (1, 1); +affected rows: 3 +info: +select * from tIssue1012; +a b +1 1 +drop table if exists t1; +create table t1(a int primary key, b int); +insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5); +replace into t1 values(1,1); +affected rows: 1 +info: +replace into t1 values(1,1),(2,2); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +replace into t1 values(4,14),(5,15),(6,16),(7,17),(8,18); +affected rows: 7 +info: Records: 5 Duplicates: 2 Warnings: 0 +replace into t1 select * from (select 1, 2) as tmp; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +drop table if exists t1, t2; +create table t1 (a int primary key, b int default 20, c int default 30); +insert into t1 value (1, 2, 3); +replace t1 set a=1, b=default; +select * from t1; +a b c +1 20 30 +replace t1 set a=2, b=default, c=default; +select * from t1; +a b c +1 20 30 +2 20 30 +replace t1 set a=2, b=default(c), c=default(b); +select * from t1; +a b c +1 20 30 +2 30 20 +replace t1 set a=default(b)+default(c); +select * from t1; +a b c +1 20 30 +2 30 20 +50 20 30 +create table t2 (pk int primary key, a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored); +replace t2 set pk=1, b=default; +select * from t2; +pk a b c +1 1 -1 -1 +replace t2 set pk=2, a=10, b=default; +select * from t2; +pk a b c +1 1 -1 -1 +2 10 -10 -10 +replace t2 set pk=2, c=default, a=20; +select * from t2; +pk a b c +1 1 -1 -1 +2 20 -20 -20 +replace t2 set pk=2, a=default, b=default, c=default; +select * from t2; +pk a b c +1 1 -1 -1 +2 1 -1 -1 +replace t2 set pk=3, a=default(a), b=default, c=default; +select * from t2; +pk a b c +1 1 -1 -1 +2 1 -1 -1 +3 1 -1 -1 +replace t2 set b=default(a); +Error 3105 (HY000): The value specified for generated column 'b' in table 't2' is not allowed. +replace t2 set a=default(b), b=default(b); +Error 3105 (HY000): The value specified for generated column 'a' in table 't2' is not allowed. +replace t2 set a=default(a), c=default(c); +Error 1364 (HY000): Field 'pk' doesn't have a default value +replace t2 set c=default(a); +Error 3105 (HY000): The value specified for generated column 'c' in table 't2' is not allowed. +drop table t1, t2; diff --git a/tests/integrationtest/r/planner/core/binary_plan.result b/tests/integrationtest/r/planner/core/binary_plan.result new file mode 100644 index 0000000000000..776df6e6ec382 --- /dev/null +++ b/tests/integrationtest/r/planner/core/binary_plan.result @@ -0,0 +1,18 @@ +select tidb_decode_binary_plan('some random bytes'); +tidb_decode_binary_plan('some random bytes') + +show warnings; +Level Code Message +Warning 1105 illegal base64 data at input byte 4 +select tidb_decode_binary_plan('c29tZSByYW5kb20gYnl0ZXM='); +tidb_decode_binary_plan('c29tZSByYW5kb20gYnl0ZXM=') + +show warnings; +Level Code Message +Warning 1105 snappy: corrupt input +select tidb_decode_binary_plan('EUBzb21lIHJhbmRvbSBieXRlcw=='); +tidb_decode_binary_plan('EUBzb21lIHJhbmRvbSBieXRlcw==') + +show warnings; +Level Code Message +Warning 1105 proto: illegal wireType 7 diff --git a/tests/integrationtest/r/planner/core/cbo.result b/tests/integrationtest/r/planner/core/cbo.result index f9d2ff5a8a726..230e53fe949ca 100644 --- a/tests/integrationtest/r/planner/core/cbo.result +++ b/tests/integrationtest/r/planner/core/cbo.result @@ -69,14 +69,14 @@ insert into t2 values (2, 22), (3, 33), (5, 55), (233, 2), (333, 3), (3434, 5); analyze table t1, t2; explain analyze select t1.a, t1.b, sum(t1.c) from t1 join t2 on t1.a = t2.b where t1.a > 1; id estRows actRows task access object execution info operator info memory disk -Projection_9 1.00 1 root NULL time:, loops:, RU:, Concurrency:OFF planner__core__cbo.t1.a, planner__core__cbo.t1.b, Column#8 KB N/A -└─StreamAgg_11 1.00 1 root NULL time:, loops: funcs:sum(Column#16)->Column#8, funcs:firstrow(Column#17)->planner__core__cbo.t1.a, funcs:firstrow(Column#18)->planner__core__cbo.t1.b KB N/A - └─Projection_53 4.00 3 root NULL time:, loops:, Concurrency:OFF cast(planner__core__cbo.t1.c, decimal(10,0) BINARY)->Column#16, planner__core__cbo.t1.a->Column#17, planner__core__cbo.t1.b->Column#18 KB N/A - └─HashJoin_51 4.00 3 root NULL time:, loops:, build_hash_table:{total:, fetch:, build:}, probe:{concurrency:, total:, max:, probe:, fetch:} inner join, equal:[eq(planner__core__cbo.t1.a, planner__core__cbo.t2.b)] KB Bytes - ├─TableReader_30(Build) 6.00 6 root NULL time:, loops:, cop_task: {num:, max:, proc_keys:, rpc_num:, rpc_time:, copr_cache_hit_ratio:, build_task_duration:, max_distsql_concurrency:} data:Selection_29 Bytes N/A +Projection_9 1.00 1 root NULL time:, loops:, RU:, Concurrency:OFF planner__core__cbo.t1.a, planner__core__cbo.t1.b, Column#8 N/A +└─StreamAgg_11 1.00 1 root NULL time:, loops: funcs:sum(Column#16)->Column#8, funcs:firstrow(Column#17)->planner__core__cbo.t1.a, funcs:firstrow(Column#18)->planner__core__cbo.t1.b N/A + └─Projection_53 4.00 3 root NULL time:, loops:, Concurrency:OFF cast(planner__core__cbo.t1.c, decimal(10,0) BINARY)->Column#16, planner__core__cbo.t1.a->Column#17, planner__core__cbo.t1.b->Column#18 N/A + └─HashJoin_51 4.00 3 root NULL time:, loops:, build_hash_table:{total:, fetch:, build:}, probe:{concurrency:, total:, max:, probe:, fetch:} inner join, equal:[eq(planner__core__cbo.t1.a, planner__core__cbo.t2.b)] + ├─TableReader_30(Build) 6.00 6 root NULL time:, loops:, cop_task: {num:, max:, proc_keys:, rpc_num:, rpc_time:, copr_cache_hit_ratio:, build_task_duration:, max_distsql_concurrency:} data:Selection_29 N/A │ └─Selection_29 6.00 6 cop[tikv] NULL tikv_task:{time:, loops:} gt(planner__core__cbo.t2.b, 1), not(isnull(planner__core__cbo.t2.b)) N/A N/A │ └─TableFullScan_28 6.00 6 cop[tikv] table:t2 tikv_task:{time:, loops:} keep order:false N/A N/A - └─TableReader_33(Probe) 4.00 4 root NULL time:, loops:, cop_task: {num:, max:, proc_keys:, rpc_num:, rpc_time:, copr_cache_hit_ratio:, build_task_duration:, max_distsql_concurrency:} data:Selection_32 Bytes N/A + └─TableReader_33(Probe) 4.00 4 root NULL time:, loops:, cop_task: {num:, max:, proc_keys:, rpc_num:, rpc_time:, copr_cache_hit_ratio:, build_task_duration:, max_distsql_concurrency:} data:Selection_32 N/A └─Selection_32 4.00 4 cop[tikv] NULL tikv_task:{time:, loops:} gt(planner__core__cbo.t1.a, 1), not(isnull(planner__core__cbo.t1.a)) N/A N/A └─TableFullScan_31 5.00 5 cop[tikv] table:t1 tikv_task:{time:, loops:} keep order:false N/A N/A set sql_mode=default; diff --git a/tests/integrationtest/r/planner/core/plan.result b/tests/integrationtest/r/planner/core/plan.result index 7a49cd0486ea4..76bee6627810d 100644 --- a/tests/integrationtest/r/planner/core/plan.result +++ b/tests/integrationtest/r/planner/core/plan.result @@ -392,3 +392,35 @@ explain select a from t; id estRows task access object operator info IndexReader_7 10000.00 root index:IndexFullScan_6 └─IndexFullScan_6 10000.00 cop[tikv] table:t, index:hypo_a(a) keep order:false, stats:pseudo +drop table if exists t; +CREATE TABLE t (c1 mediumint(9) DEFAULT '-4747160',c2 year(4) NOT NULL DEFAULT '2075',c3 double DEFAULT '1.1559030660251948',c4 enum('wbv4','eli','d8ym','m3gsx','lz7td','o','d1k7l','y1x','xcxq','bj','n7') DEFAULT 'xcxq',c5 int(11) DEFAULT '255080866',c6 tinyint(1) DEFAULT '1',PRIMARY KEY (c2),KEY `c4d86d54-091c-4307-957b-b164c9652b7f` (c6,c4) ); +insert into t values (-4747160, 2075, 722.5719203870632, 'xcxq', 1576824797, 1); +select /*+ stream_agg() */ bit_or(t.c5) as r0 from t where t.c3 in (select c6 from t where not(t.c6 <> 1) and not(t.c3 in(9263.749352636818))) group by t.c1; +r0 +affected rows: 0 +info: +drop table if exists t1; +drop table if exists t2; +CREATE TABLE `t1`(`c1` bigint(20) NOT NULL DEFAULT '-2312745469307452950', `c2` datetime DEFAULT '5316-02-03 06:54:49', `c3` tinyblob DEFAULT NULL, PRIMARY KEY (`c1`) /*T![clustered_index] CLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +CREATE TABLE `t2`(`c1` set('kn8pu','7et','vekx6','v3','liwrh','q14','1met','nnd5i','5o0','8cz','l') DEFAULT '7et,vekx6,liwrh,q14,1met', `c2` float DEFAULT '1.683167', KEY `k1` (`c2`,`c1`), KEY `k2` (`c2`)) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci; +(select /*+ agg_to_cop()*/ locate(t1.c3, t1.c3) as r0, t1.c3 as r1 from t1 where not( IsNull(t1.c1)) order by r0,r1) union all (select concat_ws(',', t2.c2, t2.c1) as r0, t2.c1 as r1 from t2 order by r0, r1) order by 1 limit 273; +r0 r1 +affected rows: 0 +info: +drop table if exists golang1, golang2; +CREATE TABLE golang1 ( `fcbpdt` CHAR (8) COLLATE utf8_general_ci NOT NULL, `fcbpsq` VARCHAR (20) COLLATE utf8_general_ci NOT NULL, `procst` char (4) COLLATE utf8_general_ci DEFAULT NULL,`cipstx` VARCHAR (105) COLLATE utf8_general_ci DEFAULT NULL, `cipsst` CHAR (4) COLLATE utf8_general_ci DEFAULT NULL, `dyngtg` VARCHAR(4) COLLATE utf8_general_ci DEFAULT NULL, `blncdt` VARCHAR (8) COLLATE utf8_general_ci DEFAULT NULL, PRIMARY KEY ( fcbpdt, fcbpsq )); +insert into golang1 values('20230925','12023092502158016','abc','','','',''); +create table golang2 (`sysgrp` varchar(20) NOT NULL,`procst` varchar(8) NOT NULL,`levlid` int(11) NOT NULL,PRIMARY key (procst));; +insert into golang2 VALUES('COMMON','ACSC',90); +insert into golang2 VALUES('COMMON','abc',8); +insert into golang2 VALUES('COMMON','CH02',6); +UPDATE golang1 a SET procst =(CASE WHEN ( SELECT levlid FROM golang2 b WHERE b.sysgrp = 'COMMON' AND b.procst = 'ACSC' ) > ( SELECT levlid FROM golang2 c WHERE c.sysgrp = 'COMMON' AND c.procst = a.procst ) THEN 'ACSC' ELSE a.procst END ), cipstx = 'CI010000', cipsst = 'ACSC', dyngtg = 'EAYT', blncdt= '20230925' WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'; +select * from golang1; +fcbpdt fcbpsq procst cipstx cipsst dyngtg blncdt +20230925 12023092502158016 ACSC CI010000 ACSC EAYT 20230925 +UPDATE golang1 a SET procst= (SELECT 1 FROM golang2 c WHERE c.procst = a.procst) WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'; +select * from golang1; +fcbpdt fcbpsq procst cipstx cipsst dyngtg blncdt +20230925 12023092502158016 1 CI010000 ACSC EAYT 20230925 +EXPLAIN FORMAT = TRADITIONAL ((VALUES ROW ()) ORDER BY 1); +Error 1051 (42S02): Unknown table '' diff --git a/tests/integrationtest/r/planner/core/plan_cache.result b/tests/integrationtest/r/planner/core/plan_cache.result index 69db6972f45d8..cfcc58922905a 100644 --- a/tests/integrationtest/r/planner/core/plan_cache.result +++ b/tests/integrationtest/r/planner/core/plan_cache.result @@ -2249,3 +2249,350 @@ select @@last_plan_from_cache; @@last_plan_from_cache 0 set tidb_enable_non_prepared_plan_cache=DEFAULT; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +100 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +100 +set @@tidb_prepared_plan_cache_size = 200; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +200 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +200 +set @@tidb_session_plan_cache_size = 300; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +300 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +300 +set global tidb_prepared_plan_cache_size = 400; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +400 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +400 +set global tidb_session_plan_cache_size = 500; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +500 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +500 +set global tidb_prepared_plan_cache_size = default; +set global tidb_session_plan_cache_size = default; +set @@tidb_session_plan_cache_size = default; +set @@tidb_prepared_plan_cache_size = default; +drop table if exists t; +create table t(a int); +set tidb_enable_non_prepared_plan_cache=1; +select * from t where a=1; +a +explain select * from t where a=2; +id estRows task access object operator info +TableReader_7 10.00 root data:Selection_6 +└─Selection_6 10.00 cop[tikv] eq(planner__core__plan_cache.t.a, 2) + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +explain format=verbose select * from t where a=2; +id estRows estCost task access object operator info +TableReader_7 10.00 168975.57 root data:Selection_6 +└─Selection_6 10.00 2534000.00 cop[tikv] eq(planner__core__plan_cache.t.a, 2) + └─TableFullScan_5 10000.00 2035000.00 cop[tikv] table:t keep order:false, stats:pseudo +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +explain analyze select * from t where a=2; +id estRows actRows task access object execution info operator info memory disk +TableReader_7 10.00 0 root +└─Selection_6 10.00 0 cop[tikv] + └─TableFullScan_5 10000.00 0 cop[tikv] +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +set tidb_enable_non_prepared_plan_cache=default; +drop table if exists t1, t2, t3, t4; +set @@tidb_opt_fix_control = "45798:on"; +create table t1 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city')))); +create table t2 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) virtual); +create table t3 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) stored); +create table t4 (a int, info json, index zips( (CAST(info->'$.zipcode' AS UNSIGNED ARRAY)))); +set @a=1; +set @b=2; +prepare s1 from 'select * from t1 where a=?'; +show warnings; +Level Code Message +execute s1 using @a; +a info city +execute s1 using @b; +a info city +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +prepare s1 from 'select * from t2 where a=?'; +show warnings; +Level Code Message +execute s1 using @a; +a info city +execute s1 using @b; +a info city +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +prepare s1 from 'select * from t3 where a=?'; +show warnings; +Level Code Message +execute s1 using @a; +a info city +execute s1 using @b; +a info city +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +prepare s1 from 'select * from t4 where a=?'; +show warnings; +Level Code Message +execute s1 using @a; +a info +execute s1 using @b; +a info +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +set @@tidb_opt_fix_control = default; +drop table if exists t1, t2; +set @@tidb_opt_fix_control = "45798:on"; +CREATE TABLE t1 ( +ipk varbinary(255) NOT NULL, +i_id varchar(45) DEFAULT NULL, +i_set_id varchar(45) DEFAULT NULL, +p_id varchar(45) DEFAULT NULL, +p_set_id varchar(45) DEFAULT NULL, +m_id bigint(20) DEFAULT NULL, +m_i_id varchar(127) DEFAULT NULL, +m_i_set_id varchar(127) DEFAULT NULL, +d json DEFAULT NULL, +p_sources json DEFAULT NULL, +nslc json DEFAULT NULL, +cl json DEFAULT NULL, +fii json DEFAULT NULL, +fpi json DEFAULT NULL, +PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */, +UNIQUE KEY i_id (i_id), +KEY d ((cast(d as char(253) array))), +KEY m_i_id (m_i_id), +KEY m_i_set_id (m_i_set_id), +KEY fpi ((cast(fpi as unsigned array))), +KEY nslc ((cast(nslc as char(1000) array))), +KEY cl ((cast(cl as char(3000) array))), +KEY fii ((cast(fii as unsigned array))), +KEY m_id (m_id), +KEY i_set_id (i_set_id), +KEY m_i_and_m_id (m_i_id,m_id)); +CREATE TABLE t2 ( +ipk varbinary(255) NOT NULL, +created_time bigint(20) DEFAULT NULL, +arrival_time bigint(20) DEFAULT NULL, +updated_time bigint(20) DEFAULT NULL, +timestamp_data json DEFAULT NULL, +PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */); +prepare stmt from 'select * +from ( t1 left outer join t2 on ( t1 . ipk = t2 . ipk ) ) +where ( t1 . i_id = ? )'; +show warnings; +Level Code Message +set @a='a', @b='b'; +execute stmt using @a; +ipk i_id i_set_id p_id p_set_id m_id m_i_id m_i_set_id d p_sources nslc cl fii fpi ipk created_time arrival_time updated_time timestamp_data +execute stmt using @b; +ipk i_id i_set_id p_id p_set_id m_id m_i_id m_i_set_id d p_sources nslc cl fii fpi ipk created_time arrival_time updated_time timestamp_data +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +set @@tidb_opt_fix_control = default; +drop table if exists t; +create table t (a int); +prepare st from 'select * from t where mod(a, 2)=1'; +execute st; +a +execute st; +a +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +insert into mysql.expr_pushdown_blacklist(name) values('mod'); +admin reload expr_pushdown_blacklist; +execute st; +a +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +execute st; +a +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +delete from mysql.expr_pushdown_blacklist; +admin reload expr_pushdown_blacklist; +drop table if exists t, tt; +create table t(a int, index(a)); +create table tt(a varchar(10)); +set tidb_enable_non_prepared_plan_cache=1; +select a+1 from t where a<10; +a+1 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select a+1 from t where a<20; +a+1 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select a+2 from t where a<30; +a+2 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select a+2 from t where a<40; +a+2 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select a,a+1 from t where a<30; +a a+1 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select a,a+1 from t where a<40; +a a+1 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select a+'123' from tt where a='1'; +a+'123' +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select a+'123' from tt where a='2'; +a+'123' +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select 1 from t where a<10; +1 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select 1 from t where a<20; +1 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select 2 from t where a<10; +2 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select 2 from t where a<20; +2 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select 1,2 from t where a<10; +1 2 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select 1,2 from t where a<20; +1 2 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +set tidb_enable_non_prepared_plan_cache=default; +drop table if exists t; +create table t(a int); +insert into t values(1); +prepare s from "select * from t where tidb_parse_tso(a) > unix_timestamp()"; +execute s; +a +1 +drop table if exists t; +set tidb_enable_non_prepared_plan_cache=1; +create table t (a int, b varchar(32), c datetime, key(a)); +select * from t where mod(a, 5) < 2; +a b c +select * from t where mod(a, 5) < 2; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select * from t where c < now(); +a b c +select * from t where c < now(); +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select date_format(c, '%Y-%m-%d') from t where a < 10; +date_format(c, '%Y-%m-%d') +select date_format(c, '%Y-%m-%d') from t where a < 10; +date_format(c, '%Y-%m-%d') +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select str_to_date(b, '%Y-%m-%d') from t where a < 10; +str_to_date(b, '%Y-%m-%d') +select str_to_date(b, '%Y-%m-%d') from t where a < 10; +str_to_date(b, '%Y-%m-%d') +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select * from t where a-2 < 20; +a b c +select * from t where a-2 < 20; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select * from t where a+b > 100; +a b c +select * from t where a+b > 100; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select * from t where -a > 10; +a b c +select * from t where -a > 10; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select * from t where a < 1 and b like '%abc%'; +a b c +select * from t where a < 1 and b like '%abc%'; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select database() from t; +database() +select database() from t; +database() +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +set tidb_enable_non_prepared_plan_cache=default; +drop table if exists t; +CREATE TABLE t (a int(11) DEFAULT NULL, b date DEFAULT NULL); +INSERT INTO t VALUES (1, current_date()); +PREPARE stmt FROM 'SELECT a FROM t WHERE b=current_date()'; +EXECUTE stmt; +a +1 diff --git a/tests/integrationtest/run-tests.sh b/tests/integrationtest/run-tests.sh index 8aaf08847f048..e0142e77d0692 100755 --- a/tests/integrationtest/run-tests.sh +++ b/tests/integrationtest/run-tests.sh @@ -88,7 +88,7 @@ function build_mysql_tester() { echo "building mysql-tester binary: $mysql_tester" rm -rf $mysql_tester - GOBIN=$PWD go install github.com/pingcap/mysql-tester/src@fc3c1e6c6233a10e1a77b168d182f2906ad0b4ee + GOBIN=$PWD go install github.com/pingcap/mysql-tester/src@77628a8d2fae0c2f4cbc059d45785ae9615c817a mv src mysql_tester } diff --git a/tests/integrationtest/t/executor/cte.test b/tests/integrationtest/t/executor/cte.test index 25bdab72cdee4..1176a509c1456 100644 --- a/tests/integrationtest/t/executor/cte.test +++ b/tests/integrationtest/t/executor/cte.test @@ -1,3 +1,6 @@ +## Delete it when https://github.com/pingcap/tidb/issues/48808 merged +set tidb_max_chunk_size=default; + # TestBasicCTE with recursive cte1 as (select 1 c1 union all select c1 + 1 c1 from cte1 where c1 < 5) select * from cte1; with recursive cte1 as (select 1 c1 union all select 2 c1 union all select c1 + 1 c1 from cte1 where c1 < 10) select * from cte1 order by c1; diff --git a/tests/integrationtest/t/executor/delete.test b/tests/integrationtest/t/executor/delete.test index ed853f680a438..45b5f3ec4df8e 100644 --- a/tests/integrationtest/t/executor/delete.test +++ b/tests/integrationtest/t/executor/delete.test @@ -13,3 +13,88 @@ set @a=1; execute stmt using @a; select * from t; +# TestDelete +drop table if exists delete_test; +drop view if exists v; +drop sequence if exists seq; +create table delete_test(id int not null default 1, name varchar(255), PRIMARY KEY(id)); +insert INTO delete_test VALUES (1, "hello"); +insert into delete_test values (2, "hello"); +--enable_info +update delete_test set name = "abc" where id = 2; +delete from delete_test where id = 2 limit 1; +delete from delete_test where 0; +--disable_info +insert into delete_test values (2, 'abc'); +--enable_info +delete from delete_test where delete_test.id = 2 limit 1; +--disable_info +begin; +SELECT * from delete_test limit 2; +commit; +insert into delete_test values (2, 'abc'); +## TODO: https://github.com/pingcap/tidb/issues/48120 +--replace_regex /INTEGER/DOUBLE/ +-- error 1292 +delete from delete_test where id = (select '2a'); +--enable_info +delete ignore from delete_test where id = (select '2a'); +--disable_info +SHOW WARNINGS; +--enable_info +delete from delete_test; +--disable_info +create view v as select * from delete_test; +-- error 1356 +delete from v where name = 'aaa'; +drop view v; +create sequence seq; +-- error 1105 +delete from seq; +drop sequence seq; + +# TestQualifiedDelete +drop table if exists t1, t2; +create table t1 (c1 int, c2 int, index (c1)); +create table t2 (c1 int, c2 int); +insert into t1 values (1, 1), (2, 2); +--enable_info +delete from t1 where t1.c1 = 1; +delete from t1 where t1.c2 = 2; +--disable_info +select * from t1; +insert into t1 values (1, 3); +--enable_info +delete from t1 as a where a.c1 = 1; +--disable_info +insert into t1 values (1, 1), (2, 2); +insert into t2 values (2, 1), (3,1); +--enable_info +delete t1, t2 from t1 join t2 where t1.c1 = t2.c2; +--disable_info +insert into t2 values (2, 1), (3,1); +--enable_info +delete a, b from t1 as a join t2 as b where a.c2 = b.c1; +--disable_info +-- error 1109 +delete t1, t2 from t1 as a join t2 as b where a.c2 = b.c1; + +# TestMultiTableDelete +drop table if exists t1, t2, t3; +create table t1 (id int, data int); +--enable_info +insert into t1 values (11, 121), (12, 122), (13, 123); +--disable_info +create table t2 (id int, data int); +--enable_info +insert into t2 values (11, 221), (22, 222), (23, 223); +--disable_info +create table t3 (id int, data int); +--enable_info +insert into t3 values (11, 321), (22, 322), (23, 323); +delete t1, t2 from t1 inner join t2 inner join t3 where t1.id=t2.id and t2.id=t3.id; +--disable_info +--sorted_result +select * from t3; + + diff --git a/tests/integrationtest/t/executor/executor.test b/tests/integrationtest/t/executor/executor.test index dde049d75536f..bf5ac6da7dda2 100644 --- a/tests/integrationtest/t/executor/executor.test +++ b/tests/integrationtest/t/executor/executor.test @@ -2671,3 +2671,180 @@ set @@tidb_enable_collect_execution_info=0; select /*+ use_index_merge(t1, primary, t1a) */ * from t1 where id < 2 or a > 4 order by a; set @@tidb_enable_collect_execution_info=default; +# TestIndexLookupRuntimeStats +drop table if exists t1; +create table t1 (a int, b int, index(a)); +insert into t1 values (1,2),(2,3),(3,4); +--replace_regex /.*time:.*loops:.*index_task:.*table_task: {total_time.*num.*concurrency.*}.*/.*time:.*loops:.*index_task:.*table_task: {total_time.*num.*concurrency.*}.*/ /.*time:.*loops:.*cop_task:.*/.*time:.*loops:.*cop_task:.*/ /[.0-9]+ KB/ KB/ /[0-9]+ Bytes/ Bytes/ +explain analyze select * from t1 use index(a) where a > 1; + +# TestHashAggRuntimeStats +drop table if exists t1; +create table t1 (a int, b int); +insert into t1 values (1,2),(2,3),(3,4); +--replace_regex /.*time:.*loops:.*partial_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*final_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*/.*time:.*loops:.*partial_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*final_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*/ /time:.*loops:.*cop_task.*/time.*loops.*cop_task.*/ /tikv_task:.*/tikv_task:.*/ /[.0-9]+ KB/ KB/ /[.0-9]+ Bytes/ Bytes/ +explain analyze SELECT /*+ HASH_AGG() */ count(*) FROM t1 WHERE a < 10; + +# TestSelectForUpdate +set global tidb_txn_mode=''; +drop table if exists t, t1; +create table t (c1 int, c2 int, c3 int); +insert t values (11, 2, 3); +insert t values (12, 2, 3); +insert t values (13, 2, 3); +create table t1 (c1 int); +insert t1 values (11); + +connect (conn1, localhost, root,, executor__executor); +begin; +select * from t where c1=11 for update; + +connect (conn2, localhost, root,, executor__executor); +begin; +update t set c2=211 where c1=11; +commit; + +connection conn1; +--replace_regex /txnStartTS.*reason/ reason/ +--error 9007 +commit; + +begin; +select * from t where exists(select null from t1 where t1.c1=t.c1) for update; + +connection conn2; +begin; +update t set c2=211 where c1=12; +commit; + +connection conn1; +commit; + +begin; +select * from t where c1=11 for update; + +connection conn2; +begin; +update t set c2=22 where c1=12; +commit; + +connection conn1; +commit; + +set @@autocommit=1; +select * from t where c1=11 for update; + +connection conn2; +begin; +update t set c2=211 where c1=11; +commit; + +connection conn1; +commit; + +begin; +--sorted_result +select * from (select * from t for update) t join t1 for update; + +connection conn2; +begin; +update t1 set c1 = 13; +commit; + +connection conn1; +--replace_regex /txnStartTS.*reason/ reason/ +--error 9007 +commit; + +disconnect conn1; +disconnect conn2; +set global tidb_txn_mode=pessimistic; + +# TestSelectForUpdateOf +drop table if exists t, t1; +create table t (i int); +create table t1 (i int); +insert t values (1); +insert t1 values (1); +begin pessimistic; +select * from t, t1 where t.i = t1.i for update of t; + +connect (conn1, localhost, root,, executor__executor); +begin pessimistic; +select * from t1 for update; +--error 3572 +select * from t for update nowait; + +connection default; +rollback; + +connection conn1; +select * from t for update nowait; +rollback; +disconnect conn1; + +# TestForSelectScopeInUnion +set session tidb_txn_mode=''; +# A union B for update, the "for update" option belongs to union statement, so +# it should works on both A and B. +drop table if exists t; +create table t(a int); +insert into t values (1); +begin; +select 1 as a union select a from t for update; + +connect (conn1, localhost, root,, executor__executor); +set session tidb_txn_mode=''; +update t set a = a + 1; + +connection default; +## As tk1 use select 'for update', it should detect conflict and fail. +--replace_regex /txnStartTS.*reason/ reason/ +--error 9007 +commit; + +begin; +--sorted_result +select 1 as a union select a from t limit 5 for update; +select 1 as a union select a from t order by a for update; + +connection conn1; +update t set a = a + 1; + +connection default; +--replace_regex /txnStartTS.*reason/ reason/ +--error 9007 +commit; + +disconnect conn1; +set session tidb_txn_mode=pessimistic; + +# TestAdminShowDDLJobsRowCount +# https://github.com/pingcap/tidb/issues/25968 +drop table if exists t; +create table t (id bigint key,b int); +split table t by (10),(20),(30); +insert into t values (0,0),(10,10),(20,20),(30,30); +alter table t add index idx1(b); +--replace_column 1 4 6 7 9 10 11 +admin show ddl jobs 1; + +insert into t values (1,0),(2,10),(3,20),(4,30); +alter table t add index idx2(b); +--replace_column 1 4 6 7 9 10 11 +admin show ddl jobs 1; + +# TestSummaryFailedUpdate +drop table if exists t; +create table t(a int, b int as(-a)); +insert into t(a) values(1), (3), (7); +SET GLOBAL tidb_mem_oom_action='CANCEL'; +set @@tidb_mem_quota_query=1; +--replace_regex /conn=[0-9]+/conn=/ +--error 8175 +update t set t.a = t.a - 1 where t.a in (select a from t where a < 4); +set @@tidb_mem_quota_query=1000000000; +select stmt_type from information_schema.statements_summary where digest_text = 'update `t` set `t` . `a` = `t` . `a` - ? where `t` . `a` in ( select `a` from `t` where `a` < ? )'; + +set @@tidb_mem_quota_query=default; +set global tidb_mem_oom_action=default; diff --git a/tests/integrationtest/t/executor/insert.test b/tests/integrationtest/t/executor/insert.test index 15bae116746ab..8d241c2a5d7e6 100644 --- a/tests/integrationtest/t/executor/insert.test +++ b/tests/integrationtest/t/executor/insert.test @@ -967,3 +967,651 @@ commit; disconnect conn1; set global tidb_disable_txn_auto_retry=default; + +# TestIssue38950 +drop table if exists t; +create table t (id smallint auto_increment primary key); +alter table t add column c1 int default 1; +--enable_info +insert ignore into t(id) values (194626268); +--disable_info +select * from t; +--enable_info +insert ignore into t(id) values ('*') on duplicate key update c1 = 2; +--disable_info +select * from t; + +# TestInsertIgnoreOnDup +drop table if exists t; +create table t (i int not null primary key, j int unique key); +--enable_info +insert into t values (1, 1), (2, 2); +insert ignore into t values(1, 1) on duplicate key update i = 2; +--disable_info +select * from t; +--enable_info +insert ignore into t values(1, 1) on duplicate key update j = 2; +--disable_info +select * from t; + +drop table if exists t2; +create table t2(`col_25` set('Alice','Bob','Charlie','David') NOT NULL,`col_26` date NOT NULL DEFAULT '2016-04-15', PRIMARY KEY (`col_26`) clustered, UNIQUE KEY `idx_9` (`col_25`,`col_26`),UNIQUE KEY `idx_10` (`col_25`)); +insert into t2(col_25, col_26) values('Bob', '1989-03-23'),('Alice', '2023-11-24'), ('Charlie', '2023-12-05'); +insert ignore into t2 (col_25,col_26) values ( 'Bob','1977-11-23' ) on duplicate key update col_25 = 'Alice', col_26 = '2036-12-13'; +show warnings; +select * from t2; + +drop table if exists t4; +create table t4(id int primary key clustered, k int, v int, unique key uk1(k)); +insert into t4 values (1, 10, 100), (3, 30, 300); +insert ignore into t4 (id, k, v) values(1, 0, 0) on duplicate key update id = 2, k = 30; +show warnings; +select * from t4; + +drop table if exists t5; +create table t5(k1 varchar(100), k2 varchar(100), uk1 int, v int, primary key(k1, k2) clustered, unique key ukk1(uk1), unique key ukk2(v)); +insert into t5(k1, k2, uk1, v) values('1', '1', 1, '100'), ('1', '3', 2, '200'); +update ignore t5 set k2 = '2', uk1 = 2 where k1 = '1' and k2 = '1'; +show warnings; +select * from t5; + +drop table if exists t6; +create table t6 (a int, b int, c int, primary key(a, b) clustered, unique key idx_14(b), unique key idx_15(b), unique key idx_16(a, b)); +insert into t6 select 10, 10, 20; +insert ignore into t6 set a = 20, b = 10 on duplicate key update a = 100; +select * from t6; +insert ignore into t6 set a = 200, b= 10 on duplicate key update c = 1000; +select * from t6; + +# TestInsertAutoInc +drop table if exists insert_autoinc_test; +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(c1) values (1), (2); +begin; +select * from insert_autoinc_test; +commit; +begin; +insert into insert_autoinc_test(id, c1) values (5,5); +insert into insert_autoinc_test(c1) values (6); +commit; +begin; +select * from insert_autoinc_test; +commit; +begin; +insert into insert_autoinc_test(id, c1) values (3,3); +commit; +begin; +select * from insert_autoinc_test; +commit; +begin; +insert into insert_autoinc_test(c1) values (7); +commit; +begin; +select * from insert_autoinc_test; +commit; +drop table if exists insert_autoinc_test; + +## issue-962 +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(id, c1) values (0.3, 1); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (-0.3, 2); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (-3.3, 3); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (4.3, 4); +select * from insert_autoinc_test; +insert into insert_autoinc_test(c1) values (5); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (null, 6); +select * from insert_autoinc_test; +drop table if exists insert_autoinc_test; + +## SQL_MODE=NO_AUTO_VALUE_ON_ZERO +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(id, c1) values (5, 1); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (0, 2); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (0, 3); +select * from insert_autoinc_test; +set SQL_MODE=NO_AUTO_VALUE_ON_ZERO; +insert into insert_autoinc_test(id, c1) values (0, 4); +select * from insert_autoinc_test; +-- error 1062 +insert into insert_autoinc_test(id, c1) values (0, 5); +insert into insert_autoinc_test(c1) values (6); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (null, 7); +select * from insert_autoinc_test; +set SQL_MODE=''; +insert into insert_autoinc_test(id, c1) values (0, 8); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (null, 9); +select * from insert_autoinc_test; +set sql_mode = default; + +# TestInsert +drop table if exists insert_test; +create table insert_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1); +--enable_info +insert insert_test (c1) values (1),(2),(NULL); +--disable_info +begin; +-- error 1136 +insert insert_test (c1) values (); +rollback; +begin; +-- error 1136 +insert insert_test (c1, c2) values (1,2),(1); +rollback; +begin; +-- error 1054 +insert insert_test (xxx) values (3); +rollback; +begin; +-- error 1146 +insert insert_test_xxx (c1) values (); +rollback; +--enable_info +insert insert_test set c1 = 3; +--disable_info +begin; +-- error 1110 +insert insert_test set c1 = 4, c1 = 5; +rollback; +begin; +-- error 1054 +insert insert_test set xxx = 6; +rollback; + +drop table if exists insert_test_1, insert_test_2; +create table insert_test_1 (id int, c1 int); +--enable_info +insert insert_test_1 select id, c1 from insert_test; +--disable_info +create table insert_test_2 (id int, c1 int); +--enable_info +insert insert_test_1 select id, c1 from insert_test union select id * 10, c1 * 10 from insert_test; +--disable_info +begin; +-- error 1136 +insert insert_test_1 select c1 from insert_test; +rollback; +begin; +-- error 1136 +insert insert_test_1 values(default, default, default, default, default); +rollback; +select * from insert_test where id = 1; +--enable_info +insert into insert_test (id, c3) values (1, 2) on duplicate key update id=values(id), c2=10; +--disable_info +select * from insert_test where id = 1; +--enable_info +insert into insert_test (id, c2) values (1, 1) on duplicate key update insert_test.c2=10; +--disable_info +-- error 1054 +insert into insert_test (id, c2) values(1, 1) on duplicate key update t.c2 = 10; +--enable_info +INSERT INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3; +--disable_info +select * from insert_test where id = 1; +--enable_info +INSERT IGNORE INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3; +--disable_info +select * from insert_test where id = 1; + +drop table if exists insert_err; +create table insert_err (id int, c1 varchar(8)); +-- error 1406 +insert insert_err values (1, 'abcdabcdabcd'); +insert insert_err values (1, '你好,世界'); +create table TEST1 (ID INT NOT NULL, VALUE INT DEFAULT NULL, PRIMARY KEY (ID)); +--enable_info +INSERT INTO TEST1(id,value) VALUE(3,3) on DUPLICATE KEY UPDATE VALUE=4; +--disable_info + +drop table if exists t; +create table t (id int); +insert into t values(1); +update t t1 set id = (select count(*) + 1 from t t2 where t1.id = t2.id); +select * from t; + +## issue 3235 +drop table if exists t; +create table t(c decimal(5, 5)); +insert into t value(0); +-- error 1264 +insert into t value(1); + +drop table if exists t; +create table t(c binary(255)); +insert into t value(1); +select length(c) from t; + +drop table if exists t; +create table t(c varbinary(255)); +insert into t value(1); +select length(c) from t; + +## issue 3509 +drop table if exists t; +create table t(c int); +set @@time_zone = '+08:00'; +insert into t value(Unix_timestamp('2002-10-27 01:00')); +select * from t; +set @@time_zone = default; + +## issue 3832 +drop table if exists t1; +create table t1 (b char(0)); +insert into t1 values (""); + +## issue 3895 +DROP TABLE IF EXISTS t; +CREATE TABLE t(a DECIMAL(4,2)); +INSERT INTO t VALUES (1.000001); +SHOW WARNINGS; +INSERT INTO t VALUES (1.000000); +SHOW WARNINGS; + +## issue 4653 +DROP TABLE IF EXISTS t; +CREATE TABLE t(a datetime); +-- error 1292 +INSERT INTO t VALUES('2017-00-00'); +set sql_mode = ''; +INSERT INTO t VALUES('2017-00-00'); +SELECT * FROM t; +set sql_mode = 'strict_all_tables'; +SELECT * FROM t; +set sql_mode = default; + +drop table if exists test; +CREATE TABLE test(id int(10) UNSIGNED NOT NULL AUTO_INCREMENT, p int(10) UNSIGNED NOT NULL, PRIMARY KEY(p), KEY(id)); +insert into test(p) value(1); +select * from test; +select * from test use index (id) where id = 1; +insert into test values(NULL, 2); +select * from test use index (id) where id = 2; +insert into test values(2, 3); +select * from test use index (id) where id = 2; + +## issue 6360 +drop table if exists t; +create table t(a bigint unsigned); +set @@sql_mode = 'strict_all_tables'; +-- error 1264 +insert into t value (-1); +set @@sql_mode = ''; +insert into t value (-1); +show warnings; +insert into t select -1; +show warnings; +insert into t select cast(-1 as unsigned); +insert into t value (-1.111); +show warnings; +insert into t value ('-1.111'); +show warnings; +update t set a = -1 limit 1; +show warnings; +select * from t; +set @@sql_mode = default; + +# issue 6424 & issue 20207 +drop table if exists t; +create table t(a time(6)); +insert into t value('20070219173709.055870'), ('20070219173709.055'), ('20070219173709.055870123'); +select * from t; +truncate table t; +insert into t value(20070219173709.055870), (20070219173709.055), (20070219173709.055870123); +select * from t; +-- error 1292 +insert into t value(-20070219173709.055870); + +drop table if exists t; +set @@sql_mode=''; +create table t(a float unsigned, b double unsigned); +insert into t value(-1.1, -1.1), (-2.1, -2.1), (0, 0), (1.1, 1.1); +show warnings; +select * from t; +set @@sql_mode=default; + +## issue 7061 +drop table if exists t; +create table t(a int default 1, b int default 2); +insert into t values(default, default); +select * from t; +truncate table t; +insert into t values(default(b), default(a)); +select * from t; +truncate table t; +insert into t (b) values(default); +select * from t; +truncate table t; +insert into t (b) values(default(a)); +select * from t; + +drop view if exists v; +create view v as select * from t; +-- error 1105 +insert into v values(1,2); +-- error 1105 +replace into v values(1,2); +drop view v; + +drop sequence if exists seq; +create sequence seq; +-- error 1105 +insert into seq values(); +-- error 1105 +replace into seq values(); +drop sequence seq; + +## issue 22851 +drop table if exists t; +create table t(name varchar(255), b int, c int, primary key(name(2))); +insert into t(name, b) values("cha", 3); +-- error 1062 +insert into t(name, b) values("chb", 3); +insert into t(name, b) values("测试", 3); +-- error 1062 +insert into t(name, b) values("测试", 3); + +# TestInsertOnDup +drop table if exists t; +create table t (i int unique key); +--enable_info +insert into t values (1),(2); +--disable_info +select * from t; +--enable_info +insert into t values (1), (2) on duplicate key update i = values(i); +--disable_info +select * from t; +--enable_info +insert into t values (2), (3) on duplicate key update i = 3; +--disable_info +select * from t; + +drop table if exists t; +create table t (i int primary key, j int unique key); +--enable_info +insert into t values (-1, 1); +--disable_info +select * from t; +--enable_info +insert into t values (1, 1) on duplicate key update j = values(j); +--disable_info +select * from t; + +drop table if exists test; +create table test (i int primary key, j int unique); +begin; +insert into test values (1,1); +insert into test values (2,1) on duplicate key update i = -i, j = -j; +commit; +select * from test; +delete from test; +insert into test values (1, 1); +begin; +delete from test where i = 1; +insert into test values (2, 1) on duplicate key update i = -i, j = -j; +commit; +select * from test; +delete from test; +insert into test values (1, 1); +begin; +update test set i = 2, j = 2 where i = 1; +insert into test values (1, 3) on duplicate key update i = -i, j = -j; +insert into test values (2, 4) on duplicate key update i = -i, j = -j; +commit; +select * from test order by i; +delete from test; +begin; +insert into test values (1, 3), (1, 3) on duplicate key update i = values(i), j = values(j); +commit; +select * from test order by i; +create table tmp (id int auto_increment, code int, primary key(id, code)); +create table m (id int primary key auto_increment, code int unique); +insert tmp (code) values (1); +insert tmp (code) values (1); +set tidb_init_chunk_size=1; +insert m (code) select code from tmp on duplicate key update code = values(code); +select * from m; + +## The following two cases are used for guaranteeing the last_insert_id +## to be set as the value of on-duplicate-update assigned. +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT AUTO_INCREMENT PRIMARY KEY, +f2 VARCHAR(5) NOT NULL UNIQUE); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +--disable_info +SELECT LAST_INSERT_ID(); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +--disable_info +SELECT LAST_INSERT_ID(); + +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT AUTO_INCREMENT UNIQUE, +f2 VARCHAR(5) NOT NULL UNIQUE); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +--disable_info +SELECT LAST_INSERT_ID(); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +--disable_info +SELECT LAST_INSERT_ID(); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = 2; +--disable_info +SELECT LAST_INSERT_ID(); + +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT); +--enable_info +INSERT t1 VALUES (1) ON DUPLICATE KEY UPDATE f1 = 1; +--disable_info +SELECT * FROM t1; + +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT NOT NULL UNIQUE); +--enable_info +INSERT t1 VALUES (1, 1); +INSERT t1 VALUES (1, 1), (1, 1) ON DUPLICATE KEY UPDATE f1 = 2, f2 = 2; +--disable_info +SELECT * FROM t1 order by f1; +-- error 1048 +INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +--enable_info +INSERT IGNORE t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +--disable_info +show warnings; +SELECT * FROM t1 order by f1; + +SET sql_mode=''; +INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +show warnings; +SELECT * FROM t1 order by f1; +set sql_mode=default; + +set tidb_init_chunk_size=default; + + +# TestInsertOnDuplicateKey +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +--enable_info +insert into t1 values(1, 100); +insert into t2 values(1, 200); +insert into t1 select a2, b2 from t2 on duplicate key update b1 = a2; +--disable_info +select * from t1; +--enable_info +insert into t1 select a2, b2 from t2 on duplicate key update b1 = b2; +--disable_info +select * from t1; +--enable_info +insert into t1 select a2, b2 from t2 on duplicate key update a1 = a2; +--disable_info +select * from t1; +--enable_info +insert into t1 select a2, b2 from t2 on duplicate key update b1 = 300; +--disable_info +select * from t1; +--enable_info +insert into t1 values(1, 1) on duplicate key update b1 = 400; +--disable_info +select * from t1; +--enable_info +insert into t1 select 1, 500 from t2 on duplicate key update b1 = 400; +--disable_info +select * from t1; + +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +-- error 1054 +insert into t1 select * from t2 on duplicate key update c = t2.b; + +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +-- error 1052 +insert into t1 select * from t2 on duplicate key update a = b; + +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +-- error 1054 +insert into t1 select * from t2 on duplicate key update c = b; + +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +-- error 1054 +insert into t1 select * from t2 on duplicate key update a1 = values(b2); + +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +--enable_info +insert into t1 values(1, 100); +insert into t2 values(1, 200); +insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2; +--disable_info +select * from t1; +--enable_info +insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2; +--disable_info +select * from t1; + +drop table if exists t; +create table t(k1 bigint, k2 bigint, val bigint, primary key(k1, k2)); +--enable_info +insert into t (val, k1, k2) values (3, 1, 2); +--disable_info +select * from t; +--enable_info +insert into t (val, k1, k2) select c, a, b from (select 1 as a, 2 as b, 4 as c) tmp on duplicate key update val = tmp.c; +--disable_info +select * from t; + +drop table if exists t; +create table t(k1 double, k2 double, v double, primary key(k1, k2)); +--enable_info +insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c; +--disable_info +select * from t; +--enable_info +insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c; +--disable_info +select * from t; + +drop table if exists t1, t2; +create table t1(id int, a int, b int); +--enable_info +insert into t1 values (1, 1, 1); +insert into t1 values (2, 2, 1); +insert into t1 values (3, 3, 1); +--disable_info +create table t2(a int primary key, b int, unique(b)); +--enable_info +insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b; +--disable_info +select * from t2 order by a; + +drop table if exists t1, t2; +create table t1(id int, a int, b int); +--enable_info +insert into t1 values (1, 1, 1); +insert into t1 values (2, 1, 2); +insert into t1 values (3, 3, 1); +--disable_info +create table t2(a int primary key, b int, unique(b)); +--enable_info +insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b; +--disable_info +select * from t2 order by a; + +drop table if exists t1, t2; +create table t1(id int, a int, b int, c int); +--enable_info +insert into t1 values (1, 1, 1, 1); +insert into t1 values (2, 2, 1, 2); +insert into t1 values (3, 3, 2, 2); +insert into t1 values (4, 4, 2, 2); +--disable_info +create table t2(a int primary key, b int, c int, unique(b), unique(c)); +--enable_info +insert into t2 select a, b, c from t1 order by id on duplicate key update b=t2.b, c=t2.c; +--disable_info +select * from t2 order by a; + +drop table if exists t1; +create table t1(a int primary key, b int); +--enable_info +insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5); +insert into t1 values(4,14),(5,15),(6,16),(7,17),(8,18) on duplicate key update b=b+10; +--disable_info + +drop table if exists a, b; +create table a(x int primary key); +create table b(x int, y int); +--enable_info +insert into a values(1); +insert into b values(1, 2); +insert into a select x from b ON DUPLICATE KEY UPDATE a.x=b.y; +--disable_info +select * from a; + +--echo ## Test issue 28078. +--echo ## Use different types of columns so that there's likely to be error if the types mismatches. +drop table if exists a, b; +create table a(id int, a1 timestamp, a2 varchar(10), a3 float, unique(id)); +create table b(id int, b1 time, b2 varchar(10), b3 int); +--enable_info +insert into a values (1, '2022-01-04 07:02:04', 'a', 1.1), (2, '2022-01-04 07:02:05', 'b', 2.2); +insert into b values (2, '12:34:56', 'c', 10), (3, '01:23:45', 'd', 20); +insert into a (id) select id from b on duplicate key update a.a2 = b.b2, a.a3 = 3.3; +--disable_info +select * from a; +--enable_info +insert into a (id) select 4 from b where b3 = 20 on duplicate key update a.a3 = b.b3; +--disable_info +select * from a; +--enable_info +insert into a (a2, a3) select 'x', 1.2 from b on duplicate key update a.a2 = b.b3; +--disable_info +select * from a; + +--echo ## reproduce insert on duplicate key update bug under new row format. +drop table if exists t1; +create table t1(c1 decimal(6,4), primary key(c1)); +insert into t1 set c1 = 0.1; +insert into t1 set c1 = 0.1 on duplicate key update c1 = 1; +select * from t1 use index(primary); diff --git a/tests/integrationtest/t/executor/partition/write.test b/tests/integrationtest/t/executor/partition/write.test new file mode 100644 index 0000000000000..6eb3724786d05 --- /dev/null +++ b/tests/integrationtest/t/executor/partition/write.test @@ -0,0 +1,604 @@ +--echo # TestWriteListPartitionTable2 +--echo # test for write list partition when the partition expression is complicated and contain generated column. +set @@session.tidb_enable_list_partition = ON; +drop table if exists t; +create table t (id int, name varchar(10),b int generated always as (length(name)+1) virtual) + partition by list (id*2 + b*b + b*b - b*b*2 - abs(id)) ( + partition p0 values in (3,5,6,9,17), + partition p1 values in (1,2,10,11,19,20), + partition p2 values in (4,12,13,14,18), + partition p3 values in (7,8,15,16,null) +); + +analyze table t; + +--echo ## Test add unique index failed. +insert into t (id,name) values (1, 'a'),(1,'b'); +--error 1062 +alter table t add unique index idx (id,b); +--echo ## Test add unique index success. +delete from t where name='b'; +alter table t add unique index idx (id,b); + +--echo ## --------------------------Test insert--------------------------- +--echo ## Test insert 1 partition. +delete from t; +insert into t (id,name) values (1, 'a'),(2,'b'),(10,'c'); +select id,name from t partition(p1) order by id; +--echo ## Test insert multi-partitions. +delete from t; +insert into t (id,name) values (1, 'a'),(3,'c'),(4,'e'); +select id,name from t partition(p0) order by id; +select id,name from t partition(p1) order by id; +select id,name from t partition(p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test insert on duplicate. +insert into t (id,name) values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'; +select id,name from t partition(p0) order by id; +select id,name from t partition(p1) order by id; +select id,name from t partition(p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test insert on duplicate error +--error 1062 +insert into t (id,name) values (3, 'a'), (11,'x') on duplicate key update id=id+1; +select id,name from t order by id; +--echo ## Test insert ignore with duplicate +insert ignore into t (id,name) values (1, 'b'), (5,'a'),(null,'y'); +show warnings; +select id,name from t partition(p0) order by id; +select id,name from t partition(p1) order by id; +select id,name from t partition(p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test insert ignore without duplicate +insert ignore into t (id,name) values (15, 'a'),(17,'a'); +select id,name from t partition(p0,p1,p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test insert meet no partition error. +--error 1526 +insert into t (id,name) values (100, 'd'); + +--echo ## --------------------------Test update--------------------------- +--echo ## Test update 1 partition. +delete from t; +insert into t (id,name) values (1, 'a'),(2,'b'),(3,'c'); +update t set name='b' where id=2;; +select id,name from t partition(p1); +update t set name='x' where id in (1,2); +select id,name from t partition(p1); +update t set name='y' where id < 3; +select id,name from t order by id; +--echo ## Test update meet duplicate error. +--error 1062 +update t set id=2 where id = 1; +select id,name from t order by id; + +--echo ## Test update multi-partitions +update t set name='z' where id in (1,2,3);; +select id,name from t order by id; +update t set name='a' limit 3; +select id,name from t order by id; +update t set id=id*10 where id in (1,2); +select id,name from t order by id; +--echo ## Test update meet duplicate error. +--error 1062 +update t set id=id+17 where id in (3,10); +select id,name from t order by id; +--echo ## Test update meet no partition error. +--error 1526 +update t set id=id*2 where id in (3,20); +select id,name from t order by id; + +--echo ## --------------------------Test replace--------------------------- +--echo ## Test replace 1 partition. +delete from t; +replace into t (id,name) values (1, 'a'),(2,'b'); +select id,name from t order by id; +--echo ## Test replace multi-partitions. +replace into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +select id,name from t partition(p0) order by id; +select id,name from t partition(p1) order by id; +select id,name from t partition(p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test replace on duplicate. +replace into t (id,name) values (1, 'x'),(7,'x'); +select id,name from t order by id; +--echo ## Test replace meet no partition error. +--error 1526 +replace into t (id,name) values (10,'x'),(50,'x'); +select id,name from t order by id; + +--echo ## --------------------------Test delete--------------------------- +--echo ## Test delete 1 partition. +delete from t where id = 3; +select id,name from t partition(p0) order by id; +delete from t where id in (1,2); +select id,name from t partition(p1) order by id; +--echo ## Test delete multi-partitions. +delete from t where id in (4,7,10,11); +select id,name from t; +insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +delete from t where id < 10; +select id,name from t; +insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +delete from t limit 3; +select id,name from t; +set @@session.tidb_enable_list_partition = default; + +--echo # TestWriteListColumnsPartitionTable1 +set @@session.tidb_enable_list_partition = ON; + +drop table if exists t; +create table t (id int, name varchar(10)) partition by list columns (id) ( + partition p0 values in (3,5,6,9,17), + partition p1 values in (1,2,10,11,19,20), + partition p2 values in (4,12,13,14,18), + partition p3 values in (7,8,15,16,null) +); + +analyze table t; + +--echo ## Test add unique index failed. +insert into t values (1, 'a'),(1,'b'); +--error 1062 +alter table t add unique index idx (id); +--echo ## Test add unique index success. +delete from t where name='b'; +alter table t add unique index idx (id); + +--echo ## --------------------------Test insert--------------------------- +--echo ## Test insert 1 partition. +delete from t; +insert into t values (1, 'a'),(2,'b'),(10,'c'); +select * from t partition(p1) order by id; +--echo ## Test insert multi-partitions. +delete from t; +insert into t values (1, 'a'),(3,'c'),(4,'e'); +select * from t partition(p0) order by id; +select * from t partition(p1) order by id; +select * from t partition(p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test insert on duplicate. +insert into t values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'; +select * from t partition(p0) order by id; +select * from t partition(p1) order by id; +select * from t partition(p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test insert on duplicate error +--error 1062 +insert into t values (3, 'a'), (11,'x') on duplicate key update id=id+1; +select * from t order by id; +--echo ## Test insert ignore with duplicate +insert ignore into t values (1, 'b'), (5,'a'),(null,'y'); +show warnings; +select * from t partition(p0) order by id; +select * from t partition(p1) order by id; +select * from t partition(p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test insert ignore without duplicate +insert ignore into t values (15, 'a'),(17,'a'); +select * from t partition(p0,p1,p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test insert meet no partition error. +--error 1526 +insert into t values (100, 'd'); + +--echo ## --------------------------Test update--------------------------- +--echo ## Test update 1 partition. +delete from t; +insert into t values (1, 'a'),(2,'b'),(3,'c'); +update t set name='b' where id=2;; +select * from t partition(p1); +update t set name='x' where id in (1,2); +select * from t partition(p1); +update t set name='y' where id < 3; +select * from t order by id; +--echo ## Test update meet duplicate error. +--error 1062 +update t set id=2 where id = 1; +select * from t order by id; + +--echo ## Test update multi-partitions +update t set name='z' where id in (1,2,3);; +select * from t order by id; +update t set name='a' limit 3; +select * from t order by id; +update t set id=id*10 where id in (1,2); +select * from t order by id; +--echo ## Test update meet duplicate error. +--error 1062 +update t set id=id+17 where id in (3,10); +select * from t order by id; +--echo ## Test update meet no partition error. +--error 1526 +update t set id=id*2 where id in (3,20); +select * from t order by id; + +--echo ## --------------------------Test replace--------------------------- +--echo ## Test replace 1 partition. +delete from t; +replace into t values (1, 'a'),(2,'b'); +select * from t order by id; +--echo ## Test replace multi-partitions. +replace into t values (3, 'c'),(4,'d'),(7,'f'); +select * from t partition(p0) order by id; +select * from t partition(p1) order by id; +select * from t partition(p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test replace on duplicate. +replace into t values (1, 'x'),(7,'x'); +select * from t order by id; +--echo ## Test replace meet no partition error. +--error 1526 +replace into t values (10,'x'),(100,'x'); +select * from t order by id; + +--echo ## --------------------------Test delete--------------------------- +--echo ## Test delete 1 partition. +delete from t where id = 3; +select * from t partition(p0) order by id; +delete from t where id in (1,2); +select * from t partition(p1) order by id; +--echo ## Test delete multi-partitions. +delete from t where id in (4,7,10,11); +select * from t; +insert into t values (3, 'c'),(4,'d'),(7,'f'); +delete from t where id < 10; +select * from t; +insert into t values (3, 'c'),(4,'d'),(7,'f'); +delete from t limit 3; +select * from t; + +set @@session.tidb_enable_list_partition = default; + + +# TestPartitionedTableReplace +set tidb_opt_fix_control = "44262:ON"; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) + partition by range (id) ( + PARTITION p0 VALUES LESS THAN (3), + PARTITION p1 VALUES LESS THAN (5), + PARTITION p2 VALUES LESS THAN (7), + PARTITION p3 VALUES LESS THAN (9)); +--enable_info +replace replace_test (c1) values (1),(2),(NULL); +--disable_info +begin; +-- error 1136 +replace replace_test (c1) values (); +rollback; +begin; +-- error 1136 +replace replace_test (c1, c2) values (1,2),(1); +rollback; +begin; +-- error 1054 +replace replace_test (xxx) values (3); +rollback; +begin; +-- error 1146 +replace replace_test_xxx (c1) values (); +rollback; +--enable_info +replace replace_test set c1 = 3; +--disable_info +begin; +-- error 1110 +replace replace_test set c1 = 4, c1 = 5; +rollback; +begin; +-- error 1054 +replace replace_test set xxx = 6; +rollback; + +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int) partition by range (id) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (6), + PARTITION p2 VALUES LESS THAN (8), + PARTITION p3 VALUES LESS THAN (10), + PARTITION p4 VALUES LESS THAN (100)); +--enable_info +replace replace_test_1 select id, c1 from replace_test; +--disable_info + +drop table if exists replace_test_2; +create table replace_test_2 (id int, c1 int) partition by range (id) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (50), + PARTITION p2 VALUES LESS THAN (100), + PARTITION p3 VALUES LESS THAN (300)); +--enable_info +replace replace_test_2 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +--disable_info +begin; +-- error 1136 +replace replace_test_2 select c1 from replace_test; +rollback; + +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by range (c2) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11)); +--enable_info +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c1=8, c2=8; +replace into replace_test_3 set c2=NULL; +replace into replace_test_3 set c2=NULL; +--disable_info + +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by range (c1) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11)); +--enable_info +replace into replace_test_4 set c2=NULL; +replace into replace_test_4 set c2=NULL; +--disable_info + +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by range (c2) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11)); +--enable_info +replace into replace_test_5 set c1=1, c2=2; +replace into replace_test_5 set c1=1, c2=2; +--disable_info + +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by range (b) ( + PARTITION p1 VALUES LESS THAN (100), + PARTITION p2 VALUES LESS THAN (200)); +--enable_info +insert into tIssue989 (a, b) values (1, 2); +replace into tIssue989(a, b) values (111, 2); +--disable_info +select * from tIssue989; + +set tidb_opt_fix_control = default; + + +# TestPartitionedTableUpdate +set tidb_opt_fix_control = "44262:ON"; +drop table if exists t; +create table t (id int not null default 1, name varchar(255)) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21)); +insert INTO t VALUES (1, "hello"); +insert INTO t VALUES (7, "hello"); + +--echo ## update non partition column +--enable_info +UPDATE t SET name = "abc" where id > 0; +--disable_info +SELECT * from t order by id limit 2; + +--echo ## update partition column +--enable_info +update t set id = id + 1; +--disable_info +SELECT * from t order by id limit 2; + +--echo ## update partition column, old and new record locates on different partitions +--enable_info +update t set id = 20 where id = 8; +--disable_info +SELECT * from t order by id limit 2; + +--echo ## table option is auto-increment +drop table if exists t; +create table t (id int not null auto_increment, name varchar(255), primary key(id)) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21)); +insert into t(name) values ('aa'); +--enable_info +update t set id = 8 where name = 'aa'; +--disable_info +insert into t(name) values ('bb'); +select * from t; +-- error 1048 +update t set id = null where name = 'aa'; + +--echo ## Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. +drop table if exists t; +create table t (id int, name int unique) + PARTITION BY RANGE ( name ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21)); +insert t values (1, 1), (2, 2); +-- error 1062 +update t set name = 1 where id = 2; +select * from t; + +--echo ## test update ignore for pimary key +drop table if exists t; +create table t(a bigint, primary key (a)) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11)); +insert into t values (5); +insert into t values (7); +update ignore t set a = 5 where a = 7; +SHOW WARNINGS; +select * from t order by a; + +--echo ## test update ignore for truncate as warning +update ignore t set a = 1 where a = (select '2a'); +SHOW WARNINGS; + +--echo ## test update ignore for unique key +drop table if exists t; +create table t(a bigint, unique key I_uniq (a)) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11)); +insert into t values (5); +insert into t values (7); +--enable_info +update ignore t set a = 5 where a = 7; +--disable_info +SHOW WARNINGS; +select * from t order by a; +set tidb_opt_fix_control = default; + + +# TestPartitionedTableDelete +drop table if exists t; +set tidb_opt_fix_control = "44262:ON"; +CREATE TABLE t (id int not null default 1, name varchar(255), index(id)) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21)); +insert into t values (1, "hello"),(2, "hello"),(3, "hello"),(4, "hello"),(5, "hello"),(6, "hello"),(7, "hello"),(8, "hello"),(9, "hello"),(10, "hello"),(11, "hello"),(12, "hello"),(13, "hello"),(14, "hello"),(15, "hello"),(16, "hello"),(17, "hello"),(18, "hello"),(19, "hello"),(20, "hello"); +--enable_info +delete from t where id = 2 limit 1; + +--echo ## Test delete with false condition +delete from t where 0; +--disable_info + +insert into t values (2, 'abc'); +--enable_info +delete from t where t.id = 2 limit 1; +--disable_info + +--echo ## Test delete ignore +insert into t values (2, 'abc'); +## TODO: https://github.com/pingcap/tidb/issues/48120 +--replace_regex /INTEGER/DOUBLE/ +-- error 1292 +delete from t where id = (select '2a'); +--enable_info +delete ignore from t where id = (select '2a'); +--disable_info +SHOW WARNINGS; + +--echo ## Test delete without using index, involve multiple partitions. +--enable_info +delete from t ignore index(id) where id >= 13 and id <= 17; +--disable_info + +admin check table t; +--enable_info +delete from t; +--disable_info + +--echo ## Fix that partitioned table should not use PointGetPlan. +drop table if exists t1; +create table t1 (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range (c1) (partition p0 values less than (3440)); +insert into t1 values (379, 379, 379); +--enable_info +delete from t1 where c1 = 379; +--disable_info +drop table t1; + +set tidb_opt_fix_control=default; + + +# TestHashPartitionedTableReplace +set @@session.tidb_enable_table_partition = '1'; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) + partition by hash(id) partitions 4; +replace replace_test (c1) values (1),(2),(NULL); +begin; +-- error 1136 +replace replace_test (c1) values (); +rollback; +begin; +-- error 1136 +replace replace_test (c1, c2) values (1,2),(1); +rollback; +begin; +-- error 1054 +replace replace_test (xxx) values (3); +rollback; +begin; +-- error 1146 +replace replace_test_xxx (c1) values (); +rollback; +begin; +-- error 1110 +replace replace_test set c1 = 4, c1 = 5; +rollback; +begin; +-- error 1054 +replace replace_test set xxx = 6; +rollback; +replace replace_test set c1 = 3; +replace replace_test set c1 = 4; +replace replace_test set c1 = 5; +replace replace_test set c1 = 6; +replace replace_test set c1 = 7; + +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int) partition by hash(id) partitions 5; +replace replace_test_1 select id, c1 from replace_test; + +drop table if exists replace_test_2; +create table replace_test_2 (id int, c1 int) partition by hash(id) partitions 6; +replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +begin; +-- error 1136 +replace replace_test_1 select c1 from replace_test; +rollback; + +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by hash(c2) partitions 7; +replace into replace_test_3 set c2=8; +--enable_info +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c1=8, c2=8; +--disable_info +replace into replace_test_3 set c2=NULL; +--enable_info +replace into replace_test_3 set c2=NULL; +--disable_info + +replace into replace_test_3 set c2=0; +replace into replace_test_3 set c2=1; +replace into replace_test_3 set c2=2; +replace into replace_test_3 set c2=3; +replace into replace_test_3 set c2=4; +replace into replace_test_3 set c2=5; +replace into replace_test_3 set c2=6; +replace into replace_test_3 set c2=7; +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c2=9; +select count(*) from replace_test_3; + +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by hash(c1) partitions 8; +replace into replace_test_4 set c2=NULL; +--enable_info +replace into replace_test_4 set c2=NULL; +--disable_info + +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by hash (c2) partitions 9; +replace into replace_test_5 set c1=1, c2=2; +--enable_info +replace into replace_test_5 set c1=1, c2=2; +--disable_info + +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by hash (b) partitions 10; +insert into tIssue989 (a, b) values (1, 2); +replace into tIssue989(a, b) values (111, 2); +select * from tIssue989; + +set @@session.tidb_enable_table_partition = default; diff --git a/tests/integrationtest/t/executor/stale_txn.test b/tests/integrationtest/t/executor/stale_txn.test index 0d11911ead762..77c0998c0aceb 100644 --- a/tests/integrationtest/t/executor/stale_txn.test +++ b/tests/integrationtest/t/executor/stale_txn.test @@ -32,3 +32,10 @@ set tidb_txn_mode = default; set tx_isolation = default; set autocommit = default; +# TestIssue33728 +drop table if exists t1; +create table t1 (id int primary key, v int); +--error 8135 +select * from t1 as of timestamp NULL; +--error 8135 +start transaction read only as of timestamp NULL; diff --git a/tests/integrationtest/t/executor/update.test b/tests/integrationtest/t/executor/update.test index 2eed96a2cd228..26989f69b92ce 100644 --- a/tests/integrationtest/t/executor/update.test +++ b/tests/integrationtest/t/executor/update.test @@ -266,3 +266,433 @@ update t1 set a='1000000000000000000' where id=2; select id, a from t1 order by id asc; set sql_mode=default; +# TestIssue21447 +drop table if exists t1; +create table t1(id int primary key, name varchar(40)); +insert into t1 values(1, 'abc'); + +begin pessimistic; + +connect (conn1, localhost, root,, executor__update); +begin pessimistic; +--enable_info +update t1 set name='xyz' where id=1; +--disable_info +select * from t1 where id = 1; +commit; +disconnect conn1; + +--enable_info +update t1 set name='xyz' where id=1; +--disable_info +select * from t1 where id = 1; +select * from t1 where id = 1 for update; +select * from t1 where id in (1, 2); +select * from t1 where id in (1, 2) for update; +commit; + +# TestUpdate +drop table if exists update_test; +create table update_test(id int not null default 1, name varchar(255), PRIMARY KEY(id)); +insert INTO update_test VALUES (1, "hello"); +insert into update_test values (2, "hello"); +--enable_info +UPDATE update_test SET name = "abc" where id > 0; +--disable_info + +## select data +begin; +SELECT * from update_test limit 2; +commit; + +--enable_info +UPDATE update_test SET name = "foo"; +--disable_info + +## table option is auto-increment +begin; +drop table if exists update_test; +commit; +begin; +create table update_test(id int not null auto_increment, name varchar(255), primary key(id)); +insert into update_test(name) values ('aa'); +--enable_info +update update_test set id = 8 where name = 'aa'; +--disable_info +insert into update_test(name) values ('bb'); +commit; +begin; +select * from update_test; +commit; +begin; +drop table if exists update_test; +commit; +begin; +create table update_test(id int not null auto_increment, name varchar(255), index(id)); +insert into update_test(name) values ('aa'); +-- error 1048 +update update_test set id = null where name = 'aa'; + +drop table update_test; +create table update_test(id int); +begin; +insert into update_test(id) values (1); +--enable_info +update update_test set id = 2 where id = 1 limit 1; +--disable_info +select * from update_test; +commit; + +## Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. +drop table if exists update_unique; +create table update_unique (id int primary key, name int unique); +insert update_unique values (1, 1), (2, 2); +begin; +-- error 1062 +update update_unique set name = 1 where id = 2; +commit; +select * from update_unique; + +## test update ignore for pimary key +drop table if exists t; +create table t(a bigint, primary key (a)); +insert into t values (1); +insert into t values (2); +--enable_info +update ignore t set a = 1 where a = 2; +--disable_info +SHOW WARNINGS; +select * from t; + +## test update ignore for truncate as warning +update ignore t set a = 1 where a = (select '2a'); +SHOW WARNINGS; + +update ignore t set a = 42 where a = 2; +select * from t; + +## test update ignore for unique key +drop table if exists t; +create table t(a bigint, unique key I_uniq (a)); +insert into t values (1); +insert into t values (2); +--enable_info +update ignore t set a = 1 where a = 2; +--disable_info +SHOW WARNINGS; +select * from t; + +## test issue21965 +drop table if exists t; +set @@session.tidb_enable_list_partition = ON; +create table t (a int) partition by list (a) (partition p0 values in (0,1)); +analyze table t; +insert ignore into t values (1); +--enable_info +update ignore t set a=2 where a=1; +--disable_info +drop table if exists t; +create table t (a int key) partition by list (a) (partition p0 values in (0,1)); +insert ignore into t values (1); +--enable_info +update ignore t set a=2 where a=1; +--disable_info +set @@session.tidb_enable_list_partition = default; + +drop table if exists t; +create table t(id integer auto_increment, t1 datetime, t2 datetime, primary key (id)); +insert into t(t1, t2) values('2000-10-01 01:01:01', '2017-01-01 10:10:10'); +select * from t; +--enable_info +update t set t1 = '2017-10-01 10:10:11', t2 = date_add(t1, INTERVAL 10 MINUTE) where id = 1; +--disable_info +select * from t; + +## for issue #5132 +drop table if exists tt1; +CREATE TABLE `tt1` (`a` int(11) NOT NULL,`b` varchar(32) DEFAULT NULL,`c` varchar(32) DEFAULT NULL,PRIMARY KEY (`a`),UNIQUE KEY `b_idx` (`b`)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +insert into tt1 values(1, 'a', 'a'); +insert into tt1 values(2, 'd', 'b'); +select * from tt1; +--enable_info +update tt1 set a=5 where c='b'; +--disable_info +select * from tt1; + +## Automatic Updating for TIMESTAMP +drop table if exists tsup; +CREATE TABLE `tsup` (`a` int,`ts` TIMESTAMP DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,KEY `idx` (`ts`)); +set @@sql_mode=''; +insert into tsup values(1, '0000-00-00 00:00:00'); +--enable_info +update tsup set a=5; +--disable_info +select t1.ts = t2.ts from (select ts from tsup use index (idx)) as t1, (select ts from tsup use index ()) as t2; +update tsup set ts='2019-01-01'; +select ts from tsup; +set @@sql_mode=default; + +## issue 5532 +drop table if exists decimals; +create table decimals (a decimal(20, 0) not null); +insert into decimals values (201); +## A warning rather than data truncated error. +--enable_info +update decimals set a = a + 1.23; +--disable_info +show warnings; +select * from decimals; + +drop table t; +CREATE TABLE `t` ( `c1` year DEFAULT NULL, `c2` year DEFAULT NULL, `c3` date DEFAULT NULL, `c4` datetime DEFAULT NULL, KEY `idx` (`c1`,`c2`)); +UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2; +-- error 1288 +update (select * from t) t set c1 = 1111111; + +## test update ignore for bad null error +drop table if exists t; +create table t (i int not null default 10); +insert into t values (1); +--enable_info +update ignore t set i = null; +--disable_info +SHOW WARNINGS; +select * from t; + +## issue 7237, update subquery table should be forbidden +drop table t; +create table t (k int, v int); +-- error 1288 +update t, (select * from t) as b set b.k = t.k; +update t, (select * from t) as b set t.k = b.k; + +## issue 8045 +drop table if exists t1; +CREATE TABLE t1 (c1 float); +INSERT INTO t1 SET c1 = 1; +--enable_info +UPDATE t1 SET c1 = 1.2 WHERE c1=1; +--disable_info + +## issue 8119 +drop table if exists t; +create table t (c1 float(1,1)); +insert into t values (0.0); +-- error 1264 +update t set c1 = 2.0; + +drop table if exists t; +create table t(a datetime not null, b datetime); +insert into t value('1999-12-12', '1999-12-13'); +set @@sql_mode=''; +select * from t; +update t set a = ''; +select * from t; +update t set b = ''; +select * from t; +set @@sql_mode=default; + +drop view if exists v; +create view v as select * from t; +-- error 1356 +update v set a = '2000-11-11'; +drop view v; + +drop sequence if exists seq; +create sequence seq; +-- error 1054 +update seq set minvalue=1; +drop sequence seq; + +drop table if exists t1, t2; +create table t1(a int, b int, c int, d int, e int, index idx(a)); +create table t2(a int, b int, c int); +update t1 join t2 on t1.a=t2.a set t1.a=1 where t2.b=1 and t2.c=2; + +## Assign `DEFAULT` in `UPDATE` statement +drop table if exists t1, t2; +create table t1 (a int default 1, b int default 2); +insert into t1 values (10, 10), (20, 20); +update t1 set a=default where b=10; +select * from t1; +update t1 set a=30, b=default where a=20; +select * from t1; +update t1 set a=default, b=default where a=30; +select * from t1; +insert into t1 values (40, 40); +update t1 set a=default, b=default; +select * from t1; +update t1 set a=default(b), b=default(a); +select * from t1; +## With generated columns +create table t2 (a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored); +insert into t2 values (10, default, default), (20, default, default); +update t2 set b=default; +select * from t2; +update t2 set a=30, b=default where a=10; +select * from t2; +update t2 set c=default, a=40 where c=-20; +select * from t2; +update t2 set a=default, b=default, c=default where b=-30; +select * from t2; +update t2 set a=default(a), b=default, c=default; +select * from t2; +## Same as in MySQL 8.0.27, but still weird behavior: a=default(b) => NULL +update t2 set a=default(b), b=default, c=default; +select * from t2; +-- error 3105 +update t2 set b=default(a); +update t2 set a=default(a), c=default(c); +select * from t2; +## Same as in MySQL 8.0.27, but still weird behavior: a=default(b) => NULL +update t2 set a=default(b), b=default(b); +select * from t2; +update t2 set a=default(a), c=default(c); +select * from t2; +## Allowed in MySQL, but should probably not be allowed. +-- error 3105 +update t2 set a=default(a), c=default(a); +drop table t1, t2; + +# TestUpdateSelect +drop table if exists msg, detail; +create table msg (id varchar(8), b int, status int, primary key (id, b)); +insert msg values ('abc', 1, 1); +create table detail (id varchar(8), start varchar(8), status int, index idx_start(start)); +insert detail values ('abc', '123', 2); +--enable_info +UPDATE msg SET msg.status = (SELECT detail.status FROM detail WHERE msg.id = detail.id); +--disable_info +admin check table msg; + +# TestUpdateDelete +drop table if exists ttt; +CREATE TABLE ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host)); +insert into ttt values (8,8),(9,9); +begin; +--enable_info +update ttt set id = 0, host='9' where id = 9 limit 1; +--disable_info +delete from ttt where id = 0 limit 1; +select * from ttt use index (i_host) order by host; +--enable_info +update ttt set id = 0, host='8' where id = 8 limit 1; +--disable_info +delete from ttt where id = 0 limit 1; +select * from ttt use index (i_host) order by host; +commit; +admin check table ttt; +drop table ttt; + +# TestUpdateAffectRowCnt +drop table if exists a; +create table a(id int auto_increment, a int default null, primary key(id)); +insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1); +--enable_info +update a set id = id*10 where a = 1001; +--disable_info +drop table a; +create table a ( a bigint, b bigint); +insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1); +--enable_info +update a set a = a*10 where b = 1001; +--disable_info + +# TestMultipleTableUpdate +drop table if exists items, month; +CREATE TABLE items (id int, price TEXT); +--enable_info +insert into items values (11, "items_price_11"), (12, "items_price_12"), (13, "items_price_13"); +--disable_info +CREATE TABLE month (mid int, mprice TEXT); +--enable_info +insert into month values (11, "month_price_11"), (22, "month_price_22"), (13, "month_price_13"); +UPDATE items, month SET items.price=month.mprice WHERE items.id=month.mid; +--disable_info +begin; +SELECT * FROM items; +commit; + +## Single-table syntax but with multiple tables +--enable_info +UPDATE items join month on items.id=month.mid SET items.price=month.mid; +--disable_info +begin; +SELECT * FROM items; +commit; + +## JoinTable with alias table name. +--enable_info +UPDATE items T0 join month T1 on T0.id=T1.mid SET T0.price=T1.mprice; +--disable_info +begin; +SELECT * FROM items; +commit; + +## fix https://github.com/pingcap/tidb/issues/369 +DROP TABLE IF EXISTS t1, t2; +create table t1 (c int); +create table t2 (c varchar(256)); +insert into t1 values (1), (2); +insert into t2 values ("a"), ("b"); +--enable_info +update t1, t2 set t1.c = 10, t2.c = "abc"; +--disable_info + +## fix https://github.com/pingcap/tidb/issues/376 +DROP TABLE IF EXISTS t1, t2; +create table t1 (c1 int); +create table t2 (c2 int); +insert into t1 values (1), (2); +insert into t2 values (1), (2); +--enable_info +update t1, t2 set t1.c1 = 10, t2.c2 = 2 where t2.c2 = 1; +--disable_info +select * from t1; + +## test https://github.com/pingcap/tidb/issues/3604 +drop table if exists t; +create table t (a int, b int); +--enable_info +insert into t values(1, 1), (2, 2), (3, 3); +update t m, t n set m.a = m.a + 1; +--disable_info +select * from t; +--enable_info +update t m, t n set n.a = n.a - 1, n.b = n.b + 1; +--disable_info +select * from t; + +# TestUpdateCastOnlyModifiedValues for issue #4514. +drop table if exists update_modified; +create table update_modified (col_1 int, col_2 enum('a', 'b')); +set SQL_MODE=''; +insert into update_modified values (0, 3); +SELECT * FROM update_modified; +set SQL_MODE=STRICT_ALL_TABLES; +--enable_info +update update_modified set col_1 = 1; +--disable_info +SELECT * FROM update_modified; +-- error 1265 +update update_modified set col_1 = 2, col_2 = 'c'; +SELECT * FROM update_modified; +--enable_info +update update_modified set col_1 = 3, col_2 = 'a'; +--disable_info +SELECT * FROM update_modified; + +## Test update a field with different column type. +drop table if exists update_with_diff_type; +CREATE TABLE update_with_diff_type (a int, b JSON); +INSERT INTO update_with_diff_type VALUES(3, '{"a": "测试"}'); +--enable_info +UPDATE update_with_diff_type SET a = '300'; +--disable_info +SELECT a FROM update_with_diff_type; +--enable_info +UPDATE update_with_diff_type SET b = '{"a": "\\u6d4b\\u8bd5"}'; +--disable_info +SELECT b FROM update_with_diff_type; +set SQL_MODE=default; + diff --git a/tests/integrationtest/t/executor/write.test b/tests/integrationtest/t/executor/write.test index 6ff53470eb9ff..a76250b25ab29 100644 --- a/tests/integrationtest/t/executor/write.test +++ b/tests/integrationtest/t/executor/write.test @@ -1216,3 +1216,137 @@ select b from t where a=1; set @@session.tidb_enable_list_partition = default; set @@allow_auto_random_explicit_insert = default; + + +# TestReplace +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1); +--enable_info +replace replace_test (c1) values (1),(2),(NULL); +--disable_info +begin; +-- error 1136 +replace replace_test (c1) values (); +rollback; +begin; +-- error 1136 +replace replace_test (c1, c2) values (1,2),(1); +rollback; +begin; +-- error 1054 +replace replace_test (xxx) values (3); +rollback; +begin; +-- error 1146 +replace replace_test_xxx (c1) values (); +rollback; +--enable_info +replace replace_test set c1 = 3; +--disable_info +begin; +-- error 1110 +replace replace_test set c1 = 4, c1 = 5; +rollback; +begin; +-- error 1054 +replace replace_test set xxx = 6; +rollback; + +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int); +--enable_info +replace replace_test_1 select id, c1 from replace_test; +--disable_info +begin; +-- error 1135 +replace replace_test_0 select c1 from replace_test; +rollback; + +create table replace_test_2 (id int, c1 int); +--enable_info +replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +--disable_info + +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)); +--enable_info +replace into replace_test_3 set c2=1; +replace into replace_test_3 set c2=1; +replace into replace_test_3 set c1=1, c2=1; +replace into replace_test_3 set c2=NULL; +replace into replace_test_3 set c2=NULL; +--disable_info + +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)); +--enable_info +replace into replace_test_4 set c2=NULL; +replace into replace_test_4 set c2=NULL; +--disable_info + +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)); +--enable_info +replace into replace_test_5 set c1=1, c2=2; +replace into replace_test_5 set c1=1, c2=2; +--disable_info + +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b)); +--enable_info +insert into tIssue989 (a, b) values (1, 2); +replace into tIssue989(a, b) values (111, 2); +--disable_info +select * from tIssue989; + +drop table if exists tIssue1012; +CREATE TABLE tIssue1012 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b)); +insert into tIssue1012 (a, b) values (1, 2); +insert into tIssue1012 (a, b) values (2, 1); +--enable_info +replace into tIssue1012(a, b) values (1, 1); +--disable_info +select * from tIssue1012; + +drop table if exists t1; +create table t1(a int primary key, b int); +insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5); +--enable_info +replace into t1 values(1,1); +replace into t1 values(1,1),(2,2); +replace into t1 values(4,14),(5,15),(6,16),(7,17),(8,18); +replace into t1 select * from (select 1, 2) as tmp; +--disable_info + +drop table if exists t1, t2; +create table t1 (a int primary key, b int default 20, c int default 30); +insert into t1 value (1, 2, 3); +replace t1 set a=1, b=default; +select * from t1; +replace t1 set a=2, b=default, c=default; +select * from t1; +replace t1 set a=2, b=default(c), c=default(b); +select * from t1; +replace t1 set a=default(b)+default(c); +select * from t1; +create table t2 (pk int primary key, a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored); +replace t2 set pk=1, b=default; +select * from t2; +replace t2 set pk=2, a=10, b=default; +select * from t2; +replace t2 set pk=2, c=default, a=20; +select * from t2; +replace t2 set pk=2, a=default, b=default, c=default; +select * from t2; +replace t2 set pk=3, a=default(a), b=default, c=default; +select * from t2; +-- error 3105 +replace t2 set b=default(a); +-- error 3105 +replace t2 set a=default(b), b=default(b); +-- error 1364 +replace t2 set a=default(a), c=default(c); +-- error 3105 +replace t2 set c=default(a); +drop table t1, t2; + diff --git a/tests/integrationtest/t/planner/core/binary_plan.test b/tests/integrationtest/t/planner/core/binary_plan.test new file mode 100644 index 0000000000000..5aae531269fd8 --- /dev/null +++ b/tests/integrationtest/t/planner/core/binary_plan.test @@ -0,0 +1,10 @@ +# TestInvalidDecodeBinaryPlan +select tidb_decode_binary_plan('some random bytes'); +show warnings; +## base64.StdEncoding.EncodeToString("some random bytes") +select tidb_decode_binary_plan('c29tZSByYW5kb20gYnl0ZXM='); +show warnings; +## base64.StdEncoding.EncodeToString(snappy.Encode(nil, "some random bytes")) +select tidb_decode_binary_plan('EUBzb21lIHJhbmRvbSBieXRlcw=='); +show warnings; + diff --git a/tests/integrationtest/t/planner/core/cbo.test b/tests/integrationtest/t/planner/core/cbo.test index 09bb8aa09203e..7eab31bc66b5e 100644 --- a/tests/integrationtest/t/planner/core/cbo.test +++ b/tests/integrationtest/t/planner/core/cbo.test @@ -59,6 +59,6 @@ create table t2(a int, b int); insert into t1 values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5); insert into t2 values (2, 22), (3, 33), (5, 55), (233, 2), (333, 3), (3434, 5); analyze table t1, t2; ---replace_regex /:[ ]?[.0-9]+[nµms]*/:/ /, scan_detail: {.*}// / tot_proc:.*?, tot_wait:.*?,// /[0-9]+ Bytes/ Bytes/ /[.0-9]+ KB/ KB/ +--replace_regex /:[ ]?[.0-9]+[nµms]*/:/ /, scan_detail: {.*}// / tot_proc:.*?, tot_wait:.*?,// /[.0-9]+ ((KB)|(Bytes))// explain analyze select t1.a, t1.b, sum(t1.c) from t1 join t2 on t1.a = t2.b where t1.a > 1; set sql_mode=default; diff --git a/tests/integrationtest/t/planner/core/plan.test b/tests/integrationtest/t/planner/core/plan.test index fcdd997eef335..896fd62e082f7 100644 --- a/tests/integrationtest/t/planner/core/plan.test +++ b/tests/integrationtest/t/planner/core/plan.test @@ -162,3 +162,39 @@ alter table t set hypo tiflash replica 0; explain select a from t; +# TestIssue40857 +drop table if exists t; +CREATE TABLE t (c1 mediumint(9) DEFAULT '-4747160',c2 year(4) NOT NULL DEFAULT '2075',c3 double DEFAULT '1.1559030660251948',c4 enum('wbv4','eli','d8ym','m3gsx','lz7td','o','d1k7l','y1x','xcxq','bj','n7') DEFAULT 'xcxq',c5 int(11) DEFAULT '255080866',c6 tinyint(1) DEFAULT '1',PRIMARY KEY (c2),KEY `c4d86d54-091c-4307-957b-b164c9652b7f` (c6,c4) ); +insert into t values (-4747160, 2075, 722.5719203870632, 'xcxq', 1576824797, 1); +--enable_info +select /*+ stream_agg() */ bit_or(t.c5) as r0 from t where t.c3 in (select c6 from t where not(t.c6 <> 1) and not(t.c3 in(9263.749352636818))) group by t.c1; +--disable_info + + +# TestIssue40535 +drop table if exists t1; +drop table if exists t2; +CREATE TABLE `t1`(`c1` bigint(20) NOT NULL DEFAULT '-2312745469307452950', `c2` datetime DEFAULT '5316-02-03 06:54:49', `c3` tinyblob DEFAULT NULL, PRIMARY KEY (`c1`) /*T![clustered_index] CLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +CREATE TABLE `t2`(`c1` set('kn8pu','7et','vekx6','v3','liwrh','q14','1met','nnd5i','5o0','8cz','l') DEFAULT '7et,vekx6,liwrh,q14,1met', `c2` float DEFAULT '1.683167', KEY `k1` (`c2`,`c1`), KEY `k2` (`c2`)) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci; +--enable_info +(select /*+ agg_to_cop()*/ locate(t1.c3, t1.c3) as r0, t1.c3 as r1 from t1 where not( IsNull(t1.c1)) order by r0,r1) union all (select concat_ws(',', t2.c2, t2.c1) as r0, t2.c1 as r1 from t2 order by r0, r1) order by 1 limit 273; +--disable_info + + +# TestIssue47445 +drop table if exists golang1, golang2; +CREATE TABLE golang1 ( `fcbpdt` CHAR (8) COLLATE utf8_general_ci NOT NULL, `fcbpsq` VARCHAR (20) COLLATE utf8_general_ci NOT NULL, `procst` char (4) COLLATE utf8_general_ci DEFAULT NULL,`cipstx` VARCHAR (105) COLLATE utf8_general_ci DEFAULT NULL, `cipsst` CHAR (4) COLLATE utf8_general_ci DEFAULT NULL, `dyngtg` VARCHAR(4) COLLATE utf8_general_ci DEFAULT NULL, `blncdt` VARCHAR (8) COLLATE utf8_general_ci DEFAULT NULL, PRIMARY KEY ( fcbpdt, fcbpsq )); +insert into golang1 values('20230925','12023092502158016','abc','','','',''); +create table golang2 (`sysgrp` varchar(20) NOT NULL,`procst` varchar(8) NOT NULL,`levlid` int(11) NOT NULL,PRIMARY key (procst));; +insert into golang2 VALUES('COMMON','ACSC',90); +insert into golang2 VALUES('COMMON','abc',8); +insert into golang2 VALUES('COMMON','CH02',6); +UPDATE golang1 a SET procst =(CASE WHEN ( SELECT levlid FROM golang2 b WHERE b.sysgrp = 'COMMON' AND b.procst = 'ACSC' ) > ( SELECT levlid FROM golang2 c WHERE c.sysgrp = 'COMMON' AND c.procst = a.procst ) THEN 'ACSC' ELSE a.procst END ), cipstx = 'CI010000', cipsst = 'ACSC', dyngtg = 'EAYT', blncdt= '20230925' WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'; +select * from golang1; +UPDATE golang1 a SET procst= (SELECT 1 FROM golang2 c WHERE c.procst = a.procst) WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'; +select * from golang1; + + +# TestExplainValuesStatement +--error 1051 +EXPLAIN FORMAT = TRADITIONAL ((VALUES ROW ()) ORDER BY 1); diff --git a/tests/integrationtest/t/planner/core/plan_cache.test b/tests/integrationtest/t/planner/core/plan_cache.test index 1be4a42c71b6a..23d20b496bb1d 100644 --- a/tests/integrationtest/t/planner/core/plan_cache.test +++ b/tests/integrationtest/t/planner/core/plan_cache.test @@ -1431,3 +1431,227 @@ select database() from t; select @@last_plan_from_cache; set tidb_enable_non_prepared_plan_cache=DEFAULT; +# TestPlanCacheSizeSwitch +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; +set @@tidb_prepared_plan_cache_size = 200; +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; +set @@tidb_session_plan_cache_size = 300; +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; + +set global tidb_prepared_plan_cache_size = 400; +connect (conn1, localhost, root,,); +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; +disconnect conn1; + +set global tidb_session_plan_cache_size = 500; +connect (conn1, localhost, root,,); +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; +disconnect conn1; + +set global tidb_prepared_plan_cache_size = default; +set global tidb_session_plan_cache_size = default; +set @@tidb_session_plan_cache_size = default; +set @@tidb_prepared_plan_cache_size = default; + +# TestNonPreparedPlanCacheWithExplain +drop table if exists t; +create table t(a int); +set tidb_enable_non_prepared_plan_cache=1; +select * from t where a=1; +explain select * from t where a=2; +select @@last_plan_from_cache; +explain format=verbose select * from t where a=2; +select @@last_plan_from_cache; +--replace_column 5 6 7 8 9 +explain analyze select * from t where a=2; +select @@last_plan_from_cache; +set tidb_enable_non_prepared_plan_cache=default; + +# TestPlanCacheGeneratedCols +drop table if exists t1, t2, t3, t4; +set @@tidb_opt_fix_control = "45798:on"; +create table t1 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city')))); +create table t2 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) virtual); +create table t3 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) stored); +create table t4 (a int, info json, index zips( (CAST(info->'$.zipcode' AS UNSIGNED ARRAY)))); +set @a=1; +set @b=2; +prepare s1 from 'select * from t1 where a=?'; +show warnings; +execute s1 using @a; +execute s1 using @b; +select @@last_plan_from_cache; +prepare s1 from 'select * from t2 where a=?'; +show warnings; +execute s1 using @a; +execute s1 using @b; +select @@last_plan_from_cache; +prepare s1 from 'select * from t3 where a=?'; +show warnings; +execute s1 using @a; +execute s1 using @b; +select @@last_plan_from_cache; +prepare s1 from 'select * from t4 where a=?'; +show warnings; +execute s1 using @a; +execute s1 using @b; +select @@last_plan_from_cache; + +set @@tidb_opt_fix_control = default; + +# TestPlanCacheGeneratedCols2 +drop table if exists t1, t2; +set @@tidb_opt_fix_control = "45798:on"; +CREATE TABLE t1 ( + ipk varbinary(255) NOT NULL, + i_id varchar(45) DEFAULT NULL, + i_set_id varchar(45) DEFAULT NULL, + p_id varchar(45) DEFAULT NULL, + p_set_id varchar(45) DEFAULT NULL, + m_id bigint(20) DEFAULT NULL, + m_i_id varchar(127) DEFAULT NULL, + m_i_set_id varchar(127) DEFAULT NULL, + d json DEFAULT NULL, + p_sources json DEFAULT NULL, + nslc json DEFAULT NULL, + cl json DEFAULT NULL, + fii json DEFAULT NULL, + fpi json DEFAULT NULL, + PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */, + UNIQUE KEY i_id (i_id), + KEY d ((cast(d as char(253) array))), + KEY m_i_id (m_i_id), + KEY m_i_set_id (m_i_set_id), + KEY fpi ((cast(fpi as unsigned array))), + KEY nslc ((cast(nslc as char(1000) array))), + KEY cl ((cast(cl as char(3000) array))), + KEY fii ((cast(fii as unsigned array))), + KEY m_id (m_id), + KEY i_set_id (i_set_id), + KEY m_i_and_m_id (m_i_id,m_id)); +CREATE TABLE t2 ( + ipk varbinary(255) NOT NULL, + created_time bigint(20) DEFAULT NULL, + arrival_time bigint(20) DEFAULT NULL, + updated_time bigint(20) DEFAULT NULL, + timestamp_data json DEFAULT NULL, + PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */); +prepare stmt from 'select * + from ( t1 left outer join t2 on ( t1 . ipk = t2 . ipk ) ) + where ( t1 . i_id = ? )'; +show warnings; +set @a='a', @b='b'; +execute stmt using @a; +execute stmt using @b; +select @@last_plan_from_cache; + +set @@tidb_opt_fix_control = default; + +# TestPlanCacheExprBlacklistCompatibility +drop table if exists t; +create table t (a int); +prepare st from 'select * from t where mod(a, 2)=1'; +execute st; +execute st; +select @@last_plan_from_cache; +insert into mysql.expr_pushdown_blacklist(name) values('mod'); +admin reload expr_pushdown_blacklist; +execute st; +select @@last_plan_from_cache; +execute st; +select @@last_plan_from_cache; + +delete from mysql.expr_pushdown_blacklist; +admin reload expr_pushdown_blacklist; + +# TestNonPreparedPlanCacheFieldNames +drop table if exists t, tt; +create table t(a int, index(a)); +create table tt(a varchar(10)); +set tidb_enable_non_prepared_plan_cache=1; +select a+1 from t where a<10; +select @@last_plan_from_cache; +select a+1 from t where a<20; +select @@last_plan_from_cache; +select a+2 from t where a<30; +select @@last_plan_from_cache; +select a+2 from t where a<40; +select @@last_plan_from_cache; +select a,a+1 from t where a<30; +select @@last_plan_from_cache; +select a,a+1 from t where a<40; +select @@last_plan_from_cache; +select a+'123' from tt where a='1'; +select @@last_plan_from_cache; +select a+'123' from tt where a='2'; +select @@last_plan_from_cache; +select 1 from t where a<10; +select @@last_plan_from_cache; +select 1 from t where a<20; +select @@last_plan_from_cache; +select 2 from t where a<10; +select @@last_plan_from_cache; +select 2 from t where a<20; +select @@last_plan_from_cache; +select 1,2 from t where a<10; +select @@last_plan_from_cache; +select 1,2 from t where a<20; +select @@last_plan_from_cache; +set tidb_enable_non_prepared_plan_cache=default; + +# TestIssue48165 +drop table if exists t; +create table t(a int); +insert into t values(1); +prepare s from "select * from t where tidb_parse_tso(a) > unix_timestamp()"; +execute s; + +# TestNonPreparedPlanCacheBuiltinFuncs +drop table if exists t; +set tidb_enable_non_prepared_plan_cache=1; +create table t (a int, b varchar(32), c datetime, key(a)); + +## normal builtin functions can be supported +select * from t where mod(a, 5) < 2; +select * from t where mod(a, 5) < 2; +select @@last_plan_from_cache; +select * from t where c < now(); +select * from t where c < now(); +select @@last_plan_from_cache; +select date_format(c, '%Y-%m-%d') from t where a < 10; +select date_format(c, '%Y-%m-%d') from t where a < 10; +select @@last_plan_from_cache; +select str_to_date(b, '%Y-%m-%d') from t where a < 10; +select str_to_date(b, '%Y-%m-%d') from t where a < 10; +select @@last_plan_from_cache; +select * from t where a-2 < 20; +select * from t where a-2 < 20; +select @@last_plan_from_cache; +select * from t where a+b > 100; +select * from t where a+b > 100; +select @@last_plan_from_cache; + +## '-' cannot support +select * from t where -a > 10; +select * from t where -a > 10; +select @@last_plan_from_cache; +## LIKE +select * from t where a < 1 and b like '%abc%'; +select * from t where a < 1 and b like '%abc%'; +select @@last_plan_from_cache; +select database() from t; +select database() from t; +select @@last_plan_from_cache; +set tidb_enable_non_prepared_plan_cache=default; + +# TestIssue45086 +drop table if exists t; +CREATE TABLE t (a int(11) DEFAULT NULL, b date DEFAULT NULL); +INSERT INTO t VALUES (1, current_date()); +PREPARE stmt FROM 'SELECT a FROM t WHERE b=current_date()'; +EXECUTE stmt;