Skip to content

Commit

Permalink
Merge remote-tracking branch 'origin/19571' into 19571
Browse files Browse the repository at this point in the history
  • Loading branch information
Patrick0308 committed Oct 15, 2020
2 parents a4f1a02 + 994af5c commit b923e4d
Show file tree
Hide file tree
Showing 63 changed files with 2,394 additions and 553 deletions.
82 changes: 44 additions & 38 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -405,25 +405,26 @@ type Status struct {
type Performance struct {
MaxProcs uint `toml:"max-procs" json:"max-procs"`
// Deprecated: use ServerMemoryQuota instead
MaxMemory uint64 `toml:"max-memory" json:"max-memory"`
ServerMemoryQuota uint64 `toml:"server-memory-quota" json:"server-memory-quota"`
StatsLease string `toml:"stats-lease" json:"stats-lease"`
StmtCountLimit uint `toml:"stmt-count-limit" json:"stmt-count-limit"`
FeedbackProbability float64 `toml:"feedback-probability" json:"feedback-probability"`
QueryFeedbackLimit uint `toml:"query-feedback-limit" json:"query-feedback-limit"`
PseudoEstimateRatio float64 `toml:"pseudo-estimate-ratio" json:"pseudo-estimate-ratio"`
ForcePriority string `toml:"force-priority" json:"force-priority"`
BindInfoLease string `toml:"bind-info-lease" json:"bind-info-lease"`
TxnEntrySizeLimit uint64 `toml:"txn-entry-size-limit" json:"txn-entry-size-limit"`
TxnTotalSizeLimit uint64 `toml:"txn-total-size-limit" json:"txn-total-size-limit"`
TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"`
CrossJoin bool `toml:"cross-join" json:"cross-join"`
RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"`
DistinctAggPushDown bool `toml:"distinct-agg-push-down" json:"agg-push-down-join"`
CommitterConcurrency int `toml:"committer-concurrency" json:"committer-concurrency"`
MaxTxnTTL uint64 `toml:"max-txn-ttl" json:"max-txn-ttl"`
MemProfileInterval string `toml:"mem-profile-interval" json:"mem-profile-interval"`
IndexUsageSyncLease string `toml:"index-usage-sync-lease" json:"index-usage-sync-lease"`
MaxMemory uint64 `toml:"max-memory" json:"max-memory"`
ServerMemoryQuota uint64 `toml:"server-memory-quota" json:"server-memory-quota"`
MemoryUsageAlarmRatio float64 `toml:"memory-usage-alarm-ratio" json:"memory-usage-alarm-ratio"`
StatsLease string `toml:"stats-lease" json:"stats-lease"`
StmtCountLimit uint `toml:"stmt-count-limit" json:"stmt-count-limit"`
FeedbackProbability float64 `toml:"feedback-probability" json:"feedback-probability"`
QueryFeedbackLimit uint `toml:"query-feedback-limit" json:"query-feedback-limit"`
PseudoEstimateRatio float64 `toml:"pseudo-estimate-ratio" json:"pseudo-estimate-ratio"`
ForcePriority string `toml:"force-priority" json:"force-priority"`
BindInfoLease string `toml:"bind-info-lease" json:"bind-info-lease"`
TxnEntrySizeLimit uint64 `toml:"txn-entry-size-limit" json:"txn-entry-size-limit"`
TxnTotalSizeLimit uint64 `toml:"txn-total-size-limit" json:"txn-total-size-limit"`
TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"`
CrossJoin bool `toml:"cross-join" json:"cross-join"`
RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"`
DistinctAggPushDown bool `toml:"distinct-agg-push-down" json:"agg-push-down-join"`
CommitterConcurrency int `toml:"committer-concurrency" json:"committer-concurrency"`
MaxTxnTTL uint64 `toml:"max-txn-ttl" json:"max-txn-ttl"`
MemProfileInterval string `toml:"mem-profile-interval" json:"mem-profile-interval"`
IndexUsageSyncLease string `toml:"index-usage-sync-lease" json:"index-usage-sync-lease"`
}

// PlanCache is the PlanCache section of the config.
Expand Down Expand Up @@ -655,25 +656,26 @@ var defaultConf = Config{
RecordQPSbyDB: false,
},
Performance: Performance{
MaxMemory: 0,
ServerMemoryQuota: 0,
TCPKeepAlive: true,
CrossJoin: true,
StatsLease: "3s",
RunAutoAnalyze: true,
StmtCountLimit: 5000,
FeedbackProbability: 0.05,
QueryFeedbackLimit: 512,
PseudoEstimateRatio: 0.8,
ForcePriority: "NO_PRIORITY",
BindInfoLease: "3s",
TxnEntrySizeLimit: DefTxnEntrySizeLimit,
TxnTotalSizeLimit: DefTxnTotalSizeLimit,
DistinctAggPushDown: false,
CommitterConcurrency: 16,
MaxTxnTTL: 60 * 60 * 1000, // 1hour
MemProfileInterval: "1m",
IndexUsageSyncLease: "60s",
MaxMemory: 0,
ServerMemoryQuota: 0,
MemoryUsageAlarmRatio: 0.8,
TCPKeepAlive: true,
CrossJoin: true,
StatsLease: "3s",
RunAutoAnalyze: true,
StmtCountLimit: 5000,
FeedbackProbability: 0.05,
QueryFeedbackLimit: 512,
PseudoEstimateRatio: 0.8,
ForcePriority: "NO_PRIORITY",
BindInfoLease: "3s",
TxnEntrySizeLimit: DefTxnEntrySizeLimit,
TxnTotalSizeLimit: DefTxnTotalSizeLimit,
DistinctAggPushDown: false,
CommitterConcurrency: 16,
MaxTxnTTL: 60 * 60 * 1000, // 1hour
MemProfileInterval: "1m",
IndexUsageSyncLease: "60s",
},
ProxyProtocol: ProxyProtocol{
Networks: "",
Expand Down Expand Up @@ -930,6 +932,10 @@ func (c *Config) Valid() error {
return fmt.Errorf("txn-total-size-limit should be less than %d", 10<<30)
}

if c.Performance.MemoryUsageAlarmRatio > 1 || c.Performance.MemoryUsageAlarmRatio < 0 {
return fmt.Errorf("memory-usage-alarm-ratio in [Performance] must be greater than or equal to 0 and less than or equal to 1")
}

if c.StmtSummary.MaxStmtCount <= 0 {
return fmt.Errorf("max-stmt-count in [stmt-summary] should be greater than 0")
}
Expand Down
8 changes: 8 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,14 @@ max-procs = 0
# Memory size quota for tidb server, 0 means unlimited
server-memory-quota = 0

# The alarm threshold when memory usage of the tidb-server exceeds. The valid value range is greater than or equal to 0
# and less than or equal to 1. The default value is 0.8.
# If this configuration is set to 0 or 1, it'll disable the alarm.
# Otherwise, related information will be recorded in the directory `tmp-storage-path/record`.
# Note: If the configuration `server-memory-quota` is set and larger than 0, the alarm threshold will be
# `memory-usage-alarm-ratio * server-memory-quota`; otherwise, it'll be `memory-usage-alarm-ratio * system memory size`.
memory-usage-alarm-ratio = 0.8

# StmtCountLimit limits the max count of statement inside a transaction.
stmt-count-limit = 5000

Expand Down
1 change: 1 addition & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,7 @@ type DDL interface {
RepairTable(ctx sessionctx.Context, table *ast.TableName, createStmt *ast.CreateTableStmt) error
CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStmt) error
DropSequence(ctx sessionctx.Context, tableIdent ast.Ident, ifExists bool) (err error)
AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error

// CreateSchemaWithInfo creates a database (schema) given its database info.
//
Expand Down
43 changes: 43 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -5320,6 +5320,49 @@ func (d *ddl) CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStm
return d.CreateTableWithInfo(ctx, ident.Schema, tbInfo, onExist, false /*tryRetainID*/)
}

func (d *ddl) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error {
ident := ast.Ident{Name: stmt.Name.Name, Schema: stmt.Name.Schema}
is := d.GetInfoSchemaWithInterceptor(ctx)
// Check schema existence.
db, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema)
}
// Check table existence.
tbl, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
if stmt.IfExists {
ctx.GetSessionVars().StmtCtx.AppendNote(err)
return nil
}
return err
}
if !tbl.Meta().IsSequence() {
return ErrWrongObject.GenWithStackByArgs(ident.Schema, ident.Name, "SEQUENCE")
}

