Skip to content

Commit

Permalink
test: add unit test for estimateTableRowSize (#52144)
Browse files Browse the repository at this point in the history
ref #52085
  • Loading branch information
tangenta committed Apr 3, 2024
1 parent 923b071 commit 9f505ef
Show file tree
Hide file tree
Showing 3 changed files with 80 additions and 3 deletions.
6 changes: 3 additions & 3 deletions pkg/ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -2214,6 +2214,9 @@ func (w *worker) executeDistTask(t table.Table, reorgInfo *reorgInfo) error {
return err
}

// EstimateTableRowSizeForTest is used for test.
var EstimateTableRowSizeForTest = estimateTableRowSize

// estimateTableRowSize estimates the row size in bytes of a table.
// This function tries to retrieve row size in following orders:
// 1. AVG_ROW_LENGTH column from information_schema.tables.
Expand Down Expand Up @@ -2266,9 +2269,6 @@ func estimateRowSizeFromRegion(ctx context.Context, store kv.Storage, tbl table.
return 0, err
}
pid := tbl.Meta().ID
if part := tbl.GetPartitionedTable(); part != nil {
pid = part.Meta().ID
}
sk, ek := tablecodec.GetTableHandleKeyRange(pid)
sRegion, err := pdCli.GetRegionByKey(ctx, codec.EncodeBytes(nil, sk))
if err != nil {
Expand Down
2 changes: 2 additions & 0 deletions tests/realtikvtest/addindextest3/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ go_test(
name = "addindextest3_test",
timeout = "long",
srcs = [
"functional_test.go",
"main_test.go",
"operator_test.go",
],
Expand All @@ -26,6 +27,7 @@ go_test(
"@com_github_ngaut_pools//:pools",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//util",
"@org_golang_x_sync//errgroup",
],
)
75 changes: 75 additions & 0 deletions tests/realtikvtest/addindextest3/functional_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
// 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 addindextest

import (
"context"
"fmt"
"testing"

"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/tests/realtikvtest"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/util"
)

func TestDDLTestEstimateTableRowSize(t *testing.T) {
store, dom := realtikvtest.CreateMockStoreAndDomainAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
tk.MustExec("create table t (a int, b int);")
tk.MustExec("insert into t values (1, 1);")

ctx := context.Background()
ctx = util.WithInternalSourceType(ctx, "estimate_row_size")
tkSess := tk.Session()
exec := tkSess.GetRestrictedSQLExecutor()
tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)

size := ddl.EstimateTableRowSizeForTest(ctx, store, exec, tbl)
require.Equal(t, 0, size) // No data in information_schema.columns.
tk.MustExec("analyze table t;")
size = ddl.EstimateTableRowSizeForTest(ctx, store, exec, tbl)
require.Equal(t, 16, size)

tk.MustExec("alter table t add column c varchar(255);")
tk.MustExec("update t set c = repeat('a', 50) where a = 1;")
tk.MustExec("analyze table t;")
size = ddl.EstimateTableRowSizeForTest(ctx, store, exec, tbl)
require.Equal(t, 67, size)

tk.MustExec("drop table t;")
tk.MustExec("create table t (id bigint primary key, b text) partition by hash(id) partitions 4;")
for i := 1; i < 10; i++ {
insertSQL := fmt.Sprintf("insert into t values (%d, repeat('a', 10))", i*10000)
tk.MustExec(insertSQL)
}
tk.MustQuery("split table t between (0) and (1000000) regions 2;").Check(testkit.Rows("4 1"))
tk.MustExec("analyze table t;")
tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
size = ddl.EstimateTableRowSizeForTest(ctx, store, exec, tbl)
require.Equal(t, 19, size)
ptbl := tbl.GetPartitionedTable()
pids := ptbl.GetAllPartitionIDs()
for _, pid := range pids {
partition := ptbl.GetPartition(pid)
size = ddl.EstimateTableRowSizeForTest(ctx, store, exec, partition)
require.Equal(t, 19, size)
}
}

0 comments on commit 9f505ef

Please sign in to comment.