table: Add GetExchangePartitionDMLSupport and remove GetDomainInfoSchema in MutateContext (#55435)

ref pingcap/tidb#53388
This commit is contained in:
王超
2024-08-19 20:28:41 +08:00
committed by GitHub
parent 509d1bd93b
commit 7ef2d46ce1
7 changed files with 40 additions and 11 deletions

View File

@ -694,7 +694,7 @@ func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue
// Handle exchange partition
tbl := e.Table.Meta()
if tbl.ExchangePartitionInfo != nil && tbl.GetPartitionInfo() == nil {
if err := checkRowForExchangePartition(e.Ctx().GetTableCtx(), row, tbl); err != nil {
if err := checkRowForExchangePartition(e.Ctx(), row, tbl); err != nil {
return nil, err
}
}

View File

@ -83,7 +83,7 @@ func updateRecord(
// Handle exchange partition
tbl := t.Meta()
if tbl.ExchangePartitionInfo != nil && tbl.GetPartitionInfo() == nil {
if err := checkRowForExchangePartition(sctx.GetTableCtx(), newData, tbl); err != nil {
if err := checkRowForExchangePartition(sctx, newData, tbl); err != nil {
return false, err
}
}
@ -332,7 +332,7 @@ func resetErrDataTooLong(colName string, rowIdx int, _ error) error {
// checkRowForExchangePartition is only used for ExchangePartition by non-partitionTable during write only state.
// It check if rowData inserted or updated violate partition definition or checkConstraints of partitionTable.
func checkRowForExchangePartition(sctx table.MutateContext, row []types.Datum, tbl *model.TableInfo) error {
func checkRowForExchangePartition(sctx sessionctx.Context, row []types.Datum, tbl *model.TableInfo) error {
is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema)
pt, tableFound := is.TableByID(context.Background(), tbl.ExchangePartitionInfo.ExchangePartitionTableID)
if !tableFound {

View File

@ -103,13 +103,18 @@ type TemporaryTableSupport interface {
AddTemporaryTableToTxn(tblInfo *model.TableInfo) (TemporaryTableHandler, bool)
}
// ExchangePartitionDMLSupport is used for DML operations when the table exchanging a partition.
type ExchangePartitionDMLSupport interface {
// GetInfoSchemaToCheckExchangeConstraint is used by DML to get the exchanged table to check
// constraints when exchanging partition.
GetInfoSchemaToCheckExchangeConstraint() infoschema.MetaOnlyInfoSchema
}
// MutateContext is used to when mutating a table.
type MutateContext interface {
AllocatorContext
// GetExprCtx returns the context to build or evaluate expressions
GetExprCtx() exprctx.ExprContext
// GetDomainInfoSchema returns the latest information schema in domain
GetDomainInfoSchema() infoschema.MetaOnlyInfoSchema
// ConnectionID returns the id of the current connection.
// If the current environment is not in a query from the client, the return value is 0.
ConnectionID() uint64
@ -141,6 +146,9 @@ type MutateContext interface {
// GetTemporaryTableSupport returns a `TemporaryTableSupport` if the context supports it.
// If the context does not support temporary table, the second return value will be false.
GetTemporaryTableSupport() (TemporaryTableSupport, bool)
// GetExchangePartitionDMLSupport returns a `ExchangePartitionDMLSupport` if the context supports it.
// ExchangePartitionDMLSupport is used by DMLs when the table is exchanging a partition.
GetExchangePartitionDMLSupport() (ExchangePartitionDMLSupport, bool)
}
// AllocatorContext is used to provide context for method `table.Allocators`.

View File

@ -7,6 +7,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/expression/context",
"//pkg/infoschema/context",
"//pkg/meta/autoid",
"//pkg/parser/model",
"//pkg/sessionctx",

View File

@ -17,6 +17,7 @@ package contextimpl
import (
"github.com/pingcap/failpoint"
exprctx "github.com/pingcap/tidb/pkg/expression/context"
infoschema "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/meta/autoid"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx"
@ -180,6 +181,16 @@ func (ctx *TableContextImpl) GetTemporaryTableSupport() (context.TemporaryTableS
return ctx, true
}
// GetInfoSchemaToCheckExchangeConstraint implements the ExchangePartitionDMLSupport interface.
func (ctx *TableContextImpl) GetInfoSchemaToCheckExchangeConstraint() infoschema.MetaOnlyInfoSchema {
return ctx.Context.GetDomainInfoSchema()
}
// GetExchangePartitionDMLSupport implements the MutateContext interface.
func (ctx *TableContextImpl) GetExchangePartitionDMLSupport() (context.ExchangePartitionDMLSupport, bool) {
return ctx, true
}
// GetTemporaryTableSizeLimit implements TemporaryTableSupport interface.
func (ctx *TableContextImpl) GetTemporaryTableSizeLimit() int64 {
return ctx.vars().TMPTableSize

View File

@ -169,4 +169,7 @@ func TestMutateContextImplFields(t *testing.T) {
require.Equal(t, int64(333), tmpTblHandler.GetDirtySize())
tmpTblHandler.UpdateTxnDeltaSize(-1)
require.Equal(t, int64(332), tmpTblHandler.GetDirtySize())
exchange, ok := ctx.GetExchangePartitionDMLSupport()
require.True(t, ok)
require.Same(t, ctx.GetDomainInfoSchema(), exchange.GetInfoSchemaToCheckExchangeConstraint())
}

View File

@ -1568,25 +1568,31 @@ func (t *partitionTableWithGivenSets) GetPartitionByRow(ctx expression.EvalConte
// checkConstraintForExchangePartition is only used for ExchangePartition by partitionTable during write only state.
// It check if rowData inserted or updated violate checkConstraints of non-partitionTable.
func checkConstraintForExchangePartition(sctx table.MutateContext, row []types.Datum, partID, ntID int64) error {
func checkConstraintForExchangePartition(ctx table.MutateContext, row []types.Datum, partID, ntID int64) error {
support, ok := ctx.GetExchangePartitionDMLSupport()
if !ok {
return errors.New("ctx does not support operations when exchanging a partition")
}
type InfoSchema interface {
TableByID(ctx stdctx.Context, id int64) (val table.Table, ok bool)
}
is, ok := sctx.GetDomainInfoSchema().(InfoSchema)
is, ok := support.GetInfoSchemaToCheckExchangeConstraint().(InfoSchema)
if !ok {
return errors.Errorf("exchange partition process assert inforSchema failed")
}
ctx := stdctx.Background()
nt, tableFound := is.TableByID(ctx, ntID)
gCtx := stdctx.Background()
nt, tableFound := is.TableByID(gCtx, ntID)
if !tableFound {
// Now partID is nt tableID.
nt, tableFound = is.TableByID(ctx, partID)
nt, tableFound = is.TableByID(gCtx, partID)
if !tableFound {
return errors.Errorf("exchange partition process table by id failed")
}
}
evalCtx := sctx.GetExprCtx().GetEvalCtx()
evalCtx := ctx.GetExprCtx().GetEvalCtx()
if err := table.CheckRowConstraintWithDatum(evalCtx, nt.WritableConstraint(), row); err != nil {
// TODO: make error include ExchangePartition info.
return err