Skip to content

Commit

Permalink
planner: Move the Selectivity function from the stats package into ca…
Browse files Browse the repository at this point in the history
…rdinality package (pingcap#46359)

ref pingcap#46358
  • Loading branch information
qw4990 authored Aug 23, 2023
1 parent 05aa171 commit 2aa82f5
Show file tree
Hide file tree
Showing 28 changed files with 3,552 additions and 3,317 deletions.
1 change: 1 addition & 0 deletions build/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -338,6 +338,7 @@
},
"lll": {
"exclude_files": {
".*_test\\.go$": "ignore test code",
"/cgo/": "ignore cgo code",
"external/": "no need to vet third party code",
".*_generated\\.go$": "ignore generated code",
Expand Down
70 changes: 70 additions & 0 deletions planner/cardinality/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "cardinality",
srcs = [
"pseudo.go",
"selectivity.go",
"trace.go",
],
importpath = "github.com/pingcap/tidb/planner/cardinality",
visibility = ["//visibility:public"],
deps = [
"//expression",
"//parser/ast",
"//parser/format",
"//parser/mysql",
"//planner/util",
"//planner/util/debugtrace",
"//sessionctx",
"//statistics",
"//types",
"//types/parser_driver",
"//util/chunk",
"//util/logutil",
"//util/ranger",
"//util/tracing",
"@com_github_pingcap_errors//:errors",
"@org_golang_x_exp//maps",
"@org_uber_go_zap//:zap",
],
)

go_test(
name = "cardinality_test",
timeout = "short",
srcs = [
"main_test.go",
"selectivity_test.go",
],
data = glob(["testdata/**"]),
embed = [":cardinality"],
flaky = True,
shard_count = 25,
deps = [
"//config",
"//domain",
"//kv",
"//parser/model",
"//parser/mysql",
"//planner/core",
"//session",
"//sessionctx",
"//sessionctx/stmtctx",
"//sessionctx/variable",
"//statistics",
"//statistics/handle",
"//testkit",
"//testkit/testdata",
"//testkit/testmain",
"//testkit/testsetup",
"//types",
"//util/codec",
"//util/collate",
"//util/mock",
"//util/ranger",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
"@org_uber_go_goleak//:goleak",
],
)
65 changes: 65 additions & 0 deletions planner/cardinality/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
// 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 cardinality

import (
"flag"
"testing"

"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/testkit/testdata"
"github.com/pingcap/tidb/testkit/testmain"
"github.com/pingcap/tidb/testkit/testsetup"
"go.uber.org/goleak"
)

var testDataMap = make(testdata.BookKeeper, 3)

func TestMain(m *testing.M) {
testsetup.SetupForCommonTest()

if !flag.Parsed() {
flag.Parse()
}

config.UpdateGlobal(func(conf *config.Config) {
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
})

testDataMap.LoadTestSuiteData("testdata", "cardinality_suite")

opts := []goleak.Option{
goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"),
goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"),
goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"),
goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"),
}

callback := func(i int) int {
testDataMap.GenerateOutputIfNeeded()
return i
}
goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...)
}

func GetCardinalitySuiteData() testdata.TestData {
return testDataMap["cardinality_suite"]
}

// MockStatsNode is only used for test.
func MockStatsNode(id int64, m int64, num int) *StatsNode {
return &StatsNode{ID: id, mask: m, numCols: num}
}
82 changes: 82 additions & 0 deletions planner/cardinality/pseudo.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
// 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 cardinality

import (
"math"

"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/statistics"
)

const (
pseudoEqualRate = 1000
pseudoLessRate = 3
)

// If one condition can't be calculated, we will assume that the selectivity of this condition is 0.8.
const selectionFactor = 0.8

func pseudoSelectivity(coll *statistics.HistColl, exprs []expression.Expression) float64 {
minFactor := selectionFactor
colExists := make(map[string]bool)
for _, expr := range exprs {
fun, ok := expr.(*expression.ScalarFunction)
if !ok {
continue
}
colID := getConstantColumnID(fun.GetArgs())
if colID == unknownColumnID {
continue
}
switch fun.FuncName.L {
case ast.EQ, ast.NullEQ, ast.In:
minFactor = math.Min(minFactor, 1.0/pseudoEqualRate)
col, ok := coll.Columns[colID]
if !ok {
continue
}
colExists[col.Info.Name.L] = true
if mysql.HasUniKeyFlag(col.Info.GetFlag()) {
return 1.0 / float64(coll.RealtimeCount)
}
case ast.GE, ast.GT, ast.LE, ast.LT:
minFactor = math.Min(minFactor, 1.0/pseudoLessRate)
// FIXME: To resolve the between case.
}
}
if len(colExists) == 0 {
return minFactor
}
// use the unique key info
for _, idx := range coll.Indices {
if !idx.Info.Unique {
continue
}
unique := true
for _, col := range idx.Info.Columns {
if !colExists[col.Name.L] {
unique = false
break
}
}
if unique {
return 1.0 / float64(coll.RealtimeCount)
}
}
return minFactor
}
Loading

0 comments on commit 2aa82f5

Please sign in to comment.