Skip to content

Commit

Permalink
Merge branch 'release-6.1' into cherry-pick-36051-to-release-6.1
Browse files Browse the repository at this point in the history
  • Loading branch information
3pointer authored Jan 18, 2023
2 parents d688a7f + 3af286a commit 17a382c
Show file tree
Hide file tree
Showing 105 changed files with 2,147 additions and 577 deletions.
4 changes: 4 additions & 0 deletions bindinfo/session_handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -414,6 +414,10 @@ func (msm *mockSessionManager) GetInternalSessionStartTSList() []uint64 {
return nil
}

func (msm *mockSessionManager) GetMinStartTS(lowerBound uint64) uint64 {
return 0
}

func TestIssue19836(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
Expand Down
7 changes: 6 additions & 1 deletion br/pkg/lightning/restore/check_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -1156,7 +1156,12 @@ func tableContainsData(ctx context.Context, db utils.DBExecutor, tableName strin
failpoint.Inject("CheckTableEmptyFailed", func() {
failpoint.Return(false, errors.New("mock error"))
})
query := "select 1 from " + tableName + " limit 1"
// Here we use the `USE INDEX()` hint to skip fetch the record from index.
// In Lightning, if previous importing is halted half-way, it is possible that
// the data is partially imported, but the index data has not been imported.
// In this situation, if no hint is added, the SQL executor might fetch the record from index,
// which is empty. This will result in missing check.
query := "SELECT 1 FROM " + tableName + " USE INDEX() LIMIT 1"
exec := common.SQLWithRetry{
DB: db,
Logger: log.L(),
Expand Down
22 changes: 11 additions & 11 deletions br/pkg/lightning/restore/check_info_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -460,11 +460,11 @@ func TestCheckTableEmpty(t *testing.T) {
require.NoError(t, err)
mock.MatchExpectationsInOrder(false)
rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone)
mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test1`.`tbl1` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows))
mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test1`.`tbl2` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows))
mock.ExpectQuery("select 1 from `test2`.`tbl1` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test2`.`tbl1` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows))
// not error, need not to init check template
err = rc.checkTableEmpty(ctx)
Expand All @@ -477,13 +477,13 @@ func TestCheckTableEmpty(t *testing.T) {
rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone)
mock.MatchExpectationsInOrder(false)
// test auto retry retryable error
mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test1`.`tbl1` USE INDEX\\(\\) LIMIT 1").
WillReturnError(&gmysql.MySQLError{Number: errno.ErrPDServerTimeout})
mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test1`.`tbl1` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows))
mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test1`.`tbl2` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows))
mock.ExpectQuery("select 1 from `test2`.`tbl1` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test2`.`tbl1` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).AddRow(1))
rc.checkTemplate = NewSimpleTemplate()
err = rc.checkTableEmpty(ctx)
Expand All @@ -499,11 +499,11 @@ func TestCheckTableEmpty(t *testing.T) {
require.NoError(t, err)
rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone)
mock.MatchExpectationsInOrder(false)
mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test1`.`tbl1` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).AddRow(1))
mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test1`.`tbl2` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows))
mock.ExpectQuery("select 1 from `test2`.`tbl1` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test2`.`tbl1` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).AddRow(1))
rc.checkTemplate = NewSimpleTemplate()
err = rc.checkTableEmpty(ctx)
Expand Down Expand Up @@ -542,7 +542,7 @@ func TestCheckTableEmpty(t *testing.T) {
require.NoError(t, err)
rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone)
// only need to check the one that is not in checkpoint
mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1").
mock.ExpectQuery("SELECT 1 FROM `test1`.`tbl2` USE INDEX\\(\\) LIMIT 1").
WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows))
err = rc.checkTableEmpty(ctx)
require.NoError(t, err)
Expand Down
5 changes: 5 additions & 0 deletions br/pkg/lightning/restore/table_restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -359,6 +359,11 @@ func (tr *TableRestore) restoreEngines(pCtx context.Context, rc *Controller, cp
if cp.Status < checkpoints.CheckpointStatusIndexImported {
var err error
if indexEngineCp.Status < checkpoints.CheckpointStatusImported {
failpoint.Inject("FailBeforeStartImportingIndexEngine", func() {
errMsg := "fail before importing index KV data"
tr.logger.Warn(errMsg)
failpoint.Return(errors.New(errMsg))
})
err = tr.importKV(ctx, closedIndexEngine, rc, indexEngineID)
failpoint.Inject("FailBeforeIndexEngineImported", func() {
panic("forcing failure due to FailBeforeIndexEngineImported")
Expand Down
5 changes: 5 additions & 0 deletions br/tests/lightning_check_partial_imported/config.toml
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
[tikv-importer]
backend = "local"

[mydumper.csv]
header = true
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
CREATE TABLE tbl01 (
`id` INTEGER,
`val` VARCHAR(64),
`aaa` CHAR(66) DEFAULT NULL,
`bbb` CHAR(10) NOT NULL,
`ccc` CHAR(42) DEFAULT NULL,
`ddd` CHAR(42) DEFAULT NULL,
`eee` CHAR(66) DEFAULT NULL,
`fff` VARCHAR(128) DEFAULT NULL,
KEY `aaa` (`aaa`),
PRIMARY KEY (`id`)
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;
6 changes: 6 additions & 0 deletions br/tests/lightning_check_partial_imported/data/db01.tbl01.csv
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
id,val,aaa,bbb,ccc,ddd,eee,fff
1,"v01","a01","b01","c01","d01","e01","f01"
2,"v02","a02","b02","c02","d02","e02","f02"
3,"v03","a03","b03","c03","d03","e03","f03"
4,"v04","a04","b04","c04","d04","e04","f04"
5,"v05","a05","b05","c05","d05","e05","f05"
47 changes: 47 additions & 0 deletions br/tests/lightning_check_partial_imported/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
#!/bin/bash
#
# Copyright 2022 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.

MYDIR=$(dirname "${BASH_SOURCE[0]}")
set -eux

check_cluster_version 4 0 0 'local backend' || exit 0

export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/lightning/restore/FailBeforeStartImportingIndexEngine=return"
set +e
if run_lightning; then
echo "The first import doesn't fail as expected" >&2
exit 1
fi
set -e

data_records=$(tail -n +2 "${MYDIR}/data/db01.tbl01.csv" | wc -l | xargs echo )
run_sql "SELECT COUNT(*) FROM db01.tbl01 USE INDEX();"
check_contains "${data_records}"

export GO_FAILPOINTS=""
set +e
if run_lightning --check-requirements=1; then
echo "The pre-check doesn't find out the non-empty table problem"
exit 2
fi
set -e

run_sql "TRUNCATE TABLE db01.tbl01;"
run_lightning --check-requirements=1
run_sql "SELECT COUNT(*) FROM db01.tbl01;"
check_contains "${data_records}"
run_sql "SELECT COUNT(*) FROM db01.tbl01 USE INDEX();"
check_contains "${data_records}"
28 changes: 28 additions & 0 deletions cmd/explaintest/r/subquery.result
Original file line number Diff line number Diff line change
Expand Up @@ -46,3 +46,31 @@ create table t1(a int(11));
create table t2(a decimal(40,20) unsigned, b decimal(40,20));
select count(*) as x from t1 group by a having x not in (select a from t2 where x = t2.b);
x
drop table if exists stu;
drop table if exists exam;
create table stu(id int, name varchar(100));
insert into stu values(1, null);
create table exam(stu_id int, course varchar(100), grade int);
insert into exam values(1, 'math', 100);
set names utf8 collate utf8_general_ci;
explain format = 'brief' select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
id estRows task access object operator info
Apply 10000.00 root CARTESIAN anti semi join, other cond:eq(test.stu.name, Column#8)
├─TableReader(Build) 10000.00 root data:TableFullScan
│ └─TableFullScan 10000.00 cop[tikv] table:stu keep order:false, stats:pseudo
└─Projection(Probe) 10.00 root guo->Column#8
└─TableReader 10.00 root data:Selection
└─Selection 10.00 cop[tikv] eq(test.exam.stu_id, test.stu.id)
└─TableFullScan 10000.00 cop[tikv] table:exam keep order:false, stats:pseudo
select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
id name
set names utf8mb4;
explain format = 'brief' select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
id estRows task access object operator info
HashJoin 8000.00 root anti semi join, equal:[eq(test.stu.id, test.exam.stu_id)], other cond:eq(test.stu.name, "guo")
├─TableReader(Build) 10000.00 root data:TableFullScan
│ └─TableFullScan 10000.00 cop[tikv] table:exam keep order:false, stats:pseudo
└─TableReader(Probe) 10000.00 root data:TableFullScan
└─TableFullScan 10000.00 cop[tikv] table:stu keep order:false, stats:pseudo
select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
id name
13 changes: 13 additions & 0 deletions cmd/explaintest/t/subquery.test
Original file line number Diff line number Diff line change
Expand Up @@ -20,3 +20,16 @@ drop table if exists t1, t2;
create table t1(a int(11));
create table t2(a decimal(40,20) unsigned, b decimal(40,20));
select count(*) as x from t1 group by a having x not in (select a from t2 where x = t2.b);

drop table if exists stu;
drop table if exists exam;
create table stu(id int, name varchar(100));
insert into stu values(1, null);
create table exam(stu_id int, course varchar(100), grade int);
insert into exam values(1, 'math', 100);
set names utf8 collate utf8_general_ci;
explain format = 'brief' select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
set names utf8mb4;
explain format = 'brief' select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
4 changes: 4 additions & 0 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -844,6 +844,10 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
if tblInfo.Partition != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table"))
}

if modifyInfo.changingCol == nil {
changingColPos := &ast.ColumnPosition{Tp: ast.ColumnPositionNone}
Expand Down
24 changes: 24 additions & 0 deletions ddl/column_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -643,6 +643,30 @@ func TestColumnModifyingDefinition(t *testing.T) {
tk.MustGetErrCode("alter table test2 change c1 a1 bigint not null;", mysql.WarnDataTruncated)
}

func TestColumnModifyingDefaultValue(t *testing.T) {
store, clean := testkit.CreateMockStoreWithSchemaLease(t, columnModifyLease)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustExec("create table t (a int default 1);")
tk.MustExec("alter table t change a a int default 0.00;")
ret := tk.MustQuery("show create table t").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` int(11) DEFAULT '0'"))

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1.25);")
tk.MustExec("alter table t change a a int default 2.8;")
ret = tk.MustQuery("show create table t").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` int(11) DEFAULT '3'"))

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a float default 1.25);")
tk.MustExec("alter table t change a a float default '0012.32';")
ret = tk.MustQuery("show create table t").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` float DEFAULT '12.32'"))
}

func TestTransactionWithWriteOnlyColumn(t *testing.T) {
store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, columnModifyLease)
defer clean()
Expand Down
42 changes: 42 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"fmt"
"math/rand"
"strings"
"sync"
"sync/atomic"
"testing"
"time"
Expand Down Expand Up @@ -3669,3 +3670,44 @@ func TestDuplicatePartitionNames(t *testing.T) {
"(PARTITION `p2` VALUES IN (2),\n" +
" PARTITION `p3` VALUES IN (3))"))
}

func TestIssue40135Ver2(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
dom := domain.GetDomain(tk.Session())
originHook := dom.DDL().GetHook()
tk.MustExec("use test")

tk1 := testkit.NewTestKit(t, store)
tk1.MustExec("use test")

tk3 := testkit.NewTestKit(t, store)
tk3.MustExec("use test")

tk.MustExec("CREATE TABLE t40135 ( a int DEFAULT NULL, b varchar(32) DEFAULT 'md', index(a)) PARTITION BY HASH (a) PARTITIONS 6")
tk.MustExec("insert into t40135 values (1, 'md'), (2, 'ma'), (3, 'md'), (4, 'ma'), (5, 'md'), (6, 'ma')")
one := true
hook := &ddl.TestDDLCallback{Do: dom}
var checkErr error
var wg sync.WaitGroup
wg.Add(1)
hook.OnJobRunBeforeExported = func(job *model.Job) {
if job.SchemaState == model.StateDeleteOnly {
tk3.MustExec("delete from t40135 where a = 1")
}
if one {
one = false
go func() {
_, checkErr = tk1.Exec("alter table t40135 modify column a int NULL")
wg.Done()
}()
}
}
defer dom.DDL().SetHook(originHook)
dom.DDL().SetHook(hook)
tk.MustExec("alter table t40135 modify column a bigint NULL DEFAULT '6243108' FIRST")
wg.Wait()
require.ErrorContains(t, checkErr, "[ddl:8200]Unsupported modify column: table is partition table")
tk.MustExec("admin check table t40135")
}
49 changes: 49 additions & 0 deletions ddl/db_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -314,6 +314,55 @@ func TestCreateTableWithEnumCol(t *testing.T) {
tk.MustQuery("select * from t_enum").Check(testkit.Rows("c"))
}

func TestCreateTableWithIntegerColWithDefault(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
// It's for failure cases.
tk.MustExec("drop table if exists t1")
failedSQL := "create table t1 (a tinyint unsigned default -1.25);"
tk.MustGetErrCode(failedSQL, errno.ErrInvalidDefault)
failedSQL = "create table t1 (a tinyint default 999999999);"
tk.MustGetErrCode(failedSQL, errno.ErrInvalidDefault)

// It's for successful cases
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (a tinyint unsigned default 1.25);")
ret := tk.MustQuery("show create table t1").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` tinyint(3) unsigned DEFAULT '1'"))

tk.MustExec("drop table t1")
tk.MustExec("create table t1 (a smallint default -1.25);")
ret = tk.MustQuery("show create table t1").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` smallint(6) DEFAULT '-1'"))

tk.MustExec("drop table t1")
tk.MustExec("create table t1 (a mediumint default 2.8);")
ret = tk.MustQuery("show create table t1").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` mediumint(9) DEFAULT '3'"))

tk.MustExec("drop table t1")
tk.MustExec("create table t1 (a int default -2.8);")
ret = tk.MustQuery("show create table t1").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` int(11) DEFAULT '-3'"))

tk.MustExec("drop table t1")
tk.MustExec("create table t1 (a bigint unsigned default 0.0);")
ret = tk.MustQuery("show create table t1").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` bigint(20) unsigned DEFAULT '0'"))

tk.MustExec("drop table t1")
tk.MustExec("create table t1 (a float default '0012.43');")
ret = tk.MustQuery("show create table t1").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` float DEFAULT '12.43'"))

tk.MustExec("drop table t1")
tk.MustExec("create table t1 (a double default '12.4300');")
ret = tk.MustQuery("show create table t1").Rows()[0][1]
require.True(t, strings.Contains(ret.(string), "`a` double DEFAULT '12.43'"))
}

func TestAlterTableWithValidation(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
Expand Down
Loading

0 comments on commit 17a382c

Please sign in to comment.