From 7ef2d46ce117e8bf5e47156dbff7bf760797e7c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Mon, 19 Aug 2024 20:28:41 +0800 Subject: [PATCH] table: Add `GetExchangePartitionDMLSupport` and remove `GetDomainInfoSchema` in `MutateContext` (#55435) ref pingcap/tidb#53388 --- pkg/executor/insert_common.go | 2 +- pkg/executor/write.go | 4 ++-- pkg/table/context/table.go | 12 ++++++++++-- pkg/table/contextimpl/BUILD.bazel | 1 + pkg/table/contextimpl/table.go | 11 +++++++++++ pkg/table/contextimpl/table_test.go | 3 +++ pkg/table/tables/partition.go | 18 ++++++++++++------ 7 files changed, 40 insertions(+), 11 deletions(-) diff --git a/pkg/executor/insert_common.go b/pkg/executor/insert_common.go index edd4ebf4dd..27d6918a46 100644 --- a/pkg/executor/insert_common.go +++ b/pkg/executor/insert_common.go @@ -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 } } diff --git a/pkg/executor/write.go b/pkg/executor/write.go index a5683ee40c..56538d29a7 100644 --- a/pkg/executor/write.go +++ b/pkg/executor/write.go @@ -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 { diff --git a/pkg/table/context/table.go b/pkg/table/context/table.go index dc498d57b2..06c06359c2 100644 --- a/pkg/table/context/table.go +++ b/pkg/table/context/table.go @@ -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`. diff --git a/pkg/table/contextimpl/BUILD.bazel b/pkg/table/contextimpl/BUILD.bazel index ccc6afe99f..1f39291ead 100644 --- a/pkg/table/contextimpl/BUILD.bazel +++ b/pkg/table/contextimpl/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/expression/context", + "//pkg/infoschema/context", "//pkg/meta/autoid", "//pkg/parser/model", "//pkg/sessionctx", diff --git a/pkg/table/contextimpl/table.go b/pkg/table/contextimpl/table.go index 1f701246ab..329bba43f6 100644 --- a/pkg/table/contextimpl/table.go +++ b/pkg/table/contextimpl/table.go @@ -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 diff --git a/pkg/table/contextimpl/table_test.go b/pkg/table/contextimpl/table_test.go index 7c5481b192..a50ea689ed 100644 --- a/pkg/table/contextimpl/table_test.go +++ b/pkg/table/contextimpl/table_test.go @@ -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()) } diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index 25e62b0dd4..87a759d63d 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -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