You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 

687 lines
22 KiB

// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package tables
import (
"bytes"
"context"
stderr "errors"
"fmt"
"sort"
"strconv"
"strings"
"sync"
"github.com/pingcap/errors"
"github.com/pingcap/parser"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mock"
"go.uber.org/zap"
)
// Both partition and partitionedTable implement the table.Table interface.
var _ table.Table = &partition{}
var _ table.Table = &partitionedTable{}
// partitionedTable implements the table.PartitionedTable interface.
var _ table.PartitionedTable = &partitionedTable{}
// partition is a feature from MySQL:
// See https://dev.mysql.com/doc/refman/8.0/en/partitioning.html
// A partition table may contain many partitions, each partition has a unique partition
// id. The underlying representation of a partition and a normal table (a table with no
// partitions) is basically the same.
// partition also implements the table.Table interface.
type partition struct {
TableCommon
}
// GetPhysicalID implements table.Table GetPhysicalID interface.
func (p *partition) GetPhysicalID() int64 {
return p.physicalTableID
}
// partitionedTable implements the table.PartitionedTable interface.
// partitionedTable is a table, it contains many Partitions.
type partitionedTable struct {
TableCommon
partitionExpr *PartitionExpr
partitions map[int64]*partition
evalBufferTypes []*types.FieldType
evalBufferPool sync.Pool
}
func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Table, error) {
ret := &partitionedTable{TableCommon: *tbl}
partitionExpr, err := newPartitionExpr(tblInfo)
if err != nil {
return nil, errors.Trace(err)
}
ret.partitionExpr = partitionExpr
initEvalBufferType(ret)
ret.evalBufferPool = sync.Pool{
New: func() interface{} {
return initEvalBuffer(ret)
},
}
if err := initTableIndices(&ret.TableCommon); err != nil {
return nil, errors.Trace(err)
}
pi := tblInfo.GetPartitionInfo()
partitions := make(map[int64]*partition, len(pi.Definitions))
for _, p := range pi.Definitions {
var t partition
err := initTableCommonWithIndices(&t.TableCommon, tblInfo, p.ID, tbl.Columns, tbl.allocs)
if err != nil {
return nil, errors.Trace(err)
}
partitions[p.ID] = &t
}
ret.partitions = partitions
return ret, nil
}
func newPartitionExpr(tblInfo *model.TableInfo) (*PartitionExpr, error) {
ctx := mock.NewContext()
dbName := model.NewCIStr(ctx.GetSessionVars().CurrentDB)
columns, names := expression.ColumnInfos2ColumnsAndNames(ctx, dbName, tblInfo.Name, tblInfo.Columns, tblInfo)
pi := tblInfo.GetPartitionInfo()
switch pi.Type {
case model.PartitionTypeRange:
return generateRangePartitionExpr(ctx, pi, columns, names)
case model.PartitionTypeHash:
return generateHashPartitionExpr(ctx, pi, columns, names)
}
panic("cannot reach here")
}
// PartitionExpr is the partition definition expressions.
type PartitionExpr struct {
// UpperBounds: (x < y1); (x < y2); (x < y3), used by locatePartition.
UpperBounds []expression.Expression
// OrigExpr is the partition expression ast used in point get.
OrigExpr ast.ExprNode
// Expr is the hash partition expression.
Expr expression.Expression
// Used in the range pruning process.
*ForRangePruning
// Used in the range column pruning process.
*ForRangeColumnsPruning
}
func initEvalBufferType(t *partitionedTable) {
hasExtraHandle := false
numCols := len(t.Cols())
if !t.Meta().PKIsHandle {
hasExtraHandle = true
numCols++
}
t.evalBufferTypes = make([]*types.FieldType, numCols)
for i, col := range t.Cols() {
t.evalBufferTypes[i] = &col.FieldType
}
if hasExtraHandle {
t.evalBufferTypes[len(t.evalBufferTypes)-1] = types.NewFieldType(mysql.TypeLonglong)
}
}
func initEvalBuffer(t *partitionedTable) *chunk.MutRow {
evalBuffer := chunk.MutRowFromTypes(t.evalBufferTypes)
return &evalBuffer
}
// ForRangeColumnsPruning is used for range partition pruning.
type ForRangeColumnsPruning struct {
LessThan []expression.Expression
MaxValue bool
}
func dataForRangeColumnsPruning(ctx sessionctx.Context, pi *model.PartitionInfo, schema *expression.Schema, names []*types.FieldName, p *parser.Parser) (*ForRangeColumnsPruning, error) {
var res ForRangeColumnsPruning
res.LessThan = make([]expression.Expression, len(pi.Definitions))
for i := 0; i < len(pi.Definitions); i++ {
if strings.EqualFold(pi.Definitions[i].LessThan[0], "MAXVALUE") {
// Use a bool flag instead of math.MaxInt64 to avoid the corner cases.
res.MaxValue = true
} else {
tmp, err := parseSimpleExprWithNames(p, ctx, pi.Definitions[i].LessThan[0], schema, names)
if err != nil {
return nil, err
}
res.LessThan[i] = tmp
}
}
return &res, nil
}
// parseSimpleExprWithNames parses simple expression string to Expression.
// The expression string must only reference the column in the given NameSlice.
func parseSimpleExprWithNames(p *parser.Parser, ctx sessionctx.Context, exprStr string, schema *expression.Schema, names types.NameSlice) (expression.Expression, error) {
exprNode, err := parseExpr(p, exprStr)
if err != nil {
return nil, errors.Trace(err)
}
return expression.RewriteSimpleExprWithNames(ctx, exprNode, schema, names)
}
// ForRangePruning is used for range partition pruning.
type ForRangePruning struct {
LessThan []int64
MaxValue bool
Unsigned bool
}
// dataForRangePruning extracts the less than parts from 'partition p0 less than xx ... partitoin p1 less than ...'
func dataForRangePruning(sctx sessionctx.Context, pi *model.PartitionInfo) (*ForRangePruning, error) {
var maxValue bool
var unsigned bool
lessThan := make([]int64, len(pi.Definitions))
for i := 0; i < len(pi.Definitions); i++ {
if strings.EqualFold(pi.Definitions[i].LessThan[0], "MAXVALUE") {
// Use a bool flag instead of math.MaxInt64 to avoid the corner cases.
maxValue = true
} else {
var err error
lessThan[i], err = strconv.ParseInt(pi.Definitions[i].LessThan[0], 10, 64)
var numErr *strconv.NumError
if stderr.As(err, &numErr) && numErr.Err == strconv.ErrRange {
var tmp uint64
tmp, err = strconv.ParseUint(pi.Definitions[i].LessThan[0], 10, 64)
lessThan[i] = int64(tmp)
unsigned = true
}
if err != nil {
val, ok := fixOldVersionPartitionInfo(sctx, pi.Definitions[i].LessThan[0])
if !ok {
logutil.BgLogger().Error("wrong partition definition", zap.String("less than", pi.Definitions[i].LessThan[0]))
return nil, errors.WithStack(err)
}
lessThan[i] = val
}
}
}
return &ForRangePruning{
LessThan: lessThan,
MaxValue: maxValue,
Unsigned: unsigned,
}, nil
}
func fixOldVersionPartitionInfo(sctx sessionctx.Context, str string) (int64, bool) {
// less than value should be calculate to integer before persistent.
// Old version TiDB may not do it and store the raw expression.
tmp, err := parseSimpleExprWithNames(parser.New(), sctx, str, nil, nil)
if err != nil {
return 0, false
}
ret, isNull, err := tmp.EvalInt(sctx, chunk.Row{})
if err != nil || isNull {
return 0, false
}
return ret, true
}
// rangePartitionString returns the partition string for a range typed partition.
func rangePartitionString(pi *model.PartitionInfo) string {
// partition by range expr
if len(pi.Columns) == 0 {
return pi.Expr
}
// partition by range columns (c1)
if len(pi.Columns) == 1 {
return pi.Columns[0].L
}
// partition by range columns (c1, c2, ...)
panic("create table assert len(columns) = 1")
}
func generateRangePartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo,
columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) {
// The caller should assure partition info is not nil.
locateExprs := make([]expression.Expression, 0, len(pi.Definitions))
var buf bytes.Buffer
p := parser.New()
schema := expression.NewSchema(columns...)
partStr := rangePartitionString(pi)
for i := 0; i < len(pi.Definitions); i++ {
if strings.EqualFold(pi.Definitions[i].LessThan[0], "MAXVALUE") {
// Expr less than maxvalue is always true.
fmt.Fprintf(&buf, "true")
} else {
fmt.Fprintf(&buf, "((%s) < (%s))", partStr, pi.Definitions[i].LessThan[0])
}
expr, err := parseSimpleExprWithNames(p, ctx, buf.String(), schema, names)
if err != nil {
// If it got an error here, ddl may hang forever, so this error log is important.
logutil.BgLogger().Error("wrong table partition expression", zap.String("expression", buf.String()), zap.Error(err))
return nil, errors.Trace(err)
}
locateExprs = append(locateExprs, expr)
buf.Reset()
}
ret := &PartitionExpr{
UpperBounds: locateExprs,
}
switch len(pi.Columns) {
case 0:
exprs, err := parseSimpleExprWithNames(p, ctx, pi.Expr, schema, names)
if err != nil {
return nil, err
}
tmp, err := dataForRangePruning(ctx, pi)
if err != nil {
return nil, errors.Trace(err)
}
ret.Expr = exprs
ret.ForRangePruning = tmp
case 1:
tmp, err := dataForRangeColumnsPruning(ctx, pi, schema, names, p)
if err != nil {
return nil, errors.Trace(err)
}
ret.ForRangeColumnsPruning = tmp
default:
panic("range column partition currently support only one column")
}
return ret, nil
}
func generateHashPartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo,
columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) {
// The caller should assure partition info is not nil.
schema := expression.NewSchema(columns...)
origExpr, err := parseExpr(parser.New(), pi.Expr)
if err != nil {
return nil, err
}
exprs, err := rewritePartitionExpr(ctx, origExpr, schema, names)
if err != nil {
// If it got an error here, ddl may hang forever, so this error log is important.
logutil.BgLogger().Error("wrong table partition expression", zap.String("expression", pi.Expr), zap.Error(err))
return nil, errors.Trace(err)
}
exprs.HashCode(ctx.GetSessionVars().StmtCtx)
return &PartitionExpr{
Expr: exprs,
OrigExpr: origExpr,
}, nil
}
// PartitionExpr returns the partition expression.
func (t *partitionedTable) PartitionExpr() (*PartitionExpr, error) {
return t.partitionExpr, nil
}
// PartitionRecordKey is exported for test.
func PartitionRecordKey(pid int64, handle int64) kv.Key {
recordPrefix := tablecodec.GenTableRecordPrefix(pid)
return tablecodec.EncodeRecordKey(recordPrefix, handle)
}
// locatePartition returns the partition ID of the input record.
func (t *partitionedTable) locatePartition(ctx sessionctx.Context, pi *model.PartitionInfo, r []types.Datum) (int64, error) {
var err error
var idx int
switch t.meta.Partition.Type {
case model.PartitionTypeRange:
if len(pi.Columns) == 0 {
idx, err = t.locateRangePartition(ctx, pi, r)
} else {
idx, err = t.locateRangeColumnPartition(ctx, pi, r)
}
case model.PartitionTypeHash:
idx, err = t.locateHashPartition(ctx, pi, r)
}
if err != nil {
return 0, errors.Trace(err)
}
return pi.Definitions[idx].ID, nil
}
func (t *partitionedTable) locateRangeColumnPartition(ctx sessionctx.Context, pi *model.PartitionInfo, r []types.Datum) (int, error) {
var err error
var isNull bool
partitionExprs := t.partitionExpr.UpperBounds
evalBuffer := t.evalBufferPool.Get().(*chunk.MutRow)
defer t.evalBufferPool.Put(evalBuffer)
idx := sort.Search(len(partitionExprs), func(i int) bool {
evalBuffer.SetDatums(r...)
ret, isNull, err := partitionExprs[i].EvalInt(ctx, evalBuffer.ToRow())
if err != nil {
return true // Break the search.
}
if isNull {
// If the column value used to determine the partition is NULL, the row is inserted into the lowest partition.
// See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-handling-nulls.html
return true // Break the search.
}
return ret > 0
})
if err != nil {
return 0, errors.Trace(err)
}
if isNull {
idx = 0
}
if idx < 0 || idx >= len(partitionExprs) {
// The data does not belong to any of the partition returns `table has no partition for value %s`.
var valueMsg string
if pi.Expr != "" {
e, err := expression.ParseSimpleExprWithTableInfo(ctx, pi.Expr, t.meta)
if err == nil {
val, _, err := e.EvalInt(ctx, chunk.MutRowFromDatums(r).ToRow())
if err == nil {
valueMsg = fmt.Sprintf("%d", val)
}
}
} else {
// When the table is partitioned by range columns.
valueMsg = "from column_list"
}
return 0, table.ErrNoPartitionForGivenValue.GenWithStackByArgs(valueMsg)
}
return idx, nil
}
func (t *partitionedTable) locateRangePartition(ctx sessionctx.Context, pi *model.PartitionInfo, r []types.Datum) (int, error) {
var (
ret int64
val int64
isNull bool
err error
)
if col, ok := t.partitionExpr.Expr.(*expression.Column); ok {
if r[col.Index].IsNull() {
isNull = true
}
ret = r[col.Index].GetInt64()
} else {
evalBuffer := t.evalBufferPool.Get().(*chunk.MutRow)
defer t.evalBufferPool.Put(evalBuffer)
evalBuffer.SetDatums(r...)
val, isNull, err = t.partitionExpr.Expr.EvalInt(ctx, evalBuffer.ToRow())
if err != nil {
return 0, err
}
ret = val
}
unsigned := mysql.HasUnsignedFlag(t.partitionExpr.Expr.GetType().Flag)
ranges := t.partitionExpr.ForRangePruning
length := len(ranges.LessThan)
pos := sort.Search(length, func(i int) bool {
if isNull {
return true
}
return ranges.compare(i, ret, unsigned) > 0
})
if isNull {
pos = 0
}
if pos < 0 || pos >= length {
// The data does not belong to any of the partition returns `table has no partition for value %s`.
var valueMsg string
if pi.Expr != "" {
e, err := expression.ParseSimpleExprWithTableInfo(ctx, pi.Expr, t.meta)
if err == nil {
val, _, err := e.EvalInt(ctx, chunk.MutRowFromDatums(r).ToRow())
if err == nil {
valueMsg = fmt.Sprintf("%d", val)
}
}
} else {
// When the table is partitioned by range columns.
valueMsg = "from column_list"
}
return 0, table.ErrNoPartitionForGivenValue.GenWithStackByArgs(valueMsg)
}
return pos, nil
}
// TODO: supports linear hashing
func (t *partitionedTable) locateHashPartition(ctx sessionctx.Context, pi *model.PartitionInfo, r []types.Datum) (int, error) {
if col, ok := t.partitionExpr.Expr.(*expression.Column); ok {
var data types.Datum
switch r[col.Index].Kind() {
case types.KindInt64, types.KindUint64:
data = r[col.Index]
default:
var err error
data, err = r[col.Index].ConvertTo(ctx.GetSessionVars().StmtCtx, types.NewFieldType(mysql.TypeLong))
if err != nil {
return 0, err
}
}
ret := data.GetInt64()
ret = ret % int64(t.meta.Partition.Num)
if ret < 0 {
ret = -ret
}
return int(ret), nil
}
evalBuffer := t.evalBufferPool.Get().(*chunk.MutRow)
defer t.evalBufferPool.Put(evalBuffer)
evalBuffer.SetDatums(r...)
ret, isNull, err := t.partitionExpr.Expr.EvalInt(ctx, evalBuffer.ToRow())
if err != nil {
return 0, err
}
if isNull {
return 0, nil
}
ret = ret % int64(t.meta.Partition.Num)
if ret < 0 {
ret = -ret
}
return int(ret), nil
}
// GetPartition returns a Table, which is actually a partition.
func (t *partitionedTable) GetPartition(pid int64) table.PhysicalTable {
// Attention, can't simply use `return t.partitions[pid]` here.
// Because A nil of type *partition is a kind of `table.PhysicalTable`
p, ok := t.partitions[pid]
if !ok {
return nil
}
return p
}
// GetPartitionByRow returns a Table, which is actually a Partition.
func (t *partitionedTable) GetPartitionByRow(ctx sessionctx.Context, r []types.Datum) (table.PhysicalTable, error) {
pid, err := t.locatePartition(ctx, t.Meta().GetPartitionInfo(), r)
if err != nil {
return nil, errors.Trace(err)
}
return t.partitions[pid], nil
}
// AddRecord implements the AddRecord method for the table.Table interface.
func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) {
return partitionedTableAddRecord(ctx, t, r, nil, opts)
}
func partitionedTableAddRecord(ctx sessionctx.Context, t *partitionedTable, r []types.Datum, partitionSelection map[int64]struct{}, opts []table.AddRecordOption) (recordID int64, err error) {
partitionInfo := t.meta.GetPartitionInfo()
pid, err := t.locatePartition(ctx, partitionInfo, r)
if err != nil {
return 0, errors.Trace(err)
}
if partitionSelection != nil {
if _, ok := partitionSelection[pid]; !ok {
return 0, errors.WithStack(table.ErrRowDoesNotMatchGivenPartitionSet)
}
}
tbl := t.GetPartition(pid)
return tbl.AddRecord(ctx, r, opts...)
}
// partitionTableWithGivenSets is used for this kind of grammar: partition (p0,p1)
// Basically it is the same as partitionedTable except that partitionTableWithGivenSets
// checks the given partition set for AddRecord/UpdateRecord operations.
type partitionTableWithGivenSets struct {
*partitionedTable
partitions map[int64]struct{}
}
// NewPartitionTableithGivenSets creates a new partition table from a partition table.
func NewPartitionTableithGivenSets(tbl table.PartitionedTable, partitions map[int64]struct{}) table.PartitionedTable {
if raw, ok := tbl.(*partitionedTable); ok {
return &partitionTableWithGivenSets{
partitionedTable: raw,
partitions: partitions,
}
}
return tbl
}
// AddRecord implements the AddRecord method for the table.Table interface.
func (t *partitionTableWithGivenSets) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) {
return partitionedTableAddRecord(ctx, t.partitionedTable, r, t.partitions, opts)
}
// RemoveRecord implements table.Table RemoveRecord interface.
func (t *partitionedTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error {
partitionInfo := t.meta.GetPartitionInfo()
pid, err := t.locatePartition(ctx, partitionInfo, r)
if err != nil {
return errors.Trace(err)
}
tbl := t.GetPartition(pid)
return tbl.RemoveRecord(ctx, h, r)
}
// UpdateRecord implements table.Table UpdateRecord interface.
// `touched` means which columns are really modified, used for secondary indices.
// Length of `oldData` and `newData` equals to length of `t.WritableCols()`.
func (t *partitionedTable) UpdateRecord(ctx context.Context, sctx sessionctx.Context, h int64, currData, newData []types.Datum, touched []bool) error {
return partitionedTableUpdateRecord(ctx, sctx, t, h, currData, newData, touched, nil)
}
func (t *partitionTableWithGivenSets) UpdateRecord(ctx context.Context, sctx sessionctx.Context, h int64, currData, newData []types.Datum, touched []bool) error {
return partitionedTableUpdateRecord(ctx, sctx, t.partitionedTable, h, currData, newData, touched, t.partitions)
}
func partitionedTableUpdateRecord(ctx context.Context, sctx sessionctx.Context, t *partitionedTable, h int64, currData, newData []types.Datum, touched []bool, partitionSelection map[int64]struct{}) error {
partitionInfo := t.meta.GetPartitionInfo()
from, err := t.locatePartition(sctx, partitionInfo, currData)
if err != nil {
return errors.Trace(err)
}
to, err := t.locatePartition(sctx, partitionInfo, newData)
if err != nil {
return errors.Trace(err)
}
if partitionSelection != nil {
if _, ok := partitionSelection[to]; !ok {
return errors.WithStack(table.ErrRowDoesNotMatchGivenPartitionSet)
}
}
// The old and new data locate in different partitions.
// Remove record from old partition and add record to new partition.
if from != to {
_, err = t.GetPartition(to).AddRecord(sctx, newData)
if err != nil {
return errors.Trace(err)
}
// UpdateRecord should be side effect free, but there're two steps here.
// What would happen if step1 succeed but step2 meets error? It's hard
// to rollback.
// So this special order is chosen: add record first, errors such as
// 'Key Already Exists' will generally happen during step1, errors are
// unlikely to happen in step2.
err = t.GetPartition(from).RemoveRecord(sctx, h, currData)
if err != nil {
logutil.BgLogger().Error("update partition record fails", zap.String("message", "new record inserted while old record is not removed"), zap.Error(err))
return errors.Trace(err)
}
return nil
}
tbl := t.GetPartition(to)
return tbl.UpdateRecord(ctx, sctx, h, currData, newData, touched)
}
// FindPartitionByName finds partition in table meta by name.
func FindPartitionByName(meta *model.TableInfo, parName string) (int64, error) {
// Hash partition table use p0, p1, p2, p3 as partition names automatically.
parName = strings.ToLower(parName)
for _, def := range meta.Partition.Definitions {
if strings.EqualFold(def.Name.L, parName) {
return def.ID, nil
}
}
return -1, errors.Trace(table.ErrUnknownPartition.GenWithStackByArgs(parName, meta.Name.O))
}
func parseExpr(p *parser.Parser, exprStr string) (ast.ExprNode, error) {
exprStr = "select " + exprStr
stmts, _, err := p.Parse(exprStr, "", "")
if err != nil {
return nil, util.SyntaxWarn(err)
}
fields := stmts[0].(*ast.SelectStmt).Fields.Fields
return fields[0].Expr, nil
}
func rewritePartitionExpr(ctx sessionctx.Context, field ast.ExprNode, schema *expression.Schema, names types.NameSlice) (expression.Expression, error) {
expr, err := expression.RewriteSimpleExprWithNames(ctx, field, schema, names)
return expr, err
}
func compareUnsigned(v1, v2 int64) int {
switch {
case uint64(v1) > uint64(v2):
return 1
case uint64(v1) == uint64(v2):
return 0
}
return -1
}
func (lt *ForRangePruning) compare(ith int, v int64, unsigned bool) int {
if ith == len(lt.LessThan)-1 {
if lt.MaxValue {
return 1
}
}
if unsigned {
return compareUnsigned(lt.LessThan[ith], v)
}
switch {
case lt.LessThan[ith] > v:
return 1
case lt.LessThan[ith] == v:
return 0
}
return -1
}