Skip to content

Commit

Permalink
Merge branch 'master' into m/deadlocks-sql-text
Browse files Browse the repository at this point in the history
  • Loading branch information
MyonKeminta authored Aug 4, 2021
2 parents babc379 + 853c41e commit 1877f9f
Show file tree
Hide file tree
Showing 95 changed files with 2,286 additions and 970 deletions.
28 changes: 28 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2132,3 +2132,31 @@ func (s *testSuite) TestTemporaryTable(c *C) {
tk.MustGetErrCode("create binding for update t set a = 1 where b = 1 and c > 1 using update /*+ use_index(t, c) */ t set a = 1 where b = 1 and c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for delete from t where b = 1 and c > 1 using delete /*+ use_index(t, c) */ from t where b = 1 and c > 1", errno.ErrOptOnTemporaryTable)
}

func (s *testSuite) TestBindingLastUpdateTime(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("drop table if exists t0;")
tk.MustExec("create table t0(a int, key(a));")
tk.MustExec("create global binding for select * from t0 using select * from t0 use index(a);")
tk.MustExec("admin reload bindings;")

bindHandle := bindinfo.NewBindHandle(tk.Se)
err := bindHandle.Update(true)
c.Check(err, IsNil)
sql, hash := parser.NormalizeDigest("select * from test . t0")
bindData := bindHandle.GetBindRecord(hash.String(), sql, "test")
c.Assert(len(bindData.Bindings), Equals, 1)
bind := bindData.Bindings[0]
updateTime := bind.UpdateTime.String()

rows1 := tk.MustQuery("show status like 'last_plan_binding_update_time';").Rows()
updateTime1 := rows1[0][1]
c.Assert(updateTime1, Equals, updateTime)

rows2 := tk.MustQuery("show session status like 'last_plan_binding_update_time';").Rows()
updateTime2 := rows2[0][1]
c.Assert(updateTime2, Equals, updateTime)
tk.MustQuery(`show global status like 'last_plan_binding_update_time';`).Check(testkit.Rows())
}
1 change: 1 addition & 0 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,7 @@ func NewBindHandle(ctx sessionctx.Context) *BindHandle {
// BindSQL has already been validated when coming here, so we use nil sctx parameter.
return handle.AddBindRecord(nil, record)
}
variable.RegisterStatistics(handle)
return handle
}

Expand Down
39 changes: 39 additions & 0 deletions bindinfo/stat.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
// Copyright 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package bindinfo

import (
"github.com/pingcap/tidb/sessionctx/variable"
)

var (
lastPlanBindingUpdateTime = "last_plan_binding_update_time"
)

// GetScope gets the status variables scope.
func (h *BindHandle) GetScope(status string) variable.ScopeFlag {
return variable.ScopeSession
}

