Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

executor: make tablesample work under different partition prune modes (#52405) #52546

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion pkg/executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -5144,7 +5144,7 @@ func (b *executorBuilder) buildTableSample(v *plannercore.PhysicalTableSample) *
e.sampler = &emptySampler{}
} else if v.TableSampleInfo.AstNode.SampleMethod == ast.SampleMethodTypeTiDBRegion {
e.sampler = newTableRegionSampler(
b.ctx, v.TableInfo, startTS, v.TableSampleInfo.Partitions, v.Schema(),
b.ctx, v.TableInfo, startTS, v.PhysicalTableID, v.TableSampleInfo.Partitions, v.Schema(),
v.TableSampleInfo.FullSchema, e.RetFieldTypes(), v.Desc)
}

Expand Down
80 changes: 50 additions & 30 deletions pkg/executor/sample.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,10 +75,12 @@ type rowSampler interface {
}

type tableRegionSampler struct {
ctx sessionctx.Context
table table.Table
startTS uint64
partTables []table.PartitionedTable
ctx sessionctx.Context
table table.Table
startTS uint64
physicalTableID int64
partTables []table.PartitionedTable

schema *expression.Schema
fullSchema *expression.Schema
isDesc bool
Expand All @@ -89,18 +91,28 @@ type tableRegionSampler struct {
isFinished bool
}

func newTableRegionSampler(ctx sessionctx.Context, t table.Table, startTs uint64, partTables []table.PartitionedTable,
schema *expression.Schema, fullSchema *expression.Schema, retTypes []*types.FieldType, desc bool) *tableRegionSampler {
func newTableRegionSampler(
ctx sessionctx.Context,
t table.Table,
startTs uint64,
pyhsicalTableID int64,
partTables []table.PartitionedTable,
schema *expression.Schema,
fullSchema *expression.Schema,
retTypes []*types.FieldType,
desc bool,
) *tableRegionSampler {
return &tableRegionSampler{
ctx: ctx,
table: t,
startTS: startTs,
partTables: partTables,
schema: schema,
fullSchema: fullSchema,
isDesc: desc,
retTypes: retTypes,
rowMap: make(map[int64]types.Datum),
ctx: ctx,
table: t,
startTS: startTs,
partTables: partTables,
physicalTableID: pyhsicalTableID,
schema: schema,
fullSchema: fullSchema,
isDesc: desc,
retTypes: retTypes,
rowMap: make(map[int64]types.Datum),
}
}

Expand Down Expand Up @@ -176,23 +188,31 @@ func (s *tableRegionSampler) writeChunkFromRanges(ranges []kv.KeyRange, req *chu
}

func (s *tableRegionSampler) splitTableRanges() ([]kv.KeyRange, error) {
if len(s.partTables) != 0 {
var ranges []kv.KeyRange
for _, t := range s.partTables {
for _, pid := range t.GetAllPartitionIDs() {
start := tablecodec.GenTableRecordPrefix(pid)
end := start.PrefixNext()
rs, err := splitIntoMultiRanges(s.ctx.GetStore(), start, end)
if err != nil {
return nil, err
}
ranges = append(ranges, rs...)
}
partitionTable := s.table.GetPartitionedTable()
if partitionTable == nil {
startKey, endKey := s.table.RecordPrefix(), s.table.RecordPrefix().PrefixNext()
return splitIntoMultiRanges(s.ctx.GetStore(), startKey, endKey)
}

var partIDs []int64
if partitionTable.Meta().ID == s.physicalTableID {
for _, p := range s.partTables {
partIDs = append(partIDs, p.GetAllPartitionIDs()...)
}
return ranges, nil
} else {
partIDs = []int64{s.physicalTableID}
}
startKey, endKey := s.table.RecordPrefix(), s.table.RecordPrefix().PrefixNext()
return splitIntoMultiRanges(s.ctx.GetStore(), startKey, endKey)
ranges := make([]kv.KeyRange, 0, len(partIDs))
for _, pid := range partIDs {
start := tablecodec.GenTableRecordPrefix(pid)
end := start.PrefixNext()
rs, err := splitIntoMultiRanges(s.ctx.GetStore(), start, end)
if err != nil {
return nil, err
}
ranges = append(ranges, rs...)
}
return ranges, nil
}

func splitIntoMultiRanges(store kv.Storage, startKey, endKey kv.Key) ([]kv.KeyRange, error) {
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -2540,6 +2540,7 @@ func (ds *DataSource) convertToSampleTable(prop *property.PhysicalProperty,
p := PhysicalTableSample{
TableSampleInfo: ds.SampleInfo,
TableInfo: ds.table,
PhysicalTableID: ds.physicalTableID,
Desc: candidate.isMatchProp && prop.SortItems[0].Desc,
}.Init(ds.SCtx(), ds.QueryBlockOffset())
p.schema = ds.schema
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -5017,7 +5017,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as
ds.SetSchema(schema)
ds.names = names
ds.setPreferredStoreType(b.TableHints())
ds.SampleInfo = NewTableSampleInfo(tn.TableSample, schema.Clone(), b.partitionedTable)
ds.SampleInfo = NewTableSampleInfo(tn.TableSample, schema, b.partitionedTable)
b.isSampling = ds.SampleInfo != nil

for i, colExpr := range ds.Schema().Columns {
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -2520,6 +2520,7 @@ type PhysicalTableSample struct {
physicalSchemaProducer
TableSampleInfo *TableSampleInfo
TableInfo table.Table
PhysicalTableID int64
Desc bool
}

Expand Down Expand Up @@ -2553,7 +2554,7 @@ func NewTableSampleInfo(node *ast.TableSample, fullSchema *expression.Schema, pt
}
return &TableSampleInfo{
AstNode: node,
FullSchema: fullSchema,
FullSchema: fullSchema.Clone(),
Partitions: pt,
}
}
Expand Down
14 changes: 14 additions & 0 deletions tests/integrationtest/r/executor/sample.result
Original file line number Diff line number Diff line change
Expand Up @@ -207,3 +207,17 @@ pk v
500 a
9223372036854775809 b
set @@global.tidb_scatter_region=default;
drop table if exists t;
create table t (a int, b varchar(255), primary key (a)) partition by hash(a) partitions 2;
insert into t values (1, '1'), (2, '2'), (3, '3');
set @@tidb_partition_prune_mode='static';
select * from t tablesample regions() order by a;
a b
1 1
2 2
set @@tidb_partition_prune_mode='dynamic';
select * from t tablesample regions() order by a;
a b
1 1
2 2
set @@tidb_partition_prune_mode=default;
10 changes: 10 additions & 0 deletions tests/integrationtest/t/executor/sample.test
Original file line number Diff line number Diff line change
Expand Up @@ -131,3 +131,13 @@ SPLIT TABLE a BY (500);
SELECT * FROM a TABLESAMPLE REGIONS() ORDER BY pk;

set @@global.tidb_scatter_region=default;

# TestTableSamplePartitionPruneMode
drop table if exists t;
create table t (a int, b varchar(255), primary key (a)) partition by hash(a) partitions 2;
insert into t values (1, '1'), (2, '2'), (3, '3');
set @@tidb_partition_prune_mode='static';
select * from t tablesample regions() order by a;
set @@tidb_partition_prune_mode='dynamic';
select * from t tablesample regions() order by a;
set @@tidb_partition_prune_mode=default;