diff --git a/Makefile b/Makefile index 8ce2d695eb3bd..d70f78cbf2473 100644 --- a/Makefile +++ b/Makefile @@ -245,9 +245,9 @@ enterprise-server: .PHONY: server_check server_check: ifeq ($(TARGET), "") - $(GOBUILD) -cover $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o bin/tidb-server ./cmd/tidb-server + $(GOBUILD) -cover $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' --tags deadlock,enableassert -o bin/tidb-server ./cmd/tidb-server else - $(GOBUILD) -cover $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o '$(TARGET)' ./cmd/tidb-server + $(GOBUILD) -cover $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' --tags deadlock,enableassert -o '$(TARGET)' ./cmd/tidb-server endif .PHONY: linux diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 835fdc31d141d..73dc7d93603b9 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -313,7 +313,8 @@ func (e *memtableRetriever) setDataForVariablesInfo(ctx sessionctx.Context) erro func (e *memtableRetriever) setDataForUserAttributes(ctx context.Context, sctx sessionctx.Context) error { exec := sctx.GetRestrictedSQLExecutor() - chunkRows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT user, host, JSON_UNQUOTE(JSON_EXTRACT(user_attributes, '$.metadata')) FROM mysql.user`) + wrappedCtx := kv.WithInternalSourceType(ctx, kv.InternalTxnOthers) + chunkRows, _, err := exec.ExecRestrictedSQL(wrappedCtx, nil, `SELECT user, host, JSON_UNQUOTE(JSON_EXTRACT(user_attributes, '$.metadata')) FROM mysql.user`) if err != nil { return err } diff --git a/pkg/expression/expropt/sessionvars.go b/pkg/expression/expropt/sessionvars.go index 779f122a71166..8c0528baf9d06 100644 --- a/pkg/expression/expropt/sessionvars.go +++ b/pkg/expression/expropt/sessionvars.go @@ -54,7 +54,7 @@ func (SessionVarsPropReader) GetSessionVars(ctx exprctx.EvalContext) (*variable. return nil, err } - if intest.InTest { + if intest.EnableAssert { exprctx.AssertLocationWithSessionVars(ctx.Location(), p.vars.GetSessionVars()) } diff --git a/pkg/expression/scalar_function.go b/pkg/expression/scalar_function.go index fb032fe849c9c..f5ae914448347 100644 --- a/pkg/expression/scalar_function.go +++ b/pkg/expression/scalar_function.go @@ -52,7 +52,7 @@ type ScalarFunction struct { // VecEvalInt evaluates this expression in a vectorized manner. func (sf *ScalarFunction) VecEvalInt(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.vecEvalInt(ctx, input, result) @@ -61,7 +61,7 @@ func (sf *ScalarFunction) VecEvalInt(ctx EvalContext, input *chunk.Chunk, result // VecEvalReal evaluates this expression in a vectorized manner. func (sf *ScalarFunction) VecEvalReal(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.vecEvalReal(ctx, input, result) @@ -70,7 +70,7 @@ func (sf *ScalarFunction) VecEvalReal(ctx EvalContext, input *chunk.Chunk, resul // VecEvalString evaluates this expression in a vectorized manner. func (sf *ScalarFunction) VecEvalString(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.vecEvalString(ctx, input, result) @@ -79,7 +79,7 @@ func (sf *ScalarFunction) VecEvalString(ctx EvalContext, input *chunk.Chunk, res // VecEvalDecimal evaluates this expression in a vectorized manner. func (sf *ScalarFunction) VecEvalDecimal(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.vecEvalDecimal(ctx, input, result) @@ -88,7 +88,7 @@ func (sf *ScalarFunction) VecEvalDecimal(ctx EvalContext, input *chunk.Chunk, re // VecEvalTime evaluates this expression in a vectorized manner. func (sf *ScalarFunction) VecEvalTime(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.vecEvalTime(ctx, input, result) @@ -97,7 +97,7 @@ func (sf *ScalarFunction) VecEvalTime(ctx EvalContext, input *chunk.Chunk, resul // VecEvalDuration evaluates this expression in a vectorized manner. func (sf *ScalarFunction) VecEvalDuration(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.vecEvalDuration(ctx, input, result) @@ -106,7 +106,7 @@ func (sf *ScalarFunction) VecEvalDuration(ctx EvalContext, input *chunk.Chunk, r // VecEvalJSON evaluates this expression in a vectorized manner. func (sf *ScalarFunction) VecEvalJSON(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.vecEvalJSON(ctx, input, result) @@ -489,7 +489,7 @@ func (sf *ScalarFunction) Eval(ctx EvalContext, row chunk.Row) (d types.Datum, e // EvalInt implements Expression interface. func (sf *ScalarFunction) EvalInt(ctx EvalContext, row chunk.Row) (int64, bool, error) { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.evalInt(ctx, row) @@ -498,7 +498,7 @@ func (sf *ScalarFunction) EvalInt(ctx EvalContext, row chunk.Row) (int64, bool, // EvalReal implements Expression interface. func (sf *ScalarFunction) EvalReal(ctx EvalContext, row chunk.Row) (float64, bool, error) { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.evalReal(ctx, row) @@ -507,7 +507,7 @@ func (sf *ScalarFunction) EvalReal(ctx EvalContext, row chunk.Row) (float64, boo // EvalDecimal implements Expression interface. func (sf *ScalarFunction) EvalDecimal(ctx EvalContext, row chunk.Row) (*types.MyDecimal, bool, error) { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.evalDecimal(ctx, row) @@ -516,7 +516,7 @@ func (sf *ScalarFunction) EvalDecimal(ctx EvalContext, row chunk.Row) (*types.My // EvalString implements Expression interface. func (sf *ScalarFunction) EvalString(ctx EvalContext, row chunk.Row) (string, bool, error) { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.evalString(ctx, row) @@ -525,7 +525,7 @@ func (sf *ScalarFunction) EvalString(ctx EvalContext, row chunk.Row) (string, bo // EvalTime implements Expression interface. func (sf *ScalarFunction) EvalTime(ctx EvalContext, row chunk.Row) (types.Time, bool, error) { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.evalTime(ctx, row) @@ -534,7 +534,7 @@ func (sf *ScalarFunction) EvalTime(ctx EvalContext, row chunk.Row) (types.Time, // EvalDuration implements Expression interface. func (sf *ScalarFunction) EvalDuration(ctx EvalContext, row chunk.Row) (types.Duration, bool, error) { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.evalDuration(ctx, row) @@ -543,7 +543,7 @@ func (sf *ScalarFunction) EvalDuration(ctx EvalContext, row chunk.Row) (types.Du // EvalJSON implements Expression interface. func (sf *ScalarFunction) EvalJSON(ctx EvalContext, row chunk.Row) (types.BinaryJSON, bool, error) { intest.Assert(ctx != nil) - if intest.InTest { + if intest.EnableAssert { ctx = wrapEvalAssert(ctx, sf.Function) } return sf.Function.evalJSON(ctx, row) diff --git a/pkg/expression/sessionexpr/sessionctx.go b/pkg/expression/sessionexpr/sessionctx.go index 1dae8c5a6a7a6..abba39ae2444a 100644 --- a/pkg/expression/sessionexpr/sessionctx.go +++ b/pkg/expression/sessionexpr/sessionctx.go @@ -191,7 +191,7 @@ func (ctx *EvalContext) SQLMode() mysql.SQLMode { // TypeCtx returns the types.Context func (ctx *EvalContext) TypeCtx() (tc types.Context) { tc = ctx.sctx.GetSessionVars().StmtCtx.TypeCtx() - if intest.InTest { + if intest.EnableAssert { exprctx.AssertLocationWithSessionVars(tc.Location(), ctx.sctx.GetSessionVars()) } return diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 5e0d3399f9d1e..fb3346fbce230 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -212,10 +212,8 @@ func setRequestSourceForInnerTxn(ctx context.Context, txn Transaction) { } // panic in test mode in case there are requests without source in the future. // log warnings in production mode. - if intest.InTest { - panic("unexpected no source type context, if you see this error, " + - "the `RequestSourceTypeKey` is missing in your context") - } + intest.Assert(true, "unexpected no source type context, if you see this error, "+ + "the `RequestSourceTypeKey` is missing in your context") logutil.Logger(ctx).Warn("unexpected no source type context, if you see this warning, " + "the `RequestSourceTypeKey` is missing in the context") } diff --git a/pkg/planner/core/operator/logicalop/logical_join.go b/pkg/planner/core/operator/logicalop/logical_join.go index b5b4e1461fafb..33e1407ec7e7c 100644 --- a/pkg/planner/core/operator/logicalop/logical_join.go +++ b/pkg/planner/core/operator/logicalop/logical_join.go @@ -1123,10 +1123,25 @@ func (p *LogicalJoin) ExtractUsedCols(parentUsedCols []*expression.Column) (left } lChild := p.Children()[0] rChild := p.Children()[1] + lSchema := lChild.Schema() + rSchema := rChild.Schema() + // parentused col = t2.a + // leftChild schema = t1.a(t2.a) + and others + // rightChild schema = t3 related + and others + if join, ok := lChild.(*LogicalJoin); ok { + if join.FullSchema != nil { + lSchema = join.FullSchema + } + } + if join, ok := rChild.(*LogicalJoin); ok { + if join.FullSchema != nil { + rSchema = join.FullSchema + } + } for _, col := range parentUsedCols { - if lChild.Schema().Contains(col) { + if lSchema != nil && lSchema.Contains(col) { leftCols = append(leftCols, col) - } else if rChild.Schema().Contains(col) { + } else if rSchema != nil && rSchema.Contains(col) { rightCols = append(rightCols, col) } } diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index d4aa1c206f724..5da631be92d20 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -804,6 +804,9 @@ func extractPartitionExprColumns(ctx expression.BuildContext, expr string, partC var cols []*expression.Column var partExpr expression.Expression if len(partCols) == 0 { + if expr == "" { + return nil, nil, nil, errors.New("expression should not be an empty string") + } schema := expression.NewSchema(columns...) expr, err := expression.ParseSimpleExpr(ctx, expr, expression.WithInputSchemaAndNames(schema, names, nil)) if err != nil { diff --git a/pkg/testkit/testkit.go b/pkg/testkit/testkit.go index 332ca40a2e7ed..eac57bf400518 100644 --- a/pkg/testkit/testkit.go +++ b/pkg/testkit/testkit.go @@ -73,7 +73,7 @@ func NewTestKit(t testing.TB, store kv.Storage) *TestKit { if _, ok := t.(*testing.B); !ok { // Don't check `intest.InTest` for benchmark. We should allow to run benchmarks without `intest` tag, because some assert may have significant performance // impact. - require.True(t, intest.InTest, "you should add --tags=intest when to test, see https://pingcap.github.io/tidb-dev-guide/get-started/setup-an-ide.html for help") + require.True(t, intest.InTest && intest.EnableAssert, "you should add --tags=intest when to test, see https://pingcap.github.io/tidb-dev-guide/get-started/setup-an-ide.html for help") } testenv.SetGOMAXPROCSForTest() tk := &TestKit{ diff --git a/pkg/util/intest/BUILD.bazel b/pkg/util/intest/BUILD.bazel index d34d4b1cbdbad..c50f71ee6e645 100644 --- a/pkg/util/intest/BUILD.bazel +++ b/pkg/util/intest/BUILD.bazel @@ -4,8 +4,9 @@ go_library( name = "intest", srcs = [ "assert.go", #keep - "common.go", - "intest.go", #keep + "in_unittest.go", #keep + "no_assert.go", + "not_in_unittest.go", ], importpath = "github.com/pingcap/tidb/pkg/util/intest", visibility = ["//visibility:public"], diff --git a/pkg/util/intest/assert.go b/pkg/util/intest/assert.go index 1d96c38ff1999..3b4e604a58ba4 100644 --- a/pkg/util/intest/assert.go +++ b/pkg/util/intest/assert.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:build intest +//go:build intest || enableassert package intest @@ -21,23 +21,26 @@ import ( "reflect" ) +// EnableAssert checks if the assert function should work. +const EnableAssert = true + // Assert asserts a condition is true func Assert(cond bool, msgAndArgs ...any) { - if InTest && !cond { + if EnableAssert && !cond { doPanic("", msgAndArgs...) } } // AssertNoError asserts an error is nil func AssertNoError(err error, msgAndArgs ...any) { - if InTest && err != nil { + if EnableAssert && err != nil { doPanic(fmt.Sprintf("error is not nil: %+v", err), msgAndArgs...) } } // AssertNotNil asserts an object is not nil func AssertNotNil(obj any, msgAndArgs ...any) { - if InTest { + if EnableAssert { Assert(obj != nil, msgAndArgs...) value := reflect.ValueOf(obj) switch value.Kind() { @@ -49,7 +52,7 @@ func AssertNotNil(obj any, msgAndArgs ...any) { // AssertFunc asserts a function condition func AssertFunc(fn func() bool, msgAndArgs ...any) { - if InTest { + if EnableAssert { Assert(fn != nil, msgAndArgs...) Assert(fn(), msgAndArgs...) } diff --git a/pkg/util/intest/intest.go b/pkg/util/intest/in_unittest.go similarity index 100% rename from pkg/util/intest/intest.go rename to pkg/util/intest/in_unittest.go diff --git a/pkg/util/intest/common.go b/pkg/util/intest/no_assert.go similarity index 90% rename from pkg/util/intest/common.go rename to pkg/util/intest/no_assert.go index a2ebccd4b3507..7d1d268a6ca30 100644 --- a/pkg/util/intest/common.go +++ b/pkg/util/intest/no_assert.go @@ -12,12 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:build !intest +//go:build !intest && !enableassert package intest -// InTest checks if the code is running in test. -const InTest = false +// EnableAssert checks if the code is running in integration test. +const EnableAssert = false // Assert is a stub function in release build. // See the same function in `util/intest/assert.go` for the real implement in test. diff --git a/pkg/util/intest/not_in_unittest.go b/pkg/util/intest/not_in_unittest.go new file mode 100644 index 0000000000000..0955803b56ea8 --- /dev/null +++ b/pkg/util/intest/not_in_unittest.go @@ -0,0 +1,20 @@ +// Copyright 2024 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//go:build !intest + +package intest + +// InTest checks if the code is running in test. +const InTest = false diff --git a/tests/integrationtest/r/planner/core/indexmerge_path.result b/tests/integrationtest/r/planner/core/indexmerge_path.result index 73aa80d810bfe..b36d507346341 100644 --- a/tests/integrationtest/r/planner/core/indexmerge_path.result +++ b/tests/integrationtest/r/planner/core/indexmerge_path.result @@ -673,6 +673,7 @@ insert into t value(1,1,1, '{"b":[1,2,3,4]}'); insert into t value(2,2,2, '{"b":[3,4,5,6]}'); set tidb_analyze_version=2; analyze table t all columns; +set @@tidb_stats_load_sync_wait=1000; explain select * from t use index (iad) where a = 1; id estRows task access object operator info TableReader_7 1.00 root data:Selection_6 diff --git a/tests/integrationtest/t/planner/core/indexmerge_path.test b/tests/integrationtest/t/planner/core/indexmerge_path.test index 06bf1256d07dd..47f9fac32b9e6 100644 --- a/tests/integrationtest/t/planner/core/indexmerge_path.test +++ b/tests/integrationtest/t/planner/core/indexmerge_path.test @@ -230,6 +230,8 @@ insert into t value(1,1,1, '{"b":[1,2,3,4]}'); insert into t value(2,2,2, '{"b":[3,4,5,6]}'); set tidb_analyze_version=2; analyze table t all columns; +# The below explain is flaky, try to increase the timeout of sync load first. +set @@tidb_stats_load_sync_wait=1000; explain select * from t use index (iad) where a = 1; explain select * from t use index (iad) where a = 1 and (2 member of (d->'$.b'));