Skip to content

Commit

Permalink
Merge branch 'master' into fix_cte_ilj_dead_lock
Browse files Browse the repository at this point in the history
  • Loading branch information
guo-shaoge authored Aug 23, 2021
2 parents 54e227c + 79aab25 commit 8c6827c
Show file tree
Hide file tree
Showing 31 changed files with 474 additions and 54 deletions.
32 changes: 32 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"fmt"
"os"
"strconv"
"strings"
"testing"
"time"

Expand Down Expand Up @@ -2453,6 +2454,37 @@ func (s *testSerialSuite) TestOptimizeOnlyOnce(c *C) {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/planner/checkOptimizeCountOne"), IsNil)
}

func (s *testSerialSuite) TestIssue26377(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("set tidb_enable_global_temporary_table = true")
tk.MustExec("set @@tidb_enable_noop_functions=1;")
tk.MustExec("drop table if exists t1,tmp1")
tk.MustExec("create table t1(a int(11))")
tk.MustExec("create global temporary table tmp1(a int(11), key idx_a(a)) on commit delete rows;")
tk.MustExec("create temporary table tmp2(a int(11), key idx_a(a));")

queries := []string{
"create global binding for select * from t1 inner join tmp1 on t1.a=tmp1.a using select * from t1 inner join tmp1 on t1.a=tmp1.a;",
"create global binding for select * from t1 where t1.a in (select a from tmp1) using select * from t1 where t1.a in (select a from tmp1 use index (idx_a));",
"create global binding for select a from t1 union select a from tmp1 using select a from t1 union select a from tmp1 use index (idx_a);",
"create global binding for select t1.a, (select a from tmp1 where tmp1.a=1) as t2 from t1 using select t1.a, (select a from tmp1 where tmp1.a=1) as t2 from t1;",
"create global binding for select * from (select * from tmp1) using select * from (select * from tmp1);",
}
genLocalTemporarySQL := func(sql string) string {
return strings.Replace(sql, "tmp1", "tmp2", -1)
}
for _, query := range queries {
localSQL := genLocalTemporarySQL(query)
queries = append(queries, localSQL)
}

for _, q := range queries {
tk.MustGetErrCode(q, errno.ErrOptOnTemporaryTable)
}
}

