From ce2af881dad7ff1701d92204fcdcc62ed187e97b Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 10 Aug 2023 15:30:27 +0100 Subject: [PATCH] This is an automated cherry-pick of #45877 Signed-off-by: ti-chi-bot --- ddl/db_partition_test.go | 231 +++ ddl/ddl_api.go | 1 - ddl/ddl_worker.go | 29 +- ddl/failtest/fail_db_test.go | 4 + ddl/partition.go | 791 +++++++++- ddl/placement_policy_test.go | 176 ++- ddl/rollingback.go | 84 ++ executor/insert_common.go | 27 + executor/write.go | 27 + infoschema/builder.go | 73 +- parser/model/model.go | 9 + table/tables/partition_test.go | 2534 ++++++++++++++++++++++++++++++++ 12 files changed, 3939 insertions(+), 47 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 0d822ad96b981..7f80110470fe4 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -2098,8 +2098,239 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { c.Assert(err, IsNil) } +<<<<<<< HEAD func (s *testSerialDBSuite1) TestExchangePartitionExpressIndex(c *C) { tk := testkit.NewTestKit(c, s.store) +======= +func TestExchangePartitionMultiTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk1 := testkit.NewTestKit(t, store) + + dbName := "ExchangeMultiTable" + tk1.MustExec(`create schema ` + dbName) + tk1.MustExec(`use ` + dbName) + tk1.MustExec(`CREATE TABLE t1 (a int)`) + tk1.MustExec(`CREATE TABLE t2 (a int)`) + tk1.MustExec(`CREATE TABLE tp (a int) partition by hash(a) partitions 3`) + tk1.MustExec(`insert into t1 values (0)`) + tk1.MustExec(`insert into t2 values (3)`) + tk1.MustExec(`insert into tp values (6)`) + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec(`use ` + dbName) + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec(`use ` + dbName) + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec(`use ` + dbName) + waitFor := func(col int, tableName, s string) { + for { + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec(`use test`) + sql := `admin show ddl jobs where db_name = '` + strings.ToLower(dbName) + `' and table_name = '` + tableName + `' and job_type = 'exchange partition'` + res := tk4.MustQuery(sql).Rows() + if len(res) == 1 && res[0][col] == s { + break + } + time.Sleep(10 * time.Millisecond) + } + } + alterChan1 := make(chan error) + alterChan2 := make(chan error) + tk3.MustExec(`BEGIN`) + tk3.MustExec(`insert into tp values (1)`) + go func() { + alterChan1 <- tk1.ExecToErr(`alter table tp exchange partition p0 with table t1`) + }() + waitFor(11, "t1", "running") + go func() { + alterChan2 <- tk2.ExecToErr(`alter table tp exchange partition p0 with table t2`) + }() + waitFor(11, "t2", "queueing") + tk3.MustExec(`rollback`) + require.NoError(t, <-alterChan1) + err := <-alterChan2 + tk3.MustQuery(`select * from t1`).Check(testkit.Rows("6")) + tk3.MustQuery(`select * from t2`).Check(testkit.Rows("0")) + tk3.MustQuery(`select * from tp`).Check(testkit.Rows("3")) + require.NoError(t, err) +} + +func TestExchangePartitionValidation(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + dbName := "ExchangeValidation" + tk.MustExec(`create schema ` + dbName) + tk.MustExec(`use ` + dbName) + tk.MustExec(`CREATE TABLE t1 ( + d date NOT NULL , + name varchar(10) NOT NULL, + UNIQUE KEY (d,name))`) + + tk.MustExec(`CREATE TABLE t1p ( + d date NOT NULL , + name varchar(10) NOT NULL, + UNIQUE KEY (d,name) + ) + PARTITION BY RANGE COLUMNS(d) + (PARTITION p202307 VALUES LESS THAN ('2023-08-01'), + PARTITION p202308 VALUES LESS THAN ('2023-09-01'), + PARTITION p202309 VALUES LESS THAN ('2023-10-01'), + PARTITION p202310 VALUES LESS THAN ('2023-11-01'), + PARTITION p202311 VALUES LESS THAN ('2023-12-01'), + PARTITION p202312 VALUES LESS THAN ('2024-01-01'), + PARTITION pfuture VALUES LESS THAN (MAXVALUE))`) + + tk.MustExec(`insert into t1 values ("2023-08-06","0000")`) + tk.MustContainErrMsg(`alter table t1p exchange partition p202307 with table t1 with validation`, + "[ddl:1737]Found a row that does not match the partition") + tk.MustExec(`insert into t1 values ("2023-08-06","0001")`) +} + +func TestExchangePartitionPlacementPolicy(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec(`create schema ExchangePartWithPolicy`) + tk.MustExec(`use ExchangePartWithPolicy`) + tk.MustExec(`CREATE PLACEMENT POLICY rule1 FOLLOWERS=1`) + tk.MustExec(`CREATE PLACEMENT POLICY rule2 FOLLOWERS=2`) + tk.MustExec(`CREATE TABLE t1 ( + d date NOT NULL , + name varchar(10) NOT NULL, + UNIQUE KEY (d,name) + ) PLACEMENT POLICY="rule1"`) + + tk.MustExec(`CREATE TABLE t1p ( + d date NOT NULL , + name varchar(10) NOT NULL, + UNIQUE KEY (d,name) + ) PLACEMENT POLICY="rule2" + PARTITION BY RANGE COLUMNS(d) + (PARTITION p202307 VALUES LESS THAN ('2023-08-01'), + PARTITION p202308 VALUES LESS THAN ('2023-09-01'), + PARTITION p202309 VALUES LESS THAN ('2023-10-01'), + PARTITION p202310 VALUES LESS THAN ('2023-11-01'), + PARTITION p202311 VALUES LESS THAN ('2023-12-01'), + PARTITION p202312 VALUES LESS THAN ('2024-01-01'), + PARTITION pfuture VALUES LESS THAN (MAXVALUE))`) + + tk.MustContainErrMsg(`alter table t1p exchange partition p202307 with table t1`, + "[ddl:1736]Tables have different definitions") + tk.MustExec(`insert into t1 values ("2023-08-06","0000")`) +} + +func TestExchangePartitionHook(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + // why use tkCancel, not tk. + tkCancel := testkit.NewTestKit(t, store) + + tk.MustExec("set @@tidb_enable_exchange_partition=1") + defer tk.MustExec("set @@tidb_enable_exchange_partition=0") + + tk.MustExec("use test") + tk.MustExec(`create table pt (a int) partition by range(a) ( + partition p0 values less than (3), + partition p1 values less than (6), + PARTITION p2 VALUES LESS THAN (9), + PARTITION p3 VALUES LESS THAN (MAXVALUE) + );`) + tk.MustExec(`create table nt(a int);`) + + tk.MustExec(`insert into pt values (0), (4), (7)`) + tk.MustExec("insert into nt values (1)") + + hook := &callback.TestDDLCallback{Do: dom} + dom.DDL().SetHook(hook) + + hookFunc := func(job *model.Job) { + if job.Type == model.ActionExchangeTablePartition && job.SchemaState != model.StateNone { + tkCancel.MustExec("use test") + tkCancel.MustGetErrCode("insert into nt values (5)", errno.ErrRowDoesNotMatchGivenPartitionSet) + } + } + hook.OnJobUpdatedExported.Store(&hookFunc) + + tk.MustExec("alter table pt exchange partition p0 with table nt") + tk.MustQuery("select * from pt partition(p0)").Check(testkit.Rows("1")) +} + +func TestExchangePartitionAutoID(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set @@tidb_enable_exchange_partition=1") + defer tk.MustExec("set @@tidb_enable_exchange_partition=0") + + tk.MustExec("use test") + tk.MustExec(`create table pt (a int primary key auto_increment) partition by range(a) ( + partition p0 values less than (3), + partition p1 values less than (6), + PARTITION p2 values less than (9), + PARTITION p3 values less than (50000000) + );`) + tk.MustExec(`create table nt(a int primary key auto_increment);`) + tk.MustExec(`insert into pt values (0), (4)`) + tk.MustExec("insert into nt values (1)") + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/exchangePartitionAutoID", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/exchangePartitionAutoID")) + }() + + tk.MustExec("alter table pt exchange partition p0 with table nt") + tk.MustExec("insert into nt values (NULL)") + tk.MustQuery("select count(*) from nt where a >= 4000000").Check(testkit.Rows("1")) + tk.MustQuery("select count(*) from pt where a >= 4000000").Check(testkit.Rows("1")) +} + +func TestTiDBEnableExchangePartition(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec(`create table pt (a int primary key auto_increment) partition by range(a) ( + partition p0 values less than (3), + partition p1 values less than (6), + PARTITION p2 values less than (9) + );`) + // default + tk.MustQuery("select @@tidb_enable_exchange_partition").Check(testkit.Rows("1")) + tk.MustExec(`create table nt(a int primary key auto_increment);`) + tk.MustExec("alter table pt exchange partition p0 with table nt") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 after the exchange, please analyze related table of the exchange to update statistics")) + + // set tidb_enable_exchange_partition = 0 + tk.MustExec("set @@tidb_enable_exchange_partition=0") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 tidb_enable_exchange_partition is always turned on. This variable has been deprecated and will be removed in the future releases")) + tk.MustQuery("select @@tidb_enable_exchange_partition").Check(testkit.Rows("1")) + tk.MustExec("alter table pt exchange partition p0 with table nt") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 after the exchange, please analyze related table of the exchange to update statistics")) + + // set tidb_enable_exchange_partition = 1 + tk.MustExec("set @@tidb_enable_exchange_partition=1") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("select @@tidb_enable_exchange_partition").Check(testkit.Rows("1")) + tk.MustExec("alter table pt exchange partition p0 with table nt") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 after the exchange, please analyze related table of the exchange to update statistics")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 after the exchange, please analyze related table of the exchange to update statistics")) +} + +func TestExchangePartitionExpressIndex(t *testing.T) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + // Test for table lock. + conf.EnableTableLock = true + conf.Instance.SlowThreshold = 10000 + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Experimental.AllowsExpressionIndex = true + }) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) tk.MustExec("use test") tk.MustExec("set @@tidb_enable_exchange_partition=1") defer tk.MustExec("set @@tidb_enable_exchange_partition=0") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 0824a9298ee53..c62264118ebb6 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3567,7 +3567,6 @@ func checkExchangePartition(pt *model.TableInfo, nt *model.TableInfo) error { return errors.Trace(ErrPartitionExchangeForeignKey.GenWithStackByArgs(nt.Name)) } - // NOTE: if nt is temporary table, it should be checked return nil } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index aa6b47dda6d07..10b178fd960b9 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -1040,6 +1040,7 @@ func updateSchemaVersion(t *meta.Meta, job *model.Job) (int64, error) { diff.OldSchemaID = oldSchemaIDs[0] diff.AffectedOpts = affects case model.ActionExchangeTablePartition: +<<<<<<< HEAD var ( ptSchemaID int64 ptTableID int64 @@ -1048,14 +1049,30 @@ func updateSchemaVersion(t *meta.Meta, job *model.Job) (int64, error) { if err != nil { return 0, errors.Trace(err) } +======= +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) diff.OldTableID = job.TableID - affects := make([]*model.AffectedOption, 1) - affects[0] = &model.AffectedOption{ - SchemaID: ptSchemaID, - TableID: ptTableID, - OldTableID: ptTableID, + diff.OldSchemaID = job.SchemaID + if job.SchemaState != model.StatePublic { + diff.TableID = job.TableID + diff.SchemaID = job.SchemaID + } else { + // Update the partitioned table (it is only done in the last state) + var ( + ptSchemaID int64 + ptTableID int64 + ptDefID int64 // Not needed, will reload the whole table + partName string // Not used + withValidation bool // Not used + ) + // See ddl.ExchangeTablePartition + err = job.DecodeArgs(&ptDefID, &ptSchemaID, &ptTableID, &partName, &withValidation) + if err != nil { + return 0, errors.Trace(err) + } + diff.SchemaID = ptSchemaID + diff.TableID = ptTableID } - diff.AffectedOpts = affects case model.ActionTruncateTablePartition: diff.TableID = job.TableID if len(job.CtxVars) > 0 { diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 1c509ee121e68..9cff8916d9803 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -132,9 +132,13 @@ func TestHalfwayCancelOperations(t *testing.T) { tk.MustExec("insert into pt values(1), (3), (5)") tk.MustExec("create table nt(a int)") tk.MustExec("insert into nt values(7)") +<<<<<<< HEAD:ddl/failtest/fail_db_test.go tk.MustExec("set @@tidb_enable_exchange_partition=1") defer tk.MustExec("set @@tidb_enable_exchange_partition=0") _, err = tk.Exec("alter table pt exchange partition p1 with table nt") +======= + err = tk.ExecToErr("alter table pt exchange partition p1 with table nt") +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)):ddl/tests/fail/fail_db_test.go require.Error(t, err) tk.MustQuery("select * from pt").Check(testkit.Rows("1", "3", "5")) diff --git a/ddl/partition.go b/ddl/partition.go index efa3e75e1f2fc..e3dcc047c3206 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -1342,6 +1342,9 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + if job.IsRollingback() { + return rollbackExchangeTablePartition(d, t, job, nt) + } pt, err := getTableInfo(t, ptID, ptSchemaID) if err != nil { if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { @@ -1350,6 +1353,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } +<<<<<<< HEAD if pt.State != model.StatePublic { job.State = model.JobStateCancelled return ver, ErrInvalidDDLState.GenWithStack("table %s is not in public, but %s", pt.Name, pt.State) @@ -1371,11 +1375,60 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo if err != nil { return ver, errors.Trace(err) } +======= + index, partDef, err := getPartitionDef(pt, partName) + if err != nil { + return ver, errors.Trace(err) + } + if job.SchemaState == model.StateNone { + if pt.State != model.StatePublic { + job.State = model.JobStateCancelled + return ver, dbterror.ErrInvalidDDLState.GenWithStack("table %s is not in public, but %s", pt.Name, pt.State) + } + err = checkExchangePartition(pt, nt) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + err = checkTableDefCompatible(pt, nt) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + err = checkExchangePartitionPlacementPolicy(t, nt.PlacementPolicyRef, pt.PlacementPolicyRef, partDef.PlacementPolicyRef) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + nt.ExchangePartitionInfo = &model.ExchangePartitionInfo{ + ExchangePartitionID: ptID, + ExchangePartitionDefID: defID, + } + // We need an interim schema version, + // so there are no non-matching rows inserted + // into the table using the schema version + // before the exchange is made. + job.SchemaState = model.StateWriteOnly + return updateVersionAndTableInfoWithCheck(d, t, job, nt, true) + } + // From now on, nt (the non-partitioned table) has + // ExchangePartitionInfo set, meaning it is restricted + // to only allow writes that would match the + // partition to be exchange with. + // So we need to rollback that change, instead of just cancelling. + + if d.lease > 0 { + delayForAsyncCommit() + } +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) if withValidation { err = checkExchangePartitionRecordValidation(w, pt, index, ntDbInfo.Name, nt.Name) if err != nil { - job.State = model.JobStateCancelled + job.State = model.JobStateRollingback return ver, errors.Trace(err) } } @@ -1383,19 +1436,11 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo // partition table auto IDs. ptAutoIDs, err := t.GetAutoIDAccessors(ptSchemaID, ptID).Get() if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } // non-partition table auto IDs. ntAutoIDs, err := t.GetAutoIDAccessors(job.SchemaID, nt.ID).Get() if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - - _, partDef, err := getPartitionDef(pt, partName) - if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -1408,35 +1453,32 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo } } - // exchange table meta id - partDef.ID, nt.ID = nt.ID, partDef.ID - - err = t.UpdateTable(ptSchemaID, pt) + // Recreate non-partition table meta info, + // by first delete it with the old table id + err = t.DropTableOrView(job.SchemaID, nt.ID) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } - failpoint.Inject("exchangePartitionErr", func(val failpoint.Value) { - if val.(bool) { - job.State = model.JobStateCancelled - failpoint.Return(ver, errors.New("occur an error after updating partition id")) - } - }) + // exchange table meta id + partDef.ID, nt.ID = nt.ID, partDef.ID - // recreate non-partition table meta info - err = t.DropTableOrView(job.SchemaID, partDef.ID) + err = t.UpdateTable(ptSchemaID, pt) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } err = t.CreateTableOrView(job.SchemaID, nt) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } + failpoint.Inject("exchangePartitionErr", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(ver, errors.New("occur an error after updating partition id")) + } + }) + // Set both tables to the maximum auto IDs between normal table and partitioned table. newAutoIDs := meta.AutoIDGroup{ RowID: mathutil.MaxInt64(ptAutoIDs.RowID, ntAutoIDs.RowID), @@ -1445,26 +1487,40 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo } err = t.GetAutoIDAccessors(ptSchemaID, pt.ID).Put(newAutoIDs) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } err = t.GetAutoIDAccessors(job.SchemaID, nt.ID).Put(newAutoIDs) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } +<<<<<<< HEAD +======= + failpoint.Inject("exchangePartitionAutoID", func(val failpoint.Value) { + if val.(bool) { + seCtx, err := w.sessPool.Get() + defer w.sessPool.Put(seCtx) + if err != nil { + failpoint.Return(ver, err) + } + se := sess.NewSession(seCtx) + _, err = se.Execute(context.Background(), "insert ignore into test.pt values (40000000)", "exchange_partition_test") + if err != nil { + failpoint.Return(ver, err) + } + } + }) + +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) // the follow code is a swap function for rules of two partitions // though partitions has exchanged their ID, swap still take effect - bundles, err := bundlesForExchangeTablePartition(t, job, pt, partDef, nt) + bundles, err := bundlesForExchangeTablePartition(t, pt, partDef, nt) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } if err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles); err != nil { - job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } @@ -1473,7 +1529,6 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo rules, err := infosync.GetLabelRules(context.TODO(), []string{ntrID, ptrID}) if err != nil { - job.State = model.JobStateCancelled return 0, errors.Wrapf(err, "failed to get PD the label rules") } @@ -1500,11 +1555,16 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo patch := label.NewRulePatch(setRules, deleteRules) err = infosync.UpdateLabelRules(context.TODO(), patch) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the label rules") } +<<<<<<< HEAD ver, err = updateSchemaVersion(t, job) +======= + job.SchemaState = model.StatePublic + nt.ExchangePartitionInfo = nil + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, nt, true) +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) if err != nil { return ver, errors.Trace(err) } @@ -1513,7 +1573,645 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, nil } +<<<<<<< HEAD func bundlesForExchangeTablePartition(t *meta.Meta, job *model.Job, pt *model.TableInfo, newPar *model.PartitionDefinition, nt *model.TableInfo) ([]*placement.Bundle, error) { +======= +func checkReorgPartition(t *meta.Meta, job *model.Job) (*model.TableInfo, []model.CIStr, *model.PartitionInfo, []model.PartitionDefinition, []model.PartitionDefinition, error) { + schemaID := job.SchemaID + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) + if err != nil { + return nil, nil, nil, nil, nil, errors.Trace(err) + } + partInfo := &model.PartitionInfo{} + var partNames []model.CIStr + err = job.DecodeArgs(&partNames, &partInfo) + if err != nil { + job.State = model.JobStateCancelled + return nil, nil, nil, nil, nil, errors.Trace(err) + } + addingDefs := tblInfo.Partition.AddingDefinitions + droppingDefs := tblInfo.Partition.DroppingDefinitions + if len(addingDefs) == 0 { + addingDefs = []model.PartitionDefinition{} + } + if len(droppingDefs) == 0 { + droppingDefs = []model.PartitionDefinition{} + } + return tblInfo, partNames, partInfo, droppingDefs, addingDefs, nil +} + +func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + // Handle the rolling back job + if job.IsRollingback() { + ver, err := w.onDropTablePartition(d, t, job) + if err != nil { + return ver, errors.Trace(err) + } + return ver, nil + } + + tblInfo, partNamesCIStr, partInfo, _, addingDefinitions, err := checkReorgPartition(t, job) + if err != nil { + return ver, err + } + partNames := make([]string, len(partNamesCIStr)) + for i := range partNamesCIStr { + partNames[i] = partNamesCIStr[i].L + } + + // In order to skip maintaining the state check in partitionDefinition, TiDB use dropping/addingDefinition instead of state field. + // So here using `job.SchemaState` to judge what the stage of this job is. + originalState := job.SchemaState + switch job.SchemaState { + case model.StateNone: + // job.SchemaState == model.StateNone means the job is in the initial state of reorg partition. + // Here should use partInfo from job directly and do some check action. + // In case there was a race for queueing different schema changes on the same + // table and the checks was not done on the current schema version. + // The partInfo may have been checked against an older schema version for example. + // If the check is done here, it does not need to be repeated, since no other + // DDL on the same table can be run concurrently. + err = checkAddPartitionTooManyPartitions(uint64(len(tblInfo.Partition.Definitions) + + len(partInfo.Definitions) - + len(partNames))) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + err = checkReorgPartitionNames(tblInfo.Partition, partNamesCIStr, partInfo) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + // Re-check that the dropped/added partitions are compatible with current definition + firstPartIdx, lastPartIdx, idMap, err := getReplacedPartitionIDs(partNamesCIStr, tblInfo.Partition) + if err != nil { + job.State = model.JobStateCancelled + return ver, err + } + sctx := w.sess.Context + if err = checkReorgPartitionDefs(sctx, tblInfo, partInfo, firstPartIdx, lastPartIdx, idMap); err != nil { + job.State = model.JobStateCancelled + return ver, err + } + + // move the adding definition into tableInfo. + updateAddingPartitionInfo(partInfo, tblInfo) + orgDefs := tblInfo.Partition.Definitions + _ = updateDroppingPartitionInfo(tblInfo, partNames) + // Reset original partitions, and keep DroppedDefinitions + tblInfo.Partition.Definitions = orgDefs + + // modify placement settings + for _, def := range tblInfo.Partition.AddingDefinitions { + if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, def.PlacementPolicyRef); err != nil { + // job.State = model.JobStateCancelled may be set depending on error in function above. + return ver, errors.Trace(err) + } + } + + // From now on we cannot just cancel the DDL, we must roll back if changesMade! + changesMade := false + if tblInfo.TiFlashReplica != nil { + // Must set placement rule, and make sure it succeeds. + if err := infosync.ConfigureTiFlashPDForPartitions(true, &tblInfo.Partition.AddingDefinitions, tblInfo.TiFlashReplica.Count, &tblInfo.TiFlashReplica.LocationLabels, tblInfo.ID); err != nil { + logutil.BgLogger().Error("ConfigureTiFlashPDForPartitions fails", zap.Error(err)) + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + changesMade = true + // In the next step, StateDeleteOnly, wait to verify the TiFlash replicas are OK + } + + bundles, err := alterTablePartitionBundles(t, tblInfo, tblInfo.Partition.AddingDefinitions) + if err != nil { + if !changesMade { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + return convertAddTablePartitionJob2RollbackJob(d, t, job, err, tblInfo) + } + + if len(bundles) > 0 { + if err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles); err != nil { + if !changesMade { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the placement rules") + } + return convertAddTablePartitionJob2RollbackJob(d, t, job, err, tblInfo) + } + changesMade = true + } + + ids := getIDs([]*model.TableInfo{tblInfo}) + for _, p := range tblInfo.Partition.AddingDefinitions { + ids = append(ids, p.ID) + } + changed, err := alterTableLabelRule(job.SchemaName, tblInfo, ids) + changesMade = changesMade || changed + if err != nil { + if !changesMade { + job.State = model.JobStateCancelled + return ver, err + } + return convertAddTablePartitionJob2RollbackJob(d, t, job, err, tblInfo) + } + + // Doing the preSplitAndScatter here, since all checks are completed, + // and we will soon start writing to the new partitions. + if s, ok := d.store.(kv.SplittableStore); ok && s != nil { + // partInfo only contains the AddingPartitions + splitPartitionTableRegion(w.sess.Context, s, tblInfo, partInfo.Definitions, true) + } + + // Assume we cannot have more than MaxUint64 rows, set the progress to 1/10 of that. + metrics.GetBackfillProgressByLabel(metrics.LblReorgPartition, job.SchemaName, tblInfo.Name.String()).Set(0.1 / float64(math.MaxUint64)) + job.SchemaState = model.StateDeleteOnly + tblInfo.Partition.DDLState = model.StateDeleteOnly + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + + // Is really both StateDeleteOnly AND StateWriteOnly needed? + // If transaction A in WriteOnly inserts row 1 (into both new and old partition set) + // and then transaction B in DeleteOnly deletes that row (in both new and old) + // does really transaction B need to do the delete in the new partition? + // Yes, otherwise it would still be there when the WriteReorg happens, + // and WriteReorg would only copy existing rows to the new table, so unless it is + // deleted it would result in a ghost row! + // What about update then? + // Updates also need to be handled for new partitions in DeleteOnly, + // since it would not be overwritten during Reorganize phase. + // BUT if the update results in adding in one partition and deleting in another, + // THEN only the delete must happen in the new partition set, not the insert! + case model.StateDeleteOnly: + // This state is to confirm all servers can not see the new partitions when reorg is running, + // so that all deletes will be done in both old and new partitions when in either DeleteOnly + // or WriteOnly state. + // Also using the state for checking that the optional TiFlash replica is available, making it + // in a state without (much) data and easy to retry without side effects. + + // Reason for having it here, is to make it easy for retry, and better to make sure it is in-sync + // as early as possible, to avoid a long wait after the data copying. + if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Available { + // For available state, the new added partition should wait its replica to + // be finished, otherwise the query to this partition will be blocked. + count := tblInfo.TiFlashReplica.Count + needRetry, err := checkPartitionReplica(count, addingDefinitions, d) + if err != nil { + // need to rollback, since we tried to register the new + // partitions before! + return convertAddTablePartitionJob2RollbackJob(d, t, job, err, tblInfo) + } + if needRetry { + // The new added partition hasn't been replicated. + // Do nothing to the job this time, wait next worker round. + time.Sleep(tiflashCheckTiDBHTTPAPIHalfInterval) + // Set the error here which will lead this job exit when it's retry times beyond the limitation. + return ver, errors.Errorf("[ddl] add partition wait for tiflash replica to complete") + } + + // When TiFlash Replica is ready, we must move them into `AvailablePartitionIDs`. + // Since onUpdateFlashReplicaStatus cannot see the partitions yet (not public) + for _, d := range addingDefinitions { + tblInfo.TiFlashReplica.AvailablePartitionIDs = append(tblInfo.TiFlashReplica.AvailablePartitionIDs, d.ID) + } + } + + job.SchemaState = model.StateWriteOnly + tblInfo.Partition.DDLState = model.StateWriteOnly + metrics.GetBackfillProgressByLabel(metrics.LblReorgPartition, job.SchemaName, tblInfo.Name.String()).Set(0.2 / float64(math.MaxUint64)) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != job.SchemaState) + case model.StateWriteOnly: + // Insert this state to confirm all servers can see the new partitions when reorg is running, + // so that new data will be updated in both old and new partitions when reorganizing. + job.SnapshotVer = 0 + job.SchemaState = model.StateWriteReorganization + tblInfo.Partition.DDLState = model.StateWriteReorganization + metrics.GetBackfillProgressByLabel(metrics.LblReorgPartition, job.SchemaName, tblInfo.Name.String()).Set(0.3 / float64(math.MaxUint64)) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != job.SchemaState) + case model.StateWriteReorganization: + physicalTableIDs := getPartitionIDsFromDefinitions(tblInfo.Partition.DroppingDefinitions) + tbl, err2 := getTable(d.store, job.SchemaID, tblInfo) + if err2 != nil { + return ver, errors.Trace(err2) + } + // TODO: If table has global indexes, we need reorg to clean up them. + // and then add the new partition ids back... + if _, ok := tbl.(table.PartitionedTable); ok && hasGlobalIndex(tblInfo) { + err = errors.Trace(dbterror.ErrCancelledDDLJob.GenWithStack("global indexes is not supported yet for reorganize partition")) + return convertAddTablePartitionJob2RollbackJob(d, t, job, err, tblInfo) + } + var done bool + done, ver, err = doPartitionReorgWork(w, d, t, job, tbl, physicalTableIDs) + + if !done { + return ver, err + } + + firstPartIdx, lastPartIdx, idMap, err2 := getReplacedPartitionIDs(partNamesCIStr, tblInfo.Partition) + failpoint.Inject("reorgPartWriteReorgReplacedPartIDsFail", func(val failpoint.Value) { + if val.(bool) { + err2 = errors.New("Injected error by reorgPartWriteReorgReplacedPartIDsFail") + } + }) + if err2 != nil { + return ver, err2 + } + newDefs := getReorganizedDefinitions(tblInfo.Partition, firstPartIdx, lastPartIdx, idMap) + + // From now on, use the new definitions, but keep the Adding and Dropping for double write + tblInfo.Partition.Definitions = newDefs + tblInfo.Partition.Num = uint64(len(newDefs)) + + // Now all the data copying is done, but we cannot simply remove the droppingDefinitions + // since they are a part of the normal Definitions that other nodes with + // the current schema version. So we need to double write for one more schema version + job.SchemaState = model.StateDeleteReorganization + tblInfo.Partition.DDLState = model.StateDeleteReorganization + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != job.SchemaState) + + case model.StateDeleteReorganization: + // Drop the droppingDefinitions and finish the DDL + // This state is needed for the case where client A sees the schema + // with version of StateWriteReorg and would not see updates of + // client B that writes to the new partitions, previously + // addingDefinitions, since it would not double write to + // the droppingDefinitions during this time + // By adding StateDeleteReorg state, client B will write to both + // the new (previously addingDefinitions) AND droppingDefinitions + + // Register the droppingDefinitions ids for rangeDelete + // and the addingDefinitions for handling in the updateSchemaVersion + physicalTableIDs := getPartitionIDsFromDefinitions(tblInfo.Partition.DroppingDefinitions) + newIDs := getPartitionIDsFromDefinitions(partInfo.Definitions) + job.CtxVars = []interface{}{physicalTableIDs, newIDs} + definitionsToAdd := tblInfo.Partition.AddingDefinitions + tblInfo.Partition.DroppingDefinitions = nil + tblInfo.Partition.AddingDefinitions = nil + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + failpoint.Inject("reorgPartWriteReorgSchemaVersionUpdateFail", func(val failpoint.Value) { + if val.(bool) { + err = errors.New("Injected error by reorgPartWriteReorgSchemaVersionUpdateFail") + } + }) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateNone + tblInfo.Partition.DDLState = model.StateNone + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + // How to handle this? + // Seems to only trigger asynchronous update of statistics. + // Should it actually be synchronous? + asyncNotifyEvent(d, &util.Event{Tp: model.ActionReorganizePartition, TableInfo: tblInfo, PartInfo: &model.PartitionInfo{Definitions: definitionsToAdd}}) + // A background job will be created to delete old partition data. + job.Args = []interface{}{physicalTableIDs} + + default: + err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("partition", job.SchemaState) + } + + return ver, errors.Trace(err) +} + +func doPartitionReorgWork(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, physTblIDs []int64) (done bool, ver int64, err error) { + job.ReorgMeta.ReorgTp = model.ReorgTypeTxn + sctx, err1 := w.sessPool.Get() + if err1 != nil { + return done, ver, err1 + } + defer w.sessPool.Put(sctx) + rh := newReorgHandler(sess.NewSession(sctx)) + elements := BuildElements(tbl.Meta().Columns[0], tbl.Meta().Indices) + partTbl, ok := tbl.(table.PartitionedTable) + if !ok { + return false, ver, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() + } + dbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + return false, ver, errors.Trace(err) + } + reorgInfo, err := getReorgInfoFromPartitions(d.jobContext(job.ID), d, rh, job, dbInfo, partTbl, physTblIDs, elements) + err = w.runReorgJob(reorgInfo, tbl.Meta(), d.lease, func() (reorgErr error) { + defer tidbutil.Recover(metrics.LabelDDL, "doPartitionReorgWork", + func() { + reorgErr = dbterror.ErrCancelledDDLJob.GenWithStack("reorganize partition for table `%v` panic", tbl.Meta().Name) + }, false) + return w.reorgPartitionDataAndIndex(tbl, reorgInfo) + }) + if err != nil { + if dbterror.ErrPausedDDLJob.Equal(err) { + return false, ver, nil + } + + if dbterror.ErrWaitReorgTimeout.Equal(err) { + // If timeout, we should return, check for the owner and re-wait job done. + return false, ver, nil + } + if kv.IsTxnRetryableError(err) { + return false, ver, errors.Trace(err) + } + if err1 := rh.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { + logutil.BgLogger().Warn("reorg partition job failed, RemoveDDLReorgHandle failed, can't convert job to rollback", zap.String("category", "ddl"), + zap.String("job", job.String()), zap.Error(err1)) + } + logutil.BgLogger().Warn("reorg partition job failed, convert job to rollback", zap.String("category", "ddl"), zap.String("job", job.String()), zap.Error(err)) + ver, err = convertAddTablePartitionJob2RollbackJob(d, t, job, err, tbl.Meta()) + return false, ver, errors.Trace(err) + } + return true, ver, err +} + +type reorgPartitionWorker struct { + *backfillCtx + // Static allocated to limit memory allocations + rowRecords []*rowRecord + rowDecoder *decoder.RowDecoder + rowMap map[int64]types.Datum + writeColOffsetMap map[int64]int + maxOffset int + reorgedTbl table.PartitionedTable +} + +func newReorgPartitionWorker(sessCtx sessionctx.Context, i int, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *JobContext) (*reorgPartitionWorker, error) { + reorgedTbl, err := tables.GetReorganizedPartitionedTable(t) + if err != nil { + return nil, errors.Trace(err) + } + pt := t.GetPartitionedTable() + if pt == nil { + return nil, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() + } + partColIDs := pt.GetPartitionColumnIDs() + writeColOffsetMap := make(map[int64]int, len(partColIDs)) + maxOffset := 0 + for _, col := range pt.Cols() { + found := false + for _, id := range partColIDs { + if col.ID == id { + found = true + break + } + } + if !found { + continue + } + writeColOffsetMap[col.ID] = col.Offset + maxOffset = mathutil.Max[int](maxOffset, col.Offset) + } + return &reorgPartitionWorker{ + backfillCtx: newBackfillCtx(reorgInfo.d, i, sessCtx, reorgInfo.SchemaName, t, jc, "reorg_partition_rate", false), + rowDecoder: decoder.NewRowDecoder(t, t.WritableCols(), decodeColMap), + rowMap: make(map[int64]types.Datum, len(decodeColMap)), + writeColOffsetMap: writeColOffsetMap, + maxOffset: maxOffset, + reorgedTbl: reorgedTbl, + }, nil +} + +func (w *reorgPartitionWorker) BackfillData(handleRange reorgBackfillTask) (taskCtx backfillTaskContext, errInTxn error) { + oprStartTime := time.Now() + ctx := kv.WithInternalSourceAndTaskType(context.Background(), w.jobContext.ddlJobSourceType(), kvutil.ExplicitTypeDDL) + errInTxn = kv.RunInNewTxn(ctx, w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { + taskCtx.addedCount = 0 + taskCtx.scanCount = 0 + txn.SetOption(kv.Priority, handleRange.priority) + if tagger := w.GetCtx().getResourceGroupTaggerForTopSQL(handleRange.getJobID()); tagger != nil { + txn.SetOption(kv.ResourceGroupTagger, tagger) + } + + rowRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) + if err != nil { + return errors.Trace(err) + } + taskCtx.nextKey = nextKey + taskCtx.done = taskDone + + warningsMap := make(map[errors.ErrorID]*terror.Error) + warningsCountMap := make(map[errors.ErrorID]int64) + for _, prr := range rowRecords { + taskCtx.scanCount++ + + err = txn.Set(prr.key, prr.vals) + if err != nil { + return errors.Trace(err) + } + taskCtx.addedCount++ + if prr.warning != nil { + if _, ok := warningsCountMap[prr.warning.ID()]; ok { + warningsCountMap[prr.warning.ID()]++ + } else { + warningsCountMap[prr.warning.ID()] = 1 + warningsMap[prr.warning.ID()] = prr.warning + } + } + // TODO: Future optimization: also write the indexes here? + // What if the transaction limit is just enough for a single row, without index? + // Hmm, how could that be in the first place? + // For now, implement the batch-txn w.addTableIndex, + // since it already exists and is in use + } + + // Collect the warnings. + taskCtx.warnings, taskCtx.warningsCount = warningsMap, warningsCountMap + + // also add the index entries here? And make sure they are not added somewhere else + + return nil + }) + logSlowOperations(time.Since(oprStartTime), "BackfillData", 3000) + + return +} + +func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgBackfillTask) ([]*rowRecord, kv.Key, bool, error) { + w.rowRecords = w.rowRecords[:0] + startTime := time.Now() + + // taskDone means that the added handle is out of taskRange.endHandle. + taskDone := false + sysTZ := w.sessCtx.GetSessionVars().StmtCtx.TimeZone + + tmpRow := make([]types.Datum, w.maxOffset+1) + var lastAccessedHandle kv.Key + oprStartTime := startTime + err := iterateSnapshotKeys(w.jobContext, w.sessCtx.GetStore(), taskRange.priority, w.table.RecordPrefix(), txn.StartTS(), taskRange.startKey, taskRange.endKey, + func(handle kv.Handle, recordKey kv.Key, rawRow []byte) (bool, error) { + oprEndTime := time.Now() + logSlowOperations(oprEndTime.Sub(oprStartTime), "iterateSnapshotKeys in reorgPartitionWorker fetchRowColVals", 0) + oprStartTime = oprEndTime + + taskDone = recordKey.Cmp(taskRange.endKey) >= 0 + + if taskDone || len(w.rowRecords) >= w.batchCnt { + return false, nil + } + + _, err := w.rowDecoder.DecodeTheExistedColumnMap(w.sessCtx, handle, rawRow, sysTZ, w.rowMap) + if err != nil { + return false, errors.Trace(err) + } + + // Set the partitioning columns and calculate which partition to write to + for colID, offset := range w.writeColOffsetMap { + d, ok := w.rowMap[colID] + if !ok { + return false, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() + } + tmpRow[offset] = d + } + p, err := w.reorgedTbl.GetPartitionByRow(w.sessCtx, tmpRow) + if err != nil { + return false, errors.Trace(err) + } + pid := p.GetPhysicalID() + newKey := tablecodec.EncodeTablePrefix(pid) + newKey = append(newKey, recordKey[len(newKey):]...) + w.rowRecords = append(w.rowRecords, &rowRecord{ + key: newKey, vals: rawRow, + }) + + w.cleanRowMap() + lastAccessedHandle = recordKey + if recordKey.Cmp(taskRange.endKey) == 0 { + taskDone = true + return false, nil + } + return true, nil + }) + + if len(w.rowRecords) == 0 { + taskDone = true + } + + logutil.BgLogger().Debug("txn fetches handle info", zap.String("category", "ddl"), zap.Uint64("txnStartTS", txn.StartTS()), zap.String("taskRange", taskRange.String()), zap.Duration("takeTime", time.Since(startTime))) + return w.rowRecords, getNextHandleKey(taskRange, taskDone, lastAccessedHandle), taskDone, errors.Trace(err) +} + +func (w *reorgPartitionWorker) cleanRowMap() { + for id := range w.rowMap { + delete(w.rowMap, id) + } +} + +func (w *reorgPartitionWorker) AddMetricInfo(cnt float64) { + w.metricCounter.Add(cnt) +} + +func (*reorgPartitionWorker) String() string { + return typeReorgPartitionWorker.String() +} + +func (w *reorgPartitionWorker) GetCtx() *backfillCtx { + return w.backfillCtx +} + +func (w *worker) reorgPartitionDataAndIndex(t table.Table, reorgInfo *reorgInfo) error { + // First copy all table data to the new partitions + // from each of the DroppingDefinitions partitions. + // Then create all indexes on the AddingDefinitions partitions + // for each new index, one partition at a time. + + // Copy the data from the DroppingDefinitions to the AddingDefinitions + if bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) { + err := w.updatePhysicalTableRow(t, reorgInfo) + if err != nil { + return errors.Trace(err) + } + } + + failpoint.Inject("reorgPartitionAfterDataCopy", func(val failpoint.Value) { + //nolint:forcetypeassert + if val.(bool) { + panic("panic test in reorgPartitionAfterDataCopy") + } + }) + + // Rewrite this to do all indexes at once in addTableIndex + // instead of calling it once per index (meaning reading the table multiple times) + // But for now, try to understand how it works... + firstNewPartitionID := t.Meta().Partition.AddingDefinitions[0].ID + startElementOffset := 0 + //startElementOffsetToResetHandle := -1 + // This backfill job starts with backfilling index data, whose index ID is currElement.ID. + if !bytes.Equal(reorgInfo.currElement.TypeKey, meta.IndexElementKey) { + // First run, have not yet started backfilling index data + // Restart with the first new partition. + // TODO: handle remove partitioning + reorgInfo.PhysicalTableID = firstNewPartitionID + } else { + // The job was interrupted and has been restarted, + // reset and start from where it was done + for i, element := range reorgInfo.elements[1:] { + if reorgInfo.currElement.ID == element.ID { + startElementOffset = i + //startElementOffsetToResetHandle = i + break + } + } + } + + for i := startElementOffset; i < len(reorgInfo.elements[1:]); i++ { + // Now build the indexes in the new partitions + var physTbl table.PhysicalTable + if tbl, ok := t.(table.PartitionedTable); ok { + physTbl = tbl.GetPartition(reorgInfo.PhysicalTableID) + } else if tbl, ok := t.(table.PhysicalTable); ok { + // This may be used when partitioning a non-partitioned table + physTbl = tbl + } + // Get the original start handle and end handle. + currentVer, err := getValidCurrentVersion(reorgInfo.d.store) + if err != nil { + return errors.Trace(err) + } + // TODO: Can we improve this in case of a crash? + // like where the regInfo PhysicalTableID and element is the same, + // and the tableid in the key-prefix regInfo.StartKey and regInfo.EndKey matches with PhysicalTableID + // do not change the reorgInfo start/end key + startHandle, endHandle, err := getTableRange(reorgInfo.d.jobContext(reorgInfo.Job.ID), reorgInfo.d, physTbl, currentVer.Ver, reorgInfo.Job.Priority) + if err != nil { + return errors.Trace(err) + } + + // Always (re)start with the full PhysicalTable range + reorgInfo.StartKey, reorgInfo.EndKey = startHandle, endHandle + + // Update the element in the reorgInfo for updating the reorg meta below. + reorgInfo.currElement = reorgInfo.elements[i+1] + // Write the reorg info to store so the whole reorganize process can recover from panic. + err = reorgInfo.UpdateReorgMeta(reorgInfo.StartKey, w.sessPool) + logutil.BgLogger().Info("update column and indexes", zap.String("category", "ddl"), + zap.Int64("jobID", reorgInfo.Job.ID), + zap.ByteString("elementType", reorgInfo.currElement.TypeKey), + zap.Int64("elementID", reorgInfo.currElement.ID), + zap.Int64("partitionTableId", physTbl.GetPhysicalID()), + zap.String("startHandle", hex.EncodeToString(reorgInfo.StartKey)), + zap.String("endHandle", hex.EncodeToString(reorgInfo.EndKey))) + if err != nil { + return errors.Trace(err) + } + err = w.addTableIndex(t, reorgInfo) + if err != nil { + return errors.Trace(err) + } + reorgInfo.PhysicalTableID = firstNewPartitionID + } + failpoint.Inject("reorgPartitionAfterIndex", func(val failpoint.Value) { + //nolint:forcetypeassert + if val.(bool) { + panic("panic test in reorgPartitionAfterIndex") + } + }) + return nil +} + +func bundlesForExchangeTablePartition(t *meta.Meta, pt *model.TableInfo, newPar *model.PartitionDefinition, nt *model.TableInfo) ([]*placement.Bundle, error) { +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) bundles := make([]*placement.Bundle, 0, 3) ptBundle, err := placement.NewTableBundle(t, pt) @@ -1615,6 +2313,37 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, inde return nil } +<<<<<<< HEAD +======= +func checkExchangePartitionPlacementPolicy(t *meta.Meta, ntPPRef, ptPPRef, partPPRef *model.PolicyRefInfo) error { + partitionPPRef := partPPRef + if partitionPPRef == nil { + partitionPPRef = ptPPRef + } + + if ntPPRef == nil && partitionPPRef == nil { + return nil + } + if ntPPRef == nil || partitionPPRef == nil { + return dbterror.ErrTablesDifferentMetadata + } + + ptPlacementPolicyInfo, _ := getPolicyInfo(t, partitionPPRef.ID) + ntPlacementPolicyInfo, _ := getPolicyInfo(t, ntPPRef.ID) + if ntPlacementPolicyInfo == nil && ptPlacementPolicyInfo == nil { + return nil + } + if ntPlacementPolicyInfo == nil || ptPlacementPolicyInfo == nil { + return dbterror.ErrTablesDifferentMetadata + } + if ntPlacementPolicyInfo.Name.L != ptPlacementPolicyInfo.Name.L { + return dbterror.ErrTablesDifferentMetadata + } + + return nil +} + +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) func buildCheckSQLForRangeExprPartition(pi *model.PartitionInfo, index int, schemaName, tableName model.CIStr) (string, []interface{}) { var buf strings.Builder paramList := make([]interface{}, 0, 4) diff --git a/ddl/placement_policy_test.go b/ddl/placement_policy_test.go index 73a93492df50e..139fe1523dd31 100644 --- a/ddl/placement_policy_test.go +++ b/ddl/placement_policy_test.go @@ -1830,6 +1830,7 @@ func (s *testDBSuite6) TestTruncatePartitionGCWithPlacement(c *C) { c.Assert(ok, IsTrue) } +<<<<<<< HEAD func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) @@ -1838,6 +1839,27 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { tk.MustExec("drop table if exists t1, t2, tp") tk.MustExec("drop placement policy if exists p1") tk.MustExec("drop placement policy if exists p2") +======= +func TestDropPartitionWithPlacement(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`)) + defer func(originGC bool) { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed")) + if originGC { + util.EmulatorGCEnable() + } else { + util.EmulatorGCDisable() + } + }(util.IsEmulatorGCEnable()) + util.EmulatorGCDisable() + + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists tp") + tk.MustExec("drop placement policy if exists p1") + tk.MustExec("drop placement policy if exists p2") + tk.MustExec("drop placement policy if exists p3") tk.MustExec("create placement policy p1 primary_region='r1' regions='r1'") defer tk.MustExec("drop placement policy p1") @@ -1845,22 +1867,111 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { tk.MustExec("create placement policy p2 primary_region='r2' regions='r2'") defer tk.MustExec("drop placement policy p2") + tk.MustExec("create placement policy p3 primary_region='r3' regions='r3'") + defer tk.MustExec("drop placement policy p3") + + policy1, ok := dom.InfoSchema().PolicyByName(model.NewCIStr("p1")) + require.True(t, ok) + + policy3, ok := dom.InfoSchema().PolicyByName(model.NewCIStr("p3")) + require.True(t, ok) + + // test for partitioned table + tk.MustExec(`CREATE TABLE tp (id INT) placement policy p1 PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100), + PARTITION p1 VALUES LESS THAN (1000) placement policy p2, + PARTITION p2 VALUES LESS THAN (10000) placement policy p3, + PARTITION p3 VALUES LESS THAN (100000) + );`) + defer tk.MustExec("drop table tp") + + tp, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + require.NoError(t, err) + + checkOldPartitions := make([]model.PartitionDefinition, 0, 2) + for _, p := range tp.Meta().Partition.Definitions { + switch p.Name.L { + case "p1": + checkOldPartitions = append(checkOldPartitions, p.Clone()) + case "p3": + p.PlacementPolicyRef = tp.Meta().PlacementPolicyRef + checkOldPartitions = append(checkOldPartitions, p.Clone()) + } + } + + tk.MustExec("ALTER TABLE tp DROP partition p1,p3") + newTp, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + require.NoError(t, err) + require.Equal(t, tp.Meta().ID, newTp.Meta().ID) + require.Equal(t, policy1.ID, newTp.Meta().PlacementPolicyRef.ID) + require.Equal(t, 2, len(newTp.Meta().Partition.Definitions)) + require.Nil(t, newTp.Meta().Partition.Definitions[0].PlacementPolicyRef) + require.Equal(t, policy3.ID, newTp.Meta().Partition.Definitions[1].PlacementPolicyRef.ID) + require.Equal(t, tp.Meta().Partition.Definitions[0].ID, newTp.Meta().Partition.Definitions[0].ID) + require.True(t, newTp.Meta().Partition.Definitions[1].ID == tp.Meta().Partition.Definitions[2].ID) + checkExistTableBundlesInPD(t, dom, "test", "tp") + checkWaitingGCPartitionBundlesInPD(t, dom, checkOldPartitions) + + // add new partition will not override bundle waiting for GC + tk.MustExec("alter table tp add partition (partition p4 values less than(1000000))") + newTp2, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + require.NoError(t, err) + require.Equal(t, 3, len(newTp2.Meta().Partition.Definitions)) + checkWaitingGCPartitionBundlesInPD(t, dom, checkOldPartitions) + + // do GC + for _, par := range checkOldPartitions { + bundle, err := infosync.GetRuleBundle(context.TODO(), placement.GroupID(par.ID)) + require.NoError(t, err) + require.False(t, bundle.IsEmpty()) + } + + gcWorker, err := gcworker.NewMockGCWorker(store) + require.NoError(t, err) + require.Nil(t, gcWorker.DeleteRanges(context.TODO(), math.MaxInt64)) + + checkExistTableBundlesInPD(t, dom, "test", "tp") + for _, par := range checkOldPartitions { + bundle, err := infosync.GetRuleBundle(context.TODO(), placement.GroupID(par.ID)) + require.NoError(t, err) + require.True(t, bundle.IsEmpty()) + } +} + +func TestExchangePartitionWithPlacement(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + // clearAllBundles(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) + + tk.MustExec("create placement policy pp1 primary_region='r1' regions='r1'") + tk.MustExec("create placement policy pp2 primary_region='r2' regions='r2'") + tk.MustExec("create placement policy pp3 primary_region='r3' regions='r3'") + +<<<<<<< HEAD + tk.MustExec("create placement policy p2 primary_region='r2' regions='r2'") + defer tk.MustExec("drop placement policy p2") + policy1, ok := tk.Se.GetInfoSchema().(infoschema.InfoSchema).PolicyByName(model.NewCIStr("p1")) c.Assert(ok, IsTrue) policy2, ok := tk.Se.GetInfoSchema().(infoschema.InfoSchema).PolicyByName(model.NewCIStr("p2")) c.Assert(ok, IsTrue) +======= + policy1, ok := dom.InfoSchema().PolicyByName(model.NewCIStr("pp1")) + require.True(t, ok) +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) - tk.MustExec(`CREATE TABLE t1 (id INT) placement policy p1`) - defer tk.MustExec("drop table t1") - + tk.MustExec(`CREATE TABLE t1 (id INT) placement policy pp1`) tk.MustExec(`CREATE TABLE t2 (id INT)`) - defer tk.MustExec("drop table t2") + tk.MustExec(`CREATE TABLE t3 (id INT) placement policy pp3`) t1, err := tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) t1ID := t1.Meta().ID +<<<<<<< HEAD t2, err := tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) c.Assert(err, IsNil) t2ID := t2.Meta().ID @@ -1871,6 +1982,13 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { PARTITION p2 VALUES LESS THAN (10000) primary_region="r1" regions="r1,r2" );`) defer tk.MustExec("drop table tp") +======= + tk.MustExec(`CREATE TABLE tp (id INT) placement policy pp3 PARTITION BY RANGE (id) ( + PARTITION p1 VALUES LESS THAN (100) placement policy pp1, + PARTITION p2 VALUES LESS THAN (1000) placement policy pp2, + PARTITION p3 VALUES LESS THAN (10000) + )`) +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) tp, err := tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) c.Assert(err, IsNil) @@ -1879,15 +1997,16 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { par1ID := tp.Meta().Partition.Definitions[1].ID par2ID := tp.Meta().Partition.Definitions[2].ID - // exchange par0, t1 - tk.MustExec("alter table tp exchange partition p0 with table t1") + // exchange par1, t1 + tk.MustExec("alter table tp exchange partition p1 with table t1") tk.MustQuery("show create table t1").Check(testkit.Rows("" + "t1 CREATE TABLE `t1` (\n" + " `id` int(11) DEFAULT NULL\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */")) + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`pp1` */")) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + +<<<<<<< HEAD ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1\" */\n" + "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + @@ -1987,6 +2106,49 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { c.Assert(t2.Meta().DirectPlacementOpts, IsNil) c.Assert(t2.Meta().PlacementPolicyRef, IsNil) checkExistTableBundlesInPD(c, s.dom, "test", "tp") +======= + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`pp3` */\n" + + "PARTITION BY RANGE (`id`)\n" + + "(PARTITION `p1` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`pp1` */,\n" + + " PARTITION `p2` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`pp2` */,\n" + + " PARTITION `p3` VALUES LESS THAN (10000))")) + tp, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + require.NoError(t, err) + require.Equal(t, tpID, tp.Meta().ID) + require.Equal(t, t1ID, tp.Meta().Partition.Definitions[0].ID) + require.NotNil(t, tp.Meta().Partition.Definitions[0].PlacementPolicyRef) + t1, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + require.Equal(t, par0ID, t1.Meta().ID) + require.Equal(t, policy1.ID, t1.Meta().PlacementPolicyRef.ID) + checkExistTableBundlesInPD(t, dom, "test", "tp") + + // exchange par2, t1 + tk.MustGetErrCode("alter table tp exchange partition p2 with table t1", mysql.ErrTablesDifferentMetadata) + + // exchange par3, t1 + tk.MustGetErrCode("alter table tp exchange partition p3 with table t1", mysql.ErrTablesDifferentMetadata) + + // exchange par1, t2 + tk.MustGetErrCode("alter table tp exchange partition p1 with table t2", mysql.ErrTablesDifferentMetadata) + + // exchange par2, t2 + tk.MustGetErrCode("alter table tp exchange partition p2 with table t2", mysql.ErrTablesDifferentMetadata) + + // exchange par3, t2 + tk.MustGetErrCode("alter table tp exchange partition p3 with table t2", mysql.ErrTablesDifferentMetadata) + + // exchange par1, t3 + tk.MustGetErrCode("alter table tp exchange partition p1 with table t3", mysql.ErrTablesDifferentMetadata) + + // exchange par2, t3 + tk.MustGetErrCode("alter table tp exchange partition p2 with table t3", mysql.ErrTablesDifferentMetadata) + + // exchange par3, t3 + tk.MustExec("alter table tp exchange partition p3 with table t3") + checkExistTableBundlesInPD(t, dom, "test", "tp") + checkExistTableBundlesInPD(t, dom, "test", "t3") +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) } func (s *testDBSuite6) TestPDFail(c *C) { diff --git a/ddl/rollingback.go b/ddl/rollingback.go index ab4e863fb7134..97589c3ed54a1 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -337,7 +337,48 @@ func rollingbackAddIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, isP return } +<<<<<<< HEAD func convertAddTablePartitionJob2RollbackJob(t *meta.Meta, job *model.Job, otherwiseErr error, tblInfo *model.TableInfo) (ver int64, err error) { +======= +func needNotifyAndStopReorgWorker(job *model.Job) bool { + if job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 { + // If the value of SnapshotVer isn't zero, it means the reorg workers have been started. + if job.MultiSchemaInfo != nil { + // However, if the sub-job is non-revertible, it means the reorg process is finished. + // We don't need to start another round to notify reorg workers to exit. + return job.MultiSchemaInfo.Revertible + } + return true + } + return false +} + +// rollbackExchangeTablePartition will clear the non-partitioned +// table's ExchangePartitionInfo state. +func rollbackExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job, tblInfo *model.TableInfo) (int64, error) { + tblInfo.ExchangePartitionInfo = nil + job.State = model.JobStateRollbackDone + job.SchemaState = model.StatePublic + return updateVersionAndTableInfo(d, t, job, tblInfo, true) +} + +func rollingbackExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + if job.SchemaState == model.StateNone { + // Nothing is changed + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + var nt *model.TableInfo + nt, err = GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + ver, err = rollbackExchangeTablePartition(d, t, job, nt) + return ver, errors.Trace(err) +} + +func convertAddTablePartitionJob2RollbackJob(d *ddlCtx, t *meta.Meta, job *model.Job, otherwiseErr error, tblInfo *model.TableInfo) (ver int64, err error) { +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) addingDefinitions := tblInfo.Partition.AddingDefinitions partNames := make([]string, 0, len(addingDefinitions)) for _, pd := range addingDefinitions { @@ -436,7 +477,36 @@ func rollingbackTruncateTable(t *meta.Meta, job *model.Job) (ver int64, err erro if err != nil { return ver, errors.Trace(err) } +<<<<<<< HEAD return cancelOnlyNotHandledJob(job) +======= + return cancelOnlyNotHandledJob(job, model.StateNone) +} + +func rollingbackReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + if job.SchemaState == model.StateNone { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + // addingDefinitions is also in tblInfo, here pass the tblInfo as parameter directly. + // TODO: Test this with reorganize partition p1 into (partition p1 ...)! + return convertAddTablePartitionJob2RollbackJob(d, t, job, dbterror.ErrCancelledDDLJob, tblInfo) +} + +func pauseReorgWorkers(w *worker, d *ddlCtx, job *model.Job) (err error) { + if needNotifyAndStopReorgWorker(job) { + logutil.Logger(w.logCtx).Info("pausing the DDL job", zap.String("category", "ddl"), zap.String("job", job.String())) + d.notifyReorgWorkerJobStateChange(job) + } + + return dbterror.ErrPausedDDLJob.GenWithStackByArgs(job.ID) +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) } func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { @@ -463,6 +533,8 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) err = rollingbackDropTableOrView(t, job) case model.ActionDropTablePartition: ver, err = rollingbackDropTablePartition(t, job) + case model.ActionExchangeTablePartition: + ver, err = rollingbackExchangeTablePartition(d, t, job) case model.ActionDropSchema: err = rollingbackDropSchema(t, job) case model.ActionRenameIndex: @@ -476,8 +548,20 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition, model.ActionModifySchemaCharsetAndCollate, model.ActionRepairTable, model.ActionModifyTableAutoIdCache, model.ActionAlterIndexVisibility, +<<<<<<< HEAD model.ActionExchangeTablePartition, model.ActionModifySchemaDefaultPlacement: ver, err = cancelOnlyNotHandledJob(job) +======= + model.ActionModifySchemaDefaultPlacement, + model.ActionRecoverSchema, model.ActionAlterCheckConstraint: + ver, err = cancelOnlyNotHandledJob(job, model.StateNone) + case model.ActionMultiSchemaChange: + err = rollingBackMultiSchemaChange(job) + case model.ActionAddCheckConstraint: + ver, err = rollingBackAddConstraint(d, t, job) + case model.ActionDropCheckConstraint: + ver, err = rollingBackDropConstraint(t, job) +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) default: job.State = model.JobStateCancelled err = errCancelledDDLJob diff --git a/executor/insert_common.go b/executor/insert_common.go index b93d1b01c19ac..407e9cc9f08c4 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -655,6 +655,33 @@ func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue } } } +<<<<<<< HEAD +======= + tbl := e.Table.Meta() + // Handle exchange partition + if tbl.ExchangePartitionInfo != nil { + is := e.Ctx().GetDomainInfoSchema().(infoschema.InfoSchema) + pt, tableFound := is.TableByID(tbl.ExchangePartitionInfo.ExchangePartitionID) + if !tableFound { + return nil, errors.Errorf("exchange partition process table by id failed") + } + p, ok := pt.(table.PartitionedTable) + if !ok { + return nil, errors.Errorf("exchange partition process assert table partition failed") + } + err := p.CheckForExchangePartition( + e.Ctx(), + pt.Meta().Partition, + row, + tbl.ExchangePartitionInfo.ExchangePartitionDefID, + ) + if err != nil { + return nil, err + } + } + sc := e.Ctx().GetSessionVars().StmtCtx + warnCnt := int(sc.WarningCount()) +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) for i, gCol := range gCols { colIdx := gCol.ColumnInfo.Offset val, err := e.GenExprs[i].Eval(chunk.MutRowFromDatums(row).ToRow()) diff --git a/executor/write.go b/executor/write.go index 9b690d1141382..fb048113ca4bf 100644 --- a/executor/write.go +++ b/executor/write.go @@ -92,7 +92,34 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, old } } +<<<<<<< HEAD // 3. Compare datum, then handle some flags. +======= + // Handle exchange partition + tbl := t.Meta() + if tbl.ExchangePartitionInfo != nil { + is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) + pt, tableFound := is.TableByID(tbl.ExchangePartitionInfo.ExchangePartitionID) + if !tableFound { + return false, errors.Errorf("exchange partition process table by id failed") + } + p, ok := pt.(table.PartitionedTable) + if !ok { + return false, errors.Errorf("exchange partition process assert table partition failed") + } + err := p.CheckForExchangePartition( + sctx, + pt.Meta().Partition, + newData, + tbl.ExchangePartitionInfo.ExchangePartitionDefID, + ) + if err != nil { + return false, err + } + } + + // Compare datum, then handle some flags. +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) for i, col := range t.Cols() { // We should use binary collation to compare datum, otherwise the result will be incorrect. cmp, err := newData[i].Compare(sc, &oldData[i], collate.GetBinaryCollator()) diff --git a/infoschema/builder.go b/infoschema/builder.go index 12b8807f84385..54caeed54cee7 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -74,6 +74,29 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return b.applyDropPolicy(diff.SchemaID), nil case model.ActionAlterPlacementPolicy: return b.applyAlterPolicy(m, diff) +<<<<<<< HEAD +======= + case model.ActionCreateResourceGroup: + return nil, b.applyCreateOrAlterResourceGroup(m, diff) + case model.ActionAlterResourceGroup: + return nil, b.applyCreateOrAlterResourceGroup(m, diff) + case model.ActionDropResourceGroup: + return b.applyDropResourceGroup(m, diff), nil + case model.ActionTruncateTablePartition, model.ActionTruncateTable: + return b.applyTruncateTableOrPartition(m, diff) + case model.ActionDropTable, model.ActionDropTablePartition: + return b.applyDropTableOrPartition(m, diff) + case model.ActionRecoverTable: + return b.applyRecoverTable(m, diff) + case model.ActionCreateTables: + return b.applyCreateTables(m, diff) + case model.ActionReorganizePartition: + return b.applyReorganizePartition(m, diff) + case model.ActionExchangeTablePartition: + return b.applyExchangeTablePartition(m, diff) + case model.ActionFlashbackCluster: + return []int64{-1}, nil +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) default: switch diff.Type { case model.ActionTruncateTablePartition, model.ActionTruncateTable: @@ -123,6 +146,47 @@ func (b *Builder) applyDropTableOrParition(m *meta.Meta, diff *model.SchemaDiff) return tblIDs, nil } +func (b *Builder) applyExchangeTablePartition(m *meta.Meta, diff *model.SchemaDiff) ([]int64, error) { + // The partitioned table is not affected until the last stage + if diff.OldTableID == diff.TableID && diff.OldSchemaID == diff.SchemaID { + return b.applyTableUpdate(m, diff) + } + ntSchemaID := diff.OldSchemaID + ntID := diff.OldTableID + ptSchemaID := diff.SchemaID + ptID := diff.TableID + if len(diff.AffectedOpts) > 0 { + // From old version + ptID = diff.AffectedOpts[0].TableID + ptSchemaID = diff.AffectedOpts[0].SchemaID + } + // The normal table needs to be updated first: + // Just update the tables separately + currDiff := &model.SchemaDiff{ + Version: diff.Version, + TableID: ntID, + SchemaID: ntSchemaID, + } + ntIDs, err := b.applyTableUpdate(m, currDiff) + if err != nil { + return nil, errors.Trace(err) + } + b.markPartitionBundleShouldUpdate(ntID) + // Then the partitioned table + currDiff.TableID = ptID + currDiff.SchemaID = ptSchemaID + ptIDs, err := b.applyTableUpdate(m, currDiff) + if err != nil { + return nil, errors.Trace(err) + } + b.markTableBundleShouldUpdate(ptID) + err = updateAutoIDForExchangePartition(b.store, ptSchemaID, ptID, ntSchemaID, ntID) + if err != nil { + return nil, errors.Trace(err) + } + return append(ptIDs, ntIDs...), nil +} + func (b *Builder) applyRecoverTable(m *meta.Meta, diff *model.SchemaDiff) ([]int64, error) { tblIDs, err := b.applyTableUpdate(m, diff) if err != nil { @@ -178,7 +242,7 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 newTableID = diff.TableID case model.ActionDropTable, model.ActionDropView, model.ActionDropSequence: oldTableID = diff.TableID - case model.ActionTruncateTable, model.ActionCreateView, model.ActionExchangeTablePartition: + case model.ActionTruncateTable, model.ActionCreateView: oldTableID = diff.OldTableID newTableID = diff.TableID default: @@ -199,6 +263,7 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 return nil, errors.Trace(err) } case model.ActionRecoverTable: +<<<<<<< HEAD if err := b.applyPlacementUpdate(placement.GroupID(newTableID)); err != nil { return nil, errors.Trace(err) } @@ -206,6 +271,11 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 if err := b.applyPlacementUpdate(placement.GroupID(newTableID)); err != nil { return nil, errors.Trace(err) } +======= + b.markTableBundleShouldUpdate(newTableID) + case model.ActionAlterTablePlacement: + b.markTableBundleShouldUpdate(newTableID) +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) } b.copySortedTables(oldTableID, newTableID) @@ -214,7 +284,6 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 var allocs autoid.Allocators if tableIDIsValid(oldTableID) { if oldTableID == newTableID && (diff.Type != model.ActionRenameTable && diff.Type != model.ActionRenameTables) && - diff.Type != model.ActionExchangeTablePartition && // For repairing table in TiDB cluster, given 2 normal node and 1 repair node. // For normal node's information schema, repaired table is existed. // For repair node's information schema, repaired table is filtered (couldn't find it in `is`). diff --git a/parser/model/model.go b/parser/model/model.go index c1569a1f9e7b8..7cbbfd5600282 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -812,6 +812,15 @@ func (p PartitionType) String() string { return "" } +<<<<<<< HEAD +======= +// ExchangePartitionInfo provides exchange partition info. +type ExchangePartitionInfo struct { + ExchangePartitionID int64 `json:"exchange_partition_id"` + ExchangePartitionDefID int64 `json:"exchange_partition_def_id"` + // Deprecated, not used + XXXExchangePartitionFlag bool `json:"exchange_partition_flag"` +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877)) } // PartitionInfo provides table partition info. diff --git a/table/tables/partition_test.go b/table/tables/partition_test.go index 4d99301684fd3..2fff6849c0072 100644 --- a/table/tables/partition_test.go +++ b/table/tables/partition_test.go @@ -608,3 +608,2537 @@ func TestIssue24746(t *testing.T) { err = tk.ExecToErr("insert into t_24746 partition (p1) values(4,'ERROR, not allowed to read from partition p0',4) on duplicate key update a = a + 1, b = 'ERROR, not allowed to read from p0!'") require.True(t, table.ErrRowDoesNotMatchGivenPartitionSet.Equal(err)) } +<<<<<<< HEAD +======= + +func TestIssue31629(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_enable_list_partition = 1") + tk.MustExec("create database Issue31629") + defer tk.MustExec("drop database Issue31629") + tk.MustExec("use Issue31629") + // Test following partition types: + // HASH, RANGE, LIST: + // - directly on a single int column + // - with expression on multiple columns + // RANGE/LIST COLUMNS single column + // RANGE/LIST COLUMNS -- Verify that only single column is allowed and no expression + tests := []struct { + create string + fail bool + cols []string + }{ + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by range(col1) (partition p0 values less than (5),partition p1 values less than (10), partition p2 values less than maxvalue)", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by range(Col1+col3) (partition p0 values less than (5),partition p1 values less than (10), partition p2 values less than maxvalue)", false, []string{"Col1", "col3"}}, + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by hash(col1) partitions 3", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by hash(Col1+col3) partitions 3", false, []string{"Col1", "col3"}}, + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by list(col1) (partition p0 values in (5,6,7,8,9),partition p1 values in (10,11,12,13,14), partition p2 values in (20,21,22,23,24))", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by list(Col1+col3) (partition p0 values in (5,6,7,8,9),partition p1 values in (10,11,12,13,14), partition p2 values in (20,21,22,23,24))", false, []string{"Col1", "col3"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col2) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, false, []string{"col2"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col2,col3) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col1+1) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col2) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, false, []string{"col2"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col2,col3) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col1+1) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, true, nil}, + } + + for i, tt := range tests { + createTable := "create table t1 " + tt.create + res, err := tk.Exec(createTable) + if res != nil { + res.Close() + } + if err != nil { + if tt.fail { + continue + } + } + require.Falsef(t, tt.fail, "test %d succeeded but was expected to fail! %s", i, createTable) + require.NoError(t, err) + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tb, err := dom.InfoSchema().TableByName(model.NewCIStr("Issue31629"), model.NewCIStr("t1")) + require.NoError(t, err) + tbp, ok := tb.(table.PartitionedTable) + require.Truef(t, ok, "test %d does not generate a table.PartitionedTable: %s (%T, %+v)", i, createTable, tb, tb) + colNames := tbp.GetPartitionColumnNames() + checkNames := []model.CIStr{model.NewCIStr(tt.cols[0])} + for i := 1; i < len(tt.cols); i++ { + checkNames = append(checkNames, model.NewCIStr(tt.cols[i])) + } + require.ElementsMatchf(t, colNames, checkNames, "test %d %s", i, createTable) + tk.MustExec("drop table t1") + } +} + +func TestExchangePartitionStates(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + dbName := "partSchemaVer" + tk.MustExec("create database " + dbName) + tk.MustExec("use " + dbName) + tk.MustExec(`set @@global.tidb_enable_metadata_lock = ON`) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + dbName) + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec("use " + dbName) + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec("use " + dbName) + tk.MustExec(`create table t (a int primary key, b varchar(255), key (b))`) + tk.MustExec(`create table tp (a int primary key, b varchar(255), key (b)) partition by range (a) (partition p0 values less than (1000000), partition p1M values less than (2000000))`) + tk.MustExec(`insert into t values (1, "1")`) + tk.MustExec(`insert into tp values (2, "2")`) + tk.MustExec(`analyze table t,tp`) + tk.MustExec("BEGIN") + tk.MustQuery(`select * from t`).Check(testkit.Rows("1 1")) + tk.MustQuery(`select * from tp`).Check(testkit.Rows("2 2")) + alterChan := make(chan error) + go func() { + // WITH VALIDATION is the default + err := tk2.ExecToErr(`alter table tp exchange partition p0 with table t`) + alterChan <- err + }() + waitFor := func(tableName, s string, pos int) { + for { + select { + case alterErr := <-alterChan: + require.Fail(t, "Alter completed unexpectedly", "With error %v", alterErr) + default: + // Alter still running + } + res := tk4.MustQuery(`admin show ddl jobs where db_name = '` + strings.ToLower(dbName) + `' and table_name = '` + tableName + `' and job_type = 'exchange partition'`).Rows() + if len(res) == 1 && res[0][pos] == s { + logutil.BgLogger().Info("Got state", zap.String("State", s)) + break + } + gotime.Sleep(50 * gotime.Millisecond) + } + } + waitFor("t", "write only", 4) + tk3.MustExec(`BEGIN`) + tk3.MustExec(`insert into t values (4,"4")`) + tk3.MustContainErrMsg(`insert into t values (1000004,"1000004")`, "[table:1748]Found a row not matching the given partition set") + tk.MustExec(`insert into t values (5,"5")`) + // This should fail the alter table! + tk.MustExec(`insert into t values (1000005,"1000005")`) + + // MDL will block the alter to not continue until all clients + // are in StateWriteOnly, which tk is blocking until it commits + tk.MustExec(`COMMIT`) + waitFor("t", "rollback done", 11) + // MDL will block the alter from finish, tk is in 'rollbacked' schema version + // but the alter is still waiting for tk3 to commit, before continuing + tk.MustExec("BEGIN") + tk.MustExec(`insert into t values (1000006,"1000006")`) + tk.MustExec(`insert into t values (6,"6")`) + tk3.MustExec(`insert into t values (7,"7")`) + tk3.MustContainErrMsg(`insert into t values (1000007,"1000007")`, + "[table:1748]Found a row not matching the given partition set") + tk3.MustExec("COMMIT") + require.ErrorContains(t, <-alterChan, + "[ddl:1737]Found a row that does not match the partition") + tk3.MustExec(`BEGIN`) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows( + "1 1", "1000005 1000005", "1000006 1000006", "5 5", "6 6")) + tk.MustQuery(`select * from tp`).Sort().Check(testkit.Rows("2 2")) + tk3.MustQuery(`select * from t`).Sort().Check(testkit.Rows( + "1 1", "1000005 1000005", "4 4", "5 5", "7 7")) + tk3.MustQuery(`select * from tp`).Sort().Check(testkit.Rows("2 2")) + tk.MustContainErrMsg(`insert into t values (7,"7")`, + "[kv:1062]Duplicate entry '7' for key 't.PRIMARY'") + tk.MustExec(`insert into t values (8,"8")`) + tk.MustExec(`insert into t values (1000008,"1000008")`) + tk.MustExec(`insert into tp values (9,"9")`) + tk.MustExec(`insert into tp values (1000009,"1000009")`) + tk3.MustExec(`insert into t values (10,"10")`) + tk3.MustExec(`insert into t values (1000010,"1000010")`) + + tk3.MustExec(`COMMIT`) + tk.MustQuery(`show create table tp`).Check(testkit.Rows("" + + "tp CREATE TABLE `tp` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (1000000),\n" + + " PARTITION `p1M` VALUES LESS THAN (2000000))")) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustExec(`commit`) + tk.MustExec(`insert into t values (11,"11")`) + tk.MustExec(`insert into t values (1000011,"1000011")`) + tk.MustExec(`insert into tp values (12,"12")`) + tk.MustExec(`insert into tp values (1000012,"1000012")`) +} + +func TestAddKeyPartitionStates(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + dbName := "partSchemaVer" + tk.MustExec("create database " + dbName) + tk.MustExec("use " + dbName) + tk.MustExec(`set @@global.tidb_enable_metadata_lock = ON`) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + dbName) + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec("use " + dbName) + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec("use " + dbName) + tk.MustExec(`create table t (a int primary key, b varchar(255), key (b)) partition by hash (a) partitions 3`) + tk.MustExec(`insert into t values (1, "1")`) + tk.MustExec(`analyze table t`) + tk.MustExec("BEGIN") + tk.MustQuery(`select * from t`).Check(testkit.Rows("1 1")) + tk.MustExec(`insert into t values (2, "2")`) + syncChan := make(chan bool) + go func() { + tk2.MustExec(`alter table t add partition partitions 1`) + syncChan <- true + }() + waitFor := func(i int, s string) { + for { + res := tk4.MustQuery(`admin show ddl jobs where db_name = '` + strings.ToLower(dbName) + `' and table_name = 't' and job_type like 'alter table%'`).Rows() + if len(res) == 1 && res[0][i] == s { + break + } + gotime.Sleep(10 * gotime.Millisecond) + } + } + waitFor(4, "delete only") + tk3.MustExec(`BEGIN`) + tk3.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1")) + tk3.MustExec(`insert into t values (3,"3")`) + + tk.MustExec(`COMMIT`) + waitFor(4, "write only") + tk.MustExec(`BEGIN`) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "2 2")) + tk.MustExec(`insert into t values (4,"4")`) + + tk3.MustExec(`COMMIT`) + waitFor(4, "write reorganization") + tk3.MustExec(`BEGIN`) + tk3.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY HASH (`a`) PARTITIONS 3")) + tk3.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "2 2", "3 3")) + tk3.MustExec(`insert into t values (5,"5")`) + + tk.MustExec(`COMMIT`) + waitFor(4, "delete reorganization") + tk.MustExec(`BEGIN`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY HASH (`a`) PARTITIONS 4")) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "2 2", "3 3", "4 4")) + tk.MustExec(`insert into t values (6,"6")`) + + tk3.MustExec(`COMMIT`) + tk.MustExec(`COMMIT`) + <-syncChan + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "2 2", "3 3", "4 4", "5 5", "6 6")) +} + +type compoundSQL struct { + selectSQL string + point bool + batchPoint bool + pruned bool + executeExplain bool + usedPartition []string + notUsedPartition []string + rowCount int +} + +type partTableCase struct { + partitionbySQL string + selectInfo []compoundSQL +} + +func executePartTableCase(t *testing.T, tk *testkit.TestKit, testCases []partTableCase, + createSQL string, insertSQLs []string, dropSQL string) { + for i, testCase := range testCases { + // create table ... partition by key ... + ddlSQL := createSQL + testCase.partitionbySQL + fmt.Println(i, ":", ddlSQL) + executeSQLWrapper(t, tk, ddlSQL) + // insert data + for _, insertsql := range insertSQLs { + executeSQLWrapper(t, tk, insertsql) + } + // execute testcases + for j, selInfo := range testCase.selectInfo { + fmt.Println(j, ":", selInfo.selectSQL) + tk.MustQuery(selInfo.selectSQL).Check(testkit.Rows(strconv.Itoa(selInfo.rowCount))) + if selInfo.executeExplain { + result := tk.MustQuery("EXPLAIN " + selInfo.selectSQL) + if selInfo.point { + result.CheckContain("Point_Get") + } + if selInfo.batchPoint { + result.CheckContain("Batch_Point_Get") + } + if selInfo.pruned { + for _, part := range selInfo.usedPartition { + result.CheckContain(part) + } + for _, part := range selInfo.notUsedPartition { + result.CheckNotContain(part) + } + } + } + } + executeSQLWrapper(t, tk, dropSQL) + } +} + +func executeSQLWrapper(t *testing.T, tk *testkit.TestKit, SQLString string) { + res, err := tk.Exec(SQLString) + if res != nil { + res.Close() + } + require.Nil(t, err) +} + +func TestKeyPartitionTableBasic(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb") + defer tk.MustExec("drop database partitiondb") + tk.MustExec("use partitiondb") + testCases := []struct { + createSQL string + dropSQL string + insertSQL string + selectInfo []compoundSQL + }{ + { + createSQL: "CREATE TABLE tkey0 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3)) PARTITION BY KEY(col3) PARTITIONS 4", + insertSQL: "INSERT INTO tkey0 VALUES(1, '2023-02-22', 1, 1), (2, '2023-02-22', 2, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey0", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 = 3", + true, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:p0", "partition:p3"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 >1 AND col3 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 2, + }, + }, + + dropSQL: "DROP TABLE IF EXISTS tkey0", + }, + { + createSQL: "CREATE TABLE tkey7 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3,col1)) PARTITION BY KEY(col3,col1) PARTITIONS 4", + insertSQL: "INSERT INTO tkey7 VALUES(1, '2023-02-22', 1, 1), (1, '2023-02-22', 2, 1),(2, '2023-02-22', 2, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4),(4, '2023-02-22', 5, 4)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey7", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 and col1 = 3", + true, false, true, true, []string{"partition:p1"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 and col1 = 3 OR col3 = 4 and col1 = 4", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey7 WHERE col1>1 and col3 >1 AND col3 < 4 and col1<3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey7", + }, + { + createSQL: "CREATE TABLE tkey8 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,PRIMARY KEY (col3,col1)) PARTITION BY KEY(col3,col1) PARTITIONS 4", + insertSQL: "INSERT INTO tkey8 VALUES(1, '2023-02-22', 111, 1), (1, '2023-02-22', 2, 1),(2, '2023-02-22', 218, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4),(4, '2023-02-22', 5, 4),(5, '2023-02-22', 5, 5),(5, '2023-02-22', 50, 2),(6, '2023-02-22', 62, 2),(60, '2023-02-22', 6, 5),(70, '2023-02-22', 50, 2),(80, '2023-02-22', 62, 2),(100, '2023-02-22', 62, 2),(2000, '2023-02-22', 6, 5),(400, '2023-02-22', 50, 2),(90, '2023-02-22', 62, 2)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey8", + false, false, false, false, []string{}, []string{}, 16, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 7, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 and col1 = 3", + true, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 and col1 = 3 OR col3 = 4 and col1 = 4", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col1>1 and col3 >1 AND col3 < 4 and col1<3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 0, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey8", + }, + { + createSQL: "CREATE TABLE tkey6 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 VARCHAR(12) NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3)) PARTITION BY KEY(col3) PARTITIONS 4", + insertSQL: "INSERT INTO tkey6 VALUES(1, '2023-02-22', 'linpin', 1), (2, '2023-02-22', 'zhangsan', 2), (3, '2023-02-22', 'anqila', 3), (4, '2023-02-22', 'xingtian', 4),(1, '2023-02-22', 'renleifeng', 5), (2, '2023-02-22', 'peilin', 2),(1, '2023-02-22', 'abcdeeg', 7), (2, '2023-02-22', 'rpstdfed', 8)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey6", + false, false, false, false, []string{}, []string{}, 8, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 = 'linpin'", + true, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 = 'zhangsan' or col3 = 'linpin'", + true, true, true, true, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 > 'linpin' AND col3 < 'qing'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey6", + }, + { + createSQL: "CREATE TABLE tkey2 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(KHH) partitions 4", + insertSQL: "INSERT INTO tkey2 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey2", + false, false, false, false, []string{}, []string{}, 17, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH > 'nanjing' AND KHH < 'suzhou'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey2", + }, + { + createSQL: "CREATE TABLE tkey5 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (KHH, JYRQ, ZJZH))PARTITION BY KEY(KHH) partitions 4", + insertSQL: "INSERT INTO tkey5 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey5", + false, false, false, false, []string{}, []string{}, 17, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH > 'nanjing' AND KHH < 'suzhou'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey5", + }, + { + createSQL: "CREATE TABLE tkey4 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(JYRQ, KHH) partitions 4", + insertSQL: "INSERT INTO tkey4 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530'),(1,'beijing','010'),(2,'beijing','010'),(2,'zzzzwuhan','027')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey4", + false, false, false, false, []string{}, []string{}, 20, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 7, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2 or KHH = 'zhenjiang' and JYRQ = 3", + false, false, true, true, []string{"partition:p0", "partition:p1"}, []string{"partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2 or KHH = 'zhenjiang' and JYRQ = 3 or KHH = 'HUAIAN' and JYRQ = 15", + false, false, true, true, []string{"partition:p0", "partition:p1"}, []string{"partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2 OR JYRQ = 3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2 OR JYRQ = 3 OR JYRQ = 15", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ >6 AND JYRQ < 10", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ >6 and KHH>'lianyungang' AND JYRQ < 10 and KHH<'xuzhou'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey4", + }, + { + createSQL: "CREATE TABLE tkey9 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(JYRQ, KHH, ZJZH) partitions 4", + insertSQL: "INSERT INTO tkey9 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530'),(1,'beijing','010'),(2,'beijing','010'),(2,'zzzzwuhan','027')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey9", + false, false, false, false, []string{}, []string{}, 20, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 8, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH = '0517'", + true, false, true, true, []string{"partition:p0"}, []string{"partition:p3", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH='0517' or KHH = 'zhenjiang' and JYRQ = 3 and ZJZH = '0518'", + false, false, true, true, []string{"partition:p3", "partition:p0"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH='0517' or KHH = 'zhenjiang' and JYRQ = 3 and ZJZH = '0518' or KHH = 'NANJING' and JYRQ = 14 and ZJZH = '025'", + false, false, true, true, []string{"partition:p0", "partition:p3"}, []string{"partition:p2", "partition:p1"}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 OR JYRQ = 3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 OR JYRQ = 3 OR JYRQ = 15", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ >6 AND JYRQ < 10", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 and KHH = 'huaian' OR JYRQ = 3 and KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey9", + }, + } + + for i, testCase := range testCases { + fmt.Println(i, ":", testCase.createSQL) + executeSQLWrapper(t, tk, testCase.createSQL) + executeSQLWrapper(t, tk, testCase.insertSQL) + for j, selInfo := range testCase.selectInfo { + fmt.Println(j, ":", selInfo.selectSQL) + tk.MustQuery(selInfo.selectSQL).Check(testkit.Rows(strconv.Itoa(selInfo.rowCount))) + if selInfo.executeExplain { + result := tk.MustQuery("EXPLAIN " + selInfo.selectSQL) + if selInfo.point { + result.CheckContain("Point_Get") + } + if selInfo.batchPoint { + result.CheckContain("Batch_Point_Get") + } + if selInfo.pruned { + for _, part := range selInfo.usedPartition { + result.CheckContain(part) + } + } + } + } + executeSQLWrapper(t, tk, testCase.dropSQL) + } +} + +func TestKeyPartitionTableAllFeildType(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb3") + defer tk.MustExec("drop database partitiondb3") + tk.MustExec("use partitiondb3") + // partition column is numeric family + createSQL := "create table tkey_numeric(\n" + + "id1 BIT(8) not null,\n" + + "id2 TINYINT not null,\n" + + "id3 BOOL not null,\n" + + "id4 SMALLINT not null,\n" + + "id5 MEDIUMINT not null,\n" + + "id6 INT not null,\n" + + "id7 BIGINT not null,\n" + + "id8 DECIMAL(12,4) not null,\n" + + "id9 FLOAT not null,\n" + + "id10 DOUBLE not null,\n" + + "name varchar(20),\n" + + "primary key(id1,id2,id3,id4,id5,id6,id7,id8,id9,id10)\n" + + ")\n" + dropSQL := "drop table tkey_numeric" + insertSQLS := []string{ + "INSERT INTO tkey_numeric VALUES(1,1,0,1,1,1,1,1.1,120.1,367.45,'linpin'),(12,12,12,12,12,12,12,12.1,1220.1,3267.45,'anqila')", + "INSERT INTO tkey_numeric VALUES(0,2,1,2,2,2,2,2.78,16.78,17.25,'ring'),(33,33,33,33,33,33,33,33.78,336.78,37.25,'black')", + "INSERT INTO tkey_numeric VALUES(2,3,1,3,3,3,3,3.78,26.78,417.25,'liudehua'),(22,23,21,23,23,23,23,32.78,26.72,27.15,'chenchen')", + "INSERT INTO tkey_numeric VALUES(3,3,2,4,4,4,4,4.78,46.48,89.35,'guofucheng'), (4,4,4,5,5,5,5,5.78,56.48,59.35,'zhangxuyou')", + "INSERT INTO tkey_numeric VALUES(5,5,5,5,5,5,5,5.78,56.48,59.35,'xietingfeng'),(34,34,34,34,34,34,34,34.78,346.78,34.25,'dongxu')", + "INSERT INTO tkey_numeric VALUES(250,120,120,250,250,258,348,38.78,186.48,719.35,'chenguanxi'),(35,35,35,250,35,35,35,35.78,356.48,35.35,'chenguanxi')", + } + testCases := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 5, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 = 3", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 = 3 or id1 = 4", + false, false, true, true, []string{"partition:p0", "partition:p3"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 >1 AND id1 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id2) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 = 3", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p0", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 = 3 or id2 = 4", + false, false, true, true, []string{"partition:p0", "partition:p3"}, []string{"partition:p1", "partition:p2"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 >1 AND id2 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p0", "partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 = 5 or id3 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 >1 AND id3 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 5, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 = 5 or id4 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 >1 AND id4 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id5) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p3", "partition:p0"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 = 5 or id5 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 >1 AND id5 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id6) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 = 5 or id6 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 >1 AND id6 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id7) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 = 5 or id7 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 >1 AND id7 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id8) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 = 1.1", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p2", "partition:p0", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 = 1.1 or id8 = 33.78", + false, false, true, true, []string{"partition:p0", "partition:p1"}, []string{"partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 >1 AND id8 < 4", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id9) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 = 46.48", + false, false, true, true, []string{}, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 = 46.48 or id9 = 336.78", + false, false, true, true, []string{}, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 >45 AND id9 < 47", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id10) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 = 46.48", + false, false, true, true, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 = 46.48 or id10 = 336.78", + false, false, true, true, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 >366 AND id10 < 368", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases, createSQL, insertSQLS, dropSQL) + + // partition column is date/time family + createSQL2 := "create table tkey_datetime(\n" + + "id1 DATE not null,\n" + + "id2 TIME not null,\n" + + "id3 DATETIME not null,\n" + + "id4 TIMESTAMP not null,\n" + + "id5 YEAR not null,\n" + + "name varchar(20),\n" + + "primary key(id1, id2, id3, id4, id5)\n" + + ")\n" + dropSQL2 := "drop table tkey_datetime" + insertSQLS2 := []string{ + "insert into tkey_datetime values('2012-04-10', '12:12:12', '2012-04-10 12:12:12', '2012-04-10 12:12:12.12345', 2012, 'linpin')", + "insert into tkey_datetime values('2013-05-11', '13:13:13', '2013-05-11 13:13:13', '2013-05-11 13:13:13.43133', 2013, 'minghua')", + "insert into tkey_datetime values('2014-06-12', '14:14:14', '2014-06-12 14:14:14', '2014-06-12 14:14:14.32344', 2014, 'oyangfeng')", + "insert into tkey_datetime values('2015-07-13', '15:15:15', '2015-07-13 15:15:15', '2015-07-13 15:15:15.42544', 2015, 'pengdehuai')", + "insert into tkey_datetime values('2021-08-14', '16:16:16', '2021-08-14 16:16:16', '2021-08-14 16:16:16.18945', 2021, 'shenwanshan')", + "insert into tkey_datetime values('2022-12-23', '23:12:15', '2022-12-23 23:12:15', '2022-12-23 23:12:15.43133', 2022, 'tangchap')", + "insert into tkey_datetime values('2023-01-12', '20:38:14', '2023-01-12 20:38:14', '2023-01-12 20:38:14.32344', 2023, 'xinyu')", + "insert into tkey_datetime values('2018-07-13', '07:15:15', '2018-07-13 07:15:15', '2018-07-13 07:15:15.42544', 2018, 'zongyang')", + "insert into tkey_datetime values('1980-01-30', '00:12:15', '1980-01-30 00:12:15', '1980-01-30 00:12:15.42544', 1980, 'MAYUWEI')", + "insert into tkey_datetime values('1980-03-30', '00:13:15', '1980-03-30 00:13:15', '1980-03-30 00:13:15.42544', 1980, 'maqinwei')", + } + testCases2 := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 = '2012-04-10'", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 = '2012-04-10' or id1 = '2018-07-13'", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 >'2012-04-10' AND id1 < '2014-04-10'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 = '2012-04-10 12:12:12'", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 = '2012-04-10 12:12:12' or id3 = '2021-08-14 16:16:16'", + false, false, true, true, []string{"partition:p3", "partition:p1"}, []string{"partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 >'2012-04-10 12:12:12' AND id3 < '2014-04-10 12:12:12'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 = '2012-04-10 12:12:12'", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 = '2012-04-10 12:12:12' or id4 = '2021-08-14 16:16:16'", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 >'2012-04-10 12:12:12' AND id4 < '2014-04-10 12:12:12'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id5) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 = 2012", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 = 2012 or id5 = 2018", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 >2012 AND id5 < 2014", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p3", "partition:p0"}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases2, createSQL2, insertSQLS2, dropSQL2) + + // partition column is string family + createSQL3 := "create table tkey_string(\n" + + "id1 CHAR(16) not null,\n" + + "id2 VARCHAR(16) not null,\n" + + "id3 BINARY(16) not null,\n" + + "id4 VARBINARY(16) not null,\n" + + "id5 BLOB not null,\n" + + "id6 TEXT not null,\n" + + "id7 ENUM('x-small', 'small', 'medium', 'large', 'x-large') not null,\n" + + "id8 SET ('a', 'b', 'c', 'd') not null,\n" + + "name varchar(16),\n" + + "primary key(id1, id2, id3, id4, id7, id8)\n" + + ")\n" + dropSQL3 := "drop table tkey_string" + insertSQLS3 := []string{ + "INSERT INTO tkey_string VALUES('huaian','huaian','huaian','huaian','huaian','huaian','x-small','a','linpin')", + "INSERT INTO tkey_string VALUES('nanjing','nanjing','nanjing','nanjing','nanjing','nanjing','small','b','linpin')", + "INSERT INTO tkey_string VALUES('zhenjiang','zhenjiang','zhenjiang','zhenjiang','zhenjiang','zhenjiang','medium','c','linpin')", + "INSERT INTO tkey_string VALUES('suzhou','suzhou','suzhou','suzhou','suzhou','suzhou','large','d','linpin')", + "INSERT INTO tkey_string VALUES('wuxi','wuxi','wuxi','wuxi','wuxi','wuxi','x-large','a','linpin')", + } + testCases3 := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p0", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 = 'huaian' or id1 = 'suzhou'", + false, false, true, true, []string{"partition:p3", "partition:p0"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 >'huaian' AND id1 < 'suzhou'", + false, false, true, true, []string{"partition:p1", "partition:p2", "partition:p0", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id2) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 = 'huaian' or id2 = 'suzhou'", + false, false, true, true, []string{"partition:p3", "partition:p0"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 >'huaian' AND id2 < 'suzhou'", + false, false, true, true, []string{"partition:p1", "partition:p2", "partition:p0", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 = 0x73757A686F7500000000000000000000", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 = 0x73757A686F7500000000000000000000 or id3 = 0x6E616E6A696E67000000000000000000", + false, false, true, true, []string{"partition:p0", "partition:p1"}, []string{"partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 >0x67756169616E00000000000000000000 AND id3 < 0x6E616E6A696E67000000000000000000", + false, false, true, true, []string{"partition:p1", "partition:p0", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 = 0x68756169616E", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p0", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 = 0x68756169616E or id4 = 0x73757A686F75", + false, false, true, true, []string{"partition:p3", "partition:p0"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 >0x73757A686F75 AND id4 < 0x78757869", + false, false, true, true, []string{"partition:p1", "partition:p2", "partition:p0", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id7) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 = 'x-small'", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 = 'x-small' or id7 = 'large'", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 > 'large' AND id7 < 'x-small'", + false, false, true, true, []string{"partition:p1", "partition:p0", "partition:p3"}, []string{"partition:p2"}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id8) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 = 'a'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 = 'a' or id8 = 'b'", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 > 'a' AND id8 < 'c'", + false, false, true, true, []string{"partition:p1", "partition:p2", "partition:p0", "partition:p3"}, []string{}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases3, createSQL3, insertSQLS3, dropSQL3) +} + +func TestKeyPartitionTableMixed(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb2") + defer tk.MustExec("drop database partitiondb2") + tk.MustExec("use partitiondb2") + // SHOW CREATE TABLE + tk.MustExec("CREATE TABLE tkey1 (col1 INT NOT NULL, col2 DATE NOT NULL,col3 INT NOT NULL, col4 INT NOT NULL, UNIQUE KEY (col3))" + + " PARTITION BY KEY(col3)" + + "(PARTITION `p0`," + + "PARTITION `p1`," + + "PARTITION `p2`," + + "PARTITION `p3`)") + tk.MustQuery("show create table tkey1").Check(testkit.Rows("tkey1 CREATE TABLE `tkey1` (\n" + + " `col1` int(11) NOT NULL,\n" + + " `col2` date NOT NULL,\n" + + " `col3` int(11) NOT NULL,\n" + + " `col4` int(11) NOT NULL,\n" + + " UNIQUE KEY `col3` (`col3`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`col3`) PARTITIONS 4")) + + // BLOB, JSON don't support key partition + err := tk.ExecToErr("create table tkey_string(\n" + + "id5 BLOB not null,\n" + + "id6 TEXT not null,\n" + + "name varchar(16)\n" + + ") PARTITION BY KEY(id5) partitions 4\n") + require.Error(t, err) + require.Regexp(t, "Field 'id5' is of a not allowed type for this type of partitioning", err) + + // BLOB, JSON don't support key partition + err = tk.ExecToErr("create table tkey_string2(\n" + + "id5 BLOB not null,\n" + + "id6 TEXT not null,\n" + + "name varchar(16)\n" + + ") PARTITION BY KEY(id6) partitions 4\n") + require.Error(t, err) + require.Regexp(t, "Field 'id6' is of a not allowed type for this type of partitioning", err) + + err = tk.ExecToErr("CREATE TABLE tkey_json (c1 JSON) PARTITION BY KEY(c1) partitions 4") + require.Error(t, err) + require.Regexp(t, "Field 'c1' is of a not allowed type for this type of partitioning", err) + + // It doesn't support LINEAR KEY partition + tk.MustExec("CREATE TABLE tkey_linear (col1 INT, col2 CHAR(5), col3 DATE) PARTITION BY LINEAR KEY(col3) PARTITIONS 5") + result := tk.MustQuery("show warnings") + result.CheckContain("LINEAR KEY is not supported, using non-linear KEY instead") + + // It will ignore ALGORITHM=1|2 + tk.MustExec("CREATE TABLE tkey_algorithm1 (col1 INT, col2 CHAR(5), col3 DATE) PARTITION BY KEY ALGORITHM=1 (col3) PARTITIONS 5") + tk.MustExec("CREATE TABLE tkey_algorithm2 (col1 INT, col2 CHAR(5), col3 DATE) PARTITION BY KEY ALGORITHM=2 (col3) PARTITIONS 5") + + err = tk.ExecToErr("CREATE TABLE tkey_algorithm3 (col1 INT, col2 CHAR(5), col3 DATE) PARTITION BY KEY ALGORITHM=3 (col3) PARTITIONS 5") + require.Error(t, err) + require.Regexp(t, "You have an error in your SQL syntax", err) + + // Key partition can't be as subpartition + tk.MustContainErrMsg("CREATE TABLE tkey_subpartition1 (a INT not null,b VARCHAR(12) not null,c CHAR(14) not null,primary key (a, b, c)) PARTITION BY KEY (a) SUBPARTITION BY KEY(b) SUBPARTITIONS 2", "[ddl:1500]It is only possible to mix RANGE/LIST partitioning with HASH/KEY partitioning for subpartitioning") + + tk.MustExec("CREATE TABLE tkey_subpartition1 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))" + + "PARTITION BY RANGE(JYRQ)\n" + + "SUBPARTITION BY KEY(KHH) SUBPARTITIONS 2 \n" + + "(\n" + + "PARTITION p0 VALUES LESS THAN (8),\n" + + "PARTITION p1 VALUES LESS THAN (16),\n" + + "PARTITION p2 VALUES LESS THAN MAXVALUE\n" + + ")") + result = tk.MustQuery("show warnings") + result.CheckContain("Unsupported subpartitioning, only using RANGE partitioning") + + // It ignores /*!50100 */ format + tk.MustExec("CREATE TABLE tkey10 (`col1` int, `col2` char(5),`col3` date)" + + "/*!50100 PARTITION BY KEY (col3) PARTITIONS 5 */") + result = tk.MustQuery("show create table tkey10") + result.Check(testkit.Rows("tkey10 CREATE TABLE `tkey10` (\n" + + " `col1` int(11) DEFAULT NULL,\n" + + " `col2` char(5) DEFAULT NULL,\n" + + " `col3` date DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`col3`) PARTITIONS 5")) + + // It ignores /*!50100 */ format, but doesn't ignore specified partition names + tk.MustExec("CREATE TABLE tkey11 (`col1` int, `col2` char(5),`col3` date)" + + "/*!50100 PARTITION BY KEY (col1) PARTITIONS 4 \n" + + "(PARTITION `pp0`,\n" + + "PARTITION `pp1`,\n" + + "PARTITION `pp2`,\n" + + "PARTITION `pp3`)\n" + + "*/") + result = tk.MustQuery("show create table tkey11") + result.Check(testkit.Rows("tkey11 CREATE TABLE `tkey11` (\n" + + " `col1` int(11) DEFAULT NULL,\n" + + " `col2` char(5) DEFAULT NULL,\n" + + " `col3` date DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`col1`)\n" + + "(PARTITION `pp0`,\n" + + " PARTITION `pp1`,\n" + + " PARTITION `pp2`,\n" + + " PARTITION `pp3`)")) + + // It shows the comment defined in the ddl + tk.MustExec("CREATE TABLE tkey12 (`col1` int, `col2` char(5),`col3` date)" + + "PARTITION BY KEY (col1) \n" + + "(PARTITION `pp0` comment 'huaian',\n" + + "PARTITION `pp1` comment 'nanjing',\n" + + "PARTITION `pp2` comment 'zhenjiang',\n" + + "PARTITION `pp3` comment 'suzhou')\n") + result = tk.MustQuery("show create table tkey12") + result.Check(testkit.Rows("tkey12 CREATE TABLE `tkey12` (\n" + + " `col1` int(11) DEFAULT NULL,\n" + + " `col2` char(5) DEFAULT NULL,\n" + + " `col3` date DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`col1`)\n" + + "(PARTITION `pp0` COMMENT 'huaian',\n" + + " PARTITION `pp1` COMMENT 'nanjing',\n" + + " PARTITION `pp2` COMMENT 'zhenjiang',\n" + + " PARTITION `pp3` COMMENT 'suzhou')")) + + // It shows the placement policy defined in the ddl + tk.MustExec("drop placement policy if exists fivereplicas") + tk.MustExec("CREATE PLACEMENT POLICY fivereplicas FOLLOWERS=4") + tk.MustExec("CREATE TABLE tkey13 (`col1` int, `col2` char(5),`col3` date) placement policy fivereplicas\n" + + "PARTITION BY KEY (col1) PARTITIONS 4") + result = tk.MustQuery("show create table tkey13") + result.Check(testkit.Rows("tkey13 CREATE TABLE `tkey13` (\n" + + " `col1` int(11) DEFAULT NULL,\n" + + " `col2` char(5) DEFAULT NULL,\n" + + " `col3` date DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`fivereplicas` */\n" + + "PARTITION BY KEY (`col1`) PARTITIONS 4")) + + // The partition column can has null value + tk.MustExec("CREATE TABLE tkey14 (`col1` int, `col2` int,`col3` int, col4 int)\n" + + "PARTITION BY KEY (col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey14 values(20,1,1,1),(1,2,NULL,2),(3,3,3,3),(3,3,NULL,3),(4,4,4,4),(5,5,5,5),(6,6,null,6),(7,7,7,7),(8,8,8,8),(9,9,9,9),(10,10,10,5),(11,11,11,6),(12,12,12,12),(13,13,13,13),(14,14,null,14)") + tk.MustQuery("SELECT count(*) FROM tkey14 WHERE col3 = NULL").Check(testkit.Rows("0")) + tk.MustQuery("SELECT count(*) FROM tkey14 WHERE col3 IS NULL").Check(testkit.Rows("4")) + result = tk.MustQuery("EXPLAIN SELECT count(*) FROM tkey14 WHERE col3 IS NULL") + result.CheckContain("partition:p1") + result.MultiCheckNotContain([]string{"partition:p0", "partition:p2", "partition:p3"}) + + tk.MustExec("CREATE TABLE tkey15 (`col1` int, col2 DATE NOT NULL,col3 VARCHAR(12), col4 int)\n" + + "PARTITION BY KEY (col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey15 VALUES(1, '2023-02-22', 'linpin', 1), (2, '2023-02-22', NULL, 2), (3, '2023-02-22', 'anqila', 3), (4, '2023-02-22', NULL, 4)") + result = tk.MustQuery("EXPLAIN SELECT count(*) FROM tkey15 WHERE col3 IS NULL") + result.CheckContain("partition:p1") + result.MultiCheckNotContain([]string{"partition:p0", "partition:p2", "partition:p3"}) + + tk.MustExec("CREATE TABLE tkey12_2 (col1 INT, col2 INT ,col3 INT ,col4 INT , UNIQUE KEY(col2, col3)" + + ") PARTITION BY KEY(col2, col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey12_2 values(20,1,1,1),(1,2,NULL,2),(3,3,3,3),(3,3,NULL,3),(4,4,4,4)," + + "(5,5,5,5), (6,6,null,6),(7,7,7,7),(8,8,8,8),(9,9,9,9),(10,10,10,5),(11,11,11,6),(12,12,12,12)," + + "(13,13,13,13),(14,14,null,14)") + result = tk.MustQuery("EXPLAIN SELECT * FROM tkey12_2 WHERE col2 = 2 and col3 IS NULL") + result.MultiCheckNotContain([]string{"partition:p1", "partition:p0", "partition:p3"}) + tk.MustQuery("SELECT * FROM tkey12_2 WHERE col2 = 2 and col3 IS NULL").Check(testkit.Rows("1 2 2")) + result = tk.MustQuery("EXPLAIN SELECT * FROM tkey12_2 WHERE col2 = 2") + result.MultiCheckContain([]string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}) + tk.MustQuery("SELECT * FROM tkey12_2 WHERE col2 = 2").Check(testkit.Rows("1 2 2")) + tk.MustQuery("EXPLAIN SELECT * FROM tkey12_2 WHERE col2 = 2").MultiCheckContain([]string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}) + tk.MustQuery("SELECT * FROM tkey12_2 WHERE col2 IS NULL") + tk.MustQuery("EXPLAIN SELECT * FROM tkey12_2 WHERE col2 IS NULL").MultiCheckContain([]string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}) + // Get the partition information from information_schema.partitions + result = tk.MustQuery("select PARTITION_NAME,PARTITION_ORDINAL_POSITION,PARTITION_METHOD,PARTITION_EXPRESSION " + + "FROM information_schema.partitions where TABLE_NAME = 'tkey12_2'") + result.Check(testkit.Rows("p0 1 KEY `col2`,`col3`", "p1 2 KEY `col2`,`col3`", "p2 3 KEY `col2`,`col3`", "p3 4 KEY `col2`,`col3`")) + + // This tests caculating the boundary partition ID when it prunes partition table + tk.MustExec("create table tkey16 (a int) partition by key (a) partitions 12") + tk.MustExec("insert into tkey16 values (0), (1), (2), (3)") + tk.MustExec("insert into tkey16 select a + 4 from tkey16") + tk.MustExec("insert into tkey16 select a + 8 from tkey16") + tk.MustExec("select * from information_schema.partitions where partition_name is not null") +} + +func TestKeyPartitionWithDifferentCharsets(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb4") + defer tk.MustExec("drop database partitiondb4") + tk.MustExec("use partitiondb4") + + tk.MustExec("CREATE TABLE tkey29 (" + + "col1 INT NOT NULL," + + "col2 DATE NOT NULL," + + "col3 VARCHAR(12) NOT NULL," + + "col4 INT NOT NULL," + + "UNIQUE KEY (col3)" + + ") CHARSET=utf8mb4 COLLATE=utf8mb4_bin " + + "PARTITION BY KEY(col3) " + + "PARTITIONS 4") + // ignore tail spaces + err := tk.ExecToErr("INSERT INTO tkey29 VALUES(1, '2023-02-22', 'linpin', 1), (1, '2023-02-22', 'linpin ', 5)") + require.Regexp(t, "Duplicate entry 'linpin ' for key 'tkey29.col3'", err) + // case sensitive + tk.MustExec("INSERT INTO tkey29 VALUES(3, '2023-02-22', 'abc', 1), (4, '2023-02-22', 'ABC ', 5)") + + tk.MustExec("CREATE TABLE tkey30 (" + + "col1 INT NOT NULL," + + "col2 DATE NOT NULL," + + "col3 VARCHAR(12) NOT NULL," + + "col4 INT NOT NULL," + + "UNIQUE KEY (col3)" + + ") CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci " + + "PARTITION BY KEY(col3) " + + "PARTITIONS 4") + // case insensitive + err = tk.ExecToErr("INSERT INTO tkey30 VALUES(1, '2023-02-22', 'linpin', 1), (1, '2023-02-22', 'LINPIN', 5)") + require.Regexp(t, "Duplicate entry 'LINPIN' for key 'tkey30.col3'", err) + // ignore tail spaces + err = tk.ExecToErr("INSERT INTO tkey30 VALUES(1, '2023-02-22', 'linpin', 1), (1, '2023-02-22', 'LINPIN ', 5)") + require.Regexp(t, "Duplicate entry 'LINPIN ' for key 'tkey30.col3'", err) + + tk.MustExec("CREATE TABLE tkey31 (" + + "col1 INT NOT NULL," + + "col2 DATE NOT NULL," + + "col3 VARCHAR(12) NOT NULL," + + "col4 INT NOT NULL," + + "UNIQUE KEY (col3)" + + ") CHARSET=gbk COLLATE=gbk_chinese_ci " + + "PARTITION BY KEY(col3) " + + "PARTITIONS 4") + err = tk.ExecToErr("INSERT INTO tkey31 VALUES(1, '2023-02-22', '刘德华', 1), (1, '2023-02-22', '刘德华 ', 5)") + require.Regexp(t, "Duplicate entry '刘德华 ' for key 'tkey31.col3'", err) + tk.MustExec("INSERT INTO tkey31 VALUES(1, '2023-02-22', '刘德华', 1), (5, '2023-02-22', '张学友', 5),(6, '2023-02-22', '艾伦', 6), (7, '2023-02-22', '宁采臣', 7)") + tk.MustQuery("SELECT * FROM tkey31 partition(p0)").Check(testkit.Rows("1 2023-02-22 刘德华 1")) + tk.MustQuery("SELECT * FROM tkey31 partition(p1)").Check(testkit.Rows("6 2023-02-22 艾伦 6")) + tk.MustQuery("SELECT * FROM tkey31 partition(p2)").Check(testkit.Rows("5 2023-02-22 张学友 5")) + tk.MustQuery("SELECT * FROM tkey31 partition(p3)").Check(testkit.Rows("7 2023-02-22 宁采臣 7")) +} + +func TestIssue31721(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_enable_list_partition=on;") + tk.MustExec("drop tables if exists t_31721") + tk.MustExec("CREATE TABLE `t_31721` (`COL1` char(1) NOT NULL) CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY LIST COLUMNS(`COL1`) " + + "(PARTITION `P0` VALUES IN ('1')," + + "PARTITION `P1` VALUES IN ('2')," + + "PARTITION `P2` VALUES IN ('3'));") + tk.MustExec("insert into t_31721 values ('1')") + tk.MustExec("select * from t_31721 partition(p0, p1) where col1 != 2;") +} + +func TestKeyPartitionTableDDL(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb3") + defer tk.MustExec("drop database partitiondb3") + tk.MustExec("use partitiondb3") + tk.MustExec("CREATE TABLE tkey14 (\n" + + "col1 INT NOT NULL," + + "col2 INT NOT NULL," + + "col3 INT NOT NULL," + + "col4 INT NOT NULL," + + "primary KEY (col1,col3)\n" + + ")" + + "PARTITION BY KEY(col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey14 values(1,1,1,1),(1,1,2,2),(3,3,3,3),(3,3,4,3),(4,4,4,4),(5,5,5,5),(6,6,6,6),(7,7,7,7),(8,8,8,8),(9,9,9,9),(10,10,10,5),(11,11,11,6),(12,12,12,12),(13,13,13,13),(14,14,14,14)") + + tk.MustExec("CREATE TABLE tkey15 (\n" + + "col1 INT NOT NULL," + + "col2 INT NOT NULL," + + "col3 INT NOT NULL," + + "col4 INT NOT NULL," + + "primary KEY (col1,col3)\n" + + ")") + tk.MustExec("INSERT INTO tkey15 values (20,20,20,20)") + + tk.MustExec("CREATE TABLE tkey16 (\n" + + "col1 INT NOT NULL," + + "col2 INT NOT NULL," + + "col3 INT NOT NULL," + + "col4 INT NOT NULL," + + "primary KEY (col1,col3)\n" + + ")" + + "PARTITION BY KEY(col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey16 values(1,1,1,1),(1,1,2,2),(3,3,3,3),(3,3,4,3),(4,4,4,4),(5,5,5,5),(6,6,6,6),(7,7,7,7),(8,8,8,8),(9,9,9,9),(10,10,10,5),(11,11,11,6),(12,12,12,12),(13,13,13,13),(14,14,14,14)") + + err := tk.ExecToErr("ALTER TABLE tkey15 PARTITION BY KEY(col3) PARTITIONS 4") + require.Regexp(t, "alter table partition is unsupported", err) + tk.MustExec("ALTER TABLE tkey14 ADD PARTITION PARTITIONS 1") + err = tk.ExecToErr("ALTER TABLE tkey14 DROP PARTITION p4") + require.Regexp(t, "DROP PARTITION can only be used on RANGE/LIST partitions", err) + tk.MustExec("ALTER TABLE tkey14 TRUNCATE PARTITION p3") + tk.MustQuery("SELECT COUNT(*) FROM tkey14 partition(p3)").Check(testkit.Rows("0")) + tk.MustExec("ALTER TABLE tkey16 COALESCE PARTITION 2") + tk.MustExec("ALTER TABLE tkey14 ANALYZE PARTITION p3") + err = tk.ExecToErr("ALTER TABLE tkey14 CHECK PARTITION p2") + require.Regexp(t, "Unsupported check partition", err) + err = tk.ExecToErr("ALTER TABLE tkey14 OPTIMIZE PARTITION p2") + require.Regexp(t, "Unsupported optimize partition", err) + err = tk.ExecToErr("ALTER TABLE tkey14 REBUILD PARTITION p2") + require.Regexp(t, "Unsupported rebuild partition", err) + err = tk.ExecToErr("ALTER TABLE tkey14 EXCHANGE PARTITION p3 WITH TABLE tkey15") + require.Regexp(t, "Unsupported partition type of table tkey14 when exchanging partition", err) + + err = tk.ExecToErr("ALTER TABLE tkey16 REORGANIZE PARTITION") + require.Regexp(t, "Unsupported reorganize partition", err) + err = tk.ExecToErr("ALTER TABLE tkey16 REORGANIZE PARTITION p0 INTO (PARTITION p0,PARTITION p1)") + require.Regexp(t, "Unsupported reorganize partition", err) + err = tk.ExecToErr("ALTER TABLE tkey16 REORGANIZE PARTITION p0 INTO (PARTITION p0)") + require.Regexp(t, "Unsupported reorganize partition", err) + err = tk.ExecToErr("ALTER TABLE tkey16 REORGANIZE PARTITION p0 INTO (PARTITION p4)") + require.Regexp(t, "Unsupported reorganize partition", err) + err = tk.ExecToErr("ALTER TABLE tkey16 REMOVE PARTITIONING") + require.Regexp(t, "Unsupported remove partitioning", err) + + tk.MustExec("CREATE TABLE tkey17 (" + + "id INT NOT NULL PRIMARY KEY," + + "name VARCHAR(20)" + + ")" + + "PARTITION BY KEY()" + + "PARTITIONS 2") + result := tk.MustQuery("show warnings") + result.CheckContain("Unsupported partition type KEY, treat as normal table") +} + +func TestLocatePartitionErrorInfo(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop tables if exists t_44966") + tk.MustExec("create table t_44966 (a bigint unsigned) partition by range (a) (partition p0 values less than (10))") + err := tk.ExecToErr("insert into t_44966 values (0xffffffffffffffff)") + require.Regexp(t, "Table has no partition for value 18446744073709551615", err) + tk.MustExec("drop tables if exists t_44966") + tk.MustExec("create table t_44966 (a bigint unsigned) partition by list (a) (partition p0 values in (1,2))") + require.Regexp(t, "Table has no partition for value 18446744073709551615", err) +} + +func TestPruneModeWarningInfo(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("set session tidb_partition_prune_mode = 'dynamic'") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Please analyze all partition tables again for consistency between partition and global stats", + "Warning 1105 Please avoid setting partition prune mode to dynamic at session level and set partition prune mode to dynamic at global level")) + tk.MustExec("set global tidb_partition_prune_mode = 'dynamic'") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Please analyze all partition tables again for consistency between partition and global stats")) +} + +type testCallback struct { + ddl.Callback + OnJobRunBeforeExported func(job *model.Job) +} + +func newTestCallBack(t *testing.T, dom *domain.Domain) *testCallback { + defHookFactory, err := ddl.GetCustomizedHook("default_hook") + require.NoError(t, err) + return &testCallback{ + Callback: defHookFactory(dom), + } +} + +func (c *testCallback) OnJobRunBefore(job *model.Job) { + if c.OnJobRunBeforeExported != nil { + c.OnJobRunBeforeExported(job) + } +} + +func TestReorgPartExtensivePart(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "ReorgPartExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + // TODO: Handle different column types? + // TODO: Handle index for different column types / combinations as well? + + // Steps: + // - create a table (should at least test both LIST and RANGE partition, Including COLUMNS) + // - add base data + // - start DDL + // - before each (and after?) each state transition: + // - insert, update and delete concurrently, to verify that the states are correctly handled. + // - TODO: Crash (if rollback is needed, then OK, but the rest need to be tested + // - TODO: Fail + // - TODO: run queries that could clash with backfill etc. (How to handle expected errors?) + // - TODO: on both the 'current' state and 'previous' state! + // - run ADMIN CHECK TABLE + // + + tk.MustExec(`create table t (a varchar(255) collate utf8mb4_unicode_ci, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, primary key (a)) partition by range columns (a) (partition pNull values less than (""), partition pM values less than ("M"), partition pLast values less than (maxvalue))`) + tk.MustExec(`create table t2 (a varchar(255) collate utf8mb4_unicode_ci, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, primary key (a), key (b), key (c,b), key (d,c), key(e))`) + + // TODO: Test again with timestamp in col e!! + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` varchar(255) COLLATE utf8mb4_unicode_ci NOT NULL,\n" + + " `b` varchar(255) COLLATE utf8mb4_general_ci DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " `d` datetime DEFAULT NULL,\n" + + " `e` timestamp NULL DEFAULT NULL,\n" + + " `f` double DEFAULT NULL,\n" + + " `g` text DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`a`)\n" + + "(PARTITION `pNull` VALUES LESS THAN (''),\n" + + " PARTITION `pM` VALUES LESS THAN ('M'),\n" + + " PARTITION `pLast` VALUES LESS THAN (MAXVALUE))")) + + tk.MustQuery(`show create table t2`).Check(testkit.Rows("" + + "t2 CREATE TABLE `t2` (\n" + + " `a` varchar(255) COLLATE utf8mb4_unicode_ci NOT NULL,\n" + + " `b` varchar(255) COLLATE utf8mb4_general_ci DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " `d` datetime DEFAULT NULL,\n" + + " `e` timestamp NULL DEFAULT NULL,\n" + + " `f` double DEFAULT NULL,\n" + + " `g` text DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`),\n" + + " KEY `d` (`d`,`c`),\n" + + " KEY `e` (`e`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + + dom := domain.GetDomain(tk.Session()) + originHook := dom.DDL().GetHook() + defer dom.DDL().SetHook(originHook) + hook := newTestCallBack(t, dom) + dom.DDL().SetHook(hook) + + rows := 10000 + pkInserts := 200 + pkUpdates := 200 + pkDeletes := 100 // Enough to delete half of what is inserted? + pkMap := make(map[string]struct{}, rows) + pkArray := make([]string, 0, len(pkMap)) + seed := rand.Int63() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + getNewPK := func(m map[string]struct{}, suf string) string { + newPK := randStr(2+reorgRand.Intn(5), reorgRand) + suf + lowerPK := strings.ToLower(newPK) + for _, ok := m[lowerPK]; ok; { + newPK = randStr(2+reorgRand.Intn(5), reorgRand) + lowerPK = strings.ToLower(newPK) + _, ok = m[lowerPK] + } + m[lowerPK] = struct{}{} + return newPK + } + cnt := 0 + getValues := func(pk string, asAssignment bool) string { + s := `('%s', '%s', %d, '%s', '%s', %f, '%s')` + if asAssignment { + s = `a = '%s', b = '%s', c = %d, d = '%s', e = '%s', f = %f, g = '%s'` + } + cnt++ + return fmt.Sprintf(s, + pk, + randStr(reorgRand.Intn(19), reorgRand), + cnt, //reorgRand.Int31(), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + reorgRand.Float64(), + randStr(512+reorgRand.Intn(1024), reorgRand)) + } + // Generate a start set: + for i := 0; i < rows; i++ { + pk := getNewPK(pkMap, "-o") + pkArray = append(pkArray, pk) + values := getValues(pk, false) + tk.MustExec(`insert into t values ` + values) + tk.MustExec(`insert into t2 values ` + values) + } + tk.MustExec(`analyze table t`) + tk.MustExec(`analyze table t2`) + tk.MustQuery(`select * from t except select * from t2`).Check(testkit.Rows()) + tk.MustQuery(`select * from t2 except select * from t`).Check(testkit.Rows()) + + // How to arrange data for possible collisions? + // change both PK column, SK column and non indexed column! + // Run various changes in transactions, in two concurrent sessions + // + mirror those transactions on a copy of the same table and data without DDL + // to verify expected transaction conflicts! + // We should try to collide: + // Current data : 1-1000 + // insert vN 1-200 // random order, random length of transaction? + // insert vN-1 100-300 // interleaved with vN, random order+length of txn? + // update vN 1-20, 100-120, 200-220, 300-320.. + // update vN-1 10-30, 110-130, 210-230, ... + // delete vN + // delete vN-1 + // insert update delete <- + // insert + // update + // delete + // Note: update the PK so it moves between different before and after partitions + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + currentState := model.StateNone + transitions := 0 + var currTbl table.Table + currSchema := sessiontxn.GetTxnManager(tk2.Session()).GetTxnInfoSchema() + prevTbl, err := currSchema.TableByName(model.NewCIStr(schemaName), model.NewCIStr("t")) + require.NoError(t, err) + var hookErr error + hook.OnJobRunBeforeExported = func(job *model.Job) { + if hookErr != nil { + // Enough to find a single error + return + } + if job.Type == model.ActionReorganizePartition && job.SchemaState != currentState { + transitions++ + // use random generation to possibly trigger txn collisions / deadlocks? + // insert (dup in new/old , non dup) + // update (dup in new/old , non dup as in same old/new partition -> update, different new/old -> insert + delete) + // delete + // verify with select after commit? + + logutil.BgLogger().Info("State before ins/upd/del", zap.Int("transitions", transitions), + zap.Int("rows", len(pkMap)), zap.Stringer("SchemaState", job.SchemaState)) + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + // Start with PK changes (non duplicate keys) + insPK := make([]string, 0, pkInserts) + values := make([]string, 0, pkInserts) + for i := 0; i < pkInserts; i += 2 { + pk := getNewPK(pkMap, "-i0") + logutil.BgLogger().Debug("insert1", zap.String("pk", pk)) + pkArray = append(pkArray, pk) + insPK = append(insPK, pk) + values = append(values, getValues(pk, false)) + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + hookErr = tk2.ExecToErr(`insert into t values ` + strings.Join(values, ",")) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`insert into t2 values ` + strings.Join(values, ",")) + if hookErr != nil { + return + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + currSchema = sessiontxn.GetTxnManager(tk2.Session()).GetTxnInfoSchema() + currTbl, hookErr = currSchema.TableByName(model.NewCIStr(schemaName), model.NewCIStr("t")) + + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + values = values[:0] + for i := 1; i < pkInserts; i += 2 { + pk := getNewPK(pkMap, "-i1") + logutil.BgLogger().Debug("insert2", zap.String("pk", pk)) + pkArray = append(pkArray, pk) + insPK = append(insPK, pk) + values = append(values, getValues(pk, false)) + } + hookErr = tk2.ExecToErr(`insert into t values ` + strings.Join(values, ",")) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`insert into t2 values ` + strings.Join(values, ",")) + if hookErr != nil { + return + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + rs, err := tk2.Exec(`select count(*) from t`) + if err != nil { + hookErr = err + return + } + tRows := tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + rs, err = tk2.Exec(`select count(*) from t2`) + if err != nil { + hookErr = err + return + } + t2Rows := tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + if tRows != t2Rows { + logutil.BgLogger().Error("rows do not match", zap.String("t", tRows), zap.String("t2", t2Rows), zap.Stringer("state", job.SchemaState)) + } + + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + + // Half from insert (1/4 in current schema version) + values = values[:0] + for i := 0; i < pkUpdates; i += 4 { + insIdx := reorgRand.Intn(len(insPK)) + oldPK := insPK[insIdx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + newPK := getNewPK(pkMap, "-u0") + insPK[insIdx] = newPK + idx := len(pkArray) - len(insPK) + insIdx + pkArray[idx] = newPK + value := getValues(newPK, true) + + logutil.BgLogger().Debug("update1", zap.String("old", oldPK), zap.String("value", value)) + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + + // Also do some non-pk column updates! + insIdx = reorgRand.Intn(len(insPK)) + oldPK = insPK[insIdx] + value = getValues(oldPK, true) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + // Half from insert (1/4 in previous schema version) + values = values[:0] + for i := 1; i < pkUpdates; i += 4 { + insIdx := reorgRand.Intn(len(insPK)) + oldPK := insPK[insIdx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + newPK := getNewPK(pkMap, "-u1") + insPK[insIdx] = newPK + idx := len(pkArray) - len(insPK) + insIdx + pkArray[idx] = newPK + value := getValues(newPK, true) + logutil.BgLogger().Debug("update2", zap.String("old", oldPK), zap.String("value", value)) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + + // Also do some non-pk column updates! + // Note: if PK changes it does RemoveRecord + AddRecord + insIdx = reorgRand.Intn(len(insPK)) + oldPK = insPK[insIdx] + value = getValues(oldPK, true) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + // Half from Old + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + + // Half from old (1/4 in current schema version) + values = values[:0] + for i := 2; i < pkUpdates; i += 4 { + idx := reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK := pkArray[idx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + newPK := getNewPK(pkMap, "-u2") + pkArray[idx] = newPK + value := getValues(newPK, true) + logutil.BgLogger().Debug("update3", zap.String("old", oldPK), zap.String("value", value)) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + + // Also do some non-pk column updates! + idx = reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK = pkArray[idx] + value = getValues(oldPK, true) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + // Half from old (1/4 in previous schema version) + values = values[:0] + for i := 3; i < pkUpdates; i += 4 { + idx := reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK := pkArray[idx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + newPK := getNewPK(pkMap, "-u3") + pkArray[idx] = newPK + value := getValues(newPK, true) + logutil.BgLogger().Debug("update4", zap.String("old", oldPK), zap.String("value", value)) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + + // Also do some non-pk column updates! + idx = reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK = pkArray[idx] + value = getValues(oldPK, true) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + rs, err = tk2.Exec(`select count(*) from t`) + if err != nil { + hookErr = err + return + } + tRows = tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + rs, err = tk2.Exec(`select count(*) from t2`) + if err != nil { + hookErr = err + return + } + t2Rows = tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + if tRows != t2Rows { + logutil.BgLogger().Error("rows do not match", zap.String("t", tRows), zap.String("t2", t2Rows), zap.Stringer("state", job.SchemaState)) + } + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + + // Half from insert (1/4 in current schema version) + values = values[:0] + for i := 0; i < pkDeletes; i += 4 { + insIdx := reorgRand.Intn(len(insPK)) + oldPK := insPK[insIdx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + idx := len(pkArray) - len(insPK) + insIdx + insPK = append(insPK[:insIdx], insPK[insIdx+1:]...) + pkArray = append(pkArray[:idx], pkArray[idx+1:]...) + logutil.BgLogger().Debug("delete0", zap.String("pk", oldPK)) + + hookErr = tk2.ExecToErr(`delete from t where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`delete from t2 where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + // Half from insert (1/4 in previous schema version) + values = values[:0] + for i := 1; i < pkDeletes; i += 4 { + insIdx := reorgRand.Intn(len(insPK)) + oldPK := insPK[insIdx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + idx := len(pkArray) - len(insPK) + insIdx + insPK = append(insPK[:insIdx], insPK[insIdx+1:]...) + pkArray = append(pkArray[:idx], pkArray[idx+1:]...) + logutil.BgLogger().Debug("delete1", zap.String("pk", oldPK)) + + hookErr = tk2.ExecToErr(`delete from t where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`delete from t2 where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + // Half from Old + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + + // Half from old (1/4 in current schema version) + values = values[:0] + for i := 2; i < pkDeletes; i += 4 { + idx := reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK := pkArray[idx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + pkArray = append(pkArray[:idx], pkArray[idx+1:]...) + logutil.BgLogger().Debug("delete2", zap.String("pk", oldPK)) + + hookErr = tk2.ExecToErr(`delete from t where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`delete from t2 where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + // Half from old (1/4 in previous schema version) + values = values[:0] + for i := 3; i < pkDeletes; i += 4 { + idx := reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK := pkArray[idx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + pkArray = append(pkArray[:idx], pkArray[idx+1:]...) + logutil.BgLogger().Debug("delete3", zap.String("pk", oldPK)) + + hookErr = tk2.ExecToErr(`delete from t where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`delete from t2 where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + rs, err = tk2.Exec(`select count(*) from t`) + if err != nil { + hookErr = err + return + } + tRows = tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + rs, err = tk2.Exec(`select count(*) from t2`) + if err != nil { + hookErr = err + return + } + t2Rows = tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + if tRows != t2Rows { + logutil.BgLogger().Error("rows do not match", zap.String("t", tRows), zap.String("t2", t2Rows), zap.Stringer("state", job.SchemaState)) + } + + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + prevTbl = currTbl + logutil.BgLogger().Info("State after ins/upd/del", zap.Int("transitions", transitions), + zap.Int("rows", len(pkMap)), zap.Stringer("SchemaState", job.SchemaState)) + } + } + tk.MustExec(`alter table t reorganize partition pNull, pM, pLast into (partition pI values less than ("I"), partition pQ values less than ("q"), partition pLast values less than (MAXVALUE))`) + require.NoError(t, hookErr) + tk.MustExec(`admin check table t`) + tk.MustExec(`admin check table t2`) + tk.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + tk.MustQuery(`select * from t except select * from t2 LIMIT 1`).Check(testkit.Rows()) + tk.MustQuery(`select * from t2 except select * from t LIMIT 1`).Check(testkit.Rows()) +} + +// Emojis fold to a single rune, and ö compares as o, so just complicated having other runes. +// Enough to just distribute between A and Z + testing simple folding +var runes = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ") + +func randStr(n int, r *rand.Rand) string { + var sb strings.Builder + sb.Grow(n) + for i := 0; i < n; i++ { + _, _ = sb.WriteRune(runes[r.Intn(len(runes))]) + } + return sb.String() +} +>>>>>>> c7c7000165a (ddl: Exchange partition rollback (#45877))