Skip to content

Commit

Permalink
Merge branch 'master' into refine_hashjoin
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Aug 3, 2022
2 parents 4f908ee + 6a6fa8a commit 9e61c37
Show file tree
Hide file tree
Showing 22 changed files with 201 additions and 229 deletions.
1 change: 0 additions & 1 deletion ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,6 @@ go_test(
"placement_policy_test.go",
"placement_sql_test.go",
"primary_key_handle_test.go",
"reorg_test.go",
"repair_table_test.go",
"restart_test.go",
"rollingback_test.go",
Expand Down
9 changes: 9 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1995,6 +1995,11 @@ func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo
if err := checkGeneratedColumn(ctx, s.Cols); err != nil {
return errors.Trace(err)
}

// Check if table has a primary key if required.
if !ctx.GetSessionVars().InRestrictedSQL && ctx.GetSessionVars().PrimaryKeyRequired && len(tbInfo.GetPkName().String()) == 0 {
return infoschema.ErrTableWithoutPrimaryKey
}
if tbInfo.Partition != nil {
if err := checkPartitionDefinitionConstraints(ctx, tbInfo); err != nil {
return errors.Trace(err)
Expand Down Expand Up @@ -6135,6 +6140,10 @@ func (d *ddl) dropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI
return err
}

if !ctx.GetSessionVars().InRestrictedSQL && ctx.GetSessionVars().PrimaryKeyRequired && isPK {
return infoschema.ErrTableWithoutPrimaryKey
}

if indexInfo == nil {
err = dbterror.ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName)
if ifExists {
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -402,7 +402,7 @@ func (d *ddl) addBatchDDLJobs2Table(tasks []*limitJobTask) error {
sess, err1 := d.sessPool.get()
if err1 == nil {
sess.SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull)
err1 = insertDDLJobs2Table(newSession(sess), jobTasks, true)
err1 = insertDDLJobs2Table(newSession(sess), true, jobTasks...)
d.sessPool.put(sess)
}
err = err1
Expand Down
30 changes: 19 additions & 11 deletions ddl/job_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,7 @@ const (
updateDDLJobSQL = "update mysql.tidb_ddl_job set job_meta = %s where job_id = %d"
)

func insertDDLJobs2Table(sess *session, jobs []*model.Job, updateRawArgs bool) error {
func insertDDLJobs2Table(sess *session, updateRawArgs bool, jobs ...*model.Job) error {
failpoint.Inject("mockAddBatchDDLJobsErr", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(errors.Errorf("mockAddBatchDDLJobsErr"))
Expand Down Expand Up @@ -432,7 +432,7 @@ func getJobsBySQL(sess *session, tbl, condition string) ([]*model.Job, error) {
}

// MoveJobFromQueue2Table move existing DDLs in queue to table.
func (d *ddl) MoveJobFromQueue2Table(force bool) error {
func (d *ddl) MoveJobFromQueue2Table(inBootstrap bool) error {
sess, err := d.sessPool.get()
if err != nil {
return err
Expand All @@ -445,7 +445,11 @@ func (d *ddl) MoveJobFromQueue2Table(force bool) error {
}
t := meta.NewMeta(txn)
isConcurrentDDL, err := t.IsConcurrentDDL()
if !force && (isConcurrentDDL || err != nil) {
if !inBootstrap && (isConcurrentDDL || err != nil) {
return errors.Trace(err)
}
systemDBID, err := t.GetSystemDBID()
if err != nil {
return errors.Trace(err)
}
for _, tp := range []workerType{addIdxWorker, generalWorker} {
Expand All @@ -454,15 +458,19 @@ func (d *ddl) MoveJobFromQueue2Table(force bool) error {
if err != nil {
return errors.Trace(err)
}
err = insertDDLJobs2Table(se, jobs, false)
if err != nil {
return errors.Trace(err)
}
if tp == generalWorker {
// general job do not have reorg info.
continue
}
for _, job := range jobs {
// In bootstrap, we can ignore the internal DDL.
if inBootstrap && job.SchemaID == systemDBID {
continue
}
err = insertDDLJobs2Table(se, false, job)
if err != nil {
return errors.Trace(err)
}
if tp == generalWorker {
// General job do not have reorg info.
continue
}
element, start, end, pid, err := t.GetDDLReorgHandle(job)
if meta.ErrDDLReorgElementNotExist.Equal(err) {
continue
Expand Down
102 changes: 0 additions & 102 deletions ddl/reorg_test.go

This file was deleted.

1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -912,6 +912,7 @@ const (
ErrFunctionalIndexDataIsTooLong = 3907
ErrFunctionalIndexNotApplicable = 3909
ErrDynamicPrivilegeNotRegistered = 3929
ErrTableWithoutPrimaryKey = 3750
// MariaDB errors.
ErrOnlyOneDefaultPartionAllowed = 4030
ErrWrongPartitionTypeExpectedSystemTime = 4113
Expand Down
1 change: 1 addition & 0 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -913,6 +913,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrCTERecursiveForbiddenJoinOrder: mysql.Message("In recursive query block of Recursive Common Table Expression '%s', the recursive table must neither be in the right argument of a LEFT JOIN, nor be forced to be non-first with join order hints", nil),
ErrInvalidRequiresSingleReference: mysql.Message("In recursive query block of Recursive Common Table Expression '%s', the recursive table must be referenced only once, and not in any subquery", nil),
ErrCTEMaxRecursionDepth: mysql.Message("Recursive query aborted after %d iterations. Try increasing @@cte_max_recursion_depth to a larger value", nil),
ErrTableWithoutPrimaryKey: mysql.Message("Unable to create or change a table without a primary key, when the system variable 'sql_require_primary_key' is set. Add a primary key to the table or unset this variable to avoid this message. Note that tables without a primary key can cause performance problems in row-based replication, so please consult your DBA before changing this setting.", nil),
// MariaDB errors.
ErrOnlyOneDefaultPartionAllowed: mysql.Message("Only one DEFAULT partition allowed", nil),
ErrWrongPartitionTypeExpectedSystemTime: mysql.Message("Wrong partitioning type, expected type: `SYSTEM_TIME`", nil),
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -2256,6 +2256,11 @@ error = '''
User %s already exists.
'''

["schema:3750"]
error = '''
Unable to create or change a table without a primary key, when the system variable 'sql_require_primary_key' is set. Add a primary key to the table or unset this variable to avoid this message. Note that tables without a primary key can cause performance problems in row-based replication, so please consult your DBA before changing this setting.
'''

["schema:4139"]
error = '''
Unknown SEQUENCE: '%-.300s'
Expand Down
53 changes: 53 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6531,6 +6531,59 @@ func TestIssue24953(t *testing.T) {
tk.MustQuery("(select col_76,col_1,col_143,col_2 from tbl_0) union (select col_54,col_57,col_55,col_56 from tbl_9);").Check(testkit.Rows("-5765442 ZdfkUJiHcOfi -597990898 384599625723370089"))
}

// issue https://github.com/pingcap/tidb/issues/28544
func TestPrimaryKeyRequiredSysvar(t *testing.T) {
store := testkit.CreateMockStore(t)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`CREATE TABLE t (
name varchar(60),
age int
)`)
tk.MustExec(`DROP TABLE t`)

tk.MustExec("set @@sql_require_primary_key=true")

// creating table without primary key should now fail
tk.MustGetErrCode(`CREATE TABLE t (
name varchar(60),
age int
)`, errno.ErrTableWithoutPrimaryKey)
// but with primary key should work as usual
tk.MustExec(`CREATE TABLE t (
id bigint(20) NOT NULL PRIMARY KEY AUTO_RANDOM,
name varchar(60),
age int
)`)
tk.MustGetErrMsg(`ALTER TABLE t
DROP COLUMN id`, "[ddl:8200]Unsupported drop integer primary key")

// test with non-clustered primary key
tk.MustExec(`CREATE TABLE t2 (
id int(11) NOT NULL,
c1 int(11) DEFAULT NULL,
PRIMARY KEY(id) NONCLUSTERED)`)
tk.MustGetErrMsg(`ALTER TABLE t2
DROP COLUMN id`, "[ddl:8200]can't drop column id with composite index covered or Primary Key covered now")
tk.MustGetErrCode(`ALTER TABLE t2 DROP PRIMARY KEY`, errno.ErrTableWithoutPrimaryKey)

// this sysvar is ignored in internal sessions
tk.Session().GetSessionVars().InRestrictedSQL = true
ctx := context.Background()
ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnOthers)
sql := `CREATE TABLE t3 (
id int(11) NOT NULL,
c1 int(11) DEFAULT NULL)`
stmts, err := tk.Session().Parse(ctx, sql)
require.NoError(t, err)
res, err := tk.Session().ExecuteStmt(ctx, stmts[0])
require.NoError(t, err)
if res != nil {
require.NoError(t, res.Close())
}
}

// issue https://github.com/pingcap/tidb/issues/26111
func TestRailsFKUsage(t *testing.T) {
store := testkit.CreateMockStore(t)
Expand Down
2 changes: 2 additions & 0 deletions infoschema/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,4 +82,6 @@ var (
ErrEmptyDatabase = dbterror.ClassSchema.NewStd(mysql.ErrBadDB)
// ErrForbidSchemaChange returns when the schema change is illegal
ErrForbidSchemaChange = dbterror.ClassSchema.NewStd(mysql.ErrForbidSchemaChange)
// ErrTableWithoutPrimaryKey returns when there is no primary key on a table and sql_require_primary_key is set
ErrTableWithoutPrimaryKey = dbterror.ClassSchema.NewStd(mysql.ErrTableWithoutPrimaryKey)
)
27 changes: 18 additions & 9 deletions meta/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -544,17 +544,12 @@ func (m *Meta) SetDDLTables() error {

// CreateMySQLDatabaseIfNotExists creates mysql schema and return its DB ID.
func (m *Meta) CreateMySQLDatabaseIfNotExists() (int64, error) {
dbs, err := m.ListDatabases()
if err != nil {
return 0, err
}
for _, db := range dbs {
if db.Name.L == mysql.SystemDB {
return db.ID, nil
}
id, err := m.GetSystemDBID()
if id != 0 || err != nil {
return id, err
}

id, err := m.GenGlobalID()
id, err = m.GenGlobalID()
if err != nil {
return 0, errors.Trace(err)
}
Expand All @@ -569,6 +564,20 @@ func (m *Meta) CreateMySQLDatabaseIfNotExists() (int64, error) {
return db.ID, err
}

// GetSystemDBID gets the system DB ID. return (0, nil) indicates that the system DB does not exist.
func (m *Meta) GetSystemDBID() (int64, error) {
dbs, err := m.ListDatabases()
if err != nil {
return 0, err
}
for _, db := range dbs {
if db.Name.L == mysql.SystemDB {
return db.ID, nil
}
}
return 0, nil
}

// CheckDDLTableExists check if the tables related to concurrent DDL exists.
func (m *Meta) CheckDDLTableExists() (bool, error) {
v, err := m.txn.Get(mDDLTableVersion)
Expand Down
4 changes: 2 additions & 2 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -631,15 +631,15 @@ func (e *Explain) prepareSchema() error {
e.Format = types.ExplainFormatROW
}
switch {
case (format == types.ExplainFormatROW && (!e.Analyze && e.RuntimeStatsColl == nil)) || (format == types.ExplainFormatBrief):
case (format == types.ExplainFormatROW || format == types.ExplainFormatBrief) && (!e.Analyze && e.RuntimeStatsColl == nil):
fieldNames = []string{"id", "estRows", "task", "access object", "operator info"}
case format == types.ExplainFormatVerbose || format == types.ExplainFormatTrueCardCost:
if e.Analyze || e.RuntimeStatsColl != nil {
fieldNames = []string{"id", "estRows", "estCost", "actRows", "task", "access object", "execution info", "operator info", "memory", "disk"}
} else {
fieldNames = []string{"id", "estRows", "estCost", "task", "access object", "operator info"}
}
case format == types.ExplainFormatROW && (e.Analyze || e.RuntimeStatsColl != nil):
case (format == types.ExplainFormatROW || format == types.ExplainFormatBrief) && (e.Analyze || e.RuntimeStatsColl != nil):
fieldNames = []string{"id", "estRows", "actRows", "task", "access object", "execution info", "operator info", "memory", "disk"}
case format == types.ExplainFormatDOT:
fieldNames = []string{"dot contents"}
Expand Down
23 changes: 23 additions & 0 deletions planner/core/plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -710,6 +710,29 @@ func BenchmarkEncodeFlatPlan(b *testing.B) {
}
}

func TestIssue35090(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
tk.MustExec("drop table if exists p, t;")
tk.MustExec("create table p (id int, c int, key i_id(id), key i_c(c));")
tk.MustExec("create table t (id int);")
tk.MustExec("insert into p values (3,3), (4,4), (6,6), (9,9);")
tk.MustExec("insert into t values (4), (9);")
tk.MustExec("select /*+ INL_JOIN(p) */ * from p, t where p.id = t.id;")
rows := [][]interface{}{
{"IndexJoin"},
{"├─TableReader(Build)"},
{"│ └─Selection"},
{"│ └─TableFullScan"},
{"└─IndexLookUp(Probe)"},
{" ├─Selection(Build)"},
{" │ └─IndexRangeScan"},
{" └─TableRowIDScan(Probe)"},
}
tk.MustQuery("explain analyze format='brief' select /*+ INL_JOIN(p) */ * from p, t where p.id = t.id;").CheckAt([]int{0}, rows)
}

// Close issue 25729
func TestIssue25729(t *testing.T) {
config.UpdateGlobal(func(conf *config.Config) {
Expand Down
2 changes: 1 addition & 1 deletion planner/core/testdata/expression_rewriter_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -14,4 +14,4 @@
"select bit_or(a * b) from t"
]
}
]
]
Loading

0 comments on commit 9e61c37

Please sign in to comment.