func (s *testSuite) TestCaptureFilter(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/lightning/backend/local/local.go
Original file line number Diff line number Diff line change
Expand Up @@ -2755,8 +2755,8 @@ func (w *Writer) appendRowsUnsorted(ctx context.Context, kvs []common.KvPair) er
l := len(w.writeBatch)
cnt := w.batchCount
var lastKey []byte
if len(w.writeBatch) > 0 {
lastKey = w.writeBatch[len(w.writeBatch)-1].Key
if cnt > 0 {
lastKey = w.writeBatch[cnt-1].Key
}
for _, pair := range kvs {
if w.isWriteBatchSorted && bytes.Compare(lastKey, pair.Key) > 0 {
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/lightning/backend/local/local_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -349,7 +349,7 @@ func testLocalWriter(c *C, needSort bool, partitialSort bool) {
f.wg.Add(1)
go f.ingestSSTLoop()
sorted := needSort && !partitialSort
w, err := openLocalWriter(context.Background(), &backend.LocalWriterConfig{IsKVSorted: sorted}, f, 1<<20)
w, err := openLocalWriter(context.Background(), &backend.LocalWriterConfig{IsKVSorted: sorted}, f, 1024)
c.Assert(err, IsNil)

ctx := context.Background()
Expand Down
5 changes: 3 additions & 2 deletions br/pkg/restore/backoff.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,8 @@ func (bo *importerBackoffer) NextBackoff(err error) time.Duration {
bo.delayTime = 2 * bo.delayTime
bo.attempt--
} else {
switch errors.Cause(err) { // nolint:errorlint
e := errors.Cause(err)
switch e { // nolint:errorlint
case berrors.ErrKVEpochNotMatch, berrors.ErrKVDownloadFailed, berrors.ErrKVIngestFailed:
bo.delayTime = 2 * bo.delayTime
bo.attempt--
Expand All @@ -65,7 +66,7 @@ func (bo *importerBackoffer) NextBackoff(err error) time.Duration {
bo.delayTime = 0
bo.attempt = 0
default:
switch status.Code(err) {
switch status.Code(e) {
case codes.Unavailable, codes.Aborted:
bo.delayTime = 2 * bo.delayTime
bo.attempt--
Expand Down
4 changes: 4 additions & 0 deletions br/pkg/restore/import.go
Original file line number Diff line number Diff line change
Expand Up @@ -317,6 +317,10 @@ func (importer *FileImporter) Import(
log.Debug("failpoint restore-storage-error injected.", zap.String("msg", msg))
e = errors.Annotate(e, msg)
})
failpoint.Inject("restore-gRPC-error", func(_ failpoint.Value) {
log.Warn("the connection to TiKV has been cut by a neko, meow :3")
e = status.Error(codes.Unavailable, "the connection to TiKV has been cut by a neko, meow :3")
})
if e != nil {
remainFiles = remainFiles[i:]
return errors.Trace(e)
Expand Down
2 changes: 1 addition & 1 deletion br/tests/br_full/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ for ct in limit lz4 zstd; do

# restore full
echo "restore with $ct backup start..."
export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/restore-storage-error=1*return(\"connection refused\")"
export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/restore-storage-error=1*return(\"connection refused\");github.com/pingcap/tidb/br/pkg/restore/restore-gRPC-error=1*return(true)"
run_br restore full -s "local://$TEST_DIR/$DB-$ct" --pd $PD_ADDR --ratelimit 1024
export GO_FAILPOINTS=""

Expand Down
13 changes: 13 additions & 0 deletions cmd/explaintest/r/explain_generate_column_substitute.result
Original file line number Diff line number Diff line change
Expand Up @@ -496,6 +496,19 @@ desc format = 'brief' select b from t;
id estRows task access object operator info
IndexReader 10000.00 root index:IndexFullScan
└─IndexFullScan 10000.00 cop[tikv] table:t, index:b(b) keep order:false, stats:pseudo
create table t01(a varchar(20));
insert into t01 values ("齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙");
alter table t01 add index eidx ((concat_ws('expression_index', a, 'test')));
select * from t01 use index (eidx) where (concat_ws('expression_index', a, 'test')) not like (concat_ws('expression_index', "齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙", 'test'));
a
insert into t01 values ("齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙");
select * from t01 use index (eidx) where (concat_ws('expression_index', a, 'test')) like (concat_ws('expression_index', "齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙", 'test'));
a
齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙
齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙
drop table if exists t1;
create table t1(a char, b varchar(20), c char, d varchar(20));
alter table t1 add index eidx ((export_set(3, a, c, ',', 5)));
create table t02 (a varchar(20));
insert into t02 values ('a'), ('b'), ('c');
select * from t02 where lower(a) < 'c';
Expand Down
10 changes: 10 additions & 0 deletions cmd/explaintest/t/explain_generate_column_substitute.test
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,16 @@ create table t(a int, b int as (a+1), key((a+1)), key(b));
desc format = 'brief' select a+1 from t;
desc format = 'brief' select b from t;

create table t01(a varchar(20));
insert into t01 values ("齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙");
alter table t01 add index eidx ((concat_ws('expression_index', a, 'test')));
select * from t01 use index (eidx) where (concat_ws('expression_index', a, 'test')) not like (concat_ws('expression_index', "齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙", 'test'));
insert into t01 values ("齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙");
select * from t01 use index (eidx) where (concat_ws('expression_index', a, 'test')) like (concat_ws('expression_index', "齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙", 'test'));

drop table if exists t1;
create table t1(a char, b varchar(20), c char, d varchar(20));
alter table t1 add index eidx ((export_set(3, a, c, ',', 5)));
create table t02 (a varchar(20));
insert into t02 values ('a'), ('b'), ('c');
select * from t02 where lower(a) < 'c';
Expand Down
3 changes: 3 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -548,6 +548,8 @@ type IsolationRead struct {
type Experimental struct {
// Whether enable global kill.
EnableGlobalKill bool `toml:"enable-global-kill" json:"-"`
// Whether enable charset feature.
EnableNewCharset bool `toml:"enable-new-charset" json:"-"`
}

var defTiKVCfg = tikvcfg.DefaultConfig()
Expand Down Expand Up @@ -670,6 +672,7 @@ var defaultConf = Config{
},
Experimental: Experimental{
EnableGlobalKill: false,
EnableNewCharset: false,
},
EnableCollectExecutionInfo: true,
EnableTelemetry: true,
Expand Down
1 change: 1 addition & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -286,6 +286,7 @@ deadlock-history-collect-retryable = true
require.Equal(t, uint64(30), conf.StoresRefreshInterval)
require.Equal(t, uint(123), conf.PessimisticTxn.DeadlockHistoryCapacity)
require.True(t, conf.PessimisticTxn.DeadlockHistoryCollectRetryable)
require.False(t, conf.Experimental.EnableNewCharset)

_, err = f.WriteString(`
[log.file]
Expand Down
2 changes: 2 additions & 0 deletions ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -295,4 +295,6 @@ var (
errFunctionalIndexOnJSONOrGeometryFunction = dbterror.ClassDDL.NewStd(mysql.ErrFunctionalIndexOnJSONOrGeometryFunction)
// errDependentByFunctionalIndex returns when the dropped column depends by expression index.
errDependentByFunctionalIndex = dbterror.ClassDDL.NewStd(mysql.ErrDependentByFunctionalIndex)
// errFunctionalIndexOnBlob when the expression of expression index returns blob or text.
errFunctionalIndexOnBlob = dbterror.ClassDDL.NewStd(mysql.ErrFunctionalIndexOnBlob)
)
3 changes: 3 additions & 0 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,9 @@ func checkIndexColumn(col *model.ColumnInfo, indexColumnLen int) error {
// Length must be specified and non-zero for BLOB and TEXT column indexes.
if types.IsTypeBlob(col.FieldType.Tp) {
if indexColumnLen == types.UnspecifiedLength {
if col.Hidden {
return errFunctionalIndexOnBlob
}
return errors.Trace(errBlobKeyWithoutLength.GenWithStackByArgs(col.Name.O))
}
if indexColumnLen == types.ErrorLength {
Expand Down
2 changes: 1 addition & 1 deletion errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -891,7 +891,7 @@ const (
ErrFunctionalIndexRefAutoIncrement = 3754
ErrCannotDropColumnFunctionalIndex = 3755
ErrFunctionalIndexPrimaryKey = 3756
ErrFunctionalIndexOnLob = 3757
ErrFunctionalIndexOnBlob = 3757
ErrFunctionalIndexFunctionIsNotAllowed = 3758
ErrFulltextFunctionalIndex = 3759
ErrSpatialFunctionalIndex = 3760
Expand Down
2 changes: 1 addition & 1 deletion errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -887,7 +887,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrFunctionalIndexRefAutoIncrement: mysql.Message("Expression index '%s' cannot refer to an auto-increment column", nil),
ErrCannotDropColumnFunctionalIndex: mysql.Message("Cannot drop column '%s' because it is used by an expression index. In order to drop the column, you must remove the expression index", nil),
ErrFunctionalIndexPrimaryKey: mysql.Message("The primary key cannot be an expression index", nil),
ErrFunctionalIndexOnLob: mysql.Message("Cannot create an expression index on an expression that returns a BLOB or TEXT. Please consider using CAST", nil),
ErrFunctionalIndexOnBlob: mysql.Message("Cannot create an expression index on an expression that returns a BLOB or TEXT. Please consider using CAST", nil),
ErrFunctionalIndexFunctionIsNotAllowed: mysql.Message("Expression of expression index '%s' contains a disallowed function", nil),
ErrFulltextFunctionalIndex: mysql.Message("Fulltext expression index is not supported", nil),
ErrSpatialFunctionalIndex: mysql.Message("Spatial expression index is not supported", nil),
Expand Down
85 changes: 83 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/deadlockhistory"
"github.com/pingcap/tidb/util/gcutil"
"github.com/pingcap/tidb/util/israce"
Expand Down Expand Up @@ -4979,6 +4980,16 @@ func (s *testSuiteP2) TestAddDateBuiltinWithWarnings(c *C) {
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Incorrect datetime value: '2001-01-00'"))
}

func (s *testSuiteP2) TestIssue27232(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a timestamp)")
tk.MustExec("insert into t values (\"1970-07-23 10:04:59\"), (\"2038-01-19 03:14:07\")")
tk.MustQuery("select * from t where date_sub(a, interval 10 month) = date_sub(\"1970-07-23 10:04:59\", interval 10 month)").Check(testkit.Rows("1970-07-23 10:04:59"))
tk.MustQuery("select * from t where timestampadd(hour, 1, a ) = timestampadd(hour, 1, \"2038-01-19 03:14:07\")").Check(testkit.Rows("2038-01-19 03:14:07"))
}

func (s *testSuiteP2) TestStrToDateBuiltinWithWarnings(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("set @@sql_mode='NO_ZERO_DATE'")
Expand Down Expand Up @@ -5627,6 +5638,72 @@ func (s *testSerialSuite2) TestUnsignedFeedback(c *C) {
c.Assert(result.Rows()[2][6], Equals, "range:[0,+inf], keep order:false")
}

func (s *testSerialSuite2) TestCharsetFeature(c *C) {
collate.SetCharsetFeatEnabledForTest(true)
defer collate.SetCharsetFeatEnabledForTest(false)

tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustQuery("show charset").Check(testkit.Rows(
"ascii US ASCII ascii_bin 1",
"binary binary binary 1",
"gbk Chinese Internal Code Specification gbk_bin 2",
"latin1 Latin1 latin1_bin 1",
"utf8 UTF-8 Unicode utf8_bin 3",
"utf8mb4 UTF-8 Unicode utf8mb4_bin 4",
))
tk.MustQuery("show collation").Check(testkit.Rows(
"ascii_bin ascii 65 Yes Yes 1",
"binary binary 63 Yes Yes 1",
"gbk_bin gbk 87 Yes 1",
"latin1_bin latin1 47 Yes Yes 1",
"utf8_bin utf8 83 Yes Yes 1",
"utf8_general_ci utf8 33 Yes 1",
"utf8_unicode_ci utf8 192 Yes 1",
"utf8mb4_bin utf8mb4 46 Yes Yes 1",
"utf8mb4_general_ci utf8mb4 45 Yes 1",
"utf8mb4_unicode_ci utf8mb4 224 Yes 1",
))

tk.MustExec("set names gbk;")
tk.MustQuery("select @@character_set_connection;").Check(testkit.Rows("gbk"))
tk.MustQuery("select @@collation_connection;").Check(testkit.Rows("gbk_bin"))
tk.MustExec("set @@character_set_client=gbk;")
tk.MustQuery("select @@character_set_client;").Check(testkit.Rows("gbk"))
tk.MustExec("set names utf8mb4;")
tk.MustExec("set @@character_set_connection=gbk;")
tk.MustQuery("select @@character_set_connection;").Check(testkit.Rows("gbk"))
tk.MustQuery("select @@collation_connection;").Check(testkit.Rows("gbk_bin"))

tk.MustQuery("select _gbk 'a'").Check(testkit.Rows("a"))

tk.MustExec("use test")
tk.MustExec("create table t1(a char(10) charset gbk);")
tk.MustExec("create table t2(a char(10) charset gbk collate gbk_bin);")
tk.MustExec("create table t3(a char(10)) charset gbk;")
tk.MustExec("alter table t3 add column b char(10) charset gbk;")
tk.MustQuery("show create table t3").Check(testkit.Rows("t3 CREATE TABLE `t3` (\n" +
" `a` char(10) DEFAULT NULL,\n" +
" `b` char(10) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_bin",
))
tk.MustExec("create table t4(a char(10));")
tk.MustExec("alter table t4 add column b char(10) charset gbk;")
tk.MustQuery("show create table t4").Check(testkit.Rows("t4 CREATE TABLE `t4` (\n" +
" `a` char(10) DEFAULT NULL,\n" +
" `b` char(10) CHARACTER SET gbk COLLATE gbk_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin",
))

tk.MustExec("create database test_gbk charset gbk;")
tk.MustExec("use test_gbk")
tk.MustExec("create table t1(a char(10));")
tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" +
" `a` char(10) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_bin",
))
}

func (s *testSerialSuite2) TestIssue23567(c *C) {
tk := testkit.NewTestKit(c, s.store)
oriProbability := statistics.FeedbackProbability.Load()
Expand Down Expand Up @@ -8546,6 +8623,9 @@ func (s testSerialSuite) TestTemporaryTableNoNetwork(c *C) {
}

func (s testSerialSuite) assertTemporaryTableNoNetwork(c *C, temporaryTableType model.TempTableType) {
var done sync.WaitGroup
defer done.Wait()

// Test that table reader/index reader/index lookup on the temporary table do not need to visit TiKV.
tk := testkit.NewTestKit(c, s.store)
tk1 := testkit.NewTestKit(c, s.store)
Expand Down Expand Up @@ -8579,10 +8659,11 @@ func (s testSerialSuite) assertTemporaryTableNoNetwork(c *C, temporaryTableType
c.Assert(err, IsNil)
blocked := make(chan struct{}, 1)
ctx, cancelFunc := context.WithCancel(context.Background())
done.Add(1)
go func() {
_, err := session.ResultSetToStringSlice(ctx, tk1.Se, rs)
defer done.Done()
_, _ = session.ResultSetToStringSlice(ctx, tk1.Se, rs)
blocked <- struct{}{}
c.Assert(err, NotNil)
}()
select {
case <-blocked:
Expand Down
32 changes: 32 additions & 0 deletions executor/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
parsertypes "github.com/pingcap/parser/types"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/infoschema"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/privilege/privileges"
"github.com/pingcap/tidb/session"
Expand Down Expand Up @@ -1044,6 +1045,37 @@ func (s *testAutoRandomSuite) TestAutoIdCache(c *C) {
))
}

func (s *testSuite5) TestShowCreateStmtIgnoreLocalTemporaryTables(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("set tidb_enable_noop_functions=true")

// SHOW CREATE VIEW ignores local temporary table with the same name
tk.MustExec("drop view if exists v1")
tk.MustExec("create view v1 as select 1")
tk.MustExec("create temporary table v1 (a int)")
tk.MustQuery("show create table v1").Check(testutil.RowsWithSep("|",
""+
"v1 CREATE TEMPORARY TABLE `v1` (\n"+
" `a` int(11) DEFAULT NULL\n"+
") ENGINE=memory DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin",
))
tk.MustExec("drop view v1")
err := tk.ExecToErr("show create view v1")
c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue)

// SHOW CREATE SEQUENCE ignores local temporary table with the same name
tk.MustExec("drop view if exists seq1")
tk.MustExec("create sequence seq1")
tk.MustExec("create temporary table seq1 (a int)")
tk.MustQuery("show create sequence seq1").Check(testutil.RowsWithSep("|",
"seq1 CREATE SEQUENCE `seq1` start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 cache 1000 nocycle ENGINE=InnoDB",
))
tk.MustExec("drop sequence seq1")
err = tk.ExecToErr("show create sequence seq1")
c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue)
}

func (s *testAutoRandomSuite) TestAutoRandomBase(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
Expand Down
Loading

0 comments on commit 8c6827c

Please sign in to comment.