Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

store/copr: set concurrency to partition number for limit statements #41500

Merged
merged 4 commits into from
Feb 17, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 11 additions & 3 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,7 +158,11 @@ func (builder *RequestBuilder) SetDAGRequest(dag *tipb.DAGRequest) *RequestBuild
if len(dag.Executors) == 2 && dag.Executors[1].GetLimit() != nil {
limit := dag.Executors[1].GetLimit()
if limit != nil && limit.Limit < estimatedRegionRowCount {
builder.Request.Concurrency = 1
if kr := builder.Request.KeyRanges; kr != nil {
builder.Request.Concurrency = kr.PartitionNum()
} else {
builder.Request.Concurrency = 1
}
}
builder.Request.LimitSize = limit.GetLimit()
}
Expand Down Expand Up @@ -265,9 +269,13 @@ func (*RequestBuilder) getKVPriority(sv *variable.SessionVars) int {
// SetFromSessionVars sets the following fields for "kv.Request" from session variables:
// "Concurrency", "IsolationLevel", "NotFillCache", "TaskID", "Priority", "ReplicaRead", "ResourceGroupTagger".
func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *RequestBuilder {
distsqlConcurrency := sv.DistSQLScanConcurrency()
if builder.Request.Concurrency == 0 {
// Concurrency may be set to 1 by SetDAGRequest
builder.Request.Concurrency = sv.DistSQLScanConcurrency()
// Concurrency unset.
builder.Request.Concurrency = distsqlConcurrency
} else if builder.Request.Concurrency > distsqlConcurrency {
// Concurrency is set in SetDAGRequest, check the upper limit.
builder.Request.Concurrency = distsqlConcurrency
}
replicaReadType := sv.GetReplicaRead()
if sv.StmtCtx.WeakConsistency {
Expand Down
18 changes: 18 additions & 0 deletions executor/distsqltest/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "distsqltest_test",
srcs = [
"distsql_test.go",
"main_test.go",
],
deps = [
"//config",
"//kv",
"//meta/autoid",
"//testkit",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//tikv",
"@org_uber_go_goleak//:goleak",
],
)
78 changes: 78 additions & 0 deletions executor/distsqltest/distsql_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
// Copyright 2023 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 distsql_test

import (
"context"
"fmt"
"strings"
"testing"

"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
)

func TestDistsqlPartitionTableConcurrency(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2, t3")
tk.MustExec("create table t1(id int primary key , val int)")
partitions := make([]string, 0, 20)
for i := 0; i < 20; i++ {
pid := i + 1
partitions = append(partitions, fmt.Sprintf("PARTITION p%d VALUES LESS THAN (%d00)", pid, pid))
}
tk.MustExec("create table t2(id int primary key, val int)" +
"partition by range(id)" +
"(" + strings.Join(partitions[:10], ",") + ")")
tk.MustExec("create table t3(id int primary key, val int)" +
"partition by range(id)" +
"(" + strings.Join(partitions, ",") + ")")
for i := 0; i < 20; i++ {
for _, tbl := range []string{"t1", "t2", "t3"} {
tk.MustExec(fmt.Sprintf("insert into %s values(%d, %d)", tbl, i*50, i*50))
}
}
tk.MustExec("analyze table t1, t2, t3")
// non-partitioned table checker
ctx1 := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) {
require.Equal(t, req.KeyRanges.PartitionNum(), 1)
require.Equal(t, req.Concurrency, 1)
})
// 10-ranges-partitioned table checker
ctx2 := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) {
require.Equal(t, req.KeyRanges.PartitionNum(), 10)
require.Equal(t, req.Concurrency, 10)
})
// 20-ranges-partitioned table checker
ctx3 := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) {
require.Equal(t, req.KeyRanges.PartitionNum(), 20)
require.Equal(t, req.Concurrency, 15)
})
ctxs := []context.Context{ctx1, ctx2, ctx3}
for i, tbl := range []string{"t1", "t2", "t3"} {
ctx := ctxs[i]
tk.MustQueryWithContext(ctx, fmt.Sprintf("select * from %s order by id asc limit 1", tbl)).
Check(testkit.Rows("0 0"))
tk.MustQueryWithContext(ctx, fmt.Sprintf("select * from %s order by id asc limit 5", tbl)).
Check(testkit.Rows("0 0", "50 50", "100 100", "150 150", "200 200"))
tk.MustQueryWithContext(ctx, fmt.Sprintf("select * from %s order by id desc limit 1", tbl)).
Check(testkit.Rows("950 950"))
tk.MustQueryWithContext(ctx, fmt.Sprintf("select * from %s order by id desc limit 5", tbl)).
Check(testkit.Rows("950 950", "900 900", "850 850", "800 800", "750 750"))
}
}
44 changes: 44 additions & 0 deletions executor/distsqltest/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
// Copyright 2023 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 distsql_test

import (
"testing"

"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/meta/autoid"
"github.com/tikv/client-go/v2/tikv"
"go.uber.org/goleak"
)

func TestMain(m *testing.M) {
autoid.SetStep(5000)
config.UpdateGlobal(func(conf *config.Config) {
conf.Log.SlowThreshold = 30000 // 30s
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
conf.Experimental.AllowsExpressionIndex = true
})
tikv.EnableFailpoints()

opts := []goleak.Option{
goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"),
goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"),
goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"),
goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"),
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"),
}
goleak.VerifyTestMain(m, opts...)
}
14 changes: 8 additions & 6 deletions planner/core/partition_pruner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -545,7 +545,9 @@ func TestListColumnsPartitionPrunerRandom(t *testing.T) {

func TestIssue22635(t *testing.T) {
failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`)
defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune")
defer func() {
failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune")
}()
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("USE test;")
Expand All @@ -557,19 +559,19 @@ CREATE TABLE t1 (
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
PARTITION BY HASH( a )
PARTITIONS 4`)
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows()) // work fine without any error
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt ORDER BY a ASC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows()) // work fine without any error

tk.MustExec("insert into t1 values (1, 1)")
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 1"))
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt ORDER BY a ASC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 1"))

tk.MustExec("insert into t1 values (2, 2), (2, 2)")
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 2"))
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt ORDER BY a ASC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 2"))

tk.MustExec("insert into t1 values (3, 3), (3, 3), (3, 3)")
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 3"))
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt ORDER BY a ASC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 3"))

tk.MustExec("insert into t1 values (4, 4), (4, 4), (4, 4), (4, 4)")
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("4 4"))
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt ORDER BY a DESC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("4 4"))
}

func TestIssue22898(t *testing.T) {
Expand Down
7 changes: 7 additions & 0 deletions store/copr/coprocessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,7 +217,14 @@ func (c *CopClient) BuildCopIterator(ctx context.Context, req *kv.Request, vars
// For ReqTypeAnalyze, we keep its concurrency to avoid slow analyze(see https://github.com/pingcap/tidb/issues/40162 for details).
if it.concurrency > 2 && it.req.Tp != kv.ReqTypeAnalyze {
oldConcurrency := it.concurrency
partitionNum := req.KeyRanges.PartitionNum()
if partitionNum > it.concurrency {
partitionNum = it.concurrency
}
it.concurrency = 2
if it.concurrency < partitionNum {
it.concurrency = partitionNum
}

failpoint.Inject("testRateLimitActionMockConsumeAndAssert", func(val failpoint.Value) {
if val.(bool) {
Expand Down