config: cleanup and pass config as parameter (#14)

* config: cleanup and pass config as parameter

Signed-off-by: disksing <i@disksing.com>

* fix review comments

Signed-off-by: disksing <i@disksing.com>

* fix conflicts

Signed-off-by: disksing <i@disksing.com>

* address comment

Signed-off-by: disksing <i@disksing.com>
This commit is contained in:
disksing 2019-04-19 17:01:34 +08:00 committed by GitHub
parent ca5f30f8bb
commit 940d55927c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
36 changed files with 611 additions and 369 deletions

View File

@ -13,191 +13,20 @@
package config
import (
"crypto/tls"
"crypto/x509"
"io/ioutil"
"time"
"github.com/pkg/errors"
)
// Security is SSL configuration.
type Security struct {
SSLCA string `toml:"ssl-ca" json:"ssl-ca"`
SSLCert string `toml:"ssl-cert" json:"ssl-cert"`
SSLKey string `toml:"ssl-key" json:"ssl-key"`
// Config contains configurations for tikv client.
type Config struct {
RPC RPC
Raw Raw
Txn Txn
RegionCache RegionCache
}
// ToTLSConfig generates tls's config based on security section of the config.
func (s *Security) ToTLSConfig() (*tls.Config, error) {
var tlsConfig *tls.Config
if len(s.SSLCA) != 0 {
var certificates = make([]tls.Certificate, 0)
if len(s.SSLCert) != 0 && len(s.SSLKey) != 0 {
// Load the client certificates from disk
certificate, err := tls.LoadX509KeyPair(s.SSLCert, s.SSLKey)
if err != nil {
return nil, errors.Errorf("could not load client key pair: %s", err)
}
certificates = append(certificates, certificate)
}
// Create a certificate pool from the certificate authority
certPool := x509.NewCertPool()
ca, err := ioutil.ReadFile(s.SSLCA)
if err != nil {
return nil, errors.Errorf("could not read ca certificate: %s", err)
}
// Append the certificates from the CA
if !certPool.AppendCertsFromPEM(ca) {
return nil, errors.New("failed to append ca certs")
}
tlsConfig = &tls.Config{
Certificates: certificates,
RootCAs: certPool,
}
// Default returns the default config.
func Default() Config {
return Config{
RPC: DefaultRPC(),
Raw: DefaultRaw(),
Txn: DefaultTxn(),
RegionCache: DefaultRegionCache(),
}
return tlsConfig, nil
}
// EnableOpenTracing is the flag to enable open tracing.
var EnableOpenTracing = false
var (
// OverloadThreshold is a threshold of TiKV load.
// If TiKV load is greater than this, TiDB will wait for a while to avoid little batch.
OverloadThreshold uint = 200
// BatchWaitSize is the max wait size for batch.
BatchWaitSize uint = 8
// MaxBatchSize is the max batch size when calling batch commands API.
MaxBatchSize uint = 128
// MaxBatchWaitTime in nanosecond is the max wait time for batch.
MaxBatchWaitTime time.Duration
)
// Those limits are enforced to make sure the transaction can be well handled by TiKV.
var (
// TxnEntrySizeLimit is limit of single entry size (len(key) + len(value)).
TxnEntrySizeLimit = 6 * 1024 * 1024
// TxnEntryCountLimit is a limit of the number of entries in the MemBuffer.
TxnEntryCountLimit uint64 = 300 * 1000
// TxnTotalSizeLimit is limit of the sum of all entry size.
TxnTotalSizeLimit = 100 * 1024 * 1024
// MaxTxnTimeUse is the max time a transaction can run.
MaxTxnTimeUse = 590
)
// Local latches for transactions. Enable it when
// there are lots of conflicts between transactions.
var (
EnableTxnLocalLatch = false
TxnLocalLatchCapacity uint = 2048000
)
// RegionCache configurations.
var (
RegionCacheBTreeDegree = 32
RegionCacheTTL = time.Minute * 10
)
// RawKV configurations.
var (
// MaxRawKVScanLimit is the maximum scan limit for rawkv Scan.
MaxRawKVScanLimit = 10240
// RawBatchPutSize is the maximum size limit for rawkv each batch put request.
RawBatchPutSize = 16 * 1024
// RawBatchPairCount is the maximum limit for rawkv each batch get/delete request.
RawBatchPairCount = 512
)
// RPC configurations.
var (
// MaxConnectionCount is the max gRPC connections that will be established with
// each tikv-server.
MaxConnectionCount uint = 16
// GrpcKeepAliveTime is the duration of time after which if the client doesn't see
// any activity it pings the server to see if the transport is still alive.
GrpcKeepAliveTime = time.Duration(10) * time.Second
// GrpcKeepAliveTimeout is the duration of time for which the client waits after having
// pinged for keepalive check and if no activity is seen even after that the connection
// is closed.
GrpcKeepAliveTimeout = time.Duration(3) * time.Second
// MaxSendMsgSize set max gRPC request message size sent to server. If any request message size is larger than
// current value, an error will be reported from gRPC.
MaxSendMsgSize = 1<<31 - 1
// MaxCallMsgSize set max gRPC receive message size received from server. If any message size is larger than
// current value, an error will be reported from gRPC.
MaxCallMsgSize = 1<<31 - 1
DialTimeout = 5 * time.Second
ReadTimeoutShort = 20 * time.Second // For requests that read/write several key-values.
ReadTimeoutMedium = 60 * time.Second // For requests that may need scan region.
ReadTimeoutLong = 150 * time.Second // For requests that may need scan region multiple times.
GCTimeout = 5 * time.Minute
UnsafeDestroyRangeTimeout = 5 * time.Minute
GrpcInitialWindowSize = 1 << 30
GrpcInitialConnWindowSize = 1 << 30
)
// KV configurations.
var (
// DefaultTxnMembufCap is the default transaction membuf capability.
DefaultTxnMembufCap = 4 * 1024
)
// Latch configurations.
var (
LatchExpireDuration = 2 * time.Minute
LatchCheckInterval = 1 * time.Minute
LatchCheckCounter = 50000
LatchListCount = 5
LatchLockChanSize = 100
)
// Oracle configurations.
var (
TsoSlowThreshold = 30 * time.Millisecond
// update oracle's lastTS every 2000ms.
OracleUpdateInterval = 2000
)
// Txn configurations.
var (
// TiKV recommends each RPC packet should be less than ~1MB. We keep each packet's
// Key+Value size below 16KB.
TxnCommitBatchSize = 16 * 1024
// By default, locks after 3000ms is considered unusual (the client created the
// lock might be dead). Other client may cleanup this kind of lock.
// For locks created recently, we will do backoff and retry.
TxnDefaultLockTTL uint64 = 3000
// TODO: Consider if it's appropriate.
TxnMaxLockTTL uint64 = 120000
// ttl = ttlFactor * sqrt(writeSizeInMiB)
TxnTTLFactor = 6000
// TxnResolvedCacheSize is max number of cached txn status.
TxnResolvedCacheSize = 2048
// SafePoint.
// This is almost the same as 'tikv_gc_safe_point' in the table 'mysql.tidb',
// save this to pd instead of tikv, because we can't use interface of table
// if the safepoint on tidb is expired.
GcSavedSafePoint = "/tidb/store/gcworker/saved_safe_point"
GcSafePointCacheInterval = time.Second * 100
GcCPUTimeInaccuracyBound = time.Second
GcSafePointUpdateInterval = time.Second * 10
GcSafePointQuickRepeatInterval = time.Second
TxnScanBatchSize = 256
TxnBatchGetSize = 5120
)

35
config/raw.go Normal file
View File

@ -0,0 +1,35 @@
// Copyright 2019 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package config
// Raw is rawkv configurations.
type Raw struct {
// MaxScanLimit is the maximum scan limit for rawkv Scan.
MaxScanLimit int
// MaxBatchPutSize is the maximum size limit for rawkv each batch put request.
MaxBatchPutSize int
// BatchPairCount is the maximum limit for rawkv each batch get/delete request.
BatchPairCount int
}
// DefaultRaw returns default rawkv configuration.
func DefaultRaw() Raw {
return Raw{
MaxScanLimit: 10240,
MaxBatchPutSize: 16 * 1024,
BatchPairCount: 512,
}
}

30
config/regioncache.go Normal file
View File

@ -0,0 +1,30 @@
// Copyright 2019 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package config
import "time"
// RegionCache contains the configurations for region cache.
type RegionCache struct {
BTreeDegree int
CacheTTL time.Duration
}
// DefaultRegionCache returns the default region cache config.
func DefaultRegionCache() RegionCache {
return RegionCache{
BTreeDegree: 32,
CacheTTL: 10 * time.Minute,
}
}

168
config/rpc.go Normal file
View File

@ -0,0 +1,168 @@
// Copyright 2019 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package config
import (
"crypto/tls"
"crypto/x509"
"io/ioutil"
"time"
"github.com/pkg/errors"
)
// RPC configurations.
type RPC struct {
// MaxConnectionCount is the max gRPC connections that will be established with
// each tikv-server.
MaxConnectionCount uint
// GrpcKeepAliveTime is the duration of time after which if the client doesn't see
// any activity it pings the server to see if the transport is still alive.
GrpcKeepAliveTime time.Duration
// GrpcKeepAliveTimeout is the duration of time for which the client waits after having
// pinged for keepalive check and if no activity is seen even after that the connection
// is closed.
GrpcKeepAliveTimeout time.Duration
// GrpcMaxSendMsgSize set max gRPC request message size sent to server. If any request message size is larger than
// current value, an error will be reported from gRPC.
GrpcMaxSendMsgSize int
// GrpcMaxCallMsgSize set max gRPC receive message size received from server. If any message size is larger than
// current value, an error will be reported from gRPC.
GrpcMaxCallMsgSize int
// The value for initial window size on a gRPC stream.
GrpcInitialWindowSize int
// The value for initial windows size on a gRPC connection.
GrpcInitialConnWindowSize int32
// The max time to establish a gRPC connection.
DialTimeout time.Duration
// For requests that read/write several key-values.
ReadTimeoutShort time.Duration
// For requests that may need scan region.
ReadTimeoutMedium time.Duration
// For requests that may need scan region multiple times.
ReadTimeoutLong time.Duration
// The flag to enable open tracing.
EnableOpenTracing bool
// Batch system configurations.
Batch Batch
Security Security
}
// DefaultRPC returns the default RPC config.
func DefaultRPC() RPC {
return RPC{
MaxConnectionCount: 16,
GrpcKeepAliveTime: 10 * time.Second,
GrpcKeepAliveTimeout: 3 * time.Second,
GrpcMaxSendMsgSize: 1<<31 - 1,
GrpcMaxCallMsgSize: 1<<31 - 1,
GrpcInitialWindowSize: 1 << 30,
GrpcInitialConnWindowSize: 1 << 30,
DialTimeout: 5 * time.Second,
ReadTimeoutShort: 20 * time.Second,
ReadTimeoutMedium: 60 * time.Second,
ReadTimeoutLong: 150 * time.Second,
EnableOpenTracing: false,
Batch: DefaultBatch(),
Security: DefaultSecurity(),
}
}
// Batch contains configurations for message batch.
type Batch struct {
// MaxBatchSize is the max batch size when calling batch commands API. Set 0 to
// turn off message batch.
MaxBatchSize uint
// OverloadThreshold is a threshold of TiKV load. If TiKV load is greater than
// this, TiDB will wait for a while to avoid little batch.
OverloadThreshold uint
// MaxWaitSize is the max wait size for batch.
MaxWaitSize uint
// MaxWaitTime is the max wait time for batch.
MaxWaitTime time.Duration
}
// DefaultBatch returns the default Batch config.
func DefaultBatch() Batch {
return Batch{
MaxBatchSize: 0,
OverloadThreshold: 200,
MaxWaitSize: 8,
MaxWaitTime: 0,
}
}
// Security is SSL configuration.
type Security struct {
SSLCA string `toml:"ssl-ca" json:"ssl-ca"`
SSLCert string `toml:"ssl-cert" json:"ssl-cert"`
SSLKey string `toml:"ssl-key" json:"ssl-key"`
}
// ToTLSConfig generates tls's config based on security section of the config.
func (s *Security) ToTLSConfig() (*tls.Config, error) {
var tlsConfig *tls.Config
if len(s.SSLCA) != 0 {
var certificates = make([]tls.Certificate, 0)
if len(s.SSLCert) != 0 && len(s.SSLKey) != 0 {
// Load the client certificates from disk
certificate, err := tls.LoadX509KeyPair(s.SSLCert, s.SSLKey)
if err != nil {
return nil, errors.Errorf("could not load client key pair: %s", err)
}
certificates = append(certificates, certificate)
}
// Create a certificate pool from the certificate authority
certPool := x509.NewCertPool()
ca, err := ioutil.ReadFile(s.SSLCA)
if err != nil {
return nil, errors.Errorf("could not read ca certificate: %s", err)
}
// Append the certificates from the CA
if !certPool.AppendCertsFromPEM(ca) {
return nil, errors.New("failed to append ca certs")
}
tlsConfig = &tls.Config{
Certificates: certificates,
RootCAs: certPool,
}
}
return tlsConfig, nil
}
// DefaultSecurity returns the default Security config.
func DefaultSecurity() Security {
return Security{}
}

125
config/txn.go Normal file
View File

@ -0,0 +1,125 @@
// Copyright 2019 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package config
import "time"
// Txn contains the configurations for transactional kv.
type Txn struct {
// EntrySizeLimit is limit of single entry size (len(key) + len(value)).
EntrySizeLimit int
// EntryCountLimit is a limit of the number of entries in the MemBuffer.
EntryCountLimit int
// TotalSizeLimit is limit of the sum of all entry size.
TotalSizeLimit int
// MaxTimeUse is the max time a transaction can run.
MaxTimeUse int
// DefaultMembufCap is the default transaction membuf capability.
DefaultMembufCap int
// TiKV recommends each RPC packet should be less than ~1MB. We keep each
// packet's Key+Value size below 16KB by default.
CommitBatchSize int
// ScanBatchSize is the limit of an iterator's scan request.
ScanBatchSize int
// BatchGetSize is the max number of keys in a BatchGet request.
BatchGetSize int
// By default, locks after 3000ms is considered unusual (the client created the
// lock might be dead). Other client may cleanup this kind of lock.
// For locks created recently, we will do backoff and retry.
DefaultLockTTL uint64
// The maximum value of a txn's lock TTL.
MaxLockTTL uint64
// ttl = ttlFactor * sqrt(writeSizeInMiB)
TTLFactor int
// ResolveCacheSize is max number of cached txn status.
ResolveCacheSize int
GcSavedSafePoint string
GcSafePointCacheInterval time.Duration
GcCPUTimeInaccuracyBound time.Duration
GcSafePointUpdateInterval time.Duration
GcSafePointQuickRepeatInterval time.Duration
GCTimeout time.Duration
UnsafeDestroyRangeTimeout time.Duration
TsoSlowThreshold time.Duration
OracleUpdateInterval time.Duration
Latch Latch
}
// DefaultTxn returns the default txn config.
func DefaultTxn() Txn {
return Txn{
EntrySizeLimit: 6 * 1024 * 1024,
EntryCountLimit: 300 * 1000,
TotalSizeLimit: 100 * 1024 * 1024,
MaxTimeUse: 590,
DefaultMembufCap: 4 * 1024,
CommitBatchSize: 16 * 1024,
ScanBatchSize: 256,
BatchGetSize: 5120,
DefaultLockTTL: 3000,
MaxLockTTL: 120000,
TTLFactor: 6000,
ResolveCacheSize: 2048,
GcSavedSafePoint: "/tidb/store/gcworker/saved_safe_point",
GcSafePointCacheInterval: time.Second * 100,
GcCPUTimeInaccuracyBound: time.Second,
GcSafePointUpdateInterval: time.Second * 10,
GcSafePointQuickRepeatInterval: time.Second,
GCTimeout: 5 * time.Minute,
UnsafeDestroyRangeTimeout: 5 * time.Minute,
TsoSlowThreshold: 30 * time.Millisecond,
OracleUpdateInterval: 2 * time.Second,
Latch: DefaultLatch(),
}
}
// Latch is the configuration for local latch.
type Latch struct {
// Enable it when there are lots of conflicts between transactions.
Enable bool
Capacity uint
ExpireDuration time.Duration
CheckInterval time.Duration
CheckCounter int
ListCount int
LockChanSize int
}
// DefaultLatch returns the default Latch config.
func DefaultLatch() Latch {
return Latch{
Enable: false,
Capacity: 2048000,
ExpireDuration: 2 * time.Minute,
CheckInterval: time.Minute,
CheckCounter: 50000,
ListCount: 5,
LockChanSize: 100,
}
}

View File

@ -21,7 +21,7 @@ import (
)
func main() {
cli, err := rawkv.NewClient([]string{"127.0.0.1:2379"}, config.Security{})
cli, err := rawkv.NewClient([]string{"127.0.0.1:2379"}, config.Default())
if err != nil {
panic(err)
}

View File

@ -41,7 +41,7 @@ var (
// Init initializes information.
func initStore() {
var err error
client, err = txnkv.NewClient([]string{*pdAddr}, config.Security{})
client, err = txnkv.NewClient([]string{*pdAddr}, config.Default())
if err != nil {
panic(err)
}

View File

@ -38,14 +38,15 @@ type CachedRegion struct {
lastAccess int64
}
func (c *CachedRegion) isValid() bool {
func (c *CachedRegion) isValid(ttl time.Duration) bool {
lastAccess := atomic.LoadInt64(&c.lastAccess)
lastAccessTime := time.Unix(lastAccess, 0)
return time.Since(lastAccessTime) < config.RegionCacheTTL
return time.Since(lastAccessTime) < ttl
}
// RegionCache caches Regions loaded from PD.
type RegionCache struct {
conf *config.RegionCache
pdClient pd.Client
mu struct {
@ -60,12 +61,13 @@ type RegionCache struct {
}
// NewRegionCache creates a RegionCache.
func NewRegionCache(pdClient pd.Client) *RegionCache {
func NewRegionCache(pdClient pd.Client, conf *config.RegionCache) *RegionCache {
c := &RegionCache{
conf: conf,
pdClient: pdClient,
}
c.mu.regions = make(map[RegionVerID]*CachedRegion)
c.mu.sorted = btree.New(config.RegionCacheBTreeDegree)
c.mu.sorted = btree.New(conf.BTreeDegree)
c.storeMu.stores = make(map[uint64]*Store)
return c
}
@ -279,7 +281,7 @@ func (c *RegionCache) getCachedRegion(id RegionVerID) *Region {
if !ok {
return nil
}
if cachedRegion.isValid() {
if cachedRegion.isValid(c.conf.CacheTTL) {
atomic.StoreInt64(&cachedRegion.lastAccess, time.Now().Unix())
return cachedRegion.region
}

View File

@ -48,7 +48,7 @@ type RawResponse struct {
}
func (h rawkvHandler) New(vars map[string]string, r *RawRequest) (*RawResponse, int, error) {
id, err := h.p.New(r.PDAddrs, config.Security{})
id, err := h.p.New(r.PDAddrs, config.Default())
if err != nil {
return nil, http.StatusInternalServerError, err
}

View File

@ -52,7 +52,7 @@ type TxnResponse struct {
}
func (h txnkvHandler) New(vars map[string]string, r *TxnRequest) (*TxnResponse, int, error) {
id, err := h.p.New(r.PDAddrs, config.Security{})
id, err := h.p.New(r.PDAddrs, config.Default())
if err != nil {
return nil, http.StatusInternalServerError, err
}

View File

@ -35,8 +35,8 @@ func NewRaw() RawKVProxy {
}
// New creates a new client and returns the client's UUID.
func (p RawKVProxy) New(pdAddrs []string, security config.Security) (UUID, error) {
client, err := rawkv.NewClient(pdAddrs, security)
func (p RawKVProxy) New(pdAddrs []string, conf config.Config) (UUID, error) {
client, err := rawkv.NewClient(pdAddrs, conf)
if err != nil {
return "", err
}

View File

@ -43,8 +43,8 @@ func NewTxn() TxnKVProxy {
}
// New creates a new client and returns the client's UUID.
func (p TxnKVProxy) New(pdAddrs []string, security config.Security) (UUID, error) {
client, err := txnkv.NewClient(pdAddrs, security)
func (p TxnKVProxy) New(pdAddrs []string, conf config.Config) (UUID, error) {
client, err := txnkv.NewClient(pdAddrs, conf)
if err != nil {
return "", err
}

View File

@ -37,26 +37,28 @@ var (
// only GET/PUT/DELETE commands are supported.
type Client struct {
clusterID uint64
conf *config.Config
regionCache *locate.RegionCache
pdClient pd.Client
rpcClient rpc.Client
}
// NewClient creates a client with PD cluster addrs.
func NewClient(pdAddrs []string, security config.Security) (*Client, error) {
func NewClient(pdAddrs []string, conf config.Config) (*Client, error) {
pdCli, err := pd.NewClient(pdAddrs, pd.SecurityOption{
CAPath: security.SSLCA,
CertPath: security.SSLCert,
KeyPath: security.SSLKey,
CAPath: conf.RPC.Security.SSLCA,
CertPath: conf.RPC.Security.SSLCert,
KeyPath: conf.RPC.Security.SSLKey,
})
if err != nil {
return nil, err
}
return &Client{
clusterID: pdCli.GetClusterID(context.TODO()),
regionCache: locate.NewRegionCache(pdCli),
conf: &conf,
regionCache: locate.NewRegionCache(pdCli, &conf.RegionCache),
pdClient: pdCli,
rpcClient: rpc.NewRPCClient(security),
rpcClient: rpc.NewRPCClient(&conf.RPC),
}, nil
}
@ -269,7 +271,7 @@ func (c *Client) Scan(startKey, endKey []byte, limit int) (keys [][]byte, values
start := time.Now()
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("raw_scan").Observe(time.Since(start).Seconds()) }()
if limit > config.MaxRawKVScanLimit {
if limit > c.conf.Raw.MaxScanLimit {
return nil, nil, errors.WithStack(ErrMaxScanLimitExceeded)
}
@ -315,7 +317,7 @@ func (c *Client) ReverseScan(startKey, endKey []byte, limit int) (keys [][]byte,
metrics.RawkvCmdHistogram.WithLabelValues("raw_reverse_scan").Observe(time.Since(start).Seconds())
}()
if limit > config.MaxRawKVScanLimit {
if limit > c.conf.Raw.MaxScanLimit {
return nil, nil, errors.WithStack(ErrMaxScanLimitExceeded)
}
@ -357,7 +359,7 @@ func (c *Client) sendReq(key []byte, req *rpc.Request) (*rpc.Response, *locate.K
if err != nil {
return nil, nil, err
}
resp, err := sender.SendReq(bo, req, loc.Region, config.ReadTimeoutShort)
resp, err := sender.SendReq(bo, req, loc.Region, c.conf.RPC.ReadTimeoutShort)
if err != nil {
return nil, nil, err
}
@ -384,7 +386,7 @@ func (c *Client) sendBatchReq(bo *retry.Backoffer, keys [][]byte, cmdType rpc.Cm
var batches []batch
for regionID, groupKeys := range groups {
batches = appendKeyBatches(batches, regionID, groupKeys, config.RawBatchPairCount)
batches = appendKeyBatches(batches, regionID, groupKeys, c.conf.Raw.BatchPairCount)
}
bo, cancel := bo.Fork()
ches := make(chan singleBatchResp, len(batches))
@ -443,7 +445,7 @@ func (c *Client) doBatchReq(bo *retry.Backoffer, batch batch, cmdType rpc.CmdTyp
}
sender := rpc.NewRegionRequestSender(c.regionCache, c.rpcClient)
resp, err := sender.SendReq(bo, req, batch.regionID, config.ReadTimeoutShort)
resp, err := sender.SendReq(bo, req, batch.regionID, c.conf.RPC.ReadTimeoutShort)
batchResp := singleBatchResp{}
if err != nil {
@ -511,7 +513,7 @@ func (c *Client) sendDeleteRangeReq(startKey []byte, endKey []byte) (*rpc.Respon
},
}
resp, err := sender.SendReq(bo, req, loc.Region, config.ReadTimeoutShort)
resp, err := sender.SendReq(bo, req, loc.Region, c.conf.RPC.ReadTimeoutShort)
if err != nil {
return nil, nil, err
}
@ -542,7 +544,7 @@ func (c *Client) sendBatchPut(bo *retry.Backoffer, keys, values [][]byte) error
var batches []batch
// split the keys by size and RegionVerID
for regionID, groupKeys := range groups {
batches = appendBatches(batches, regionID, groupKeys, keyToValue, config.RawBatchPutSize)
batches = appendBatches(batches, regionID, groupKeys, keyToValue, c.conf.Raw.MaxBatchPutSize)
}
bo, cancel := bo.Fork()
ch := make(chan error, len(batches))
@ -621,7 +623,7 @@ func (c *Client) doBatchPut(bo *retry.Backoffer, batch batch) error {
}
sender := rpc.NewRegionRequestSender(c.regionCache, c.rpcClient)
resp, err := sender.SendReq(bo, req, batch.regionID, config.ReadTimeoutShort)
resp, err := sender.SendReq(bo, req, batch.regionID, c.conf.RPC.ReadTimeoutShort)
if err != nil {
return err
}

View File

@ -43,9 +43,11 @@ func (s *testRawKVSuite) SetUpTest(c *C) {
mocktikv.BootstrapWithSingleStore(s.cluster)
pdClient := mocktikv.NewPDClient(s.cluster)
mvccStore := mocktikv.MustNewMVCCStore()
conf := config.Default()
s.client = &Client{
conf: &conf,
clusterID: 0,
regionCache: locate.NewRegionCache(pdClient),
regionCache: locate.NewRegionCache(pdClient, &conf.RegionCache),
pdClient: pdClient,
rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore),
}
@ -199,7 +201,7 @@ func (s *testRawKVSuite) TestRawBatch(c *C) {
size := 0
var testKeys [][]byte
var testValues [][]byte
for i := 0; size/config.RawBatchPutSize < 4; i++ {
for i := 0; size/s.client.conf.Raw.MaxBatchPutSize < 4; i++ {
key := fmt.Sprint("key", i)
size += len(key)
testKeys = append(testKeys, []byte(key))

View File

@ -47,8 +47,9 @@ type Client interface {
}
type connArray struct {
conf *config.RPC
index uint32
v []*grpc.ClientConn
conns []*grpc.ClientConn
// Bind with a background goroutine to process coprocessor streaming timeout.
streamTimeout chan *Lease
@ -59,6 +60,7 @@ type connArray struct {
}
type batchCommandsClient struct {
conf *config.Batch
conn *grpc.ClientConn
client tikvpb.Tikv_BatchCommandsClient
batched sync.Map
@ -142,32 +144,33 @@ func (c *batchCommandsClient) batchRecvLoop() {
}
transportLayerLoad := resp.GetTransportLayerLoad()
if transportLayerLoad > 0.0 && config.MaxBatchWaitTime > 0 {
if transportLayerLoad > 0.0 && c.conf.MaxWaitTime > 0 {
// We need to consider TiKV load only if batch-wait strategy is enabled.
atomic.StoreUint64(c.transportLayerLoad, transportLayerLoad)
}
}
}
func newConnArray(maxSize uint, addr string, security config.Security) (*connArray, error) {
func newConnArray(addr string, conf *config.RPC) (*connArray, error) {
a := &connArray{
conf: conf,
index: 0,
v: make([]*grpc.ClientConn, maxSize),
conns: make([]*grpc.ClientConn, conf.GrpcMaxCallMsgSize),
streamTimeout: make(chan *Lease, 1024),
batchCommandsCh: make(chan *batchCommandsEntry, config.MaxBatchSize),
batchCommandsClients: make([]*batchCommandsClient, 0, maxSize),
batchCommandsCh: make(chan *batchCommandsEntry, conf.Batch.MaxBatchSize),
batchCommandsClients: make([]*batchCommandsClient, 0, conf.GrpcMaxCallMsgSize),
transportLayerLoad: 0,
}
if err := a.Init(addr, security); err != nil {
if err := a.Init(addr); err != nil {
return nil, err
}
return a, nil
}
func (a *connArray) Init(addr string, security config.Security) error {
func (a *connArray) Init(addr string) error {
opt := grpc.WithInsecure()
if len(security.SSLCA) != 0 {
tlsConfig, err := security.ToTLSConfig()
if len(a.conf.Security.SSLCA) != 0 {
tlsConfig, err := a.conf.Security.ToTLSConfig()
if err != nil {
return err
}
@ -176,7 +179,7 @@ func (a *connArray) Init(addr string, security config.Security) error {
unaryInterceptor := grpc_prometheus.UnaryClientInterceptor
streamInterceptor := grpc_prometheus.StreamClientInterceptor
if config.EnableOpenTracing {
if a.conf.EnableOpenTracing {
unaryInterceptor = grpc_middleware.ChainUnaryClient(
unaryInterceptor,
grpc_opentracing.UnaryClientInterceptor(),
@ -187,23 +190,23 @@ func (a *connArray) Init(addr string, security config.Security) error {
)
}
allowBatch := config.MaxBatchSize > 0
for i := range a.v {
ctx, cancel := context.WithTimeout(context.Background(), config.DialTimeout)
allowBatch := a.conf.Batch.MaxBatchSize > 0
for i := range a.conns {
ctx, cancel := context.WithTimeout(context.Background(), a.conf.DialTimeout)
conn, err := grpc.DialContext(
ctx,
addr,
opt,
grpc.WithInitialWindowSize(int32(config.GrpcInitialWindowSize)),
grpc.WithInitialConnWindowSize(int32(config.GrpcInitialConnWindowSize)),
grpc.WithInitialWindowSize(int32(a.conf.GrpcInitialWindowSize)),
grpc.WithInitialConnWindowSize(int32(a.conf.GrpcInitialConnWindowSize)),
grpc.WithUnaryInterceptor(unaryInterceptor),
grpc.WithStreamInterceptor(streamInterceptor),
grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(config.MaxCallMsgSize)),
grpc.WithDefaultCallOptions(grpc.MaxCallSendMsgSize(config.MaxSendMsgSize)),
grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(a.conf.GrpcMaxCallMsgSize)),
grpc.WithDefaultCallOptions(grpc.MaxCallSendMsgSize(a.conf.GrpcMaxSendMsgSize)),
grpc.WithBackoffMaxDelay(time.Second*3),
grpc.WithKeepaliveParams(keepalive.ClientParameters{
Time: config.GrpcKeepAliveTime,
Timeout: config.GrpcKeepAliveTimeout,
Time: a.conf.GrpcKeepAliveTime,
Timeout: a.conf.GrpcKeepAliveTimeout,
PermitWithoutStream: true,
}),
)
@ -213,7 +216,7 @@ func (a *connArray) Init(addr string, security config.Security) error {
a.Close()
return errors.WithStack(err)
}
a.v[i] = conn
a.conns[i] = conn
if allowBatch {
// Initialize batch streaming clients.
@ -224,6 +227,7 @@ func (a *connArray) Init(addr string, security config.Security) error {
return errors.WithStack(err)
}
batchClient := &batchCommandsClient{
conf: &a.conf.Batch,
conn: conn,
client: streamClient,
batched: sync.Map{},
@ -244,8 +248,8 @@ func (a *connArray) Init(addr string, security config.Security) error {
}
func (a *connArray) Get() *grpc.ClientConn {
next := atomic.AddUint32(&a.index, 1) % uint32(len(a.v))
return a.v[next]
next := atomic.AddUint32(&a.index, 1) % uint32(len(a.conns))
return a.conns[next]
}
func (a *connArray) Close() {
@ -255,10 +259,10 @@ func (a *connArray) Close() {
atomic.StoreInt32(&c.closed, 1)
}
close(a.batchCommandsCh)
for i, c := range a.v {
for i, c := range a.conns {
if c != nil {
c.Close()
a.v[i] = nil
a.conns[i] = nil
}
}
close(a.streamTimeout)
@ -356,13 +360,15 @@ func (a *connArray) batchSendLoop() {
}
}()
entries := make([]*batchCommandsEntry, 0, config.MaxBatchSize)
requests := make([]*tikvpb.BatchCommandsRequest_Request, 0, config.MaxBatchSize)
requestIDs := make([]uint64, 0, config.MaxBatchSize)
conf := &a.conf.Batch
entries := make([]*batchCommandsEntry, 0, conf.MaxBatchSize)
requests := make([]*tikvpb.BatchCommandsRequest_Request, 0, conf.MaxBatchSize)
requestIDs := make([]uint64, 0, conf.MaxBatchSize)
for {
// Choose a connection by round-robbin.
next := atomic.AddUint32(&a.index, 1) % uint32(len(a.v))
next := atomic.AddUint32(&a.index, 1) % uint32(len(a.conns))
batchCommandsClient := a.batchCommandsClients[next]
entries = entries[:0]
@ -370,15 +376,15 @@ func (a *connArray) batchSendLoop() {
requestIDs = requestIDs[:0]
metrics.PendingBatchRequests.Set(float64(len(a.batchCommandsCh)))
fetchAllPendingRequests(a.batchCommandsCh, int(config.MaxBatchSize), &entries, &requests)
fetchAllPendingRequests(a.batchCommandsCh, int(conf.MaxBatchSize), &entries, &requests)
if len(entries) < int(config.MaxBatchSize) && config.MaxBatchWaitTime > 0 {
if len(entries) < int(conf.MaxBatchSize) && conf.MaxWaitTime > 0 {
transportLayerLoad := atomic.LoadUint64(batchCommandsClient.transportLayerLoad)
// If the target TiKV is overload, wait a while to collect more requests.
if uint(transportLayerLoad) >= config.OverloadThreshold {
if uint(transportLayerLoad) >= conf.OverloadThreshold {
fetchMorePendingRequests(
a.batchCommandsCh, int(config.MaxBatchSize), int(config.BatchWaitSize),
config.MaxBatchWaitTime, &entries, &requests,
a.batchCommandsCh, int(conf.MaxBatchSize), int(conf.MaxWaitSize),
conf.MaxWaitTime, &entries, &requests,
)
}
}
@ -420,14 +426,14 @@ type rpcClient struct {
sync.RWMutex
isClosed bool
conns map[string]*connArray
security config.Security
conf *config.RPC
}
// NewRPCClient manages connections and rpc calls with tikv-servers.
func NewRPCClient(security config.Security) Client {
func NewRPCClient(conf *config.RPC) Client {
return &rpcClient{
conns: make(map[string]*connArray),
security: security,
conns: make(map[string]*connArray),
conf: conf,
}
}
@ -455,7 +461,7 @@ func (c *rpcClient) createConnArray(addr string) (*connArray, error) {
array, ok := c.conns[addr]
if !ok {
var err error
array, err = newConnArray(config.MaxConnectionCount, addr, c.security)
array, err = newConnArray(addr, c.conf)
if err != nil {
return nil, err
}
@ -526,7 +532,7 @@ func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *Request,
return nil, err
}
if config.MaxBatchSize > 0 {
if c.conf.Batch.MaxBatchSize > 0 {
if batchReq := req.ToBatchCommandsRequest(); batchReq != nil {
return sendBatchRequest(ctx, addr, connArray, batchReq, timeout)
}

View File

@ -27,8 +27,8 @@ type Client struct {
}
// NewClient creates a client with PD addresses.
func NewClient(pdAddrs []string, security config.Security) (*Client, error) {
tikvStore, err := store.NewStore(pdAddrs, security)
func NewClient(pdAddrs []string, config config.Config) (*Client, error) {
tikvStore, err := store.NewStore(pdAddrs, config)
if err != nil {
return nil, err
}

View File

@ -31,13 +31,10 @@ type BufferStore struct {
}
// NewBufferStore creates a BufferStore using r for read.
func NewBufferStore(r Retriever, cap int) *BufferStore {
if cap <= 0 {
cap = config.DefaultTxnMembufCap
}
func NewBufferStore(r Retriever, conf *config.Txn) *BufferStore {
return &BufferStore{
r: r,
MemBuffer: &lazyMemBuffer{cap: cap},
MemBuffer: &lazyMemBuffer{conf: conf},
}
}

View File

@ -32,7 +32,8 @@ type testBufferStoreSuite struct{}
var _ = Suite(testBufferStoreSuite{})
func (s testBufferStoreSuite) TestGetSet(c *C) {
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(config.DefaultTxnMembufCap)}, config.DefaultTxnMembufCap)
conf := config.DefaultTxn()
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(&conf, 0)}, &conf)
key := key.Key("key")
_, err := bs.Get(key)
c.Check(err, NotNil)
@ -46,7 +47,8 @@ func (s testBufferStoreSuite) TestGetSet(c *C) {
}
func (s testBufferStoreSuite) TestSaveTo(c *C) {
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(config.DefaultTxnMembufCap)}, config.DefaultTxnMembufCap)
conf := config.DefaultTxn()
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(&conf, 0)}, &conf)
var buf bytes.Buffer
for i := 0; i < 10; i++ {
fmt.Fprint(&buf, i)
@ -56,7 +58,7 @@ func (s testBufferStoreSuite) TestSaveTo(c *C) {
}
bs.Set(key.Key("novalue"), nil)
mutator := NewMemDbBuffer(config.DefaultTxnMembufCap)
mutator := NewMemDbBuffer(&conf, 0)
err := bs.SaveTo(mutator)
c.Check(err, IsNil)

View File

@ -37,12 +37,14 @@ type testKVSuite struct {
}
func (s *testKVSuite) SetUpSuite(c *C) {
conf := config.DefaultTxn()
s.bs = make([]MemBuffer, 1)
s.bs[0] = NewMemDbBuffer(config.DefaultTxnMembufCap)
s.bs[0] = NewMemDbBuffer(&conf, 0)
}
func (s *testKVSuite) ResetMembuffers() {
s.bs[0] = NewMemDbBuffer(config.DefaultTxnMembufCap)
conf := config.DefaultTxn()
s.bs[0] = NewMemDbBuffer(&conf, 0)
}
func insertData(c *C, buffer MemBuffer) {
@ -185,7 +187,8 @@ func (s *testKVSuite) TestNewIteratorMin(c *C) {
}
func (s *testKVSuite) TestBufferLimit(c *C) {
buffer := NewMemDbBuffer(config.DefaultTxnMembufCap).(*memDbBuffer)
conf := config.DefaultTxn()
buffer := NewMemDbBuffer(&conf, 0).(*memDbBuffer)
buffer.bufferSizeLimit = 1000
buffer.entrySizeLimit = 500
@ -197,7 +200,7 @@ func (s *testKVSuite) TestBufferLimit(c *C) {
err = buffer.Set([]byte("yz"), make([]byte, 499))
c.Assert(err, NotNil) // buffer size limit
buffer = NewMemDbBuffer(config.DefaultTxnMembufCap).(*memDbBuffer)
buffer = NewMemDbBuffer(&conf, 0).(*memDbBuffer)
buffer.bufferLenLimit = 10
for i := 0; i < 10; i++ {
err = buffer.Set([]byte{byte(i)}, []byte{byte(i)})
@ -210,35 +213,39 @@ func (s *testKVSuite) TestBufferLimit(c *C) {
var opCnt = 100000
func BenchmarkMemDbBufferSequential(b *testing.B) {
conf := config.DefaultTxn()
data := make([][]byte, opCnt)
for i := 0; i < opCnt; i++ {
data[i] = encodeInt(i)
}
buffer := NewMemDbBuffer(config.DefaultTxnMembufCap)
buffer := NewMemDbBuffer(&conf, 0)
benchmarkSetGet(b, buffer, data)
b.ReportAllocs()
}
func BenchmarkMemDbBufferRandom(b *testing.B) {
conf := config.DefaultTxn()
data := make([][]byte, opCnt)
for i := 0; i < opCnt; i++ {
data[i] = encodeInt(i)
}
shuffle(data)
buffer := NewMemDbBuffer(config.DefaultTxnMembufCap)
buffer := NewMemDbBuffer(&conf, 0)
benchmarkSetGet(b, buffer, data)
b.ReportAllocs()
}
func BenchmarkMemDbIter(b *testing.B) {
buffer := NewMemDbBuffer(config.DefaultTxnMembufCap)
conf := config.DefaultTxn()
buffer := NewMemDbBuffer(&conf, 0)
benchIterator(b, buffer)
b.ReportAllocs()
}
func BenchmarkMemDbCreation(b *testing.B) {
conf := config.DefaultTxn()
for i := 0; i < b.N; i++ {
NewMemDbBuffer(config.DefaultTxnMembufCap)
NewMemDbBuffer(&conf, 0)
}
b.ReportAllocs()
}

View File

@ -17,14 +17,13 @@ package kv
import (
"fmt"
"sync/atomic"
"github.com/pkg/errors"
"github.com/pingcap/goleveldb/leveldb"
"github.com/pingcap/goleveldb/leveldb/comparer"
"github.com/pingcap/goleveldb/leveldb/iterator"
"github.com/pingcap/goleveldb/leveldb/memdb"
"github.com/pingcap/goleveldb/leveldb/util"
"github.com/pkg/errors"
"github.com/tikv/client-go/config"
"github.com/tikv/client-go/key"
)
@ -33,7 +32,7 @@ import (
type memDbBuffer struct {
db *memdb.DB
entrySizeLimit int
bufferLenLimit uint64
bufferLenLimit int
bufferSizeLimit int
}
@ -43,12 +42,15 @@ type memDbIter struct {
}
// NewMemDbBuffer creates a new memDbBuffer.
func NewMemDbBuffer(cap int) MemBuffer {
func NewMemDbBuffer(conf *config.Txn, cap int) MemBuffer {
if cap <= 0 {
cap = conf.DefaultMembufCap
}
return &memDbBuffer{
db: memdb.New(comparer.DefaultComparer, cap),
entrySizeLimit: config.TxnEntrySizeLimit,
bufferLenLimit: atomic.LoadUint64(&config.TxnEntryCountLimit),
bufferSizeLimit: config.TxnTotalSizeLimit,
entrySizeLimit: conf.EntrySizeLimit,
bufferLenLimit: conf.EntryCountLimit,
bufferSizeLimit: conf.TotalSizeLimit,
}
}

View File

@ -72,9 +72,9 @@ type unionStore struct {
}
// NewUnionStore builds a new UnionStore.
func NewUnionStore(snapshot Snapshot) UnionStore {
func NewUnionStore(conf *config.Txn, snapshot Snapshot) UnionStore {
return &unionStore{
BufferStore: NewBufferStore(snapshot, config.DefaultTxnMembufCap),
BufferStore: NewBufferStore(snapshot, conf),
snapshot: snapshot,
lazyConditionPairs: make(map[string]*conditionPair),
opts: make(map[Option]interface{}),
@ -105,8 +105,9 @@ func (it invalidIterator) Close() {}
// lazyMemBuffer wraps a MemBuffer which is to be initialized when it is modified.
type lazyMemBuffer struct {
mb MemBuffer
cap int
mb MemBuffer
cap int
conf *config.Txn
}
func (lmb *lazyMemBuffer) Get(k key.Key) ([]byte, error) {
@ -119,7 +120,7 @@ func (lmb *lazyMemBuffer) Get(k key.Key) ([]byte, error) {
func (lmb *lazyMemBuffer) Set(key key.Key, value []byte) error {
if lmb.mb == nil {
lmb.mb = NewMemDbBuffer(lmb.cap)
lmb.mb = NewMemDbBuffer(lmb.conf, lmb.cap)
}
return lmb.mb.Set(key, value)
@ -127,7 +128,7 @@ func (lmb *lazyMemBuffer) Set(key key.Key, value []byte) error {
func (lmb *lazyMemBuffer) Delete(k key.Key) error {
if lmb.mb == nil {
lmb.mb = NewMemDbBuffer(lmb.cap)
lmb.mb = NewMemDbBuffer(lmb.conf, lmb.cap)
}
return lmb.mb.Delete(k)

View File

@ -27,8 +27,9 @@ type testUnionStoreSuite struct {
}
func (s *testUnionStoreSuite) SetUpTest(c *C) {
s.store = NewMemDbBuffer(config.DefaultTxnMembufCap)
s.us = NewUnionStore(&mockSnapshot{s.store})
conf := config.DefaultTxn()
s.store = NewMemDbBuffer(&conf, 0)
s.us = NewUnionStore(&conf, &mockSnapshot{s.store})
}
func (s *testUnionStoreSuite) TestGetSet(c *C) {

View File

@ -37,6 +37,7 @@ type node struct {
// latch stores a key's waiting transactions information.
type latch struct {
conf *config.Latch
queue *node
count int
waiting []*Lock
@ -94,6 +95,7 @@ func (l *Lock) SetCommitTS(commitTS uint64) {
// Each latch is indexed by a slot's ID, hence the term latch and slot are used in interchangeable,
// but conceptually a latch is a queue, and a slot is an index to the queue
type Latches struct {
conf *config.Latch
slots []latch
}
@ -113,10 +115,11 @@ func (s bytesSlice) Less(i, j int) bool {
// NewLatches create a Latches with fixed length,
// the size will be rounded up to the power of 2.
func NewLatches(size uint) *Latches {
powerOfTwoSize := 1 << uint32(bits.Len32(uint32(size-1)))
func NewLatches(conf *config.Latch) *Latches {
powerOfTwoSize := 1 << uint32(bits.Len32(uint32(conf.Capacity-1)))
slots := make([]latch, powerOfTwoSize)
return &Latches{
conf: conf,
slots: slots,
}
}
@ -228,8 +231,8 @@ func (latches *Latches) acquireSlot(lock *Lock) acquireResult {
defer latch.Unlock()
// Try to recycle to limit the memory usage.
if latch.count >= config.LatchListCount {
latch.recycle(lock.startTS)
if latch.count >= latches.conf.ListCount {
latch.recycle(lock.startTS, latches.conf.ExpireDuration)
}
find := findNode(latch.queue, key)
@ -264,12 +267,12 @@ func (latches *Latches) acquireSlot(lock *Lock) acquireResult {
}
// recycle is not thread safe, the latch should acquire its lock before executing this function.
func (l *latch) recycle(currentTS uint64) int {
func (l *latch) recycle(currentTS uint64, expireDuration time.Duration) int {
total := 0
fakeHead := node{next: l.queue}
prev := &fakeHead
for curr := prev.next; curr != nil; curr = curr.next {
if tsoSub(currentTS, curr.maxCommitTS) >= config.LatchExpireDuration && curr.value == nil {
if tsoSub(currentTS, curr.maxCommitTS) >= expireDuration && curr.value == nil {
l.count--
prev.next = curr.next
total++
@ -286,7 +289,7 @@ func (latches *Latches) recycle(currentTS uint64) {
for i := 0; i < len(latches.slots); i++ {
latch := &latches.slots[i]
latch.Lock()
total += latch.recycle(currentTS)
total += latch.recycle(currentTS, latches.conf.ExpireDuration)
latch.Unlock()
}
log.Debugf("recycle run at %v, recycle count = %d...\n", time.Now(), total)

View File

@ -36,7 +36,9 @@ type testLatchSuite struct {
}
func (s *testLatchSuite) SetUpTest(c *C) {
s.latches = NewLatches(256)
conf := config.DefaultLatch()
conf.Capacity = 256
s.latches = NewLatches(&conf)
}
func (s *testLatchSuite) newLock(keys [][]byte) (startTS uint64, lock *Lock) {
@ -109,7 +111,9 @@ func (s *testLatchSuite) TestFirstAcquireFailedWithStale(c *C) {
}
func (s *testLatchSuite) TestRecycle(c *C) {
latches := NewLatches(8)
conf := config.DefaultLatch()
conf.Capacity = 8
latches := NewLatches(&conf)
now := time.Now()
startTS := oracle.ComposeTS(oracle.GetPhysical(now), 0)
lock := latches.genLock(startTS, [][]byte{
@ -143,7 +147,7 @@ func (s *testLatchSuite) TestRecycle(c *C) {
}
c.Assert(allEmpty, IsFalse)
currentTS := oracle.ComposeTS(oracle.GetPhysical(now.Add(config.LatchExpireDuration)), 3)
currentTS := oracle.ComposeTS(oracle.GetPhysical(now.Add(conf.ExpireDuration)), 3)
latches.recycle(currentTS)
for i := 0; i < len(latches.slots); i++ {

View File

@ -23,6 +23,7 @@ import (
// LatchesScheduler is used to schedule latches for transactions.
type LatchesScheduler struct {
conf *config.Latch
latches *Latches
unlockCh chan *Lock
closed bool
@ -31,10 +32,11 @@ type LatchesScheduler struct {
}
// NewScheduler create the LatchesScheduler.
func NewScheduler(size uint) *LatchesScheduler {
latches := NewLatches(size)
unlockCh := make(chan *Lock, config.LatchLockChanSize)
func NewScheduler(conf *config.Latch) *LatchesScheduler {
latches := NewLatches(conf)
unlockCh := make(chan *Lock, conf.LockChanSize)
scheduler := &LatchesScheduler{
conf: conf,
latches: latches,
unlockCh: unlockCh,
closed: false,
@ -55,7 +57,7 @@ func (scheduler *LatchesScheduler) run() {
if lock.commitTS > lock.startTS {
currentTS := lock.commitTS
elapsed := tsoSub(currentTS, scheduler.lastRecycleTime)
if elapsed > config.LatchCheckInterval || counter > config.LatchCheckCounter {
if elapsed > scheduler.conf.CheckInterval || counter > scheduler.conf.CheckCounter {
go scheduler.latches.recycle(lock.commitTS)
scheduler.lastRecycleTime = currentTS
counter = 0

View File

@ -20,6 +20,7 @@ import (
"time"
. "github.com/pingcap/check"
"github.com/tikv/client-go/config"
)
var _ = Suite(&testSchedulerSuite{})
@ -31,7 +32,9 @@ func (s *testSchedulerSuite) SetUpTest(c *C) {
}
func (s *testSchedulerSuite) TestWithConcurrency(c *C) {
sched := NewScheduler(7)
conf := config.DefaultLatch()
conf.Capacity = 7
sched := NewScheduler(&conf)
defer sched.Close()
rand.Seed(time.Now().Unix())

View File

@ -29,23 +29,25 @@ var _ oracle.Oracle = &pdOracle{}
// pdOracle is an Oracle that uses a placement driver client as source.
type pdOracle struct {
conf *config.Txn
c pd.Client
lastTS uint64
quit chan struct{}
}
// NewPdOracle create an Oracle that uses a pd client source.
// Refer https://github.com/pingcap/pd/blob/master/client/client.go for more details.
// NewPdOracle create an Oracle that uses a pd client source. Refer
// https://github.com/pingcap/pd/blob/master/client/client.go for more details.
// PdOracle mantains `lastTS` to store the last timestamp got from PD server. If
// `GetTimestamp()` is not called after `updateInterval`, it will be called by
// itself to keep up with the timestamp on PD server.
func NewPdOracle(pdClient pd.Client, updateInterval time.Duration) (oracle.Oracle, error) {
// `GetTimestamp()` is not called after `conf.OracleUpdateInterval`, it will be
// called by itself to keep up with the timestamp on PD server.
func NewPdOracle(pdClient pd.Client, conf *config.Txn) (oracle.Oracle, error) {
o := &pdOracle{
conf: conf,
c: pdClient,
quit: make(chan struct{}),
}
ctx := context.TODO()
go o.updateTS(ctx, updateInterval)
go o.updateTS(ctx, conf.OracleUpdateInterval)
// Initialize lastTS by Get.
_, err := o.GetTimestamp(ctx)
if err != nil {
@ -102,7 +104,7 @@ func (o *pdOracle) getTimestamp(ctx context.Context) (uint64, error) {
return 0, err
}
dist := time.Since(now)
if dist > config.TsoSlowThreshold {
if dist > o.conf.TsoSlowThreshold {
log.Warnf("get timestamp too slow: %s", dist)
}
return oracle.ComposeTS(physical, logical), nil

View File

@ -19,7 +19,6 @@ import (
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pkg/errors"
"github.com/tikv/client-go/config"
"github.com/tikv/client-go/retry"
"github.com/tikv/client-go/rpc"
)
@ -51,6 +50,8 @@ func NewDeleteRangeTask(ctx context.Context, store *TiKVStore, startKey []byte,
// Execute performs the delete range operation.
func (t *DeleteRangeTask) Execute() error {
conf := t.store.GetConfig()
startKey, rangeEndKey := t.startKey, t.endKey
for {
select {
@ -80,7 +81,7 @@ func (t *DeleteRangeTask) Execute() error {
},
}
resp, err := t.store.SendReq(bo, req, loc.Region, config.ReadTimeoutMedium)
resp, err := t.store.SendReq(bo, req, loc.Region, conf.RPC.ReadTimeoutMedium)
if err != nil {
return err
}

View File

@ -32,6 +32,7 @@ import (
// LockResolver resolves locks and also caches resolved txn status.
type LockResolver struct {
store *TiKVStore
conf *config.Config
mu struct {
sync.RWMutex
// resolved caches resolved txns (FIFO, txn id -> txnStatus).
@ -43,6 +44,7 @@ type LockResolver struct {
func newLockResolver(store *TiKVStore) *LockResolver {
r := &LockResolver{
store: store,
conf: store.GetConfig(),
}
r.mu.resolved = make(map[uint64]TxnStatus)
r.mu.recentResolved = list.New()
@ -55,8 +57,8 @@ var _ = NewLockResolver
// NewLockResolver creates a LockResolver.
// It is exported for other pkg to use. For instance, binlog service needs
// to determine a transaction's commit state.
func NewLockResolver(etcdAddrs []string, security config.Security) (*LockResolver, error) {
s, err := NewStore(etcdAddrs, security)
func NewLockResolver(etcdAddrs []string, conf config.Config) (*LockResolver, error) {
s, err := NewStore(etcdAddrs, conf)
if err != nil {
return nil, err
}
@ -82,10 +84,10 @@ type Lock struct {
}
// NewLock creates a new *Lock.
func NewLock(l *kvrpcpb.LockInfo) *Lock {
func NewLock(l *kvrpcpb.LockInfo, defaultTTL uint64) *Lock {
ttl := l.GetLockTtl()
if ttl == 0 {
ttl = config.TxnDefaultLockTTL
ttl = defaultTTL
}
return &Lock{
Key: l.GetKey(),
@ -104,7 +106,7 @@ func (lr *LockResolver) saveResolved(txnID uint64, status TxnStatus) {
}
lr.mu.resolved[txnID] = status
lr.mu.recentResolved.PushBack(txnID)
if len(lr.mu.resolved) > config.TxnResolvedCacheSize {
if len(lr.mu.resolved) > lr.conf.Txn.ResolveCacheSize {
front := lr.mu.recentResolved.Front()
delete(lr.mu.resolved, front.Value.(uint64))
lr.mu.recentResolved.Remove(front)
@ -171,7 +173,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *retry.Backoffer, locks []*Lock, lo
},
}
startTime = time.Now()
resp, err := lr.store.SendReq(bo, req, loc, config.ReadTimeoutShort)
resp, err := lr.store.SendReq(bo, req, loc, lr.conf.RPC.ReadTimeoutShort)
if err != nil {
return false, err
}
@ -282,7 +284,7 @@ func (lr *LockResolver) getTxnStatus(bo *retry.Backoffer, txnID uint64, primary
if err != nil {
return status, err
}
resp, err := lr.store.SendReq(bo, req, loc.Region, config.ReadTimeoutShort)
resp, err := lr.store.SendReq(bo, req, loc.Region, lr.conf.RPC.ReadTimeoutShort)
if err != nil {
return status, err
}
@ -336,7 +338,7 @@ func (lr *LockResolver) resolveLock(bo *retry.Backoffer, l *Lock, status TxnStat
if status.IsCommitted() {
req.ResolveLock.CommitVersion = status.CommitTS()
}
resp, err := lr.store.SendReq(bo, req, loc.Region, config.ReadTimeoutShort)
resp, err := lr.store.SendReq(bo, req, loc.Region, lr.conf.RPC.ReadTimeoutShort)
if err != nil {
return err
}

View File

@ -24,7 +24,6 @@ import (
grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
"github.com/pkg/errors"
log "github.com/sirupsen/logrus"
"github.com/tikv/client-go/config"
"google.golang.org/grpc"
)
@ -120,7 +119,7 @@ func (w *EtcdSafePointKV) Get(k string) (string, error) {
func saveSafePoint(kv SafePointKV, key string, t uint64) error {
s := strconv.FormatUint(t, 10)
err := kv.Put(config.GcSavedSafePoint, s)
err := kv.Put(key, s)
if err != nil {
log.Error("save safepoint failed:", err)
return err
@ -129,7 +128,7 @@ func saveSafePoint(kv SafePointKV, key string, t uint64) error {
}
func loadSafePoint(kv SafePointKV, key string) (uint64, error) {
str, err := kv.Get(config.GcSavedSafePoint)
str, err := kv.Get(key)
if err != nil {
return 0, err

View File

@ -30,6 +30,7 @@ import (
// Scanner support tikv scan
type Scanner struct {
snapshot *TiKVSnapshot
conf *config.Config
batchSize int
valid bool
cache []*pb.KvPair
@ -42,10 +43,11 @@ type Scanner struct {
func newScanner(snapshot *TiKVSnapshot, startKey []byte, endKey []byte, batchSize int) (*Scanner, error) {
// It must be > 1. Otherwise scanner won't skipFirst.
if batchSize <= 1 {
batchSize = config.TxnScanBatchSize
batchSize = snapshot.conf.Txn.ScanBatchSize
}
scanner := &Scanner{
snapshot: snapshot,
conf: snapshot.conf,
batchSize: batchSize,
valid: true,
nextStartKey: startKey,
@ -175,7 +177,7 @@ func (s *Scanner) getData(bo *retry.Backoffer) error {
NotFillCache: s.snapshot.NotFillCache,
},
}
resp, err := sender.SendReq(bo, req, loc.Region, config.ReadTimeoutMedium)
resp, err := sender.SendReq(bo, req, loc.Region, s.conf.RPC.ReadTimeoutMedium)
if err != nil {
return err
}
@ -205,7 +207,7 @@ func (s *Scanner) getData(bo *retry.Backoffer) error {
// Check if kvPair contains error, it should be a Lock.
for _, pair := range kvPairs {
if keyErr := pair.GetError(); keyErr != nil {
lock, err := extractLockFromKeyErr(keyErr)
lock, err := extractLockFromKeyErr(keyErr, s.conf.Txn.DefaultLockTTL)
if err != nil {
return err
}

View File

@ -36,6 +36,7 @@ import (
type TiKVSnapshot struct {
store *TiKVStore
ts uint64
conf *config.Config
Priority pb.CommandPri
NotFillCache bool
@ -47,6 +48,7 @@ func newTiKVSnapshot(store *TiKVStore, ts uint64) *TiKVSnapshot {
return &TiKVSnapshot{
store: store,
ts: ts,
conf: store.GetConfig(),
Priority: pb.CommandPri_Normal,
}
}
@ -98,7 +100,7 @@ func (s *TiKVSnapshot) batchGetKeysByRegions(bo *retry.Backoffer, keys [][]byte,
var batches []batchKeys
for id, g := range groups {
batches = appendBatchBySize(batches, id, g, func([]byte) int { return 1 }, config.TxnBatchGetSize)
batches = appendBatchBySize(batches, id, g, func([]byte) int { return 1 }, s.conf.Txn.BatchGetSize)
}
if len(batches) == 0 {
@ -141,7 +143,7 @@ func (s *TiKVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys
NotFillCache: s.NotFillCache,
},
}
resp, err := sender.SendReq(bo, req, batch.region, config.ReadTimeoutMedium)
resp, err := sender.SendReq(bo, req, batch.region, s.conf.RPC.ReadTimeoutMedium)
if err != nil {
return err
}
@ -170,7 +172,7 @@ func (s *TiKVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys
collectF(pair.GetKey(), pair.GetValue())
continue
}
lock, err := extractLockFromKeyErr(keyErr)
lock, err := extractLockFromKeyErr(keyErr, s.conf.Txn.DefaultLockTTL)
if err != nil {
return err
}
@ -226,7 +228,7 @@ func (s *TiKVSnapshot) get(bo *retry.Backoffer, k key.Key) ([]byte, error) {
if err != nil {
return nil, err
}
resp, err := sender.SendReq(bo, req, loc.Region, config.ReadTimeoutShort)
resp, err := sender.SendReq(bo, req, loc.Region, s.conf.RPC.ReadTimeoutShort)
if err != nil {
return nil, err
}
@ -247,7 +249,7 @@ func (s *TiKVSnapshot) get(bo *retry.Backoffer, k key.Key) ([]byte, error) {
}
val := cmdGetResp.GetValue()
if keyErr := cmdGetResp.GetError(); keyErr != nil {
lock, err := extractLockFromKeyErr(keyErr)
lock, err := extractLockFromKeyErr(keyErr, s.conf.Txn.DefaultLockTTL)
if err != nil {
return nil, err
}
@ -269,7 +271,7 @@ func (s *TiKVSnapshot) get(bo *retry.Backoffer, k key.Key) ([]byte, error) {
// Iter returns a list of key-value pair after `k`.
func (s *TiKVSnapshot) Iter(k key.Key, upperBound key.Key) (kv.Iterator, error) {
scanner, err := newScanner(s, k, upperBound, config.TxnScanBatchSize)
scanner, err := newScanner(s, k, upperBound, s.conf.Txn.ScanBatchSize)
return scanner, err
}
@ -283,9 +285,9 @@ func (s *TiKVSnapshot) SetPriority(priority int) {
s.Priority = pb.CommandPri(priority)
}
func extractLockFromKeyErr(keyErr *pb.KeyError) (*Lock, error) {
func extractLockFromKeyErr(keyErr *pb.KeyError, defaultTTL uint64) (*Lock, error) {
if locked := keyErr.GetLocked(); locked != nil {
return NewLock(locked), nil
return NewLock(locked, defaultTTL), nil
}
if keyErr.Conflict != nil {
err := errors.New(conflictToString(keyErr.Conflict))

View File

@ -20,7 +20,6 @@ import (
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pkg/errors"
log "github.com/sirupsen/logrus"
"github.com/tikv/client-go/config"
"github.com/tikv/client-go/key"
"github.com/tikv/client-go/retry"
"github.com/tikv/client-go/rpc"
@ -39,6 +38,7 @@ func SplitRegion(store *TiKVStore, splitKey key.Key) error {
},
}
req.Context.Priority = kvrpcpb.CommandPri_Normal
conf := store.GetConfig()
for {
loc, err := store.GetRegionCache().LocateKey(bo, splitKey)
if err != nil {
@ -48,7 +48,7 @@ func SplitRegion(store *TiKVStore, splitKey key.Key) error {
log.Infof("skip split_region region at %q", splitKey)
return nil
}
res, err := sender.SendReq(bo, req, loc.Region, config.ReadTimeoutShort)
res, err := sender.SendReq(bo, req, loc.Region, conf.RPC.ReadTimeoutShort)
if err != nil {
return err
}

View File

@ -35,6 +35,7 @@ import (
// TiKVStore contains methods to interact with a TiKV cluster.
type TiKVStore struct {
conf *config.Config
clusterID uint64
uuid string
oracle oracle.Oracle
@ -54,22 +55,24 @@ type TiKVStore struct {
}
// NewStore creates a TiKVStore instance.
func NewStore(pdAddrs []string, security config.Security) (*TiKVStore, error) {
func NewStore(pdAddrs []string, conf config.Config) (*TiKVStore, error) {
pdCli, err := pd.NewClient(pdAddrs, pd.SecurityOption{
CAPath: security.SSLCA,
CertPath: security.SSLCert,
KeyPath: security.SSLKey,
CAPath: conf.RPC.Security.SSLCA,
CertPath: conf.RPC.Security.SSLCert,
KeyPath: conf.RPC.Security.SSLKey,
})
if err != nil {
return nil, err
}
oracle, err := oracles.NewPdOracle(pdCli, time.Duration(config.OracleUpdateInterval)*time.Millisecond)
pdClient := &locate.CodecPDClient{Client: pdCli}
oracle, err := oracles.NewPdOracle(pdCli, &conf.Txn)
if err != nil {
return nil, err
}
tlsConfig, err := security.ToTLSConfig()
tlsConfig, err := conf.RPC.Security.ToTLSConfig()
if err != nil {
return nil, err
}
@ -82,12 +85,13 @@ func NewStore(pdAddrs []string, security config.Security) (*TiKVStore, error) {
clusterID := pdCli.GetClusterID(context.TODO())
store := &TiKVStore{
conf: &conf,
clusterID: clusterID,
uuid: fmt.Sprintf("tikv-%d", clusterID),
oracle: oracle,
client: rpc.NewRPCClient(security),
pdClient: &locate.CodecPDClient{Client: pdCli},
regionCache: locate.NewRegionCache(pdCli),
client: rpc.NewRPCClient(&conf.RPC),
pdClient: pdClient,
regionCache: locate.NewRegionCache(pdClient, &conf.RegionCache),
etcdAddrs: pdAddrs,
tlsConfig: tlsConfig,
spkv: spkv,
@ -97,14 +101,19 @@ func NewStore(pdAddrs []string, security config.Security) (*TiKVStore, error) {
store.lockResolver = newLockResolver(store)
if config.EnableTxnLocalLatch {
store.txnLatches = latch.NewScheduler(config.TxnLocalLatchCapacity)
if conf.Txn.Latch.Enable {
store.txnLatches = latch.NewScheduler(&conf.Txn.Latch)
}
go store.runSafePointChecker()
return store, nil
}
// GetConfig returns the store's configurations.
func (s *TiKVStore) GetConfig() *config.Config {
return s.conf
}
// GetLockResolver returns the lock resolver instance.
func (s *TiKVStore) GetLockResolver() *LockResolver {
return s.lockResolver
@ -177,21 +186,21 @@ func (s *TiKVStore) GetTimestampWithRetry(bo *retry.Backoffer) (uint64, error) {
}
func (s *TiKVStore) runSafePointChecker() {
d := config.GcSafePointUpdateInterval
d := s.conf.Txn.GcSafePointUpdateInterval
for {
select {
case spCachedTime := <-time.After(d):
cachedSafePoint, err := loadSafePoint(s.spkv, config.GcSavedSafePoint)
cachedSafePoint, err := loadSafePoint(s.spkv, s.conf.Txn.GcSavedSafePoint)
if err == nil {
metrics.LoadSafepointCounter.WithLabelValues("ok").Inc()
s.spMutex.Lock()
s.safePoint, s.spTime = cachedSafePoint, spCachedTime
s.spMutex.Unlock()
d = config.GcSafePointUpdateInterval
d = s.conf.Txn.GcSafePointUpdateInterval
} else {
metrics.LoadSafepointCounter.WithLabelValues("fail").Inc()
log.Errorf("fail to load safepoint from pd: %v", err)
d = config.GcSafePointQuickRepeatInterval
d = s.conf.Txn.GcSafePointQuickRepeatInterval
}
case <-s.Closed():
return
@ -208,7 +217,7 @@ func (s *TiKVStore) CheckVisibility(startTS uint64) error {
s.spMutex.RUnlock()
diff := time.Since(cachedTime)
if diff > (config.GcSafePointCacheInterval - config.GcCPUTimeInaccuracyBound) {
if diff > (s.conf.Txn.GcSafePointCacheInterval - s.conf.Txn.GcCPUTimeInaccuracyBound) {
return errors.WithStack(ErrPDServerTimeout)
}

View File

@ -63,6 +63,7 @@ type TxnCommitter struct {
ConnID uint64 // ConnID is used for log.
store *TiKVStore
conf *config.Config
startTS uint64
keys [][]byte
mutations map[string]*pb.Mutation
@ -91,6 +92,7 @@ func NewTxnCommitter(store *TiKVStore, startTS uint64, startTime time.Time, muta
lockCnt int
)
conf := store.GetConfig()
for key, mut := range mutations {
switch mut.Op {
case pb.Op_Put, pb.Op_Insert:
@ -102,7 +104,7 @@ func NewTxnCommitter(store *TiKVStore, startTS uint64, startTime time.Time, muta
}
keys = append(keys, []byte(key))
entrySize := len(mut.Key) + len(mut.Value)
if entrySize > config.TxnEntrySizeLimit {
if entrySize > conf.Txn.EntrySizeLimit {
return nil, kv.ErrEntryTooLarge
}
size += entrySize
@ -112,21 +114,22 @@ func NewTxnCommitter(store *TiKVStore, startTS uint64, startTime time.Time, muta
return nil, nil
}
if len(keys) > int(config.TxnEntryCountLimit) || size > config.TxnTotalSizeLimit {
if len(keys) > int(conf.Txn.EntryCountLimit) || size > conf.Txn.TotalSizeLimit {
return nil, kv.ErrTxnTooLarge
}
// Convert from sec to ms
maxTxnTimeUse := uint64(config.MaxTxnTimeUse) * 1000
maxTxnTimeUse := uint64(conf.Txn.MaxTimeUse) * 1000
metrics.TxnWriteKVCountHistogram.Observe(float64(len(keys)))
metrics.TxnWriteSizeHistogram.Observe(float64(size))
return &TxnCommitter{
store: store,
conf: conf,
startTS: startTS,
keys: keys,
mutations: mutations,
lockTTL: txnLockTTL(startTime, size),
lockTTL: txnLockTTL(conf, startTime, size),
maxTxnTimeUse: maxTxnTimeUse,
detail: CommitDetails{WriteSize: size, WriteKeys: len(keys)},
}, nil
@ -138,20 +141,20 @@ func (c *TxnCommitter) primary() []byte {
const bytesPerMiB = 1024 * 1024
func txnLockTTL(startTime time.Time, txnSize int) uint64 {
func txnLockTTL(conf *config.Config, startTime time.Time, txnSize int) uint64 {
// Increase lockTTL for large transactions.
// The formula is `ttl = ttlFactor * sqrt(sizeInMiB)`.
// When writeSize is less than 256KB, the base ttl is defaultTTL (3s);
// When writeSize is 1MiB, 100MiB, or 400MiB, ttl is 6s, 60s, 120s correspondingly;
lockTTL := config.TxnDefaultLockTTL
if txnSize >= config.TxnCommitBatchSize {
lockTTL := conf.Txn.DefaultLockTTL
if txnSize >= conf.Txn.CommitBatchSize {
sizeMiB := float64(txnSize) / bytesPerMiB
lockTTL = uint64(float64(config.TxnTTLFactor) * math.Sqrt(sizeMiB))
if lockTTL < config.TxnDefaultLockTTL {
lockTTL = config.TxnDefaultLockTTL
lockTTL = uint64(float64(conf.Txn.TTLFactor) * math.Sqrt(sizeMiB))
if lockTTL < conf.Txn.DefaultLockTTL {
lockTTL = conf.Txn.DefaultLockTTL
}
if lockTTL > config.TxnMaxLockTTL {
lockTTL = config.TxnMaxLockTTL
if lockTTL > conf.Txn.MaxLockTTL {
lockTTL = conf.Txn.MaxLockTTL
}
}
@ -183,10 +186,11 @@ func (c *TxnCommitter) doActionOnKeys(bo *retry.Backoffer, action commitAction,
atomic.AddInt32(&c.detail.PrewriteRegionNum, int32(len(groups)))
}
// Make sure the group that contains primary key goes first.
batches = appendBatchBySize(batches, firstRegion, groups[firstRegion], sizeFunc, config.TxnCommitBatchSize)
commitBatchSize := c.conf.Txn.CommitBatchSize
batches = appendBatchBySize(batches, firstRegion, groups[firstRegion], sizeFunc, commitBatchSize)
delete(groups, firstRegion)
for id, g := range groups {
batches = appendBatchBySize(batches, id, g, sizeFunc, config.TxnCommitBatchSize)
batches = appendBatchBySize(batches, id, g, sizeFunc, commitBatchSize)
}
firstIsPrimary := bytes.Equal(keys[0], c.primary())
@ -318,7 +322,7 @@ func (c *TxnCommitter) prewriteSingleBatch(bo *retry.Backoffer, batch batchKeys)
},
}
for {
resp, err := c.store.SendReq(bo, req, batch.region, config.ReadTimeoutShort)
resp, err := c.store.SendReq(bo, req, batch.region, c.conf.RPC.ReadTimeoutShort)
if err != nil {
return err
}
@ -349,7 +353,7 @@ func (c *TxnCommitter) prewriteSingleBatch(bo *retry.Backoffer, batch batchKeys)
}
// Extract lock from key error
lock, err1 := extractLockFromKeyErr(keyErr)
lock, err1 := extractLockFromKeyErr(keyErr, c.conf.Txn.DefaultLockTTL)
if err1 != nil {
return err1
}
@ -399,7 +403,7 @@ func (c *TxnCommitter) commitSingleBatch(bo *retry.Backoffer, batch batchKeys) e
req.Context.Priority = c.Priority
sender := rpc.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetRPCClient())
resp, err := sender.SendReq(bo, req, batch.region, config.ReadTimeoutShort)
resp, err := sender.SendReq(bo, req, batch.region, c.conf.RPC.ReadTimeoutShort)
// If we fail to receive response for the request that commits primary key, it will be undetermined whether this
// transaction has been successfully committed.
@ -470,7 +474,7 @@ func (c *TxnCommitter) cleanupSingleBatch(bo *retry.Backoffer, batch batchKeys)
SyncLog: c.SyncLog,
},
}
resp, err := c.store.SendReq(bo, req, batch.region, config.ReadTimeoutShort)
resp, err := c.store.SendReq(bo, req, batch.region, c.conf.RPC.ReadTimeoutShort)
if err != nil {
return err
}

View File

@ -44,7 +44,7 @@ type Transaction struct {
func newTransaction(tikvStore *store.TiKVStore, ts uint64) *Transaction {
snapshot := tikvStore.GetSnapshot(ts)
us := kv.NewUnionStore(snapshot)
us := kv.NewUnionStore(&tikvStore.GetConfig().Txn, snapshot)
return &Transaction{
tikvStore: tikvStore,
snapshot: snapshot,