Skip to content

Commit

Permalink
*: remove the LightningMode from TiDB (#12466)
Browse files Browse the repository at this point in the history
  • Loading branch information
lonng authored and sre-bot committed Sep 29, 2019
1 parent 1b03d16 commit 41ac571
Show file tree
Hide file tree
Showing 12 changed files with 20 additions and 1,126 deletions.
13 changes: 5 additions & 8 deletions executor/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,15 +51,12 @@ func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) error {
defer sessVars.CleanBuffers()
ignoreErr := sessVars.StmtCtx.DupKeyAsWarning

if !sessVars.LightningMode {
txn, err := e.ctx.Txn(true)
if err != nil {
return err
}
sessVars.GetWriteStmtBufs().BufStore = kv.NewBufferStore(txn, kv.TempTxnMemBufCap)
txn, err := e.ctx.Txn(true)
if err != nil {
return err
}

e.ctx.GetSessionVars().StmtCtx.AddRecordRows(uint64(len(rows)))
sessVars.GetWriteStmtBufs().BufStore = kv.NewBufferStore(txn, kv.TempTxnMemBufCap)
sessVars.StmtCtx.AddRecordRows(uint64(len(rows)))
// If you use the IGNORE keyword, duplicate-key error that occurs while executing the INSERT statement are ignored.
// For example, without IGNORE, a row that duplicates an existing UNIQUE index or PRIMARY KEY value in
// the table causes a duplicate-key error and the statement is aborted. With IGNORE, the row is discarded and no error occurs.
Expand Down
10 changes: 4 additions & 6 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -396,13 +396,11 @@ func (e *InsertValues) doBatchInsert(ctx context.Context) error {
// We should return a special error for batch insert.
return ErrBatchInsertFail.GenWithStack("BatchInsert failed with error: %v", err)
}
if !sessVars.LightningMode {
txn, err := e.ctx.Txn(true)
if err != nil {
return err
}
sessVars.GetWriteStmtBufs().BufStore = kv.NewBufferStore(txn, kv.TempTxnMemBufCap)
txn, err := e.ctx.Txn(true)
if err != nil {
return err
}
sessVars.GetWriteStmtBufs().BufStore = kv.NewBufferStore(txn, kv.TempTxnMemBufCap)
return nil
}

Expand Down
2 changes: 1 addition & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error {
Params: sorter.markers,
SchemaVersion: e.is.SchemaMetaVersion(),
}
prepared.UseCache = plannercore.PreparedPlanCacheEnabled() && (vars.LightningMode || plannercore.Cacheable(stmt))
prepared.UseCache = plannercore.PreparedPlanCacheEnabled() && plannercore.Cacheable(stmt)

// We try to build the real statement of preparedStmt.
for i := range prepared.Params {
Expand Down
2 changes: 0 additions & 2 deletions kv/buffer_store.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@ import (
var (
// DefaultTxnMembufCap is the default transaction membuf capability.
DefaultTxnMembufCap = 4 * 1024
// ImportingTxnMembufCap is the capability of tidb importing data situation.
ImportingTxnMembufCap = 32 * 1024
// TempTxnMemBufCap is the capability of temporary membuf.
TempTxnMemBufCap = 64
)
Expand Down
4 changes: 0 additions & 4 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,10 +246,6 @@ func (s *session) DDLOwnerChecker() owner.DDLOwnerChecker {
}

func (s *session) getMembufCap() int {
if s.sessionVars.LightningMode {
return kv.ImportingTxnMembufCap
}

return kv.DefaultTxnMembufCap
}

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

/* TiDB system variables */

// LightningMode is true when the lightning use the kvencoder to transfer sql to raw kv.
LightningMode bool

// SkipUTF8Check check on input value.
SkipUTF8Check bool

Expand Down Expand Up @@ -593,9 +590,7 @@ func (s *SessionVars) GetSplitRegionTimeout() time.Duration {

// CleanBuffers cleans the temporary bufs
func (s *SessionVars) CleanBuffers() {
if !s.LightningMode {
s.GetWriteStmtBufs().clean()
}
s.GetWriteStmtBufs().clean()
}

// AllocPlanColumnID allocates column id for plan.
Expand Down
7 changes: 4 additions & 3 deletions table/tables/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,9 +198,10 @@ func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedV
fn(&opt)
}
ss := opt.AssertionProto
writeBufs := sctx.GetSessionVars().GetWriteStmtBufs()
skipCheck := sctx.GetSessionVars().LightningMode || sctx.GetSessionVars().StmtCtx.BatchCheck
key, distinct, err := c.GenIndexKey(sctx.GetSessionVars().StmtCtx, indexedValues, h, writeBufs.IndexKeyBuf)
vars := sctx.GetSessionVars()
writeBufs := vars.GetWriteStmtBufs()
skipCheck := vars.StmtCtx.BatchCheck
key, distinct, err := c.GenIndexKey(vars.StmtCtx, indexedValues, h, writeBufs.IndexKeyBuf)
if err != nil {
return 0, err
}
Expand Down
14 changes: 4 additions & 10 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -413,10 +413,6 @@ func adjustRowValuesBuf(writeBufs *variable.WriteStmtBufs, rowLen int) {
// getRollbackableMemStore get a rollbackable BufferStore, when we are importing data,
// Just add the kv to transaction's membuf directly.
func (t *tableCommon) getRollbackableMemStore(ctx sessionctx.Context) (kv.RetrieverMutator, error) {
if ctx.GetSessionVars().LightningMode {
return ctx.Txn(true)
}

bs := ctx.GetSessionVars().GetWriteStmtBufs().BufStore
if bs == nil {
txn, err := ctx.Txn(true)
Expand Down Expand Up @@ -528,12 +524,10 @@ func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ..
}
txn.SetAssertion(key, kv.None)

if !sessVars.LightningMode {
if err = rm.(*kv.BufferStore).SaveTo(txn); err != nil {
return 0, err
}
ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.physicalTableID, recordID)
if err = rm.(*kv.BufferStore).SaveTo(txn); err != nil {
return 0, err
}
ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.physicalTableID, recordID)

if shouldWriteBinlog(ctx) {
// For insert, TiDB and Binlog can use same row and schema.
Expand Down Expand Up @@ -594,7 +588,7 @@ func (t *tableCommon) addIndices(sctx sessionctx.Context, recordID int64, r []ty
} else {
ctx = context.Background()
}
skipCheck := sctx.GetSessionVars().LightningMode || sctx.GetSessionVars().StmtCtx.BatchCheck
skipCheck := sctx.GetSessionVars().StmtCtx.BatchCheck
if t.meta.PKIsHandle && !skipCheck && !opt.SkipHandleCheck {
if err := CheckHandleExists(ctx, sctx, t, recordID, nil); err != nil {
return recordID, err
Expand Down
77 changes: 0 additions & 77 deletions util/kvencoder/allocator.go

This file was deleted.

Loading

0 comments on commit 41ac571

Please sign in to comment.