// Stats returns the server statistics.
func (h *BindHandle) Stats(vars *variable.SessionVars) (map[string]interface{}, error) {
h.bindInfo.Lock()
defer func() {
h.bindInfo.Unlock()
}()
m := make(map[string]interface{})
m[lastPlanBindingUpdateTime] = h.bindInfo.lastUpdateTime.String()

return m, nil
}
35 changes: 29 additions & 6 deletions cmd/explaintest/r/generated_columns.result
Original file line number Diff line number Diff line change
Expand Up @@ -105,14 +105,37 @@ PARTITION p5 VALUES LESS THAN (6),
PARTITION max VALUES LESS THAN MAXVALUE);
EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a <= 1;
id estRows task access object operator info
TableReader 3323.33 root partition:p0,p1 data:Selection
└─Selection 3323.33 cop[tikv] le(test.sgc3.a, 1)
└─TableFullScan 10000.00 cop[tikv] table:sgc3 keep order:false, stats:pseudo
PartitionUnion 6646.67 root
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] le(test.sgc3.a, 1)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p0 keep order:false, stats:pseudo
└─TableReader 3323.33 root data:Selection
└─Selection 3323.33 cop[tikv] le(test.sgc3.a, 1)
└─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p1 keep order:false, stats:pseudo
EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a < 7;
id estRows task access object operator info
TableReader 3323.33 root partition:all data:Selection
└─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
└─TableFullScan 10000.00 cop[tikv] table:sgc3 keep order:false, stats:pseudo
PartitionUnion 23263.33 root
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p0 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p1 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p2 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p3 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p4 keep order:false, stats:pseudo
├─TableReader 3323.33 root data:Selection
│ └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
│ └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:p5 keep order:false, stats:pseudo
└─TableReader 3323.33 root data:Selection
└─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7)
└─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:max keep order:false, stats:pseudo
DROP TABLE IF EXISTS t1;
CREATE TABLE t1(a INT, b INT AS (a+1) VIRTUAL, c INT AS (b+1) VIRTUAL, d INT AS (c+1) VIRTUAL, KEY(b), INDEX IDX(c, d));
INSERT INTO t1 (a) VALUES (0);
Expand Down
22 changes: 15 additions & 7 deletions cmd/explaintest/r/select.result
Original file line number Diff line number Diff line change
Expand Up @@ -359,17 +359,25 @@ insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8);
insert into th values (-1,-1),(-2,-2),(-3,-3),(-4,-4),(-5,-5),(-6,-6),(-7,-7),(-8,-8);
desc select * from th where a=-2;
id estRows task access object operator info
TableReader_7 10.00 root partition:p2 data:Selection_6
└─Selection_6 10.00 cop[tikv] eq(test.th.a, -2)
└─TableFullScan_5 10000.00 cop[tikv] table:th keep order:false, stats:pseudo
TableReader_9 10.00 root data:Selection_8
└─Selection_8 10.00 cop[tikv] eq(test.th.a, -2)
└─TableFullScan_7 10000.00 cop[tikv] table:th, partition:p2 keep order:false, stats:pseudo
desc select * from th;
id estRows task access object operator info
TableReader_5 10000.00 root partition:all data:TableFullScan_4
└─TableFullScan_4 10000.00 cop[tikv] table:th keep order:false, stats:pseudo
PartitionUnion_9 30000.00 root
├─TableReader_11 10000.00 root data:TableFullScan_10
│ └─TableFullScan_10 10000.00 cop[tikv] table:th, partition:p0 keep order:false, stats:pseudo
├─TableReader_13 10000.00 root data:TableFullScan_12
│ └─TableFullScan_12 10000.00 cop[tikv] table:th, partition:p1 keep order:false, stats:pseudo
└─TableReader_15 10000.00 root data:TableFullScan_14
└─TableFullScan_14 10000.00 cop[tikv] table:th, partition:p2 keep order:false, stats:pseudo
desc select * from th partition (p2,p1);
id estRows task access object operator info
TableReader_5 10000.00 root partition:p1,p2 data:TableFullScan_4
└─TableFullScan_4 10000.00 cop[tikv] table:th keep order:false, stats:pseudo
PartitionUnion_8 20000.00 root
├─TableReader_10 10000.00 root data:TableFullScan_9
│ └─TableFullScan_9 10000.00 cop[tikv] table:th, partition:p1 keep order:false, stats:pseudo
└─TableReader_12 10000.00 root data:TableFullScan_11
└─TableFullScan_11 10000.00 cop[tikv] table:th, partition:p2 keep order:false, stats:pseudo
drop table if exists t;
create table t(a int, b int);
explain format = 'brief' select a != any (select a from t t2) from t t1;
Expand Down
3 changes: 3 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -366,6 +366,8 @@ type Security struct {
SpilledFileEncryptionMethod string `toml:"spilled-file-encryption-method" json:"spilled-file-encryption-method"`
// EnableSEM prevents SUPER users from having full access.
EnableSEM bool `toml:"enable-sem" json:"enable-sem"`
// Allow automatic TLS certificate generation
AutoTLS bool `toml:"auto-tls" json:"auto-tls"`
}

