Skip to content

Commit ea8b881

Browse files
disksingsiddontang
authored andcommitted
Move config variables to config package (#9)
* cleanup configurations Signed-off-by: disksing <[email protected]>
1 parent 970883c commit ea8b881

23 files changed

+191
-176
lines changed

config/config.go

+104
Original file line numberDiff line numberDiff line change
@@ -97,3 +97,107 @@ var (
9797
EnableTxnLocalLatch = false
9898
TxnLocalLatchCapacity uint = 2048000
9999
)
100+
101+
// RegionCache configurations.
102+
var (
103+
RegionCacheBTreeDegree = 32
104+
RegionCacheTTL = time.Minute * 10
105+
)
106+
107+
// RawKV configurations.
108+
var (
109+
// MaxRawKVScanLimit is the maximum scan limit for rawkv Scan.
110+
MaxRawKVScanLimit = 10240
111+
// RawBatchPutSize is the maximum size limit for rawkv each batch put request.
112+
RawBatchPutSize = 16 * 1024
113+
// RawBatchPairCount is the maximum limit for rawkv each batch get/delete request.
114+
RawBatchPairCount = 512
115+
)
116+
117+
// RPC configurations.
118+
var (
119+
// MaxConnectionCount is the max gRPC connections that will be established with
120+
// each tikv-server.
121+
MaxConnectionCount uint = 16
122+
123+
// GrpcKeepAliveTime is the duration of time after which if the client doesn't see
124+
// any activity it pings the server to see if the transport is still alive.
125+
GrpcKeepAliveTime = time.Duration(10) * time.Second
126+
127+
// GrpcKeepAliveTimeout is the duration of time for which the client waits after having
128+
// pinged for keepalive check and if no activity is seen even after that the connection
129+
// is closed.
130+
GrpcKeepAliveTimeout = time.Duration(3) * time.Second
131+
132+
// MaxSendMsgSize set max gRPC request message size sent to server. If any request message size is larger than
133+
// current value, an error will be reported from gRPC.
134+
MaxSendMsgSize = 1<<31 - 1
135+
136+
// MaxCallMsgSize set max gRPC receive message size received from server. If any message size is larger than
137+
// current value, an error will be reported from gRPC.
138+
MaxCallMsgSize = 1<<31 - 1
139+
140+
DialTimeout = 5 * time.Second
141+
ReadTimeoutShort = 20 * time.Second // For requests that read/write several key-values.
142+
ReadTimeoutMedium = 60 * time.Second // For requests that may need scan region.
143+
ReadTimeoutLong = 150 * time.Second // For requests that may need scan region multiple times.
144+
GCTimeout = 5 * time.Minute
145+
UnsafeDestroyRangeTimeout = 5 * time.Minute
146+
147+
GrpcInitialWindowSize = 1 << 30
148+
GrpcInitialConnWindowSize = 1 << 30
149+
)
150+
151+
// KV configurations.
152+
var (
153+
// DefaultTxnMembufCap is the default transaction membuf capability.
154+
DefaultTxnMembufCap = 4 * 1024
155+
)
156+
157+
// Latch configurations.
158+
var (
159+
LatchExpireDuration = 2 * time.Minute
160+
LatchCheckInterval = 1 * time.Minute
161+
LatchCheckCounter = 50000
162+
LatchListCount = 5
163+
LatchLockChanSize = 100
164+
)
165+
166+
// Oracle configurations.
167+
var (
168+
TsoSlowThreshold = 30 * time.Millisecond
169+
// update oracle's lastTS every 2000ms.
170+
OracleUpdateInterval = 2000
171+
)
172+
173+
// Txn configurations.
174+
var (
175+
// TiKV recommends each RPC packet should be less than ~1MB. We keep each packet's
176+
// Key+Value size below 16KB.
177+
TxnCommitBatchSize = 16 * 1024
178+
179+
// By default, locks after 3000ms is considered unusual (the client created the
180+
// lock might be dead). Other client may cleanup this kind of lock.
181+
// For locks created recently, we will do backoff and retry.
182+
TxnDefaultLockTTL uint64 = 3000
183+
// TODO: Consider if it's appropriate.
184+
TxnMaxLockTTL uint64 = 120000
185+
// ttl = ttlFactor * sqrt(writeSizeInMiB)
186+
TxnTTLFactor = 6000
187+
// TxnResolvedCacheSize is max number of cached txn status.
188+
TxnResolvedCacheSize = 2048
189+
190+
// SafePoint.
191+
// This is almost the same as 'tikv_gc_safe_point' in the table 'mysql.tidb',
192+
// save this to pd instead of tikv, because we can't use interface of table
193+
// if the safepoint on tidb is expired.
194+
GcSavedSafePoint = "/tidb/store/gcworker/saved_safe_point"
195+
196+
GcSafePointCacheInterval = time.Second * 100
197+
GcCPUTimeInaccuracyBound = time.Second
198+
GcSafePointUpdateInterval = time.Second * 10
199+
GcSafePointQuickRepeatInterval = time.Second
200+
201+
TxnScanBatchSize = 256
202+
TxnBatchGetSize = 5120
203+
)

locate/region_cache.go

+3-7
Original file line numberDiff line numberDiff line change
@@ -27,15 +27,11 @@ import (
2727
"github.com/pkg/errors"
2828
log "github.com/sirupsen/logrus"
2929
"github.com/tikv/client-go/codec"
30+
"github.com/tikv/client-go/config"
3031
"github.com/tikv/client-go/metrics"
3132
"github.com/tikv/client-go/retry"
3233
)
3334

34-
const (
35-
btreeDegree = 32
36-
rcDefaultRegionCacheTTL = time.Minute * 10
37-
)
38-
3935
// CachedRegion encapsulates {Region, TTL}
4036
type CachedRegion struct {
4137
region *Region
@@ -45,7 +41,7 @@ type CachedRegion struct {
4541
func (c *CachedRegion) isValid() bool {
4642
lastAccess := atomic.LoadInt64(&c.lastAccess)
4743
lastAccessTime := time.Unix(lastAccess, 0)
48-
return time.Since(lastAccessTime) < rcDefaultRegionCacheTTL
44+
return time.Since(lastAccessTime) < config.RegionCacheTTL
4945
}
5046

5147
// RegionCache caches Regions loaded from PD.
@@ -69,7 +65,7 @@ func NewRegionCache(pdClient pd.Client) *RegionCache {
6965
pdClient: pdClient,
7066
}
7167
c.mu.regions = make(map[RegionVerID]*CachedRegion)
72-
c.mu.sorted = btree.New(btreeDegree)
68+
c.mu.sorted = btree.New(config.RegionCacheBTreeDegree)
7369
c.storeMu.stores = make(map[uint64]*Store)
7470
return c
7571
}

rawkv/rawkv.go

+7-16
Original file line numberDiff line numberDiff line change
@@ -29,19 +29,10 @@ import (
2929
)
3030

3131
var (
32-
// MaxRawKVScanLimit is the maximum scan limit for rawkv Scan.
33-
MaxRawKVScanLimit = 10240
3432
// ErrMaxScanLimitExceeded is returned when the limit for rawkv Scan is to large.
3533
ErrMaxScanLimitExceeded = errors.New("limit should be less than MaxRawKVScanLimit")
3634
)
3735

38-
const (
39-
// rawBatchPutSize is the maximum size limit for rawkv each batch put request.
40-
rawBatchPutSize = 16 * 1024
41-
// rawBatchPairCount is the maximum limit for rawkv each batch get/delete request.
42-
rawBatchPairCount = 512
43-
)
44-
4536
// RawKVClient is a client of TiKV server which is used as a key-value storage,
4637
// only GET/PUT/DELETE commands are supported.
4738
type RawKVClient struct {
@@ -278,7 +269,7 @@ func (c *RawKVClient) Scan(startKey, endKey []byte, limit int) (keys [][]byte, v
278269
start := time.Now()
279270
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("raw_scan").Observe(time.Since(start).Seconds()) }()
280271

281-
if limit > MaxRawKVScanLimit {
272+
if limit > config.MaxRawKVScanLimit {
282273
return nil, nil, errors.WithStack(ErrMaxScanLimitExceeded)
283274
}
284275

@@ -319,7 +310,7 @@ func (c *RawKVClient) sendReq(key []byte, req *rpc.Request) (*rpc.Response, *loc
319310
if err != nil {
320311
return nil, nil, err
321312
}
322-
resp, err := sender.SendReq(bo, req, loc.Region, rpc.ReadTimeoutShort)
313+
resp, err := sender.SendReq(bo, req, loc.Region, config.ReadTimeoutShort)
323314
if err != nil {
324315
return nil, nil, err
325316
}
@@ -346,7 +337,7 @@ func (c *RawKVClient) sendBatchReq(bo *retry.Backoffer, keys [][]byte, cmdType r
346337

347338
var batches []batch
348339
for regionID, groupKeys := range groups {
349-
batches = appendKeyBatches(batches, regionID, groupKeys, rawBatchPairCount)
340+
batches = appendKeyBatches(batches, regionID, groupKeys, config.RawBatchPairCount)
350341
}
351342
bo, cancel := bo.Fork()
352343
ches := make(chan singleBatchResp, len(batches))
@@ -405,7 +396,7 @@ func (c *RawKVClient) doBatchReq(bo *retry.Backoffer, batch batch, cmdType rpc.C
405396
}
406397

407398
sender := rpc.NewRegionRequestSender(c.regionCache, c.rpcClient)
408-
resp, err := sender.SendReq(bo, req, batch.regionID, rpc.ReadTimeoutShort)
399+
resp, err := sender.SendReq(bo, req, batch.regionID, config.ReadTimeoutShort)
409400

410401
batchResp := singleBatchResp{}
411402
if err != nil {
@@ -473,7 +464,7 @@ func (c *RawKVClient) sendDeleteRangeReq(startKey []byte, endKey []byte) (*rpc.R
473464
},
474465
}
475466

476-
resp, err := sender.SendReq(bo, req, loc.Region, rpc.ReadTimeoutShort)
467+
resp, err := sender.SendReq(bo, req, loc.Region, config.ReadTimeoutShort)
477468
if err != nil {
478469
return nil, nil, err
479470
}
@@ -504,7 +495,7 @@ func (c *RawKVClient) sendBatchPut(bo *retry.Backoffer, keys, values [][]byte) e
504495
var batches []batch
505496
// split the keys by size and RegionVerID
506497
for regionID, groupKeys := range groups {
507-
batches = appendBatches(batches, regionID, groupKeys, keyToValue, rawBatchPutSize)
498+
batches = appendBatches(batches, regionID, groupKeys, keyToValue, config.RawBatchPutSize)
508499
}
509500
bo, cancel := bo.Fork()
510501
ch := make(chan error, len(batches))
@@ -583,7 +574,7 @@ func (c *RawKVClient) doBatchPut(bo *retry.Backoffer, batch batch) error {
583574
}
584575

585576
sender := rpc.NewRegionRequestSender(c.regionCache, c.rpcClient)
586-
resp, err := sender.SendReq(bo, req, batch.regionID, rpc.ReadTimeoutShort)
577+
resp, err := sender.SendReq(bo, req, batch.regionID, config.ReadTimeoutShort)
587578
if err != nil {
588579
return err
589580
}

rawkv/rawkv_test.go

+2-1
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ import (
2020
"testing"
2121

2222
. "github.com/pingcap/check"
23+
"github.com/tikv/client-go/config"
2324
"github.com/tikv/client-go/locate"
2425
"github.com/tikv/client-go/mockstore/mocktikv"
2526
"github.com/tikv/client-go/retry"
@@ -178,7 +179,7 @@ func (s *testRawKVSuite) TestRawBatch(c *C) {
178179
size := 0
179180
var testKeys [][]byte
180181
var testValues [][]byte
181-
for i := 0; size/rawBatchPutSize < 4; i++ {
182+
for i := 0; size/config.RawBatchPutSize < 4; i++ {
182183
key := fmt.Sprint("key", i)
183184
size += len(key)
184185
testKeys = append(testKeys, []byte(key))

rpc/client.go

+8-42
Original file line numberDiff line numberDiff line change
@@ -37,40 +37,6 @@ import (
3737
gstatus "google.golang.org/grpc/status"
3838
)
3939

40-
// MaxConnectionCount is the max gRPC connections that will be established with
41-
// each tikv-server.
42-
var MaxConnectionCount uint = 16
43-
44-
// GrpcKeepAliveTime is the duration of time after which if the client doesn't see
45-
// any activity it pings the server to see if the transport is still alive.
46-
var GrpcKeepAliveTime = time.Duration(10) * time.Second
47-
48-
// GrpcKeepAliveTimeout is the duration of time for which the client waits after having
49-
// pinged for keepalive check and if no activity is seen even after that the connection
50-
// is closed.
51-
var GrpcKeepAliveTimeout = time.Duration(3) * time.Second
52-
53-
// MaxSendMsgSize set max gRPC request message size sent to server. If any request message size is larger than
54-
// current value, an error will be reported from gRPC.
55-
var MaxSendMsgSize = 1<<31 - 1
56-
57-
// MaxCallMsgSize set max gRPC receive message size received from server. If any message size is larger than
58-
// current value, an error will be reported from gRPC.
59-
var MaxCallMsgSize = 1<<31 - 1
60-
61-
// Timeout durations.
62-
const (
63-
dialTimeout = 5 * time.Second
64-
ReadTimeoutShort = 20 * time.Second // For requests that read/write several key-values.
65-
ReadTimeoutMedium = 60 * time.Second // For requests that may need scan region.
66-
ReadTimeoutLong = 150 * time.Second // For requests that may need scan region multiple times.
67-
GCTimeout = 5 * time.Minute
68-
UnsafeDestroyRangeTimeout = 5 * time.Minute
69-
70-
grpcInitialWindowSize = 1 << 30
71-
grpcInitialConnWindowSize = 1 << 30
72-
)
73-
7440
// Client is a client that sends RPC.
7541
// It should not be used after calling Close().
7642
type Client interface {
@@ -223,21 +189,21 @@ func (a *connArray) Init(addr string, security config.Security) error {
223189

224190
allowBatch := config.MaxBatchSize > 0
225191
for i := range a.v {
226-
ctx, cancel := context.WithTimeout(context.Background(), dialTimeout)
192+
ctx, cancel := context.WithTimeout(context.Background(), config.DialTimeout)
227193
conn, err := grpc.DialContext(
228194
ctx,
229195
addr,
230196
opt,
231-
grpc.WithInitialWindowSize(grpcInitialWindowSize),
232-
grpc.WithInitialConnWindowSize(grpcInitialConnWindowSize),
197+
grpc.WithInitialWindowSize(int32(config.GrpcInitialWindowSize)),
198+
grpc.WithInitialConnWindowSize(int32(config.GrpcInitialConnWindowSize)),
233199
grpc.WithUnaryInterceptor(unaryInterceptor),
234200
grpc.WithStreamInterceptor(streamInterceptor),
235-
grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(MaxCallMsgSize)),
236-
grpc.WithDefaultCallOptions(grpc.MaxCallSendMsgSize(MaxSendMsgSize)),
201+
grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(config.MaxCallMsgSize)),
202+
grpc.WithDefaultCallOptions(grpc.MaxCallSendMsgSize(config.MaxSendMsgSize)),
237203
grpc.WithBackoffMaxDelay(time.Second*3),
238204
grpc.WithKeepaliveParams(keepalive.ClientParameters{
239-
Time: GrpcKeepAliveTime,
240-
Timeout: GrpcKeepAliveTimeout,
205+
Time: config.GrpcKeepAliveTime,
206+
Timeout: config.GrpcKeepAliveTimeout,
241207
PermitWithoutStream: true,
242208
}),
243209
)
@@ -489,7 +455,7 @@ func (c *rpcClient) createConnArray(addr string) (*connArray, error) {
489455
array, ok := c.conns[addr]
490456
if !ok {
491457
var err error
492-
array, err = newConnArray(MaxConnectionCount, addr, c.security)
458+
array, err = newConnArray(config.MaxConnectionCount, addr, c.security)
493459
if err != nil {
494460
return nil, err
495461
}

txnkv/kv/buffer_store.go

+2-10
Original file line numberDiff line numberDiff line change
@@ -14,18 +14,10 @@
1414
package kv
1515

1616
import (
17+
"github.com/tikv/client-go/config"
1718
"github.com/tikv/client-go/key"
1819
)
1920

20-
var (
21-
// DefaultTxnMembufCap is the default transaction membuf capability.
22-
DefaultTxnMembufCap = 4 * 1024
23-
// ImportingTxnMembufCap is the capability of tidb importing data situation.
24-
ImportingTxnMembufCap = 32 * 1024
25-
// TempTxnMemBufCap is the capability of temporary membuf.
26-
TempTxnMemBufCap = 64
27-
)
28-
2921
// BufferStore wraps a Retriever for read and a MemBuffer for buffered write.
3022
// Common usage pattern:
3123
// bs := NewBufferStore(r) // use BufferStore to wrap a Retriever
@@ -41,7 +33,7 @@ type BufferStore struct {
4133
// NewBufferStore creates a BufferStore using r for read.
4234
func NewBufferStore(r Retriever, cap int) *BufferStore {
4335
if cap <= 0 {
44-
cap = DefaultTxnMembufCap
36+
cap = config.DefaultTxnMembufCap
4537
}
4638
return &BufferStore{
4739
r: r,

txnkv/kv/buffer_store_test.go

+4-3
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ import (
1919
"testing"
2020

2121
. "github.com/pingcap/check"
22+
"github.com/tikv/client-go/config"
2223
"github.com/tikv/client-go/key"
2324
)
2425

@@ -31,7 +32,7 @@ type testBufferStoreSuite struct{}
3132
var _ = Suite(testBufferStoreSuite{})
3233

3334
func (s testBufferStoreSuite) TestGetSet(c *C) {
34-
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(DefaultTxnMembufCap)}, DefaultTxnMembufCap)
35+
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(config.DefaultTxnMembufCap)}, config.DefaultTxnMembufCap)
3536
key := key.Key("key")
3637
_, err := bs.Get(key)
3738
c.Check(err, NotNil)
@@ -45,7 +46,7 @@ func (s testBufferStoreSuite) TestGetSet(c *C) {
4546
}
4647

4748
func (s testBufferStoreSuite) TestSaveTo(c *C) {
48-
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(DefaultTxnMembufCap)}, DefaultTxnMembufCap)
49+
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(config.DefaultTxnMembufCap)}, config.DefaultTxnMembufCap)
4950
var buf bytes.Buffer
5051
for i := 0; i < 10; i++ {
5152
fmt.Fprint(&buf, i)
@@ -55,7 +56,7 @@ func (s testBufferStoreSuite) TestSaveTo(c *C) {
5556
}
5657
bs.Set(key.Key("novalue"), nil)
5758

58-
mutator := NewMemDbBuffer(DefaultTxnMembufCap)
59+
mutator := NewMemDbBuffer(config.DefaultTxnMembufCap)
5960
err := bs.SaveTo(mutator)
6061
c.Check(err, IsNil)
6162

0 commit comments

Comments
 (0)