use global variable instead of config (#458)

Signed-off-by: Jack Yu <jackysp@gmail.com>
This commit is contained in:
Jack Yu 2022-03-29 17:20:50 +08:00 committed by GitHub
parent 8489c3e8c3
commit 6bf6951325
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 15 additions and 8 deletions

View File

@ -44,8 +44,6 @@ import (
const (
// DefStoreLivenessTimeout is the default value for store liveness timeout.
DefStoreLivenessTimeout = "1s"
// DefTxnCommitBatchSize recommends each RPC packet should be less than ~1MB
DefTxnCommitBatchSize = 16 * 1024
)
// TiKVClient is the config for tikv client.
@ -86,7 +84,6 @@ type TiKVClient struct {
// TTLRefreshedTxnSize controls whether a transaction should update its TTL or not.
TTLRefreshedTxnSize int64 `toml:"ttl-refreshed-txn-size" json:"ttl-refreshed-txn-size"`
ResolveLockLiteThreshold uint64 `toml:"resolve-lock-lite-threshold" json:"resolve-lock-lite-threshold"`
TxnCommitBatchSize uint `toml:"txn-commit-batch-size" json:"txn-commit-batch-size"`
}
// AsyncCommit is the config for the async commit feature. The switch to enable it is a system variable.
@ -155,7 +152,6 @@ func DefaultTiKVClient() TiKVClient {
},
ResolveLockLiteThreshold: 16,
TxnCommitBatchSize: DefTxnCommitBatchSize,
}
}

View File

@ -41,6 +41,17 @@ import (
// StoreLimit will update from config reload and global variable set.
var StoreLimit atomic.Int64
// DefTxnCommitBatchSize is the default value of TxnCommitBatchSize.
const DefTxnCommitBatchSize uint64 = 16 * 1024
// TxnCommitBatchSize controls the batch size of transaction commit related requests sent by client to TiKV,
// TiKV recommends each RPC packet should be less than ~1MB.
var TxnCommitBatchSize atomic.Uint64
func init() {
TxnCommitBatchSize.Store(DefTxnCommitBatchSize)
}
// ReplicaReadType is the type of replica to read data from
type ReplicaReadType byte

View File

@ -700,7 +700,7 @@ func txnLockTTL(startTime time.Time, txnSize int) uint64 {
// When writeSize is less than 256KB, the base ttl is defaultTTL (3s);
// When writeSize is 1MiB, 4MiB, or 10MiB, ttl is 6s, 12s, 20s correspondingly;
lockTTL := defaultLockTTL
if txnSize >= int(config.GetGlobalConfig().TiKVClient.TxnCommitBatchSize) {
if txnSize >= int(kv.TxnCommitBatchSize.Load()) {
sizeMiB := float64(txnSize) / bytesPerMiB
lockTTL = uint64(float64(ttlFactor) * math.Sqrt(sizeMiB))
if lockTTL < defaultLockTTL {
@ -875,7 +875,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *retry.Backoffer, action
batchBuilder := newBatched(c.primary())
for _, group := range groups {
batchBuilder.appendBatchMutationsBySize(group.region, group.mutations, sizeFunc,
int(config.GetGlobalConfig().TiKVClient.TxnCommitBatchSize))
int(kv.TxnCommitBatchSize.Load()))
}
firstIsPrimary := batchBuilder.setPrimary()

View File

@ -20,10 +20,10 @@ import (
"sync/atomic"
"time"
"github.com/tikv/client-go/v2/config"
"github.com/tikv/client-go/v2/internal/locate"
"github.com/tikv/client-go/v2/internal/retry"
"github.com/tikv/client-go/v2/internal/unionstore"
"github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/tikvrpc"
"github.com/tikv/client-go/v2/txnkv/txnsnapshot"
)
@ -330,7 +330,7 @@ type ConfigProbe struct{}
// GetTxnCommitBatchSize returns the batch size to commit txn.
func (c ConfigProbe) GetTxnCommitBatchSize() uint64 {
return uint64(config.GetGlobalConfig().TiKVClient.TxnCommitBatchSize)
return kv.TxnCommitBatchSize.Load()
}
// GetPessimisticLockMaxBackoff returns pessimisticLockMaxBackoff