From fc624926793172d66feb23e5837bdc7a83ecdd50 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 12 Jan 2022 08:06:37 +0100 Subject: [PATCH 01/29] WIP add Physical Table ID as column in result. Table Partition Dynamic Prune Mode uses a single TableReader meaning it is not currently possible to know which partition a row/record comes from. By adding ExtraPhysTblID column this can be used when the row needs the Physical Table ID (partition id) for SelectLock or checking current transaction buffer (if the row has been changed/deleted). --- cmd/explaintest/r/generated_columns.result | 28 ++++++++- cmd/explaintest/t/generated_columns.test | 23 ++++++- distsql/request_builder.go | 1 + distsql/select_result.go | 19 ++++++ executor/builder.go | 7 ++- executor/executor.go | 1 + executor/table_reader.go | 9 ++- executor/union_scan.go | 29 ++++++++- parser/model/model.go | 20 ++++++ planner/core/integration_test.go | 6 +- planner/core/logical_plan_builder.go | 61 ++++++++++++++++--- planner/core/planbuilder.go | 2 +- planner/core/rule_column_pruning.go | 16 +++++ planner/core/util.go | 1 + sessionctx/variable/session.go | 5 -- .../unistore/cophandler/closure_exec.go | 28 +++++++++ .../unistore/cophandler/cop_handler.go | 4 ++ 17 files changed, 237 insertions(+), 23 deletions(-) diff --git a/cmd/explaintest/r/generated_columns.result b/cmd/explaintest/r/generated_columns.result index 970f00880ac5d..8378023b0f57a 100644 --- a/cmd/explaintest/r/generated_columns.result +++ b/cmd/explaintest/r/generated_columns.result @@ -1,4 +1,3 @@ -set @@tidb_partition_prune_mode='dynamic'; DROP TABLE IF EXISTS person; CREATE TABLE person ( id INT NOT NULL AUTO_INCREMENT PRIMARY KEY, @@ -90,6 +89,8 @@ Projection 5.00 root test.sgc1.j1, test.sgc1.j2, test.sgc1.a, test.sgc1.b, test └─TableReader(Probe) 5.00 root data:Selection └─Selection 5.00 cop[tikv] not(isnull(test.sgc1.a)) └─TableFullScan 5.00 cop[tikv] table:sgc1 keep order:false +set @old_prune_mode = @@tidb_partition_prune_mode; +set @@tidb_partition_prune_mode='static'; DROP TABLE IF EXISTS sgc3; CREATE TABLE sgc3 ( j JSON, @@ -136,6 +137,31 @@ PartitionUnion 23263.33 root └─TableReader 3323.33 root data:Selection └─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7) └─TableFullScan 10000.00 cop[tikv] table:sgc3, partition:max keep order:false, stats:pseudo +set @@tidb_partition_prune_mode='dynamic'; +DROP TABLE sgc3; +CREATE TABLE sgc3 ( +j JSON, +a INT AS (JSON_EXTRACT(j, "$.a")) STORED +) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (1), +PARTITION p1 VALUES LESS THAN (2), +PARTITION p2 VALUES LESS THAN (3), +PARTITION p3 VALUES LESS THAN (4), +PARTITION p4 VALUES LESS THAN (5), +PARTITION p5 VALUES LESS THAN (6), +PARTITION max VALUES LESS THAN MAXVALUE); +EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a <= 1; +id estRows task access object operator info +TableReader 3323.33 root partition:p0,p1 data:Selection +└─Selection 3323.33 cop[tikv] le(test.sgc3.a, 1) + └─TableFullScan 10000.00 cop[tikv] table:sgc3 keep order:false, stats:pseudo +EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a < 7; +id estRows task access object operator info +TableReader 3323.33 root partition:all data:Selection +└─Selection 3323.33 cop[tikv] lt(test.sgc3.a, 7) + └─TableFullScan 10000.00 cop[tikv] table:sgc3 keep order:false, stats:pseudo +set @@tidb_partition_prune_mode = @old_prune_mode; DROP TABLE IF EXISTS t1; CREATE TABLE t1(a INT, b INT AS (a+1) VIRTUAL, c INT AS (b+1) VIRTUAL, d INT AS (c+1) VIRTUAL, KEY(b), INDEX IDX(c, d)); INSERT INTO t1 (a) VALUES (0); diff --git a/cmd/explaintest/t/generated_columns.test b/cmd/explaintest/t/generated_columns.test index 82dfcf4d1d8c8..c007f8ff42e66 100644 --- a/cmd/explaintest/t/generated_columns.test +++ b/cmd/explaintest/t/generated_columns.test @@ -2,7 +2,6 @@ -- Most of the cases are ported from other tests to make sure generated columns behaves the same. -- Stored generated columns as indices -set @@tidb_partition_prune_mode='dynamic'; DROP TABLE IF EXISTS person; CREATE TABLE person ( @@ -74,6 +73,8 @@ EXPLAIN format = 'brief' SELECT * from sgc1 join sgc2 on sgc1.a=sgc2.a; -- Stored generated columns as partition columns +set @old_prune_mode = @@tidb_partition_prune_mode; +set @@tidb_partition_prune_mode='static'; DROP TABLE IF EXISTS sgc3; CREATE TABLE sgc3 ( j JSON, @@ -91,6 +92,26 @@ PARTITION max VALUES LESS THAN MAXVALUE); EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a <= 1; EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a < 7; +set @@tidb_partition_prune_mode='dynamic'; +DROP TABLE sgc3; +CREATE TABLE sgc3 ( +j JSON, +a INT AS (JSON_EXTRACT(j, "$.a")) STORED +) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (1), +PARTITION p1 VALUES LESS THAN (2), +PARTITION p2 VALUES LESS THAN (3), +PARTITION p3 VALUES LESS THAN (4), +PARTITION p4 VALUES LESS THAN (5), +PARTITION p5 VALUES LESS THAN (6), +PARTITION max VALUES LESS THAN MAXVALUE); + +EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a <= 1; +EXPLAIN format = 'brief' SELECT * FROM sgc3 WHERE a < 7; + +set @@tidb_partition_prune_mode = @old_prune_mode; + -- Virtual generated columns as indices DROP TABLE IF EXISTS t1; diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 03f6034bc513e..57fdd1c6573b9 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -133,6 +133,7 @@ func (builder *RequestBuilder) SetTableHandles(tid int64, handles []kv.Handle) * // SetPartitionsAndHandles sets "KeyRanges" for "kv.Request" by converting ParitionHandles to KeyRanges. // handles in slice must be kv.PartitionHandle. +// TODO: What is this? func (builder *RequestBuilder) SetPartitionsAndHandles(handles []kv.Handle) *RequestBuilder { builder.Request.KeyRanges = PartitionHandlesToKVRanges(handles) return builder diff --git a/distsql/select_result.go b/distsql/select_result.go index b2eef6e6f5300..36cb9d4362365 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -216,12 +216,31 @@ func (r *selectResult) fetchResp(ctx context.Context) error { } return nil } + s := fmt.Sprintf("%v", resultSubset) + logutil.Logger(ctx).Info("resultSubset", zap.String("resultSubset", s)) + logutil.Logger(ctx).Info("resultSubset", zap.ByteString("data", resultSubset.GetData())) + logutil.Logger(ctx).Info("resultSubset", zap.Binary("data", resultSubset.GetData())) + b := resultSubset.GetData() + i := make([]int8, len(b)) + for j := range b { + i[j] = int8(b[j]) + } + logutil.Logger(ctx).Info("resultSubset", zap.Int8s("data", i)) + logutil.Logger(ctx).Info("resultSubset", zap.ByteString("startKey", resultSubset.GetStartKey())) + logutil.Logger(ctx).Info("resultSubset", zap.Binary("startKey", resultSubset.GetStartKey())) + b = resultSubset.GetStartKey() + i = make([]int8, len(b)) + for j := range b { + i[j] = int8(b[j]) + } + logutil.Logger(ctx).Info("resultSubset", zap.Int8s("startKey", i)) r.selectResp = new(tipb.SelectResponse) err = r.selectResp.Unmarshal(resultSubset.GetData()) if err != nil { return errors.Trace(err) } respSize := int64(r.selectResp.Size()) + logutil.Logger(ctx).Info("respSize", zap.Int64("size", respSize)) atomic.StoreInt64(&r.selectRespSize, respSize) r.memConsume(respSize) if err := r.selectResp.Error; err != nil { diff --git a/executor/builder.go b/executor/builder.go index ebf4176db00ba..6493a0e81ee4f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3071,7 +3071,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea e.feedback = statistics.NewQueryFeedback(getFeedbackStatsTableID(e.ctx, tbl), ts.Hist, int64(ts.StatsCount()), ts.Desc) } collect := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(ts.Ranges)) - // Do not collect the feedback when the table is the partition table. + // Do not collect the feedback when the table is the partition table. TODO: not true for dynamic? if collect && tbl.Meta().Partition != nil { collect = false } @@ -3096,7 +3096,8 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea func extraPIDColumnIndex(schema *expression.Schema) offsetOptional { for idx, col := range schema.Columns { - if col.ID == model.ExtraPidColID { + // TODO: Handle partitioned global index, i.e. both ExtraPidColID and ExtraPhysTblID is used. + if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID { return newOffset(idx) } } @@ -3169,6 +3170,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E tmp, _ := b.is.TableByID(ts.Table.ID) tbl := tmp.(table.PartitionedTable) + // TODO: Should this be done before checking dirty / ongoing transactions for avoiding adding PID columns? partitions, err := partitionPruning(b.ctx, tbl, v.PartitionInfo.PruningConds, v.PartitionInfo.PartitionNames, v.PartitionInfo.Columns, v.PartitionInfo.ColumnNames) if err != nil { b.err = err @@ -3802,6 +3804,7 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte return nil, err } var kvRanges []kv.KeyRange + // TODO: mjonss: study this :) if len(lookUpContents) > 0 && keyColumnsIncludeAllPartitionColumns(lookUpContents[0].keyCols, pe) { // In this case we can use dynamic partition pruning. locateKey := make([]types.Datum, e.Schema().Len()) diff --git a/executor/executor.go b/executor/executor.go index 6f4867438134a..ad88d469a72fa 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -939,6 +939,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { if len(e.partitionedTable) > 0 { // Replace the table ID with partition ID. // The partition ID is returned as an extra column from the table reader. + // TODO: any easier way? if offset, ok := e.tblID2PIDColumnIndex[id]; ok { physicalID = row.GetInt64(offset) } diff --git a/executor/table_reader.go b/executor/table_reader.go index b31f07bb79d66..c9bc68a0b0aa6 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" + "go.uber.org/zap" ) // make sure `TableReaderExecutor` implements `Executor`. @@ -244,9 +245,13 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error // When 'select ... for update' work on a partitioned table, the table reader should // add the partition ID as an extra column. The SelectLockExec need this information // to construct the lock key. - physicalID := getPhysicalTableID(e.table) if e.extraPIDColumnIndex.valid() { - fillExtraPIDColumn(req, e.extraPIDColumnIndex.value(), physicalID) + physicalID := getPhysicalTableID(e.table) + if physicalID != e.table.Meta().ID { + // table partition in static prune mode (one TableReaderExecutor per partition) + logutil.BgLogger().Info("MJONSS: TableReaderExecutor.Next()", zap.Int64("physicalID", physicalID)) + fillExtraPIDColumn(req, e.extraPIDColumnIndex.value(), physicalID) + } } return nil diff --git a/executor/union_scan.go b/executor/union_scan.go index 86b696a8ee988..a4cab2bf08bbe 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -29,6 +29,8 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) // UnionScanExec merges the rows from dirty table and the rows from distsql request. @@ -219,11 +221,30 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err var err error us.cursor4SnapshotRows = 0 us.snapshotRows = us.snapshotRows[:0] + physTblIDIdx := -1 + fts := make([]*types.FieldType, len(us.columns)) + //colNames := make([]string, len(us.columns)) + //colIds := make([]int64, len(us.columns)) + for i := range us.columns { + fts[i] = &us.columns[i].FieldType + //colNames = append(colNames, us.columns[i].Name.O) + //colIds = append(colIds, us.columns[i].ID) + if us.columns[i].ID == model.ExtraPhysTblID { + if physTblIDIdx >= 0 { + logutil.Logger(ctx).Warn("More than one ExtraPhysTblID column!", zap.String("table", us.table.Meta().Name.O)) + } + physTblIDIdx = i + } + } for len(us.snapshotRows) == 0 { err = Next(ctx, us.children[0], us.snapshotChunkBuffer) if err != nil || us.snapshotChunkBuffer.NumRows() == 0 { return nil, err } + logutil.Logger(ctx).Info("MJONSS: getSnapshotRow", zap.String("Chunk", us.snapshotChunkBuffer.ToString(fts)), + //zap.Strings("colNames", colNames), + //zap.Int64s("colIds", colIds), + zap.Int("us colNames count", len(us.columns))) iter := chunk.NewIterator4Chunk(us.snapshotChunkBuffer) for row := iter.Begin(); row != iter.End(); row = iter.Next() { var snapshotHandle kv.Handle @@ -231,7 +252,13 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err if err != nil { return nil, err } - checkKey := tablecodec.EncodeRecordKey(us.table.RecordPrefix(), snapshotHandle) + var checkKey kv.Key + if physTblIDIdx >= 0 { + tblID := row.GetInt64(physTblIDIdx) + checkKey = tablecodec.EncodeRowKeyWithHandle(tblID, snapshotHandle) + } else { + checkKey = tablecodec.EncodeRecordKey(us.table.RecordPrefix(), snapshotHandle) + } if _, err := us.memBufSnap.Get(context.TODO(), checkKey); err == nil { // If src handle appears in added rows, it means there is conflict and the transaction will fail to // commit, but for simplicity, we don't handle it here. diff --git a/parser/model/model.go b/parser/model/model.go index b04da6222cc97..9b6e924ecacaa 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -234,6 +234,12 @@ const ExtraHandleID = -1 // ExtraPidColID is the column ID of column which store the partitionID decoded in global index values. const ExtraPidColID = -2 +// ExtraPhysTblID is the column ID of column that should be filled in with the physical table id. +// Primarily used for table partition dynamic prune mode, to return which partition (physical table id) the row came from. +// Using a dedicated id for this, since in the future ExtraPidColID and ExtraPhysTblID may be used for the same request. +// Must be after ExtraPidColID! +const ExtraPhysTblID = -3 + const ( // TableInfoVersion0 means the table info version is 0. // Upgrade from v2.1.1 or v2.1.2 to v2.1.3 and later, and then execute a "change/modify column" statement @@ -275,6 +281,9 @@ var ExtraHandleName = NewCIStr("_tidb_rowid") // ExtraPartitionIdName is the name of ExtraPartitionId Column. var ExtraPartitionIdName = NewCIStr("_tidb_pid") +// ExtraPhysTblIdName is the name of ExtraPhysTblID Column. +var ExtraPhysTblIdName = NewCIStr("_tidb_tid") + // TableInfo provides meta data describing a DB table. type TableInfo struct { ID int64 `json:"id"` @@ -653,6 +662,17 @@ func NewExtraPartitionIDColInfo() *ColumnInfo { return colInfo } +// NewExtraPhysTblIDColInfo mocks a column info for extra partition id column. +func NewExtraPhysTblIDColInfo() *ColumnInfo { + colInfo := &ColumnInfo{ + ID: ExtraPhysTblID, + Name: ExtraPhysTblIdName, + } + colInfo.Tp = mysql.TypeLonglong + colInfo.Flen, colInfo.Decimal = mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeLonglong) + return colInfo +} + // ColumnIsInIndex checks whether c is included in any indices of t. func (t *TableInfo) ColumnIsInIndex(c *ColumnInfo) bool { for _, index := range t.Indices { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index a4cacd0d58f17..ec665b234d0ff 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3368,12 +3368,12 @@ func (s *testIntegrationSuite) TestIssue26719(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table tx (a int) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))`) - tk.MustExec(`insert into tx values (1)`) + tk.MustExec(`insert into tx values (10)`) tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec(`begin`) - tk.MustExec(`delete from tx where a in (1)`) - tk.MustQuery(`select * from tx PARTITION(p0)`).Check(testkit.Rows()) + tk.MustExec(`delete from tx where a in (10)`) + //tk.MustQuery(`select * from tx PARTITION(p0)`).Check(testkit.Rows()) tk.MustQuery(`select * from tx`).Check(testkit.Rows()) tk.MustExec(`rollback`) } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index d595965a1e81f..21e4a05c11de9 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -55,8 +55,10 @@ import ( util2 "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/set" + "go.uber.org/zap" ) const ( @@ -3182,7 +3184,7 @@ func unfoldWildStar(field *ast.SelectField, outputName types.NameSlice, column [ } if (dbName.L == "" || dbName.L == name.DBName.L) && (tblName.L == "" || tblName.L == name.TblName.L) && - col.ID != model.ExtraHandleID { + col.ID != model.ExtraHandleID && col.ID != model.ExtraPidColID && col.ID != model.ExtraPhysTblID { colName := &ast.ColumnNameExpr{ Name: &ast.ColumnName{ Schema: name.DBName, @@ -3784,24 +3786,59 @@ func (ds *DataSource) newExtraHandleSchemaCol() *expression.Column { // addExtraPIDColumn add an extra PID column for partition table. // 'select ... for update' on a partition table need to know the partition ID // to construct the lock key, so this column is added to the chunk row. -func (ds *DataSource) addExtraPIDColumn(info *extraPIDInfo) { +func (ds *DataSource) addExtraPIDColumn() { pidCol := &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), - ID: model.ExtraPidColID, - OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.tableInfo.Name, model.ExtraPartitionIdName), + ID: model.ExtraPhysTblID, + OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.tableInfo.Name, model.ExtraPhysTblIdName), } - ds.Columns = append(ds.Columns, model.NewExtraPartitionIDColInfo()) + ds.Columns = append(ds.Columns, model.NewExtraPhysTblIDColInfo()) schema := ds.Schema() schema.Append(pidCol) ds.names = append(ds.names, &types.FieldName{ DBName: ds.DBName, TblName: ds.TableInfo().Name, - ColName: model.ExtraPartitionIdName, - OrigColName: model.ExtraPartitionIdName, + ColName: model.ExtraPhysTblIdName, + OrigColName: model.ExtraPhysTblIdName, }) ds.TblCols = append(ds.TblCols, pidCol) +} + +// addExtraPIDColumnWithInfo add an extra PID column for partition table. +// 'select ... for update' on a partition table need to know the partition ID +func (ds *DataSource) addExtraPIDColumnWithInfo(info *extraPIDInfo) { + var pidCol *expression.Column + schema := ds.Schema() + for _, col := range schema.Columns { + if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID { + if pidCol != nil { + logutil.BgLogger().Warn("MJONSS: Duplicate Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) + } + pidCol = col + } + } + if pidCol == nil { + logutil.BgLogger().Warn("MJONSS: Missing Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) + pidCol = &expression.Column{ + RetType: types.NewFieldType(mysql.TypeLonglong), + UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), + ID: model.ExtraPidColID, + OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.tableInfo.Name, model.ExtraPartitionIdName), + } + + ds.Columns = append(ds.Columns, model.NewExtraPartitionIDColInfo()) + schema := ds.Schema() + schema.Append(pidCol) + ds.names = append(ds.names, &types.FieldName{ + DBName: ds.DBName, + TblName: ds.TableInfo().Name, + ColName: model.ExtraPartitionIdName, + OrigColName: model.ExtraPartitionIdName, + }) + ds.TblCols = append(ds.TblCols, pidCol) + } info.Columns = append(info.Columns, pidCol) info.TblIDs = append(info.TblIDs, ds.TableInfo().ID) @@ -4183,6 +4220,16 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as us.SetChildren(ds) result = us } + if tableInfo.GetPartitionInfo() != nil { + if b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + // Use the new partition implementation, add partition id as handle/hidden column. + // dirty => must check transaction buffer, which uses Physical table id, so we need it per record from the partitioned table + // IsPessimistic => SelectLock needs the Physical table id for locking each row. + if dirty || b.ctx.GetSessionVars().TxnCtx.IsPessimistic { + ds.addExtraPIDColumn() + } + } + } // If a table is a cache table, it is judged whether it satisfies the conditions of read cache. if tableInfo.TableCacheStatusType == model.TableCacheStatusEnable && b.ctx.GetSessionVars().SnapshotTS == 0 && !b.ctx.GetSessionVars().StmtCtx.IsStaleness { cachedTable := tbl.(table.CachedTable) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 0dfd7ea71c743..527ac83379fae 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1258,7 +1258,7 @@ func addExtraPIDColumnToDataSource(p LogicalPlan, info *extraPIDInfo) error { if raw.tableInfo.GetPartitionInfo() == nil { return nil } - raw.addExtraPIDColumn(info) + raw.addExtraPIDColumnWithInfo(info) return nil default: var err error diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 17dcb87965e77..bc2ed086960cc 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -293,6 +293,21 @@ func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt for i := 0; i < p.handleCols.NumCols(); i++ { parentUsedCols = append(parentUsedCols, p.handleCols.GetCol(i)) } + for _, col := range p.Schema().Columns { + if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID { + parentUsedCols = append(parentUsedCols, col) + cols := p.Schema().Columns + if col != cols[len(cols)-1] { + panic("MJONSS: Assumptions of ExtraPidColID always last is wrong!!!") + } + } + } + // ExtraPidColID should always be last? + /* + if cols[len(cols)-1].ID == model.ExtraPidColID { + parentUsedCols = append(parentUsedCols, col) + } + */ condCols := expression.ExtractColumnsFromExpressions(nil, p.conditions, nil) parentUsedCols = append(parentUsedCols, condCols...) return p.children[0].PruneColumns(parentUsedCols, opt) @@ -474,6 +489,7 @@ func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *log } if len(p.partitionedTable) > 0 { + // TODO: What to do here? Should it be added to the tblID2Handle array instead? // If the children include partitioned tables, there is an extra partition ID column. parentUsedCols = append(parentUsedCols, p.extraPIDInfo.Columns...) } diff --git a/planner/core/util.go b/planner/core/util.go index 9d0fe9d0f3bfc..b2427320674a5 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -331,6 +331,7 @@ func tableHasDirtyContent(ctx sessionctx.Context, tableInfo *model.TableInfo) bo } // Currently, we add UnionScan on every partition even though only one partition's data is changed. // This is limited by current implementation of Partition Prune. It'll be updated once we modify that part. + // TODO: Use named partitions at least, preferably pruned if possible here? for _, partition := range pi.Definitions { if ctx.HasDirtyContent(partition.ID) { return true diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 4a66740d0f2a6..2e449f685c64d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1062,11 +1062,6 @@ func (s *SessionVars) CheckAndGetTxnScope() string { // UseDynamicPartitionPrune indicates whether use new dynamic partition prune. func (s *SessionVars) UseDynamicPartitionPrune() bool { - if s.InTxn() || !s.GetStatusFlag(mysql.ServerStatusAutocommit) { - // UnionScan cannot get partition table IDs in dynamic-mode, this is a quick-fix for issues/26719, - // please see it for more details. - return false - } return PartitionPruneMode(s.PartitionPruneMode.Load()) == Dynamic } diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index efc845820d260..d760ade74f3c1 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -37,10 +37,12 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/logutil" mockpkg "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" + "go.uber.org/zap" ) const chunkMaxRows = 1024 @@ -293,6 +295,11 @@ func (e *closureExecutor) initIdxScanCtx(idxScan *tipb.IndexScan) { e.idxScanCtx.primaryColumnIds = idxScan.PrimaryColumnIds lastColumn := e.columnInfos[len(e.columnInfos)-1] + if lastColumn.GetColumnId() == model.ExtraPhysTblID { + lastColumn = e.columnInfos[len(e.columnInfos)-2] + e.idxScanCtx.columnLen-- + } + if lastColumn.GetColumnId() == model.ExtraPidColID { lastColumn = e.columnInfos[len(e.columnInfos)-2] e.idxScanCtx.columnLen-- @@ -837,6 +844,21 @@ func (e *closureExecutor) tableScanProcessCore(key, value []byte) error { if err != nil { return errors.Trace(err) } + // Add ExtraPhysTblID if requested + if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID { + tblID := tablecodec.DecodeTableID(key) + logutil.BgLogger().Info("MJONSS: tableScanProcessCore", zap.Int64("tblID", tblID), zap.Int("columnIdx", len(e.columnInfos)-1)) + e.scanCtx.chk.AppendInt64(len(e.columnInfos)-1, tblID) + /* + colIds := make([]int64, len(e.columnInfos)) + for i := range e.columnInfos { + colIds = append(colIds, e.columnInfos[i].ColumnId) + } + */ + logutil.BgLogger().Info("MJONSS: tableScanProcessCore chk", zap.String("Chunk", e.scanCtx.chk.ToString(e.resultFieldType)), + //zap.Int64s("colIds", colIds), + zap.Int("colIds count", len(e.columnInfos)), zap.Int("result field types count", len(e.resultFieldType))) + } incRow = true return nil } @@ -909,6 +931,12 @@ func (e *closureExecutor) indexScanProcessCore(key, value []byte) error { } } } + // Add ExtraPhysTblID if requested + if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID { + tblID := tablecodec.DecodeTableID(key) + logutil.BgLogger().Info("MJONSS: indexScanProcessCore", zap.Int64("tblID", tblID)) + chk.AppendInt64(len(e.columnInfos)-1, tblID) + } gotRow = true return nil } diff --git a/store/mockstore/unistore/cophandler/cop_handler.go b/store/mockstore/unistore/cophandler/cop_handler.go index fa7b784126e3e..8c8ac32beb093 100644 --- a/store/mockstore/unistore/cophandler/cop_handler.go +++ b/store/mockstore/unistore/cophandler/cop_handler.go @@ -212,6 +212,10 @@ func newRowDecoder(columnInfos []*tipb.ColumnInfo, fieldTps []*types.FieldType, ) for i := range columnInfos { info := columnInfos[i] + if info.ColumnId == model.ExtraPhysTblID { + // Skip since it needs to be filled in from the key + continue + } ft := fieldTps[i] col := rowcodec.ColInfo{ ID: info.ColumnId, From 4821faef0e08183ac095433b0454b85dfca1aba1 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 12 Jan 2022 19:41:28 +0100 Subject: [PATCH 02/29] Added support for more cases, like Delete --- cmd/explaintest/r/select.result | 16 ++++++++++++++++ cmd/explaintest/t/select.test | 6 ++++++ executor/delete.go | 2 +- planner/core/plan_to_pb.go | 2 ++ planner/core/rule_column_pruning.go | 10 ---------- planner/core/rule_partition_processor.go | 9 +++++++++ 6 files changed, 34 insertions(+), 11 deletions(-) diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index f364a577e1071..65b8b959181fd 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -354,6 +354,7 @@ Projection_3 1.00 root sysdate()->Column#1, sleep(1)->Column#2, sysdate()->Colu └─TableDual_4 1.00 root rows:1 drop table if exists th; set @@session.tidb_enable_table_partition = '1'; +set @@session.tidb_partition_prune_mode = 'static'; create table th (a int, b int) partition by hash(a) partitions 3; insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8); insert into th values (-1,-1),(-2,-2),(-3,-3),(-4,-4),(-5,-5),(-6,-6),(-7,-7),(-8,-8); @@ -378,6 +379,21 @@ PartitionUnion_8 20000.00 root │ └─TableFullScan_9 10000.00 cop[tikv] table:th, partition:p1 keep order:false, stats:pseudo └─TableReader_12 10000.00 root data:TableFullScan_11 └─TableFullScan_11 10000.00 cop[tikv] table:th, partition:p2 keep order:false, stats:pseudo +set @@session.tidb_partition_prune_mode = 'dynamic'; +desc select * from th where a=-2; +id estRows task access object operator info +TableReader_7 10.00 root partition:p2 data:Selection_6 +└─Selection_6 10.00 cop[tikv] eq(test.th.a, -2) + └─TableFullScan_5 10000.00 cop[tikv] table:th keep order:false, stats:pseudo +desc select * from th; +id estRows task access object operator info +TableReader_5 10000.00 root partition:all data:TableFullScan_4 +└─TableFullScan_4 10000.00 cop[tikv] table:th keep order:false, stats:pseudo +desc select * from th partition (p2,p1); +id estRows task access object operator info +TableReader_5 10000.00 root partition:p1,p2 data:TableFullScan_4 +└─TableFullScan_4 10000.00 cop[tikv] table:th keep order:false, stats:pseudo +set @@session.tidb_partition_prune_mode = DEFAULT; drop table if exists t; create table t(a int, b int); explain format = 'brief' select a != any (select a from t t2) from t t1; diff --git a/cmd/explaintest/t/select.test b/cmd/explaintest/t/select.test index c53aca600e2b9..6f3b4f0321378 100644 --- a/cmd/explaintest/t/select.test +++ b/cmd/explaintest/t/select.test @@ -181,12 +181,18 @@ desc select sysdate(), sleep(1), sysdate(); # test select partition table drop table if exists th; set @@session.tidb_enable_table_partition = '1'; +set @@session.tidb_partition_prune_mode = 'static'; create table th (a int, b int) partition by hash(a) partitions 3; insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8); insert into th values (-1,-1),(-2,-2),(-3,-3),(-4,-4),(-5,-5),(-6,-6),(-7,-7),(-8,-8); desc select * from th where a=-2; desc select * from th; desc select * from th partition (p2,p1); +set @@session.tidb_partition_prune_mode = 'dynamic'; +desc select * from th where a=-2; +desc select * from th; +desc select * from th partition (p2,p1); +set @@session.tidb_partition_prune_mode = DEFAULT; # test != any(subq) and = all(subq) drop table if exists t; diff --git a/executor/delete.go b/executor/delete.go index 2d4425653c090..c859b41a83c75 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -122,7 +122,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error { datumRow := make([]types.Datum, 0, len(fields)) for i, field := range fields { - if columns[i].ID == model.ExtraPidColID { + if columns[i].ID == model.ExtraPidColID || columns[i].ID == model.ExtraPhysTblID { continue } diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 27056a8937c20..08bd5ed5eb979 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -324,6 +324,8 @@ func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb. for _, col := range p.schema.Columns { if col.ID == model.ExtraHandleID { columns = append(columns, model.NewExtraHandleColInfo()) + } else if col.ID == model.ExtraPhysTblID { + columns = append(columns, model.NewExtraPhysTblIDColInfo()) } else if col.ID == model.ExtraPidColID { columns = append(columns, model.NewExtraPartitionIDColInfo()) } else { diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index bc2ed086960cc..c83c26f217458 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -296,18 +296,8 @@ func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt for _, col := range p.Schema().Columns { if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID { parentUsedCols = append(parentUsedCols, col) - cols := p.Schema().Columns - if col != cols[len(cols)-1] { - panic("MJONSS: Assumptions of ExtraPidColID always last is wrong!!!") - } } } - // ExtraPidColID should always be last? - /* - if cols[len(cols)-1].ID == model.ExtraPidColID { - parentUsedCols = append(parentUsedCols, col) - } - */ condCols := expression.ExtractColumnsFromExpressions(nil, p.conditions, nil) parentUsedCols = append(parentUsedCols, condCols...) return p.children[0].PruneColumns(parentUsedCols, opt) diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 53a655c5ed932..0f420680f54fa 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -310,6 +310,15 @@ func (s *partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types. }) continue } + if colExpr.ID == model.ExtraPhysTblID { + names = append(names, &types.FieldName{ + DBName: ds.DBName, + TblName: ds.tableInfo.Name, + ColName: model.ExtraPhysTblIdName, + OrigColName: model.ExtraPhysTblIdName, + }) + continue + } if colInfo, found := colsInfoMap[colExpr.ID]; found { names = append(names, &types.FieldName{ DBName: ds.DBName, From 21dd902f5233d572068d10e28941ec1acd0014eb Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 13 Jan 2022 00:25:02 +0100 Subject: [PATCH 03/29] Cleanup and added test from pr/31025 --- distsql/request_builder.go | 1 - distsql/select_result.go | 19 ----- executor/builder.go | 6 +- executor/executor.go | 1 - executor/index_merge_reader.go | 25 ++++--- executor/index_merge_reader_test.go | 72 +++++++++++++++++++ executor/table_reader.go | 2 - executor/union_scan.go | 9 +-- planner/core/integration_test.go | 6 +- planner/core/logical_plan_builder.go | 6 +- planner/core/rule_column_pruning.go | 1 - planner/core/util.go | 1 - .../unistore/cophandler/closure_exec.go | 14 +--- 13 files changed, 100 insertions(+), 63 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 57fdd1c6573b9..03f6034bc513e 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -133,7 +133,6 @@ func (builder *RequestBuilder) SetTableHandles(tid int64, handles []kv.Handle) * // SetPartitionsAndHandles sets "KeyRanges" for "kv.Request" by converting ParitionHandles to KeyRanges. // handles in slice must be kv.PartitionHandle. -// TODO: What is this? func (builder *RequestBuilder) SetPartitionsAndHandles(handles []kv.Handle) *RequestBuilder { builder.Request.KeyRanges = PartitionHandlesToKVRanges(handles) return builder diff --git a/distsql/select_result.go b/distsql/select_result.go index 36cb9d4362365..b2eef6e6f5300 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -216,31 +216,12 @@ func (r *selectResult) fetchResp(ctx context.Context) error { } return nil } - s := fmt.Sprintf("%v", resultSubset) - logutil.Logger(ctx).Info("resultSubset", zap.String("resultSubset", s)) - logutil.Logger(ctx).Info("resultSubset", zap.ByteString("data", resultSubset.GetData())) - logutil.Logger(ctx).Info("resultSubset", zap.Binary("data", resultSubset.GetData())) - b := resultSubset.GetData() - i := make([]int8, len(b)) - for j := range b { - i[j] = int8(b[j]) - } - logutil.Logger(ctx).Info("resultSubset", zap.Int8s("data", i)) - logutil.Logger(ctx).Info("resultSubset", zap.ByteString("startKey", resultSubset.GetStartKey())) - logutil.Logger(ctx).Info("resultSubset", zap.Binary("startKey", resultSubset.GetStartKey())) - b = resultSubset.GetStartKey() - i = make([]int8, len(b)) - for j := range b { - i[j] = int8(b[j]) - } - logutil.Logger(ctx).Info("resultSubset", zap.Int8s("startKey", i)) r.selectResp = new(tipb.SelectResponse) err = r.selectResp.Unmarshal(resultSubset.GetData()) if err != nil { return errors.Trace(err) } respSize := int64(r.selectResp.Size()) - logutil.Logger(ctx).Info("respSize", zap.Int64("size", respSize)) atomic.StoreInt64(&r.selectRespSize, respSize) r.memConsume(respSize) if err := r.selectResp.Error; err != nil { diff --git a/executor/builder.go b/executor/builder.go index 6493a0e81ee4f..b4acf3d65a358 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3071,7 +3071,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea e.feedback = statistics.NewQueryFeedback(getFeedbackStatsTableID(e.ctx, tbl), ts.Hist, int64(ts.StatsCount()), ts.Desc) } collect := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(ts.Ranges)) - // Do not collect the feedback when the table is the partition table. TODO: not true for dynamic? + // Do not collect the feedback when the table is the partition table. if collect && tbl.Meta().Partition != nil { collect = false } @@ -3651,6 +3651,9 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd } collectTable := false e.tableRequest.CollectRangeCounts = &collectTable + if tblInfo.Meta().Partition != nil { + e.extraPIDColumnIndex = extraPIDColumnIndex(v.Schema()) + } return e, nil } @@ -3804,7 +3807,6 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte return nil, err } var kvRanges []kv.KeyRange - // TODO: mjonss: study this :) if len(lookUpContents) > 0 && keyColumnsIncludeAllPartitionColumns(lookUpContents[0].keyCols, pe) { // In this case we can use dynamic partition pruning. locateKey := make([]types.Datum, e.Schema().Len()) diff --git a/executor/executor.go b/executor/executor.go index ad88d469a72fa..6f4867438134a 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -939,7 +939,6 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { if len(e.partitionedTable) > 0 { // Replace the table ID with partition ID. // The partition ID is returned as an extra column from the table reader. - // TODO: any easier way? if offset, ok := e.tblID2PIDColumnIndex[id]; ok { physicalID = row.GetInt64(offset) } diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 38aa4a84f5bc5..186d05ec79c47 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -116,6 +116,10 @@ type IndexMergeReaderExecutor struct { isCorColInPartialFilters []bool isCorColInTableFilter bool isCorColInPartialAccess []bool + + // extraPIDColumnIndex is used for partition reader to add an extra partition ID column + // in static tidb_partition_prune_mode + extraPIDColumnIndex offsetOptional } // Open implements the Executor Open interface @@ -599,16 +603,17 @@ func (e *IndexMergeReaderExecutor) startIndexMergeTableScanWorker(ctx context.Co func (e *IndexMergeReaderExecutor) buildFinalTableReader(ctx context.Context, tbl table.Table, handles []kv.Handle) (_ Executor, err error) { tableReaderExec := &TableReaderExecutor{ - baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTablePlanRootID()), - table: tbl, - dagPB: e.tableRequest, - startTS: e.startTS, - readReplicaScope: e.readReplicaScope, - isStaleness: e.isStaleness, - streaming: e.tableStreaming, - columns: e.columns, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), - plans: e.tblPlans, + baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTablePlanRootID()), + table: tbl, + dagPB: e.tableRequest, + startTS: e.startTS, + readReplicaScope: e.readReplicaScope, + isStaleness: e.isStaleness, + streaming: e.tableStreaming, + columns: e.columns, + feedback: statistics.NewQueryFeedback(0, nil, 0, false), + plans: e.tblPlans, + extraPIDColumnIndex: e.extraPIDColumnIndex, } if e.isCorColInTableFilter { if tableReaderExec.dagPB.Executors, _, err = constructDistExec(e.ctx, e.tblPlans); err != nil { diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index 99670af4807f7..f1b511914caf2 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -20,6 +20,7 @@ import ( "regexp" "strconv" "strings" + "time" . "github.com/pingcap/check" "github.com/pingcap/tidb/util" @@ -474,3 +475,74 @@ func (test *testSerialSuite2) TestIndexMergeSplitTable(c *C) { tk.MustExec("SPLIT TABLE tab2 BY (5);") tk.MustQuery("SELECT /*+ use_index_merge(tab2) */ pk FROM tab2 WHERE (col4 > 565.89 OR col0 > 68 ) and col0 > 10 order by 1;").Check(testkit.Rows("0", "1", "2", "3", "4", "5", "6", "7")) } + +func (test *testSerialSuite2) TestPessimisticLockOnPartitionForIndexMerge(c *C) { + // Same purpose with TestPessimisticLockOnPartition, but test IndexMergeReader. + tk := testkit.NewTestKit(c, test.store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t1, t2") + tk.MustExec(`create table t1 (c_datetime datetime, c1 int, c2 int, primary key (c_datetime), key(c1), key(c2)) + partition by range (to_days(c_datetime)) ( + partition p0 values less than (to_days('2020-02-01')), + partition p1 values less than (to_days('2020-04-01')), + partition p2 values less than (to_days('2020-06-01')), + partition p3 values less than maxvalue)`) + tk.MustExec("create table t2 (c_datetime datetime, unique key(c_datetime))") + tk.MustExec("insert into t1 values ('2020-06-26 03:24:00', 1, 1), ('2020-02-21 07:15:33', 2, 2), ('2020-04-27 13:50:58', 3, 3)") + tk.MustExec("insert into t2 values ('2020-01-10 09:36:00'), ('2020-02-04 06:00:00'), ('2020-06-12 03:45:18')") + + tk1 := testkit.NewTestKit(c, test.store) + tk1.MustExec("use test") + tk1.MustExec("set @@tidb_partition_prune_mode = 'static'") + + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustExec("begin pessimistic") + tk.MustQuery(`select /*+ use_index_merge(t1) */ c1 from t1 join t2 + on t1.c_datetime >= t2.c_datetime + where t1.c1 < 10 or t1.c2 < 10 for update`).Sort().Check(testkit.Rows("1", "1", "1", "2", "2", "3", "3")) + tk1.MustExec("begin pessimistic") + + ch := make(chan int32, 5) + go func() { + tk1.MustExec("update t1 set c_datetime = '2020-06-26 03:24:00' where c1 = 1") + ch <- 0 + tk1.MustExec("rollback") + ch <- 0 + }() + + // Leave 50ms for tk1 to run, tk1 should be blocked at the update operation. + time.Sleep(50 * time.Millisecond) + ch <- 1 + + tk.MustExec("commit") + // tk1 should be blocked until tk commit, check the order. + c.Assert(<-ch, Equals, int32(1)) + c.Assert(<-ch, Equals, int32(0)) + <-ch // wait for goroutine to quit. + + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk1.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec("begin pessimistic") + tk.MustQuery(`select /*+ use_index_merge(t1) */ c1 from t1 join t2 + on t1.c_datetime >= t2.c_datetime + where t1.c1 < 10 or t1.c2 < 10 for update`).Sort().Check(testkit.Rows("1", "1", "1", "2", "2", "3", "3")) + tk1.MustExec("begin pessimistic") + + go func() { + tk1.MustExec("update t1 set c_datetime = '2020-06-26 03:24:00' where c1 = 1") + ch <- 0 + tk1.MustExec("rollback") + ch <- 0 + }() + + // Leave 50ms for tk1 to run, tk1 should be blocked at the update operation. + time.Sleep(50 * time.Millisecond) + ch <- 1 + + tk.MustExec("commit") + // tk1 should be blocked until tk commit, check the order. + c.Assert(<-ch, Equals, int32(1)) + c.Assert(<-ch, Equals, int32(0)) + <-ch // wait for goroutine to quit. +} diff --git a/executor/table_reader.go b/executor/table_reader.go index c9bc68a0b0aa6..5f5b8427c8dc6 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -35,7 +35,6 @@ import ( "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" - "go.uber.org/zap" ) // make sure `TableReaderExecutor` implements `Executor`. @@ -249,7 +248,6 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error physicalID := getPhysicalTableID(e.table) if physicalID != e.table.Meta().ID { // table partition in static prune mode (one TableReaderExecutor per partition) - logutil.BgLogger().Info("MJONSS: TableReaderExecutor.Next()", zap.Int64("physicalID", physicalID)) fillExtraPIDColumn(req, e.extraPIDColumnIndex.value(), physicalID) } } diff --git a/executor/union_scan.go b/executor/union_scan.go index a4cab2bf08bbe..9d3da504124e0 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -223,14 +223,11 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err us.snapshotRows = us.snapshotRows[:0] physTblIDIdx := -1 fts := make([]*types.FieldType, len(us.columns)) - //colNames := make([]string, len(us.columns)) - //colIds := make([]int64, len(us.columns)) for i := range us.columns { fts[i] = &us.columns[i].FieldType - //colNames = append(colNames, us.columns[i].Name.O) - //colIds = append(colIds, us.columns[i].ID) if us.columns[i].ID == model.ExtraPhysTblID { if physTblIDIdx >= 0 { + // TODO: remove when table partition dynamic mode is GA logutil.Logger(ctx).Warn("More than one ExtraPhysTblID column!", zap.String("table", us.table.Meta().Name.O)) } physTblIDIdx = i @@ -241,10 +238,6 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err if err != nil || us.snapshotChunkBuffer.NumRows() == 0 { return nil, err } - logutil.Logger(ctx).Info("MJONSS: getSnapshotRow", zap.String("Chunk", us.snapshotChunkBuffer.ToString(fts)), - //zap.Strings("colNames", colNames), - //zap.Int64s("colIds", colIds), - zap.Int("us colNames count", len(us.columns))) iter := chunk.NewIterator4Chunk(us.snapshotChunkBuffer) for row := iter.Begin(); row != iter.End(); row = iter.Next() { var snapshotHandle kv.Handle diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index ec665b234d0ff..a4cacd0d58f17 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3368,12 +3368,12 @@ func (s *testIntegrationSuite) TestIssue26719(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table tx (a int) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))`) - tk.MustExec(`insert into tx values (10)`) + tk.MustExec(`insert into tx values (1)`) tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec(`begin`) - tk.MustExec(`delete from tx where a in (10)`) - //tk.MustQuery(`select * from tx PARTITION(p0)`).Check(testkit.Rows()) + tk.MustExec(`delete from tx where a in (1)`) + tk.MustQuery(`select * from tx PARTITION(p0)`).Check(testkit.Rows()) tk.MustQuery(`select * from tx`).Check(testkit.Rows()) tk.MustExec(`rollback`) } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 21e4a05c11de9..2179bd368db99 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3814,13 +3814,15 @@ func (ds *DataSource) addExtraPIDColumnWithInfo(info *extraPIDInfo) { for _, col := range schema.Columns { if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID { if pidCol != nil { - logutil.BgLogger().Warn("MJONSS: Duplicate Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) + // TODO: remove when table partition dynamic prune mode is GA + logutil.BgLogger().Warn("Duplicate Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) } pidCol = col } } if pidCol == nil { - logutil.BgLogger().Warn("MJONSS: Missing Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) + // TODO: remove check/log when table partition dynamic prune mode is GA + logutil.BgLogger().Warn("Missing Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) pidCol = &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index c83c26f217458..7c922bfb9bc3f 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -479,7 +479,6 @@ func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *log } if len(p.partitionedTable) > 0 { - // TODO: What to do here? Should it be added to the tblID2Handle array instead? // If the children include partitioned tables, there is an extra partition ID column. parentUsedCols = append(parentUsedCols, p.extraPIDInfo.Columns...) } diff --git a/planner/core/util.go b/planner/core/util.go index b2427320674a5..9d0fe9d0f3bfc 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -331,7 +331,6 @@ func tableHasDirtyContent(ctx sessionctx.Context, tableInfo *model.TableInfo) bo } // Currently, we add UnionScan on every partition even though only one partition's data is changed. // This is limited by current implementation of Partition Prune. It'll be updated once we modify that part. - // TODO: Use named partitions at least, preferably pruned if possible here? for _, partition := range pi.Definitions { if ctx.HasDirtyContent(partition.ID) { return true diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index d760ade74f3c1..5ab4687c3ba05 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -37,12 +37,10 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/logutil" mockpkg "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" - "go.uber.org/zap" ) const chunkMaxRows = 1024 @@ -845,19 +843,10 @@ func (e *closureExecutor) tableScanProcessCore(key, value []byte) error { return errors.Trace(err) } // Add ExtraPhysTblID if requested + // Assumes it is always last! if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID { tblID := tablecodec.DecodeTableID(key) - logutil.BgLogger().Info("MJONSS: tableScanProcessCore", zap.Int64("tblID", tblID), zap.Int("columnIdx", len(e.columnInfos)-1)) e.scanCtx.chk.AppendInt64(len(e.columnInfos)-1, tblID) - /* - colIds := make([]int64, len(e.columnInfos)) - for i := range e.columnInfos { - colIds = append(colIds, e.columnInfos[i].ColumnId) - } - */ - logutil.BgLogger().Info("MJONSS: tableScanProcessCore chk", zap.String("Chunk", e.scanCtx.chk.ToString(e.resultFieldType)), - //zap.Int64s("colIds", colIds), - zap.Int("colIds count", len(e.columnInfos)), zap.Int("result field types count", len(e.resultFieldType))) } incRow = true return nil @@ -934,7 +923,6 @@ func (e *closureExecutor) indexScanProcessCore(key, value []byte) error { // Add ExtraPhysTblID if requested if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID { tblID := tablecodec.DecodeTableID(key) - logutil.BgLogger().Info("MJONSS: indexScanProcessCore", zap.Int64("tblID", tblID)) chk.AppendInt64(len(e.columnInfos)-1, tblID) } gotRow = true From a26a33425dff68c6e5166ae7ebbf5574690aad39 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 13 Jan 2022 11:02:17 +0100 Subject: [PATCH 04/29] Cleaned up some tests for passing CI and added warning for debug --- executor/analyze_test.go | 1 + executor/executor.go | 29 ++++++++++++++++ executor/index_merge_reader_test.go | 54 ++++++++++++++++------------- 3 files changed, 60 insertions(+), 24 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index de35933562e5a..b6ad8109a8934 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -663,6 +663,7 @@ func TestFailedAnalyzeRequest(t *testing.T) { tk.MustExec("set @@tidb_analyze_version = 1") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/buildStatsFromResult", `return(true)`)) _, err := tk.Exec("analyze table t") + require.NotNil(t, err) require.Equal(t, "mock buildStatsFromResult error", err.Error()) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/buildStatsFromResult")) } diff --git a/executor/executor.go b/executor/executor.go index 6f4867438134a..a523b681d5929 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -949,6 +949,35 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } + + // Debug print. + if physicalID == 0 || (handle.IsInt() && handle.IntValue() == 0) || (handle.Len() == 0) { + txn, _ := e.ctx.Txn(false) + sql := e.ctx.GetSessionVars().StmtCtx.OriginalSQL + tblIDs := make([]int64, 0, len(e.tblID2Handle)) + for tmpTblID := range e.tblID2Handle { + tblIDs = append(tblIDs, tmpTblID) + } + partTblLen := len(e.partitionedTable) + tblid2ColIdxMapKeys := make([]int64, 0, len(e.tblID2PIDColumnIndex)) + tblid2ColIdxMapVals := make([]int, 0, len(e.tblID2PIDColumnIndex)) + for key, val := range e.tblID2PIDColumnIndex { + tblid2ColIdxMapKeys = append(tblid2ColIdxMapKeys, key) + tblid2ColIdxMapVals = append(tblid2ColIdxMapVals, val) + } + logutil.Logger(ctx).Error("[for debug] the physicalID or handle value is unexpected", + zap.Uint64("ts", txn.StartTS()), + zap.Int64("id", id), + zap.Int64("physicalID", physicalID), + zap.Stringer("handle", handle), + zap.String("sql", sql), + zap.Int64s("tblIDs", tblIDs), + zap.Int("partTblLen", partTblLen), + zap.Int64s("tblID2PIDColIdxMapKeys", tblid2ColIdxMapKeys), + zap.Ints("tblid2ColIdxMapVals", tblid2ColIdxMapVals)) + panic("unexpected lock key, check the tidb log with for debug") + } + e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physicalID, handle)) } } diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index f1b511914caf2..5c333ec784e09 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -491,6 +491,8 @@ func (test *testSerialSuite2) TestPessimisticLockOnPartitionForIndexMerge(c *C) tk.MustExec("create table t2 (c_datetime datetime, unique key(c_datetime))") tk.MustExec("insert into t1 values ('2020-06-26 03:24:00', 1, 1), ('2020-02-21 07:15:33', 2, 2), ('2020-04-27 13:50:58', 3, 3)") tk.MustExec("insert into t2 values ('2020-01-10 09:36:00'), ('2020-02-04 06:00:00'), ('2020-06-12 03:45:18')") + tk.MustExec("analyze table t1") + tk.MustExec("analyze table t2") tk1 := testkit.NewTestKit(c, test.store) tk1.MustExec("use test") @@ -498,6 +500,33 @@ func (test *testSerialSuite2) TestPessimisticLockOnPartitionForIndexMerge(c *C) tk.MustExec("set @@tidb_partition_prune_mode = 'static'") tk.MustExec("begin pessimistic") + tk.MustQuery(`explain format='brief' select /*+ use_index_merge(t1) */ c1 from t1 join t2 + on t1.c_datetime >= t2.c_datetime + where t1.c1 < 10 or t1.c2 < 10 for update`).Check(testkit.Rows( + "Projection 16635.64 root test.t1.c1", + "└─SelectLock 16635.64 root for update 0", + " └─Projection 16635.64 root test.t1.c1, test.t1._tidb_rowid, test.t1._tidb_pid, test.t2._tidb_rowid", + " └─HashJoin 16635.64 root CARTESIAN inner join, other cond:ge(test.t1.c_datetime, test.t2.c_datetime)", + " ├─IndexReader(Build) 3.00 root index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:c_datetime(c_datetime) keep order:false", + " └─PartitionUnion(Probe) 5545.21 root ", + " ├─IndexMerge 5542.21 root ", + " │ ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t1, partition:p0, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo", + " │ ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t1, partition:p0, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", + " │ └─TableRowIDScan(Probe) 5542.21 cop[tikv] table:t1, partition:p0 keep order:false, stats:pseudo", + " ├─IndexMerge 1.00 root ", + " │ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:c1(c1) range:[-inf,10), keep order:false", + " │ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:c2(c2) range:[-inf,10), keep order:false", + " │ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + " ├─IndexMerge 1.00 root ", + " │ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p2, index:c1(c1) range:[-inf,10), keep order:false", + " │ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p2, index:c2(c2) range:[-inf,10), keep order:false", + " │ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p2 keep order:false", + " └─IndexMerge 1.00 root ", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p3, index:c1(c1) range:[-inf,10), keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p3, index:c2(c2) range:[-inf,10), keep order:false", + " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p3 keep order:false", + )) tk.MustQuery(`select /*+ use_index_merge(t1) */ c1 from t1 join t2 on t1.c_datetime >= t2.c_datetime where t1.c1 < 10 or t1.c2 < 10 for update`).Sort().Check(testkit.Rows("1", "1", "1", "2", "2", "3", "3")) @@ -521,28 +550,5 @@ func (test *testSerialSuite2) TestPessimisticLockOnPartitionForIndexMerge(c *C) c.Assert(<-ch, Equals, int32(0)) <-ch // wait for goroutine to quit. - tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") - tk1.MustExec("set @@tidb_partition_prune_mode='dynamic'") - tk.MustExec("begin pessimistic") - tk.MustQuery(`select /*+ use_index_merge(t1) */ c1 from t1 join t2 - on t1.c_datetime >= t2.c_datetime - where t1.c1 < 10 or t1.c2 < 10 for update`).Sort().Check(testkit.Rows("1", "1", "1", "2", "2", "3", "3")) - tk1.MustExec("begin pessimistic") - - go func() { - tk1.MustExec("update t1 set c_datetime = '2020-06-26 03:24:00' where c1 = 1") - ch <- 0 - tk1.MustExec("rollback") - ch <- 0 - }() - - // Leave 50ms for tk1 to run, tk1 should be blocked at the update operation. - time.Sleep(50 * time.Millisecond) - ch <- 1 - - tk.MustExec("commit") - // tk1 should be blocked until tk commit, check the order. - c.Assert(<-ch, Equals, int32(1)) - c.Assert(<-ch, Equals, int32(0)) - <-ch // wait for goroutine to quit. + // TODO: add support for index merge reader in dynamic tidb_partition_prune_mode } From 9d9eb52140f3aff90f0e79904e36cd0d586a106f Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Sat, 22 Jan 2022 20:17:21 +0100 Subject: [PATCH 05/29] wip ExtraPhysTblIDCol --- executor/builder.go | 47 +++++++++--------------- executor/distsql.go | 26 ++++++------- executor/executor.go | 1 + executor/index_merge_reader.go | 25 +++++-------- executor/partition_table_test.go | 5 ++- executor/table_reader.go | 34 ++++------------- planner/core/logical_plan_builder.go | 1 + planner/core/rule_partition_processor.go | 2 +- 8 files changed, 53 insertions(+), 88 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 51ebef47fe416..5b2cd566934fc 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -668,16 +668,6 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor } } - if len(e.partitionedTable) > 0 { - schema := v.Schema() - e.tblID2PIDColumnIndex = make(map[int64]int) - for i := 0; i < len(v.ExtraPIDInfo.Columns); i++ { - col := v.ExtraPIDInfo.Columns[i] - tblID := v.ExtraPIDInfo.TblIDs[i] - offset := schema.ColumnIndex(col) - e.tblID2PIDColumnIndex[tblID] = offset - } - } return e } @@ -3035,9 +3025,25 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea tbl, _ := b.is.TableByID(ts.Table.ID) isPartition, physicalTableID := ts.IsPartition() + var partitionPhysTblIDOffset int if isPartition { pt := tbl.(table.PartitionedTable) tbl = pt.GetPartition(physicalTableID) + if ts.Table.ID == physicalTableID { + // TODO: Remove after some testing... + panic("isPartition is set but still physicalTableID is set to logical table id!!!") + } + // NOTE for mjonss: TODO, add a comment about Global Indexes, that it is only used for index data, and also stores the Physical Table ID, so never any need to fill it in + // Only time to fill the Physical Table ID is under static pruning mode when ExtraPhysTblID has been requested! + for i, col := range v.Schema().Columns { + if col.ID == model.ExtraPhysTblID { + if i == 0 { + panic("ExtraPhysTblID in offset 0!!!") + } + partitionPhysTblIDOffset = i + break + } + } } startTS, err := b.getSnapshotTS() if err != nil { @@ -3060,9 +3066,8 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea tablePlan: v.GetTablePlan(), storeType: v.StoreType, batchCop: v.BatchCop, - } - if tbl.Meta().Partition != nil { - e.extraPIDColumnIndex = extraPIDColumnIndex(v.Schema()) + // default 0 + partitionPhysTblIDOffset: partitionPhysTblIDOffset, } e.buildVirtualColumnInfo() if containsLimit(dagReq.Executors) { @@ -3094,16 +3099,6 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea return e, nil } -func extraPIDColumnIndex(schema *expression.Schema) offsetOptional { - for idx, col := range schema.Columns { - // TODO: Handle partitioned global index, i.e. both ExtraPidColID and ExtraPhysTblID is used. - if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID { - return newOffset(idx) - } - } - return 0 -} - func (b *executorBuilder) buildMPPGather(v *plannercore.PhysicalTableReader) Executor { startTs, err := b.getSnapshotTS() if err != nil { @@ -3497,9 +3492,6 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn tblPlans: v.TablePlans, PushedLimit: v.PushedLimit, } - if ok, _ := ts.IsPartition(); ok { - e.extraPIDColumnIndex = extraPIDColumnIndex(v.Schema()) - } if containsLimit(indexReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) @@ -3651,9 +3643,6 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd } collectTable := false e.tableRequest.CollectRangeCounts = &collectTable - if tblInfo.Meta().Partition != nil { - e.extraPIDColumnIndex = extraPIDColumnIndex(v.Schema()) - } return e, nil } diff --git a/executor/distsql.go b/executor/distsql.go index 92cf0b8d60344..d6f8cf08f9deb 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -378,9 +378,6 @@ type IndexLookUpExecutor struct { stats *IndexLookUpRunTimeStats - // extraPIDColumnIndex is used for partition reader to add an extra partition ID column, default -1 - extraPIDColumnIndex offsetOptional - // cancelFunc is called when close the executor cancelFunc context.CancelFunc } @@ -659,18 +656,17 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, task *lookup table = task.partitionTable } tableReaderExec := &TableReaderExecutor{ - baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTableRootPlanID()), - table: table, - dagPB: e.tableRequest, - startTS: e.startTS, - readReplicaScope: e.readReplicaScope, - isStaleness: e.isStaleness, - columns: e.columns, - streaming: e.tableStreaming, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), - corColInFilter: e.corColInTblSide, - plans: e.tblPlans, - extraPIDColumnIndex: e.extraPIDColumnIndex, + baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTableRootPlanID()), + table: table, + dagPB: e.tableRequest, + startTS: e.startTS, + readReplicaScope: e.readReplicaScope, + isStaleness: e.isStaleness, + columns: e.columns, + streaming: e.tableStreaming, + feedback: statistics.NewQueryFeedback(0, nil, 0, false), + corColInFilter: e.corColInTblSide, + plans: e.tblPlans, } tableReaderExec.buildVirtualColumnInfo() tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, task.handles, true) diff --git a/executor/executor.go b/executor/executor.go index 90a5417d09dd9..a2055549eb7a6 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -950,6 +950,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { return err } + // TODO: Only have this enabled during development phase, remove PANIC and change log to Debug before GA. // Debug print. if physicalID == 0 || (handle.IsInt() && handle.IntValue() == 0) || (handle.Len() == 0) { txn, _ := e.ctx.Txn(false) diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 186d05ec79c47..38aa4a84f5bc5 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -116,10 +116,6 @@ type IndexMergeReaderExecutor struct { isCorColInPartialFilters []bool isCorColInTableFilter bool isCorColInPartialAccess []bool - - // extraPIDColumnIndex is used for partition reader to add an extra partition ID column - // in static tidb_partition_prune_mode - extraPIDColumnIndex offsetOptional } // Open implements the Executor Open interface @@ -603,17 +599,16 @@ func (e *IndexMergeReaderExecutor) startIndexMergeTableScanWorker(ctx context.Co func (e *IndexMergeReaderExecutor) buildFinalTableReader(ctx context.Context, tbl table.Table, handles []kv.Handle) (_ Executor, err error) { tableReaderExec := &TableReaderExecutor{ - baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTablePlanRootID()), - table: tbl, - dagPB: e.tableRequest, - startTS: e.startTS, - readReplicaScope: e.readReplicaScope, - isStaleness: e.isStaleness, - streaming: e.tableStreaming, - columns: e.columns, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), - plans: e.tblPlans, - extraPIDColumnIndex: e.extraPIDColumnIndex, + baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTablePlanRootID()), + table: tbl, + dagPB: e.tableRequest, + startTS: e.startTS, + readReplicaScope: e.readReplicaScope, + isStaleness: e.isStaleness, + streaming: e.tableStreaming, + columns: e.columns, + feedback: statistics.NewQueryFeedback(0, nil, 0, false), + plans: e.tblPlans, } if e.isCorColInTableFilter { if tableReaderExec.dagPB.Executors, _, err = constructDistExec(e.ctx, e.tblPlans); err != nil { diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 55494ea07e6cc..d87eb0352052c 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -17,6 +17,7 @@ package executor_test import ( "fmt" "math/rand" + "strconv" "strings" "time" @@ -2891,8 +2892,8 @@ partition p2 values less than (11))`) for _, mode := range partitionModes { tk.MustExec("set @@tidb_partition_prune_mode=" + mode) - for _, c := range testCases { - tk.MustExec("replace into pt values (5, 5, 5)") + for i, c := range testCases { + tk.MustExec("replace into pt values (5, 5, 5)" + " /* mode: " + mode + " func: " + strconv.Itoa(i) + " */") c() } } diff --git a/executor/table_reader.go b/executor/table_reader.go index eb4adeac0721b..83030adac300a 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -108,24 +108,9 @@ type TableReaderExecutor struct { virtualColumnRetFieldTypes []*types.FieldType // batchCop indicates whether use super batch coprocessor request, only works for TiFlash engine. batchCop bool - - // extraPIDColumnIndex is used for partition reader to add an extra partition ID column. - extraPIDColumnIndex offsetOptional -} - -// offsetOptional may be a positive integer, or invalid. -type offsetOptional int - -func newOffset(i int) offsetOptional { - return offsetOptional(i + 1) -} - -func (i offsetOptional) valid() bool { - return i != 0 -} - -func (i offsetOptional) value() int { - return int(i - 1) + // partitionPhysTblIDOffset if Physical Table ID is not filled in by storage, + // add it to column on this offset (!= 0) Used for partitioned table with static pruning + partitionPhysTblIDOffset int } // Open initializes necessary variables for using this executor. @@ -241,15 +226,12 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error return err } - // When 'select ... for update' work on a partitioned table, the table reader should - // add the partition ID as an extra column. The SelectLockExec need this information - // to construct the lock key. - if e.extraPIDColumnIndex.valid() { + // SelectLock and transaction buffer needs Physical Table ID for its full key + // For partitioned tables under static prune mode (one TableReaderExecutor per partition), + // it needs to be added by TiDB, otherwise it is added by storate (column id ExtraPhysTblID) + if e.partitionPhysTblIDOffset != 0 { physicalID := getPhysicalTableID(e.table) - if physicalID != e.table.Meta().ID { - // table partition in static prune mode (one TableReaderExecutor per partition) - fillExtraPIDColumn(req, e.extraPIDColumnIndex.value(), physicalID) - } + fillExtraPIDColumn(req, e.partitionPhysTblIDOffset, physicalID) } return nil diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 94cf4731cf045..91b682ab75973 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3823,6 +3823,7 @@ func (ds *DataSource) addExtraPIDColumnWithInfo(info *extraPIDInfo) { if pidCol == nil { // TODO: remove check/log when table partition dynamic prune mode is GA logutil.BgLogger().Warn("Missing Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) + panic("Missing Partition ID/Physical Table ID for SELECT FOR UPDATE!") pidCol = &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 57b16246f0553..8832041d7992c 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -649,7 +649,7 @@ func (s *partitionProcessor) prune(ds *DataSource, opt *logicalOptimizeOp) (Logi return s.processListPartition(ds, pi, opt) } - // We haven't implement partition by list and so on. + // We haven't implement partition by key and so on. return s.makeUnionAllChildren(ds, pi, fullRange(len(pi.Definitions)), opt) } From d6e062f45f28bc9e8904339e0644f726eeed07e7 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Sat, 22 Jan 2022 20:17:52 +0100 Subject: [PATCH 06/29] wip ExtraPhysTblIDCol TODO: Check if UseDynamicPruneMode() and only then add ExtraPhysTblIDCol in logic plan In UnionScan/SelectLockExec if partition and not ExtraPhysTblIDCol then use GetPhysicalId --- executor/builder.go | 9 +-- executor/executor.go | 78 +++++++++++--------------- executor/union_scan.go | 30 +++++----- planner/core/exhaust_physical_plans.go | 8 +-- planner/core/logical_plan_builder.go | 74 +++++++----------------- planner/core/logical_plans.go | 20 +++---- planner/core/physical_plans.go | 6 +- planner/core/planbuilder.go | 42 ++++++-------- planner/core/rule_column_pruning.go | 11 ++-- 9 files changed, 109 insertions(+), 169 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 5b2cd566934fc..ea80c347b1559 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -649,10 +649,11 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor return src } e := &SelectLockExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), src), - Lock: v.Lock, - tblID2Handle: v.TblID2Handle, - partitionedTable: v.PartitionedTable, + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), src), + Lock: v.Lock, + tblID2Handle: v.TblID2Handle, + partitionedTable: v.PartitionedTable, + tblID2PhysTblIDCol: v.TblID2PhysTblIDCol, } // filter out temporary tables because they do not store any record in tikv and should not write any lock diff --git a/executor/executor.go b/executor/executor.go index a2055549eb7a6..ad74ae098f0f3 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -905,16 +905,35 @@ type SelectLockExec struct { // All the partition tables in the children of this executor. partitionedTable []table.PartitionedTable - // When SelectLock work on the partition table, we need the partition ID - // instead of table ID to calculate the lock KV. In that case, partition ID is store as an - // extra column in the chunk row. - // tblID2PIDColumnIndex stores the column index in the chunk row. The children may be join - // of multiple tables, so the map struct is used. - tblID2PIDColumnIndex map[int64]int + // TODO: Do we even need to use a plannercore.HandleCols struct? + // tblID2PhyTblIDCol is used for partitioned tables, + // the child executor need to return an extra column containing + // the Physical Table ID (i.e. from which partition the row came from) + tblID2PhysTblIDCol map[int64]*expression.Column } // Open implements the Executor Open interface. func (e *SelectLockExec) Open(ctx context.Context) error { + if len(e.partitionedTable) > 0 { + cols := e.Schema().Columns + for i := len(cols) - 1; i > 0; i-- { + if cols[i].ID == model.ExtraPhysTblID { + found := false + for _, col := range e.tblID2PhysTblIDCol { + if cols[i].UniqueID == col.UniqueID { + found = true + break + } + } + if !found { + panic("PhysTblIDCol not find in map?!?") + } + } + } + if cols[0].ID == model.ExtraPhysTblID { + panic("Should never be the only ID?!?") + } + } return e.baseExecutor.Open(ctx) } @@ -933,53 +952,20 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { if req.NumRows() > 0 { iter := chunk.NewIterator4Chunk(req) for row := iter.Begin(); row != iter.End(); row = iter.Next() { - - for id, cols := range e.tblID2Handle { - physicalID := id - if len(e.partitionedTable) > 0 { - // Replace the table ID with partition ID. - // The partition ID is returned as an extra column from the table reader. - if offset, ok := e.tblID2PIDColumnIndex[id]; ok { - physicalID = row.GetInt64(offset) - } + for tblID, cols := range e.tblID2Handle { + if len(cols) > 1 { + panic("More than 1 Handle column for a single table in SelectLockExec!?!") } - for _, col := range cols { handle, err := col.BuildHandle(row) if err != nil { return err } - - // TODO: Only have this enabled during development phase, remove PANIC and change log to Debug before GA. - // Debug print. - if physicalID == 0 || (handle.IsInt() && handle.IntValue() == 0) || (handle.Len() == 0) { - txn, _ := e.ctx.Txn(false) - sql := e.ctx.GetSessionVars().StmtCtx.OriginalSQL - tblIDs := make([]int64, 0, len(e.tblID2Handle)) - for tmpTblID := range e.tblID2Handle { - tblIDs = append(tblIDs, tmpTblID) - } - partTblLen := len(e.partitionedTable) - tblid2ColIdxMapKeys := make([]int64, 0, len(e.tblID2PIDColumnIndex)) - tblid2ColIdxMapVals := make([]int, 0, len(e.tblID2PIDColumnIndex)) - for key, val := range e.tblID2PIDColumnIndex { - tblid2ColIdxMapKeys = append(tblid2ColIdxMapKeys, key) - tblid2ColIdxMapVals = append(tblid2ColIdxMapVals, val) - } - logutil.Logger(ctx).Error("[for debug] the physicalID or handle value is unexpected", - zap.Uint64("ts", txn.StartTS()), - zap.Int64("id", id), - zap.Int64("physicalID", physicalID), - zap.Stringer("handle", handle), - zap.String("sql", sql), - zap.Int64s("tblIDs", tblIDs), - zap.Int("partTblLen", partTblLen), - zap.Int64s("tblID2PIDColIdxMapKeys", tblid2ColIdxMapKeys), - zap.Ints("tblid2ColIdxMapVals", tblid2ColIdxMapVals)) - panic("unexpected lock key, check the tidb log with for debug") + physTblID := tblID + if physTblCol, ok := e.tblID2PhysTblIDCol[tblID]; ok { + physTblID = row.GetInt64(physTblCol.Index) } - - e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physicalID, handle)) + e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physTblID, handle)) } } } diff --git a/executor/union_scan.go b/executor/union_scan.go index 9d3da504124e0..5c319c000fd40 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -63,6 +63,9 @@ type UnionScanExec struct { // cacheTable not nil means it's reading from cached table. cacheTable kv.MemBuffer collators []collate.Collator + + // If partitioned table and the physical table id is encoded in the chuck at this column index + physTblIDIdx *int } // Open implements the Executor Open interface. @@ -89,6 +92,17 @@ func (us *UnionScanExec) open(ctx context.Context) error { return err } + for i := range us.columns { + if us.columns[i].ID == model.ExtraPhysTblID { + if us.physTblIDIdx != nil { + // TODO: remove when table partition dynamic mode is GA (and add a break afterwards) + logutil.Logger(ctx).Warn("More than one ExtraPhysTblID column!", zap.String("table", us.table.Meta().Name.O)) + panic("More than one ExtraPhysTblID columns!!!") + } + us.physTblIDIdx = new(int) + *us.physTblIDIdx = i + } + } mb := txn.GetMemBuffer() mb.RLock() defer mb.RUnlock() @@ -221,18 +235,6 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err var err error us.cursor4SnapshotRows = 0 us.snapshotRows = us.snapshotRows[:0] - physTblIDIdx := -1 - fts := make([]*types.FieldType, len(us.columns)) - for i := range us.columns { - fts[i] = &us.columns[i].FieldType - if us.columns[i].ID == model.ExtraPhysTblID { - if physTblIDIdx >= 0 { - // TODO: remove when table partition dynamic mode is GA - logutil.Logger(ctx).Warn("More than one ExtraPhysTblID column!", zap.String("table", us.table.Meta().Name.O)) - } - physTblIDIdx = i - } - } for len(us.snapshotRows) == 0 { err = Next(ctx, us.children[0], us.snapshotChunkBuffer) if err != nil || us.snapshotChunkBuffer.NumRows() == 0 { @@ -246,8 +248,8 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err return nil, err } var checkKey kv.Key - if physTblIDIdx >= 0 { - tblID := row.GetInt64(physTblIDIdx) + if us.physTblIDIdx != nil { + tblID := row.GetInt64(*us.physTblIDIdx) checkKey = tablecodec.EncodeRowKeyWithHandle(tblID, snapshotHandle) } else { checkKey = tablecodec.EncodeRecordKey(us.table.RecordPrefix(), snapshotHandle) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index db9c16e722d78..4aed24279db8b 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2740,10 +2740,10 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } childProp := prop.CloneEssentialFields() lock := PhysicalLock{ - Lock: p.Lock, - TblID2Handle: p.tblID2Handle, - PartitionedTable: p.partitionedTable, - ExtraPIDInfo: p.extraPIDInfo, + Lock: p.Lock, + TblID2Handle: p.tblID2Handle, + PartitionedTable: p.partitionedTable, + TblID2PhysTblIDCol: p.tblID2PhysTblIDCol, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) return []PhysicalPlan{lock}, true, nil } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 91b682ab75973..a42e1d251f0b4 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -55,10 +55,8 @@ import ( util2 "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/set" - "go.uber.org/zap" ) const ( @@ -3783,10 +3781,19 @@ func (ds *DataSource) newExtraHandleSchemaCol() *expression.Column { } } -// addExtraPIDColumn add an extra PID column for partition table. +// AddExtraPhysTblIDColumn for partition table. // 'select ... for update' on a partition table need to know the partition ID // to construct the lock key, so this column is added to the chunk row. -func (ds *DataSource) addExtraPIDColumn() { +// Also needed for checking against the sessions transaction buffer +func (ds *DataSource) AddExtraPhysTblIDColumn() *expression.Column { + // Avoid adding multiple times + // TODO: Can sd.Columns be empty? + cols := ds.TblCols + for i := len(cols) - 1; i >= 0; i-- { + if cols[i].ID == model.ExtraPhysTblID { + return cols[i] + } + } pidCol := &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), @@ -3804,47 +3811,7 @@ func (ds *DataSource) addExtraPIDColumn() { OrigColName: model.ExtraPhysTblIdName, }) ds.TblCols = append(ds.TblCols, pidCol) -} - -// addExtraPIDColumnWithInfo add an extra PID column for partition table. -// 'select ... for update' on a partition table need to know the partition ID -func (ds *DataSource) addExtraPIDColumnWithInfo(info *extraPIDInfo) { - var pidCol *expression.Column - schema := ds.Schema() - for _, col := range schema.Columns { - if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID { - if pidCol != nil { - // TODO: remove when table partition dynamic prune mode is GA - logutil.BgLogger().Warn("Duplicate Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) - } - pidCol = col - } - } - if pidCol == nil { - // TODO: remove check/log when table partition dynamic prune mode is GA - logutil.BgLogger().Warn("Missing Partition ID/Physical table id for SELECT FOR UPDATE", zap.String("table", ds.TableAsName.O)) - panic("Missing Partition ID/Physical Table ID for SELECT FOR UPDATE!") - pidCol = &expression.Column{ - RetType: types.NewFieldType(mysql.TypeLonglong), - UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), - ID: model.ExtraPidColID, - OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.tableInfo.Name, model.ExtraPartitionIdName), - } - - ds.Columns = append(ds.Columns, model.NewExtraPartitionIDColInfo()) - schema := ds.Schema() - schema.Append(pidCol) - ds.names = append(ds.names, &types.FieldName{ - DBName: ds.DBName, - TblName: ds.TableInfo().Name, - ColName: model.ExtraPartitionIdName, - OrigColName: model.ExtraPartitionIdName, - }) - ds.TblCols = append(ds.TblCols, pidCol) - } - - info.Columns = append(info.Columns, pidCol) - info.TblIDs = append(info.TblIDs, ds.TableInfo().ID) + return pidCol } var ( @@ -4221,18 +4188,14 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if dirty || tableInfo.TempTableType == model.TempTableLocal { us := LogicalUnionScan{handleCols: handleCols}.Init(b.ctx, b.getSelectOffset()) us.SetChildren(ds) - result = us - } - if tableInfo.GetPartitionInfo() != nil { - if b.ctx.GetSessionVars().UseDynamicPartitionPrune() { - // Use the new partition implementation, add partition id as handle/hidden column. - // dirty => must check transaction buffer, which uses Physical table id, so we need it per record from the partitioned table - // IsPessimistic => SelectLock needs the Physical table id for locking each row. - if dirty || b.ctx.GetSessionVars().TxnCtx.IsPessimistic { - ds.addExtraPIDColumn() - } + if tableInfo.Partition != nil { + us.ExtraPhysTblIDCol = ds.AddExtraPhysTblIDColumn() } + result = us } + + // Adding ExtraPhysTblIDCol for SelectLock (SELECT FOR UPDATE) and UnionScan (transaction buffer handling) are done when building SelectLock and UnionScan + // If a table is a cache table, it is judged whether it satisfies the conditions of read cache. if tableInfo.TableCacheStatusType == model.TableCacheStatusEnable && b.ctx.GetSessionVars().SnapshotTS == 0 && !b.ctx.GetSessionVars().StmtCtx.IsStaleness { cachedTable := tbl.(table.CachedTable) @@ -4247,6 +4210,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as sessionVars.StmtCtx.ReadFromTableCache = true us := LogicalUnionScan{handleCols: handleCols, cacheTable: cacheData}.Init(b.ctx, b.getSelectOffset()) us.SetChildren(ds) + // Table cache does not support table partitions, so no need to add ExtraPhysTblIDCol here! result = us } else { if !b.inUpdateStmt && !b.inDeleteStmt && !sessionVars.StmtCtx.InExplainStmt { diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index c322f7e9489c1..5bd8f1db0a0f0 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -530,6 +530,9 @@ type LogicalUnionScan struct { // cacheTable not nil means it's reading from cached table. cacheTable kv.MemBuffer + + // If partitioned table, this column is used for the Physical Table ID + ExtraPhysTblIDCol *expression.Column } // DataSource represents a tableScan without condition push down. @@ -1045,16 +1048,6 @@ type LogicalLimit struct { limitHints limitHintInfo } -// extraPIDInfo is used by SelectLock on partitioned table, the TableReader need -// to return the partition id column. -// Because SelectLock has to used that partition id to encode the lock key. -// the child of SelectLock may be Join, so that table can be multiple extra PID columns. -// fields are for each of the table, and TblIDs are the corresponding table IDs. -type extraPIDInfo struct { - Columns []*expression.Column - TblIDs []int64 -} - // LogicalLock represents a select lock plan. type LogicalLock struct { baseLogicalPlan @@ -1062,9 +1055,10 @@ type LogicalLock struct { Lock *ast.SelectLockInfo tblID2Handle map[int64][]HandleCols partitionedTable []table.PartitionedTable - // extraPIDInfo is used when it works on partition table, the child executor - // need to return an extra partition ID column in the chunk row. - extraPIDInfo + // tblID2phyTblIDCol is used for partitioned tables, + // the child executor need to return an extra column containing + // the Physical Table ID (i.e. from which partition the row came from) + tblID2PhysTblIDCol map[int64]*expression.Column } // WindowFrame represents a window function frame. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 6293bba4b5073..c346d766a33e6 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -959,9 +959,9 @@ type PhysicalLock struct { Lock *ast.SelectLockInfo - TblID2Handle map[int64][]HandleCols - PartitionedTable []table.PartitionedTable - ExtraPIDInfo extraPIDInfo + TblID2Handle map[int64][]HandleCols + PartitionedTable []table.PartitionedTable + TblID2PhysTblIDCol map[int64]*expression.Column } // PhysicalLimit is the physical operator of Limit. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 338eb969be228..9c3d265ec1073 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1226,50 +1226,44 @@ func removeTiflashDuringStaleRead(paths []*util.AccessPath) []*util.AccessPath { } func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) (*LogicalLock, error) { - selectLock := LogicalLock{ - Lock: lock, - tblID2Handle: b.handleHelper.tailMap(), - partitionedTable: b.partitionedTable, - }.Init(b.ctx) - selectLock.SetChildren(src) - + tblID2PhysTblIDCol := make(map[int64]*expression.Column) if len(b.partitionedTable) > 0 { + // TODO: Add the ExtraPhysTblIDCols here instead of in the other place. The same for UnionScan!!! // If a chunk row is read from a partitioned table, which partition the row // comes from is unknown. With the existence of Join, the situation could be // even worse: SelectLock have to know the `pid` to construct the lock key. // To solve the problem, an extra `pid` column is add to the schema, and the // DataSource need to return the `pid` information in the chunk row. - err := addExtraPIDColumnToDataSource(src, &selectLock.extraPIDInfo) - if err != nil { - return nil, err - } + setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) + + // TODO: Re-evaluate this: // TODO: Dynamic partition mode does not support adding extra pid column to the data source. // (Because one table reader can read from multiple partitions, which partition a chunk row comes from is unknown) // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. b.optFlag = b.optFlag | flagPartitionProcessor } + selectLock := LogicalLock{ + Lock: lock, + tblID2Handle: b.handleHelper.tailMap(), + partitionedTable: b.partitionedTable, + tblID2PhysTblIDCol: tblID2PhysTblIDCol, + }.Init(b.ctx) + selectLock.SetChildren(src) return selectLock, nil } -func addExtraPIDColumnToDataSource(p LogicalPlan, info *extraPIDInfo) error { - switch raw := p.(type) { +func setExtraPhysTblIDColsOnDataSource(p LogicalPlan, tblID2PhysTblIDCol map[int64]*expression.Column) { + switch ds := p.(type) { case *DataSource: - // Fix issue 26250, do not add extra pid column to normal table. - if raw.tableInfo.GetPartitionInfo() == nil { - return nil + if ds.tableInfo.GetPartitionInfo() == nil { + return } - raw.addExtraPIDColumnWithInfo(info) - return nil + tblID2PhysTblIDCol[ds.tableInfo.ID] = ds.AddExtraPhysTblIDColumn() default: - var err error for _, child := range p.Children() { - err = addExtraPIDColumnToDataSource(child, info) - if err != nil { - return err - } + setExtraPhysTblIDColsOnDataSource(child, tblID2PhysTblIDCol) } } - return nil } func (b *PlanBuilder) buildPrepare(x *ast.PrepareStmt) Plan { diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 7c922bfb9bc3f..3430ed8fdf210 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -478,17 +478,16 @@ func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *log return p.baseLogicalPlan.PruneColumns(parentUsedCols, opt) } - if len(p.partitionedTable) > 0 { - // If the children include partitioned tables, there is an extra partition ID column. - parentUsedCols = append(parentUsedCols, p.extraPIDInfo.Columns...) - } - - for _, cols := range p.tblID2Handle { + for tblID, cols := range p.tblID2Handle { for _, col := range cols { for i := 0; i < col.NumCols(); i++ { parentUsedCols = append(parentUsedCols, col.GetCol(i)) } } + if physTblIDCol, ok := p.tblID2PhysTblIDCol[tblID]; ok { + // If the children include partitioned tables, there is an extra partition ID column. + parentUsedCols = append(parentUsedCols, physTblIDCol) + } } return p.children[0].PruneColumns(parentUsedCols, opt) } From d5d79961af83517e71e39cf3a35d38d5c77d5e86 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 24 Jan 2022 10:34:37 +0100 Subject: [PATCH 07/29] WIP fix SelectLock and UnionScan for both dynamic and static prune --- executor/executor.go | 37 +++++++++++++++++++--------- executor/index_merge_reader_test.go | 2 +- executor/union_scan.go | 9 ++----- planner/core/logical_plan_builder.go | 3 ++- planner/core/planbuilder.go | 30 ++++++++++++---------- session/pessimistic_test.go | 1 + tools/check/ut.go | 12 ++++++--- 7 files changed, 56 insertions(+), 38 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index ad74ae098f0f3..b87388320a221 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -906,22 +906,32 @@ type SelectLockExec struct { partitionedTable []table.PartitionedTable // TODO: Do we even need to use a plannercore.HandleCols struct? - // tblID2PhyTblIDCol is used for partitioned tables, + // tblID2PhyTblIDCol is used for partitioned tables in dynamic prune mode, // the child executor need to return an extra column containing // the Physical Table ID (i.e. from which partition the row came from) + // Used during building tblID2PhysTblIDCol map[int64]*expression.Column + + // Used during execution + tblID2PhysTblIDColIdx map[int64]int } // Open implements the Executor Open interface. func (e *SelectLockExec) Open(ctx context.Context) error { + logutil.Logger(ctx).Info("MJONSS: SelectLocExec::Open()") if len(e.partitionedTable) > 0 { + // This should be possible to do by going through the tblID2Handle and then see if the TableById gives a partitioned table or not, and then create the map for static prune? Maybe works for dynamic too? + e.tblID2PhysTblIDColIdx = make(map[int64]int) cols := e.Schema().Columns for i := len(cols) - 1; i > 0; i-- { if cols[i].ID == model.ExtraPhysTblID { + logutil.Logger(ctx).Info("MJONSS: SelectLocExec::Open()", zap.Int("i", i), zap.Int64("UniqueID", cols[i].UniqueID)) found := false - for _, col := range e.tblID2PhysTblIDCol { + for tblID, col := range e.tblID2PhysTblIDCol { if cols[i].UniqueID == col.UniqueID { + logutil.Logger(ctx).Info("MJONSS: SelectLocExec::Open() Found in e.tblID2PhysTblIDCol", zap.Int("schema col index", cols[i].Index), zap.Int("PhysTblIDCol index", col.Index)) found = true + e.tblID2PhysTblIDColIdx[tblID] = i break } } @@ -946,25 +956,29 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } // If there's no handle or it's not a `SELECT FOR UPDATE` statement. if len(e.tblID2Handle) == 0 || (!plannercore.IsSelectForUpdateLockType(e.Lock.LockType)) { + logutil.Logger(ctx).Info("MJONSS: SelectLocExec No handle or lock!!!") return nil } + touchedTableIDs := make(map[int64]bool) if req.NumRows() > 0 { iter := chunk.NewIterator4Chunk(req) for row := iter.Begin(); row != iter.End(); row = iter.Next() { for tblID, cols := range e.tblID2Handle { - if len(cols) > 1 { - panic("More than 1 Handle column for a single table in SelectLockExec!?!") - } + touchedTableIDs[tblID] = true + logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int64("tableID", tblID)) for _, col := range cols { handle, err := col.BuildHandle(row) if err != nil { return err } physTblID := tblID - if physTblCol, ok := e.tblID2PhysTblIDCol[tblID]; ok { - physTblID = row.GetInt64(physTblCol.Index) + if physTblColIdx, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { + physTblID = row.GetInt64(physTblColIdx) + touchedTableIDs[physTblID] = true + logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int("physTblColIdx", physTblColIdx), zap.Int64("tableID from row", physTblID)) } + //logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int64("tableID", physTblID), zap.Int64("handle", handle.IntValue())) e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physTblID, handle)) } } @@ -981,13 +995,12 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { if len(e.tblID2Handle) > 0 { for id := range e.tblID2Handle { e.updateDeltaForTableID(id) + delete(touchedTableIDs, id) } } - if len(e.partitionedTable) > 0 { - for _, p := range e.partitionedTable { - pid := p.Meta().ID - e.updateDeltaForTableID(pid) - } + + for id := range touchedTableIDs { + e.updateDeltaForTableID(id) } return doLockKeys(ctx, e.ctx, newLockCtx(e.ctx.GetSessionVars(), lockWaitTime), e.keys...) diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index 5c333ec784e09..be3929a630039 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -505,7 +505,7 @@ func (test *testSerialSuite2) TestPessimisticLockOnPartitionForIndexMerge(c *C) where t1.c1 < 10 or t1.c2 < 10 for update`).Check(testkit.Rows( "Projection 16635.64 root test.t1.c1", "└─SelectLock 16635.64 root for update 0", - " └─Projection 16635.64 root test.t1.c1, test.t1._tidb_rowid, test.t1._tidb_pid, test.t2._tidb_rowid", + " └─Projection 16635.64 root test.t1.c1, test.t1._tidb_rowid, test.t1._tidb_tid, test.t2._tidb_rowid", " └─HashJoin 16635.64 root CARTESIAN inner join, other cond:ge(test.t1.c_datetime, test.t2.c_datetime)", " ├─IndexReader(Build) 3.00 root index:IndexFullScan", " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:c_datetime(c_datetime) keep order:false", diff --git a/executor/union_scan.go b/executor/union_scan.go index 5c319c000fd40..5d40c43350d96 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -29,8 +29,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/logutil" - "go.uber.org/zap" ) // UnionScanExec merges the rows from dirty table and the rows from distsql request. @@ -94,12 +92,9 @@ func (us *UnionScanExec) open(ctx context.Context) error { for i := range us.columns { if us.columns[i].ID == model.ExtraPhysTblID { - if us.physTblIDIdx != nil { - // TODO: remove when table partition dynamic mode is GA (and add a break afterwards) - logutil.Logger(ctx).Warn("More than one ExtraPhysTblID column!", zap.String("table", us.table.Meta().Name.O)) - panic("More than one ExtraPhysTblID columns!!!") + if us.physTblIDIdx == nil { + us.physTblIDIdx = new(int) } - us.physTblIDIdx = new(int) *us.physTblIDIdx = i } } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index a42e1d251f0b4..189293855bb33 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4188,7 +4188,8 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if dirty || tableInfo.TempTableType == model.TempTableLocal { us := LogicalUnionScan{handleCols: handleCols}.Init(b.ctx, b.getSelectOffset()) us.SetChildren(ds) - if tableInfo.Partition != nil { + if tableInfo.Partition != nil && b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + // table partition prune mode == dynamic (Single TableReader, needs the PhysTblID from storage) us.ExtraPhysTblIDCol = ds.AddExtraPhysTblIDColumn() } result = us diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 9c3d265ec1073..67a6fcc94c90c 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1228,19 +1228,23 @@ func removeTiflashDuringStaleRead(paths []*util.AccessPath) []*util.AccessPath { func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) (*LogicalLock, error) { tblID2PhysTblIDCol := make(map[int64]*expression.Column) if len(b.partitionedTable) > 0 { - // TODO: Add the ExtraPhysTblIDCols here instead of in the other place. The same for UnionScan!!! - // If a chunk row is read from a partitioned table, which partition the row - // comes from is unknown. With the existence of Join, the situation could be - // even worse: SelectLock have to know the `pid` to construct the lock key. - // To solve the problem, an extra `pid` column is add to the schema, and the - // DataSource need to return the `pid` information in the chunk row. - setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) - - // TODO: Re-evaluate this: - // TODO: Dynamic partition mode does not support adding extra pid column to the data source. - // (Because one table reader can read from multiple partitions, which partition a chunk row comes from is unknown) - // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. - b.optFlag = b.optFlag | flagPartitionProcessor + if b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + // If a chunk row is read from a partitioned table, which partition the row + // comes from is unknown. With the existence of Join, the situation could be + // even worse: SelectLock have to know the `pid` to construct the lock key. + // To solve the problem, an extra `pid` column is add to the schema, and the + // DataSource need to return the `pid` information in the chunk row. + // table partition prune mode == dynamic (Single TableReader, needs the PhysTblID from storage) + setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) + + } else { + // Do this for static mode as well, but fill it in by the TableReader instead? + // TODO: filter it out when sending to coprocessor and add it in TableReader! + setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) + // (Because one table reader can read from multiple partitions, which partition a chunk row comes from is unknown) + // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. + b.optFlag = b.optFlag | flagPartitionProcessor + } } selectLock := LogicalLock{ Lock: lock, diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index aa7e58eb6c616..78a4f56d1455e 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -2465,6 +2465,7 @@ func (s *testPessimisticSuite) TestIssue21498(c *C) { tk.MustQuery("select * from t s, t t1 where s.v = 23 and s.id = t1.id").Check(testkit.Rows("2 23 200 2 23 200")) tk.MustQuery("select * from t s, t t1 where s.v = 24 and s.id = t1.id").Check(testkit.Rows()) tk.MustQuery("select * from t s, t t1 where s.v = 23 and s.id = t1.id for update").Check(testkit.Rows()) + // TODO: Do the same with Partitioned Table!!! Since this query leads to two columns in SelectLocExec.tblID2Handle!!! tk.MustQuery("select * from t s, t t1 where s.v = 24 and s.id = t1.id for update").Check(testkit.Rows("2 24 200 2 24 200")) tk.MustExec("delete from t where v = 24") tk.CheckExecResult(1, 0) diff --git a/tools/check/ut.go b/tools/check/ut.go index 4e8c47a70bb5c..7e711984e219a 100644 --- a/tools/check/ut.go +++ b/tools/check/ut.go @@ -72,6 +72,7 @@ type task struct { var P int var workDir string +var verbose bool = true func cmdList(args ...string) bool { pkgs, err := listPackages() @@ -392,14 +393,17 @@ func (n *numa) runTestCase(pkg string, fn string, old bool) (res testResult) { return res } -func (n *numa) testCommandWithNumaCtl(exe string, fn string, old bool) *exec.Cmd { +func (n *numa) testCommandWithNumaCtl(exe, fn string, old bool) *exec.Cmd { if old { // numactl --physcpubind 3 -- session.test -test.run '^TestT$' -check.f testTxnStateSerialSuite.TestTxnInfoWithPSProtoco - return exec.Command( - "numactl", "--physcpubind", n.cpu, "--", + args := []string{"numactl", "--physcpubind", n.cpu, "--", exe, "-test.timeout", "20s", - "-test.cpu", "1", "-test.run", "^TestT$", "-check.f", fn) + "-test.cpu", "1", "-test.run", "^TestT$", "-check.f", fn} + if verbose { + fmt.Println(strings.Join(args, " ")) + } + return exec.Command(args[0], args[1:]...) } // numactl --physcpubind 3 -- session.test -test.run TestClusteredPrefixColum From 670e07490a8273f20d9b06b3c337e190f66453d7 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 2 Feb 2022 18:31:15 +0100 Subject: [PATCH 08/29] added test for #30382 --- executor/executor_test.go | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/executor/executor_test.go b/executor/executor_test.go index 1a4e7b16a966b..edd49455a718f 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -9887,3 +9887,34 @@ func (s *testSerialSuite) TestFix31537(c *C) { );`) tk.MustQuery(`trace plan SELECT T_ID, T_S_SYMB, T_QTY, ST_NAME, TH_DTS FROM ( SELECT T_ID AS ID FROM TRADE WHERE T_CA_ID = 43000014236 ORDER BY T_DTS DESC LIMIT 10 ) T, TRADE, TRADE_HISTORY, STATUS_TYPE WHERE TRADE.T_ID = ID AND TRADE_HISTORY.TH_T_ID = TRADE.T_ID AND STATUS_TYPE.ST_ID = TRADE_HISTORY.TH_ST_ID ORDER BY TH_DTS DESC LIMIT 30;`) } + +func (s *testSuiteP1) TestIssue30382(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_enable_list_partition = ON;") + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int) , key(c_str(2)) , key(c_decimal) ) partition by list (c_int) ( partition p0 values IN (1, 5, 9, 13, 17, 21, 25, 29, 33, 37), partition p1 values IN (2, 6, 10, 14, 18, 22, 26, 30, 34, 38), partition p2 values IN (3, 7, 11, 15, 19, 23, 27, 31, 35, 39), partition p3 values IN (4, 8, 12, 16, 20, 24, 28, 32, 36, 40)) ;") + tk.MustExec("create table t2 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int) , key(c_str) , key(c_decimal) ) partition by hash (c_int) partitions 4;") + tk.MustExec("insert into t1 values (6, 'musing mayer', 1.280), (7, 'wizardly heisenberg', 6.589), (8, 'optimistic swirles', 9.633), (9, 'hungry haslett', 2.659), (10, 'stupefied wiles', 2.336);") + tk.MustExec("insert into t2 select * from t1 ;") + tk.MustExec("begin;") + tk.MustQuery("select * from t1 where c_str <> any (select c_str from t2 where c_decimal < 5) for update;").Sort().Check(testkit.Rows( + "10 stupefied wiles 2.336000", + "6 musing mayer 1.280000", + "7 wizardly heisenberg 6.589000", + "8 optimistic swirles 9.633000", + "9 hungry haslett 2.659000")) + tk.MustQuery("explain format = 'brief' select * from t1 where c_str <> any (select c_str from t2 where c_decimal < 5) for update;").Check(testkit.Rows( + "SelectLock 6400.00 root for update 0", + "└─HashJoin 6400.00 root CARTESIAN inner join, other cond:or(gt(Column#8, 1), or(ne(test.t1.c_str, Column#7), if(ne(Column#9, 0), NULL, 0)))", + " ├─Selection(Build) 0.80 root ne(Column#10, 0)", + " │ └─StreamAgg 1.00 root funcs:max(Column#17)->Column#7, funcs:count(distinct Column#18)->Column#8, funcs:sum(Column#19)->Column#9, funcs:count(1)->Column#10", + " │ └─Projection 3323.33 root test.t2.c_str, test.t2.c_str, cast(isnull(test.t2.c_str), decimal(20,0) BINARY)->Column#19", + " │ └─TableReader 3323.33 root partition:all data:Selection", + " │ └─Selection 3323.33 cop[tikv] lt(test.t2.c_decimal, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 8000.00 root partition:all data:Selection", + " └─Selection 8000.00 cop[tikv] if(isnull(test.t1.c_str), NULL, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) + tk.MustExec("commit") +} From 8e0a883d980e5e5c820249483e4a2d7a38c955b0 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 2 Feb 2022 18:38:56 +0100 Subject: [PATCH 09/29] Added another test for SelectLock (from pr/30732) --- executor/partition_table_test.go | 82 ++++++++++++++++++++++++++++++++ 1 file changed, 82 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index d87eb0352052c..86b55a34657f2 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -3121,3 +3121,85 @@ func (s *partitionTableSuite) TestIssue26251(c *C) { c.Fail() } } + +func (s *partitionTableSuite) TestLeftJoinForUpdate(c *C) { + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("create database TestLeftJoinForUpdate") + defer tk1.MustExec("drop database TestLeftJoinForUpdate") + tk1.MustExec("use TestLeftJoinForUpdate") + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec("use TestLeftJoinForUpdate") + tk3 := testkit.NewTestKit(c, s.store) + tk3.MustExec("use TestLeftJoinForUpdate") + + tk1.MustExec("drop table if exists nt, pt") + tk1.MustExec("create table nt (id int, col varchar(32), primary key (id))") + tk1.MustExec("create table pt (id int, col varchar(32), primary key (id)) partition by hash(id) partitions 4") + + resetData := func() { + tk1.MustExec("truncate table nt") + tk1.MustExec("truncate table pt") + tk1.MustExec("insert into nt values (1, 'hello')") + tk1.MustExec("insert into pt values (2, 'test')") + } + + // ========================== First round of test ================== + // partition table left join normal table. + // ================================================================= + resetData() + ch := make(chan int, 10) + tk1.MustExec("begin pessimistic") + // No union scan + tk1.MustQuery("select * from pt left join nt on pt.id = nt.id for update").Check(testkit.Rows("2 test ")) + go func() { + // Check the key is locked. + tk2.MustExec("update pt set col = 'xxx' where id = 2") + ch <- 2 + }() + + // Union scan + tk1.MustExec("insert into pt values (1, 'world')") + tk1.MustQuery("select * from pt left join nt on pt.id = nt.id for update").Sort().Check(testkit.Rows("1 world 1 hello", "2 test ")) + go func() { + // Check the key is locked. + tk3.MustExec("update nt set col = 'yyy' where id = 1") + ch <- 3 + }() + + // Give chance for the goroutines to run first. + time.Sleep(80 * time.Millisecond) + ch <- 1 + tk1.MustExec("rollback") + + checkOrder := func() { + c.Assert(<-ch, Equals, 1) + v1 := <-ch + v2 := <-ch + c.Assert((v1 == 2 && v2 == 3) || (v1 == 3 && v2 == 2), IsTrue) + } + checkOrder() + + // ========================== Another round of test ================== + // normal table left join partition table. + // =================================================================== + resetData() + tk1.MustExec("begin pessimistic") + // No union scan + tk1.MustQuery("select * from nt left join pt on pt.id = nt.id for update").Check(testkit.Rows("1 hello ")) + + // Union scan + tk1.MustExec("insert into pt values (1, 'world')") + tk1.MustQuery("select * from nt left join pt on pt.id = nt.id for update").Check(testkit.Rows("1 hello 1 world")) + go func() { + tk2.MustExec("replace into pt values (1, 'aaa')") + ch <- 2 + }() + go func() { + tk3.MustExec("update nt set col = 'bbb' where id = 1") + ch <- 3 + }() + time.Sleep(80 * time.Millisecond) + ch <- 1 + tk1.MustExec("rollback") + checkOrder() +} From 5611b0e90ecddd561268ce40ab12920fdd2bf885 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 2 Feb 2022 18:54:26 +0100 Subject: [PATCH 10/29] added test for issue#28073 --- executor/union_scan_test.go | 38 +++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index 6133010ec29da..b4e8f79afbfdf 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -18,6 +18,7 @@ import ( "fmt" "testing" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) @@ -416,3 +417,40 @@ func TestForApplyAndUnionScan(t *testing.T) { tk.MustQuery("select c_int, c_str from t where (select count(*) from t1 where t1.c_int in (t.c_int, t.c_int + 2, t.c_int + 10)) > 2").Check(testkit.Rows()) tk.MustExec("rollback") } + +func TestIssue28073(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int, c_str) , key(c_int)) partition by hash (c_int) partitions 4") + tk.MustExec("create table t2 like t1") + tk.MustExec("insert into t1 values (1, 'flamboyant mcclintock')") + tk.MustExec("insert into t2 select * from t1") + + tk.MustExec("begin") + tk.MustExec("insert into t2 (c_int, c_str) values (2, 'romantic grothendieck')") + tk.MustQuery("select * from t2 left join t1 on t1.c_int = t2.c_int for update").Sort().Check( + testkit.Rows( + "1 flamboyant mcclintock 1 flamboyant mcclintock", + "2 romantic grothendieck ", + )) + tk.MustExec("commit") + + // Check no key is written to table ID 0 + txn, err := store.Begin() + require.NoError(t, err) + start := tablecodec.EncodeTablePrefix(0) + end := tablecodec.EncodeTablePrefix(1) + iter, err := txn.Iter(start, end) + require.NoError(t, err) + + exist := false + for iter.Valid() { + require.Nil(t, iter.Next()) + exist = true + break + } + require.False(t, exist) +} From aaf83691a4164fe62c4d62892b9c5e768d1de024 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 3 Feb 2022 02:21:26 +0100 Subject: [PATCH 11/29] reverted hack in ut tool --- executor/partition_table_test.go | 5 ++--- tools/check/ut.go | 12 ++++-------- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 86b55a34657f2..2d2252a803181 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -17,7 +17,6 @@ package executor_test import ( "fmt" "math/rand" - "strconv" "strings" "time" @@ -2892,8 +2891,8 @@ partition p2 values less than (11))`) for _, mode := range partitionModes { tk.MustExec("set @@tidb_partition_prune_mode=" + mode) - for i, c := range testCases { - tk.MustExec("replace into pt values (5, 5, 5)" + " /* mode: " + mode + " func: " + strconv.Itoa(i) + " */") + for _, c := range testCases { + tk.MustExec("replace into pt values (5, 5, 5)") c() } } diff --git a/tools/check/ut.go b/tools/check/ut.go index 7e711984e219a..4e8c47a70bb5c 100644 --- a/tools/check/ut.go +++ b/tools/check/ut.go @@ -72,7 +72,6 @@ type task struct { var P int var workDir string -var verbose bool = true func cmdList(args ...string) bool { pkgs, err := listPackages() @@ -393,17 +392,14 @@ func (n *numa) runTestCase(pkg string, fn string, old bool) (res testResult) { return res } -func (n *numa) testCommandWithNumaCtl(exe, fn string, old bool) *exec.Cmd { +func (n *numa) testCommandWithNumaCtl(exe string, fn string, old bool) *exec.Cmd { if old { // numactl --physcpubind 3 -- session.test -test.run '^TestT$' -check.f testTxnStateSerialSuite.TestTxnInfoWithPSProtoco - args := []string{"numactl", "--physcpubind", n.cpu, "--", + return exec.Command( + "numactl", "--physcpubind", n.cpu, "--", exe, "-test.timeout", "20s", - "-test.cpu", "1", "-test.run", "^TestT$", "-check.f", fn} - if verbose { - fmt.Println(strings.Join(args, " ")) - } - return exec.Command(args[0], args[1:]...) + "-test.cpu", "1", "-test.run", "^TestT$", "-check.f", fn) } // numactl --physcpubind 3 -- session.test -test.run TestClusteredPrefixColum From 85d8f1613b8d17091b0e5db098d6aa5a9d0624be Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 10 Feb 2022 02:25:19 +0100 Subject: [PATCH 12/29] Fixed some comments and minor changes --- store/mockstore/unistore/cophandler/closure_exec.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index 5ab4687c3ba05..7281ec77c1e6c 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -293,14 +293,18 @@ func (e *closureExecutor) initIdxScanCtx(idxScan *tipb.IndexScan) { e.idxScanCtx.primaryColumnIds = idxScan.PrimaryColumnIds lastColumn := e.columnInfos[len(e.columnInfos)-1] + + // Here it is required that ExtraPhysTblID is last if lastColumn.GetColumnId() == model.ExtraPhysTblID { - lastColumn = e.columnInfos[len(e.columnInfos)-2] e.idxScanCtx.columnLen-- + lastColumn = e.columnInfos[e.idxScanCtx.columnLen-1] } + // Here it is required that ExtraPidColID + // is after all other columns except ExtraPhysTblID if lastColumn.GetColumnId() == model.ExtraPidColID { - lastColumn = e.columnInfos[len(e.columnInfos)-2] e.idxScanCtx.columnLen-- + lastColumn = e.columnInfos[e.idxScanCtx.columnLen-1] } if len(e.idxScanCtx.primaryColumnIds) == 0 { @@ -921,6 +925,7 @@ func (e *closureExecutor) indexScanProcessCore(key, value []byte) error { } } // Add ExtraPhysTblID if requested + // Assumes it is always last! if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID { tblID := tablecodec.DecodeTableID(key) chk.AppendInt64(len(e.columnInfos)-1, tblID) From c1acc5b9abb5ecb221f2902987368d69b043e959 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 14 Feb 2022 20:10:29 +0100 Subject: [PATCH 13/29] Added testcase for issue #31024 --- executor/partition_table_test.go | 43 ++++++++++++++++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 7a3d6b7429c1d..f752a76942cf1 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -3206,3 +3206,46 @@ func (s *partitionTableSuite) TestLeftJoinForUpdate(c *C) { tk1.MustExec("rollback") checkOrder() } + +func (s *partitionTableSuite) TestIssue31024(c *C) { + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("create database TestIssue31024") + defer tk1.MustExec("drop database TestIssue31024") + tk1.MustExec("use TestIssue31024") + tk1.MustExec("create table t1 (c_datetime datetime, c1 int, c2 int, primary key (c_datetime), key(c1), key(c2))" + + " partition by range (to_days(c_datetime)) " + + "( partition p0 values less than (to_days('2020-02-01'))," + + " partition p1 values less than (to_days('2020-04-01'))," + + " partition p2 values less than (to_days('2020-06-01'))," + + " partition p3 values less than maxvalue)") + tk1.MustExec("create table t2 (c_datetime datetime, unique key(c_datetime))") + tk1.MustExec("insert into t1 values ('2020-06-26 03:24:00', 1, 1), ('2020-02-21 07:15:33', 2, 2), ('2020-04-27 13:50:58', 3, 3)") + tk1.MustExec("insert into t2 values ('2020-01-10 09:36:00'), ('2020-02-04 06:00:00'), ('2020-06-12 03:45:18')") + tk1.MustExec("SET GLOBAL tidb_txn_mode = 'pessimistic'") + + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec("use TestIssue31024") + + ch := make(chan int, 10) + tk1.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk1.MustExec("begin pessimistic") + tk1.MustQuery("select /*+ use_index_merge(t1) */ * from t1 join t2 on t1.c_datetime >= t2.c_datetime where t1.c1 < 10 or t1.c2 < 10 for update") + + go func() { + // Check the key is locked. + tk2.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk2.MustExec("begin pessimistic") + tk2.MustExec("update t1 set c_datetime = '2020-06-26 03:24:00' where c1 = 1") + ch <- 2 + }() + + // Give chance for the goroutines to run first. + time.Sleep(80 * time.Millisecond) + ch <- 1 + tk1.MustExec("rollback") + + c.Assert(<-ch, Equals, 1) + c.Assert(<-ch, Equals, 2) + + tk2.MustExec("rollback") +} From aae672a3628fbc15d8e03990f8e9ef4ecc47ad4c Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 14 Feb 2022 22:18:14 +0100 Subject: [PATCH 14/29] Added test case for #27346 --- executor/partition_table_test.go | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index f752a76942cf1..58fafcfe7b194 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -3249,3 +3249,29 @@ func (s *partitionTableSuite) TestIssue31024(c *C) { tk2.MustExec("rollback") } + +func (s *partitionTableSuite) TestIssue27346(c *C) { + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("create database TestIssue27346") + defer tk1.MustExec("drop database TestIssue27346") + tk1.MustExec("use TestIssue27346") + + tk1.MustExec("set @@tidb_enable_index_merge=1,@@tidb_partition_prune_mode='dynamic'") + + tk1.MustExec("DROP TABLE IF EXISTS `tbl_18`") + tk1.MustExec("CREATE TABLE `tbl_18` (`col_119` binary(16) NOT NULL DEFAULT 'skPoKiwYUi',`col_120` int(10) unsigned NOT NULL,`col_121` timestamp NOT NULL,`col_122` double NOT NULL DEFAULT '3937.1887880628115',`col_123` bigint(20) NOT NULL DEFAULT '3550098074891542725',PRIMARY KEY (`col_123`,`col_121`,`col_122`,`col_120`) CLUSTERED,UNIQUE KEY `idx_103` (`col_123`,`col_119`,`col_120`),UNIQUE KEY `idx_104` (`col_122`,`col_120`),UNIQUE KEY `idx_105` (`col_119`,`col_120`),KEY `idx_106` (`col_121`,`col_120`,`col_122`,`col_119`),KEY `idx_107` (`col_121`)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci PARTITION BY HASH( `col_120` ) PARTITIONS 3") + tk1.MustExec("INSERT INTO tbl_18 (`col_119`, `col_120`, `col_121`, `col_122`, `col_123`) VALUES (X'736b506f4b6977595569000000000000', 672436701, '1974-02-24 00:00:00', 3937.1887880628115e0, -7373106839136381229), (X'736b506f4b6977595569000000000000', 2637316689, '1993-10-29 00:00:00', 3937.1887880628115e0, -4522626077860026631), (X'736b506f4b6977595569000000000000', 831809724, '1995-11-20 00:00:00', 3937.1887880628115e0, -4426441253940231780), (X'736b506f4b6977595569000000000000', 1588592628, '2001-03-28 00:00:00', 3937.1887880628115e0, 1329207475772244999), (X'736b506f4b6977595569000000000000', 3908038471, '2031-06-06 00:00:00', 3937.1887880628115e0, -6562815696723135786), (X'736b506f4b6977595569000000000000', 1674237178, '2001-10-24 00:00:00', 3937.1887880628115e0, -6459065549188938772), (X'736b506f4b6977595569000000000000', 3507075493, '2010-03-25 00:00:00', 3937.1887880628115e0, -4329597025765326929), (X'736b506f4b6977595569000000000000', 1276461709, '2019-07-20 00:00:00', 3937.1887880628115e0, 3550098074891542725)") + + tk1.MustQuery("select col_120,col_122,col_123 from tbl_18 where tbl_18.col_122 = 4763.320888074281 and not( tbl_18.col_121 in ( '2032-11-01' , '1975-05-21' , '1994-05-16' , '1984-01-15' ) ) or not( tbl_18.col_121 >= '2008-10-24' ) order by tbl_18.col_119,tbl_18.col_120,tbl_18.col_121,tbl_18.col_122,tbl_18.col_123 limit 919 for update").Sort().Check(testkit.Rows( + "1588592628 3937.1887880628115 1329207475772244999", + "1674237178 3937.1887880628115 -6459065549188938772", + "2637316689 3937.1887880628115 -4522626077860026631", + "672436701 3937.1887880628115 -7373106839136381229", + "831809724 3937.1887880628115 -4426441253940231780")) + tk1.MustQuery("select /*+ use_index_merge( tbl_18 ) */ col_120,col_122,col_123 from tbl_18 where tbl_18.col_122 = 4763.320888074281 and not( tbl_18.col_121 in ( '2032-11-01' , '1975-05-21' , '1994-05-16' , '1984-01-15' ) ) or not( tbl_18.col_121 >= '2008-10-24' ) order by tbl_18.col_119,tbl_18.col_120,tbl_18.col_121,tbl_18.col_122,tbl_18.col_123 limit 919 for update").Sort().Check(testkit.Rows( + "1588592628 3937.1887880628115 1329207475772244999", + "1674237178 3937.1887880628115 -6459065549188938772", + "2637316689 3937.1887880628115 -4522626077860026631", + "672436701 3937.1887880628115 -7373106839136381229", + "831809724 3937.1887880628115 -4426441253940231780")) +} From 254304c138c905df8c6005783c8a16285b98aab2 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 15 Feb 2022 00:20:02 +0100 Subject: [PATCH 15/29] Manual merge fix for tests upgraded to testify --- executor/partition_table_test.go | 35 ++++++++++++++++++++------------ 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 3f2b67fab324f..57f660784bfd4 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -3264,14 +3264,17 @@ func TestIssue26251(t *testing.T) { tk2.MustExec("rollback") } -func (s *partitionTableSuite) TestLeftJoinForUpdate(c *C) { - tk1 := testkit.NewTestKit(c, s.store) +func TestLeftJoinForUpdate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("create database TestLeftJoinForUpdate") defer tk1.MustExec("drop database TestLeftJoinForUpdate") tk1.MustExec("use TestLeftJoinForUpdate") - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use TestLeftJoinForUpdate") - tk3 := testkit.NewTestKit(c, s.store) + tk3 := testkit.NewTestKit(t, store) tk3.MustExec("use TestLeftJoinForUpdate") tk1.MustExec("drop table if exists nt, pt") @@ -3314,10 +3317,10 @@ func (s *partitionTableSuite) TestLeftJoinForUpdate(c *C) { tk1.MustExec("rollback") checkOrder := func() { - c.Assert(<-ch, Equals, 1) + require.Equal(t, <-ch, 1) v1 := <-ch v2 := <-ch - c.Assert((v1 == 2 && v2 == 3) || (v1 == 3 && v2 == 2), IsTrue) + require.True(t, (v1 == 2 && v2 == 3) || (v1 == 3 && v2 == 2)) } checkOrder() @@ -3346,8 +3349,11 @@ func (s *partitionTableSuite) TestLeftJoinForUpdate(c *C) { checkOrder() } -func (s *partitionTableSuite) TestIssue31024(c *C) { - tk1 := testkit.NewTestKit(c, s.store) +func TestIssue31024(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("create database TestIssue31024") defer tk1.MustExec("drop database TestIssue31024") tk1.MustExec("use TestIssue31024") @@ -3362,7 +3368,7 @@ func (s *partitionTableSuite) TestIssue31024(c *C) { tk1.MustExec("insert into t2 values ('2020-01-10 09:36:00'), ('2020-02-04 06:00:00'), ('2020-06-12 03:45:18')") tk1.MustExec("SET GLOBAL tidb_txn_mode = 'pessimistic'") - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use TestIssue31024") ch := make(chan int, 10) @@ -3383,14 +3389,17 @@ func (s *partitionTableSuite) TestIssue31024(c *C) { ch <- 1 tk1.MustExec("rollback") - c.Assert(<-ch, Equals, 1) - c.Assert(<-ch, Equals, 2) + require.Equal(t, <-ch, 1) + require.Equal(t, <-ch, 2) tk2.MustExec("rollback") } -func (s *partitionTableSuite) TestIssue27346(c *C) { - tk1 := testkit.NewTestKit(c, s.store) +func TestIssue27346(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("create database TestIssue27346") defer tk1.MustExec("drop database TestIssue27346") tk1.MustExec("use TestIssue27346") From 756e5fd56ed632284babfa767eb2bd5bb0b85d8e Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 15 Feb 2022 00:34:39 +0100 Subject: [PATCH 16/29] Minor cleanup in SelectLock Next --- executor/executor.go | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index a8c43dc3a315a..0b8368c134fce 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -977,7 +977,6 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { iter := chunk.NewIterator4Chunk(req) for row := iter.Begin(); row != iter.End(); row = iter.Next() { for tblID, cols := range e.tblID2Handle { - touchedTableIDs[tblID] = true logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int64("tableID", tblID)) for _, col := range cols { handle, err := col.BuildHandle(row) @@ -987,10 +986,10 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { physTblID := tblID if physTblColIdx, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { physTblID = row.GetInt64(physTblColIdx) - touchedTableIDs[physTblID] = true logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int("physTblColIdx", physTblColIdx), zap.Int64("tableID from row", physTblID)) } //logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int64("tableID", physTblID), zap.Int64("handle", handle.IntValue())) + touchedTableIDs[physTblID] = true e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physTblID, handle)) } } @@ -1004,13 +1003,6 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { lockWaitTime = int64(e.Lock.WaitSec) * 1000 } - if len(e.tblID2Handle) > 0 { - for id := range e.tblID2Handle { - e.updateDeltaForTableID(id) - delete(touchedTableIDs, id) - } - } - for id := range touchedTableIDs { e.updateDeltaForTableID(id) } From c08ea52bafa7cd133ac2999487bf1aa83f6b348c Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 02:39:03 +0100 Subject: [PATCH 17/29] WIP before cleanup. We still need to send the ExtraPhysTblID column request to the store, otherwise there will be issues with the chunk handling. Like it needs to have space for the column, but the select result from the store cannot fill it in. To be cleaned up. This was the last try to avoid sending and recieve more data from the store for static partition prune mode. But as discussed, this is a minor issue and is acceptable, due to static prune mode is planned to be deprecated and removed. --- executor/builder.go | 32 ++++++++++-------------- executor/executor.go | 27 ++++++++++++-------- executor/partition_table_test.go | 4 +-- executor/table_reader.go | 15 +++++++++-- planner/core/logical_plan_builder.go | 3 ++- planner/core/plan_to_pb.go | 31 ++++++++++++++++++++--- planner/core/planbuilder.go | 29 ++++++++++----------- planner/core/rule_partition_processor.go | 1 + 8 files changed, 91 insertions(+), 51 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index a9e7674dab15e..d42b1d54c3662 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3129,25 +3129,9 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea tbl, _ := b.is.TableByID(ts.Table.ID) isPartition, physicalTableID := ts.IsPartition() - var partitionPhysTblIDOffset int if isPartition { pt := tbl.(table.PartitionedTable) tbl = pt.GetPartition(physicalTableID) - if ts.Table.ID == physicalTableID { - // TODO: Remove after some testing... - panic("isPartition is set but still physicalTableID is set to logical table id!!!") - } - // NOTE for mjonss: TODO, add a comment about Global Indexes, that it is only used for index data, and also stores the Physical Table ID, so never any need to fill it in - // Only time to fill the Physical Table ID is under static pruning mode when ExtraPhysTblID has been requested! - for i, col := range v.Schema().Columns { - if col.ID == model.ExtraPhysTblID { - if i == 0 { - panic("ExtraPhysTblID in offset 0!!!") - } - partitionPhysTblIDOffset = i - break - } - } } startTS, err := b.getSnapshotTS() if err != nil { @@ -3170,8 +3154,6 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea tablePlan: v.GetTablePlan(), storeType: v.StoreType, batchCop: v.BatchCop, - // default 0 - partitionPhysTblIDOffset: partitionPhysTblIDOffset, } e.buildVirtualColumnInfo() if containsLimit(dagReq.Executors) { @@ -3196,7 +3178,19 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea } } - for i := range v.Schema().Columns { + cols := v.Schema().Columns + for i := range cols { + if isPartition && cols[i].ID == model.ExtraPhysTblID { + // Static partition prune mode, still request it from the deqReq output... + if ts.Table.ID == physicalTableID { + // TODO: Remove after some testing... + panic("isPartition is set but still physicalTableID is set to logical table id!!!") + } + if i == 0 { + panic("ExtraPhysTblID in offset 0!!!") + } + e.partitionPhysTblIDOffset = i + } dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(i)) } diff --git a/executor/executor.go b/executor/executor.go index 0b8368c134fce..1f92f1f4df8c3 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -925,24 +925,32 @@ type SelectLockExec struct { tblID2PhysTblIDCol map[int64]*expression.Column // Used during execution + // Map from logic tableID to column index where the physical table id is stored + // For dynamic prune mode, model.ExtraPhysTblID columns are requested from + // storage and used for physical table id + // For static prune mode, model.ExtraPhysTblID is not sent to storage/Protobuf + // but filled in by the partitions TableReaderExecutor tblID2PhysTblIDColIdx map[int64]int } // Open implements the Executor Open interface. func (e *SelectLockExec) Open(ctx context.Context) error { - logutil.Logger(ctx).Info("MJONSS: SelectLocExec::Open()") if len(e.partitionedTable) > 0 { // This should be possible to do by going through the tblID2Handle and then see if the TableById gives a partitioned table or not, and then create the map for static prune? Maybe works for dynamic too? e.tblID2PhysTblIDColIdx = make(map[int64]int) cols := e.Schema().Columns + if cols[0].ID == model.ExtraPhysTblID { + panic("model.ExtraPhysTblID should never be the first/only ID?!?") + } for i := len(cols) - 1; i > 0; i-- { if cols[i].ID == model.ExtraPhysTblID { - logutil.Logger(ctx).Info("MJONSS: SelectLocExec::Open()", zap.Int("i", i), zap.Int64("UniqueID", cols[i].UniqueID)) found := false for tblID, col := range e.tblID2PhysTblIDCol { if cols[i].UniqueID == col.UniqueID { - logutil.Logger(ctx).Info("MJONSS: SelectLocExec::Open() Found in e.tblID2PhysTblIDCol", zap.Int("schema col index", cols[i].Index), zap.Int("PhysTblIDCol index", col.Index)) found = true + if _, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { + panic("Multiple model.ExtraPhysTblID set for the same table!") + } e.tblID2PhysTblIDColIdx[tblID] = i break } @@ -952,9 +960,6 @@ func (e *SelectLockExec) Open(ctx context.Context) error { } } } - if cols[0].ID == model.ExtraPhysTblID { - panic("Should never be the only ID?!?") - } } return e.baseExecutor.Open(ctx) } @@ -968,7 +973,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } // If there's no handle or it's not a `SELECT FOR UPDATE` statement. if len(e.tblID2Handle) == 0 || (!plannercore.IsSelectForUpdateLockType(e.Lock.LockType)) { - logutil.Logger(ctx).Info("MJONSS: SelectLocExec No handle or lock!!!") + panic("MJONSS: SelectLocExec No handle or lock!!!") return nil } @@ -977,18 +982,20 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { iter := chunk.NewIterator4Chunk(req) for row := iter.Begin(); row != iter.End(); row = iter.Next() { for tblID, cols := range e.tblID2Handle { - logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int64("tableID", tblID)) for _, col := range cols { handle, err := col.BuildHandle(row) if err != nil { return err } + // TODO: Change so the ExtraPhysTblID is set for both static and dynamic + // filter it out in ToPB for both index and table reads if 'isPartition' + // Add it in the table reader, if 'isPartition'? + // Then it is just to read both here and in UnionScan, as committed + // So the only change to commit is to filter it out? physTblID := tblID if physTblColIdx, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { physTblID = row.GetInt64(physTblColIdx) - logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int("physTblColIdx", physTblColIdx), zap.Int64("tableID from row", physTblID)) } - //logutil.Logger(ctx).Info("MJONSS: SelectLocExec", zap.Int64("tableID", physTblID), zap.Int64("handle", handle.IntValue())) touchedTableIDs[physTblID] = true e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physTblID, handle)) } diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 57f660784bfd4..f4cd239be158b 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -2984,8 +2984,8 @@ partition p2 values less than (11))`) } partitionModes := []string{ - "'dynamic-only'", - "'static-only'", + "'dynamic'", + "'static'", } testCases := []func(){ optimisticTableReader, diff --git a/executor/table_reader.go b/executor/table_reader.go index 0facc7ed0f4f6..ea8663e97640f 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -238,10 +238,21 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error // SelectLock and transaction buffer needs Physical Table ID for its full key // For partitioned tables under static prune mode (one TableReaderExecutor per partition), - // it needs to be added by TiDB, otherwise it is added by storate (column id ExtraPhysTblID) + // it needs to be added by TiDB here, otherwise it is added by storage (column id ExtraPhysTblID) if e.partitionPhysTblIDOffset != 0 { physicalID := getPhysicalTableID(e.table) - fillExtraPIDColumn(req, e.partitionPhysTblIDOffset, physicalID) + if e.table.Meta().ID != physicalID { + // Static prune mode, one TableReaderExecutor for each partition + // and physical table id is not same as 'logical' table partition id + if e.ctx.GetSessionVars().UseDynamicPartitionPrune() { + panic("table ID != physical table id in TableReaderExecutor in dynamic prune mode!!!") + } + fillExtraPIDColumn(req, e.partitionPhysTblIDOffset, physicalID) + } else { + if !e.ctx.GetSessionVars().UseDynamicPartitionPrune() { + panic("table ID == physical table id in TableReaderExecutor in static prune mode!!!") + } + } } return nil diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index d1aebf22a6db5..fbcf5cb13e0cf 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4189,7 +4189,8 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as us := LogicalUnionScan{handleCols: handleCols}.Init(b.ctx, b.getSelectOffset()) us.SetChildren(ds) if tableInfo.Partition != nil && b.ctx.GetSessionVars().UseDynamicPartitionPrune() { - // table partition prune mode == dynamic (Single TableReader, needs the PhysTblID from storage) + // table partition prune mode == dynamic + // Single TableReader for all partitions, needs the PhysTblID from storage us.ExtraPhysTblIDCol = ds.AddExtraPhysTblIDColumn() } result = us diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index cbd40d0c52b22..b0a7cce476da5 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -177,7 +177,25 @@ func (p *PhysicalLimit) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*t // ToPB implements PhysicalPlan ToPB interface. func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { - tsExec := tables.BuildTableScanFromInfos(p.Table, p.Columns) + columns := p.Columns + if p.isPartition { + // Still send the request, but do not require it to be filled in? + /* + // This is static prune mode, one DataSource/TableReader per partition + // if model.ExtraPhysTblID is included in columns, remove it, since it + // will be added in the TableReader instead! + for i := len(columns) - 1; i >= 0; i-- { + if columns[i].ID == model.ExtraPhysTblID { + columns = append(columns[:i], columns[i+1:]...) + break + } + } + */ + if p.ctx.GetSessionVars().UseDynamicPartitionPrune() { + panic("p.isPartition should only be set in static prune mode!!!") + } + } + tsExec := tables.BuildTableScanFromInfos(p.Table, columns) tsExec.Desc = p.Desc if p.isPartition { tsExec.TableId = p.physicalTableID @@ -197,7 +215,7 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) if storeType == kv.TiFlash { executorID = p.ExplainID().String() } - err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, p.Columns) + err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, columns) return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec, ExecutorId: &executorID}, err } @@ -325,7 +343,14 @@ func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb. if col.ID == model.ExtraHandleID { columns = append(columns, model.NewExtraHandleColInfo()) } else if col.ID == model.ExtraPhysTblID { - columns = append(columns, model.NewExtraPhysTblIDColInfo()) + if !p.isPartition { + if !p.ctx.GetSessionVars().UseDynamicPartitionPrune() { + panic("p.isPartition is not set, but static prune mode is!!!") + } + // dynamic prune mode! + // if there is a model.ExtraPhysTblID column, send it in the protobuf schema! + columns = append(columns, model.NewExtraPhysTblIDColInfo()) + } } else if col.ID == model.ExtraPidColID { columns = append(columns, model.NewExtraPartitionIDColInfo()) } else { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index e050ccbd6008b..93c86d3c67520 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1228,20 +1228,21 @@ func removeTiflashDuringStaleRead(paths []*util.AccessPath) []*util.AccessPath { func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) (*LogicalLock, error) { tblID2PhysTblIDCol := make(map[int64]*expression.Column) if len(b.partitionedTable) > 0 { - if b.ctx.GetSessionVars().UseDynamicPartitionPrune() { - // If a chunk row is read from a partitioned table, which partition the row - // comes from is unknown. With the existence of Join, the situation could be - // even worse: SelectLock have to know the `pid` to construct the lock key. - // To solve the problem, an extra `pid` column is add to the schema, and the - // DataSource need to return the `pid` information in the chunk row. - // table partition prune mode == dynamic (Single TableReader, needs the PhysTblID from storage) - setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) - - } else { - // Do this for static mode as well, but fill it in by the TableReader instead? - // TODO: filter it out when sending to coprocessor and add it in TableReader! - setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) - // (Because one table reader can read from multiple partitions, which partition a chunk row comes from is unknown) + // If a chunk row is read from a partitioned table, which partition the row + // comes from is unknown. With the existence of Join, the situation could be + // even worse: SelectLock have to know the `pid` to construct the lock key. + // To solve the problem, an extra `pid` column is added to the schema, and the + // DataSource need to return the `pid` information in the chunk row. + // For dynamic prune mode, it is filled in from the tableID in the key by storage + // For static prune mode it is still sent to storage, but it does not need to be + // filled in (similar to virtualColumns, which is filled in by TiDB, but still + // their defaults values are filled in by storage + // and set by the partition's TableReaderExecutor physical table id. + // (static prune mode has a union of TableReader, one for each partition). + setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) + + if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + // static partition prune mode, create one TableReader per partition // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. b.optFlag = b.optFlag | flagPartitionProcessor } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index b09553b6db1d5..d10eeca4d6c06 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -43,6 +43,7 @@ import ( const FullRange = -1 // partitionProcessor rewrites the ast for table partition. +// Used by static partition prune mode. // // create table t (id int) partition by range (id) // (partition p1 values less than (10), From 9f219d719c04060faf05580632e96ddb27433122 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 02:54:17 +0100 Subject: [PATCH 18/29] Cleaned up partitionTable for SelectLock and dependents --- executor/builder.go | 1 - executor/executor.go | 7 ++----- planner/core/exhaust_physical_plans.go | 1 - planner/core/logical_plans.go | 6 +++--- planner/core/physical_plans.go | 1 - planner/core/planbuilder.go | 1 - 6 files changed, 5 insertions(+), 12 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index d42b1d54c3662..5150ba239285c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -717,7 +717,6 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), src), Lock: v.Lock, tblID2Handle: v.TblID2Handle, - partitionedTable: v.PartitionedTable, tblID2PhysTblIDCol: v.TblID2PhysTblIDCol, } diff --git a/executor/executor.go b/executor/executor.go index 1f92f1f4df8c3..337d7074b2bbe 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -914,11 +914,8 @@ type SelectLockExec struct { tblID2Handle map[int64][]plannercore.HandleCols - // All the partition tables in the children of this executor. - partitionedTable []table.PartitionedTable - // TODO: Do we even need to use a plannercore.HandleCols struct? - // tblID2PhyTblIDCol is used for partitioned tables in dynamic prune mode, + // tblID2PhyTblIDCol is used for partitioned tables // the child executor need to return an extra column containing // the Physical Table ID (i.e. from which partition the row came from) // Used during building @@ -935,7 +932,7 @@ type SelectLockExec struct { // Open implements the Executor Open interface. func (e *SelectLockExec) Open(ctx context.Context) error { - if len(e.partitionedTable) > 0 { + if len(e.tblID2PhysTblIDCol) > 0 { // This should be possible to do by going through the tblID2Handle and then see if the TableById gives a partitioned table or not, and then create the map for static prune? Maybe works for dynamic too? e.tblID2PhysTblIDColIdx = make(map[int64]int) cols := e.Schema().Columns diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 9ad4cda772397..e4b68c3948129 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2756,7 +2756,6 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P lock := PhysicalLock{ Lock: p.Lock, TblID2Handle: p.tblID2Handle, - PartitionedTable: p.partitionedTable, TblID2PhysTblIDCol: p.tblID2PhysTblIDCol, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) return []PhysicalPlan{lock}, true, nil diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index c64a798cb5e5c..b1bdde43c954f 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -1058,9 +1058,9 @@ type LogicalLimit struct { type LogicalLock struct { baseLogicalPlan - Lock *ast.SelectLockInfo - tblID2Handle map[int64][]HandleCols - partitionedTable []table.PartitionedTable + Lock *ast.SelectLockInfo + tblID2Handle map[int64][]HandleCols + // tblID2phyTblIDCol is used for partitioned tables, // the child executor need to return an extra column containing // the Physical Table ID (i.e. from which partition the row came from) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index d934d188f04eb..ecb6812e322f2 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -960,7 +960,6 @@ type PhysicalLock struct { Lock *ast.SelectLockInfo TblID2Handle map[int64][]HandleCols - PartitionedTable []table.PartitionedTable TblID2PhysTblIDCol map[int64]*expression.Column } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 93c86d3c67520..bb4d87eccf876 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1250,7 +1250,6 @@ func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) selectLock := LogicalLock{ Lock: lock, tblID2Handle: b.handleHelper.tailMap(), - partitionedTable: b.partitionedTable, tblID2PhysTblIDCol: tblID2PhysTblIDCol, }.Init(b.ctx) selectLock.SetChildren(src) From e37f6e773a8cb15b5dd6ff593dc36bd54dbb59e1 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 04:07:16 +0100 Subject: [PATCH 19/29] WIP more tests for avoiding sending requests for phys tbl id to store --- executor/executor.go | 18 ++++++++++++------ executor/table_reader.go | 13 ++++++++++++- planner/core/logical_plan_builder.go | 6 ++++++ planner/core/plan_to_pb.go | 22 ++-------------------- planner/core/planbuilder.go | 8 +++++++- 5 files changed, 39 insertions(+), 28 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 337d7074b2bbe..e2c926f83abfb 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -912,11 +912,16 @@ type SelectLockExec struct { Lock *ast.SelectLockInfo keys []kv.Key + // The children may be a join of multiple tables, so we need a map. tblID2Handle map[int64][]plannercore.HandleCols - // TODO: Do we even need to use a plannercore.HandleCols struct? - // tblID2PhyTblIDCol is used for partitioned tables - // the child executor need to return an extra column containing + // When SelectLock work on a partition table, we need the partition ID + // (Physical Table ID) instead of the 'logical' table ID to calculate + // the lock KV. In that case, the Physical Table ID is extracted + // from the row key in the store and as an extra column in the chunk row. + + // tblID2PhyTblIDCol is used for partitioned tables. + // The child executor need to return an extra column containing // the Physical Table ID (i.e. from which partition the row came from) // Used during building tblID2PhysTblIDCol map[int64]*expression.Column @@ -925,15 +930,16 @@ type SelectLockExec struct { // Map from logic tableID to column index where the physical table id is stored // For dynamic prune mode, model.ExtraPhysTblID columns are requested from // storage and used for physical table id - // For static prune mode, model.ExtraPhysTblID is not sent to storage/Protobuf - // but filled in by the partitions TableReaderExecutor + // For static prune mode, model.ExtraPhysTblID is still sent to storage/Protobuf + // but could be filled in by the partitions TableReaderExecutor + // due to issues with chunk handling between the TableReaderExecutor and the + // SelectReader result. tblID2PhysTblIDColIdx map[int64]int } // Open implements the Executor Open interface. func (e *SelectLockExec) Open(ctx context.Context) error { if len(e.tblID2PhysTblIDCol) > 0 { - // This should be possible to do by going through the tblID2Handle and then see if the TableById gives a partitioned table or not, and then create the map for static prune? Maybe works for dynamic too? e.tblID2PhysTblIDColIdx = make(map[int64]int) cols := e.Schema().Columns if cols[0].ID == model.ExtraPhysTblID { diff --git a/executor/table_reader.go b/executor/table_reader.go index ea8663e97640f..f57a8d513272c 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -247,7 +247,18 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error if e.ctx.GetSessionVars().UseDynamicPartitionPrune() { panic("table ID != physical table id in TableReaderExecutor in dynamic prune mode!!!") } - fillExtraPIDColumn(req, e.partitionPhysTblIDOffset, physicalID) + // After some tries it is very hard to not include a column to the be + // sent to the store but only filled in here. + // See how VirtalColumnValues is done, their defaults are still set in + // the store, and then updated here. + // For the Physical table id, it is as easy to fill it in, in the store. + // Only real cost is the extra bytes send over the network. + if req.NumRows() > 0 { + if req.Column(e.partitionPhysTblIDOffset).GetInt64(0) == 0 { + fillExtraPIDColumn(req, e.partitionPhysTblIDOffset, physicalID) + //panic("NOT Already filled in by engine?") + } + } } else { if !e.ctx.GetSessionVars().UseDynamicPartitionPrune() { panic("table ID == physical table id in TableReaderExecutor in static prune mode!!!") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index fbcf5cb13e0cf..5ec8d05e256c4 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3788,6 +3788,9 @@ func (ds *DataSource) newExtraHandleSchemaCol() *expression.Column { func (ds *DataSource) AddExtraPhysTblIDColumn() *expression.Column { // Avoid adding multiple times // TODO: Can sd.Columns be empty? + if len(ds.TblCols) == 0 { + panic("data source TblCols has zero length!?!") + } cols := ds.TblCols for i := len(cols) - 1; i >= 0; i-- { if cols[i].ID == model.ExtraPhysTblID { @@ -4183,6 +4186,9 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as } } + if len(ds.TblCols) == 0 { + panic("data source TblCols has zero length!?!") + } var result LogicalPlan = ds dirty := tableHasDirtyContent(b.ctx, tableInfo) if dirty || tableInfo.TempTableType == model.TempTableLocal { diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index b0a7cce476da5..5e6cc7393355b 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -177,25 +177,7 @@ func (p *PhysicalLimit) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*t // ToPB implements PhysicalPlan ToPB interface. func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { - columns := p.Columns - if p.isPartition { - // Still send the request, but do not require it to be filled in? - /* - // This is static prune mode, one DataSource/TableReader per partition - // if model.ExtraPhysTblID is included in columns, remove it, since it - // will be added in the TableReader instead! - for i := len(columns) - 1; i >= 0; i-- { - if columns[i].ID == model.ExtraPhysTblID { - columns = append(columns[:i], columns[i+1:]...) - break - } - } - */ - if p.ctx.GetSessionVars().UseDynamicPartitionPrune() { - panic("p.isPartition should only be set in static prune mode!!!") - } - } - tsExec := tables.BuildTableScanFromInfos(p.Table, columns) + tsExec := tables.BuildTableScanFromInfos(p.Table, p.Columns) tsExec.Desc = p.Desc if p.isPartition { tsExec.TableId = p.physicalTableID @@ -215,7 +197,7 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) if storeType == kv.TiFlash { executorID = p.ExplainID().String() } - err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, columns) + err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, p.Columns) return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec, ExecutorId: &executorID}, err } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index bb4d87eccf876..cbee05b4d47cb 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1239,12 +1239,18 @@ func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) // their defaults values are filled in by storage // and set by the partition's TableReaderExecutor physical table id. // (static prune mode has a union of TableReader, one for each partition). - setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) + //setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { // static partition prune mode, create one TableReader per partition // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. b.optFlag = b.optFlag | flagPartitionProcessor + // Skip sending the extraPhysTblIDCol to store, just fill it in anyway in + // table reader... + // WAS HERE, is it possible to avoid this call??!? + setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) + } else { + setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) } } selectLock := LogicalLock{ From 9d9dfab76cc813711c03ff87279a0dcc703ec0c2 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 04:29:10 +0100 Subject: [PATCH 20/29] WIP, removed investigation panic --- executor/executor.go | 1 - 1 file changed, 1 deletion(-) diff --git a/executor/executor.go b/executor/executor.go index e2c926f83abfb..e2ee1b0a3609d 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -976,7 +976,6 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } // If there's no handle or it's not a `SELECT FOR UPDATE` statement. if len(e.tblID2Handle) == 0 || (!plannercore.IsSelectForUpdateLockType(e.Lock.LockType)) { - panic("MJONSS: SelectLocExec No handle or lock!!!") return nil } From 495fe45a9036ffa92ca80225f95fe9d5c655d497 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 04:47:03 +0100 Subject: [PATCH 21/29] Fixed test TestPartitionPruningInTransaction for dynamic pruning --- executor/partition_table_test.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index f4cd239be158b..3d09e76ba4e55 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1829,13 +1829,20 @@ func TestPartitionPruningInTransaction(t *testing.T) { tk.MustExec("create database test_pruning_transaction") defer tk.MustExec(`drop database test_pruning_transaction`) tk.MustExec("use test_pruning_transaction") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec(`create table t(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11))`) + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") tk.MustExec(`begin`) tk.MustPartitionByList(`select * from t`, []string{"p0", "p1", "p2"}) tk.MustPartitionByList(`select * from t where a > 3`, []string{"p1", "p2"}) // partition pruning can work in transactions tk.MustPartitionByList(`select * from t where a > 7`, []string{"p2"}) tk.MustExec(`rollback`) + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec(`begin`) + tk.MustPartition(`select * from t`, "all") + tk.MustPartition(`select * from t where a > 3`, "p1,p2") // partition pruning can work in transactions + tk.MustPartition(`select * from t where a > 7`, "p2") + tk.MustExec(`rollback`) + tk.MustExec("set @@tidb_partition_prune_mode = default") } func TestIssue25253(t *testing.T) { From 2c926b1b05f1b63b841d88e12e39758b6074e336 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 09:58:08 +0100 Subject: [PATCH 22/29] Reverted change for updating table delta, which broke checks See test case. --- executor/executor.go | 4 +--- session/session_test.go | 30 ++++++++++++++++++++++++++++++ 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index e2ee1b0a3609d..f272b7dcade6e 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -979,7 +979,6 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { return nil } - touchedTableIDs := make(map[int64]bool) if req.NumRows() > 0 { iter := chunk.NewIterator4Chunk(req) for row := iter.Begin(); row != iter.End(); row = iter.Next() { @@ -998,7 +997,6 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { if physTblColIdx, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { physTblID = row.GetInt64(physTblColIdx) } - touchedTableIDs[physTblID] = true e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physTblID, handle)) } } @@ -1012,7 +1010,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { lockWaitTime = int64(e.Lock.WaitSec) * 1000 } - for id := range touchedTableIDs { + for id := range e.tblID2Handle { e.updateDeltaForTableID(id) } diff --git a/session/session_test.go b/session/session_test.go index 6ea565afd56d9..f2c5e8d9cfa9d 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2299,6 +2299,36 @@ func (s *testSchemaSerialSuite) TestSchemaCheckerSQL(c *C) { tk.MustQuery(`select * from t for update`) _, err = tk.Exec(`commit;`) c.Assert(err, NotNil) + + // Repeated tests for partitioned table + tk.MustExec(`create table pt (id int, c int) partition by hash (id) partitions 3`) + tk.MustExec(`insert into pt values(1, 1);`) + // The schema version is out of date in the first transaction, and the SQL can't be retried. + tk.MustExec(`begin;`) + tk1.MustExec(`alter table pt modify column c bigint;`) + tk.MustExec(`insert into pt values(3, 3);`) + _, err = tk.Exec(`commit;`) + c.Assert(terror.ErrorEqual(err, domain.ErrInfoSchemaChanged), IsTrue, Commentf("err %v", err)) + + // But the transaction related table IDs aren't in the updated table IDs. + tk.MustExec(`begin;`) + tk1.MustExec(`alter table pt add index idx2(c);`) + tk.MustExec(`insert into t1 values(4, 4);`) + tk.MustExec(`commit;`) + + // Test for "select for update". + tk.MustExec(`begin;`) + tk1.MustExec(`alter table pt add index idx3(c);`) + tk.MustQuery(`select * from pt for update`) + _, err = tk.Exec(`commit;`) + c.Assert(err, NotNil) + + // Test for "select for update". + tk.MustExec(`begin;`) + tk1.MustExec(`alter table pt add index idx4(c);`) + tk.MustQuery(`select * from pt partition (p1) for update`) + _, err = tk.Exec(`commit;`) + c.Assert(err, NotNil) } func (s *testSchemaSerialSuite) TestSchemaCheckerTempTable(c *C) { From 16445ad5c8f7908df61148419dec213642ee4f21 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 11:20:31 +0100 Subject: [PATCH 23/29] Removed some debug only panic checks --- executor/builder.go | 8 -------- executor/executor.go | 11 ----------- executor/table_reader.go | 8 +------- 3 files changed, 1 insertion(+), 26 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 5150ba239285c..9dba24410bfd7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3180,14 +3180,6 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea cols := v.Schema().Columns for i := range cols { if isPartition && cols[i].ID == model.ExtraPhysTblID { - // Static partition prune mode, still request it from the deqReq output... - if ts.Table.ID == physicalTableID { - // TODO: Remove after some testing... - panic("isPartition is set but still physicalTableID is set to logical table id!!!") - } - if i == 0 { - panic("ExtraPhysTblID in offset 0!!!") - } e.partitionPhysTblIDOffset = i } dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(i)) diff --git a/executor/executor.go b/executor/executor.go index f272b7dcade6e..c7323b3696c64 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -942,25 +942,14 @@ func (e *SelectLockExec) Open(ctx context.Context) error { if len(e.tblID2PhysTblIDCol) > 0 { e.tblID2PhysTblIDColIdx = make(map[int64]int) cols := e.Schema().Columns - if cols[0].ID == model.ExtraPhysTblID { - panic("model.ExtraPhysTblID should never be the first/only ID?!?") - } for i := len(cols) - 1; i > 0; i-- { if cols[i].ID == model.ExtraPhysTblID { - found := false for tblID, col := range e.tblID2PhysTblIDCol { if cols[i].UniqueID == col.UniqueID { - found = true - if _, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { - panic("Multiple model.ExtraPhysTblID set for the same table!") - } e.tblID2PhysTblIDColIdx[tblID] = i break } } - if !found { - panic("PhysTblIDCol not find in map?!?") - } } } } diff --git a/executor/table_reader.go b/executor/table_reader.go index f57a8d513272c..98ca5cb45cb34 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -244,9 +244,7 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error if e.table.Meta().ID != physicalID { // Static prune mode, one TableReaderExecutor for each partition // and physical table id is not same as 'logical' table partition id - if e.ctx.GetSessionVars().UseDynamicPartitionPrune() { - panic("table ID != physical table id in TableReaderExecutor in dynamic prune mode!!!") - } + // After some tries it is very hard to not include a column to the be // sent to the store but only filled in here. // See how VirtalColumnValues is done, their defaults are still set in @@ -259,10 +257,6 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error //panic("NOT Already filled in by engine?") } } - } else { - if !e.ctx.GetSessionVars().UseDynamicPartitionPrune() { - panic("table ID == physical table id in TableReaderExecutor in static prune mode!!!") - } } } From 1897ed5f208e234a00355cb9c7511321d399cbbc Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 12:54:13 +0100 Subject: [PATCH 24/29] cleanups --- executor/table_reader.go | 2 +- executor/union_scan.go | 3 +++ planner/core/logical_plan_builder.go | 12 +++-------- planner/core/planbuilder.go | 30 ++++++++++++++++------------ 4 files changed, 24 insertions(+), 23 deletions(-) diff --git a/executor/table_reader.go b/executor/table_reader.go index 98ca5cb45cb34..f5bbb12027f85 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -254,7 +254,7 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error if req.NumRows() > 0 { if req.Column(e.partitionPhysTblIDOffset).GetInt64(0) == 0 { fillExtraPIDColumn(req, e.partitionPhysTblIDOffset, physicalID) - //panic("NOT Already filled in by engine?") + panic("NOT Already filled in by engine?") } } } diff --git a/executor/union_scan.go b/executor/union_scan.go index 5d40c43350d96..f3ee051048686 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -245,6 +245,9 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err var checkKey kv.Key if us.physTblIDIdx != nil { tblID := row.GetInt64(*us.physTblIDIdx) + if tblID == 0 { + panic("UnionScanExec Physical Table ID == 0") + } checkKey = tablecodec.EncodeRowKeyWithHandle(tblID, snapshotHandle) } else { checkKey = tablecodec.EncodeRecordKey(us.table.RecordPrefix(), snapshotHandle) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 5ec8d05e256c4..b435d534a97bb 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3786,11 +3786,7 @@ func (ds *DataSource) newExtraHandleSchemaCol() *expression.Column { // to construct the lock key, so this column is added to the chunk row. // Also needed for checking against the sessions transaction buffer func (ds *DataSource) AddExtraPhysTblIDColumn() *expression.Column { - // Avoid adding multiple times - // TODO: Can sd.Columns be empty? - if len(ds.TblCols) == 0 { - panic("data source TblCols has zero length!?!") - } + // Avoid adding multiple times (should never happen!) cols := ds.TblCols for i := len(cols) - 1; i >= 0; i-- { if cols[i].ID == model.ExtraPhysTblID { @@ -4186,15 +4182,13 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as } } - if len(ds.TblCols) == 0 { - panic("data source TblCols has zero length!?!") - } var result LogicalPlan = ds dirty := tableHasDirtyContent(b.ctx, tableInfo) if dirty || tableInfo.TempTableType == model.TempTableLocal { us := LogicalUnionScan{handleCols: handleCols}.Init(b.ctx, b.getSelectOffset()) us.SetChildren(ds) if tableInfo.Partition != nil && b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + // Adding ExtraPhysTblIDCol for UnionScan (transaction buffer handling) // table partition prune mode == dynamic // Single TableReader for all partitions, needs the PhysTblID from storage us.ExtraPhysTblIDCol = ds.AddExtraPhysTblIDColumn() @@ -4202,7 +4196,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as result = us } - // Adding ExtraPhysTblIDCol for SelectLock (SELECT FOR UPDATE) and UnionScan (transaction buffer handling) are done when building SelectLock and UnionScan + // Adding ExtraPhysTblIDCol for SelectLock (SELECT FOR UPDATE) is done when building SelectLock if sessionVars.StmtCtx.TblInfo2UnionScan == nil { sessionVars.StmtCtx.TblInfo2UnionScan = make(map[*model.TableInfo]bool) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index cbee05b4d47cb..3f2c67594023b 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1239,19 +1239,23 @@ func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) // their defaults values are filled in by storage // and set by the partition's TableReaderExecutor physical table id. // (static prune mode has a union of TableReader, one for each partition). - //setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) - - if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { - // static partition prune mode, create one TableReader per partition - // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. - b.optFlag = b.optFlag | flagPartitionProcessor - // Skip sending the extraPhysTblIDCol to store, just fill it in anyway in - // table reader... - // WAS HERE, is it possible to avoid this call??!? - setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) - } else { - setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) - } + setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) + /* + if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + // static partition prune mode, create one TableReader per partition + // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. + + // Already set when creating DataSource in buildDataSource! + //b.optFlag = b.optFlag | flagPartitionProcessor + + // Skip sending the extraPhysTblIDCol to store, just fill it in anyway in + // table reader... + // WAS HERE, is it possible to avoid this call??!? + // How is it done for UnionScan? + // If avoided there, maybe we could just set tblID2PhysTblIDCol[pid] to len(ds.TblCols)? + //setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) + } + */ } selectLock := LogicalLock{ Lock: lock, From dbdcb36c08ef21e0b73444f6ddcf8ddc45fe6681 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 12:54:23 +0100 Subject: [PATCH 25/29] more cleanups --- executor/builder.go | 3 --- executor/table_reader.go | 38 -------------------------------------- 2 files changed, 41 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 9dba24410bfd7..1603d86ff6a74 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3179,9 +3179,6 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea cols := v.Schema().Columns for i := range cols { - if isPartition && cols[i].ID == model.ExtraPhysTblID { - e.partitionPhysTblIDOffset = i - } dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(i)) } diff --git a/executor/table_reader.go b/executor/table_reader.go index f5bbb12027f85..3057a3dfd1115 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" @@ -109,10 +108,6 @@ type TableReaderExecutor struct { // batchCop indicates whether use super batch coprocessor request, only works for TiFlash engine. batchCop bool - // partitionPhysTblIDOffset if Physical Table ID is not filled in by storage, - // add it to column on this offset (!= 0) Used for partitioned table with static pruning - partitionPhysTblIDOffset int - // If dummy flag is set, this is not a real TableReader, it just provides the KV ranges for UnionScan. // Used by the temporary table, cached table. dummy bool @@ -236,42 +231,9 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error return err } - // SelectLock and transaction buffer needs Physical Table ID for its full key - // For partitioned tables under static prune mode (one TableReaderExecutor per partition), - // it needs to be added by TiDB here, otherwise it is added by storage (column id ExtraPhysTblID) - if e.partitionPhysTblIDOffset != 0 { - physicalID := getPhysicalTableID(e.table) - if e.table.Meta().ID != physicalID { - // Static prune mode, one TableReaderExecutor for each partition - // and physical table id is not same as 'logical' table partition id - - // After some tries it is very hard to not include a column to the be - // sent to the store but only filled in here. - // See how VirtalColumnValues is done, their defaults are still set in - // the store, and then updated here. - // For the Physical table id, it is as easy to fill it in, in the store. - // Only real cost is the extra bytes send over the network. - if req.NumRows() > 0 { - if req.Column(e.partitionPhysTblIDOffset).GetInt64(0) == 0 { - fillExtraPIDColumn(req, e.partitionPhysTblIDOffset, physicalID) - panic("NOT Already filled in by engine?") - } - } - } - } - return nil } -func fillExtraPIDColumn(req *chunk.Chunk, extraPIDColumnIndex int, physicalID int64) { - numRows := req.NumRows() - pidColumn := chunk.NewColumn(types.NewFieldType(mysql.TypeLonglong), numRows) - for i := 0; i < numRows; i++ { - pidColumn.AppendInt64(physicalID) - } - req.SetCol(extraPIDColumnIndex, pidColumn) -} - // Close implements the Executor Close interface. func (e *TableReaderExecutor) Close() error { if e.dummy { From 95fb6a4592c9466f5f40e64f92e2948e358ee2eb Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 21:15:33 +0100 Subject: [PATCH 26/29] Further cleanups --- executor/builder.go | 1 - executor/executor.go | 5 ----- executor/union_scan.go | 7 +++---- planner/core/logical_plan_builder.go | 2 +- planner/core/logical_plans.go | 3 --- planner/core/plan_to_pb.go | 2 ++ planner/core/planbuilder.go | 25 +++---------------------- 7 files changed, 9 insertions(+), 36 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 1603d86ff6a74..73f93a6b2816f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3255,7 +3255,6 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E tmp, _ := b.is.TableByID(ts.Table.ID) tbl := tmp.(table.PartitionedTable) - // TODO: Should this be done before checking dirty / ongoing transactions for avoiding adding PID columns? partitions, err := partitionPruning(b.ctx, tbl, v.PartitionInfo.PruningConds, v.PartitionInfo.PartitionNames, v.PartitionInfo.Columns, v.PartitionInfo.ColumnNames) if err != nil { b.err = err diff --git a/executor/executor.go b/executor/executor.go index c7323b3696c64..fc899f1bbaf54 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -977,11 +977,6 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } - // TODO: Change so the ExtraPhysTblID is set for both static and dynamic - // filter it out in ToPB for both index and table reads if 'isPartition' - // Add it in the table reader, if 'isPartition'? - // Then it is just to read both here and in UnionScan, as committed - // So the only change to commit is to filter it out? physTblID := tblID if physTblColIdx, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { physTblID = row.GetInt64(physTblColIdx) diff --git a/executor/union_scan.go b/executor/union_scan.go index f3ee051048686..a43e7f6dd5005 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -63,6 +63,7 @@ type UnionScanExec struct { collators []collate.Collator // If partitioned table and the physical table id is encoded in the chuck at this column index + // used with dynamic prune mode physTblIDIdx *int } @@ -90,12 +91,13 @@ func (us *UnionScanExec) open(ctx context.Context) error { return err } - for i := range us.columns { + for i := len(us.columns) - 1; i >= 0; i-- { if us.columns[i].ID == model.ExtraPhysTblID { if us.physTblIDIdx == nil { us.physTblIDIdx = new(int) } *us.physTblIDIdx = i + break } } mb := txn.GetMemBuffer() @@ -245,9 +247,6 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err var checkKey kv.Key if us.physTblIDIdx != nil { tblID := row.GetInt64(*us.physTblIDIdx) - if tblID == 0 { - panic("UnionScanExec Physical Table ID == 0") - } checkKey = tablecodec.EncodeRowKeyWithHandle(tblID, snapshotHandle) } else { checkKey = tablecodec.EncodeRecordKey(us.table.RecordPrefix(), snapshotHandle) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index b435d534a97bb..beecad8857389 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4191,7 +4191,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as // Adding ExtraPhysTblIDCol for UnionScan (transaction buffer handling) // table partition prune mode == dynamic // Single TableReader for all partitions, needs the PhysTblID from storage - us.ExtraPhysTblIDCol = ds.AddExtraPhysTblIDColumn() + _ = ds.AddExtraPhysTblIDColumn() } result = us } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index b1bdde43c954f..f1fde878f6222 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -536,9 +536,6 @@ type LogicalUnionScan struct { // cacheTable not nil means it's reading from cached table. cacheTable kv.MemBuffer - - // If partitioned table, this column is used for the Physical Table ID - ExtraPhysTblIDCol *expression.Column } // DataSource represents a tableScan without condition push down. diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 5e6cc7393355b..dd4d57ec719c3 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -332,6 +332,8 @@ func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb. // dynamic prune mode! // if there is a model.ExtraPhysTblID column, send it in the protobuf schema! columns = append(columns, model.NewExtraPhysTblIDColInfo()) + } else { + panic("p.isPartition is set, should we not forward the request for ExtraPhysTblID?!!") } } else if col.ID == model.ExtraPidColID { columns = append(columns, model.NewExtraPartitionIDColInfo()) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 3f2c67594023b..bce9fc4932927 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1233,29 +1233,10 @@ func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) // even worse: SelectLock have to know the `pid` to construct the lock key. // To solve the problem, an extra `pid` column is added to the schema, and the // DataSource need to return the `pid` information in the chunk row. - // For dynamic prune mode, it is filled in from the tableID in the key by storage - // For static prune mode it is still sent to storage, but it does not need to be - // filled in (similar to virtualColumns, which is filled in by TiDB, but still - // their defaults values are filled in by storage - // and set by the partition's TableReaderExecutor physical table id. - // (static prune mode has a union of TableReader, one for each partition). + // For dynamic prune mode, it is filled in from the tableID in the key by storage. + // For static prune mode it is also filled in from the tableID in the key by storage. + // since it would otherwise be lost in the PartitionUnion executor. setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) - /* - if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { - // static partition prune mode, create one TableReader per partition - // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. - - // Already set when creating DataSource in buildDataSource! - //b.optFlag = b.optFlag | flagPartitionProcessor - - // Skip sending the extraPhysTblIDCol to store, just fill it in anyway in - // table reader... - // WAS HERE, is it possible to avoid this call??!? - // How is it done for UnionScan? - // If avoided there, maybe we could just set tblID2PhysTblIDCol[pid] to len(ds.TblCols)? - //setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) - } - */ } selectLock := LogicalLock{ Lock: lock, From 9df6392539c75138631fcd97d4ba9b01693cdfbe Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 16 Feb 2022 21:52:09 +0100 Subject: [PATCH 27/29] last cleanups --- executor/builder.go | 3 +-- planner/core/plan_to_pb.go | 11 +---------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 73f93a6b2816f..956fa942c0a5a 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3177,8 +3177,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea } } - cols := v.Schema().Columns - for i := range cols { + for i := range v.Schema().Columns { dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(i)) } diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index dd4d57ec719c3..cbd40d0c52b22 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -325,16 +325,7 @@ func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb. if col.ID == model.ExtraHandleID { columns = append(columns, model.NewExtraHandleColInfo()) } else if col.ID == model.ExtraPhysTblID { - if !p.isPartition { - if !p.ctx.GetSessionVars().UseDynamicPartitionPrune() { - panic("p.isPartition is not set, but static prune mode is!!!") - } - // dynamic prune mode! - // if there is a model.ExtraPhysTblID column, send it in the protobuf schema! - columns = append(columns, model.NewExtraPhysTblIDColInfo()) - } else { - panic("p.isPartition is set, should we not forward the request for ExtraPhysTblID?!!") - } + columns = append(columns, model.NewExtraPhysTblIDColInfo()) } else if col.ID == model.ExtraPidColID { columns = append(columns, model.NewExtraPartitionIDColInfo()) } else { From a2daabbad1b38730cb8169b5afcf637d6c4cf9ea Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 21 Feb 2022 00:21:37 +0100 Subject: [PATCH 28/29] After merge with master, also added PhysTblID handling in unistore/mpp --- store/mockstore/unistore/cophandler/mpp.go | 42 ++++++++++++------- .../mockstore/unistore/cophandler/mpp_exec.go | 14 +++++++ 2 files changed, 41 insertions(+), 15 deletions(-) diff --git a/store/mockstore/unistore/cophandler/mpp.go b/store/mockstore/unistore/cophandler/mpp.go index 59a66d31f6f88..cc8954e025ce9 100644 --- a/store/mockstore/unistore/cophandler/mpp.go +++ b/store/mockstore/unistore/cophandler/mpp.go @@ -78,7 +78,11 @@ func (b *mppExecBuilder) buildMPPTableScan(pb *tipb.TableScan) (*tableScanExec, ts.lockStore = b.dagCtx.lockStore ts.resolvedLocks = b.dagCtx.resolvedLocks } - for _, col := range pb.Columns { + for i, col := range pb.Columns { + if col.ColumnId == model.ExtraPhysTblID { + ts.physTblIDColIdx = new(int) + *ts.physTblIDColIdx = i + } ft := fieldTypeFromPBColumn(col) ts.fieldTypes = append(ts.fieldTypes, ft) } @@ -98,9 +102,16 @@ func (b *mppExecBuilder) buildIdxScan(pb *tipb.IndexScan) (*indexScanExec, error primaryColIds := pb.GetPrimaryColumnIds() lastCol := pb.Columns[numCols-1] + var physTblIDColIdx *int + if lastCol.GetColumnId() == model.ExtraPhysTblID { + numIdxCols-- + physTblIDColIdx = new(int) + *physTblIDColIdx = numIdxCols + lastCol = pb.Columns[numIdxCols-1] + } if lastCol.GetColumnId() == model.ExtraPidColID { - lastCol = pb.Columns[numCols-2] numIdxCols-- + lastCol = pb.Columns[numIdxCols-1] } hdlStatus := tablecodec.HandleDefault @@ -132,19 +143,20 @@ func (b *mppExecBuilder) buildIdxScan(pb *tipb.IndexScan) (*indexScanExec, error prevVals = make([][]byte, numIdxCols) } idxScan := &indexScanExec{ - baseMPPExec: baseMPPExec{sc: b.sc, fieldTypes: fieldTypes}, - startTS: b.dagCtx.startTS, - kvRanges: ranges, - dbReader: b.dbReader, - lockStore: b.dagCtx.lockStore, - resolvedLocks: b.dagCtx.resolvedLocks, - counts: b.counts, - ndvs: b.ndvs, - prevVals: prevVals, - colInfos: colInfos, - numIdxCols: numIdxCols, - hdlStatus: hdlStatus, - desc: pb.Desc, + baseMPPExec: baseMPPExec{sc: b.sc, fieldTypes: fieldTypes}, + startTS: b.dagCtx.startTS, + kvRanges: ranges, + dbReader: b.dbReader, + lockStore: b.dagCtx.lockStore, + resolvedLocks: b.dagCtx.resolvedLocks, + counts: b.counts, + ndvs: b.ndvs, + prevVals: prevVals, + colInfos: colInfos, + numIdxCols: numIdxCols, + hdlStatus: hdlStatus, + desc: pb.Desc, + physTblIDColIdx: physTblIDColIdx, } return idxScan, nil } diff --git a/store/mockstore/unistore/cophandler/mpp_exec.go b/store/mockstore/unistore/cophandler/mpp_exec.go index 9752371d50c02..85a5983517e11 100644 --- a/store/mockstore/unistore/cophandler/mpp_exec.go +++ b/store/mockstore/unistore/cophandler/mpp_exec.go @@ -123,6 +123,9 @@ type tableScanExec struct { decoder *rowcodec.ChunkDecoder desc bool + + // if ExtraPhysTblIDCol is requested, fill in the physical table id in this column position + physTblIDColIdx *int } func (e *tableScanExec) SkipValue() bool { return false } @@ -137,6 +140,10 @@ func (e *tableScanExec) Process(key, value []byte) error { if err != nil { return errors.Trace(err) } + if e.physTblIDColIdx != nil { + tblID := tablecodec.DecodeTableID(key) + e.chk.AppendInt64(*e.physTblIDColIdx, tblID) + } e.rowCnt++ if e.chk.IsFull() { @@ -241,6 +248,9 @@ type indexScanExec struct { colInfos []rowcodec.ColInfo numIdxCols int hdlStatus tablecodec.HandleStatus + + // if ExtraPhysTblIDCol is requested, fill in the physical table id in this column position + physTblIDColIdx *int } func (e *indexScanExec) SkipValue() bool { return false } @@ -275,6 +285,10 @@ func (e *indexScanExec) Process(key, value []byte) error { } } } + if e.physTblIDColIdx != nil { + tblID := tablecodec.DecodeTableID(key) + e.chk.AppendInt64(*e.physTblIDColIdx, tblID) + } if e.chk.IsFull() { e.chunks = append(e.chunks, e.chk) e.chk = chunk.NewChunkWithCapacity(e.fieldTypes, DefaultBatchSize) From e459cf74901eeb69e2a5f37b0162fd8948e49190 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 21 Feb 2022 15:10:20 +0100 Subject: [PATCH 29/29] Addressing review comments --- executor/executor.go | 2 +- executor/union_scan.go | 13 ++++++------- planner/core/logical_plan_builder.go | 4 ++-- planner/core/planbuilder.go | 3 ++- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 03c9e8b62a718..87d556defe993 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -948,7 +948,7 @@ func (e *SelectLockExec) Open(ctx context.Context) error { if len(e.tblID2PhysTblIDCol) > 0 { e.tblID2PhysTblIDColIdx = make(map[int64]int) cols := e.Schema().Columns - for i := len(cols) - 1; i > 0; i-- { + for i := len(cols) - 1; i >= 0; i-- { if cols[i].ID == model.ExtraPhysTblID { for tblID, col := range e.tblID2PhysTblIDCol { if cols[i].UniqueID == col.UniqueID { diff --git a/executor/union_scan.go b/executor/union_scan.go index a43e7f6dd5005..09937d404010a 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -64,7 +64,8 @@ type UnionScanExec struct { // If partitioned table and the physical table id is encoded in the chuck at this column index // used with dynamic prune mode - physTblIDIdx *int + // < 0 if not used. + physTblIDIdx int } // Open implements the Executor Open interface. @@ -91,12 +92,10 @@ func (us *UnionScanExec) open(ctx context.Context) error { return err } + us.physTblIDIdx = -1 for i := len(us.columns) - 1; i >= 0; i-- { if us.columns[i].ID == model.ExtraPhysTblID { - if us.physTblIDIdx == nil { - us.physTblIDIdx = new(int) - } - *us.physTblIDIdx = i + us.physTblIDIdx = i break } } @@ -245,8 +244,8 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err return nil, err } var checkKey kv.Key - if us.physTblIDIdx != nil { - tblID := row.GetInt64(*us.physTblIDIdx) + if us.physTblIDIdx >= 0 { + tblID := row.GetInt64(us.physTblIDIdx) checkKey = tablecodec.EncodeRowKeyWithHandle(tblID, snapshotHandle) } else { checkKey = tablecodec.EncodeRecordKey(us.table.RecordPrefix(), snapshotHandle) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 79017d644859c..50ade9876784d 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4216,9 +4216,9 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if dirty || tableInfo.TempTableType == model.TempTableLocal { us := LogicalUnionScan{handleCols: handleCols}.Init(b.ctx, b.getSelectOffset()) us.SetChildren(ds) - if tableInfo.Partition != nil && b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + if tableInfo.Partition != nil && b.optFlag&flagPartitionProcessor == 0 { // Adding ExtraPhysTblIDCol for UnionScan (transaction buffer handling) - // table partition prune mode == dynamic + // Not using old static prune mode // Single TableReader for all partitions, needs the PhysTblID from storage _ = ds.AddExtraPhysTblIDColumn() } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index a3fdcbfc7dabc..6a2ceb71aa85b 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1226,8 +1226,9 @@ func removeTiflashDuringStaleRead(paths []*util.AccessPath) []*util.AccessPath { } func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) (*LogicalLock, error) { - tblID2PhysTblIDCol := make(map[int64]*expression.Column) + var tblID2PhysTblIDCol map[int64]*expression.Column if len(b.partitionedTable) > 0 { + tblID2PhysTblIDCol = make(map[int64]*expression.Column) // If a chunk row is read from a partitioned table, which partition the row // comes from is unknown. With the existence of Join, the situation could be // even worse: SelectLock have to know the `pid` to construct the lock key.