Skip to content

Commit

Permalink
infoschema_v2: add bench test (#52200)
Browse files Browse the repository at this point in the history
ref #50959
  • Loading branch information
ywqzzy authored Apr 8, 2024
1 parent 0ce42ed commit e79a26f
Show file tree
Hide file tree
Showing 9 changed files with 140 additions and 54 deletions.
4 changes: 0 additions & 4 deletions pkg/disttask/framework/integrationtests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,16 +26,12 @@ go_test(
"//pkg/disttask/framework/taskexecutor",
"//pkg/disttask/framework/testutil",
"//pkg/domain",
"//pkg/parser/terror",
"//pkg/session",
"//pkg/store/driver",
"//pkg/testkit",
"//pkg/testkit/testsetup",
"//pkg/util",
"//pkg/util/metricsutil",
"@com_github_gorilla_mux//:mux",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_prometheus_client_golang//prometheus/promhttp",
"@com_github_stretchr_testify//require",
"@io_opencensus_go//stats/view",
"@org_uber_go_goleak//:goleak",
Expand Down
42 changes: 1 addition & 41 deletions pkg/disttask/framework/integrationtests/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,9 @@ import (
"context"
"flag"
"fmt"
"net"
"net/http"
"net/http/pprof"
"testing"
"time"

"github.com/gorilla/mux"
"github.com/pingcap/tidb/pkg/disttask/framework/handle"
"github.com/pingcap/tidb/pkg/disttask/framework/proto"
"github.com/pingcap/tidb/pkg/disttask/framework/scheduler"
Expand All @@ -33,13 +29,9 @@ import (
"github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor"
"github.com/pingcap/tidb/pkg/disttask/framework/testutil"
"github.com/pingcap/tidb/pkg/domain"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/session"
"github.com/pingcap/tidb/pkg/store/driver"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/metricsutil"
"github.com/prometheus/client_golang/prometheus/promhttp"
"github.com/stretchr/testify/require"
"go.opencensus.io/stats/view"
"go.uber.org/mock/gomock"
Expand All @@ -63,7 +55,7 @@ var (
// bench.test -test.v -run ^$ -test.bench=BenchmarkSchedulerOverhead --with-tikv "upstream-pd:2379?disableGC=true"
func BenchmarkSchedulerOverhead(b *testing.B) {
ctx, cancel := context.WithCancel(context.Background())
statusWG := mockTiDBStatusPort(ctx, b)
statusWG := testkit.MockTiDBStatusPort(ctx, b, "10080")
defer func() {
cancel()
statusWG.Wait()
Expand Down Expand Up @@ -138,38 +130,6 @@ func prepareForBenchTest(b *testing.B) {
tk.MustExec("delete from mysql.tidb_background_subtask_history")
}

// we run this test on a k8s environment, so we need to mock the TiDB server status port
// to have metrics.
func mockTiDBStatusPort(ctx context.Context, b *testing.B) *util.WaitGroupWrapper {
var wg util.WaitGroupWrapper
err := metricsutil.RegisterMetrics()
terror.MustNil(err)
router := mux.NewRouter()
router.Handle("/metrics", promhttp.Handler())
serverMux := http.NewServeMux()
serverMux.Handle("/", router)
serverMux.HandleFunc("/debug/pprof/", pprof.Index)
serverMux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline)
serverMux.HandleFunc("/debug/pprof/profile", pprof.Profile)
serverMux.HandleFunc("/debug/pprof/symbol", pprof.Symbol)
serverMux.HandleFunc("/debug/pprof/trace", pprof.Trace)

statusListener, err := net.Listen("tcp", "0.0.0.0:10080")
require.NoError(b, err)
statusServer := &http.Server{Handler: serverMux}
wg.RunWithLog(func() {
if err := statusServer.Serve(statusListener); err != nil {
b.Logf("status server serve failed: %v", err)
}
})
wg.RunWithLog(func() {
<-ctx.Done()
_ = statusServer.Close()
})

return &wg
}

func registerTaskTypeForBench(c *testutil.TestDXFContext) {
stepTransition := map[proto.Step]proto.Step{
proto.StepInit: proto.StepOne,
Expand Down
4 changes: 4 additions & 0 deletions pkg/infoschema/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ go_test(
name = "infoschema_test",
timeout = "short",
srcs = [
"bench_test.go",
"infoschema_test.go",
"infoschema_v2_test.go",
"main_test.go",
Expand All @@ -93,16 +94,19 @@ go_test(
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/sessionctx/variable",
"//pkg/store/driver",
"//pkg/table",
"//pkg/testkit",
"//pkg/testkit/testsetup",
"//pkg/testkit/testutil",
"//pkg/types",
"//pkg/util",
"//pkg/util/logutil",
"//pkg/util/set",
"@com_github_pingcap_errors//:errors",
"@com_github_prometheus_prometheus//promql/parser",
"@com_github_stretchr_testify//require",
"@org_uber_go_goleak//:goleak",
"@org_uber_go_zap//:zap",
],
)
77 changes: 77 additions & 0 deletions pkg/infoschema/bench_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
// Copyright 2024 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package infoschema_test

import (
"context"
"flag"
"strconv"
"testing"
"time"

"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/infoschema/internal"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/store/driver"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/stretchr/testify/require"
"go.uber.org/zap"
)

var (
tableCnt = flag.Int("table-cnt", 100, "table count")
version = flag.Int("version", 2, "infoschema version")
port = flag.String("port", "10080", "port of metric server")
)

// test overhead of infoschema
//
// GOOS=linux GOARCH=amd64 go test -tags intest -c -o bench.test ./pkg/infoschema
//
// bench.test -test.v -run ^$ -test.bench=BenchmarkInfoschemaOverhead --with-tikv "upstream-pd:2379?disableGC=true"
func BenchmarkInfoschemaOverhead(b *testing.B) {
wg := testkit.MockTiDBStatusPort(context.Background(), b, *port)

var d driver.TiKVDriver
var err error
store, err := d.Open("tikv://" + *testkit.WithTiKV)
require.NoError(b, err)

re := internal.CreateAutoIDRequirementWithStore(b, store)
defer func() {
err := re.Store().Close()
require.NoError(b, err)
}()

if *version == 2 {
variable.SchemaCacheSize.Store(1000000)
}

tc := &infoschemaTestContext{
t: b,
re: re,
ctx: kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL),
data: infoschema.NewData(),
}
startTime := time.Now()
for j := 0; j < *tableCnt; j++ {
tc.runCreateTable("test" + strconv.Itoa(j))
}
logutil.BgLogger().Info("all table created", zap.Duration("cost time", time.Since(startTime)))
// TODO: add more scenes.
wg.Wait()
}
2 changes: 1 addition & 1 deletion pkg/infoschema/infoschema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -984,7 +984,7 @@ func TestEnableInfoSchemaV2(t *testing.T) {
type infoschemaTestContext struct {
// only test one db.
dbInfo *model.DBInfo
t *testing.T
t testing.TB
re autoid.Requirement
ctx context.Context
data *infoschema.Data
Expand Down
22 changes: 15 additions & 7 deletions pkg/infoschema/internal/testkit.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,14 @@ func CreateAutoIDRequirement(t testing.TB, opts ...mockstore.MockTiKVStoreOption
}
}

// CreateAutoIDRequirementWithStore create autoid requirement with storage for testing.
func CreateAutoIDRequirementWithStore(t testing.TB, store kv.Storage) autoid.Requirement {
return &mockAutoIDRequirement{
store: store,
client: nil,
}
}

// GenGlobalID generates next id globally for testing.
func GenGlobalID(store kv.Storage) (int64, error) {
var globalID int64
Expand All @@ -139,7 +147,7 @@ func GenGlobalID(store kv.Storage) (int64, error) {
}

// MockDBInfo mock DBInfo for testing.
func MockDBInfo(t *testing.T, store kv.Storage, DBName string) *model.DBInfo {
func MockDBInfo(t testing.TB, store kv.Storage, DBName string) *model.DBInfo {
id, err := GenGlobalID(store)
require.NoError(t, err)
return &model.DBInfo{
Expand All @@ -151,7 +159,7 @@ func MockDBInfo(t *testing.T, store kv.Storage, DBName string) *model.DBInfo {
}

// MockTableInfo mock TableInfo for testing.
func MockTableInfo(t *testing.T, store kv.Storage, tblName string) *model.TableInfo {
func MockTableInfo(t testing.TB, store kv.Storage, tblName string) *model.TableInfo {
colID, err := GenGlobalID(store)
require.NoError(t, err)
colInfo := &model.ColumnInfo{
Expand Down Expand Up @@ -211,7 +219,7 @@ func MockPolicyRefInfo(t *testing.T, store kv.Storage, policyName string) *model
}

// AddTable add mock table for testing.
func AddTable(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblInfo *model.TableInfo) {
func AddTable(t testing.TB, store kv.Storage, dbInfo *model.DBInfo, tblInfo *model.TableInfo) {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
err := meta.NewMeta(txn).CreateTableOrView(dbInfo.ID, dbInfo.Name.O, tblInfo)
Expand All @@ -233,7 +241,7 @@ func UpdateTable(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblInfo *
}

// DropTable drop mock table for testing.
func DropTable(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblID int64, tblName string) {
func DropTable(t testing.TB, store kv.Storage, dbInfo *model.DBInfo, tblID int64, tblName string) {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
err := meta.NewMeta(txn).DropTableOrView(dbInfo.ID, dbInfo.Name.O, tblID, tblName)
Expand All @@ -244,7 +252,7 @@ func DropTable(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblID int64
}

// AddDB add mock db for testing.
func AddDB(t *testing.T, store kv.Storage, dbInfo *model.DBInfo) {
func AddDB(t testing.TB, store kv.Storage, dbInfo *model.DBInfo) {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
err := meta.NewMeta(txn).CreateDatabase(dbInfo)
Expand All @@ -255,7 +263,7 @@ func AddDB(t *testing.T, store kv.Storage, dbInfo *model.DBInfo) {
}

// DropDB drop mock db for testing.
func DropDB(t *testing.T, store kv.Storage, dbInfo *model.DBInfo) {
func DropDB(t testing.TB, store kv.Storage, dbInfo *model.DBInfo) {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)

err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
Expand All @@ -267,7 +275,7 @@ func DropDB(t *testing.T, store kv.Storage, dbInfo *model.DBInfo) {
}

// UpdateDB update mock db for testing.
func UpdateDB(t *testing.T, store kv.Storage, dbInfo *model.DBInfo) {
func UpdateDB(t testing.TB, store kv.Storage, dbInfo *model.DBInfo) {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
err := meta.NewMeta(txn).UpdateDatabase(dbInfo)
Expand Down
2 changes: 1 addition & 1 deletion pkg/store/batch_coprocessor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package store
package store_test

import (
"context"
Expand Down
3 changes: 3 additions & 0 deletions pkg/testkit/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -40,10 +40,13 @@ go_library(
"//pkg/util/gctuner",
"//pkg/util/intest",
"//pkg/util/logutil",
"//pkg/util/metricsutil",
"//pkg/util/sqlexec",
"@com_github_gorilla_mux//:mux",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_tipb//go-binlog",
"@com_github_prometheus_client_golang//prometheus/promhttp",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//tikv",
Expand Down
38 changes: 38 additions & 0 deletions pkg/testkit/testkit.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,15 @@ package testkit
import (
"context"
"fmt"
"net"
"net/http"
"net/http/pprof"
"strings"
"sync"
"testing"
"time"

"github.com/gorilla/mux"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/expression"
Expand All @@ -35,10 +39,13 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/testkit/testenv"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/intest"
"github.com/pingcap/tidb/pkg/util/metricsutil"
"github.com/pingcap/tidb/pkg/util/sqlexec"
"github.com/pingcap/tipb/go-binlog"
"github.com/prometheus/client_golang/prometheus/promhttp"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/tikv"
Expand Down Expand Up @@ -698,3 +705,34 @@ func EnableFailPoint(t testing.TB, name, expr string) {
require.NoError(t, failpoint.Disable(name))
})
}

// MockTiDBStatusPort mock the TiDB server status port to have metrics.
func MockTiDBStatusPort(ctx context.Context, b *testing.B, port string) *util.WaitGroupWrapper {
var wg util.WaitGroupWrapper
err := metricsutil.RegisterMetrics()
terror.MustNil(err)
router := mux.NewRouter()
router.Handle("/metrics", promhttp.Handler())
serverMux := http.NewServeMux()
serverMux.Handle("/", router)
serverMux.HandleFunc("/debug/pprof/", pprof.Index)
serverMux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline)
serverMux.HandleFunc("/debug/pprof/profile", pprof.Profile)
serverMux.HandleFunc("/debug/pprof/symbol", pprof.Symbol)
serverMux.HandleFunc("/debug/pprof/trace", pprof.Trace)

statusListener, err := net.Listen("tcp", "0.0.0.0:"+port)
require.NoError(b, err)
statusServer := &http.Server{Handler: serverMux}
wg.RunWithLog(func() {
if err := statusServer.Serve(statusListener); err != nil {
b.Logf("status server serve failed: %v", err)
}
})
wg.RunWithLog(func() {
<-ctx.Done()
_ = statusServer.Close()
})

return &wg
}

0 comments on commit e79a26f

Please sign in to comment.