// The ErrConfigValidationFailed error is used so that external callers can do a type assertion
Expand Down Expand Up @@ -674,6 +676,7 @@ var defaultConf = Config{
Security: Security{
SpilledFileEncryptionMethod: SpilledFileEncryptionMethodPlaintext,
EnableSEM: false,
AutoTLS: true,
},
DeprecateIntegerDisplayWidth: false,
EnableEnumLengthLimit: true,
Expand Down
3 changes: 3 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -205,6 +205,9 @@ spilled-file-encryption-method = "plaintext"
# Security Enhanced Mode (SEM) restricts the "SUPER" privilege and requires fine-grained privileges instead.
enable-sem = false

# Automatic creation of TLS certificates
auto-tls = true

[status]
# If enable status report HTTP service.
report-status = true
Expand Down
11 changes: 11 additions & 0 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
Expand Down Expand Up @@ -1871,3 +1872,13 @@ func (s *serialTestStateChangeSuite) TestCreateExpressionIndex(c *C) {
tk.MustExec("admin check table t")
tk.MustQuery("select * from t order by a, b").Check(testkit.Rows("0 9", "0 11", "0 11", "1 7", "2 7", "5 7", "8 8", "10 10", "10 10"))
}

func (s *testStateChangeSuite) TestExpressionIndexDDLError(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test_db_state")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, index idx((a+b)))")
tk.MustGetErrCode("alter table t rename column b to b2", errno.ErrDependentByFunctionalIndex)
tk.MustGetErrCode("alter table t drop column b", errno.ErrDependentByFunctionalIndex)
tk.MustExec("drop table t")
}
97 changes: 97 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3134,3 +3134,100 @@ func (s *testIntegrationSuite3) TestDropTemporaryTable(c *C) {
c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.a_local_temp_table_9_not_exist'")
tk.MustQuery("select * from a_local_temp_table_8").Check(testkit.Rows())
}

