Skip to content

Commit

Permalink
*: Add support for GET_LOCK() / RELEASE_LOCK() / RELEASE_ALL_LOCKS() (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo committed Apr 29, 2022
1 parent 6ea8302 commit 6d8669e
Show file tree
Hide file tree
Showing 18 changed files with 440 additions and 74 deletions.
2 changes: 2 additions & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -826,6 +826,8 @@ const (
ErrInvalidFieldSize = 3013
ErrInvalidArgumentForLogarithm = 3020
ErrAggregateOrderNonAggQuery = 3029
ErrUserLockWrongName = 3057
ErrUserLockDeadlock = 3058
ErrIncorrectType = 3064
ErrFieldInOrderNotSelect = 3065
ErrAggregateInOrderNotSelect = 3066
Expand Down
2 changes: 2 additions & 0 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1048,6 +1048,8 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrLockExpire: mysql.Message("TTL manager has timed out, pessimistic locks may expire, please commit or rollback this transaction", nil),
ErrTableOptionUnionUnsupported: mysql.Message("CREATE/ALTER table with union option is not supported", nil),
ErrTableOptionInsertMethodUnsupported: mysql.Message("CREATE/ALTER table with insert method option is not supported", nil),
ErrUserLockDeadlock: mysql.Message("Deadlock found when trying to get user-level lock; try rolling back transaction/releasing locks and restarting lock acquisition.", nil),
ErrUserLockWrongName: mysql.Message("Incorrect user-level lock name '%s'.", nil),

ErrBRIEBackupFailed: mysql.Message("Backup failed: %s", nil),
ErrBRIERestoreFailed: mysql.Message("Restore failed: %s", nil),
Expand Down
2 changes: 1 addition & 1 deletion executor/infoschema_cluster_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -330,7 +330,7 @@ func (s *infosSchemaClusterTableSuite) TestTableStorageStats() {
"test 2",
))
rows := tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows()
result := 31
result := 32
s.Require().Len(rows, result)

// More tests about the privileges.
Expand Down
14 changes: 1 addition & 13 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1336,28 +1336,16 @@ func TestEnableNoopFunctionsVar(t *testing.T) {
tk.MustQuery(`select @@global.tidb_enable_noop_functions;`).Check(testkit.Rows("OFF"))
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("OFF"))

err := tk.ExecToErr(`select get_lock('lock1', 2);`)
require.True(t, terror.ErrorEqual(err, expression.ErrFunctionsNoopImpl), fmt.Sprintf("err %v", err))
err = tk.ExecToErr(`select release_lock('lock1');`)
require.True(t, terror.ErrorEqual(err, expression.ErrFunctionsNoopImpl), fmt.Sprintf("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("ON"))
tk.MustQuery(`select @@global.tidb_enable_noop_functions;`).Check(testkit.Rows("OFF"))
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("OFF"))
tk.MustQuery(`select @@global.tidb_enable_noop_functions;`).Check(testkit.Rows("OFF"))

err = tk.ExecToErr(`select get_lock('lock2', 10);`)
require.True(t, terror.ErrorEqual(err, expression.ErrFunctionsNoopImpl), fmt.Sprintf("err %v", err))
err = tk.ExecToErr(`select release_lock('lock2');`)
require.True(t, terror.ErrorEqual(err, expression.ErrFunctionsNoopImpl), fmt.Sprintf("err %v", err))

// set test
require.Error(t, tk.ExecToErr(`set tidb_enable_noop_functions='abc'`))
require.Error(t, tk.ExecToErr(`set tidb_enable_noop_functions=11`))
Expand All @@ -1368,7 +1356,7 @@ func TestEnableNoopFunctionsVar(t *testing.T) {
tk.MustExec(`set tidb_enable_noop_functions=0;`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("OFF"))

err = tk.ExecToErr("SET SESSION tx_read_only = 1")
err := tk.ExecToErr("SET SESSION tx_read_only = 1")
require.True(t, terror.ErrorEqual(err, variable.ErrFunctionsNoopImpl), fmt.Sprintf("err %v", err))

tk.MustExec("SET SESSION tx_read_only = 0")
Expand Down
2 changes: 0 additions & 2 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -743,8 +743,6 @@ var funcs = map[string]functionClass{
ast.BinToUUID: &binToUUIDFunctionClass{baseFunctionClass{ast.BinToUUID, 1, 2}},
ast.TiDBShard: &tidbShardFunctionClass{baseFunctionClass{ast.TiDBShard, 1, 1}},

// get_lock() and release_lock() are parsed but do nothing.
// It is used for preventing error in Ruby's activerecord migrations.
ast.GetLock: &lockFunctionClass{baseFunctionClass{ast.GetLock, 2, 2}},
ast.ReleaseLock: &releaseLockFunctionClass{baseFunctionClass{ast.ReleaseLock, 1, 1}},

Expand Down
112 changes: 104 additions & 8 deletions expression/builtin_miscellaneous.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,16 @@ import (
"fmt"
"math"
"net"
"strconv"
"strings"
"time"

"github.com/google/uuid"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -65,6 +69,7 @@ var (
_ builtinFunc = &builtinSleepSig{}
_ builtinFunc = &builtinLockSig{}
_ builtinFunc = &builtinReleaseLockSig{}
_ builtinFunc = &builtinReleaseAllLocksSig{}
_ builtinFunc = &builtinDecimalAnyValueSig{}
_ builtinFunc = &builtinDurationAnyValueSig{}
_ builtinFunc = &builtinIntAnyValueSig{}
Expand Down Expand Up @@ -186,9 +191,55 @@ func (b *builtinLockSig) Clone() builtinFunc {

// evalInt evals a builtinLockSig.
// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_get-lock
// The lock function will do nothing.
// Warning: get_lock() function is parsed but ignored.
func (b *builtinLockSig) evalInt(_ chunk.Row) (int64, bool, error) {
func (b *builtinLockSig) evalInt(row chunk.Row) (int64, bool, error) {
lockName, isNull, err := b.args[0].EvalString(b.ctx, row)
if err != nil {
return 0, isNull, err
}
// Validate that lockName is NOT NULL or empty string
if isNull {
return 0, false, errUserLockWrongName.GenWithStackByArgs("NULL")
}
if lockName == "" || len(lockName) > 64 {
return 0, false, errUserLockWrongName.GenWithStackByArgs(lockName)
}
maxTimeout := int64(variable.GetSysVar(variable.InnodbLockWaitTimeout).MaxValue)
timeout, isNullTimeout, err := b.args[1].EvalInt(b.ctx, row)
if err != nil {
return 0, false, err
}
if isNullTimeout {
timeout = maxTimeout // Observed behavior in MySQL
}
// A timeout less than zero is expected to be treated as unlimited.
// Because of our implementation being based on pessimistic locks,
// We can't have a timeout greater than innodb_lock_wait_timeout.
// So users are aware, we also attach a warning.
if timeout < 0 || timeout > maxTimeout {
err := errTruncatedWrongValue.GenWithStackByArgs("get_lock", strconv.FormatInt(timeout, 10))
b.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
timeout = maxTimeout
}
// Lock names are case insensitive. Because we can't rely on collations
// being enabled on the internal table, we have to lower it.
lockName = strings.ToLower(lockName)
if len(lockName) > 64 {
return 0, false, errIncorrectArgs.GenWithStackByArgs("get_lock")
}
err = b.ctx.GetAdvisoryLock(lockName, timeout)
if err != nil {
switch errors.Cause(err).(*terror.Error).Code() {
case mysql.ErrLockWaitTimeout:
return 0, false, nil // Another user has the lock
case mysql.ErrLockDeadlock:
// Currently this code is not reachable because each Advisory Lock
// Uses a separate session. Deadlock detection does not work across
// independent sessions.
return 0, false, errUserLockDeadlock
default:
return 0, false, err
}
}
return 1, false, nil
}

Expand Down Expand Up @@ -221,10 +272,29 @@ func (b *builtinReleaseLockSig) Clone() builtinFunc {

// evalInt evals a builtinReleaseLockSig.
// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_release-lock
// The release lock function will do nothing.
// Warning: release_lock() function is parsed but ignored.
func (b *builtinReleaseLockSig) evalInt(_ chunk.Row) (int64, bool, error) {
return 1, false, nil
func (b *builtinReleaseLockSig) evalInt(row chunk.Row) (int64, bool, error) {
lockName, isNull, err := b.args[0].EvalString(b.ctx, row)
if err != nil {
return 0, isNull, err
}
// Validate that lockName is NOT NULL or empty string
if isNull {
return 0, false, errUserLockWrongName.GenWithStackByArgs("NULL")
}
if lockName == "" || len(lockName) > 64 {
return 0, false, errUserLockWrongName.GenWithStackByArgs(lockName)
}
// Lock names are case insensitive. Because we can't rely on collations
// being enabled on the internal table, we have to lower it.
lockName = strings.ToLower(lockName)
if len(lockName) > 64 {
return 0, false, errIncorrectArgs.GenWithStackByArgs("release_lock")
}
released := int64(0)
if b.ctx.ReleaseAdvisoryLock(lockName) {
released = 1
}
return released, false, nil
}

type anyValueFunctionClass struct {
Expand Down Expand Up @@ -1070,7 +1140,33 @@ type releaseAllLocksFunctionClass struct {
}

func (c *releaseAllLocksFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) {
return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", "RELEASE_ALL_LOCKS")
if err := c.verifyArgs(args); err != nil {
return nil, err
}
bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt)
if err != nil {
return nil, err
}
sig := &builtinReleaseAllLocksSig{bf}
bf.tp.SetFlen(1)
return sig, nil
}

type builtinReleaseAllLocksSig struct {
baseBuiltinFunc
}

func (b *builtinReleaseAllLocksSig) Clone() builtinFunc {
newSig := &builtinReleaseAllLocksSig{}
newSig.cloneFrom(&b.baseBuiltinFunc)
return newSig
}

// evalInt evals a builtinReleaseAllLocksSig.
// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_release-all-locks
func (b *builtinReleaseAllLocksSig) evalInt(_ chunk.Row) (int64, bool, error) {
count := b.ctx.ReleaseAllAdvisoryLocks()
return int64(count), false, nil
}

type uuidFunctionClass struct {
Expand Down
34 changes: 0 additions & 34 deletions expression/builtin_miscellaneous_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -227,23 +227,6 @@ func (b *builtinNameConstDurationSig) vecEvalDuration(input *chunk.Chunk, result
return b.args[1].VecEvalDuration(b.ctx, input, result)
}

func (b *builtinLockSig) vectorized() bool {
return true
}

// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_get-lock
// The lock function will do nothing.
// Warning: get_lock() function is parsed but ignored.
func (b *builtinLockSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
n := input.NumRows()
result.ResizeInt64(n, false)
i64s := result.Int64s()
for i := range i64s {
i64s[i] = 1
}
return nil
}

func (b *builtinDurationAnyValueSig) vectorized() bool {
return true
}
Expand Down Expand Up @@ -633,23 +616,6 @@ func (b *builtinNameConstRealSig) vecEvalReal(input *chunk.Chunk, result *chunk.
return b.args[1].VecEvalReal(b.ctx, input, result)
}

func (b *builtinReleaseLockSig) vectorized() bool {
return true
}

// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_release-lock
// The release lock function will do nothing.
// Warning: release_lock() function is parsed but ignored.
func (b *builtinReleaseLockSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
n := input.NumRows()
result.ResizeInt64(n, false)
i64s := result.Int64s()
for i := range i64s {
i64s[i] = 1
}
return nil
}

func (b *builtinVitessHashSig) vectorized() bool {
return true
}
Expand Down
4 changes: 2 additions & 2 deletions expression/builtin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,14 +139,14 @@ func TestIsNullFunc(t *testing.T) {
func TestLock(t *testing.T) {
ctx := createContext(t)
lock := funcs[ast.GetLock]
f, err := lock.getFunction(ctx, datumsToConstants(types.MakeDatums(nil, 1)))
f, err := lock.getFunction(ctx, datumsToConstants(types.MakeDatums("mylock", 1)))
require.NoError(t, err)
v, err := evalBuiltinFunc(f, chunk.Row{})
require.NoError(t, err)
require.Equal(t, int64(1), v.GetInt64())

releaseLock := funcs[ast.ReleaseLock]
f, err = releaseLock.getFunction(ctx, datumsToConstants(types.MakeDatums(1)))
f, err = releaseLock.getFunction(ctx, datumsToConstants(types.MakeDatums("mylock")))
require.NoError(t, err)
v, err = evalBuiltinFunc(f, chunk.Row{})
require.NoError(t, err)
Expand Down
2 changes: 2 additions & 0 deletions expression/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,8 @@ var (
errWrongValueForType = dbterror.ClassExpression.NewStd(mysql.ErrWrongValueForType)
errUnknown = dbterror.ClassExpression.NewStd(mysql.ErrUnknown)
errSpecificAccessDenied = dbterror.ClassExpression.NewStd(mysql.ErrSpecificAccessDenied)
errUserLockDeadlock = dbterror.ClassExpression.NewStd(mysql.ErrUserLockDeadlock)
errUserLockWrongName = dbterror.ClassExpression.NewStd(mysql.ErrUserLockWrongName)

// Sequence usage privilege check.
errSequenceAccessDenied = dbterror.ClassExpression.NewStd(mysql.ErrTableaccessDenied)
Expand Down
8 changes: 2 additions & 6 deletions expression/function_traits.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,13 +226,9 @@ var mutableEffectsFunctions = map[string]struct{}{
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)
// some functions 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: {},
}
var noopFuncs = map[string]struct{}{}

// booleanFunctions stores boolean functions
var booleanFunctions = map[string]struct{}{
Expand Down
2 changes: 0 additions & 2 deletions expression/integration_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4181,8 +4181,6 @@ func TestNoopFunctions(t *testing.T) {
"SELECT * FROM t1 LOCK IN SHARE MODE",
"SELECT * FROM t1 GROUP BY a DESC",
"SELECT * FROM t1 GROUP BY a ASC",
"SELECT GET_LOCK('acdc', 10)",
"SELECT RELEASE_LOCK('acdc')",
}

for _, stmt := range stmts {
Expand Down
Loading

0 comments on commit 6d8669e

Please sign in to comment.