Skip to content

Commit

Permalink
*: disable batch DMLs by default (pingcap#13081)
Browse files Browse the repository at this point in the history
  • Loading branch information
jackysp authored and XiaTianliang committed Dec 21, 2019
1 parent 4f4488d commit 2a015f8
Show file tree
Hide file tree
Showing 8 changed files with 47 additions and 9 deletions.
2 changes: 2 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ type Config struct {
OOMAction string `toml:"oom-action" json:"oom-action"`
MemQuotaQuery int64 `toml:"mem-quota-query" json:"mem-quota-query"`
EnableStreaming bool `toml:"enable-streaming" json:"enable-streaming"`
EnableBatchDML bool `toml:"enable-batch-dml" json:"enable-batch-dml"`
TxnLocalLatches TxnLocalLatches `toml:"txn-local-latches" json:"txn-local-latches"`
// Set sys variable lower-case-table-names, ref: https://dev.mysql.com/doc/refman/5.7/en/identifier-case-sensitivity.html.
// TODO: We actually only support mode 2, which keeps the original case, but the comparison is case-insensitive.
Expand Down Expand Up @@ -426,6 +427,7 @@ var defaultConf = Config{
OOMAction: "log",
MemQuotaQuery: 32 << 30,
EnableStreaming: false,
EnableBatchDML: false,
CheckMb4ValueInUTF8: true,
TreatOldVersionUTF8AsUTF8MB4: true,
EnableTableLock: false,
Expand Down
3 changes: 3 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,9 @@ oom-action = "log"
# Enable coprocessor streaming.
enable-streaming = false

# Enable batch commit for the DMLs.
enable-batch-dml = false

# Set system variable 'lower_case_table_names'
lower-case-table-names = 2

Expand Down
2 changes: 2 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,7 @@ token-limit = 0
enable-table-lock = true
delay-clean-table-lock = 5
split-region-max-num=10000
enable-batch-dml = true
[performance]
txn-total-size-limit=2000
[tikv-client]
Expand Down Expand Up @@ -209,6 +210,7 @@ max-sql-length=1024
c.Assert(conf.SplitRegionMaxNum, Equals, uint64(10000))
c.Assert(conf.StmtSummary.MaxStmtCount, Equals, uint(1000))
c.Assert(conf.StmtSummary.MaxSQLLength, Equals, uint(1024))
c.Assert(conf.EnableBatchDML, Equals, true)
c.Assert(f.Close(), IsNil)
c.Assert(os.Remove(configFile), IsNil)

Expand Down
3 changes: 2 additions & 1 deletion executor/delete.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package executor
import (
"context"

"github.com/pingcap/tidb/config"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -73,7 +74,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error {
}

// If tidb_batch_delete is ON and not in a transaction, we could use BatchDelete mode.
batchDelete := e.ctx.GetSessionVars().BatchDelete && !e.ctx.GetSessionVars().InTxn()
batchDelete := e.ctx.GetSessionVars().BatchDelete && !e.ctx.GetSessionVars().InTxn() && config.GetGlobalConfig().EnableBatchDML
batchDMLSize := e.ctx.GetSessionVars().DMLBatchSize
fields := retTypes(e.children[0])
chk := newFirstChunk(e.children[0])
Expand Down
5 changes: 3 additions & 2 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -202,7 +203,7 @@ func insertRows(ctx context.Context, base insertCommon) (err error) {
return err
}
sessVars := e.ctx.GetSessionVars()
batchInsert := sessVars.BatchInsert && !sessVars.InTxn()
batchInsert := sessVars.BatchInsert && !sessVars.InTxn() && config.GetGlobalConfig().EnableBatchDML
batchSize := sessVars.DMLBatchSize

e.lazyFillAutoID = true
Expand Down Expand Up @@ -370,7 +371,7 @@ func insertRowsFromSelect(ctx context.Context, base insertCommon) error {
// If StrictSQLMode is disabled and it is a insert-select statement, it also handle BadNullAsWarning.
sessVars.StmtCtx.BadNullAsWarning = true
}
batchInsert := sessVars.BatchInsert && !sessVars.InTxn()
batchInsert := sessVars.BatchInsert && !sessVars.InTxn() && config.GetGlobalConfig().EnableBatchDML
batchSize := sessVars.DMLBatchSize

for {
Expand Down
15 changes: 15 additions & 0 deletions executor/seqtest/seq_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -845,6 +845,13 @@ func (s *seqTestSuite) TestBatchInsertDelete(c *C) {
r = tk.MustQuery("select count(*) from batch_insert;")
r.Check(testkit.Rows("320"))

// Test tidb_batch_insert could not work if enable-batch-dml is disabled.
tk.MustExec("set @@session.tidb_batch_insert=1;")
_, err = tk.Exec("insert into batch_insert (c) select * from batch_insert;")
c.Assert(err, NotNil)
c.Assert(kv.ErrTxnTooLarge.Equal(err), IsTrue)
tk.MustExec("set @@session.tidb_batch_insert=0;")

// for on duplicate key
_, err = tk.Exec(`insert into batch_insert_on_duplicate select * from batch_insert_on_duplicate as tt
on duplicate key update batch_insert_on_duplicate.id=batch_insert_on_duplicate.id+1000;`)
Expand All @@ -853,6 +860,14 @@ func (s *seqTestSuite) TestBatchInsertDelete(c *C) {
r = tk.MustQuery("select count(*) from batch_insert;")
r.Check(testkit.Rows("320"))

cfg := config.GetGlobalConfig()
newCfg := *cfg
newCfg.EnableBatchDML = true
config.StoreGlobalConfig(&newCfg)
defer func() {
config.StoreGlobalConfig(cfg)
}()

// Change to batch inset mode and batch size to 50.
tk.MustExec("set @@session.tidb_batch_insert=1;")
tk.MustExec("set @@session.tidb_dml_batch_size=50;")
Expand Down
9 changes: 9 additions & 0 deletions executor/write_concurrent_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"context"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/util/testkit"
)

Expand All @@ -29,6 +30,14 @@ func (s *testSuite) TestBatchInsertWithOnDuplicate(c *C) {
tk.MustExec(ctx, "create table duplicate_test(id int auto_increment, k1 int, primary key(id), unique key uk(k1))")
tk.MustExec(ctx, "insert into duplicate_test(k1) values(?),(?),(?),(?),(?)", tk.PermInt(5)...)

cfg := config.GetGlobalConfig()
newCfg := *cfg
newCfg.EnableBatchDML = true
config.StoreGlobalConfig(&newCfg)
defer func() {
config.StoreGlobalConfig(cfg)
}()

tk.ConcurrentRun(c, 3, 2, // concurrent: 3, loops: 2,
// prepare data for each loop.
func(ctx context.Context, tk *testkit.CTestKit, concurrent int, currentLoop int) [][][]interface{} {
Expand Down
17 changes: 11 additions & 6 deletions statistics/handle/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -564,12 +564,17 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch

func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error {
h.mu.Lock()
h.mu.ctx.GetSessionVars().BatchDelete = true
sql := fmt.Sprintf("delete from mysql.stats_feedback where table_id = %d and hist_id = %d and is_index = %d", tableID, histID, isIndex)
_, err := h.mu.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), sql)
h.mu.ctx.GetSessionVars().BatchDelete = false
h.mu.Unlock()
return errors.Trace(err)
defer h.mu.Unlock()
hasData := true
for hasData {
sql := fmt.Sprintf("delete from mysql.stats_feedback where table_id = %d and hist_id = %d and is_index = %d limit 10000", tableID, histID, isIndex)
_, err := h.mu.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), sql)
if err != nil {
return errors.Trace(err)
}
hasData = h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() > 0
}
return nil
}

func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.QueryFeedback, hist *statistics.Histogram, cms *statistics.CMSketch) error {
Expand Down

0 comments on commit 2a015f8

Please sign in to comment.