Skip to content

Commit

Permalink
*: make expression index as experimental feature (pingcap#28307)
Browse files Browse the repository at this point in the history
  • Loading branch information
wjhuang2016 authored Oct 29, 2021
1 parent 83e559d commit 9dc4630
Show file tree
Hide file tree
Showing 15 changed files with 91 additions and 20 deletions.
6 changes: 6 additions & 0 deletions br/tests/config/tidb.toml
Original file line number Diff line number Diff line change
Expand Up @@ -11,3 +11,9 @@ ssl-key = "/tmp/backup_restore_test/certs/tidb.key"
cluster-ssl-ca = "/tmp/backup_restore_test/certs/ca.pem"
cluster-ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem"
cluster-ssl-key = "/tmp/backup_restore_test/certs/tidb.key"

# experimental section controls the features that are still experimental: their semantics,
# interfaces are subject to change, using these features in the production environment is not recommended.
[experimental]
# enable creating expression index.
allow-expression-index = true
1 change: 1 addition & 0 deletions cmd/explaintest/config.toml
Original file line number Diff line number Diff line change
Expand Up @@ -25,3 +25,4 @@ stats-lease = "0"

[experimental]
enable-new-charset = true
allow-expression-index = true
3 changes: 3 additions & 0 deletions cmd/explaintest/r/explain_generate_column_substitute.result
Original file line number Diff line number Diff line change
Expand Up @@ -520,3 +520,6 @@ select * from t02 use index(eidx) where lower(a) < 'c';
a
a
b
select @@tidb_allow_function_for_expression_index;
@@tidb_allow_function_for_expression_index
lower, md5, reverse, upper, vitess_hash
2 changes: 2 additions & 0 deletions cmd/explaintest/t/explain_generate_column_substitute.test
Original file line number Diff line number Diff line change
Expand Up @@ -232,3 +232,5 @@ insert into t02 values ('a'), ('b'), ('c');
select * from t02 where lower(a) < 'c';
create index eidx on t02 ((lower(a)));
select * from t02 use index(eidx) where lower(a) < 'c';

select @@tidb_allow_function_for_expression_index;
31 changes: 16 additions & 15 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -561,6 +561,8 @@ type IsolationRead struct {
// Experimental controls the features that are still experimental: their semantics, interfaces are subject to change.
// Using these features in the production environment is not recommended.
type Experimental struct {
// Whether enable creating expression index.
AllowsExpressionIndex bool `toml:"allow-expression-index" json:"allow-expression-index"`
// Whether enable global kill.
EnableGlobalKill bool `toml:"enable-global-kill" json:"-"`
// Whether enable charset feature.
Expand Down Expand Up @@ -732,21 +734,20 @@ func StoreGlobalConfig(config *Config) {
}

var deprecatedConfig = map[string]struct{}{
"pessimistic-txn.ttl": {},
"pessimistic-txn.enable": {},
"log.file.log-rotate": {},
"log.log-slow-query": {},
"txn-local-latches": {},
"txn-local-latches.enabled": {},
"txn-local-latches.capacity": {},
"performance.max-memory": {},
"max-txn-time-use": {},
"experimental.allow-auto-random": {},
"enable-redact-log": {}, // use variable tidb_redact_log instead
"tikv-client.copr-cache.enable": {},
"alter-primary-key": {}, // use NONCLUSTERED keyword instead
"enable-streaming": {},
"experimental.allow-expression-index": {},
"pessimistic-txn.ttl": {},
"pessimistic-txn.enable": {},
"log.file.log-rotate": {},
"log.log-slow-query": {},
"txn-local-latches": {},
"txn-local-latches.enabled": {},
"txn-local-latches.capacity": {},
"performance.max-memory": {},
"max-txn-time-use": {},
"experimental.allow-auto-random": {},
"enable-redact-log": {}, // use variable tidb_redact_log instead
"tikv-client.copr-cache.enable": {},
"alter-primary-key": {}, // use NONCLUSTERED keyword instead
"enable-streaming": {},
}

func isAllDeprecatedConfigItems(items []string) bool {
Expand Down
2 changes: 2 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -483,6 +483,8 @@ history-size = 24
# experimental section controls the features that are still experimental: their semantics,
# interfaces are subject to change, using these features in the production environment is not recommended.
[experimental]
# enable creating expression index.
allow-expression-index = false

# server level isolation read by engines and labels
[isolation-read]
Expand Down
2 changes: 2 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,7 @@ max-sql-length=1024
refresh-interval=100
history-size=100
[experimental]
allow-expression-index = true
[isolation-read]
engines = ["tiflash"]
[labels]
Expand Down Expand Up @@ -292,6 +293,7 @@ receiver-address = "127.0.0.1:10100"
require.True(t, conf.PessimisticTxn.DeadlockHistoryCollectRetryable)
require.False(t, conf.Experimental.EnableNewCharset)
require.Equal(t, "127.0.0.1:10100", conf.TopSQL.ReceiverAddress)
require.True(t, conf.Experimental.AllowsExpressionIndex)

_, err = f.WriteString(`
[log.file]
Expand Down
12 changes: 12 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2326,6 +2326,18 @@ func (s *testSerialDBSuite1) TestAddExpressionIndex(c *C) {
tk.MustExec("create table t(a int, key((a+1)), key((a+2)), key idx((a+3)), key((a+4)));")
tk.MustExec("drop table if exists t")
tk.MustExec("CREATE TABLE t (A INT, B INT, UNIQUE KEY ((A * 2)));")

// Test experiment switch.
config.UpdateGlobal(func(conf *config.Config) {
conf.Experimental.AllowsExpressionIndex = false
})
tk.MustGetErrMsg("create index d on t((repeat(a, 2)))", "[ddl:8200]Unsupported creating expression index containing unsafe functions without allow-expression-index in config")
tk.MustGetErrMsg("create table t(a char(10), key ((repeat(a, 2))));", "[ddl:8200]Unsupported creating expression index containing unsafe functions without allow-expression-index in config")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a char(10), key ((lower(a))))")
config.UpdateGlobal(func(conf *config.Config) {
conf.Experimental.AllowsExpressionIndex = true
})
}

func (s *testSerialDBSuite1) TestCreateExpressionIndexError(c *C) {
Expand Down
1 change: 1 addition & 0 deletions ddl/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ func TestT(t *testing.T) {
conf.Log.SlowThreshold = 10000
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
conf.Experimental.AllowsExpressionIndex = true
})
tikv.EnableFailpoints()

Expand Down
2 changes: 2 additions & 0 deletions ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,6 +235,8 @@ var (
ErrColumnTypeUnsupportedNextValue = dbterror.ClassDDL.NewStd(mysql.ErrColumnTypeUnsupportedNextValue)
// ErrAddColumnWithSequenceAsDefault is returned when the new added column with sequence's nextval as it's default value.
ErrAddColumnWithSequenceAsDefault = dbterror.ClassDDL.NewStd(mysql.ErrAddColumnWithSequenceAsDefault)
// ErrUnsupportedExpressionIndex is returned when create an expression index without allow-expression-index.
ErrUnsupportedExpressionIndex = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "creating expression index containing unsafe functions without allow-expression-index in config"), nil))
// ErrPartitionExchangePartTable is returned when exchange table partition with another table is partitioned.
ErrPartitionExchangePartTable = dbterror.ClassDDL.NewStd(mysql.ErrPartitionExchangePartTable)
// ErrTablesDifferentMetadata is returned when exchanges tables is not compatible.
Expand Down
20 changes: 15 additions & 5 deletions ddl/generated_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,13 @@ import (
"fmt"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
)

Expand Down Expand Up @@ -265,11 +267,12 @@ func checkModifyGeneratedColumn(sctx sessionctx.Context, tbl table.Table, oldCol
}

type illegalFunctionChecker struct {
hasIllegalFunc bool
hasAggFunc bool
hasRowVal bool // hasRowVal checks whether the functional index refers to a row value
hasWindowFunc bool
otherErr error
hasIllegalFunc bool
hasAggFunc bool
hasRowVal bool // hasRowVal checks whether the functional index refers to a row value
hasWindowFunc bool
hasNotGAFunc4ExprIdx bool
otherErr error
}

func (c *illegalFunctionChecker) Enter(inNode ast.Node) (outNode ast.Node, skipChildren bool) {
Expand All @@ -286,6 +289,10 @@ func (c *illegalFunctionChecker) Enter(inNode ast.Node) (outNode ast.Node, skipC
c.otherErr = err
return inNode, true
}
_, isFuncGA := variable.GAFunction4ExpressionIndex[node.FnName.L]
if !isFuncGA {
c.hasNotGAFunc4ExprIdx = true
}
case *ast.SubqueryExpr, *ast.ValuesExpr, *ast.VariableExpr:
// Subquery & `values(x)` & variable is not allowed
c.hasIllegalFunc = true
Expand Down Expand Up @@ -344,6 +351,9 @@ func checkIllegalFn4Generated(name string, genType int, expr ast.ExprNode) error
if c.otherErr != nil {
return c.otherErr
}
if genType == typeIndex && c.hasNotGAFunc4ExprIdx && !config.GetGlobalConfig().Experimental.AllowsExpressionIndex {
return ErrUnsupportedExpressionIndex
}
return nil
}

Expand Down
1 change: 1 addition & 0 deletions executor/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ func TestMain(m *testing.M) {
conf.Log.SlowThreshold = 30000 // 30s
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
conf.Experimental.AllowsExpressionIndex = true
})
tikv.EnableFailpoints()
tmpDir := config.GetGlobalConfig().TempStoragePath
Expand Down
1 change: 1 addition & 0 deletions expression/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ func TestMain(m *testing.M) {
config.UpdateGlobal(func(conf *config.Config) {
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
conf.Experimental.AllowsExpressionIndex = true
})
tikv.EnableFailpoints()

Expand Down
4 changes: 4 additions & 0 deletions planner/core/plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"strings"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/model"
Expand Down Expand Up @@ -593,6 +594,9 @@ func (s *testPlanNormalize) BenchmarkEncodePlan(c *C) {

// Close issue 25729
func (s *testPlanNormalize) TestIssue25729(c *C) {
config.UpdateGlobal(func(conf *config.Config) {
conf.Experimental.AllowsExpressionIndex = true
})
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists tt")
Expand Down
23 changes: 23 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"fmt"
"math"
"runtime"
"sort"
"strconv"
"strings"
"sync"
Expand All @@ -29,6 +30,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -1835,6 +1837,25 @@ var defaultSysVars = []*SysVar{

{Scope: ScopeNone, Name: "version_compile_os", Value: runtime.GOOS},
{Scope: ScopeNone, Name: "version_compile_machine", Value: runtime.GOARCH},
{Scope: ScopeNone, Name: TiDBAllowFunctionForExpressionIndex, ReadOnly: true, Value: collectAllowFuncName4ExpressionIndex()},
}

func collectAllowFuncName4ExpressionIndex() string {
var str []string
for funcName := range GAFunction4ExpressionIndex {
str = append(str, funcName)
}
sort.Strings(str)
return strings.Join(str, ", ")
}

// GAFunction4ExpressionIndex stores functions GA for expression index.
var GAFunction4ExpressionIndex = map[string]struct{}{
ast.Lower: {},
ast.Upper: {},
ast.MD5: {},
ast.Reverse: {},
ast.VitessHash: {},
}

// FeedbackProbability points to the FeedbackProbability in statistics package.
Expand Down Expand Up @@ -2137,6 +2158,8 @@ const (
LastInsertID = "last_insert_id"
// Identity is the name of 'identity' system variable.
Identity = "identity"
// TiDBAllowFunctionForExpressionIndex is the name of `TiDBAllowFunctionForExpressionIndex` system variable.
TiDBAllowFunctionForExpressionIndex = "tidb_allow_function_for_expression_index"
)

// GlobalVarAccessor is the interface for accessing global scope system and status variables.
Expand Down

0 comments on commit 9dc4630

Please sign in to comment.