table: Add GetExchangePartitionDMLSupport and remove GetDomainInfoSchema in MutateContext (#55435)
ref pingcap/tidb#53388
This commit is contained in:
@ -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
|
||||
}
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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`.
|
||||
|
||||
@ -7,6 +7,7 @@ go_library(
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
"//pkg/expression/context",
|
||||
"//pkg/infoschema/context",
|
||||
"//pkg/meta/autoid",
|
||||
"//pkg/parser/model",
|
||||
"//pkg/sessionctx",
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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())
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
Reference in New Issue
Block a user