From 6afe4d980ccdbe8ebedd21a6716310e91f1011fc Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 24 Dec 2021 15:45:04 +0800 Subject: [PATCH 1/5] test: merge serial tests in cmd, planner, server, util --- .../{column_serial_test.go => column_test.go} | 0 .../{ddl_serial_test.go => ddl_test.go} | 0 .../{index_serial_test.go => index_test.go} | 0 ...o_pb_serial_test.go => plan_to_pb_test.go} | 0 ...ial_test.go => statistics_handler_test.go} | 0 ...ext_serial_test.go => txn_context_test.go} | 0 util/chunk/row_container_serial_test.go | 111 ------ util/chunk/row_container_test.go | 87 ++++ util/ranger/ranger_serial_test.go | 374 ------------------ util/ranger/ranger_test.go | 345 ++++++++++++++++ 10 files changed, 432 insertions(+), 485 deletions(-) rename cmd/ddltest/{column_serial_test.go => column_test.go} (100%) rename cmd/ddltest/{ddl_serial_test.go => ddl_test.go} (100%) rename cmd/ddltest/{index_serial_test.go => index_test.go} (100%) rename planner/core/{plan_to_pb_serial_test.go => plan_to_pb_test.go} (100%) rename server/{statistics_handler_serial_test.go => statistics_handler_test.go} (100%) rename sessiontxn/{txn_context_serial_test.go => txn_context_test.go} (100%) delete mode 100644 util/chunk/row_container_serial_test.go delete mode 100644 util/ranger/ranger_serial_test.go diff --git a/cmd/ddltest/column_serial_test.go b/cmd/ddltest/column_test.go similarity index 100% rename from cmd/ddltest/column_serial_test.go rename to cmd/ddltest/column_test.go diff --git a/cmd/ddltest/ddl_serial_test.go b/cmd/ddltest/ddl_test.go similarity index 100% rename from cmd/ddltest/ddl_serial_test.go rename to cmd/ddltest/ddl_test.go diff --git a/cmd/ddltest/index_serial_test.go b/cmd/ddltest/index_test.go similarity index 100% rename from cmd/ddltest/index_serial_test.go rename to cmd/ddltest/index_test.go diff --git a/planner/core/plan_to_pb_serial_test.go b/planner/core/plan_to_pb_test.go similarity index 100% rename from planner/core/plan_to_pb_serial_test.go rename to planner/core/plan_to_pb_test.go diff --git a/server/statistics_handler_serial_test.go b/server/statistics_handler_test.go similarity index 100% rename from server/statistics_handler_serial_test.go rename to server/statistics_handler_test.go diff --git a/sessiontxn/txn_context_serial_test.go b/sessiontxn/txn_context_test.go similarity index 100% rename from sessiontxn/txn_context_serial_test.go rename to sessiontxn/txn_context_test.go diff --git a/util/chunk/row_container_serial_test.go b/util/chunk/row_container_serial_test.go deleted file mode 100644 index 0be690a444a35..0000000000000 --- a/util/chunk/row_container_serial_test.go +++ /dev/null @@ -1,111 +0,0 @@ -// Copyright 2021 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. - -package chunk - -import ( - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/memory" - "github.com/stretchr/testify/require" -) - -func TestSpillActionDeadLock(t *testing.T) { - // Maybe get deadlock if we use two RLock in one goroutine, for oom-action call stack. - // Now the implement avoids the situation. - // Goroutine 1: rc.Add() (RLock) -> list.Add() -> tracker.Consume() -> SpillDiskAction -> rc.AlreadySpilledSafeForTest() (RLock) - // Goroutine 2: ------------------> SpillDiskAction -> new Goroutine to spill -> ------------------ - // new Goroutine created by 2: ---> rc.SpillToDisk (Lock) - // In golang, RLock will be blocked after try to get Lock. So it will cause deadlock. - require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock", "return(true)")) - defer func() { - require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock")) - }() - sz := 4 - fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} - rc := NewRowContainer(fields, sz) - - chk := NewChunkWithCapacity(fields, sz) - for i := 0; i < sz; i++ { - chk.AppendInt64(0, int64(i)) - } - var tracker *memory.Tracker - var err error - tracker = rc.GetMemTracker() - tracker.SetBytesLimit(1) - ac := rc.ActionSpillForTest() - tracker.FallbackOldAndSetNewAction(ac) - require.False(t, rc.AlreadySpilledSafeForTest()) - go func() { - time.Sleep(200 * time.Millisecond) - ac.Action(tracker) - }() - err = rc.Add(chk) - require.NoError(t, err) - rc.actionSpill.WaitForTest() - require.True(t, rc.AlreadySpilledSafeForTest()) -} - -func TestActionBlocked(t *testing.T) { - sz := 4 - fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} - rc := NewRowContainer(fields, sz) - - chk := NewChunkWithCapacity(fields, sz) - for i := 0; i < sz; i++ { - chk.AppendInt64(0, int64(i)) - } - var tracker *memory.Tracker - var err error - // Case 1, test Broadcast in Action. - tracker = rc.GetMemTracker() - tracker.SetBytesLimit(1450) - ac := rc.ActionSpill() - tracker.FallbackOldAndSetNewAction(ac) - for i := 0; i < 10; i++ { - err = rc.Add(chk) - require.NoError(t, err) - } - - ac.cond.L.Lock() - for ac.cond.status == notSpilled || - ac.cond.status == spilling { - ac.cond.Wait() - } - ac.cond.L.Unlock() - ac.cond.L.Lock() - require.Equal(t, spilledYet, ac.cond.status) - ac.cond.L.Unlock() - require.Equal(t, int64(0), tracker.BytesConsumed()) - require.Greater(t, tracker.MaxConsumed(), int64(0)) - require.Greater(t, rc.GetDiskTracker().BytesConsumed(), int64(0)) - - // Case 2, test Action will block when spilling. - rc = NewRowContainer(fields, sz) - tracker = rc.GetMemTracker() - ac = rc.ActionSpill() - starttime := time.Now() - ac.setStatus(spilling) - go func() { - time.Sleep(200 * time.Millisecond) - ac.setStatus(spilledYet) - ac.cond.Broadcast() - }() - ac.Action(tracker) - require.GreaterOrEqual(t, time.Since(starttime), 200*time.Millisecond) -} diff --git a/util/chunk/row_container_test.go b/util/chunk/row_container_test.go index 553863ddf3141..2bf8ef26644bc 100644 --- a/util/chunk/row_container_test.go +++ b/util/chunk/row_container_test.go @@ -16,7 +16,9 @@ package chunk import ( "testing" + "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/memory" @@ -216,3 +218,88 @@ func TestRowContainerResetAndAction(t *testing.T) { rc.actionSpill.WaitForTest() require.Greater(t, rc.GetDiskTracker().BytesConsumed(), int64(0)) } + +func TestSpillActionDeadLock(t *testing.T) { + // Maybe get deadlock if we use two RLock in one goroutine, for oom-action call stack. + // Now the implement avoids the situation. + // Goroutine 1: rc.Add() (RLock) -> list.Add() -> tracker.Consume() -> SpillDiskAction -> rc.AlreadySpilledSafeForTest() (RLock) + // Goroutine 2: ------------------> SpillDiskAction -> new Goroutine to spill -> ------------------ + // new Goroutine created by 2: ---> rc.SpillToDisk (Lock) + // In golang, RLock will be blocked after try to get Lock. So it will cause deadlock. + require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock", "return(true)")) + defer func() { + require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock")) + }() + sz := 4 + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} + rc := NewRowContainer(fields, sz) + + chk := NewChunkWithCapacity(fields, sz) + for i := 0; i < sz; i++ { + chk.AppendInt64(0, int64(i)) + } + var tracker *memory.Tracker + var err error + tracker = rc.GetMemTracker() + tracker.SetBytesLimit(1) + ac := rc.ActionSpillForTest() + tracker.FallbackOldAndSetNewAction(ac) + require.False(t, rc.AlreadySpilledSafeForTest()) + go func() { + time.Sleep(200 * time.Millisecond) + ac.Action(tracker) + }() + err = rc.Add(chk) + require.NoError(t, err) + rc.actionSpill.WaitForTest() + require.True(t, rc.AlreadySpilledSafeForTest()) +} + +func TestActionBlocked(t *testing.T) { + sz := 4 + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} + rc := NewRowContainer(fields, sz) + + chk := NewChunkWithCapacity(fields, sz) + for i := 0; i < sz; i++ { + chk.AppendInt64(0, int64(i)) + } + var tracker *memory.Tracker + var err error + // Case 1, test Broadcast in Action. + tracker = rc.GetMemTracker() + tracker.SetBytesLimit(1450) + ac := rc.ActionSpill() + tracker.FallbackOldAndSetNewAction(ac) + for i := 0; i < 10; i++ { + err = rc.Add(chk) + require.NoError(t, err) + } + + ac.cond.L.Lock() + for ac.cond.status == notSpilled || + ac.cond.status == spilling { + ac.cond.Wait() + } + ac.cond.L.Unlock() + ac.cond.L.Lock() + require.Equal(t, spilledYet, ac.cond.status) + ac.cond.L.Unlock() + require.Equal(t, int64(0), tracker.BytesConsumed()) + require.Greater(t, tracker.MaxConsumed(), int64(0)) + require.Greater(t, rc.GetDiskTracker().BytesConsumed(), int64(0)) + + // Case 2, test Action will block when spilling. + rc = NewRowContainer(fields, sz) + tracker = rc.GetMemTracker() + ac = rc.ActionSpill() + starttime := time.Now() + ac.setStatus(spilling) + go func() { + time.Sleep(200 * time.Millisecond) + ac.setStatus(spilledYet) + ac.cond.Broadcast() + }() + ac.Action(tracker) + require.GreaterOrEqual(t, time.Since(starttime), 200*time.Millisecond) +} diff --git a/util/ranger/ranger_serial_test.go b/util/ranger/ranger_serial_test.go deleted file mode 100644 index 49cfc3638d060..0000000000000 --- a/util/ranger/ranger_serial_test.go +++ /dev/null @@ -1,374 +0,0 @@ -// Copyright 2021 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. - -package ranger_test - -import ( - "context" - "fmt" - "testing" - - "github.com/pingcap/tidb/expression" - plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/ranger" - "github.com/stretchr/testify/require" -) - -func TestIndexRange(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec(` -create table t( - a varchar(50), - b int, - c double, - d varchar(10), - e binary(10), - f varchar(10) collate utf8mb4_general_ci, - g enum('A','B','C') collate utf8mb4_general_ci, - index idx_ab(a(50), b), - index idx_cb(c, a), - index idx_d(d(2)), - index idx_e(e(2)), - index idx_f(f), - index idx_de(d(2), e), - index idx_g(g) -)`) - - tests := []struct { - indexPos int - exprStr string - accessConds string - filterConds string - resultStr string - }{ - { - indexPos: 0, - exprStr: `a LIKE 'abc%'`, - accessConds: `[like(test.t.a, abc%, 92)]`, - filterConds: "[]", - resultStr: "[[\"abc\",\"abd\")]", - }, - { - indexPos: 0, - exprStr: "a LIKE 'abc_'", - accessConds: "[like(test.t.a, abc_, 92)]", - filterConds: "[like(test.t.a, abc_, 92)]", - resultStr: "[(\"abc\",\"abd\")]", - }, - { - indexPos: 0, - exprStr: "a LIKE 'abc'", - accessConds: "[like(test.t.a, abc, 92)]", - filterConds: "[]", - resultStr: "[[\"abc\",\"abc\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE "ab\_c"`, - accessConds: "[like(test.t.a, ab\\_c, 92)]", - filterConds: "[]", - resultStr: "[[\"ab_c\",\"ab_c\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE '%'`, - accessConds: "[]", - filterConds: `[like(test.t.a, %, 92)]`, - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 0, - exprStr: `a LIKE '\%a'`, - accessConds: "[like(test.t.a, \\%a, 92)]", - filterConds: "[]", - resultStr: `[["%a","%a"]]`, - }, - { - indexPos: 0, - exprStr: `a LIKE "\\"`, - accessConds: "[like(test.t.a, \\, 92)]", - filterConds: "[]", - resultStr: "[[\"\\\",\"\\\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE "\\\\a%"`, - accessConds: `[like(test.t.a, \\a%, 92)]`, - filterConds: "[]", - resultStr: "[[\"\\a\",\"\\b\")]", - }, - { - indexPos: 0, - exprStr: `a > NULL`, - accessConds: "[gt(test.t.a, )]", - filterConds: "[]", - resultStr: `[]`, - }, - { - indexPos: 0, - exprStr: `a = 'a' and b in (1, 2, 3)`, - accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, 3)]", - filterConds: "[]", - resultStr: "[[\"a\" 1,\"a\" 1] [\"a\" 2,\"a\" 2] [\"a\" 3,\"a\" 3]]", - }, - { - indexPos: 0, - exprStr: `a = 'a' and b not in (1, 2, 3)`, - accessConds: "[eq(test.t.a, a) not(in(test.t.b, 1, 2, 3))]", - filterConds: "[]", - resultStr: "[(\"a\" NULL,\"a\" 1) (\"a\" 3,\"a\" +inf]]", - }, - { - indexPos: 0, - exprStr: `a in ('a') and b in ('1', 2.0, NULL)`, - accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, )]", - filterConds: "[]", - resultStr: `[["a" 1,"a" 1] ["a" 2,"a" 2]]`, - }, - { - indexPos: 1, - exprStr: `c in ('1.1', 1, 1.1) and a in ('1', 'a', NULL)`, - accessConds: "[in(test.t.c, 1.1, 1, 1.1) in(test.t.a, 1, a, )]", - filterConds: "[]", - resultStr: "[[1 \"1\",1 \"1\"] [1 \"a\",1 \"a\"] [1.1 \"1\",1.1 \"1\"] [1.1 \"a\",1.1 \"a\"]]", - }, - { - indexPos: 1, - exprStr: "c in (1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)", - accessConds: "[in(test.t.c, 1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)]", - filterConds: "[]", - resultStr: "[[1,1] [2,2] [3,3] [4,4]]", - }, - { - indexPos: 1, - exprStr: "c not in (1, 2, 3)", - accessConds: "[not(in(test.t.c, 1, 2, 3))]", - filterConds: "[]", - resultStr: "[(NULL,1) (1,2) (2,3) (3,+inf]]", - }, - { - indexPos: 1, - exprStr: "c in (1, 2) and c in (1, 3)", - accessConds: "[eq(test.t.c, 1)]", - filterConds: "[]", - resultStr: "[[1,1]]", - }, - { - indexPos: 1, - exprStr: "c = 1 and c = 2", - accessConds: "[]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "a in (NULL)", - accessConds: "[eq(test.t.a, )]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "a not in (NULL, '1', '2', '3')", - accessConds: "[not(in(test.t.a, , 1, 2, 3))]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL, '1', '2', '3') and a > '2')", - accessConds: "[or(in(test.t.a, , 1, 2, 3), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[[-inf,\"2\"] [\"3\",\"3\"]]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL) and a > '2')", - accessConds: "[or(eq(test.t.a, ), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[[-inf,\"2\"]]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL) or a > '2')", - accessConds: "[and(eq(test.t.a, ), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "(a > 'b' and a < 'bbb') or (a < 'cb' and a > 'a')", - accessConds: "[or(and(gt(test.t.a, b), lt(test.t.a, bbb)), and(lt(test.t.a, cb), gt(test.t.a, a)))]", - filterConds: "[]", - resultStr: "[(\"a\",\"cb\")]", - }, - { - indexPos: 0, - exprStr: "(a > 'a' and a < 'b') or (a >= 'b' and a < 'c')", - accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[]", - resultStr: "[(\"a\",\"c\")]", - }, - { - indexPos: 0, - exprStr: "(a > 'a' and a < 'b' and b < 1) or (a >= 'b' and a < 'c')", - accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[or(and(and(gt(test.t.a, a), lt(test.t.a, b)), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - resultStr: "[(\"a\",\"c\")]", - }, - { - indexPos: 0, - exprStr: "(a in ('a', 'b') and b < 1) or (a >= 'b' and a < 'c')", - accessConds: "[or(and(in(test.t.a, a, b), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[]", - resultStr: `[["a" -inf,"a" 1) ["b","c")]`, - }, - { - indexPos: 0, - exprStr: "(a > 'a') or (c > 1)", - accessConds: "[]", - filterConds: "[or(gt(test.t.a, a), gt(test.t.c, 1))]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 2, - exprStr: `d = "你好啊"`, - accessConds: "[eq(test.t.d, 你好啊)]", - filterConds: "[eq(test.t.d, 你好啊)]", - resultStr: "[[\"你好\",\"你好\"]]", - }, - { - indexPos: 3, - exprStr: `e = "你好啊"`, - accessConds: "[eq(test.t.e, 你好啊)]", - filterConds: "[eq(test.t.e, 你好啊)]", - resultStr: "[[0xE4BD,0xE4BD]]", - }, - { - indexPos: 2, - exprStr: `d in ("你好啊", "再见")`, - accessConds: "[in(test.t.d, 你好啊, 再见)]", - filterConds: "[in(test.t.d, 你好啊, 再见)]", - resultStr: "[[\"你好\",\"你好\"] [\"再见\",\"再见\"]]", - }, - { - indexPos: 2, - exprStr: `d not in ("你好啊")`, - accessConds: "[]", - filterConds: "[ne(test.t.d, 你好啊)]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 2, - exprStr: `d < "你好" || d > "你好"`, - accessConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", - filterConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", - resultStr: "[[-inf,+inf]]", - }, - { - indexPos: 2, - exprStr: `not(d < "你好" || d > "你好")`, - accessConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", - filterConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", - resultStr: "[[\"你好\",\"你好\"]]", - }, - { - indexPos: 4, - exprStr: "f >= 'a' and f <= 'B'", - accessConds: "[ge(test.t.f, a) le(test.t.f, B)]", - filterConds: "[]", - resultStr: "[[\"a\",\"B\"]]", - }, - { - indexPos: 4, - exprStr: "f in ('a', 'B')", - accessConds: "[in(test.t.f, a, B)]", - filterConds: "[]", - resultStr: "[[\"a\",\"a\"] [\"B\",\"B\"]]", - }, - { - indexPos: 4, - exprStr: "f = 'a' and f = 'B' collate utf8mb4_bin", - accessConds: "[eq(test.t.f, a)]", - filterConds: "[eq(test.t.f, B)]", - resultStr: "[[\"a\",\"a\"]]", - }, - { - indexPos: 4, - exprStr: "f like '@%' collate utf8mb4_bin", - accessConds: "[]", - filterConds: "[like(test.t.f, @%, 92)]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 5, - exprStr: "d in ('aab', 'aac') and e = 'a'", - accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", - filterConds: "[in(test.t.d, aab, aac)]", - resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", - }, - { - indexPos: 6, - exprStr: "g = 'a'", - accessConds: "[eq(test.t.g, a)]", - filterConds: "[]", - resultStr: "[[\"A\",\"A\"]]", - }, - } - - collate.SetNewCollationEnabledForTest(true) - defer func() { collate.SetNewCollationEnabledForTest(false) }() - ctx := context.Background() - for _, tt := range tests { - t.Run(tt.exprStr, func(t *testing.T) { - sql := "select * from t where " + tt.exprStr - sctx := testKit.Session().(sessionctx.Context) - stmts, err := session.Parse(sctx, sql) - require.NoError(t, err) - require.Len(t, stmts, 1) - ret := &plannercore.PreprocessorReturn{} - err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) - require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) - require.NoError(t, err) - selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) - tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() - require.NotNil(t, selection) - conds := make([]expression.Expression, len(selection.Conditions)) - for i, cond := range selection.Conditions { - conds[i] = expression.PushDownNot(sctx, cond) - } - cols, lengths := expression.IndexInfo2PrefixCols(tbl.Columns, selection.Schema().Columns, tbl.Indices[tt.indexPos]) - require.NotNil(t, cols) - res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, conds, cols, lengths) - require.NoError(t, err) - require.Equal(t, tt.accessConds, fmt.Sprintf("%s", res.AccessConds)) - require.Equal(t, tt.filterConds, fmt.Sprintf("%s", res.RemainedConds)) - got := fmt.Sprintf("%v", res.Ranges) - require.Equal(t, tt.resultStr, got) - }) - } -} diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 03de4f42c2eea..6c6ad0bf9abdb 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" ) @@ -1420,3 +1421,347 @@ func TestPrefixIndexAppendPointRanges(t *testing.T) { testKit.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) } } + +func TestIndexRange(t *testing.T) { + dom, store, err := newDomainStoreWithBootstrap(t) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + require.NoError(t, err) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec(` +create table t( + a varchar(50), + b int, + c double, + d varchar(10), + e binary(10), + f varchar(10) collate utf8mb4_general_ci, + g enum('A','B','C') collate utf8mb4_general_ci, + index idx_ab(a(50), b), + index idx_cb(c, a), + index idx_d(d(2)), + index idx_e(e(2)), + index idx_f(f), + index idx_de(d(2), e), + index idx_g(g) +)`) + + tests := []struct { + indexPos int + exprStr string + accessConds string + filterConds string + resultStr string + }{ + { + indexPos: 0, + exprStr: `a LIKE 'abc%'`, + accessConds: `[like(test.t.a, abc%, 92)]`, + filterConds: "[]", + resultStr: "[[\"abc\",\"abd\")]", + }, + { + indexPos: 0, + exprStr: "a LIKE 'abc_'", + accessConds: "[like(test.t.a, abc_, 92)]", + filterConds: "[like(test.t.a, abc_, 92)]", + resultStr: "[(\"abc\",\"abd\")]", + }, + { + indexPos: 0, + exprStr: "a LIKE 'abc'", + accessConds: "[like(test.t.a, abc, 92)]", + filterConds: "[]", + resultStr: "[[\"abc\",\"abc\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE "ab\_c"`, + accessConds: "[like(test.t.a, ab\\_c, 92)]", + filterConds: "[]", + resultStr: "[[\"ab_c\",\"ab_c\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE '%'`, + accessConds: "[]", + filterConds: `[like(test.t.a, %, 92)]`, + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 0, + exprStr: `a LIKE '\%a'`, + accessConds: "[like(test.t.a, \\%a, 92)]", + filterConds: "[]", + resultStr: `[["%a","%a"]]`, + }, + { + indexPos: 0, + exprStr: `a LIKE "\\"`, + accessConds: "[like(test.t.a, \\, 92)]", + filterConds: "[]", + resultStr: "[[\"\\\",\"\\\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE "\\\\a%"`, + accessConds: `[like(test.t.a, \\a%, 92)]`, + filterConds: "[]", + resultStr: "[[\"\\a\",\"\\b\")]", + }, + { + indexPos: 0, + exprStr: `a > NULL`, + accessConds: "[gt(test.t.a, )]", + filterConds: "[]", + resultStr: `[]`, + }, + { + indexPos: 0, + exprStr: `a = 'a' and b in (1, 2, 3)`, + accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, 3)]", + filterConds: "[]", + resultStr: "[[\"a\" 1,\"a\" 1] [\"a\" 2,\"a\" 2] [\"a\" 3,\"a\" 3]]", + }, + { + indexPos: 0, + exprStr: `a = 'a' and b not in (1, 2, 3)`, + accessConds: "[eq(test.t.a, a) not(in(test.t.b, 1, 2, 3))]", + filterConds: "[]", + resultStr: "[(\"a\" NULL,\"a\" 1) (\"a\" 3,\"a\" +inf]]", + }, + { + indexPos: 0, + exprStr: `a in ('a') and b in ('1', 2.0, NULL)`, + accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, )]", + filterConds: "[]", + resultStr: `[["a" 1,"a" 1] ["a" 2,"a" 2]]`, + }, + { + indexPos: 1, + exprStr: `c in ('1.1', 1, 1.1) and a in ('1', 'a', NULL)`, + accessConds: "[in(test.t.c, 1.1, 1, 1.1) in(test.t.a, 1, a, )]", + filterConds: "[]", + resultStr: "[[1 \"1\",1 \"1\"] [1 \"a\",1 \"a\"] [1.1 \"1\",1.1 \"1\"] [1.1 \"a\",1.1 \"a\"]]", + }, + { + indexPos: 1, + exprStr: "c in (1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)", + accessConds: "[in(test.t.c, 1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)]", + filterConds: "[]", + resultStr: "[[1,1] [2,2] [3,3] [4,4]]", + }, + { + indexPos: 1, + exprStr: "c not in (1, 2, 3)", + accessConds: "[not(in(test.t.c, 1, 2, 3))]", + filterConds: "[]", + resultStr: "[(NULL,1) (1,2) (2,3) (3,+inf]]", + }, + { + indexPos: 1, + exprStr: "c in (1, 2) and c in (1, 3)", + accessConds: "[eq(test.t.c, 1)]", + filterConds: "[]", + resultStr: "[[1,1]]", + }, + { + indexPos: 1, + exprStr: "c = 1 and c = 2", + accessConds: "[]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "a in (NULL)", + accessConds: "[eq(test.t.a, )]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "a not in (NULL, '1', '2', '3')", + accessConds: "[not(in(test.t.a, , 1, 2, 3))]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL, '1', '2', '3') and a > '2')", + accessConds: "[or(in(test.t.a, , 1, 2, 3), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[[-inf,\"2\"] [\"3\",\"3\"]]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL) and a > '2')", + accessConds: "[or(eq(test.t.a, ), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[[-inf,\"2\"]]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL) or a > '2')", + accessConds: "[and(eq(test.t.a, ), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "(a > 'b' and a < 'bbb') or (a < 'cb' and a > 'a')", + accessConds: "[or(and(gt(test.t.a, b), lt(test.t.a, bbb)), and(lt(test.t.a, cb), gt(test.t.a, a)))]", + filterConds: "[]", + resultStr: "[(\"a\",\"cb\")]", + }, + { + indexPos: 0, + exprStr: "(a > 'a' and a < 'b') or (a >= 'b' and a < 'c')", + accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[]", + resultStr: "[(\"a\",\"c\")]", + }, + { + indexPos: 0, + exprStr: "(a > 'a' and a < 'b' and b < 1) or (a >= 'b' and a < 'c')", + accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[or(and(and(gt(test.t.a, a), lt(test.t.a, b)), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + resultStr: "[(\"a\",\"c\")]", + }, + { + indexPos: 0, + exprStr: "(a in ('a', 'b') and b < 1) or (a >= 'b' and a < 'c')", + accessConds: "[or(and(in(test.t.a, a, b), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[]", + resultStr: `[["a" -inf,"a" 1) ["b","c")]`, + }, + { + indexPos: 0, + exprStr: "(a > 'a') or (c > 1)", + accessConds: "[]", + filterConds: "[or(gt(test.t.a, a), gt(test.t.c, 1))]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 2, + exprStr: `d = "你好啊"`, + accessConds: "[eq(test.t.d, 你好啊)]", + filterConds: "[eq(test.t.d, 你好啊)]", + resultStr: "[[\"你好\",\"你好\"]]", + }, + { + indexPos: 3, + exprStr: `e = "你好啊"`, + accessConds: "[eq(test.t.e, 你好啊)]", + filterConds: "[eq(test.t.e, 你好啊)]", + resultStr: "[[0xE4BD,0xE4BD]]", + }, + { + indexPos: 2, + exprStr: `d in ("你好啊", "再见")`, + accessConds: "[in(test.t.d, 你好啊, 再见)]", + filterConds: "[in(test.t.d, 你好啊, 再见)]", + resultStr: "[[\"你好\",\"你好\"] [\"再见\",\"再见\"]]", + }, + { + indexPos: 2, + exprStr: `d not in ("你好啊")`, + accessConds: "[]", + filterConds: "[ne(test.t.d, 你好啊)]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 2, + exprStr: `d < "你好" || d > "你好"`, + accessConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", + filterConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", + resultStr: "[[-inf,+inf]]", + }, + { + indexPos: 2, + exprStr: `not(d < "你好" || d > "你好")`, + accessConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", + filterConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", + resultStr: "[[\"你好\",\"你好\"]]", + }, + { + indexPos: 4, + exprStr: "f >= 'a' and f <= 'B'", + accessConds: "[ge(test.t.f, a) le(test.t.f, B)]", + filterConds: "[]", + resultStr: "[[\"a\",\"B\"]]", + }, + { + indexPos: 4, + exprStr: "f in ('a', 'B')", + accessConds: "[in(test.t.f, a, B)]", + filterConds: "[]", + resultStr: "[[\"a\",\"a\"] [\"B\",\"B\"]]", + }, + { + indexPos: 4, + exprStr: "f = 'a' and f = 'B' collate utf8mb4_bin", + accessConds: "[eq(test.t.f, a)]", + filterConds: "[eq(test.t.f, B)]", + resultStr: "[[\"a\",\"a\"]]", + }, + { + indexPos: 4, + exprStr: "f like '@%' collate utf8mb4_bin", + accessConds: "[]", + filterConds: "[like(test.t.f, @%, 92)]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 5, + exprStr: "d in ('aab', 'aac') and e = 'a'", + accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", + filterConds: "[in(test.t.d, aab, aac)]", + resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", + }, + { + indexPos: 6, + exprStr: "g = 'a'", + accessConds: "[eq(test.t.g, a)]", + filterConds: "[]", + resultStr: "[[\"A\",\"A\"]]", + }, + } + + collate.SetNewCollationEnabledForTest(true) + defer func() { collate.SetNewCollationEnabledForTest(false) }() + ctx := context.Background() + for _, tt := range tests { + t.Run(tt.exprStr, func(t *testing.T) { + sql := "select * from t where " + tt.exprStr + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, sql) + require.NoError(t, err) + require.Len(t, stmts, 1) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoError(t, err) + selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() + require.NotNil(t, selection) + conds := make([]expression.Expression, len(selection.Conditions)) + for i, cond := range selection.Conditions { + conds[i] = expression.PushDownNot(sctx, cond) + } + cols, lengths := expression.IndexInfo2PrefixCols(tbl.Columns, selection.Schema().Columns, tbl.Indices[tt.indexPos]) + require.NotNil(t, cols) + res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, conds, cols, lengths) + require.NoError(t, err) + require.Equal(t, tt.accessConds, fmt.Sprintf("%s", res.AccessConds)) + require.Equal(t, tt.filterConds, fmt.Sprintf("%s", res.RemainedConds)) + got := fmt.Sprintf("%v", res.Ranges) + require.Equal(t, tt.resultStr, got) + }) + } +} From f1c3f60f45fc5604fbacb12d5916a403776f2063 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Fri, 24 Dec 2021 16:48:45 +0800 Subject: [PATCH 2/5] flatten the json output (#30905) Signed-off-by: yisaer Co-authored-by: Ti Chi Robot --- util/tracing/opt_trace.go | 57 +++++++++++++++--- util/tracing/opt_trace_test.go | 105 +++++++++++++++++++++++++++++++++ 2 files changed, 154 insertions(+), 8 deletions(-) create mode 100644 util/tracing/opt_trace_test.go diff --git a/util/tracing/opt_trace.go b/util/tracing/opt_trace.go index e3d243b16ed33..506db98ee0d8d 100644 --- a/util/tracing/opt_trace.go +++ b/util/tracing/opt_trace.go @@ -16,17 +16,17 @@ package tracing // LogicalPlanTrace indicates for the LogicalPlan trace information type LogicalPlanTrace struct { - ID int `json:"id"` - TP string `json:"type"` - Children []*LogicalPlanTrace `json:"children"` + ID int + TP string + Children []*LogicalPlanTrace // ExplainInfo should be implemented by each implemented LogicalPlan - ExplainInfo string `json:"info"` + ExplainInfo string } // LogicalOptimizeTracer indicates the trace for the whole logicalOptimize processing type LogicalOptimizeTracer struct { - FinalLogicalPlan *LogicalPlanTrace `json:"final"` + FinalLogicalPlan []FlattenLogicalPlanTrace `json:"final"` Steps []*LogicalRuleOptimizeTracer `json:"steps"` // curRuleTracer indicates the current rule Tracer during optimize by rule curRuleTracer *LogicalRuleOptimizeTracer @@ -53,14 +53,14 @@ func (tracer *LogicalOptimizeTracer) AppendRuleTracerStepToCurrent(id int, tp, r // RecordFinalLogicalPlan add plan trace after logical optimize func (tracer *LogicalOptimizeTracer) RecordFinalLogicalPlan(final *LogicalPlanTrace) { - tracer.FinalLogicalPlan = final + tracer.FinalLogicalPlan = toFlattenLogicalPlanTrace(final) } // LogicalRuleOptimizeTracer indicates the trace for the LogicalPlan tree before and after // logical rule optimize type LogicalRuleOptimizeTracer struct { Index int `json:"index"` - Before *LogicalPlanTrace `json:"before"` + Before []FlattenLogicalPlanTrace `json:"before"` RuleName string `json:"name"` Steps []LogicalRuleOptimizeTraceStep `json:"steps"` } @@ -69,7 +69,7 @@ type LogicalRuleOptimizeTracer struct { func buildLogicalRuleOptimizeTracerBeforeOptimize(index int, name string, before *LogicalPlanTrace) *LogicalRuleOptimizeTracer { return &LogicalRuleOptimizeTracer{ Index: index, - Before: before, + Before: toFlattenLogicalPlanTrace(before), RuleName: name, Steps: make([]LogicalRuleOptimizeTraceStep, 0), } @@ -85,6 +85,47 @@ type LogicalRuleOptimizeTraceStep struct { Index int `json:"index"` } +// FlattenLogicalPlanTrace indicates the flatten LogicalPlanTrace +type FlattenLogicalPlanTrace struct { + ID int `json:"id"` + TP string `json:"type"` + Children []int `json:"children"` + + // ExplainInfo should be implemented by each implemented LogicalPlan + ExplainInfo string `json:"info"` +} + +// toFlattenLogicalPlanTrace transform LogicalPlanTrace into FlattenLogicalPlanTrace +func toFlattenLogicalPlanTrace(root *LogicalPlanTrace) []FlattenLogicalPlanTrace { + wrapper := &flattenWrapper{flatten: make([]FlattenLogicalPlanTrace, 0)} + flattenLogicalPlanTrace(root, wrapper) + return wrapper.flatten +} + +type flattenWrapper struct { + flatten []FlattenLogicalPlanTrace +} + +func flattenLogicalPlanTrace(node *LogicalPlanTrace, wrapper *flattenWrapper) { + flattenNode := FlattenLogicalPlanTrace{ + ID: node.ID, + TP: node.TP, + ExplainInfo: node.ExplainInfo, + Children: make([]int, 0), + } + if len(node.Children) < 1 { + wrapper.flatten = append(wrapper.flatten, flattenNode) + return + } + for _, child := range node.Children { + flattenNode.Children = append(flattenNode.Children, child.ID) + } + for _, child := range node.Children { + flattenLogicalPlanTrace(child, wrapper) + } + wrapper.flatten = append(wrapper.flatten, flattenNode) +} + // CETraceRecord records an expression and related cardinality estimation result. type CETraceRecord struct { TableID int64 `json:"-"` diff --git a/util/tracing/opt_trace_test.go b/util/tracing/opt_trace_test.go new file mode 100644 index 0000000000000..800749429ec0d --- /dev/null +++ b/util/tracing/opt_trace_test.go @@ -0,0 +1,105 @@ +// Copyright 2021 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. + +package tracing + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestFlattenLogicalPlanTrace(t *testing.T) { + root1 := &LogicalPlanTrace{ + ID: 1, + TP: "foo1", + ExplainInfo: "bar1", + Children: []*LogicalPlanTrace{ + { + ID: 2, + TP: "foo2", + ExplainInfo: "bar2", + Children: nil, + }, + }, + } + root2 := &LogicalPlanTrace{ + ID: 1, + TP: "foo1", + ExplainInfo: "bar1", + Children: []*LogicalPlanTrace{ + { + ID: 2, + TP: "foo2", + ExplainInfo: "bar2", + Children: nil, + }, + { + ID: 3, + TP: "foo3", + ExplainInfo: "bar3", + Children: []*LogicalPlanTrace{ + { + ID: 4, + TP: "foo4", + ExplainInfo: "bar4", + Children: nil, + }, + }, + }, + }, + } + expect1 := []FlattenLogicalPlanTrace{ + { + ID: 2, + TP: "foo2", + ExplainInfo: "bar2", + Children: []int{}, + }, + { + ID: 1, + TP: "foo1", + ExplainInfo: "bar1", + Children: []int{2}, + }, + } + expect2 := []FlattenLogicalPlanTrace{ + { + ID: 2, + TP: "foo2", + ExplainInfo: "bar2", + Children: []int{}, + }, + { + ID: 4, + TP: "foo4", + ExplainInfo: "bar4", + Children: []int{}, + }, + { + ID: 3, + TP: "foo3", + ExplainInfo: "bar3", + Children: []int{4}, + }, + { + ID: 1, + TP: "foo1", + ExplainInfo: "bar1", + Children: []int{2, 3}, + }, + } + require.EqualValues(t, toFlattenLogicalPlanTrace(root1), expect1) + require.EqualValues(t, toFlattenLogicalPlanTrace(root2), expect2) +} From 391a8c04fc42216aa5f77adba31c9045ce87e957 Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 24 Dec 2021 17:03:48 +0800 Subject: [PATCH 3/5] test: control log level with environment variables (#30871) --- Makefile | 33 ++++------------- Makefile.common | 3 -- bindinfo/main_test.go | 2 +- br/pkg/conn/main_test.go | 2 +- br/pkg/kv/main_test.go | 2 +- br/pkg/metautil/main_test.go | 2 +- br/pkg/pdutil/main_test.go | 2 +- br/pkg/rtree/main_test.go | 2 +- br/pkg/summary/main_test.go | 2 +- br/pkg/trace/main_test.go | 2 +- br/pkg/utils/main_test.go | 2 +- cmd/ddltest/main_test.go | 2 +- config/main_test.go | 2 +- ddl/failtest/main_test.go | 2 +- ddl/label/main_test.go | 2 +- ddl/util/main_test.go | 2 +- distsql/main_test.go | 2 +- domain/globalconfigsync/globalconfig_test.go | 2 +- domain/infosync/info_test.go | 2 +- domain/main_test.go | 2 +- errno/main_test.go | 2 +- executor/aggfuncs/main_test.go | 2 +- executor/main_test.go | 2 +- executor/oomtest/oom_test.go | 2 +- executor/seqtest/main_test.go | 2 +- expression/aggregation/main_test.go | 2 +- expression/main_test.go | 2 +- infoschema/main_test.go | 2 +- infoschema/perfschema/main_test.go | 2 +- kv/main_test.go | 2 +- meta/autoid/main_test.go | 2 +- meta/main_test.go | 2 +- metrics/main_test.go | 2 +- owner/main_test.go | 2 +- planner/cascades/main_test.go | 2 +- planner/core/main_test.go | 2 +- planner/implementation/main_test.go | 2 +- planner/memo/main_test.go | 2 +- planner/util/main_test.go | 2 +- plugin/conn_ip_example/main_test.go | 2 +- plugin/main_test.go | 2 +- privilege/privileges/main_test.go | 2 +- server/main_test.go | 2 +- session/main_test.go | 2 +- sessionctx/binloginfo/main_test.go | 2 +- sessionctx/main_test.go | 2 +- sessionctx/stmtctx/main_test.go | 2 +- sessionctx/variable/main_test.go | 2 +- sessiontxn/txn_context_test.go | 2 +- statistics/handle/main_test.go | 2 +- statistics/main_test.go | 2 +- store/copr/main_test.go | 2 +- store/driver/error/error_test.go | 2 +- store/driver/main_test.go | 2 +- store/driver/txn/main_test.go | 2 +- store/gcworker/main_test.go | 2 +- store/helper/main_test.go | 2 +- store/main_test.go | 2 +- store/mockstore/main_test.go | 2 +- store/mockstore/mockcopr/main_test.go | 2 +- .../unistore/cophandler/main_test.go | 2 +- .../mockstore/unistore/lockstore/main_test.go | 2 +- store/mockstore/unistore/main_test.go | 2 +- store/mockstore/unistore/tikv/main_test.go | 2 +- .../unistore/util/lockwaiter/main_test.go | 2 +- structure/main_test.go | 2 +- table/main_test.go | 2 +- table/tables/main_test.go | 2 +- table/temptable/main_test.go | 2 +- tablecodec/main_test.go | 2 +- tablecodec/rowindexcodec/main_test.go | 2 +- telemetry/cte_test/cte_test.go | 2 +- telemetry/main_test.go | 2 +- tests/globalkilltest/main_test.go | 2 +- tests/graceshutdown/main_test.go | 2 +- tests/readonlytest/main_test.go | 2 +- tidb-server/main_test.go | 2 +- tools/check/go.mod | 2 - tools/check/go.sum | 4 -- types/json/main_test.go | 2 +- types/main_test.go | 2 +- types/parser_driver/main_test.go | 2 +- util/admin/main_test.go | 2 +- util/arena/main_test.go | 2 +- util/benchdaily/main_test.go | 2 +- util/bitmap/main_test.go | 2 +- util/checksum/main_test.go | 2 +- util/chunk/main_test.go | 2 +- util/codec/main_test.go | 2 +- util/collate/main_test.go | 2 +- util/cteutil/main_test.go | 2 +- util/dbterror/main_test.go | 2 +- util/deadlockhistory/main_test.go | 2 +- util/disjointset/main_test.go | 2 +- util/disk/main_test.go | 2 +- util/encrypt/main_test.go | 2 +- util/execdetails/main_test.go | 2 +- util/expensivequery/expensivequerey_test.go | 2 +- util/fastrand/main_test.go | 2 +- util/format/main_test.go | 2 +- util/generatedexpr/main_test.go | 2 +- util/hack/main_test.go | 2 +- util/keydecoder/main_test.go | 2 +- util/kvcache/main_test.go | 2 +- util/localpool/main_test.go | 2 +- util/logutil/main_test.go | 2 +- util/main_test.go | 2 +- util/math/main_test.go | 2 +- util/memory/main_test.go | 12 +----- util/mock/main_test.go | 2 +- util/mvmap/main_test.go | 2 +- util/parser/main_test.go | 2 +- util/plancodec/main_test.go | 2 +- util/printer/main_test.go | 2 +- util/profile/main_test.go | 2 +- util/ranger/main_test.go | 2 +- util/resourcegrouptag/main_test.go | 2 +- util/rowDecoder/main_test.go | 2 +- util/rowcodec/main_test.go | 2 +- util/selection/main_test.go | 2 +- util/sem/main_test.go | 2 +- util/set/main_test.go | 2 +- util/slice/main_test.go | 2 +- util/sqlexec/main_test.go | 2 +- util/stmtsummary/main_test.go | 2 +- util/stringutil/main_test.go | 2 +- util/sys/linux/main_test.go | 2 +- util/sys/storage/main_test.go | 2 +- util/systimemon/main_test.go | 2 +- util/testbridge/bridge.go | 37 +++++++++++++++++-- util/texttree/main_test.go | 2 +- util/timeutil/main_test.go | 2 +- util/topsql/main_test.go | 2 +- util/topsql/reporter/main_test.go | 2 +- util/topsql/stmtstats/main_test.go | 2 +- .../stmtstats/stmtstatstest/main_test.go | 2 +- util/topsql/tracecpu/main_test.go | 2 +- util/tracing/main_test.go | 2 +- util/vitess/main_test.go | 2 +- 139 files changed, 175 insertions(+), 182 deletions(-) diff --git a/Makefile b/Makefile index 30e01b2c1a1e1..e0ba263d192ab 100644 --- a/Makefile +++ b/Makefile @@ -136,24 +136,13 @@ gotest: failpoint-enable $(GOTEST) -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -timeout 20m -cover $(PACKAGES_TIDB_TESTS) -coverprofile=coverage.txt -check.p true > gotest.log || { $(FAILPOINT_DISABLE); cat 'gotest.log'; exit 1; } @$(FAILPOINT_DISABLE) -gotest_in_verify_ci_part_1: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml - @echo "Running gotest_in_verify_ci_part_1." +gotest_in_verify_ci: failpoint-enable tools/bin/gotestsum + @echo "Running gotest_in_verify_ci" @mkdir -p $(TEST_COVERAGE_DIR) - @export log_level=info; export TZ='Asia/Shanghai'; \ - CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) --race \ - -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \ - $(PACKAGES_TIDB_TESTS_EXPENSIVE) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } - tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/tidb-coverage.xml" - @$(FAILPOINT_DISABLE) - -gotest_in_verify_ci_part_2: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml - @echo "Running gotest_in_verify_ci_part_2." - @mkdir -p $(TEST_COVERAGE_DIR) - @export log_level=info; export TZ='Asia/Shanghai'; \ - CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) --race \ + @export TZ='Asia/Shanghai'; \ + CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) \ -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \ - $(PACKAGES_TIDB_TESTS_OTHERS) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } - tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/tidb-coverage.xml" + $(PACKAGES_TIDB_TESTS) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) race: failpoint-enable @@ -341,13 +330,12 @@ br_unit_test: $(GOTEST) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -tags leak $(ARGS) -coverprofile=coverage.txt || ( make failpoint-disable && exit 1 ) @make failpoint-disable br_unit_test_in_verify_ci: export ARGS=$$($(BR_PACKAGES)) -br_unit_test_in_verify_ci: tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml +br_unit_test_in_verify_ci: tools/bin/gotestsum @make failpoint-enable @export TZ='Asia/Shanghai'; @mkdir -p $(TEST_COVERAGE_DIR) CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/br-junit-report.xml" -- $(RACE_FLAG) -ldflags '$(LDFLAGS)' \ -tags leak $(ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/br_cov.unit_test.out" || ( make failpoint-disable && exit 1 ) - tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/br_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/br-coverage.xml" @make failpoint-disable br_integration_test: br_bins build_br build_for_br_integration_test @@ -407,11 +395,10 @@ dumpling_unit_test: failpoint-enable $(DUMPLING_GOTEST) $(RACE_FLAG) -coverprofile=coverage.txt -covermode=atomic -tags leak $(DUMPLING_ARGS) || ( make failpoint-disable && exit 1 ) @make failpoint-disable dumpling_unit_test_in_verify_ci: export DUMPLING_ARGS=$$($(DUMPLING_PACKAGES)) -dumpling_unit_test_in_verify_ci: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml +dumpling_unit_test_in_verify_ci: failpoint-enable tools/bin/gotestsum @mkdir -p $(TEST_COVERAGE_DIR) CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/dumpling-junit-report.xml" -- -tags leak $(DUMPLING_ARGS) \ $(RACE_FLAG) -coverprofile="$(TEST_COVERAGE_DIR)/dumpling_cov.unit_test.out" || ( make failpoint-disable && exit 1 ) - tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/dumpling_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/dumpling-coverage.xml" @make failpoint-disable dumpling_integration_test: dumpling_bins failpoint-enable build_dumpling @@ -435,9 +422,3 @@ dumpling_bins: tools/bin/gotestsum: tools/check/go.mod cd tools/check && $(GO) build -o ../bin/gotestsum gotest.tools/gotestsum - -tools/bin/gocov: tools/check/go.mod - cd tools/check && $(GO) build -o ../bin/gocov github.com/axw/gocov/gocov - -tools/bin/gocov-xml: tools/check/go.mod - cd tools/check && $(GO) build -o ../bin/gocov-xml github.com/AlekSi/gocov-xml diff --git a/Makefile.common b/Makefile.common index 2a8ea369521b1..8ea85d6a24694 100644 --- a/Makefile.common +++ b/Makefile.common @@ -47,11 +47,8 @@ MAC := "Darwin" PACKAGE_LIST := go list ./... PACKAGE_LIST_TIDB_TESTS := go list ./... | grep -vE "github.com\/pingcap\/tidb\/br|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/dumpling" -PACKAGE_LIST_TEST_OTHERS := go list ./... | grep -vE "github.com\/pingcap\/tidb\/br|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/dumpling|github.com\/pingcap\/tidb\/executor|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/ddl" PACKAGES ?= $$($(PACKAGE_LIST)) PACKAGES_TIDB_TESTS ?= $$($(PACKAGE_LIST_TIDB_TESTS)) -PACKAGES_TIDB_TESTS_EXPENSIVE ?= "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/ddl" -PACKAGES_TIDB_TESTS_OTHERS ?= $$($(PACKAGE_LIST_TEST_OTHERS)) PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||' PACKAGE_DIRECTORIES_TIDB_TESTS := $(PACKAGE_LIST_TIDB_TESTS) | sed 's|github.com/pingcap/$(PROJECT)/||' FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go") diff --git a/bindinfo/main_test.go b/bindinfo/main_test.go index 85151366ea0ef..6104329761360 100644 --- a/bindinfo/main_test.go +++ b/bindinfo/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/br/pkg/conn/main_test.go b/br/pkg/conn/main_test.go index b1299da7358de..7b46a892be79d 100644 --- a/br/pkg/conn/main_test.go +++ b/br/pkg/conn/main_test.go @@ -26,6 +26,6 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/br/pkg/kv/main_test.go b/br/pkg/kv/main_test.go index d7e28c807d792..72bbbcfbf4cb0 100644 --- a/br/pkg/kv/main_test.go +++ b/br/pkg/kv/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/br/pkg/metautil/main_test.go b/br/pkg/metautil/main_test.go index e73ef73e16f2d..44cb7f4a19097 100644 --- a/br/pkg/metautil/main_test.go +++ b/br/pkg/metautil/main_test.go @@ -25,6 +25,6 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/br/pkg/pdutil/main_test.go b/br/pkg/pdutil/main_test.go index 861c3921a3eb3..653d973bcd499 100644 --- a/br/pkg/pdutil/main_test.go +++ b/br/pkg/pdutil/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/br/pkg/rtree/main_test.go b/br/pkg/rtree/main_test.go index 85dc99665acaf..8d4ae4216a894 100644 --- a/br/pkg/rtree/main_test.go +++ b/br/pkg/rtree/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/br/pkg/summary/main_test.go b/br/pkg/summary/main_test.go index e1b89ff3d0a0f..77dfc58c09d57 100644 --- a/br/pkg/summary/main_test.go +++ b/br/pkg/summary/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/br/pkg/trace/main_test.go b/br/pkg/trace/main_test.go index f253ad281ecdb..adbee42408e38 100644 --- a/br/pkg/trace/main_test.go +++ b/br/pkg/trace/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/br/pkg/utils/main_test.go b/br/pkg/utils/main_test.go index 47d7fd9b63b21..09976b09ac199 100644 --- a/br/pkg/utils/main_test.go +++ b/br/pkg/utils/main_test.go @@ -25,6 +25,6 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/cmd/ddltest/main_test.go b/cmd/ddltest/main_test.go index 890a52a7f4ab8..810ba80b9a974 100644 --- a/cmd/ddltest/main_test.go +++ b/cmd/ddltest/main_test.go @@ -26,7 +26,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() err := logutil.InitLogger(&logutil.LogConfig{Config: zaplog.Config{Level: *logLevel}}) if err != nil { fmt.Fprint(os.Stderr, err.Error()) diff --git a/config/main_test.go b/config/main_test.go index 0ef0a65458301..847d12a0a12ce 100644 --- a/config/main_test.go +++ b/config/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/ddl/failtest/main_test.go b/ddl/failtest/main_test.go index 7c9e3d4bae618..aea99a6687c23 100644 --- a/ddl/failtest/main_test.go +++ b/ddl/failtest/main_test.go @@ -25,7 +25,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.SafeWindow = 0 diff --git a/ddl/label/main_test.go b/ddl/label/main_test.go index 25784de1cd97e..577972a1f5a95 100644 --- a/ddl/label/main_test.go +++ b/ddl/label/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/ddl/util/main_test.go b/ddl/util/main_test.go index 3dc8c61daacda..9446758e36923 100644 --- a/ddl/util/main_test.go +++ b/ddl/util/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), } diff --git a/distsql/main_test.go b/distsql/main_test.go index 1aaddbe954620..d000a32e9561f 100644 --- a/distsql/main_test.go +++ b/distsql/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/domain/globalconfigsync/globalconfig_test.go b/domain/globalconfigsync/globalconfig_test.go index c7be9137ddf68..a1705adffea1d 100644 --- a/domain/globalconfigsync/globalconfig_test.go +++ b/domain/globalconfigsync/globalconfig_test.go @@ -30,7 +30,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/domain/infosync/info_test.go b/domain/infosync/info_test.go index bb0ded60e7e25..f8e0d5d728650 100644 --- a/domain/infosync/info_test.go +++ b/domain/infosync/info_test.go @@ -37,7 +37,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), } diff --git a/domain/main_test.go b/domain/main_test.go index 069ed9590f222..969d50e4a0f67 100644 --- a/domain/main_test.go +++ b/domain/main_test.go @@ -23,7 +23,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/errno/main_test.go b/errno/main_test.go index a38aae7f1cb39..f0064431962cc 100644 --- a/errno/main_test.go +++ b/errno/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() os.Exit(m.Run()) } diff --git a/executor/aggfuncs/main_test.go b/executor/aggfuncs/main_test.go index f46a63bba4d9a..a385a98c57215 100644 --- a/executor/aggfuncs/main_test.go +++ b/executor/aggfuncs/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/executor/main_test.go b/executor/main_test.go index bff65b72d6a2d..88be5b84f0750 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -34,7 +34,7 @@ var prepareMergeSuiteData testdata.TestData var aggMergeSuiteData testdata.TestData func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() testDataMap.LoadTestSuiteData("testdata", "prepare_suite") testDataMap.LoadTestSuiteData("testdata", "agg_suite") diff --git a/executor/oomtest/oom_test.go b/executor/oomtest/oom_test.go index d592456bb61cd..b6a4d452bbe4a 100644 --- a/executor/oomtest/oom_test.go +++ b/executor/oomtest/oom_test.go @@ -34,7 +34,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() registerHook() domain.RunAutoAnalyze = false config.UpdateGlobal(func(conf *config.Config) { diff --git a/executor/seqtest/main_test.go b/executor/seqtest/main_test.go index 0a85b8ca61055..737a21822d083 100644 --- a/executor/seqtest/main_test.go +++ b/executor/seqtest/main_test.go @@ -23,7 +23,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.SafeWindow = 0 conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 diff --git a/expression/aggregation/main_test.go b/expression/aggregation/main_test.go index 4078dc6cf4b48..53126014ead5b 100644 --- a/expression/aggregation/main_test.go +++ b/expression/aggregation/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/expression/main_test.go b/expression/main_test.go index 9a1e170078f65..590c1c33ba72f 100644 --- a/expression/main_test.go +++ b/expression/main_test.go @@ -32,7 +32,7 @@ import ( var testDataMap = make(testdata.BookKeeper) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() testmain.ShortCircuitForBench(m) config.UpdateGlobal(func(conf *config.Config) { diff --git a/infoschema/main_test.go b/infoschema/main_test.go index 8d3c4dba9623e..8a878a925be01 100644 --- a/infoschema/main_test.go +++ b/infoschema/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/infoschema/perfschema/main_test.go b/infoschema/perfschema/main_test.go index 068cebc3f3cb0..a13c7d1ddca79 100644 --- a/infoschema/perfschema/main_test.go +++ b/infoschema/perfschema/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/kv/main_test.go b/kv/main_test.go index e87d9dd2a9916..6d1f289de7d9a 100644 --- a/kv/main_test.go +++ b/kv/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/meta/autoid/main_test.go b/meta/autoid/main_test.go index a5e8e915db0db..c9ee29c2200f7 100644 --- a/meta/autoid/main_test.go +++ b/meta/autoid/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/meta/main_test.go b/meta/main_test.go index 858d4bbb6f6e6..7fd569a01a417 100644 --- a/meta/main_test.go +++ b/meta/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/metrics/main_test.go b/metrics/main_test.go index 2e27c2eec0544..96050f378e142 100644 --- a/metrics/main_test.go +++ b/metrics/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/owner/main_test.go b/owner/main_test.go index ba940ab9a3d5d..ec1eeb75cf4a5 100644 --- a/owner/main_test.go +++ b/owner/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), } diff --git a/planner/cascades/main_test.go b/planner/cascades/main_test.go index 590e1825b08bd..f1a3ceb81fe23 100644 --- a/planner/cascades/main_test.go +++ b/planner/cascades/main_test.go @@ -30,7 +30,7 @@ var stringerSuiteData testdata.TestData var transformationRulesSuiteData testdata.TestData func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() flag.Parse() diff --git a/planner/core/main_test.go b/planner/core/main_test.go index 236d154ff8934..c0fb896a5bfd2 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -28,7 +28,7 @@ var testDataMap = make(testdata.BookKeeper, 2) var indexMergeSuiteData testdata.TestData func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() flag.Parse() diff --git a/planner/implementation/main_test.go b/planner/implementation/main_test.go index b7a2088709314..2e4cb46332f0f 100644 --- a/planner/implementation/main_test.go +++ b/planner/implementation/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/planner/memo/main_test.go b/planner/memo/main_test.go index 784f011a9ddee..6072aab036844 100644 --- a/planner/memo/main_test.go +++ b/planner/memo/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/planner/util/main_test.go b/planner/util/main_test.go index 1b930670688d0..0a5c3ac1a47da 100644 --- a/planner/util/main_test.go +++ b/planner/util/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/plugin/conn_ip_example/main_test.go b/plugin/conn_ip_example/main_test.go index 640b8d3aa5108..24c8bc9dcec4e 100644 --- a/plugin/conn_ip_example/main_test.go +++ b/plugin/conn_ip_example/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/plugin/main_test.go b/plugin/main_test.go index 48633eff506ae..cd0e57872a41b 100644 --- a/plugin/main_test.go +++ b/plugin/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/privilege/privileges/main_test.go b/privilege/privileges/main_test.go index 0d43ce5c21a4a..9819d73c47670 100644 --- a/privilege/privileges/main_test.go +++ b/privilege/privileges/main_test.go @@ -27,7 +27,7 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() session.SetSchemaLease(0) session.DisableStats4Test() diff --git a/server/main_test.go b/server/main_test.go index 155d9f9b7294e..1320e01c821bf 100644 --- a/server/main_test.go +++ b/server/main_test.go @@ -29,7 +29,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() runInGoTest = true // flag for NewServer to known it is running in test environment diff --git a/session/main_test.go b/session/main_test.go index ee79b1b30e967..fb75aaccf0f3a 100644 --- a/session/main_test.go +++ b/session/main_test.go @@ -42,7 +42,7 @@ var testDataMap = make(testdata.BookKeeper, 1) func TestMain(m *testing.M) { testmain.ShortCircuitForBench(m) - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() flag.Parse() testDataMap.LoadTestSuiteData("testdata", "clustered_index_suite") diff --git a/sessionctx/binloginfo/main_test.go b/sessionctx/binloginfo/main_test.go index 27abcdef3e02f..c7e1ff6d026be 100644 --- a/sessionctx/binloginfo/main_test.go +++ b/sessionctx/binloginfo/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("google.golang.org/grpc.(*addrConn).resetTransport"), goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), diff --git a/sessionctx/main_test.go b/sessionctx/main_test.go index 89eeb11becb85..ae77476986062 100644 --- a/sessionctx/main_test.go +++ b/sessionctx/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/sessionctx/stmtctx/main_test.go b/sessionctx/stmtctx/main_test.go index 5ccfada0cec6c..e97d9a07961a7 100644 --- a/sessionctx/stmtctx/main_test.go +++ b/sessionctx/stmtctx/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/sessionctx/variable/main_test.go b/sessionctx/variable/main_test.go index af5506566eac2..d8c4cc64199e8 100644 --- a/sessionctx/variable/main_test.go +++ b/sessionctx/variable/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 099af86a0e689..97144263590b5 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -32,7 +32,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/statistics/handle/main_test.go b/statistics/handle/main_test.go index 5b8ea0ed4c0dc..658b377aba2b4 100644 --- a/statistics/handle/main_test.go +++ b/statistics/handle/main_test.go @@ -26,6 +26,6 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/statistics/main_test.go b/statistics/main_test.go index 7e40d650fe393..4b0960f69a735 100644 --- a/statistics/main_test.go +++ b/statistics/main_test.go @@ -32,7 +32,7 @@ import ( var testDataMap = make(testdata.BookKeeper, 3) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() if !flag.Parsed() { flag.Parse() diff --git a/store/copr/main_test.go b/store/copr/main_test.go index 411741f5e651e..e69b25bc2173e 100644 --- a/store/copr/main_test.go +++ b/store/copr/main_test.go @@ -34,6 +34,6 @@ func (m *main) Run() int { } func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(&main{m: m}) } diff --git a/store/driver/error/error_test.go b/store/driver/error/error_test.go index 3db2830179502..6d2222b29bc59 100644 --- a/store/driver/error/error_test.go +++ b/store/driver/error/error_test.go @@ -26,7 +26,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/driver/main_test.go b/store/driver/main_test.go index 1902174a16bf7..7e9de1553e8aa 100644 --- a/store/driver/main_test.go +++ b/store/driver/main_test.go @@ -37,7 +37,7 @@ var ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() tikv.EnableFailpoints() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/store/driver/txn/main_test.go b/store/driver/txn/main_test.go index 3805c435df0e3..97fa3f9904ce4 100644 --- a/store/driver/txn/main_test.go +++ b/store/driver/txn/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/gcworker/main_test.go b/store/gcworker/main_test.go index 0a6be0a87548b..89909d438672a 100644 --- a/store/gcworker/main_test.go +++ b/store/gcworker/main_test.go @@ -25,7 +25,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() tikv.EnableFailpoints() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/store/helper/main_test.go b/store/helper/main_test.go index ade457cc75183..52601cf5f3c6a 100644 --- a/store/helper/main_test.go +++ b/store/helper/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/main_test.go b/store/main_test.go index 40703aa3d2e51..b76d49ae6800c 100644 --- a/store/main_test.go +++ b/store/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/store/mockstore/main_test.go b/store/mockstore/main_test.go index 0bb01f6c23182..dea595138efbf 100644 --- a/store/mockstore/main_test.go +++ b/store/mockstore/main_test.go @@ -24,7 +24,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() callback := func(i int) int { // wait for leveldb to close, leveldb will be closed in one second time.Sleep(time.Second) diff --git a/store/mockstore/mockcopr/main_test.go b/store/mockstore/mockcopr/main_test.go index 3f3036ba87541..91d38a4cff79c 100644 --- a/store/mockstore/mockcopr/main_test.go +++ b/store/mockstore/mockcopr/main_test.go @@ -24,7 +24,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/store/mockstore/unistore/cophandler/main_test.go b/store/mockstore/unistore/cophandler/main_test.go index e740267de961d..cffc39b4fa1c0 100644 --- a/store/mockstore/unistore/cophandler/main_test.go +++ b/store/mockstore/unistore/cophandler/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/mockstore/unistore/lockstore/main_test.go b/store/mockstore/unistore/lockstore/main_test.go index 5674b940ca5e6..0fb96356bfc89 100644 --- a/store/mockstore/unistore/lockstore/main_test.go +++ b/store/mockstore/unistore/lockstore/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/mockstore/unistore/main_test.go b/store/mockstore/unistore/main_test.go index 9d44274a8c87d..8b40056fab6d9 100644 --- a/store/mockstore/unistore/main_test.go +++ b/store/mockstore/unistore/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/mockstore/unistore/tikv/main_test.go b/store/mockstore/unistore/tikv/main_test.go index 5bc57f47c8321..c74abb998b10a 100644 --- a/store/mockstore/unistore/tikv/main_test.go +++ b/store/mockstore/unistore/tikv/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/mockstore/unistore/util/lockwaiter/main_test.go b/store/mockstore/unistore/util/lockwaiter/main_test.go index 6bd0d063dae5c..4a0cf2fdd53de 100644 --- a/store/mockstore/unistore/util/lockwaiter/main_test.go +++ b/store/mockstore/unistore/util/lockwaiter/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/structure/main_test.go b/structure/main_test.go index 793fae6d90647..3874644b636a8 100644 --- a/structure/main_test.go +++ b/structure/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/table/main_test.go b/table/main_test.go index 91463fb0b5f8c..8ff3f37c63060 100644 --- a/table/main_test.go +++ b/table/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/table/tables/main_test.go b/table/tables/main_test.go index ebfceb2bd3bca..9150776cd9404 100644 --- a/table/tables/main_test.go +++ b/table/tables/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/table/temptable/main_test.go b/table/temptable/main_test.go index c7baa1e9f5208..2b312cecefc9b 100644 --- a/table/temptable/main_test.go +++ b/table/temptable/main_test.go @@ -38,7 +38,7 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/tablecodec/main_test.go b/tablecodec/main_test.go index 75ea2dc757133..9785957934b8b 100644 --- a/tablecodec/main_test.go +++ b/tablecodec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/tablecodec/rowindexcodec/main_test.go b/tablecodec/rowindexcodec/main_test.go index 55b15ba96e15d..7a48bd9d289a1 100644 --- a/tablecodec/rowindexcodec/main_test.go +++ b/tablecodec/rowindexcodec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/telemetry/cte_test/cte_test.go b/telemetry/cte_test/cte_test.go index 356b65eaad059..5849b285f97e6 100644 --- a/telemetry/cte_test/cte_test.go +++ b/telemetry/cte_test/cte_test.go @@ -33,7 +33,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/telemetry/main_test.go b/telemetry/main_test.go index f498d16a2d564..feb7337e8fac3 100644 --- a/telemetry/main_test.go +++ b/telemetry/main_test.go @@ -27,7 +27,7 @@ var ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/tests/globalkilltest/main_test.go b/tests/globalkilltest/main_test.go index ae4d8e2d63b02..71ee2c7e95780 100644 --- a/tests/globalkilltest/main_test.go +++ b/tests/globalkilltest/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() os.Exit(m.Run()) } diff --git a/tests/graceshutdown/main_test.go b/tests/graceshutdown/main_test.go index 65f1a0ac1d78e..87d7052c17762 100644 --- a/tests/graceshutdown/main_test.go +++ b/tests/graceshutdown/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("syscall.syscall6"), } diff --git a/tests/readonlytest/main_test.go b/tests/readonlytest/main_test.go index 6d7f7491b6296..9e822e0c6ddb2 100644 --- a/tests/readonlytest/main_test.go +++ b/tests/readonlytest/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/tidb-server/main_test.go b/tidb-server/main_test.go index 84f16eb033561..77c042daf8eae 100644 --- a/tidb-server/main_test.go +++ b/tidb-server/main_test.go @@ -28,7 +28,7 @@ import ( var isCoverageServer string func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/tools/check/go.mod b/tools/check/go.mod index 81ee48b2242cd..5db3382e2d8b9 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -1,10 +1,8 @@ module github.com/pingcap/tidb/_tools require ( - github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737 github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc // indirect github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf // indirect - github.com/axw/gocov v1.0.0 github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03 github.com/dnephin/govet v0.0.0-20171012192244-4a96d43e39d3 github.com/kisielk/errcheck v1.2.0 diff --git a/tools/check/go.sum b/tools/check/go.sum index 776ad3f913a32..53cc061d53eec 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -1,12 +1,8 @@ -github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737 h1:JZHBkt0GhM+ARQykshqpI49yaWCHQbJonH3XpDTwMZQ= -github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737/go.mod h1:w1KSuh2JgIL3nyRiZijboSUwbbxOrTzWwyWVFUHtXBQ= github.com/BurntSushi/toml v0.3.0 h1:e1/Ivsx3Z0FVTV0NSOv/aVgbUWyQuzj7DDnFblkRvsY= github.com/BurntSushi/toml v0.3.0/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf h1:qet1QNfXsQxTZqLG4oE62mJzwPIB8+Tee4RNCL9ulrY= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/axw/gocov v1.0.0 h1:YsqYR66hUmilVr23tu8USgnJIJvnwh3n7j5zRn7x4LU= -github.com/axw/gocov v1.0.0/go.mod h1:LvQpEYiwwIb2nYkXY2fDWhg9/AsYqkhmrCshjlUJECE= github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03 h1:0wUHjDfbCAROEAZ96zAJGwcNMkPIheFaIjtQyv3QqfM= github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03/go.mod h1:uFE9hX+zXEwvyUThZ4gDb9vkAwc5DoHUnRSEpH0VrOs= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= diff --git a/types/json/main_test.go b/types/json/main_test.go index a8cf27b5e9331..85aa89330e238 100644 --- a/types/json/main_test.go +++ b/types/json/main_test.go @@ -24,6 +24,6 @@ import ( const benchStr = `{"a":[1,"2",{"aa":"bb"},4,null],"b":true,"c":null}` func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/types/main_test.go b/types/main_test.go index 26127901cab34..73d0d58061bda 100644 --- a/types/main_test.go +++ b/types/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/types/parser_driver/main_test.go b/types/parser_driver/main_test.go index b1e6a35776689..d3be3b70a0a41 100644 --- a/types/parser_driver/main_test.go +++ b/types/parser_driver/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/admin/main_test.go b/util/admin/main_test.go index b4242e21591ee..e600c43eaefb8 100644 --- a/util/admin/main_test.go +++ b/util/admin/main_test.go @@ -23,7 +23,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.SafeWindow = 0 diff --git a/util/arena/main_test.go b/util/arena/main_test.go index 020d3424cc9b8..b0b84e0a0bf8a 100644 --- a/util/arena/main_test.go +++ b/util/arena/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/benchdaily/main_test.go b/util/benchdaily/main_test.go index de5653eaa3f98..aa5fe00c5fc55 100644 --- a/util/benchdaily/main_test.go +++ b/util/benchdaily/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/bitmap/main_test.go b/util/bitmap/main_test.go index c32a6a56b6c2a..65b018ed43e14 100644 --- a/util/bitmap/main_test.go +++ b/util/bitmap/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/checksum/main_test.go b/util/checksum/main_test.go index 4212ee3bc431c..4781cf2b6b147 100644 --- a/util/checksum/main_test.go +++ b/util/checksum/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/chunk/main_test.go b/util/chunk/main_test.go index bc9a0900ec180..d3f9eddaa54eb 100644 --- a/util/chunk/main_test.go +++ b/util/chunk/main_test.go @@ -24,7 +24,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() path, _ := os.MkdirTemp("", "oom-use-tmp-storage") config.UpdateGlobal(func(conf *config.Config) { diff --git a/util/codec/main_test.go b/util/codec/main_test.go index cdc978b14d253..b7219d2bfbff3 100644 --- a/util/codec/main_test.go +++ b/util/codec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/collate/main_test.go b/util/collate/main_test.go index e82127b7013c9..048f02f25b627 100644 --- a/util/collate/main_test.go +++ b/util/collate/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/cteutil/main_test.go b/util/cteutil/main_test.go index f882348ac05cc..74244f4f0eec1 100644 --- a/util/cteutil/main_test.go +++ b/util/cteutil/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/dbterror/main_test.go b/util/dbterror/main_test.go index 7879e9d1a6904..c409af6eea5a7 100644 --- a/util/dbterror/main_test.go +++ b/util/dbterror/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/deadlockhistory/main_test.go b/util/deadlockhistory/main_test.go index f5d8af7b1f799..ea3e208d82692 100644 --- a/util/deadlockhistory/main_test.go +++ b/util/deadlockhistory/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/disjointset/main_test.go b/util/disjointset/main_test.go index 9ae4a8269842b..9145fc2f2d696 100644 --- a/util/disjointset/main_test.go +++ b/util/disjointset/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/disk/main_test.go b/util/disk/main_test.go index 966c9a8e86fa2..bef823c25baa9 100644 --- a/util/disk/main_test.go +++ b/util/disk/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/encrypt/main_test.go b/util/encrypt/main_test.go index 29fb8827d96e0..e6df43ef9941d 100644 --- a/util/encrypt/main_test.go +++ b/util/encrypt/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/execdetails/main_test.go b/util/execdetails/main_test.go index a55b06c98e775..0388a932da8c5 100644 --- a/util/execdetails/main_test.go +++ b/util/execdetails/main_test.go @@ -21,6 +21,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/expensivequery/expensivequerey_test.go b/util/expensivequery/expensivequerey_test.go index 8f5c60179694e..9852780a20b09 100644 --- a/util/expensivequery/expensivequerey_test.go +++ b/util/expensivequery/expensivequerey_test.go @@ -27,7 +27,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/fastrand/main_test.go b/util/fastrand/main_test.go index fb2255c79e252..a0ec62f151863 100644 --- a/util/fastrand/main_test.go +++ b/util/fastrand/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/format/main_test.go b/util/format/main_test.go index 66294a0dbadfb..e0269371cc883 100644 --- a/util/format/main_test.go +++ b/util/format/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/generatedexpr/main_test.go b/util/generatedexpr/main_test.go index d5418deebddcf..51ad2d1335ce0 100644 --- a/util/generatedexpr/main_test.go +++ b/util/generatedexpr/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/hack/main_test.go b/util/hack/main_test.go index aaa708c42a033..193c9492d3063 100644 --- a/util/hack/main_test.go +++ b/util/hack/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/keydecoder/main_test.go b/util/keydecoder/main_test.go index 6900f8e304ec0..1dd3d666b6a67 100644 --- a/util/keydecoder/main_test.go +++ b/util/keydecoder/main_test.go @@ -26,6 +26,6 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/util/kvcache/main_test.go b/util/kvcache/main_test.go index 90e09a3a45e77..9b4ead41b9f2d 100644 --- a/util/kvcache/main_test.go +++ b/util/kvcache/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/localpool/main_test.go b/util/localpool/main_test.go index fb21dcc42f4f5..35b66eda56f43 100644 --- a/util/localpool/main_test.go +++ b/util/localpool/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/logutil/main_test.go b/util/logutil/main_test.go index 7d833f0bd86ef..0113d6f75ee6c 100644 --- a/util/logutil/main_test.go +++ b/util/logutil/main_test.go @@ -34,7 +34,7 @@ var ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), } diff --git a/util/main_test.go b/util/main_test.go index 1b930670688d0..0a5c3ac1a47da 100644 --- a/util/main_test.go +++ b/util/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/math/main_test.go b/util/math/main_test.go index c10b457e497a8..b315c0487ce89 100644 --- a/util/math/main_test.go +++ b/util/math/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/memory/main_test.go b/util/memory/main_test.go index 03d7f7d266e77..34ce2a33e66c3 100644 --- a/util/memory/main_test.go +++ b/util/memory/main_test.go @@ -15,22 +15,14 @@ package memory import ( - "fmt" - "os" "testing" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testbridge" "go.uber.org/goleak" ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() - logLevel := os.Getenv("log_level") - err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false)) - if err != nil { - fmt.Fprint(os.Stderr, err.Error()) - os.Exit(1) - } + testbridge.SetupForCommonTest() + goleak.VerifyTestMain(m) } diff --git a/util/mock/main_test.go b/util/mock/main_test.go index 11d845761a1ec..e855ad497df38 100644 --- a/util/mock/main_test.go +++ b/util/mock/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/mvmap/main_test.go b/util/mvmap/main_test.go index 86dd20958e3a6..39178d04f26fe 100644 --- a/util/mvmap/main_test.go +++ b/util/mvmap/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/parser/main_test.go b/util/parser/main_test.go index eef165cfd2fcf..f2cc66445b8e2 100644 --- a/util/parser/main_test.go +++ b/util/parser/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/plancodec/main_test.go b/util/plancodec/main_test.go index 31620cb293173..65fc11aefa4bd 100644 --- a/util/plancodec/main_test.go +++ b/util/plancodec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/printer/main_test.go b/util/printer/main_test.go index ae6fbd3e45509..1f5f8dda193b5 100644 --- a/util/printer/main_test.go +++ b/util/printer/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/profile/main_test.go b/util/profile/main_test.go index cd5b368a787de..d3ddfb5ac3238 100644 --- a/util/profile/main_test.go +++ b/util/profile/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/util/ranger/main_test.go b/util/ranger/main_test.go index f0f1652c18edd..410ac374b04d6 100644 --- a/util/ranger/main_test.go +++ b/util/ranger/main_test.go @@ -29,7 +29,7 @@ var testDataMap = make(testdata.BookKeeper, 1) var rangerSuiteData testdata.TestData func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() flag.Parse() diff --git a/util/resourcegrouptag/main_test.go b/util/resourcegrouptag/main_test.go index 9bd1c8df16cd4..0ab522e6ae8bb 100644 --- a/util/resourcegrouptag/main_test.go +++ b/util/resourcegrouptag/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/rowDecoder/main_test.go b/util/rowDecoder/main_test.go index 0e93bf3104a9b..1a843cbfe6b69 100644 --- a/util/rowDecoder/main_test.go +++ b/util/rowDecoder/main_test.go @@ -26,6 +26,6 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/util/rowcodec/main_test.go b/util/rowcodec/main_test.go index 7a54747ceaf72..b333276b3c672 100644 --- a/util/rowcodec/main_test.go +++ b/util/rowcodec/main_test.go @@ -25,7 +25,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/selection/main_test.go b/util/selection/main_test.go index 91e31ed6c5bb3..a35eb9a4bfe88 100644 --- a/util/selection/main_test.go +++ b/util/selection/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/sem/main_test.go b/util/sem/main_test.go index 81af71ff59d39..7c8abcb5b5049 100644 --- a/util/sem/main_test.go +++ b/util/sem/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/set/main_test.go b/util/set/main_test.go index d119da45ccb6b..1a068e7eed801 100644 --- a/util/set/main_test.go +++ b/util/set/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/slice/main_test.go b/util/slice/main_test.go index 2da7631bca34f..08c038fe064dd 100644 --- a/util/slice/main_test.go +++ b/util/slice/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/sqlexec/main_test.go b/util/sqlexec/main_test.go index 7aa194b084bf0..672e7b4343d24 100644 --- a/util/sqlexec/main_test.go +++ b/util/sqlexec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/stmtsummary/main_test.go b/util/stmtsummary/main_test.go index 99143834fd420..43e9e63d30c30 100644 --- a/util/stmtsummary/main_test.go +++ b/util/stmtsummary/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/stringutil/main_test.go b/util/stringutil/main_test.go index 7f012cc923dba..0690bad2d85f4 100644 --- a/util/stringutil/main_test.go +++ b/util/stringutil/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/sys/linux/main_test.go b/util/sys/linux/main_test.go index 4c190736347de..052580b6f3b86 100644 --- a/util/sys/linux/main_test.go +++ b/util/sys/linux/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/sys/storage/main_test.go b/util/sys/storage/main_test.go index 846992285e92a..ac3a5f06caeaa 100644 --- a/util/sys/storage/main_test.go +++ b/util/sys/storage/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/systimemon/main_test.go b/util/systimemon/main_test.go index 2d5053b2b2a93..2a216674ce419 100644 --- a/util/systimemon/main_test.go +++ b/util/systimemon/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/pingcap/tidb/util/systimemon.StartMonitor"), diff --git a/util/testbridge/bridge.go b/util/testbridge/bridge.go index 1aee33f9c0a15..a682a6574fa3c 100644 --- a/util/testbridge/bridge.go +++ b/util/testbridge/bridge.go @@ -19,18 +19,47 @@ package testbridge import ( "flag" + "fmt" + "os" + + "github.com/pingcap/log" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) -// WorkaroundGoCheckFlags registers flags of go-check for pkg does not import go-check +// SetupForCommonTest runs before all the tests. +func SetupForCommonTest() { + workaroundGoCheckFlags() + applyOSLogLevel() +} + +// workaroundGoCheckFlags registers flags of go-check for pkg does not import go-check // to workaround the go-check flags passed in Makefile. // // TODO: Remove this function when the migration from go-check to testify[1] is done. // [1] https://github.com/pingcap/tidb/issues/26022 -func WorkaroundGoCheckFlags() { +func workaroundGoCheckFlags() { if flag.Lookup("check.timeout") == nil { - _ = flag.Duration("check.timeout", 0, "WorkaroundGoCheckFlags: check.timeout") + _ = flag.Duration("check.timeout", 0, "workaroundGoCheckFlags: check.timeout") } if flag.Lookup("check.p") == nil { - _ = flag.Bool("check.p", false, "WorkaroundGoCheckFlags: check.p") + _ = flag.Bool("check.p", false, "workaroundGoCheckFlags: check.p") + } +} + +func applyOSLogLevel() { + osLoglevel := os.Getenv("log_level") + if len(osLoglevel) > 0 { + cfg := log.Config{ + Level: osLoglevel, + Format: "text", + File: log.FileLogConfig{}, + } + gl, props, err := log.InitLogger(&cfg, zap.AddStacktrace(zapcore.FatalLevel)) + if err != nil { + _, _ = fmt.Fprintf(os.Stderr, "applyOSLogLevel failed: %v", err) + os.Exit(-1) + } + log.ReplaceGlobals(gl, props) } } diff --git a/util/texttree/main_test.go b/util/texttree/main_test.go index af4e400aeaa61..8aa6c0a957425 100644 --- a/util/texttree/main_test.go +++ b/util/texttree/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/timeutil/main_test.go b/util/timeutil/main_test.go index 910501d567d05..9d30909025632 100644 --- a/util/timeutil/main_test.go +++ b/util/timeutil/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/topsql/main_test.go b/util/topsql/main_test.go index f5e3dc3f7d0cf..b567382484285 100644 --- a/util/topsql/main_test.go +++ b/util/topsql/main_test.go @@ -25,7 +25,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() // set up variable.TopSQLVariable.Enable.Store(true) diff --git a/util/topsql/reporter/main_test.go b/util/topsql/reporter/main_test.go index a828a948fb4fd..b06eb67a844cf 100644 --- a/util/topsql/reporter/main_test.go +++ b/util/topsql/reporter/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/topsql/stmtstats/main_test.go b/util/topsql/stmtstats/main_test.go index 24f6c2574c522..cee86b8217e18 100644 --- a/util/topsql/stmtstats/main_test.go +++ b/util/topsql/stmtstats/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/topsql/stmtstats/stmtstatstest/main_test.go b/util/topsql/stmtstats/stmtstatstest/main_test.go index ecf1220642ecf..033634a360cc0 100644 --- a/util/topsql/stmtstats/stmtstatstest/main_test.go +++ b/util/topsql/stmtstats/stmtstatstest/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/util/topsql/tracecpu/main_test.go b/util/topsql/tracecpu/main_test.go index 74352d78d7419..2e6d98c89b32e 100644 --- a/util/topsql/tracecpu/main_test.go +++ b/util/topsql/tracecpu/main_test.go @@ -28,7 +28,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() variable.TopSQLVariable.Enable.Store(false) config.UpdateGlobal(func(conf *config.Config) { diff --git a/util/tracing/main_test.go b/util/tracing/main_test.go index 17e67ecd40ea8..c684666e3216f 100644 --- a/util/tracing/main_test.go +++ b/util/tracing/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/vitess/main_test.go b/util/vitess/main_test.go index a66f334f9bba9..4151a95a5b7d1 100644 --- a/util/vitess/main_test.go +++ b/util/vitess/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } From bff723bf4b5dbc8a8dc26781dafab04b6378c16b Mon Sep 17 00:00:00 2001 From: you06 Date: Fri, 24 Dec 2021 18:23:53 +0800 Subject: [PATCH 4/5] planner: add usage of paging copr in optimizer (#30536) close pingcap/tidb#30578 --- distsql/distsql.go | 1 + distsql/select_result.go | 9 +++-- distsql/stream.go | 2 +- executor/builder.go | 6 ++++ executor/distsql.go | 2 ++ metrics/distsql.go | 2 +- metrics/grafana/tidb.json | 4 +-- metrics/session.go | 1 + planner/core/explain.go | 11 ++++-- planner/core/find_best_task.go | 1 + planner/core/physical_plans.go | 1 + planner/core/plan_test.go | 64 ++++++++++++++++++++++++++++++++++ planner/core/task.go | 52 ++++++++++++++++++++++++++- store/copr/coprocessor.go | 25 ++----------- store/copr/coprocessor_test.go | 3 +- util/paging/main_test.go | 27 ++++++++++++++ util/paging/paging.go | 61 ++++++++++++++++++++++++++++++++ util/paging/paging_test.go | 36 +++++++++++++++++++ 18 files changed, 275 insertions(+), 33 deletions(-) create mode 100644 util/paging/main_test.go create mode 100644 util/paging/paging.go create mode 100644 util/paging/paging_test.go diff --git a/distsql/distsql.go b/distsql/distsql.go index 77b75efc480fd..5581f9c269c99 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -133,6 +133,7 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie memTracker: kvReq.MemTracker, encodeType: encodetype, storeType: kvReq.StoreType, + paging: kvReq.Paging, }, nil } diff --git a/distsql/select_result.go b/distsql/select_result.go index 3ac7f1db94a97..b2eef6e6f5300 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -152,7 +152,8 @@ type selectResult struct { durationReported bool memTracker *memory.Tracker - stats *selectResultRuntimeStats + stats *selectResultRuntimeStats + paging bool } func (r *selectResult) fetchResp(ctx context.Context) error { @@ -206,7 +207,11 @@ func (r *selectResult) fetchResp(ctx context.Context) error { // final round of fetch // TODO: Add a label to distinguish between success or failure. // https://github.com/pingcap/tidb/issues/11397 - metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) + if r.paging { + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "paging").Observe(r.fetchDuration.Seconds()) + } else { + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "common").Observe(r.fetchDuration.Seconds()) + } r.durationReported = true } return nil diff --git a/distsql/stream.go b/distsql/stream.go index 73d8f96b8fe79..2f10e72b5c0f5 100644 --- a/distsql/stream.go +++ b/distsql/stream.go @@ -82,7 +82,7 @@ func (r *streamResult) readDataFromResponse(ctx context.Context, resp kv.Respons if !r.durationReported { // TODO: Add a label to distinguish between success or failure. // https://github.com/pingcap/tidb/issues/11397 - metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "streaming").Observe(r.fetchDuration.Seconds()) r.durationReported = true } return true, nil diff --git a/executor/builder.go b/executor/builder.go index 78c94fd02f1aa..ef932da1211fa 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3409,6 +3409,11 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn if err != nil { return nil, err } + indexPaging := false + if v.Paging { + indexPaging = true + indexStreaming = false + } tableReq, tableStreaming, tbl, err := buildTableReq(b, v.Schema().Len(), v.TablePlans) if err != nil { return nil, err @@ -3430,6 +3435,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn columns: ts.Columns, indexStreaming: indexStreaming, tableStreaming: tableStreaming, + indexPaging: indexPaging, dataReaderBuilder: &dataReaderBuilder{executorBuilder: b}, corColInIdxSide: b.corColInDistPlan(v.IndexPlans), corColInTblSide: b.corColInDistPlan(v.TablePlans), diff --git a/executor/distsql.go b/executor/distsql.go index 1c31130dc53b5..9d380883703cd 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -361,6 +361,7 @@ type IndexLookUpExecutor struct { indexStreaming bool tableStreaming bool + indexPaging bool corColInIdxSide bool corColInTblSide bool @@ -560,6 +561,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< SetDesc(e.desc). SetKeepOrder(e.keepOrder). SetStreaming(e.indexStreaming). + SetPaging(e.indexPaging). SetReadReplicaScope(e.readReplicaScope). SetIsStaleness(e.isStaleness). SetFromSessionVars(e.ctx.GetSessionVars()). diff --git a/metrics/distsql.go b/metrics/distsql.go index 9bec9d7646827..3a4527da510ae 100644 --- a/metrics/distsql.go +++ b/metrics/distsql.go @@ -27,7 +27,7 @@ var ( Name: "handle_query_duration_seconds", Help: "Bucketed histogram of processing time (s) of handled queries.", Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days - }, []string{LblType, LblSQLType}) + }, []string{LblType, LblSQLType, LblCoprType}) DistSQLScanKeysPartialHistogram = prometheus.NewHistogram( prometheus.HistogramOpts{ diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index f6606d18ef4c6..a828967a8ada9 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -6168,10 +6168,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_distsql_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_distsql_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (copr_type)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "", + "legendFormat": "{{copr_type}}", "metric": "tidb_distsql_query_total", "refId": "A", "step": 4 diff --git a/metrics/session.go b/metrics/session.go index 0058104788f21..83df91439d311 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -142,6 +142,7 @@ const ( LblDb = "db" LblResult = "result" LblSQLType = "sql_type" + LblCoprType = "copr_type" LblGeneral = "general" LblInternal = "internal" LbTxnMode = "txn_mode" diff --git a/planner/core/explain.go b/planner/core/explain.go index 26bfa775fc417..d0ae474b79e7f 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -456,17 +456,22 @@ func (p *PhysicalIndexReader) accessObject(sctx sessionctx.Context) string { // ExplainInfo implements Plan interface. func (p *PhysicalIndexLookUpReader) ExplainInfo() string { + var str strings.Builder // The children can be inferred by the relation symbol. if p.PushedLimit != nil { - var str strings.Builder str.WriteString("limit embedded(offset:") str.WriteString(strconv.FormatUint(p.PushedLimit.Offset, 10)) str.WriteString(", count:") str.WriteString(strconv.FormatUint(p.PushedLimit.Count, 10)) str.WriteString(")") - return str.String() } - return "" + if p.Paging { + if p.PushedLimit != nil { + str.WriteString(", ") + } + str.WriteString("paging:true") + } + return str.String() } func (p *PhysicalIndexLookUpReader) accessObject(sctx sessionctx.Context) string { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 5157ffb738b24..ff90a92b9b497 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1249,6 +1249,7 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid indexPlan: is, tblColHists: ds.TblColHists, tblCols: ds.TblCols, + expectCnt: uint64(prop.ExpectedCnt), } cop.partitionInfo = PartitionInfo{ PruningConds: ds.allConds, diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 17e84b6efcf47..6293bba4b5073 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -270,6 +270,7 @@ type PhysicalIndexLookUpReader struct { TablePlans []PhysicalPlan indexPlan PhysicalPlan tablePlan PhysicalPlan + Paging bool ExtraHandleCol *expression.Column // PushedLimit is used to avoid unnecessary table scan tasks of IndexLookUpReader. diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 4437a354b1757..288444c9f2c12 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -18,6 +18,7 @@ import ( "bytes" "fmt" "strings" + "testing" . "github.com/pingcap/check" "github.com/pingcap/tidb/config" @@ -26,6 +27,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" + kit "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/testkit" @@ -639,3 +641,65 @@ func (s *testPlanNormalize) TestIssue25729(c *C) { tk.MustExec("insert into t1 values(\"a\", \"adwa\");") tk.MustQuery("select * from t1 where concat(a, b) like \"aadwa\" and a = \"a\";").Check(testkit.Rows("a adwa")) } + +func TestCopPaging(t *testing.T) { + store, clean := kit.CreateMockStore(t) + defer clean() + + tk := kit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set session tidb_enable_paging = 1") + tk.MustExec("create table t(id int, c1 int, c2 int, primary key (id), key i(c1))") + defer tk.MustExec("drop table t") + for i := 0; i < 1024; i++ { + tk.MustExec("insert into t values(?, ?, ?)", i, i, i) + } + tk.MustExec("analyze table t") + + // limit 960 should go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(kit.Rows( + "Limit 4.00 root offset:0, count:960", + "└─IndexLookUp 4.00 root paging:true", + " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 4.00 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 1024.00 cop[tikv] table:t keep order:false")) + } + + // selection between limit and indexlookup, limit 960 should also go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(kit.Rows( + "Limit 3.20 root offset:0, count:960", + "└─Selection 2.56 root gt(mod(test.t.id, 2), 0)", + " └─IndexLookUp 3.20 root paging:true", + " ├─Selection(Build) 819.20 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + } + + // limit 961 exceeds the threshold, it should not go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(kit.Rows( + "Limit 4.00 root offset:0, count:961", + "└─IndexLookUp 4.00 root ", + " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 4.00 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 1024.00 cop[tikv] table:t keep order:false")) + } + + // selection between limit and indexlookup, limit 961 should not go paging too + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(kit.Rows( + "Limit 3.20 root offset:0, count:961", + "└─Selection 2.56 root gt(mod(test.t.id, 2), 0)", + " └─IndexLookUp 3.20 root ", + " ├─Selection(Build) 819.20 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + } +} diff --git a/planner/core/task.go b/planner/core/task.go index 4b9c5692ca29d..90fbcedacaa9b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/paging" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -89,6 +90,10 @@ type copTask struct { // For table partition. partitionInfo PartitionInfo + + // expectCnt is the expected row count of upper task, 0 for unlimited. + // It's used for deciding whether using paging distsql. + expectCnt uint64 } func (t *copTask) invalid() bool { @@ -914,7 +919,17 @@ func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { // (indexRows / batchSize) * batchSize * CPUFactor // Since we don't know the number of copTasks built, ignore these network cost now. indexRows := t.indexPlan.statsInfo().RowCount - newTask.cst += indexRows * sessVars.CPUFactor + idxCst := indexRows * sessVars.CPUFactor + // if the expectCnt is below the paging threshold, using paging API, recalculate idxCst. + // paging API reduces the count of index and table rows, however introduces more seek cost. + if ctx.GetSessionVars().EnablePaging && t.expectCnt > 0 && t.expectCnt <= paging.Threshold { + p.Paging = true + pagingCst := calcPagingCost(ctx, t) + // prevent enlarging the cost because we take paging as a better plan, + // if the cost is enlarged, it'll be easier to go another plan. + idxCst = math.Min(idxCst, pagingCst) + } + newTask.cst += idxCst // Add cost of worker goroutines in index lookup. numTblWorkers := float64(sessVars.IndexLookupConcurrency()) newTask.cst += (numTblWorkers + 1) * sessVars.ConcurrencyFactor @@ -951,6 +966,41 @@ func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { return newTask } +func extractRows(p PhysicalPlan) float64 { + f := float64(0) + for _, c := range p.Children() { + if len(c.Children()) != 0 { + f += extractRows(c) + } else { + f += c.statsInfo().RowCount + } + } + return f +} + +// calcPagingCost calculates the cost for paging processing which may increase the seekCnt and reduce scanned rows. +func calcPagingCost(ctx sessionctx.Context, t *copTask) float64 { + sessVars := ctx.GetSessionVars() + indexRows := t.indexPlan.statsInfo().RowCount + expectCnt := t.expectCnt + sourceRows := extractRows(t.indexPlan) + // with paging, the scanned rows is always less than or equal to source rows. + if uint64(sourceRows) < expectCnt { + expectCnt = uint64(sourceRows) + } + seekCnt := paging.CalculateSeekCnt(expectCnt) + indexSelectivity := float64(1) + if sourceRows > indexRows { + indexSelectivity = indexRows / sourceRows + } + pagingCst := seekCnt*sessVars.GetSeekFactor(nil) + float64(expectCnt)*sessVars.CPUFactor + pagingCst *= indexSelectivity + + // we want the diff between idxCst and pagingCst here, + // however, the idxCst does not contain seekFactor, so a seekFactor needs to be removed + return pagingCst - sessVars.GetSeekFactor(nil) +} + func (t *rootTask) convertToRootTask(_ sessionctx.Context) *rootTask { return t.copy().(*rootTask) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 25ed965ea15e3..04e29a8507656 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/paging" "github.com/pingcap/tidb/util/trxevents" "github.com/pingcap/tipb/go-tipb" "github.com/tikv/client-go/v2/metrics" @@ -61,18 +62,6 @@ const ( copNextMaxBackoff = 20000 ) -// A paging request may be separated into multi requests if there are more data than a page. -// The paging size grows from min to max, it's not well tuned yet. -// e.g. a paging request scans over range (r1, r200), it requires 64 rows in the first batch, -// if it's not drained, then the paging size grows, the new range is calculated like (r100, r200), then send a request again. -// Compare with the common unary request, paging request allows early access of data, it offers a streaming-like way processing data. -// TODO: may make the paging parameters configurable. -const ( - minPagingSize uint64 = 64 - maxPagingSize = minPagingSize * 128 - pagingSizeGrow uint64 = 2 -) - // CopClient is coprocessor client. type CopClient struct { kv.RequestTypeSupportedChecker @@ -212,7 +201,7 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv // the size will grow every round. pagingSize := uint64(0) if req.Paging { - pagingSize = minPagingSize + pagingSize = paging.MinPagingSize } tasks = append(tasks, &copTask{ region: loc.Location.Region, @@ -928,7 +917,7 @@ func (worker *copIteratorWorker) handleCopPagingResult(bo *Backoffer, rpcCtx *ti if task.ranges.Len() == 0 { return nil, nil } - task.pagingSize = growPagingSize(task.pagingSize) + task.pagingSize = paging.GrowPagingSize(task.pagingSize) return []*copTask{task}, nil } @@ -1332,11 +1321,3 @@ func isolationLevelToPB(level kv.IsoLevel) kvrpcpb.IsolationLevel { return kvrpcpb.IsolationLevel_SI } } - -func growPagingSize(size uint64) uint64 { - size *= pagingSizeGrow - if size > maxPagingSize { - return maxPagingSize - } - return size -} diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index 88ad5568f68eb..b628b4eaab831 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/driver/backoff" + "github.com/pingcap/tidb/util/paging" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" @@ -318,7 +319,7 @@ func TestBuildPagingTasks(t *testing.T) { require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], "a", "c") require.True(t, tasks[0].paging) - require.Equal(t, tasks[0].pagingSize, minPagingSize) + require.Equal(t, tasks[0].pagingSize, paging.MinPagingSize) } func toCopRange(r kv.KeyRange) *coprocessor.KeyRange { diff --git a/util/paging/main_test.go b/util/paging/main_test.go new file mode 100644 index 0000000000000..af568af279474 --- /dev/null +++ b/util/paging/main_test.go @@ -0,0 +1,27 @@ +// Copyright 2021 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. + +package paging + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.SetupForCommonTest() + goleak.VerifyTestMain(m) +} diff --git a/util/paging/paging.go b/util/paging/paging.go new file mode 100644 index 0000000000000..5f2618ea341db --- /dev/null +++ b/util/paging/paging.go @@ -0,0 +1,61 @@ +// Copyright 2021 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. + +package paging + +import "math" + +// A paging request may be separated into multi requests if there are more data than a page. +// The paging size grows from min to max, it's not well tuned yet. +// e.g. a paging request scans over range (r1, r200), it requires 64 rows in the first batch, +// if it's not drained, then the paging size grows, the new range is calculated like (r100, r200), then send a request again. +// Compare with the common unary request, paging request allows early access of data, it offers a streaming-like way processing data. +// TODO: may make the paging parameters configurable. +const ( + MinPagingSize uint64 = 64 + maxPagingSizeShift = 7 + pagingSizeGrow = 2 + MaxPagingSize = MinPagingSize << maxPagingSizeShift + pagingGrowingSum = ((2 << maxPagingSizeShift) - 1) * MinPagingSize + Threshold uint64 = 960 +) + +// GrowPagingSize grows the paging size and ensures it does not exceed MaxPagingSize +func GrowPagingSize(size uint64) uint64 { + size <<= 1 + if size > MaxPagingSize { + return MaxPagingSize + } + return size +} + +// CalculateSeekCnt calculates the seek count from expect count +func CalculateSeekCnt(expectCnt uint64) float64 { + if expectCnt == 0 { + return 0 + } + if expectCnt > pagingGrowingSum { + // if the expectCnt is larger than pagingGrowingSum, calculate the seekCnt for the excess. + return float64(8 + (expectCnt-pagingGrowingSum+MaxPagingSize-1)/MaxPagingSize) + } + if expectCnt > MinPagingSize { + // if the expectCnt is less than pagingGrowingSum, + // calculate the seekCnt(number of terms) from the sum of a geometric progression. + // expectCnt = minPagingSize * (pagingSizeGrow ^ seekCnt - 1) / (pagingSizeGrow - 1) + // simplify (pagingSizeGrow ^ seekCnt - 1) to pagingSizeGrow ^ seekCnt, we can infer that + // seekCnt = log((pagingSizeGrow - 1) * expectCnt / minPagingSize) / log(pagingSizeGrow) + return 1 + float64(int(math.Log(float64((pagingSizeGrow-1)*expectCnt)/float64(MinPagingSize))/math.Log(float64(pagingSizeGrow)))) + } + return 1 +} diff --git a/util/paging/paging_test.go b/util/paging/paging_test.go new file mode 100644 index 0000000000000..1890b4d754d54 --- /dev/null +++ b/util/paging/paging_test.go @@ -0,0 +1,36 @@ +// Copyright 2021 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. + +package paging + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestGrowPagingSize(t *testing.T) { + require.Equal(t, GrowPagingSize(MinPagingSize), MinPagingSize*pagingSizeGrow) + require.Equal(t, GrowPagingSize(MaxPagingSize), MaxPagingSize) + require.Equal(t, GrowPagingSize(MaxPagingSize/pagingSizeGrow+1), MaxPagingSize) +} + +func TestCalculateSeekCnt(t *testing.T) { + require.InDelta(t, CalculateSeekCnt(0), 0, 0.1) + require.InDelta(t, CalculateSeekCnt(1), 1, 0.1) + require.InDelta(t, CalculateSeekCnt(MinPagingSize), 1, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum), maxPagingSizeShift+1, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum+1), maxPagingSizeShift+2, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum+MaxPagingSize), maxPagingSizeShift+2, 0.1) +} From fd61fe9afbc6b76e5ea1b0dca6f9c6ee7f2f5a0d Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 24 Dec 2021 19:16:41 +0800 Subject: [PATCH 5/5] test: address comment --- util/ranger/ranger_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 6c6ad0bf9abdb..7fc93cca7d8cc 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -1424,11 +1424,11 @@ func TestPrefixIndexAppendPointRanges(t *testing.T) { func TestIndexRange(t *testing.T) { dom, store, err := newDomainStoreWithBootstrap(t) + require.NoError(t, err) defer func() { dom.Close() require.NoError(t, store.Close()) }() - require.NoError(t, err) testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t")