Skip to content

Commit

Permalink
Merge remote-tracking branch 'upstream/master' into txn-kv-filter
Browse files Browse the repository at this point in the history
  • Loading branch information
sticnarf committed Apr 15, 2021
2 parents 8d637ef + cf813dc commit bdcb96f
Show file tree
Hide file tree
Showing 7 changed files with 186 additions and 50 deletions.
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1616,6 +1616,11 @@ error = '''
Incorrect argument type to variable '%-.64s'
'''

["variable:1235"]
error = '''
function %s has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions
'''

["variable:1238"]
error = '''
Variable '%-.192s' is a %s variable
Expand Down
120 changes: 81 additions & 39 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (

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

testSQL := "SET @a = 1;"
tk.MustExec(testSQL)

Expand Down Expand Up @@ -214,34 +215,6 @@ func (s *testSerialSuite1) TestSetVar(c *C) {
tk.MustQuery("select @@global.tx_isolation").Check(testkit.Rows("REPEATABLE-READ"))
tk.MustQuery("select @@global.transaction_isolation").Check(testkit.Rows("REPEATABLE-READ"))

tk.MustExec("SET SESSION tx_read_only = 1")
tk.MustExec("SET SESSION tx_read_only = 0")
tk.MustQuery("select @@session.tx_read_only").Check(testkit.Rows("0"))
tk.MustQuery("select @@session.transaction_read_only").Check(testkit.Rows("0"))

tk.MustExec("SET GLOBAL tx_read_only = 1")
tk.MustExec("SET GLOBAL tx_read_only = 0")
tk.MustQuery("select @@global.tx_read_only").Check(testkit.Rows("0"))
tk.MustQuery("select @@global.transaction_read_only").Check(testkit.Rows("0"))

tk.MustExec("SET SESSION transaction_read_only = 1")
tk.MustExec("SET SESSION transaction_read_only = 0")
tk.MustQuery("select @@session.tx_read_only").Check(testkit.Rows("0"))
tk.MustQuery("select @@session.transaction_read_only").Check(testkit.Rows("0"))

tk.MustExec("SET SESSION transaction_read_only = 1")
tk.MustQuery("select @@session.tx_read_only").Check(testkit.Rows("1"))
tk.MustQuery("select @@session.transaction_read_only").Check(testkit.Rows("1"))

tk.MustExec("SET GLOBAL transaction_read_only = 1")
tk.MustExec("SET GLOBAL transaction_read_only = 0")
tk.MustQuery("select @@global.tx_read_only").Check(testkit.Rows("0"))
tk.MustQuery("select @@global.transaction_read_only").Check(testkit.Rows("0"))

tk.MustExec("SET GLOBAL transaction_read_only = 1")
tk.MustQuery("select @@global.tx_read_only").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.transaction_read_only").Check(testkit.Rows("1"))

// Even the transaction fail, set session variable would success.
tk.MustExec("BEGIN")
tk.MustExec("SET SESSION TRANSACTION ISOLATION LEVEL READ COMMITTED")
Expand Down Expand Up @@ -345,17 +318,6 @@ func (s *testSerialSuite1) TestSetVar(c *C) {
tk.MustExec("SET GLOBAL tidb_skip_isolation_level_check = 0")
tk.MustExec("SET SESSION tidb_skip_isolation_level_check = 0")

tk.MustExec("set global read_only = 0")
tk.MustQuery("select @@global.read_only;").Check(testkit.Rows("0"))
tk.MustExec("set global read_only = off")
tk.MustQuery("select @@global.read_only;").Check(testkit.Rows("0"))
tk.MustExec("set global read_only = 1")
tk.MustQuery("select @@global.read_only;").Check(testkit.Rows("1"))
tk.MustExec("set global read_only = on")
tk.MustQuery("select @@global.read_only;").Check(testkit.Rows("1"))
_, err = tk.Exec("set global read_only = abc")
c.Assert(err, NotNil)

// test for tidb_wait_split_region_finish
tk.MustQuery(`select @@session.tidb_wait_split_region_finish;`).Check(testkit.Rows("1"))
tk.MustExec("set tidb_wait_split_region_finish = 1")
Expand Down Expand Up @@ -1200,6 +1162,16 @@ func (s *testSuite5) TestSetConcurrency(c *C) {
func (s *testSuite5) TestEnableNoopFunctionsVar(c *C) {
tk := testkit.NewTestKit(c, s.store)

defer func() {
// Ensure global settings are reset.
tk.MustExec("SET GLOBAL tx_read_only = 0")
tk.MustExec("SET GLOBAL transaction_read_only = 0")
tk.MustExec("SET GLOBAL read_only = 0")
tk.MustExec("SET GLOBAL super_read_only = 0")
tk.MustExec("SET GLOBAL offline_mode = 0")
tk.MustExec("SET GLOBAL tidb_enable_noop_functions = 0")
}()

// 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"))
Expand Down Expand Up @@ -1237,6 +1209,76 @@ func (s *testSuite5) TestEnableNoopFunctionsVar(c *C) {
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("1"))
tk.MustExec(`set tidb_enable_noop_functions=0;`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("0"))

_, err = tk.Exec("SET SESSION tx_read_only = 1")
c.Assert(terror.ErrorEqual(err, variable.ErrFunctionsNoopImpl), IsTrue, Commentf("err %v", err))

tk.MustExec("SET SESSION tx_read_only = 0")
tk.MustQuery("select @@session.tx_read_only").Check(testkit.Rows("0"))
tk.MustQuery("select @@session.transaction_read_only").Check(testkit.Rows("0"))

_, err = tk.Exec("SET GLOBAL tx_read_only = 1") // should fail.
c.Assert(terror.ErrorEqual(err, variable.ErrFunctionsNoopImpl), IsTrue, Commentf("err %v", err))
tk.MustExec("SET GLOBAL tx_read_only = 0")
tk.MustQuery("select @@global.tx_read_only").Check(testkit.Rows("0"))
tk.MustQuery("select @@global.transaction_read_only").Check(testkit.Rows("0"))

_, err = tk.Exec("SET SESSION transaction_read_only = 1")
c.Assert(terror.ErrorEqual(err, variable.ErrFunctionsNoopImpl), IsTrue, Commentf("err %v", err))
tk.MustExec("SET SESSION transaction_read_only = 0")
tk.MustQuery("select @@session.tx_read_only").Check(testkit.Rows("0"))
tk.MustQuery("select @@session.transaction_read_only").Check(testkit.Rows("0"))

// works on SESSION because SESSION tidb_enable_noop_functions=1
tk.MustExec("SET tidb_enable_noop_functions = 1")
tk.MustExec("SET SESSION transaction_read_only = 1")
tk.MustQuery("select @@session.tx_read_only").Check(testkit.Rows("1"))
tk.MustQuery("select @@session.transaction_read_only").Check(testkit.Rows("1"))

// fails on GLOBAL because GLOBAL.tidb_enable_noop_functions still=0
_, err = tk.Exec("SET GLOBAL transaction_read_only = 1")
c.Assert(terror.ErrorEqual(err, variable.ErrFunctionsNoopImpl), IsTrue, Commentf("err %v", err))
tk.MustExec("SET GLOBAL tidb_enable_noop_functions = 1")
// now works
tk.MustExec("SET GLOBAL transaction_read_only = 1")
tk.MustQuery("select @@global.tx_read_only").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.transaction_read_only").Check(testkit.Rows("1"))
tk.MustExec("SET GLOBAL transaction_read_only = 0")
tk.MustQuery("select @@global.tx_read_only").Check(testkit.Rows("0"))
tk.MustQuery("select @@global.transaction_read_only").Check(testkit.Rows("0"))

_, err = tk.Exec("SET tidb_enable_noop_functions = 0") // fails because transaction_read_only/tx_read_only = 1
c.Assert(err, NotNil)

tk.MustExec("SET transaction_read_only = 0")
tk.MustExec("SET tidb_enable_noop_functions = 0") // now works.

// setting session doesn't change global, which succeeds because global.transaction_read_only/tx_read_only = 0
tk.MustExec("SET GLOBAL tidb_enable_noop_functions = 0")

// but if global.transaction_read_only=1, it would fail
tk.MustExec("SET GLOBAL tidb_enable_noop_functions = 1")
tk.MustExec("SET GLOBAL transaction_read_only = 1")
// fails
_, err = tk.Exec("SET GLOBAL tidb_enable_noop_functions = 0")
c.Assert(err, NotNil)

// reset for rest of tests.
tk.MustExec("SET GLOBAL transaction_read_only = 0")
tk.MustExec("SET GLOBAL tidb_enable_noop_functions = 0")

tk.MustExec("set global read_only = 0")
tk.MustQuery("select @@global.read_only;").Check(testkit.Rows("0"))
tk.MustExec("set global read_only = off")
tk.MustQuery("select @@global.read_only;").Check(testkit.Rows("0"))
tk.MustExec("SET global tidb_enable_noop_functions = 1")
tk.MustExec("set global read_only = 1")
tk.MustQuery("select @@global.read_only;").Check(testkit.Rows("1"))
tk.MustExec("set global read_only = on")
tk.MustQuery("select @@global.read_only;").Check(testkit.Rows("1"))
_, err = tk.Exec("set global read_only = abc")
c.Assert(err, NotNil)

}

func (s *testSuite5) TestSetClusterConfig(c *C) {
Expand Down
11 changes: 9 additions & 2 deletions executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/distsql"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
Expand Down Expand Up @@ -570,8 +571,14 @@ func (e *SimpleExec) executeUse(s *ast.UseStmt) error {
func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error {
// If `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND` is the first statement in TxnCtx, we should
// always create a new Txn instead of reusing it.
if s.ReadOnly && s.Bound != nil {
return e.executeStartTransactionReadOnlyWithTimestampBound(ctx, s)
if s.ReadOnly {
enableNoopFuncs := e.ctx.GetSessionVars().EnableNoopFuncs
if !enableNoopFuncs && s.Bound == nil {
return expression.ErrFunctionsNoopImpl.GenWithStackByArgs("READ ONLY")
}
if s.Bound != nil {
return e.executeStartTransactionReadOnlyWithTimestampBound(ctx, s)
}
}

// If BEGIN is the first statement in TxnCtx, we can reuse the existing transaction, without the
Expand Down
18 changes: 17 additions & 1 deletion expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4545,11 +4545,23 @@ func (s *testIntegrationSuite) TestSetVariables(c *C) {
r.Check(testkit.Rows("0 0 0 0"))

_, err = tk.Exec("set session transaction read only;")
c.Assert(err, NotNil)

_, err = tk.Exec("start transaction read only;")
c.Assert(err, NotNil)

_, err = tk.Exec("set tidb_enable_noop_functions=1")
c.Assert(err, IsNil)

tk.MustExec("set session transaction read only;")
tk.MustExec("start transaction read only;")

r = tk.MustQuery(`select @@session.tx_read_only, @@global.tx_read_only, @@session.transaction_read_only, @@global.transaction_read_only;`)
r.Check(testkit.Rows("1 0 1 0"))
_, err = tk.Exec("set global transaction read only;")
c.Assert(err, IsNil)
c.Assert(err, NotNil)
tk.MustExec("set global tidb_enable_noop_functions=1;")
tk.MustExec("set global transaction read only;")
r = tk.MustQuery(`select @@session.tx_read_only, @@global.tx_read_only, @@session.transaction_read_only, @@global.transaction_read_only;`)
r.Check(testkit.Rows("1 1 1 1"))

Expand All @@ -4560,6 +4572,10 @@ func (s *testIntegrationSuite) TestSetVariables(c *C) {
r = tk.MustQuery(`select @@session.tx_read_only, @@global.tx_read_only, @@session.transaction_read_only, @@global.transaction_read_only;`)
r.Check(testkit.Rows("0 0 0 0"))

// reset
tk.MustExec("set tidb_enable_noop_functions=0")
tk.MustExec("set global tidb_enable_noop_functions=1")

_, err = tk.Exec("set @@global.max_user_connections='';")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, variable.ErrWrongTypeForVar.GenWithStackByArgs("max_user_connections").Error())
Expand Down
21 changes: 18 additions & 3 deletions sessionctx/variable/noop.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,24 @@ import (
// but changing them has no effect on behavior.

var noopSysVars = []*SysVar{
// It is unsafe to pretend that any variation of "read only" is enabled when the server
// does not support it. It is possible that these features will be supported in future,
// but until then...
{Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkReadOnly(vars, normalizedValue, originalValue, scope, false)
}},
{Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkReadOnly(vars, normalizedValue, originalValue, scope, false)
}},
{Scope: ScopeGlobal, Name: OfflineMode, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkReadOnly(vars, normalizedValue, originalValue, scope, true)
}},
{Scope: ScopeGlobal, Name: SuperReadOnly, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkReadOnly(vars, normalizedValue, originalValue, scope, false)
}},
{Scope: ScopeGlobal, Name: serverReadOnly, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkReadOnly(vars, normalizedValue, originalValue, scope, false)
}},
{Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10", Type: TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear, AutoConvertOutOfRange: true},
{Scope: ScopeGlobal | ScopeSession, Name: QueryCacheWlockInvalidate, Value: BoolOff, Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: "sql_buffer_result", Value: BoolOff, IsHintUpdatable: true},
Expand Down Expand Up @@ -83,7 +101,6 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "innodb_ft_sort_pll_degree", Value: "2"},
{Scope: ScopeNone, Name: "thread_stack", Value: "262144"},
{Scope: ScopeGlobal, Name: "relay_log_info_repository", Value: "FILE"},
{Scope: ScopeGlobal, Name: SuperReadOnly, Value: "0", Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: "max_delayed_threads", Value: "20"},
{Scope: ScopeNone, Name: "protocol_version", Value: "10"},
{Scope: ScopeGlobal | ScopeSession, Name: "new", Value: BoolOff},
Expand Down Expand Up @@ -124,7 +141,6 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: "innodb_file_format_max", Value: "Antelope"},
{Scope: ScopeGlobal | ScopeSession, Name: "debug", Value: ""},
{Scope: ScopeGlobal, Name: "log_warnings", Value: "1"},
{Scope: ScopeGlobal, Name: OfflineMode, Value: "0", Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: InnodbStrictMode, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true},
{Scope: ScopeGlobal, Name: "innodb_rollback_segments", Value: "128"},
{Scope: ScopeGlobal | ScopeSession, Name: "join_buffer_size", Value: "262144", IsHintUpdatable: true},
Expand Down Expand Up @@ -286,7 +302,6 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: LogBinTrustFunctionCreators, Value: BoolOff, Type: TypeBool},
{Scope: ScopeNone, Name: "innodb_write_io_threads", Value: "4"},
{Scope: ScopeGlobal, Name: "mysql_native_password_proxy_users", Value: ""},
{Scope: ScopeGlobal, Name: serverReadOnly, Value: BoolOff, Type: TypeBool},
{Scope: ScopeNone, Name: "large_page_size", Value: "0"},
{Scope: ScopeNone, Name: "table_open_cache_instances", Value: "1"},
{Scope: ScopeGlobal, Name: InnodbStatsPersistent, Value: BoolOn, Type: TypeBool, AutoConvertNegativeBool: true},
Expand Down
40 changes: 35 additions & 5 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,14 @@ import (
"github.com/cznic/mathutil"
"github.com/pingcap/errors"
"github.com/pingcap/parser/mysql"
pmysql "github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/versioninfo"
atomic2 "go.uber.org/atomic"
Expand Down Expand Up @@ -427,8 +429,16 @@ func GetSysVars() map[string]*SysVar {
return sysVars
}

// PluginVarNames is global plugin var names set.
var PluginVarNames []string
var (
// PluginVarNames is global plugin var names set.
PluginVarNames []string
errUnknownSystemVariable = dbterror.ClassVariable.NewStd(mysql.ErrUnknownSystemVariable)
errValueNotSupportedWhen = dbterror.ClassVariable.NewStdErr(mysql.ErrNotSupportedYet, pmysql.Message("%s = OFF is not supported when %s = ON", nil))
// ErrFunctionsNoopImpl is an error to say the behavior is protected by the tidb_enable_noop_functions sysvar.
// This is copied from expression.ErrFunctionsNoopImpl to prevent circular dependencies.
// It needs to be public for tests.
ErrFunctionsNoopImpl = dbterror.ClassVariable.NewStdErr(mysql.ErrNotSupportedYet, pmysql.Message("function %s has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions", nil))
)

func init() {
sysVars = make(map[string]*SysVar)
Expand Down Expand Up @@ -583,8 +593,6 @@ var defaultSysVars = []*SysVar{
{Scope: ScopeGlobal | ScopeSession, Name: CharsetDatabase, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkCharacterValid(normalizedValue, CharsetDatabase)
}},
{Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: "0"},
{Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: "0"},
{Scope: ScopeGlobal, Name: MaxPreparedStmtCount, Value: strconv.FormatInt(DefMaxPreparedStmtCount, 10), Type: TypeInt, MinValue: -1, MaxValue: 1048576, AutoConvertOutOfRange: true},
{Scope: ScopeNone, Name: DataDir, Value: "/usr/local/mysql/data/"},
{Scope: ScopeGlobal | ScopeSession, Name: WaitTimeout, Value: strconv.FormatInt(DefWaitTimeout, 10), Type: TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear, AutoConvertOutOfRange: true},
Expand Down Expand Up @@ -1099,7 +1107,29 @@ var defaultSysVars = []*SysVar{
MemoryUsageAlarmRatio.Store(tidbOptFloat64(val, 0.8))
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: BoolToOnOff(DefTiDBEnableNoopFuncs), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: BoolToOnOff(DefTiDBEnableNoopFuncs), Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {

// The behavior is very weird if someone can turn TiDBEnableNoopFuncs OFF, but keep any of the following on:
// TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly
// To prevent this strange position, prevent setting to OFF when any of these sysVars are ON of the same scope.

if normalizedValue == BoolOff {
for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly} {
val, _ := vars.GetSystemVar(potentialIncompatibleSysVar) // session scope
if scope == ScopeGlobal { // global scope
var err error
val, err = vars.GlobalVarsAccessor.GetGlobalSysVar(potentialIncompatibleSysVar)
if err != nil {
return originalValue, errUnknownSystemVariable.GenWithStackByArgs(potentialIncompatibleSysVar)
}
}
if TiDBOptOn(val) {
return originalValue, errValueNotSupportedWhen.GenWithStackByArgs(TiDBEnableNoopFuncs, potentialIncompatibleSysVar)
}
}
}
return normalizedValue, nil
}, SetSession: func(s *SessionVars, val string) error {
s.EnableNoopFuncs = TiDBOptOn(val)
return nil
}},
Expand Down
21 changes: 21 additions & 0 deletions sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,27 @@ func checkCharacterValid(normalizedValue string, argName string) (string, error)
return cht, nil
}

// checkReadOnly requires TiDBEnableNoopFuncs=1 for the same scope otherwise an error will be returned.
func checkReadOnly(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag, offlineMode bool) (string, error) {
feature := "READ ONLY"
if offlineMode {
feature = "OFFLINE MODE"
}
if TiDBOptOn(normalizedValue) {
if !vars.EnableNoopFuncs && scope == ScopeSession {
return BoolOff, ErrFunctionsNoopImpl.GenWithStackByArgs(feature)
}
val, err := vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBEnableNoopFuncs)
if err != nil {
return originalValue, errUnknownSystemVariable.GenWithStackByArgs(TiDBEnableNoopFuncs)
}
if scope == ScopeGlobal && !TiDBOptOn(val) {
return BoolOff, ErrFunctionsNoopImpl.GenWithStackByArgs(feature)
}
}
return normalizedValue, nil
}

// GetSessionSystemVar gets a system variable.
// If it is a session only variable, use the default value defined in code.
// Returns error if there is no such variable.
Expand Down

0 comments on commit bdcb96f

Please sign in to comment.