// Validate the new sequence option value in old sequenceInfo.
oldSequenceInfo := tbl.Meta().Sequence
copySequenceInfo := *oldSequenceInfo
_, _, err = alterSequenceOptions(stmt.SeqOptions, ident, &copySequenceInfo)
if err != nil {
return err
}

job := &model.Job{
SchemaID: db.ID,
TableID: tbl.Meta().ID,
SchemaName: db.Name.L,
Type: model.ActionAlterSequence,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{ident, stmt.SeqOptions},
}

err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

func (d *ddl) DropSequence(ctx sessionctx.Context, ti ast.Ident, ifExists bool) (err error) {
schema, tbl, err := d.getSchemaAndTableByIdent(ctx, ti)
if err != nil {
Expand Down
2 changes: 2 additions & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -685,6 +685,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
ver, err = onAlterIndexVisibility(t, job)
case model.ActionAlterTableAlterPartition:
ver, err = onAlterTablePartition(t, job)
case model.ActionAlterSequence:
ver, err = onAlterSequence(t, job)
default:
// Invalid job, cancel it.
job.State = model.JobStateCancelled
Expand Down
108 changes: 108 additions & 0 deletions ddl/sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ package ddl

import (
"math"
"reflect"

"github.com/cznic/mathutil"
"github.com/pingcap/errors"
Expand Down Expand Up @@ -183,3 +184,110 @@ func buildSequenceInfo(stmt *ast.CreateSequenceStmt, ident ast.Ident) (*model.Se
}
return sequenceInfo, nil
}

func alterSequenceOptions(sequenceOptions []*ast.SequenceOption, ident ast.Ident, oldSequence *model.SequenceInfo) (bool, int64, error) {
var (
restartFlag bool
restartWithFlag bool
restartValue int64
)
// Override the old sequence value with new option.
for _, op := range sequenceOptions {
switch op.Tp {
case ast.SequenceOptionIncrementBy:
oldSequence.Increment = op.IntValue
case ast.SequenceStartWith:
oldSequence.Start = op.IntValue
case ast.SequenceMinValue:
oldSequence.MinValue = op.IntValue
case ast.SequenceMaxValue:
oldSequence.MaxValue = op.IntValue
case ast.SequenceCache:
oldSequence.CacheValue = op.IntValue
case ast.SequenceNoCache:
oldSequence.Cache = false
case ast.SequenceCycle:
oldSequence.Cycle = true
case ast.SequenceNoCycle:
oldSequence.Cycle = false
case ast.SequenceRestart:
restartFlag = true
case ast.SequenceRestartWith:
restartWithFlag = true
restartValue = op.IntValue
}
}
if !validateSequenceOptions(oldSequence) {
return false, 0, ErrSequenceInvalidData.GenWithStackByArgs(ident.Schema.L, ident.Name.L)
}
if restartWithFlag {
return true, restartValue, nil
}
if restartFlag {
return true, oldSequence.Start, nil
}
return false, 0, nil
}

func onAlterSequence(t *meta.Meta, job *model.Job) (ver int64, _ error) {
schemaID := job.SchemaID
var (
sequenceOpts []*ast.SequenceOption
ident ast.Ident
)
if err := job.DecodeArgs(&ident, &sequenceOpts); err != nil {
// Invalid arguments, cancel this job.
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}

// Get the old tableInfo.
tblInfo, err := checkTableExistAndCancelNonExistJob(t, job, schemaID)
if err != nil {
return ver, errors.Trace(err)
}

// Substitute the sequence info.
copySequenceInfo := *tblInfo.Sequence
restart, restartValue, err := alterSequenceOptions(sequenceOpts, ident, &copySequenceInfo)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
shouldUpdateVer := !reflect.DeepEqual(*tblInfo.Sequence, copySequenceInfo) || restart
tblInfo.Sequence = &copySequenceInfo

// Restart the sequence value.
// Notice: during the alter sequence process, if there is some dml continually consumes sequence (nextval/setval),
// the below cases will occur:
// Since the table schema haven't been refreshed in local/other node, dml will still use old definition of sequence
// to allocate sequence ids. Once the restart value is updated to kv here, the allocated ids in the upper layer won't
// guarantee to be consecutive and monotonous.
if restart {
err := restartSequenceValue(t, schemaID, tblInfo, restartValue)
if err != nil {
return ver, errors.Trace(err)
}
}

// Store the sequence info into kv.
ver, err = updateVersionAndTableInfo(t, job, tblInfo, shouldUpdateVer)
if err != nil {
return ver, errors.Trace(err)
}
// Finish this job.
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo)
return ver, nil
}

// Like setval does, restart sequence value won't affect current the step frequency. It will look backward for
// the first valid sequence valid rather than return the restart value directly.
func restartSequenceValue(t *meta.Meta, dbID int64, tblInfo *model.TableInfo, seqValue int64) error {
var sequenceBase int64
if tblInfo.Sequence.Increment >= 0 {
sequenceBase = seqValue - 1
} else {
sequenceBase = seqValue + 1
}
return t.RestartSequenceValue(dbID, tblInfo, sequenceBase)
}
85 changes: 85 additions & 0 deletions ddl/sequence_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@ func (s *testSequenceSuite) TestCreateSequence(c *C) {
c.Assert(sequenceTable.Meta().Sequence.Cycle, Equals, false)

// Test create privilege.
tk.MustExec("drop user if exists myuser@localhost")
tk.MustExec("create user myuser@localhost")

tk1 := testkit.NewTestKit(c, s.store)
Expand Down Expand Up @@ -989,3 +990,87 @@ func (s *testSequenceSuite) TestSequenceCacheShouldNotBeNegative(c *C) {

tk.MustExec("create sequence seq cache 1")
}

func (s *testSequenceSuite) TestAlterSequence(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

tk.MustExec("drop sequence if exists seq")
tk.MustExec("create sequence seq")
tk.MustExec("alter sequence seq increment by 2 start with 2")
tk.MustQuery("show create sequence seq").Check(testkit.Rows("seq CREATE SEQUENCE `seq` " +
"start with 2 minvalue 1 maxvalue 9223372036854775806 increment by 2 cache 1000 nocycle ENGINE=InnoDB"))

tk.MustExec("drop sequence if exists seq")
tk.MustExec("create sequence seq")
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("2"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3"))
// Alter sequence will invalidate the sequence cache in memory.
tk.MustExec("alter sequence seq increment by 2")
tk.MustQuery("show create sequence seq").Check(testkit.Rows("seq CREATE SEQUENCE `seq` " +
"start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 2 cache 1000 nocycle ENGINE=InnoDB"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1001"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1003"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1005"))

// Restart with value will reset the sequence value in kv.
tk.MustExec("alter sequence seq restart with 9")
// Like setval does, the sequence current value change won't affect the increment frequency.
// By now the step frequency is: 1, 3, 5, 7, 9, 11, 13, 15...
// After restart with 9, the current value rebased to 8, the next valid value will be 9, coincidentally equal to what we restarted.
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("9"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("11"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("13"))

tk.MustExec("alter sequence seq restart with 10")
// After restart with 10, the current value rebased to 9, the next valid value will be 11, rather than what we restart.
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("11"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("13"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("15"))

// Restart will reset the sequence value to start value by default.
tk.MustExec("alter sequence seq restart")
// After restart, the base will be pointed to 0, the first value will be 1 here, then plus the increment 2, the second value will be 3.
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("5"))

tk.MustExec("drop sequence if exists seq")
tk.MustExec("create sequence seq increment by 3")
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("4"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("7"))

tk.MustExec("alter sequence seq increment by 4")
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3001"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3005"))
tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3009"))
tk.MustExec("drop sequence if exists seq")
}

func (s *testSequenceSuite) TestAlterSequencePrivilege(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop sequence if exists my_seq")
tk.MustExec("create sequence my_seq")

// Test create privilege.
tk.MustExec("drop user if exists myuser@localhost")
tk.MustExec("create user myuser@localhost")

tk1 := testkit.NewTestKit(c, s.store)
se, err := session.CreateSession4Test(s.store)
c.Assert(err, IsNil)
c.Assert(se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil), IsTrue)
tk1.Se = se

// grant the myuser the access to database test.
tk.MustExec("grant select on test.* to 'myuser'@'localhost'")

tk1.MustExec("use test")
_, err = tk1.Exec("alter sequence my_seq increment = 2")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[planner:1142]ALTER command denied to user 'myuser'@'localhost' for table 'my_seq'")
tk.MustExec("drop sequence if exists my_seq")
}
Loading

0 comments on commit b923e4d

Please sign in to comment.