Skip to content

Commit

Permalink
*: add tidb_enable_noop_functions to default disable get_lock()/relea…
Browse files Browse the repository at this point in the history
…se_lock() (#10987)

* issue#4100 add new variable to default disable usage of get_lock and release_lock functions
  • Loading branch information
cfzjywxk authored Jul 8, 2019
1 parent 21d2590 commit a737d26
Show file tree
Hide file tree
Showing 12 changed files with 73 additions and 1 deletion.
43 changes: 43 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
. "github.com/pingcap/check"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/testkit"
Expand Down Expand Up @@ -741,3 +742,45 @@ func (s *testSuite2) TestSelectGlobalVar(c *C) {
err = tk.ExecToErr("select @@global.invalid")
c.Assert(terror.ErrorEqual(err, variable.UnknownSystemVar), IsTrue, Commentf("err %v", err))
}

func (s *testSuite2) TestEnableNoopFunctionsVar(c *C) {
tk := testkit.NewTestKit(c, s.store)

// test for tidb_enable_noop_functions
tk.MustQuery(`select @@global.tidb_enable_noop_functions;`).Check(testkit.Rows("0"))
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("0"))

_, err := tk.Exec(`select get_lock('lock1', 2);`)
c.Assert(terror.ErrorEqual(err, expression.ErrFunctionsNoopImpl), IsTrue, Commentf("err %v", err))
_, err = tk.Exec(`select release_lock('lock1');`)
c.Assert(terror.ErrorEqual(err, expression.ErrFunctionsNoopImpl), IsTrue, Commentf("err %v", err))

// change session var to 1
tk.MustExec(`set tidb_enable_noop_functions=1;`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("1"))
tk.MustQuery(`select @@global.tidb_enable_noop_functions;`).Check(testkit.Rows("0"))
tk.MustQuery(`select get_lock("lock", 10)`).Check(testkit.Rows("1"))
tk.MustQuery(`select release_lock("lock")`).Check(testkit.Rows("1"))

// restore to 0
tk.MustExec(`set tidb_enable_noop_functions=0;`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("0"))
tk.MustQuery(`select @@global.tidb_enable_noop_functions;`).Check(testkit.Rows("0"))

_, err = tk.Exec(`select get_lock('lock2', 10);`)
c.Assert(terror.ErrorEqual(err, expression.ErrFunctionsNoopImpl), IsTrue, Commentf("err %v", err))
_, err = tk.Exec(`select release_lock('lock2');`)
c.Assert(terror.ErrorEqual(err, expression.ErrFunctionsNoopImpl), IsTrue, Commentf("err %v", err))

// set test
_, err = tk.Exec(`set tidb_enable_noop_functions='abc'`)
c.Assert(err, NotNil)
_, err = tk.Exec(`set tidb_enable_noop_functions=11`)
c.Assert(err, NotNil)
tk.MustExec(`set tidb_enable_noop_functions="off";`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("off"))
tk.MustExec(`set tidb_enable_noop_functions="on";`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("on"))
tk.MustExec(`set tidb_enable_noop_functions=0;`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("0"))
}
1 change: 1 addition & 0 deletions expression/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ var (
ErrRegexp = terror.ClassExpression.New(mysql.ErrRegexp, mysql.MySQLErrName[mysql.ErrRegexp])
ErrOperandColumns = terror.ClassExpression.New(mysql.ErrOperandColumns, mysql.MySQLErrName[mysql.ErrOperandColumns])
ErrCutValueGroupConcat = terror.ClassExpression.New(mysql.ErrCutValueGroupConcat, mysql.MySQLErrName[mysql.ErrCutValueGroupConcat])
ErrFunctionsNoopImpl = terror.ClassExpression.New(mysql.ErrNotSupportedYet, "function %s has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions")

// All the un-exported errors are defined here:
errFunctionNotExists = terror.ClassExpression.New(mysql.ErrSpDoesNotExist, mysql.MySQLErrName[mysql.ErrSpDoesNotExist])
Expand Down
8 changes: 8 additions & 0 deletions expression/function_traits.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,3 +156,11 @@ var mutableEffectsFunctions = map[string]struct{}{
ast.GetVar: {},
ast.AnyValue: {},
}

// some functions like "get_lock" and "release_lock" currently do NOT have
// right implementations, but may have noop ones(like with any inputs, always return 1)
// if apps really need these "funcs" to run, we offer sys var(tidb_enable_noop_functions) to enable noop usage
var noopFuncs = map[string]struct{}{
ast.GetLock: {},
ast.ReleaseLock: {},
}
1 change: 1 addition & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -202,6 +202,7 @@ func (s *testIntegrationSuite) TestMiscellaneousBuiltin(c *C) {
tk.MustQuery("select a,any_value(b),sum(c) from t1 group by a order by a;").Check(testkit.Rows("1 10 0", "2 30 0"))

// for locks
tk.MustExec(`set tidb_enable_noop_functions=1;`)
result := tk.MustQuery(`SELECT GET_LOCK('test_lock1', 10);`)
result.Check(testkit.Rows("1"))
result = tk.MustQuery(`SELECT GET_LOCK('test_lock2', 10);`)
Expand Down
5 changes: 5 additions & 0 deletions expression/scalar_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,11 @@ func newFunctionImpl(ctx sessionctx.Context, fold bool, funcName string, retType
if !ok {
return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", funcName)
}
if !ctx.GetSessionVars().EnableNoopFuncs {
if _, ok := noopFuncs[funcName]; ok {
return nil, ErrFunctionsNoopImpl.GenWithStackByArgs(funcName)
}
}
funcArgs := make([]Expression, len(args))
copy(funcArgs, args)
f, err := fc.getFunction(ctx, funcArgs)
Expand Down
1 change: 1 addition & 0 deletions expression/typeinfer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,7 @@ func (s *testInferTypeSuite) TestInferType(c *C) {
c_year year
)`
testKit.MustExec(sql)
testKit.MustExec(`set tidb_enable_noop_functions=1;`)

var tests []typeInferTestCase
tests = append(tests, s.createTestCase4Constants()...)
Expand Down
1 change: 1 addition & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1697,6 +1697,7 @@ var builtinGlobalVariable = []string{
variable.TiDBEnableWindowFunction,
variable.TiDBEnableFastAnalyze,
variable.TiDBExpensiveQueryTimeThreshold,
variable.TiDBEnableNoopFuncs,
variable.TiDBEnableIndexMerge,
}

Expand Down
6 changes: 6 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -395,6 +395,9 @@ type SessionVars struct {

// ConnectionInfo indicates current connection info used by current session, only be lazy assigned by plugin.
ConnectionInfo *ConnectionInfo

// use noop funcs or not
EnableNoopFuncs bool
}

// ConnectionInfo present connection used by audit.
Expand Down Expand Up @@ -448,6 +451,7 @@ func NewSessionVars() *SessionVars {
WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish,
WaitSplitRegionTimeout: DefWaitSplitRegionTimeout,
EnableIndexMerge: false,
EnableNoopFuncs: DefTiDBEnableNoopFuncs,
}
vars.Concurrency = Concurrency{
IndexLookupConcurrency: DefIndexLookupConcurrency,
Expand Down Expand Up @@ -824,6 +828,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error {
s.LowResolutionTSO = TiDBOptOn(val)
case TiDBEnableIndexMerge:
s.EnableIndexMerge = TiDBOptOn(val)
case TiDBEnableNoopFuncs:
s.EnableNoopFuncs = TiDBOptOn(val)
}
s.systems[name] = val
return nil
Expand Down
1 change: 1 addition & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -702,6 +702,7 @@ var defaultSysVars = []*SysVar{
{ScopeSession, TiDBWaitSplitRegionTimeout, strconv.Itoa(DefWaitSplitRegionTimeout)},
{ScopeSession, TiDBLowResolutionTSO, "0"},
{ScopeSession, TiDBExpensiveQueryTimeThreshold, strconv.Itoa(DefTiDBExpensiveQueryTimeThreshold)},
{ScopeGlobal | ScopeSession, TiDBEnableNoopFuncs, BoolToIntStr(DefTiDBEnableNoopFuncs)},
}

// SynonymsSysVariables is synonyms of system variables.
Expand Down
4 changes: 4 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -280,6 +280,9 @@ const (

// TiDBEnableIndexMerge indicates to generate IndexMergePath.
TiDBEnableIndexMerge = "tidb_enable_index_merge"

// TiDBEnableNoopFuncs set true will enable using fake funcs(like get_lock release_lock)
TiDBEnableNoopFuncs = "tidb_enable_noop_functions"
)

// Default TiDB system variable values.
Expand Down Expand Up @@ -342,6 +345,7 @@ const (
DefTiDBExpensiveQueryTimeThreshold = 60 // 60s
DefTiDBWaitSplitRegionFinish = true
DefWaitSplitRegionTimeout = 300 // 300s
DefTiDBEnableNoopFuncs = false
)

// Process global variables.
Expand Down
2 changes: 1 addition & 1 deletion sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -419,7 +419,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string,
TiDBOptInSubqToJoinAndAgg, TiDBEnableFastAnalyze,
TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming,
TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction,
TiDBCheckMb4ValueInUTF8, TiDBLowResolutionTSO, TiDBEnableIndexMerge:
TiDBCheckMb4ValueInUTF8, TiDBLowResolutionTSO, TiDBEnableIndexMerge, TiDBEnableNoopFuncs:
if strings.EqualFold(value, "ON") || value == "1" || strings.EqualFold(value, "OFF") || value == "0" {
return value, nil
}
Expand Down
1 change: 1 addition & 0 deletions store/tikv/client_fail_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ func (s *testClientSuite) TestPanicInRecvLoop(c *C) {
time.Sleep(time.Second)
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/panicInFailPendingRequests"), IsNil)
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/gotErrorInRecvLoop"), IsNil)
time.Sleep(time.Second)

req := &tikvrpc.Request{
Type: tikvrpc.CmdEmpty,
Expand Down

0 comments on commit a737d26

Please sign in to comment.