diff --git a/config/config.go b/config/config.go index 2cd801ceefac4..61f6c961f1902 100644 --- a/config/config.go +++ b/config/config.go @@ -238,6 +238,8 @@ type TiKVClient struct { GrpcKeepAliveTimeout uint `toml:"grpc-keepalive-timeout" json:"grpc-keepalive-timeout"` // CommitTimeout is the max time which command 'commit' will wait. CommitTimeout string `toml:"commit-timeout" json:"commit-timeout"` + // MaxTxnTimeUse is the max time a Txn may use (in seconds) from its startTS to commitTS. + MaxTxnTimeUse uint `toml:"max-txn-time-use" json:"max-txn-time-use"` } // Binlog is the config for binlog. @@ -320,6 +322,7 @@ var defaultConf = Config{ GrpcKeepAliveTime: 10, GrpcKeepAliveTimeout: 3, CommitTimeout: "41s", + MaxTxnTimeUse: 590, }, Binlog: Binlog{ WriteTimeout: "15s", diff --git a/config/config.toml.example b/config/config.toml.example index 9cf4682f80282..71f35f0eb4f1a 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -229,6 +229,11 @@ grpc-keepalive-timeout = 3 # max time for commit command, must be twice bigger than raft election timeout. commit-timeout = "41s" +# The max time a Txn may use (in seconds) from its startTS to commitTS. +# We use it to guarantee GC worker will not influence any active txn. Please make sure that this +# value is less than gc_life_time - 10s. +max-txn-time-use = 590 + [txn-local-latches] # Enable local latches for transactions. Enable it when # there are lots of conflicts between transactions. diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 382663e0d3671..c20841ba9ef4e 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx/binloginfo" @@ -76,6 +77,10 @@ type twoPhaseCommitter struct { syncLog bool connID uint64 // connID is used for log. cleanWg sync.WaitGroup + // The max time a Txn may use (in ms) from its startTS to commitTS. + // We use it to guarantee GC worker will not influence any active txn. The value + // should be less than GC life time. + maxTxnTimeUse uint64 } // newTwoPhaseCommitter creates a twoPhaseCommitter. @@ -140,18 +145,22 @@ func newTwoPhaseCommitter(txn *tikvTxn, connID uint64) (*twoPhaseCommitter, erro connID, tableID, size, len(keys), putCnt, delCnt, lockCnt, txn.startTS) } + // Convert from sec to ms + maxTxnTimeUse := uint64(config.GetGlobalConfig().TiKVClient.MaxTxnTimeUse) * 1000 + metrics.TiKVTxnWriteKVCountHistogram.Observe(float64(len(keys))) metrics.TiKVTxnWriteSizeHistogram.Observe(float64(size)) return &twoPhaseCommitter{ - store: txn.store, - txn: txn, - startTS: txn.StartTS(), - keys: keys, - mutations: mutations, - lockTTL: txnLockTTL(txn.startTime, size), - priority: getTxnPriority(txn), - syncLog: getTxnSyncLog(txn), - connID: connID, + store: txn.store, + txn: txn, + startTS: txn.StartTS(), + keys: keys, + mutations: mutations, + lockTTL: txnLockTTL(txn.startTime, size), + priority: getTxnPriority(txn), + syncLog: getTxnSyncLog(txn), + connID: connID, + maxTxnTimeUse: maxTxnTimeUse, }, nil } @@ -542,11 +551,6 @@ func (c *twoPhaseCommitter) cleanupKeys(bo *Backoffer, keys [][]byte) error { return c.doActionOnKeys(bo, actionCleanup, keys) } -// The max time a Txn may use (in ms) from its startTS to commitTS. -// We use it to guarantee GC worker will not influence any active txn. The value -// should be less than `gcRunInterval`. -const maxTxnTimeUse = 590000 - func (c *twoPhaseCommitter) executeAndWriteFinishBinlog(ctx context.Context) error { err := c.execute(ctx) if err != nil { @@ -613,7 +617,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) error { return errors.Trace(err) } - if c.store.oracle.IsExpired(c.startTS, maxTxnTimeUse) { + if c.store.oracle.IsExpired(c.startTS, c.maxTxnTimeUse) { err = errors.Errorf("con:%d txn takes too much time, start: %d, commit: %d", c.connID, c.startTS, c.commitTS) return errors.Annotate(err, txnRetryableMark) }