func (s *testIntegrationSuite3) TestTruncateLocalTemporaryTable(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("set @@tidb_enable_noop_functions = 1")

tk.MustExec("drop table if exists t1, tn")
tk.MustExec("create table t1 (id int)")
tk.MustExec("create table tn (id int)")
tk.MustExec("insert into t1 values(10), (11), (12)")
tk.MustExec("create temporary table t1 (id int primary key auto_increment)")
tk.MustExec("create temporary table t2 (id int primary key)")
tk.MustExec("create database test2")
tk.MustExec("create temporary table test2.t2 (id int)")

// truncate table out of txn
tk.MustExec("insert into t1 values(1), (2), (3)")
tk.MustExec("insert into t2 values(4), (5), (6)")
tk.MustExec("insert into test2.t2 values(7), (8), (9)")
tk.MustExec("truncate table t1")
tk.MustQuery("select * from t1").Check(testkit.Rows())
tk.MustExec("insert into t1 values()")
// auto_increment will be reset for truncate
tk.MustQuery("select * from t1").Check(testkit.Rows("1"))
tk.MustQuery("select * from t2").Check(testkit.Rows("4", "5", "6"))
tk.MustExec("truncate table t2")
tk.MustQuery("select * from t2").Check(testkit.Rows())
tk.MustQuery("select * from test2.t2").Check(testkit.Rows("7", "8", "9"))
tk.MustExec("drop table t1")
tk.MustQuery("select * from t1").Check(testkit.Rows("10", "11", "12"))
tk.MustExec("create temporary table t1 (id int primary key auto_increment)")

// truncate table with format dbName.tableName
tk.MustExec("insert into t2 values(4), (5), (6)")
tk.MustExec("insert into test2.t2 values(7), (8), (9)")
tk.MustExec("truncate table test2.t2")
tk.MustQuery("select * from test2.t2").Check(testkit.Rows())
tk.MustQuery("select * from t2").Check(testkit.Rows("4", "5", "6"))
tk.MustExec("truncate table test.t2")
tk.MustQuery("select * from t2").Check(testkit.Rows())

// truncate table in txn
tk.MustExec("insert into t1 values(1), (2), (3)")
tk.MustExec("insert into t2 values(4), (5), (6)")
tk.MustExec("begin")
tk.MustExec("insert into t1 values(11), (12)")
tk.MustExec("insert into t2 values(24), (25)")
tk.MustExec("delete from t1 where id=2")
tk.MustExec("delete from t2 where id=4")
tk.MustExec("truncate table t1")
tk.MustQuery("select * from t1").Check(testkit.Rows())
tk.MustExec("insert into t1 values()")
// auto_increment will be reset for truncate
tk.MustQuery("select * from t1").Check(testkit.Rows("1"))
tk.MustQuery("select * from t2").Check(testkit.Rows("5", "6", "24", "25"))

// since transaction already committed by truncate, so query after rollback will get same result
tk.MustExec("rollback")
tk.MustQuery("select * from t1").Check(testkit.Rows("1"))
tk.MustQuery("select * from t2").Check(testkit.Rows("5", "6", "24", "25"))

// truncate a temporary table will not effect the normal table with the same name
tk.MustExec("drop table t1")
tk.MustQuery("select * from t1").Check(testkit.Rows("10", "11", "12"))
tk.MustExec("create temporary table t1 (id int primary key auto_increment)")

// truncate temporary table will clear session data
localTemporaryTables := tk.Se.GetSessionVars().LocalTemporaryTables.(*infoschema.LocalTemporaryTables)
tb1, exist := localTemporaryTables.TableByName(model.NewCIStr("test"), model.NewCIStr("t1"))
tbl1Info := tb1.Meta()
tablePrefix := tablecodec.EncodeTablePrefix(tbl1Info.ID)
endTablePrefix := tablecodec.EncodeTablePrefix(tbl1Info.ID + 1)
c.Assert(exist, IsTrue)
tk.MustExec("insert into t1 values(1), (2), (3)")
tk.MustExec("begin")
tk.MustExec("insert into t1 values(5), (6), (7)")
tk.MustExec("truncate table t1")
iter, err := tk.Se.GetSessionVars().TemporaryTableData.Iter(tablePrefix, endTablePrefix)
c.Assert(err, IsNil)
for iter.Valid() {
key := iter.Key()
if !bytes.HasPrefix(key, tablePrefix) {
break
}
value := iter.Value()
c.Assert(len(value), Equals, 0)
_ = iter.Next()
}
c.Assert(iter.Valid(), IsFalse)

// truncate after drop database should be successful
tk.MustExec("create temporary table test2.t3 (id int)")
tk.MustExec("insert into test2.t3 values(1)")
tk.MustExec("drop database test2")
tk.MustExec("truncate table test2.t3")
tk.MustQuery("select * from test2.t3").Check(testkit.Rows())
}
2 changes: 1 addition & 1 deletion ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2520,7 +2520,7 @@ func (s *testDBSuite5) TestRenameColumn(c *C) {
assertColNames("test_rename_column", "id", "col2")
s.mustExec(tk, c, "alter table test_rename_column rename column col2 to col1")
assertColNames("test_rename_column", "id", "col1")
tk.MustGetErrCode("alter table test_rename_column rename column id to id1", errno.ErrBadField)
tk.MustGetErrCode("alter table test_rename_column rename column id to id1", errno.ErrDependentByGeneratedColumn)

// Test renaming view columns.
tk.MustExec("drop table test_rename_column")
Expand Down
10 changes: 8 additions & 2 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -4207,7 +4207,10 @@ func (d *ddl) RenameColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al
dependedColNames := findColumnNamesInExpr(col.GeneratedExpr)
for _, name := range dependedColNames {
if name.Name.L == oldColName.L {
return ErrBadField.GenWithStackByArgs(oldColName.O, "generated column function")
if col.Hidden {
return errDependentByFunctionalIndex.GenWithStackByArgs(oldColName.O)
}
return errDependentByGeneratedColumn.GenWithStackByArgs(oldColName.O)
}
}
}
Expand Down Expand Up @@ -5439,7 +5442,10 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI
}

