Skip to content

Commit

Permalink
Makefile: add gotag for make server_check (#51057)
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored Oct 18, 2024
1 parent aed9e8e commit 55b3e37
Show file tree
Hide file tree
Showing 16 changed files with 80 additions and 36 deletions.
4 changes: 2 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
3 changes: 2 additions & 1 deletion pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/expropt/sessionvars.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
}

Expand Down
28 changes: 14 additions & 14 deletions pkg/expression/scalar_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand Down Expand Up @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/sessionexpr/sessionctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
6 changes: 2 additions & 4 deletions pkg/kv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
}
Expand Down
19 changes: 17 additions & 2 deletions pkg/planner/core/operator/logicalop/logical_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/table/tables/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/testkit/testkit.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand Down
5 changes: 3 additions & 2 deletions pkg/util/intest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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"],
Expand Down
13 changes: 8 additions & 5 deletions pkg/util/intest/assert.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand All @@ -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() {
Expand All @@ -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...)
}
Expand Down
File renamed without changes.
6 changes: 3 additions & 3 deletions pkg/util/intest/common.go → pkg/util/intest/no_assert.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
20 changes: 20 additions & 0 deletions pkg/util/intest/not_in_unittest.go
Original file line number Diff line number Diff line change
@@ -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
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 2 additions & 0 deletions tests/integrationtest/t/planner/core/indexmerge_path.test
Original file line number Diff line number Diff line change
Expand Up @@ -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'));

Expand Down

0 comments on commit 55b3e37

Please sign in to comment.