func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error {
if ok, dep := hasDependentByGeneratedColumn(tblInfo, colName); ok {
if ok, dep, isHidden := hasDependentByGeneratedColumn(tblInfo, colName); ok {
if isHidden {
return errDependentByFunctionalIndex.GenWithStackByArgs(dep)
}
return errDependentByGeneratedColumn.GenWithStackByArgs(dep)
}

Expand Down
8 changes: 6 additions & 2 deletions ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -286,8 +286,12 @@ var (
errUnsupportedEngineTemporary = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support this kind of engine for temporary table", nil))
errUnsupportedClusteredSecondaryKey = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("CLUSTERED/NONCLUSTERED keyword is only supported for primary key", nil))

// ErrUnsupportedLocalTempTableDDL returns when ddl operation unsupported for local temporary table
ErrUnsupportedLocalTempTableDDL = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support %s for local temporary table", nil))
// ErrInvalidAttributesSpec is returned when meeting invalid attributes.
ErrInvalidAttributesSpec = dbterror.ClassDDL.NewStd(mysql.ErrInvalidAttributesSpec)
// ErrFunctionalIndexOnJSONOrGeometryFunction returns when creating expression index and the type of the expression is JSON.
ErrFunctionalIndexOnJSONOrGeometryFunction = dbterror.ClassDDL.NewStd(mysql.ErrFunctionalIndexOnJSONOrGeometryFunction)
// errFunctionalIndexOnJSONOrGeometryFunction returns when creating expression index and the type of the expression is JSON.
errFunctionalIndexOnJSONOrGeometryFunction = dbterror.ClassDDL.NewStd(mysql.ErrFunctionalIndexOnJSONOrGeometryFunction)
// errDependentByFunctionalIndex returns when the dropped column depends by expression index.
errDependentByFunctionalIndex = dbterror.ClassDDL.NewStd(mysql.ErrDependentByFunctionalIndex)
)
8 changes: 4 additions & 4 deletions ddl/generated_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,15 +141,15 @@ func findColumnNamesInExpr(expr ast.ExprNode) []*ast.ColumnName {
}

// hasDependentByGeneratedColumn checks whether there are other columns depend on this column or not.
func hasDependentByGeneratedColumn(tblInfo *model.TableInfo, colName model.CIStr) (bool, string) {
func hasDependentByGeneratedColumn(tblInfo *model.TableInfo, colName model.CIStr) (bool, string, bool) {
for _, col := range tblInfo.Columns {
for dep := range col.Dependences {
if dep == colName.L {
return true, dep
return true, dep, col.Hidden
}
}
}
return false, ""
return false, "", false
}

func isGeneratedRelatedColumn(tblInfo *model.TableInfo, newCol, col *model.ColumnInfo) error {
Expand All @@ -158,7 +158,7 @@ func isGeneratedRelatedColumn(tblInfo *model.TableInfo, newCol, col *model.Colum
msg := fmt.Sprintf("newCol IsGenerated %v, oldCol IsGenerated %v", newCol.IsGenerated(), col.IsGenerated())
return errUnsupportedModifyColumn.GenWithStackByArgs(msg)
}
if ok, dep := hasDependentByGeneratedColumn(tblInfo, col.Name); ok {
if ok, dep, _ := hasDependentByGeneratedColumn(tblInfo, col.Name); ok {
msg := fmt.Sprintf("oldCol is a dependent column '%s' for generated column", dep)
return errUnsupportedModifyColumn.GenWithStackByArgs(msg)
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ func checkIndexColumn(col *model.ColumnInfo, indexColumnLen int) error {
// JSON column cannot index.
if col.FieldType.Tp == mysql.TypeJSON {
if col.Hidden {
return ErrFunctionalIndexOnJSONOrGeometryFunction
return errFunctionalIndexOnJSONOrGeometryFunction
}
return errors.Trace(errJSONUsedAsKey.GenWithStackByArgs(col.Name.O))
}
Expand Down
Loading

0 comments on commit 1877f9f

Please sign in to comment.