mirror of https://github.com/tikv/client-go.git
create v2 branch
Signed-off-by: disksing <i@disksing.com>
This commit is contained in:
parent
d811a08025
commit
2f09a0b49d
|
|
@ -1,7 +0,0 @@
|
|||
# TiKV Go Client
|
||||
|
||||
TiKV Go Client provides support for interacting with the TiKV server in the form of a Go library.
|
||||
|
||||
Its main codes and structure are stripped from the [pingcap/tidb](https://github.com/pingcap/tidb) repository. The main reason for extracting this repo is to provide a cleaner option without directly accessing `github.com/pingcap/tidb/store/tikv` and introducing a lot of unnecessary dependencies.
|
||||
|
||||
There are examples of how to use them in the `example/` directory. Please note that it is **not recommended or supported** to use both the raw and transactional APIs on the same keyspace.
|
||||
|
|
@ -1,83 +0,0 @@
|
|||
// Copyright 2018 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 codec
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const (
|
||||
encGroupSize = 8
|
||||
encMarker = byte(0xFF)
|
||||
encPad = byte(0x0)
|
||||
)
|
||||
|
||||
var pads = make([]byte, encGroupSize)
|
||||
|
||||
// DecodeBytes decodes a TiDB encoded byte slice.
|
||||
func DecodeBytes(b []byte) ([]byte, []byte, error) {
|
||||
buf := make([]byte, 0, len(b)/(encGroupSize+1)*encGroupSize)
|
||||
for {
|
||||
if len(b) < encGroupSize+1 {
|
||||
return nil, nil, errors.New("insufficient bytes to decode value")
|
||||
}
|
||||
|
||||
groupBytes := b[:encGroupSize+1]
|
||||
|
||||
group := groupBytes[:encGroupSize]
|
||||
marker := groupBytes[encGroupSize]
|
||||
|
||||
padCount := encMarker - marker
|
||||
if padCount > encGroupSize {
|
||||
return nil, nil, errors.Errorf("invalid marker byte, group bytes %q", groupBytes)
|
||||
}
|
||||
|
||||
realGroupSize := encGroupSize - padCount
|
||||
buf = append(buf, group[:realGroupSize]...)
|
||||
b = b[encGroupSize+1:]
|
||||
|
||||
if padCount != 0 {
|
||||
// Check validity of padding bytes.
|
||||
if !bytes.Equal(group[realGroupSize:], pads[:padCount]) {
|
||||
return nil, nil, errors.Errorf("invalid padding byte, group bytes %q", groupBytes)
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
return b, buf, nil
|
||||
}
|
||||
|
||||
// EncodeBytes encodes a byte slice into TiDB's encoded form.
|
||||
func EncodeBytes(b []byte) []byte {
|
||||
dLen := len(b)
|
||||
reallocSize := (dLen/encGroupSize + 1) * (encGroupSize + 1)
|
||||
result := make([]byte, 0, reallocSize)
|
||||
for idx := 0; idx <= dLen; idx += encGroupSize {
|
||||
remain := dLen - idx
|
||||
padCount := 0
|
||||
if remain >= encGroupSize {
|
||||
result = append(result, b[idx:idx+encGroupSize]...)
|
||||
} else {
|
||||
padCount = encGroupSize - remain
|
||||
result = append(result, b[idx:]...)
|
||||
result = append(result, pads[:padCount]...)
|
||||
}
|
||||
|
||||
marker := encMarker - byte(padCount)
|
||||
result = append(result, marker)
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
|
@ -1,37 +0,0 @@
|
|||
// Copyright 2018 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 codec
|
||||
|
||||
import (
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
)
|
||||
|
||||
// DecodeRegionMetaKey translates a region meta from encoded form to unencoded form.
|
||||
func DecodeRegionMetaKey(r *metapb.Region) error {
|
||||
if len(r.StartKey) != 0 {
|
||||
_, decoded, err := DecodeBytes(r.StartKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
r.StartKey = decoded
|
||||
}
|
||||
if len(r.EndKey) != 0 {
|
||||
_, decoded, err := DecodeBytes(r.EndKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
r.EndKey = decoded
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
@ -1,57 +0,0 @@
|
|||
// 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 codec
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const signMask uint64 = 0x8000000000000000
|
||||
|
||||
// EncodeIntToCmpUint make int v to comparable uint type
|
||||
func EncodeIntToCmpUint(v int64) uint64 {
|
||||
return uint64(v) ^ signMask
|
||||
}
|
||||
|
||||
// EncodeInt appends the encoded value to slice b and returns the appended slice.
|
||||
// EncodeInt guarantees that the encoded value is in ascending order for comparison.
|
||||
func EncodeInt(b []byte, v int64) []byte {
|
||||
var data [8]byte
|
||||
u := EncodeIntToCmpUint(v)
|
||||
binary.BigEndian.PutUint64(data[:], u)
|
||||
return append(b, data[:]...)
|
||||
}
|
||||
|
||||
// EncodeUintDesc appends the encoded value to slice b and returns the appended slice.
|
||||
// EncodeUintDesc guarantees that the encoded value is in descending order for comparison.
|
||||
func EncodeUintDesc(b []byte, v uint64) []byte {
|
||||
var data [8]byte
|
||||
binary.BigEndian.PutUint64(data[:], ^v)
|
||||
return append(b, data[:]...)
|
||||
}
|
||||
|
||||
// DecodeUintDesc decodes value encoded by EncodeInt before.
|
||||
// It returns the leftover un-decoded slice, decoded value if no error.
|
||||
func DecodeUintDesc(b []byte) ([]byte, uint64, error) {
|
||||
if len(b) < 8 {
|
||||
return nil, 0, errors.New("insufficient bytes to decode value")
|
||||
}
|
||||
|
||||
data := b[:8]
|
||||
v := binary.BigEndian.Uint64(data)
|
||||
b = b[8:]
|
||||
return b, ^v, nil
|
||||
}
|
||||
|
|
@ -1,66 +0,0 @@
|
|||
// 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 codec
|
||||
|
||||
import "github.com/tikv/client-go/key"
|
||||
|
||||
var (
|
||||
tablePrefix = []byte{'t'}
|
||||
recordPrefixSep = []byte("_r")
|
||||
indexPrefixSep = []byte("_i")
|
||||
)
|
||||
|
||||
const (
|
||||
idLen = 8
|
||||
prefixLen = 1 + idLen /*tableID*/ + 2
|
||||
recordRowKeyLen = prefixLen + idLen /*handle*/
|
||||
tablePrefixLength = 1
|
||||
recordPrefixSepLength = 2
|
||||
)
|
||||
|
||||
// appendTableRecordPrefix appends table record prefix "t[tableID]_r".
|
||||
func appendTableRecordPrefix(buf []byte, tableID int64) []byte {
|
||||
buf = append(buf, tablePrefix...)
|
||||
buf = EncodeInt(buf, tableID)
|
||||
buf = append(buf, recordPrefixSep...)
|
||||
return buf
|
||||
}
|
||||
|
||||
// GenTableRecordPrefix composes record prefix with tableID: "t[tableID]_r".
|
||||
func GenTableRecordPrefix(tableID int64) key.Key {
|
||||
buf := make([]byte, 0, len(tablePrefix)+8+len(recordPrefixSep))
|
||||
return appendTableRecordPrefix(buf, tableID)
|
||||
}
|
||||
|
||||
// appendTableIndexPrefix appends table index prefix "t[tableID]_i".
|
||||
func appendTableIndexPrefix(buf []byte, tableID int64) []byte {
|
||||
buf = append(buf, tablePrefix...)
|
||||
buf = EncodeInt(buf, tableID)
|
||||
buf = append(buf, indexPrefixSep...)
|
||||
return buf
|
||||
}
|
||||
|
||||
// GenTableIndexPrefix composes index prefix with tableID: "t[tableID]_i".
|
||||
func GenTableIndexPrefix(tableID int64) key.Key {
|
||||
buf := make([]byte, 0, len(tablePrefix)+8+len(indexPrefixSep))
|
||||
return appendTableIndexPrefix(buf, tableID)
|
||||
}
|
||||
|
||||
// EncodeTableIndexPrefix encodes index prefix with tableID and idxID.
|
||||
func EncodeTableIndexPrefix(tableID, idxID int64) key.Key {
|
||||
key := make([]byte, 0, prefixLen)
|
||||
key = appendTableIndexPrefix(key, tableID)
|
||||
key = EncodeInt(key, idxID)
|
||||
return key
|
||||
}
|
||||
|
|
@ -1,32 +0,0 @@
|
|||
// Copyright 2018 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
|
||||
|
||||
// Config contains configurations for tikv client.
|
||||
type Config struct {
|
||||
RPC RPC
|
||||
Raw Raw
|
||||
Txn Txn
|
||||
RegionCache RegionCache
|
||||
}
|
||||
|
||||
// Default returns the default config.
|
||||
func Default() Config {
|
||||
return Config{
|
||||
RPC: DefaultRPC(),
|
||||
Raw: DefaultRaw(),
|
||||
Txn: DefaultTxn(),
|
||||
RegionCache: DefaultRegionCache(),
|
||||
}
|
||||
}
|
||||
|
|
@ -1,35 +0,0 @@
|
|||
// 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,
|
||||
}
|
||||
}
|
||||
|
|
@ -1,30 +0,0 @@
|
|||
// 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
168
config/rpc.go
|
|
@ -1,168 +0,0 @@
|
|||
// 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
125
config/txn.go
|
|
@ -1,125 +0,0 @@
|
|||
// 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,
|
||||
}
|
||||
}
|
||||
|
|
@ -1,198 +0,0 @@
|
|||
// 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 main
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"flag"
|
||||
"fmt"
|
||||
"log"
|
||||
"math/rand"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"github.com/tikv/client-go/rawkv"
|
||||
"github.com/tikv/client-go/txnkv"
|
||||
)
|
||||
|
||||
var (
|
||||
pdAddr = flag.String("pd", "127.0.0.1:2379", "pd address")
|
||||
mode = flag.String("mode", "raw", "raw / txn")
|
||||
|
||||
pushAddr = flag.String("push", "127.0.0.1:9090", "pushGateway address")
|
||||
pushInterval = flag.Duration("interval", 15*time.Second, "push metrics interval")
|
||||
pushJob = flag.String("job", "bench", "job name")
|
||||
pushInstance = flag.String("instance", "bench1", "instance name")
|
||||
|
||||
keyLen = flag.Int("klen", 10, "length of key")
|
||||
valueLen = flag.Int("vlen", 20, "length of value")
|
||||
keyRange = flag.Int("range", 100000, "size of the key set")
|
||||
|
||||
rawGetP = flag.Int("raw-get-p", 1, "raw get concurrency")
|
||||
rawBatchGetP = flag.Int("raw-batch-get-p", 0, "raw batch get concurrency")
|
||||
rawBatchGetN = flag.Int("raw-batch-get-n", 10, "raw batch get batch size")
|
||||
rawPutP = flag.Int("raw-put-p", 1, "raw put concurrency")
|
||||
rawBatchPutP = flag.Int("raw-batch-put-p", 0, "raw batch put concurrency")
|
||||
rawBatchPutN = flag.Int("raw-batch-put-n", 10, "raw batch put batch size")
|
||||
rawDeleteP = flag.Int("raw-delete-p", 1, "raw delete concurrency")
|
||||
rawBatchDeleteP = flag.Int("raw-batch-delete-p", 0, "raw batch delete concurrency")
|
||||
rawBatchDeleteN = flag.Int("raw-batch-delete-n", 10, "raw batch delete batch size")
|
||||
rawScanP = flag.Int("raw-scan-p", 1, "raw scan concurrency")
|
||||
rawScanL = flag.Int("raw-scan-l", 10, "raw scan limit")
|
||||
|
||||
txn1P = flag.Int("txn1-p", 1, "txn1 concurrency")
|
||||
txn1GetN = flag.Int("txn1-get-n", 10, "txn1 get command count")
|
||||
txn1PutN = flag.Int("txn1-put-n", 0, "txn1 put command count")
|
||||
txn1DeleteN = flag.Int("txn1-delete-n", 0, "txn1 delete command count")
|
||||
txn1ScanN = flag.Int("txn1-scan-n", 1, "txn1 scan command count")
|
||||
txn1ScanL = flag.Int("txn1-scan-l", 10, "txn1 scan limit")
|
||||
|
||||
txn2P = flag.Int("txn2-p", 2, "txn2 concurrency")
|
||||
txn2GetN = flag.Int("txn2-get-n", 0, "txn2 get command count")
|
||||
txn2PutN = flag.Int("txn2-put-n", 10, "txn2 put command count")
|
||||
txn2DeleteN = flag.Int("txn2-delete-n", 1, "txn2 delete command count")
|
||||
txn2ScanN = flag.Int("txn2-scan-n", 0, "txn2 scan command count")
|
||||
txn2ScanL = flag.Int("txn2-scan-l", 10, "txn2 scan limit")
|
||||
|
||||
txn3P = flag.Int("txn3-p", 0, "txn3 concurrency")
|
||||
txn3GetN = flag.Int("txn3-get-n", 1, "txn3 get command count")
|
||||
txn3PutN = flag.Int("txn3-put-n", 1, "txn3 put command count")
|
||||
txn3DeleteN = flag.Int("txn3-delete-n", 1, "txn3 delete command count")
|
||||
txn3ScanN = flag.Int("txn3-scan-n", 1, "txn3 scan command count")
|
||||
txn3ScanL = flag.Int("txn3-scan-l", 10, "txn3 scan limit")
|
||||
)
|
||||
|
||||
func newConfig() config.Config {
|
||||
return config.Default()
|
||||
}
|
||||
|
||||
var (
|
||||
rawCli *rawkv.Client
|
||||
txnCli *txnkv.Client
|
||||
)
|
||||
|
||||
func k() []byte {
|
||||
var t string
|
||||
if *mode == "raw" {
|
||||
t = fmt.Sprintf("R%%%dd", *keyLen-1)
|
||||
} else {
|
||||
t = fmt.Sprintf("T%%%dd", *keyLen-1)
|
||||
}
|
||||
return []byte(fmt.Sprintf(t, rand.Intn(*keyRange)))
|
||||
}
|
||||
|
||||
func v() []byte {
|
||||
return bytes.Repeat([]byte{0}, *valueLen)
|
||||
}
|
||||
|
||||
func n(x int, f func() []byte) [][]byte {
|
||||
res := make([][]byte, x)
|
||||
for i := range res {
|
||||
res[i] = f()
|
||||
}
|
||||
return res
|
||||
}
|
||||
|
||||
func nk(x int) [][]byte { return n(x, k) }
|
||||
func nv(x int) [][]byte { return n(x, v) }
|
||||
|
||||
func P(p int, f func()) {
|
||||
for i := 0; i < p; i++ {
|
||||
go func() {
|
||||
for {
|
||||
f()
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
func benchRaw() {
|
||||
var err error
|
||||
rawCli, err = rawkv.NewClient(context.TODO(), strings.Split(*pdAddr, ","), newConfig())
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
P(*rawGetP, func() { rawCli.Get(context.TODO(), k()) })
|
||||
P(*rawBatchGetP, func() { rawCli.BatchGet(context.TODO(), nk(*rawBatchGetN)) })
|
||||
P(*rawPutP, func() { rawCli.Put(context.TODO(), k(), v()) })
|
||||
P(*rawBatchPutP, func() { rawCli.BatchPut(context.TODO(), nk(*rawBatchPutN), nv(*rawBatchPutN)) })
|
||||
P(*rawDeleteP, func() { rawCli.Delete(context.TODO(), k()) })
|
||||
P(*rawBatchDeleteP, func() { rawCli.BatchDelete(context.TODO(), nk(*rawBatchDeleteN)) })
|
||||
P(*rawScanP, func() { rawCli.Scan(context.TODO(), k(), nil, *rawScanL) })
|
||||
}
|
||||
|
||||
func benchTxn() {
|
||||
var err error
|
||||
txnCli, err = txnkv.NewClient(context.TODO(), strings.Split(*pdAddr, ","), newConfig())
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
t := func(getN, putN, delN, scanN, scanL int) func() {
|
||||
return func() {
|
||||
tx, err := txnCli.Begin(context.TODO())
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
for i := 0; i < getN; i++ {
|
||||
tx.Get(context.TODO(), k())
|
||||
}
|
||||
for i := 0; i < putN; i++ {
|
||||
tx.Set(k(), v())
|
||||
}
|
||||
for i := 0; i < delN; i++ {
|
||||
tx.Delete(k())
|
||||
}
|
||||
for i := 0; i < scanN; i++ {
|
||||
it, err := tx.Iter(context.TODO(), k(), nil)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
for j := 0; j < scanL && it.Valid(); j++ {
|
||||
it.Next(context.TODO())
|
||||
}
|
||||
it.Close()
|
||||
}
|
||||
tx.Commit(context.TODO())
|
||||
}
|
||||
}
|
||||
|
||||
P(*txn1P, t(*txn1GetN, *txn1PutN, *txn1DeleteN, *txn1ScanN, *txn1ScanL))
|
||||
P(*txn2P, t(*txn2GetN, *txn2PutN, *txn2DeleteN, *txn2ScanN, *txn2ScanL))
|
||||
P(*txn3P, t(*txn3GetN, *txn3PutN, *txn3DeleteN, *txn3ScanN, *txn3ScanL))
|
||||
}
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
|
||||
go metrics.PushMetrics(context.TODO(), *pushAddr, *pushInterval, *pushJob, *pushInstance)
|
||||
|
||||
switch *mode {
|
||||
case "raw":
|
||||
benchRaw()
|
||||
case "txn":
|
||||
benchTxn()
|
||||
default:
|
||||
log.Fatal("invalid mode:", *mode)
|
||||
}
|
||||
|
||||
for {
|
||||
fmt.Print(".")
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
}
|
||||
|
|
@ -1,26 +0,0 @@
|
|||
// 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 main
|
||||
|
||||
import (
|
||||
"net/http"
|
||||
|
||||
"github.com/tikv/client-go/proxy/httpproxy"
|
||||
)
|
||||
|
||||
func main() {
|
||||
h := httpproxy.NewHTTPProxyHandlerWithConfig()
|
||||
h.Config.RPC.EnableOpenTracing = true
|
||||
http.ListenAndServe(":8080", httpproxy.NewHTTPProxyHandlerWithConfig())
|
||||
}
|
||||
|
|
@ -1,24 +0,0 @@
|
|||
// 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 main
|
||||
|
||||
import (
|
||||
"net/http"
|
||||
|
||||
"github.com/tikv/client-go/proxy/httpproxy"
|
||||
)
|
||||
|
||||
func main() {
|
||||
http.ListenAndServe(":8080", httpproxy.NewHTTPProxyHandler())
|
||||
}
|
||||
|
|
@ -1,63 +0,0 @@
|
|||
// 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 main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/rawkv"
|
||||
)
|
||||
|
||||
func main() {
|
||||
cli, err := rawkv.NewClient(context.TODO(), []string{"127.0.0.1:2379"}, config.Default())
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
defer cli.Close()
|
||||
|
||||
fmt.Printf("cluster ID: %d\n", cli.ClusterID())
|
||||
|
||||
key := []byte("Company")
|
||||
val := []byte("PingCAP")
|
||||
|
||||
// put key into tikv
|
||||
err = cli.Put(context.TODO(), key, val)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Printf("Successfully put %s:%s to tikv\n", key, val)
|
||||
|
||||
// get key from tikv
|
||||
val, err = cli.Get(context.TODO(), key)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Printf("found val: %s for key: %s\n", val, key)
|
||||
|
||||
// delete key from tikv
|
||||
err = cli.Delete(context.TODO(), key)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Printf("key: %s deleted\n", key)
|
||||
|
||||
// get key again from tikv
|
||||
val, err = cli.Get(context.TODO(), key)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Printf("found val: %s for key: %s\n", val, key)
|
||||
}
|
||||
|
|
@ -1,144 +0,0 @@
|
|||
// 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 main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"flag"
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/key"
|
||||
"github.com/tikv/client-go/txnkv"
|
||||
)
|
||||
|
||||
// KV represents a Key-Value pair.
|
||||
type KV struct {
|
||||
K, V []byte
|
||||
}
|
||||
|
||||
func (kv KV) String() string {
|
||||
return fmt.Sprintf("%s => %s (%v)", kv.K, kv.V, kv.V)
|
||||
}
|
||||
|
||||
var (
|
||||
client *txnkv.Client
|
||||
pdAddr = flag.String("pd", "127.0.0.1:2379", "pd address")
|
||||
)
|
||||
|
||||
// Init initializes information.
|
||||
func initStore() {
|
||||
var err error
|
||||
client, err = txnkv.NewClient(context.TODO(), []string{*pdAddr}, config.Default())
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
// key1 val1 key2 val2 ...
|
||||
func puts(args ...[]byte) error {
|
||||
tx, err := client.Begin(context.TODO())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for i := 0; i < len(args); i += 2 {
|
||||
key, val := args[i], args[i+1]
|
||||
err := tx.Set(key, val)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return tx.Commit(context.Background())
|
||||
}
|
||||
|
||||
func get(k []byte) (KV, error) {
|
||||
tx, err := client.Begin(context.TODO())
|
||||
if err != nil {
|
||||
return KV{}, err
|
||||
}
|
||||
v, err := tx.Get(context.TODO(), k)
|
||||
if err != nil {
|
||||
return KV{}, err
|
||||
}
|
||||
return KV{K: k, V: v}, nil
|
||||
}
|
||||
|
||||
func dels(keys ...[]byte) error {
|
||||
tx, err := client.Begin(context.TODO())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, key := range keys {
|
||||
err := tx.Delete(key)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return tx.Commit(context.Background())
|
||||
}
|
||||
|
||||
func scan(keyPrefix []byte, limit int) ([]KV, error) {
|
||||
tx, err := client.Begin(context.TODO())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
it, err := tx.Iter(context.TODO(), key.Key(keyPrefix), nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer it.Close()
|
||||
var ret []KV
|
||||
for it.Valid() && limit > 0 {
|
||||
ret = append(ret, KV{K: it.Key()[:], V: it.Value()[:]})
|
||||
limit--
|
||||
it.Next(context.TODO())
|
||||
}
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
func main() {
|
||||
pdAddr := os.Getenv("PD_ADDR")
|
||||
if pdAddr != "" {
|
||||
os.Args = append(os.Args, "-pd", pdAddr)
|
||||
}
|
||||
flag.Parse()
|
||||
initStore()
|
||||
|
||||
// set
|
||||
err := puts([]byte("key1"), []byte("value1"), []byte("key2"), []byte("value2"))
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// get
|
||||
kv, err := get([]byte("key1"))
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Println(kv)
|
||||
|
||||
// scan
|
||||
ret, err := scan([]byte("key"), 10)
|
||||
for _, kv := range ret {
|
||||
fmt.Println(kv)
|
||||
}
|
||||
|
||||
// delete
|
||||
err = dels([]byte("key1"), []byte("key2"))
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
32
go.mod
32
go.mod
|
|
@ -1,32 +0,0 @@
|
|||
module github.com/tikv/client-go
|
||||
|
||||
go 1.15
|
||||
|
||||
require (
|
||||
github.com/coreos/etcd v3.3.25+incompatible
|
||||
github.com/golang/protobuf v1.3.4
|
||||
github.com/google/btree v1.0.0
|
||||
github.com/google/uuid v1.1.1
|
||||
github.com/gorilla/mux v1.7.4
|
||||
github.com/grpc-ecosystem/go-grpc-middleware v1.1.0
|
||||
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
|
||||
github.com/json-iterator/go v1.1.10 // indirect
|
||||
github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712
|
||||
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989
|
||||
github.com/pingcap/kvproto v0.0.0-20210219095907-b2375dcc80ad
|
||||
github.com/pkg/errors v0.9.1
|
||||
github.com/prometheus/client_golang v1.2.1
|
||||
github.com/prometheus/common v0.9.1
|
||||
github.com/sirupsen/logrus v1.4.2
|
||||
github.com/spaolacci/murmur3 v1.1.0
|
||||
github.com/tikv/pd v1.1.0-beta.0.20210122094357-c7aac753461a
|
||||
go.etcd.io/etcd v3.3.25+incompatible // indirect
|
||||
go.uber.org/multierr v1.6.0 // indirect
|
||||
go.uber.org/zap v1.16.0 // indirect
|
||||
golang.org/x/net v0.0.0-20201110031124-69a78807bb2b // indirect
|
||||
golang.org/x/sys v0.0.0-20201113233024-12cec1faf1ba // indirect
|
||||
golang.org/x/text v0.3.4 // indirect
|
||||
golang.org/x/tools v0.0.0-20201116002733-ac45abd4c88c // indirect
|
||||
google.golang.org/grpc v1.26.0
|
||||
gopkg.in/yaml.v2 v2.3.0 // indirect
|
||||
)
|
||||
71
key/key.go
71
key/key.go
|
|
@ -1,71 +0,0 @@
|
|||
// 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 key
|
||||
|
||||
import "bytes"
|
||||
|
||||
// Key represents high-level Key type.
|
||||
type Key []byte
|
||||
|
||||
// Next returns the next key in byte-order.
|
||||
func (k Key) Next() Key {
|
||||
// add 0x0 to the end of key
|
||||
buf := make([]byte, len([]byte(k))+1)
|
||||
copy(buf, []byte(k))
|
||||
return buf
|
||||
}
|
||||
|
||||
// PrefixNext returns the next prefix key.
|
||||
//
|
||||
// Assume there are keys like:
|
||||
//
|
||||
// rowkey1
|
||||
// rowkey1_column1
|
||||
// rowkey1_column2
|
||||
// rowKey2
|
||||
//
|
||||
// If we seek 'rowkey1' Next, we will get 'rowkey1_column1'.
|
||||
// If we seek 'rowkey1' PrefixNext, we will get 'rowkey2'.
|
||||
func (k Key) PrefixNext() Key {
|
||||
buf := make([]byte, len([]byte(k)))
|
||||
copy(buf, []byte(k))
|
||||
var i int
|
||||
for i = len(k) - 1; i >= 0; i-- {
|
||||
buf[i]++
|
||||
if buf[i] != 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if i == -1 {
|
||||
copy(buf, k)
|
||||
buf = append(buf, 0)
|
||||
}
|
||||
return buf
|
||||
}
|
||||
|
||||
// Cmp returns the comparison result of two keys.
|
||||
// The result will be 0 if a==b, -1 if a < b, and +1 if a > b.
|
||||
func (k Key) Cmp(another Key) int {
|
||||
return bytes.Compare(k, another)
|
||||
}
|
||||
|
||||
// Clone returns a copy of the Key.
|
||||
func (k Key) Clone() Key {
|
||||
return append([]byte(nil), k...)
|
||||
}
|
||||
|
||||
// HasPrefix tests whether the Key begins with prefix.
|
||||
func (k Key) HasPrefix(prefix Key) bool {
|
||||
return bytes.HasPrefix(k, prefix)
|
||||
}
|
||||
|
|
@ -1,62 +0,0 @@
|
|||
// Copyright 2018 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 locate
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/tikv/client-go/codec"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
||||
// CodecPDClient wraps a PD Client to decode the encoded keys in region meta.
|
||||
type CodecPDClient struct {
|
||||
pd.Client
|
||||
}
|
||||
|
||||
// GetRegion encodes the key before send requests to pd-server and decodes the
|
||||
// returned StartKey && EndKey from pd-server.
|
||||
func (c *CodecPDClient) GetRegion(ctx context.Context, key []byte) (*pd.Region, error) {
|
||||
encodedKey := codec.EncodeBytes(key)
|
||||
region, err := c.Client.GetRegion(ctx, encodedKey)
|
||||
return processRegionResult(region, err)
|
||||
}
|
||||
|
||||
// GetPrevRegion encodes the key before send requests to pd-server and decodes the
|
||||
// returned StartKey && EndKey from pd-server.
|
||||
func (c *CodecPDClient) GetPrevRegion(ctx context.Context, key []byte) (*pd.Region, error) {
|
||||
encodedKey := codec.EncodeBytes(key)
|
||||
region, err := c.Client.GetPrevRegion(ctx, encodedKey)
|
||||
return processRegionResult(region, err)
|
||||
}
|
||||
|
||||
// GetRegionByID decodes the returned StartKey && EndKey from pd-server.
|
||||
func (c *CodecPDClient) GetRegionByID(ctx context.Context, regionID uint64) (*pd.Region, error) {
|
||||
region, err := c.Client.GetRegionByID(ctx, regionID)
|
||||
return processRegionResult(region, err)
|
||||
}
|
||||
|
||||
func processRegionResult(region *pd.Region, err error) (*pd.Region, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if region == nil {
|
||||
return nil, nil
|
||||
}
|
||||
err = codec.DecodeRegionMetaKey(region.Meta)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return region, nil
|
||||
}
|
||||
|
|
@ -1,611 +0,0 @@
|
|||
// Copyright 2018 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 locate
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/google/btree"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/codec"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"github.com/tikv/client-go/retry"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
||||
// CachedRegion encapsulates {Region, TTL}
|
||||
type CachedRegion struct {
|
||||
region *Region
|
||||
lastAccess int64
|
||||
}
|
||||
|
||||
func (c *CachedRegion) isValid(ttl time.Duration) bool {
|
||||
lastAccess := atomic.LoadInt64(&c.lastAccess)
|
||||
lastAccessTime := time.Unix(lastAccess, 0)
|
||||
return time.Since(lastAccessTime) < ttl
|
||||
}
|
||||
|
||||
// RegionCache caches Regions loaded from PD.
|
||||
type RegionCache struct {
|
||||
conf *config.RegionCache
|
||||
pdClient pd.Client
|
||||
|
||||
mu struct {
|
||||
sync.RWMutex
|
||||
regions map[RegionVerID]*CachedRegion
|
||||
sorted *btree.BTree
|
||||
}
|
||||
storeMu struct {
|
||||
sync.RWMutex
|
||||
stores map[uint64]*Store
|
||||
}
|
||||
}
|
||||
|
||||
// NewRegionCache creates a 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(conf.BTreeDegree)
|
||||
c.storeMu.stores = make(map[uint64]*Store)
|
||||
return c
|
||||
}
|
||||
|
||||
// RPCContext contains data that is needed to send RPC to a region.
|
||||
type RPCContext struct {
|
||||
Region RegionVerID
|
||||
Meta *metapb.Region
|
||||
Peer *metapb.Peer
|
||||
Addr string
|
||||
}
|
||||
|
||||
// GetStoreID returns StoreID.
|
||||
func (c *RPCContext) GetStoreID() uint64 {
|
||||
if c.Peer != nil {
|
||||
return c.Peer.StoreId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
// GetRPCContext returns RPCContext for a region. If it returns nil, the region
|
||||
// must be out of date and already dropped from cache.
|
||||
func (c *RegionCache) GetRPCContext(bo *retry.Backoffer, id RegionVerID) (*RPCContext, error) {
|
||||
c.mu.RLock()
|
||||
region := c.getCachedRegion(id)
|
||||
if region == nil {
|
||||
c.mu.RUnlock()
|
||||
return nil, nil
|
||||
}
|
||||
// Note: it is safe to use region.meta and region.peer without clone after
|
||||
// unlock, because region cache will never update the content of region's meta
|
||||
// or peer. On the contrary, if we want to use `region` after unlock, then we
|
||||
// need to clone it to avoid data race.
|
||||
meta, peer := region.meta, region.peer
|
||||
c.mu.RUnlock()
|
||||
|
||||
addr, err := c.GetStoreAddr(bo, peer.GetStoreId())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if addr == "" {
|
||||
// Store not found, region must be out of date.
|
||||
c.DropRegion(id)
|
||||
return nil, nil
|
||||
}
|
||||
return &RPCContext{
|
||||
Region: id,
|
||||
Meta: meta,
|
||||
Peer: peer,
|
||||
Addr: addr,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// KeyLocation is the region and range that a key is located.
|
||||
type KeyLocation struct {
|
||||
Region RegionVerID
|
||||
StartKey []byte
|
||||
EndKey []byte
|
||||
}
|
||||
|
||||
// Contains checks if key is in [StartKey, EndKey).
|
||||
func (l *KeyLocation) Contains(key []byte) bool {
|
||||
return bytes.Compare(l.StartKey, key) <= 0 &&
|
||||
(bytes.Compare(key, l.EndKey) < 0 || len(l.EndKey) == 0)
|
||||
}
|
||||
|
||||
// LocateKey searches for the region and range that the key is located.
|
||||
func (c *RegionCache) LocateKey(bo *retry.Backoffer, key []byte) (*KeyLocation, error) {
|
||||
c.mu.RLock()
|
||||
r := c.searchCachedRegion(key)
|
||||
if r != nil {
|
||||
loc := &KeyLocation{
|
||||
Region: r.VerID(),
|
||||
StartKey: r.StartKey(),
|
||||
EndKey: r.EndKey(),
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
return loc, nil
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
|
||||
r, err := c.loadRegion(bo, key)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
c.insertRegionToCache(r)
|
||||
|
||||
return &KeyLocation{
|
||||
Region: r.VerID(),
|
||||
StartKey: r.StartKey(),
|
||||
EndKey: r.EndKey(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// LocateRegionByID searches for the region with ID.
|
||||
func (c *RegionCache) LocateRegionByID(bo *retry.Backoffer, regionID uint64) (*KeyLocation, error) {
|
||||
c.mu.RLock()
|
||||
r := c.getRegionByIDFromCache(regionID)
|
||||
if r != nil {
|
||||
loc := &KeyLocation{
|
||||
Region: r.VerID(),
|
||||
StartKey: r.StartKey(),
|
||||
EndKey: r.EndKey(),
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
return loc, nil
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
|
||||
r, err := c.loadRegionByID(bo, regionID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
c.insertRegionToCache(r)
|
||||
return &KeyLocation{
|
||||
Region: r.VerID(),
|
||||
StartKey: r.StartKey(),
|
||||
EndKey: r.EndKey(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// GroupKeysByRegion separates keys into groups by their belonging Regions.
|
||||
// Specially it also returns the first key's region which may be used as the
|
||||
// 'PrimaryLockKey' and should be committed ahead of others.
|
||||
func (c *RegionCache) GroupKeysByRegion(bo *retry.Backoffer, keys [][]byte) (map[RegionVerID][][]byte, RegionVerID, error) {
|
||||
groups := make(map[RegionVerID][][]byte)
|
||||
var first RegionVerID
|
||||
var lastLoc *KeyLocation
|
||||
for i, k := range keys {
|
||||
if lastLoc == nil || !lastLoc.Contains(k) {
|
||||
var err error
|
||||
lastLoc, err = c.LocateKey(bo, k)
|
||||
if err != nil {
|
||||
return nil, first, err
|
||||
}
|
||||
}
|
||||
id := lastLoc.Region
|
||||
if i == 0 {
|
||||
first = id
|
||||
}
|
||||
groups[id] = append(groups[id], k)
|
||||
}
|
||||
return groups, first, nil
|
||||
}
|
||||
|
||||
// ListRegionIDsInKeyRange lists ids of regions in [start_key,end_key].
|
||||
func (c *RegionCache) ListRegionIDsInKeyRange(bo *retry.Backoffer, startKey, endKey []byte) (regionIDs []uint64, err error) {
|
||||
for {
|
||||
curRegion, err := c.LocateKey(bo, startKey)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
regionIDs = append(regionIDs, curRegion.Region.id)
|
||||
if curRegion.Contains(endKey) {
|
||||
break
|
||||
}
|
||||
startKey = curRegion.EndKey
|
||||
}
|
||||
return regionIDs, nil
|
||||
}
|
||||
|
||||
// DropRegion removes a cached Region.
|
||||
func (c *RegionCache) DropRegion(id RegionVerID) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
c.dropRegionFromCache(id)
|
||||
}
|
||||
|
||||
// UpdateLeader update some region cache with newer leader info.
|
||||
func (c *RegionCache) UpdateLeader(regionID RegionVerID, leaderStoreID uint64) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
r := c.getCachedRegion(regionID)
|
||||
if r == nil {
|
||||
log.Debugf("regionCache: cannot find region when updating leader %d,%d", regionID, leaderStoreID)
|
||||
return
|
||||
}
|
||||
|
||||
if !r.SwitchPeer(leaderStoreID) {
|
||||
log.Debugf("regionCache: cannot find peer when updating leader %d,%d", regionID, leaderStoreID)
|
||||
c.dropRegionFromCache(r.VerID())
|
||||
}
|
||||
}
|
||||
|
||||
// insertRegionToCache tries to insert the Region to cache.
|
||||
func (c *RegionCache) insertRegionToCache(r *Region) {
|
||||
old := c.mu.sorted.ReplaceOrInsert(newBtreeItem(r))
|
||||
if old != nil {
|
||||
delete(c.mu.regions, old.(*btreeItem).region.VerID())
|
||||
}
|
||||
c.mu.regions[r.VerID()] = &CachedRegion{
|
||||
region: r,
|
||||
lastAccess: time.Now().Unix(),
|
||||
}
|
||||
}
|
||||
|
||||
// getCachedRegion loads a region from cache. It also checks if the region has
|
||||
// not been accessed for a long time (maybe out of date). In this case, it
|
||||
// returns nil so the region will be loaded from PD again.
|
||||
// Note that it should be called with c.mu.RLock(), and the returned Region
|
||||
// should not be used after c.mu is RUnlock().
|
||||
func (c *RegionCache) getCachedRegion(id RegionVerID) *Region {
|
||||
cachedRegion, ok := c.mu.regions[id]
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
if cachedRegion.isValid(c.conf.CacheTTL) {
|
||||
atomic.StoreInt64(&cachedRegion.lastAccess, time.Now().Unix())
|
||||
return cachedRegion.region
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// searchCachedRegion finds a region from cache by key. Like `getCachedRegion`,
|
||||
// it should be called with c.mu.RLock(), and the returned Region should not be
|
||||
// used after c.mu is RUnlock().
|
||||
func (c *RegionCache) searchCachedRegion(key []byte) *Region {
|
||||
var r *Region
|
||||
c.mu.sorted.DescendLessOrEqual(newBtreeSearchItem(key), func(item btree.Item) bool {
|
||||
r = item.(*btreeItem).region
|
||||
return false
|
||||
})
|
||||
if r != nil && r.Contains(key) {
|
||||
return c.getCachedRegion(r.VerID())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// getRegionByIDFromCache tries to get region by regionID from cache. Like
|
||||
// `getCachedRegion`, it should be called with c.mu.RLock(), and the returned
|
||||
// Region should not be used after c.mu is RUnlock().
|
||||
func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region {
|
||||
for v, r := range c.mu.regions {
|
||||
if v.id == regionID {
|
||||
return r.region
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *RegionCache) dropRegionFromCache(verID RegionVerID) {
|
||||
r, ok := c.mu.regions[verID]
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
metrics.RegionCacheCounter.WithLabelValues("drop_region_from_cache", "ok").Inc()
|
||||
c.mu.sorted.Delete(newBtreeItem(r.region))
|
||||
delete(c.mu.regions, verID)
|
||||
}
|
||||
|
||||
// loadRegion loads region from pd client, and picks the first peer as leader.
|
||||
func (c *RegionCache) loadRegion(bo *retry.Backoffer, key []byte) (*Region, error) {
|
||||
var backoffErr error
|
||||
for {
|
||||
if backoffErr != nil {
|
||||
err := bo.Backoff(retry.BoPDRPC, backoffErr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
region, err := c.pdClient.GetRegion(bo.GetContext(), key)
|
||||
metrics.RegionCacheCounter.WithLabelValues("get_region", metrics.RetLabel(err)).Inc()
|
||||
if err != nil {
|
||||
backoffErr = errors.Errorf("loadRegion from PD failed, key: %q, err: %v", key, err)
|
||||
continue
|
||||
}
|
||||
if region == nil || region.Meta == nil {
|
||||
backoffErr = errors.Errorf("region not found for key %q", key)
|
||||
continue
|
||||
}
|
||||
if len(region.Meta.Peers) == 0 {
|
||||
return nil, errors.New("receive Region with no peer")
|
||||
}
|
||||
r := &Region{
|
||||
meta: region.Meta,
|
||||
peer: region.Meta.Peers[0],
|
||||
}
|
||||
if region.Leader != nil {
|
||||
r.SwitchPeer(region.Leader.GetStoreId())
|
||||
}
|
||||
return r, nil
|
||||
}
|
||||
}
|
||||
|
||||
// loadRegionByID loads region from pd client, and picks the first peer as leader.
|
||||
func (c *RegionCache) loadRegionByID(bo *retry.Backoffer, regionID uint64) (*Region, error) {
|
||||
var backoffErr error
|
||||
for {
|
||||
if backoffErr != nil {
|
||||
err := bo.Backoff(retry.BoPDRPC, backoffErr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
region, err := c.pdClient.GetRegionByID(bo.GetContext(), regionID)
|
||||
metrics.RegionCacheCounter.WithLabelValues("get_region_by_id", metrics.RetLabel(err)).Inc()
|
||||
if err != nil {
|
||||
backoffErr = errors.Errorf("loadRegion from PD failed, regionID: %v, err: %v", regionID, err)
|
||||
continue
|
||||
}
|
||||
if region.Meta == nil {
|
||||
backoffErr = errors.Errorf("region not found for regionID %q", regionID)
|
||||
continue
|
||||
}
|
||||
if len(region.Meta.Peers) == 0 {
|
||||
return nil, errors.New("receive Region with no peer")
|
||||
}
|
||||
r := &Region{
|
||||
meta: region.Meta,
|
||||
peer: region.Meta.Peers[0],
|
||||
}
|
||||
if region.Leader != nil {
|
||||
r.SwitchPeer(region.Leader.GetStoreId())
|
||||
}
|
||||
return r, nil
|
||||
}
|
||||
}
|
||||
|
||||
// GetStoreAddr returns a tikv server's address by its storeID. It checks cache
|
||||
// first, sends request to pd server when necessary.
|
||||
func (c *RegionCache) GetStoreAddr(bo *retry.Backoffer, id uint64) (string, error) {
|
||||
c.storeMu.RLock()
|
||||
if store, ok := c.storeMu.stores[id]; ok {
|
||||
c.storeMu.RUnlock()
|
||||
return store.Addr, nil
|
||||
}
|
||||
c.storeMu.RUnlock()
|
||||
return c.ReloadStoreAddr(bo, id)
|
||||
}
|
||||
|
||||
// ReloadStoreAddr reloads store's address.
|
||||
func (c *RegionCache) ReloadStoreAddr(bo *retry.Backoffer, id uint64) (string, error) {
|
||||
addr, err := c.loadStoreAddr(bo, id)
|
||||
if err != nil || addr == "" {
|
||||
return "", err
|
||||
}
|
||||
|
||||
c.storeMu.Lock()
|
||||
defer c.storeMu.Unlock()
|
||||
c.storeMu.stores[id] = &Store{
|
||||
ID: id,
|
||||
Addr: addr,
|
||||
}
|
||||
return addr, nil
|
||||
}
|
||||
|
||||
// ClearStoreByID clears store from cache with storeID.
|
||||
func (c *RegionCache) ClearStoreByID(id uint64) {
|
||||
c.storeMu.Lock()
|
||||
defer c.storeMu.Unlock()
|
||||
delete(c.storeMu.stores, id)
|
||||
}
|
||||
|
||||
func (c *RegionCache) loadStoreAddr(bo *retry.Backoffer, id uint64) (string, error) {
|
||||
for {
|
||||
store, err := c.pdClient.GetStore(bo.GetContext(), id)
|
||||
metrics.RegionCacheCounter.WithLabelValues("get_store", metrics.RetLabel(err)).Inc()
|
||||
if err != nil {
|
||||
if errors.Cause(err) == context.Canceled {
|
||||
return "", err
|
||||
}
|
||||
err = errors.Errorf("loadStore from PD failed, id: %d, err: %v", id, err)
|
||||
if err = bo.Backoff(retry.BoPDRPC, err); err != nil {
|
||||
return "", err
|
||||
}
|
||||
continue
|
||||
}
|
||||
if store == nil {
|
||||
return "", nil
|
||||
}
|
||||
return store.GetAddress(), nil
|
||||
}
|
||||
}
|
||||
|
||||
// DropStoreOnSendRequestFail is used for clearing cache when a tikv server does not respond.
|
||||
func (c *RegionCache) DropStoreOnSendRequestFail(ctx *RPCContext, err error) {
|
||||
// We need to drop the store only when the request is the first one failed on this store.
|
||||
// Because too many concurrently requests trying to drop the store will be blocked on the lock.
|
||||
failedRegionID := ctx.Region
|
||||
failedStoreID := ctx.Peer.StoreId
|
||||
c.mu.Lock()
|
||||
_, ok := c.mu.regions[failedRegionID]
|
||||
if !ok {
|
||||
// The failed region is dropped already by another request, we don't need to iterate the regions
|
||||
// and find regions on the failed store to drop.
|
||||
c.mu.Unlock()
|
||||
return
|
||||
}
|
||||
for id, r := range c.mu.regions {
|
||||
if r.region.peer.GetStoreId() == failedStoreID {
|
||||
c.dropRegionFromCache(id)
|
||||
}
|
||||
}
|
||||
c.mu.Unlock()
|
||||
|
||||
// Store's meta may be out of date.
|
||||
var failedStoreAddr string
|
||||
c.storeMu.Lock()
|
||||
store, ok := c.storeMu.stores[failedStoreID]
|
||||
if ok {
|
||||
failedStoreAddr = store.Addr
|
||||
delete(c.storeMu.stores, failedStoreID)
|
||||
}
|
||||
c.storeMu.Unlock()
|
||||
log.Infof("drop regions that on the store %d(%s) due to send request fail, err: %v",
|
||||
failedStoreID, failedStoreAddr, err)
|
||||
}
|
||||
|
||||
// OnRegionStale removes the old region and inserts new regions into the cache.
|
||||
func (c *RegionCache) OnRegionStale(ctx *RPCContext, newRegions []*metapb.Region) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
c.dropRegionFromCache(ctx.Region)
|
||||
|
||||
for _, meta := range newRegions {
|
||||
if _, ok := c.pdClient.(*CodecPDClient); ok {
|
||||
if err := codec.DecodeRegionMetaKey(meta); err != nil {
|
||||
return errors.Errorf("newRegion's range key is not encoded: %v, %v", meta, err)
|
||||
}
|
||||
}
|
||||
region := &Region{
|
||||
meta: meta,
|
||||
peer: meta.Peers[0],
|
||||
}
|
||||
region.SwitchPeer(ctx.Peer.GetStoreId())
|
||||
c.insertRegionToCache(region)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// PDClient returns the pd.Client in RegionCache.
|
||||
func (c *RegionCache) PDClient() pd.Client {
|
||||
return c.pdClient
|
||||
}
|
||||
|
||||
// btreeItem is BTree's Item that uses []byte to compare.
|
||||
type btreeItem struct {
|
||||
key []byte
|
||||
region *Region
|
||||
}
|
||||
|
||||
func newBtreeItem(r *Region) *btreeItem {
|
||||
return &btreeItem{
|
||||
key: r.StartKey(),
|
||||
region: r,
|
||||
}
|
||||
}
|
||||
|
||||
func newBtreeSearchItem(key []byte) *btreeItem {
|
||||
return &btreeItem{
|
||||
key: key,
|
||||
}
|
||||
}
|
||||
|
||||
func (item *btreeItem) Less(other btree.Item) bool {
|
||||
return bytes.Compare(item.key, other.(*btreeItem).key) < 0
|
||||
}
|
||||
|
||||
// Region stores region's meta and its leader peer.
|
||||
type Region struct {
|
||||
meta *metapb.Region
|
||||
peer *metapb.Peer
|
||||
}
|
||||
|
||||
// GetID returns id.
|
||||
func (r *Region) GetID() uint64 {
|
||||
return r.meta.GetId()
|
||||
}
|
||||
|
||||
// RegionVerID is a unique ID that can identify a Region at a specific version.
|
||||
type RegionVerID struct {
|
||||
id uint64
|
||||
confVer uint64
|
||||
ver uint64
|
||||
}
|
||||
|
||||
// GetID returns the id of the region
|
||||
func (r *RegionVerID) GetID() uint64 {
|
||||
return r.id
|
||||
}
|
||||
|
||||
// VerID returns the Region's RegionVerID.
|
||||
func (r *Region) VerID() RegionVerID {
|
||||
return RegionVerID{
|
||||
id: r.meta.GetId(),
|
||||
confVer: r.meta.GetRegionEpoch().GetConfVer(),
|
||||
ver: r.meta.GetRegionEpoch().GetVersion(),
|
||||
}
|
||||
}
|
||||
|
||||
// StartKey returns StartKey.
|
||||
func (r *Region) StartKey() []byte {
|
||||
return r.meta.StartKey
|
||||
}
|
||||
|
||||
// EndKey returns EndKey.
|
||||
func (r *Region) EndKey() []byte {
|
||||
return r.meta.EndKey
|
||||
}
|
||||
|
||||
// GetContext constructs kvprotopb.Context from region info.
|
||||
func (r *Region) GetContext() *kvrpcpb.Context {
|
||||
return &kvrpcpb.Context{
|
||||
RegionId: r.meta.Id,
|
||||
RegionEpoch: r.meta.RegionEpoch,
|
||||
Peer: r.peer,
|
||||
}
|
||||
}
|
||||
|
||||
// SwitchPeer switches current peer to the one on specific store. It returns
|
||||
// false if no peer matches the storeID.
|
||||
func (r *Region) SwitchPeer(storeID uint64) bool {
|
||||
for _, p := range r.meta.Peers {
|
||||
if p.GetStoreId() == storeID {
|
||||
r.peer = p
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// Contains checks whether the key is in the region, for the maximum region endKey is empty.
|
||||
// startKey <= key < endKey.
|
||||
func (r *Region) Contains(key []byte) bool {
|
||||
return bytes.Compare(r.meta.GetStartKey(), key) <= 0 &&
|
||||
(bytes.Compare(key, r.meta.GetEndKey()) < 0 || len(r.meta.GetEndKey()) == 0)
|
||||
}
|
||||
|
||||
// Store contains a tikv server's address.
|
||||
type Store struct {
|
||||
ID uint64
|
||||
Addr string
|
||||
}
|
||||
|
|
@ -1,235 +0,0 @@
|
|||
// Copyright 2018 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 metrics
|
||||
|
||||
import "github.com/prometheus/client_golang/prometheus"
|
||||
|
||||
// Client metrics.
|
||||
var (
|
||||
TxnCounter = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "txn_total",
|
||||
Help: "Counter of created txns.",
|
||||
})
|
||||
|
||||
TxnHistogram = prometheus.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "txn_durations_seconds",
|
||||
Help: "Bucketed histogram of processing txn",
|
||||
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20),
|
||||
},
|
||||
)
|
||||
|
||||
SnapshotCounter = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "snapshot_total",
|
||||
Help: "Counter of snapshots.",
|
||||
})
|
||||
|
||||
TxnCmdHistogram = prometheus.NewHistogramVec(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "txn_cmd_duration_seconds",
|
||||
Help: "Bucketed histogram of processing time of txn cmds.",
|
||||
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20),
|
||||
}, []string{"type"})
|
||||
|
||||
BackoffCounter = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "backoff_total",
|
||||
Help: "Counter of backoff.",
|
||||
}, []string{"type"})
|
||||
|
||||
BackoffHistogram = prometheus.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "backoff_seconds",
|
||||
Help: "total backoff seconds of a single backoffer.",
|
||||
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20),
|
||||
})
|
||||
|
||||
SendReqHistogram = prometheus.NewHistogramVec(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "request_seconds",
|
||||
Help: "Bucketed histogram of sending request duration.",
|
||||
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20),
|
||||
}, []string{"type", "store"})
|
||||
|
||||
LockResolverCounter = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "lock_resolver_actions_total",
|
||||
Help: "Counter of lock resolver actions.",
|
||||
}, []string{"type"})
|
||||
|
||||
RegionErrorCounter = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "region_err_total",
|
||||
Help: "Counter of region errors.",
|
||||
}, []string{"type"})
|
||||
|
||||
TxnWriteKVCountHistogram = prometheus.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "txn_write_kv_num",
|
||||
Help: "Count of kv pairs to write in a transaction.",
|
||||
Buckets: prometheus.ExponentialBuckets(1, 2, 21),
|
||||
})
|
||||
|
||||
TxnWriteSizeHistogram = prometheus.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "txn_write_size_bytes",
|
||||
Help: "Size of kv pairs to write in a transaction.",
|
||||
Buckets: prometheus.ExponentialBuckets(1, 2, 21),
|
||||
})
|
||||
|
||||
RawkvCmdHistogram = prometheus.NewHistogramVec(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "rawkv_cmd_seconds",
|
||||
Help: "Bucketed histogram of processing time of rawkv cmds.",
|
||||
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20),
|
||||
}, []string{"type"})
|
||||
|
||||
RawkvSizeHistogram = prometheus.NewHistogramVec(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "rawkv_kv_size_bytes",
|
||||
Help: "Size of key/value to put, in bytes.",
|
||||
Buckets: prometheus.ExponentialBuckets(1, 2, 21),
|
||||
}, []string{"type"})
|
||||
|
||||
TxnRegionsNumHistogram = prometheus.NewHistogramVec(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "txn_regions_num",
|
||||
Help: "Number of regions in a transaction.",
|
||||
Buckets: prometheus.ExponentialBuckets(1, 2, 20),
|
||||
}, []string{"type"})
|
||||
|
||||
LoadSafepointCounter = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "load_safepoint_total",
|
||||
Help: "Counter of load safepoint.",
|
||||
}, []string{"type"})
|
||||
|
||||
SecondaryLockCleanupFailureCounter = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "lock_cleanup_task_total",
|
||||
Help: "failure statistic of secondary lock cleanup task.",
|
||||
}, []string{"type"})
|
||||
|
||||
RegionCacheCounter = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "region_cache_operations_total",
|
||||
Help: "Counter of region cache.",
|
||||
}, []string{"type", "result"})
|
||||
|
||||
// PendingBatchRequests indicates the number of requests pending in the batch channel.
|
||||
PendingBatchRequests = prometheus.NewGauge(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "pending_batch_requests",
|
||||
Help: "Pending batch requests",
|
||||
})
|
||||
|
||||
BatchWaitDuration = prometheus.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "client_go",
|
||||
Name: "batch_wait_duration",
|
||||
// Min bucket is [0, 1ns).
|
||||
Buckets: prometheus.ExponentialBuckets(1, 2, 30),
|
||||
Help: "batch wait duration",
|
||||
})
|
||||
|
||||
TSFutureWaitDuration = prometheus.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tikv",
|
||||
Subsystem: "pdclient",
|
||||
Name: "ts_future_wait_seconds",
|
||||
Help: "Bucketed histogram of seconds cost for waiting timestamp future.",
|
||||
Buckets: prometheus.ExponentialBuckets(0.000005, 2, 18), // 5us ~ 128 ms
|
||||
})
|
||||
|
||||
LocalLatchWaitTimeHistogram = prometheus.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: "tidb",
|
||||
Subsystem: "tikvclient",
|
||||
Name: "local_latch_wait_seconds",
|
||||
Help: "Wait time of a get local latch.",
|
||||
Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20),
|
||||
})
|
||||
)
|
||||
|
||||
// RetLabel returns "ok" when err == nil and "err" when err != nil.
|
||||
// This could be useful when you need to observe the operation result.
|
||||
func RetLabel(err error) string {
|
||||
if err == nil {
|
||||
return "ok"
|
||||
}
|
||||
return "err"
|
||||
}
|
||||
|
||||
func init() {
|
||||
prometheus.MustRegister(TxnCounter)
|
||||
prometheus.MustRegister(SnapshotCounter)
|
||||
prometheus.MustRegister(TxnHistogram)
|
||||
prometheus.MustRegister(TxnCmdHistogram)
|
||||
prometheus.MustRegister(BackoffCounter)
|
||||
prometheus.MustRegister(BackoffHistogram)
|
||||
prometheus.MustRegister(SendReqHistogram)
|
||||
prometheus.MustRegister(LockResolverCounter)
|
||||
prometheus.MustRegister(RegionErrorCounter)
|
||||
prometheus.MustRegister(TxnWriteKVCountHistogram)
|
||||
prometheus.MustRegister(TxnWriteSizeHistogram)
|
||||
prometheus.MustRegister(RawkvCmdHistogram)
|
||||
prometheus.MustRegister(RawkvSizeHistogram)
|
||||
prometheus.MustRegister(TxnRegionsNumHistogram)
|
||||
prometheus.MustRegister(LoadSafepointCounter)
|
||||
prometheus.MustRegister(SecondaryLockCleanupFailureCounter)
|
||||
prometheus.MustRegister(RegionCacheCounter)
|
||||
prometheus.MustRegister(PendingBatchRequests)
|
||||
prometheus.MustRegister(BatchWaitDuration)
|
||||
prometheus.MustRegister(TSFutureWaitDuration)
|
||||
prometheus.MustRegister(LocalLatchWaitTimeHistogram)
|
||||
}
|
||||
|
|
@ -1,49 +0,0 @@
|
|||
// 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 metrics
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/push"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// PushMetrics pushes metrics to Prometheus Pushgateway.
|
||||
// Note:
|
||||
// * Normally, you need to start a goroutine to push metrics: `go
|
||||
// PushMetrics(...)`
|
||||
// * `instance` should be global identical -- NO 2 processes share a same
|
||||
// `instance`.
|
||||
// * `job` is used to distinguish different workloads, DO NOT use too many `job`
|
||||
// labels since there are grafana panels that groups by `job`.
|
||||
func PushMetrics(ctx context.Context, addr string, interval time.Duration, job, instance string) {
|
||||
ticker := time.NewTicker(interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
}
|
||||
|
||||
err := push.New(addr, job).Grouping("instance", instance).Gatherer(prometheus.DefaultGatherer).Push()
|
||||
if err != nil {
|
||||
log.Errorf("cannot push metrics to prometheus pushgateway: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1,549 +0,0 @@
|
|||
// Copyright 2016 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 mocktikv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"math"
|
||||
"sync"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/tikv/client-go/codec"
|
||||
)
|
||||
|
||||
// Cluster simulates a TiKV cluster. It focuses on management and the change of
|
||||
// meta data. A Cluster mainly includes following 3 kinds of meta data:
|
||||
// 1) Region: A Region is a fragment of TiKV's data whose range is [start, end).
|
||||
// The data of a Region is duplicated to multiple Peers and distributed in
|
||||
// multiple Stores.
|
||||
// 2) Peer: A Peer is a replica of a Region's data. All peers of a Region form
|
||||
// a group, each group elects a Leader to provide services.
|
||||
// 3) Store: A Store is a storage/service node. Try to think it as a TiKV server
|
||||
// process. Only the store with request's Region's leader Peer could respond
|
||||
// to client's request.
|
||||
type Cluster struct {
|
||||
sync.RWMutex
|
||||
id uint64
|
||||
stores map[uint64]*Store
|
||||
regions map[uint64]*Region
|
||||
}
|
||||
|
||||
// NewCluster creates an empty cluster. It needs to be bootstrapped before
|
||||
// providing service.
|
||||
func NewCluster() *Cluster {
|
||||
return &Cluster{
|
||||
stores: make(map[uint64]*Store),
|
||||
regions: make(map[uint64]*Region),
|
||||
}
|
||||
}
|
||||
|
||||
// AllocID creates an unique ID in cluster. The ID could be used as either
|
||||
// StoreID, RegionID, or PeerID.
|
||||
func (c *Cluster) AllocID() uint64 {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
return c.allocID()
|
||||
}
|
||||
|
||||
// AllocIDs creates multiple IDs.
|
||||
func (c *Cluster) AllocIDs(n int) []uint64 {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
var ids []uint64
|
||||
for len(ids) < n {
|
||||
ids = append(ids, c.allocID())
|
||||
}
|
||||
return ids
|
||||
}
|
||||
|
||||
func (c *Cluster) allocID() uint64 {
|
||||
c.id++
|
||||
return c.id
|
||||
}
|
||||
|
||||
// GetAllRegions gets all the regions in the cluster.
|
||||
func (c *Cluster) GetAllRegions() []*Region {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
regions := make([]*Region, 0, len(c.regions))
|
||||
for _, region := range c.regions {
|
||||
regions = append(regions, region)
|
||||
}
|
||||
return regions
|
||||
}
|
||||
|
||||
// GetStore returns a Store's meta.
|
||||
func (c *Cluster) GetStore(storeID uint64) *metapb.Store {
|
||||
c.RLock()
|
||||
defer c.RUnlock()
|
||||
|
||||
if store := c.stores[storeID]; store != nil {
|
||||
return proto.Clone(store.meta).(*metapb.Store)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// StopStore stops a store with storeID.
|
||||
func (c *Cluster) StopStore(storeID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
if store := c.stores[storeID]; store != nil {
|
||||
store.meta.State = metapb.StoreState_Offline
|
||||
}
|
||||
}
|
||||
|
||||
// StartStore starts a store with storeID.
|
||||
func (c *Cluster) StartStore(storeID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
if store := c.stores[storeID]; store != nil {
|
||||
store.meta.State = metapb.StoreState_Up
|
||||
}
|
||||
}
|
||||
|
||||
// CancelStore makes the store with cancel state true.
|
||||
func (c *Cluster) CancelStore(storeID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
//A store returns context.Cancelled Error when cancel is true.
|
||||
if store := c.stores[storeID]; store != nil {
|
||||
store.cancel = true
|
||||
}
|
||||
}
|
||||
|
||||
// UnCancelStore makes the store with cancel state false.
|
||||
func (c *Cluster) UnCancelStore(storeID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
if store := c.stores[storeID]; store != nil {
|
||||
store.cancel = false
|
||||
}
|
||||
}
|
||||
|
||||
// GetStoreByAddr returns a Store's meta by an addr.
|
||||
func (c *Cluster) GetStoreByAddr(addr string) *metapb.Store {
|
||||
c.RLock()
|
||||
defer c.RUnlock()
|
||||
|
||||
for _, s := range c.stores {
|
||||
if s.meta.GetAddress() == addr {
|
||||
return proto.Clone(s.meta).(*metapb.Store)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetAndCheckStoreByAddr checks and returns a Store's meta by an addr
|
||||
func (c *Cluster) GetAndCheckStoreByAddr(addr string) (*metapb.Store, error) {
|
||||
c.RLock()
|
||||
defer c.RUnlock()
|
||||
|
||||
for _, s := range c.stores {
|
||||
if s.cancel {
|
||||
return nil, context.Canceled
|
||||
}
|
||||
if s.meta.GetAddress() == addr {
|
||||
return proto.Clone(s.meta).(*metapb.Store), nil
|
||||
}
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// AddStore adds a new Store to the cluster.
|
||||
func (c *Cluster) AddStore(storeID uint64, addr string) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
c.stores[storeID] = newStore(storeID, addr)
|
||||
}
|
||||
|
||||
// RemoveStore removes a Store from the cluster.
|
||||
func (c *Cluster) RemoveStore(storeID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
delete(c.stores, storeID)
|
||||
}
|
||||
|
||||
// UpdateStoreAddr updates store address for cluster.
|
||||
func (c *Cluster) UpdateStoreAddr(storeID uint64, addr string) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
c.stores[storeID] = newStore(storeID, addr)
|
||||
}
|
||||
|
||||
// GetRegion returns a Region's meta and leader ID.
|
||||
func (c *Cluster) GetRegion(regionID uint64) (*metapb.Region, uint64) {
|
||||
c.RLock()
|
||||
defer c.RUnlock()
|
||||
|
||||
r, ok := c.regions[regionID]
|
||||
if !ok {
|
||||
return nil, 0
|
||||
}
|
||||
return proto.Clone(r.Meta).(*metapb.Region), r.leader
|
||||
}
|
||||
|
||||
// GetRegionByKey returns the Region and its leader whose range contains the key.
|
||||
func (c *Cluster) GetRegionByKey(key []byte) (*metapb.Region, *metapb.Peer) {
|
||||
c.RLock()
|
||||
defer c.RUnlock()
|
||||
|
||||
for _, r := range c.regions {
|
||||
if regionContains(r.Meta.StartKey, r.Meta.EndKey, key) {
|
||||
return proto.Clone(r.Meta).(*metapb.Region), proto.Clone(r.leaderPeer()).(*metapb.Peer)
|
||||
}
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// GetPrevRegionByKey returns the previous Region and its leader whose range contains the key.
|
||||
func (c *Cluster) GetPrevRegionByKey(key []byte) (*metapb.Region, *metapb.Peer) {
|
||||
c.RLock()
|
||||
defer c.RUnlock()
|
||||
|
||||
currentRegion, _ := c.GetRegionByKey(key)
|
||||
if len(currentRegion.StartKey) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
for _, r := range c.regions {
|
||||
if bytes.Equal(r.Meta.EndKey, currentRegion.StartKey) {
|
||||
return proto.Clone(r.Meta).(*metapb.Region), proto.Clone(r.leaderPeer()).(*metapb.Peer)
|
||||
}
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// GetRegionByID returns the Region and its leader whose ID is regionID.
|
||||
func (c *Cluster) GetRegionByID(regionID uint64) (*metapb.Region, *metapb.Peer) {
|
||||
c.RLock()
|
||||
defer c.RUnlock()
|
||||
|
||||
for _, r := range c.regions {
|
||||
if r.Meta.GetId() == regionID {
|
||||
return proto.Clone(r.Meta).(*metapb.Region), proto.Clone(r.leaderPeer()).(*metapb.Peer)
|
||||
}
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Bootstrap creates the first Region. The Stores should be in the Cluster before
|
||||
// bootstrap.
|
||||
func (c *Cluster) Bootstrap(regionID uint64, storeIDs, peerIDs []uint64, leaderPeerID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
c.regions[regionID] = newRegion(regionID, storeIDs, peerIDs, leaderPeerID)
|
||||
}
|
||||
|
||||
// AddPeer adds a new Peer for the Region on the Store.
|
||||
func (c *Cluster) AddPeer(regionID, storeID, peerID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
c.regions[regionID].addPeer(peerID, storeID)
|
||||
}
|
||||
|
||||
// RemovePeer removes the Peer from the Region. Note that if the Peer is leader,
|
||||
// the Region will have no leader before calling ChangeLeader().
|
||||
func (c *Cluster) RemovePeer(regionID, storeID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
c.regions[regionID].removePeer(storeID)
|
||||
}
|
||||
|
||||
// ChangeLeader sets the Region's leader Peer. Caller should guarantee the Peer
|
||||
// exists.
|
||||
func (c *Cluster) ChangeLeader(regionID, leaderPeerID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
c.regions[regionID].changeLeader(leaderPeerID)
|
||||
}
|
||||
|
||||
// GiveUpLeader sets the Region's leader to 0. The Region will have no leader
|
||||
// before calling ChangeLeader().
|
||||
func (c *Cluster) GiveUpLeader(regionID uint64) {
|
||||
c.ChangeLeader(regionID, 0)
|
||||
}
|
||||
|
||||
// Split splits a Region at the key (encoded) and creates new Region.
|
||||
func (c *Cluster) Split(regionID, newRegionID uint64, key []byte, peerIDs []uint64, leaderPeerID uint64) {
|
||||
c.SplitRaw(regionID, newRegionID, NewMvccKey(key), peerIDs, leaderPeerID)
|
||||
}
|
||||
|
||||
// SplitRaw splits a Region at the key (not encoded) and creates new Region.
|
||||
func (c *Cluster) SplitRaw(regionID, newRegionID uint64, rawKey []byte, peerIDs []uint64, leaderPeerID uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
newRegion := c.regions[regionID].split(newRegionID, rawKey, peerIDs, leaderPeerID)
|
||||
c.regions[newRegionID] = newRegion
|
||||
}
|
||||
|
||||
// Merge merges 2 regions, their key ranges should be adjacent.
|
||||
func (c *Cluster) Merge(regionID1, regionID2 uint64) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
c.regions[regionID1].merge(c.regions[regionID2].Meta.GetEndKey())
|
||||
delete(c.regions, regionID2)
|
||||
}
|
||||
|
||||
// SplitTable evenly splits the data in table into count regions.
|
||||
// Only works for single store.
|
||||
func (c *Cluster) SplitTable(mvccStore MVCCStore, tableID int64, count int) {
|
||||
tableStart := codec.GenTableRecordPrefix(tableID)
|
||||
tableEnd := tableStart.PrefixNext()
|
||||
c.splitRange(mvccStore, NewMvccKey(tableStart), NewMvccKey(tableEnd), count)
|
||||
}
|
||||
|
||||
// SplitIndex evenly splits the data in index into count regions.
|
||||
// Only works for single store.
|
||||
func (c *Cluster) SplitIndex(mvccStore MVCCStore, tableID, indexID int64, count int) {
|
||||
indexStart := codec.EncodeTableIndexPrefix(tableID, indexID)
|
||||
indexEnd := indexStart.PrefixNext()
|
||||
c.splitRange(mvccStore, NewMvccKey(indexStart), NewMvccKey(indexEnd), count)
|
||||
}
|
||||
|
||||
func (c *Cluster) splitRange(mvccStore MVCCStore, start, end MvccKey, count int) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
c.evacuateOldRegionRanges(start, end)
|
||||
regionPairs := c.getEntriesGroupByRegions(mvccStore, start, end, count)
|
||||
c.createNewRegions(regionPairs, start, end)
|
||||
}
|
||||
|
||||
// getPairsGroupByRegions groups the key value pairs into splitted regions.
|
||||
func (c *Cluster) getEntriesGroupByRegions(mvccStore MVCCStore, start, end MvccKey, count int) [][]Pair {
|
||||
startTS := uint64(math.MaxUint64)
|
||||
limit := int(math.MaxInt32)
|
||||
pairs := mvccStore.Scan(start.Raw(), end.Raw(), limit, startTS, kvrpcpb.IsolationLevel_SI)
|
||||
regionEntriesSlice := make([][]Pair, 0, count)
|
||||
quotient := len(pairs) / count
|
||||
remainder := len(pairs) % count
|
||||
i := 0
|
||||
for i < len(pairs) {
|
||||
regionEntryCount := quotient
|
||||
if remainder > 0 {
|
||||
remainder--
|
||||
regionEntryCount++
|
||||
}
|
||||
regionEntries := pairs[i : i+regionEntryCount]
|
||||
regionEntriesSlice = append(regionEntriesSlice, regionEntries)
|
||||
i += regionEntryCount
|
||||
}
|
||||
return regionEntriesSlice
|
||||
}
|
||||
|
||||
func (c *Cluster) createNewRegions(regionPairs [][]Pair, start, end MvccKey) {
|
||||
for i := range regionPairs {
|
||||
peerID := c.allocID()
|
||||
newRegion := newRegion(c.allocID(), []uint64{c.firstStoreID()}, []uint64{peerID}, peerID)
|
||||
var regionStartKey, regionEndKey MvccKey
|
||||
if i == 0 {
|
||||
regionStartKey = start
|
||||
} else {
|
||||
regionStartKey = NewMvccKey(regionPairs[i][0].Key)
|
||||
}
|
||||
if i == len(regionPairs)-1 {
|
||||
regionEndKey = end
|
||||
} else {
|
||||
// Use the next region's first key as region end key.
|
||||
regionEndKey = NewMvccKey(regionPairs[i+1][0].Key)
|
||||
}
|
||||
newRegion.updateKeyRange(regionStartKey, regionEndKey)
|
||||
c.regions[newRegion.Meta.Id] = newRegion
|
||||
}
|
||||
}
|
||||
|
||||
// evacuateOldRegionRanges evacuate the range [start, end].
|
||||
// Old regions has intersection with [start, end) will be updated or deleted.
|
||||
func (c *Cluster) evacuateOldRegionRanges(start, end MvccKey) {
|
||||
oldRegions := c.getRegionsCoverRange(start, end)
|
||||
for _, oldRegion := range oldRegions {
|
||||
startCmp := bytes.Compare(oldRegion.Meta.StartKey, start)
|
||||
endCmp := bytes.Compare(oldRegion.Meta.EndKey, end)
|
||||
if len(oldRegion.Meta.EndKey) == 0 {
|
||||
endCmp = 1
|
||||
}
|
||||
if startCmp >= 0 && endCmp <= 0 {
|
||||
// The region is within table data, it will be replaced by new regions.
|
||||
delete(c.regions, oldRegion.Meta.Id)
|
||||
} else if startCmp < 0 && endCmp > 0 {
|
||||
// A single Region covers table data, split into two regions that do not overlap table data.
|
||||
oldEnd := oldRegion.Meta.EndKey
|
||||
oldRegion.updateKeyRange(oldRegion.Meta.StartKey, start)
|
||||
peerID := c.allocID()
|
||||
newRegion := newRegion(c.allocID(), []uint64{c.firstStoreID()}, []uint64{peerID}, peerID)
|
||||
newRegion.updateKeyRange(end, oldEnd)
|
||||
c.regions[newRegion.Meta.Id] = newRegion
|
||||
} else if startCmp < 0 {
|
||||
oldRegion.updateKeyRange(oldRegion.Meta.StartKey, start)
|
||||
} else {
|
||||
oldRegion.updateKeyRange(end, oldRegion.Meta.EndKey)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Cluster) firstStoreID() uint64 {
|
||||
for id := range c.stores {
|
||||
return id
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
// getRegionsCoverRange gets regions in the cluster that has intersection with [start, end).
|
||||
func (c *Cluster) getRegionsCoverRange(start, end MvccKey) []*Region {
|
||||
var regions []*Region
|
||||
for _, region := range c.regions {
|
||||
onRight := bytes.Compare(end, region.Meta.StartKey) <= 0
|
||||
onLeft := bytes.Compare(region.Meta.EndKey, start) <= 0
|
||||
if len(region.Meta.EndKey) == 0 {
|
||||
onLeft = false
|
||||
}
|
||||
if onLeft || onRight {
|
||||
continue
|
||||
}
|
||||
regions = append(regions, region)
|
||||
}
|
||||
return regions
|
||||
}
|
||||
|
||||
// Region is the Region meta data.
|
||||
type Region struct {
|
||||
Meta *metapb.Region
|
||||
leader uint64
|
||||
}
|
||||
|
||||
func newPeerMeta(peerID, storeID uint64) *metapb.Peer {
|
||||
return &metapb.Peer{
|
||||
Id: peerID,
|
||||
StoreId: storeID,
|
||||
}
|
||||
}
|
||||
|
||||
func newRegion(regionID uint64, storeIDs, peerIDs []uint64, leaderPeerID uint64) *Region {
|
||||
if len(storeIDs) != len(peerIDs) {
|
||||
panic("length of storeIDs and peerIDs mismatch")
|
||||
}
|
||||
peers := make([]*metapb.Peer, 0, len(storeIDs))
|
||||
for i := range storeIDs {
|
||||
peers = append(peers, newPeerMeta(peerIDs[i], storeIDs[i]))
|
||||
}
|
||||
meta := &metapb.Region{
|
||||
Id: regionID,
|
||||
Peers: peers,
|
||||
}
|
||||
return &Region{
|
||||
Meta: meta,
|
||||
leader: leaderPeerID,
|
||||
}
|
||||
}
|
||||
|
||||
func (r *Region) addPeer(peerID, storeID uint64) {
|
||||
r.Meta.Peers = append(r.Meta.Peers, newPeerMeta(peerID, storeID))
|
||||
r.incConfVer()
|
||||
}
|
||||
|
||||
func (r *Region) removePeer(peerID uint64) {
|
||||
for i, peer := range r.Meta.Peers {
|
||||
if peer.GetId() == peerID {
|
||||
r.Meta.Peers = append(r.Meta.Peers[:i], r.Meta.Peers[i+1:]...)
|
||||
break
|
||||
}
|
||||
}
|
||||
if r.leader == peerID {
|
||||
r.leader = 0
|
||||
}
|
||||
r.incConfVer()
|
||||
}
|
||||
|
||||
func (r *Region) changeLeader(leaderID uint64) {
|
||||
r.leader = leaderID
|
||||
}
|
||||
|
||||
func (r *Region) leaderPeer() *metapb.Peer {
|
||||
for _, p := range r.Meta.Peers {
|
||||
if p.GetId() == r.leader {
|
||||
return p
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *Region) split(newRegionID uint64, key MvccKey, peerIDs []uint64, leaderPeerID uint64) *Region {
|
||||
if len(r.Meta.Peers) != len(peerIDs) {
|
||||
panic("length of storeIDs and peerIDs mismatch")
|
||||
}
|
||||
storeIDs := make([]uint64, 0, len(r.Meta.Peers))
|
||||
for _, peer := range r.Meta.Peers {
|
||||
storeIDs = append(storeIDs, peer.GetStoreId())
|
||||
}
|
||||
region := newRegion(newRegionID, storeIDs, peerIDs, leaderPeerID)
|
||||
region.updateKeyRange(key, r.Meta.EndKey)
|
||||
r.updateKeyRange(r.Meta.StartKey, key)
|
||||
return region
|
||||
}
|
||||
|
||||
func (r *Region) merge(endKey MvccKey) {
|
||||
r.Meta.EndKey = endKey
|
||||
r.incVersion()
|
||||
}
|
||||
|
||||
func (r *Region) updateKeyRange(start, end MvccKey) {
|
||||
r.Meta.StartKey = start
|
||||
r.Meta.EndKey = end
|
||||
r.incVersion()
|
||||
}
|
||||
|
||||
func (r *Region) incConfVer() {
|
||||
r.Meta.RegionEpoch = &metapb.RegionEpoch{
|
||||
ConfVer: r.Meta.GetRegionEpoch().GetConfVer() + 1,
|
||||
Version: r.Meta.GetRegionEpoch().GetVersion(),
|
||||
}
|
||||
}
|
||||
|
||||
func (r *Region) incVersion() {
|
||||
r.Meta.RegionEpoch = &metapb.RegionEpoch{
|
||||
ConfVer: r.Meta.GetRegionEpoch().GetConfVer(),
|
||||
Version: r.Meta.GetRegionEpoch().GetVersion() + 1,
|
||||
}
|
||||
}
|
||||
|
||||
// Store is the Store's meta data.
|
||||
type Store struct {
|
||||
meta *metapb.Store
|
||||
cancel bool // return context.Cancelled error when cancel is true.
|
||||
}
|
||||
|
||||
func newStore(storeID uint64, addr string) *Store {
|
||||
return &Store{
|
||||
meta: &metapb.Store{
|
||||
Id: storeID,
|
||||
Address: addr,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
|
@ -1,51 +0,0 @@
|
|||
// Copyright 2016 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 mocktikv
|
||||
|
||||
import "fmt"
|
||||
|
||||
// BootstrapWithSingleStore initializes a Cluster with 1 Region and 1 Store.
|
||||
func BootstrapWithSingleStore(cluster *Cluster) (storeID, peerID, regionID uint64) {
|
||||
ids := cluster.AllocIDs(3)
|
||||
storeID, peerID, regionID = ids[0], ids[1], ids[2]
|
||||
cluster.AddStore(storeID, fmt.Sprintf("store%d", storeID))
|
||||
cluster.Bootstrap(regionID, []uint64{storeID}, []uint64{peerID}, peerID)
|
||||
return
|
||||
}
|
||||
|
||||
// BootstrapWithMultiStores initializes a Cluster with 1 Region and n Stores.
|
||||
func BootstrapWithMultiStores(cluster *Cluster, n int) (storeIDs, peerIDs []uint64, regionID uint64, leaderPeer uint64) {
|
||||
storeIDs = cluster.AllocIDs(n)
|
||||
peerIDs = cluster.AllocIDs(n)
|
||||
leaderPeer = peerIDs[0]
|
||||
regionID = cluster.AllocID()
|
||||
for _, storeID := range storeIDs {
|
||||
cluster.AddStore(storeID, fmt.Sprintf("store%d", storeID))
|
||||
}
|
||||
cluster.Bootstrap(regionID, storeIDs, peerIDs, leaderPeer)
|
||||
return
|
||||
}
|
||||
|
||||
// BootstrapWithMultiRegions initializes a Cluster with multiple Regions and 1
|
||||
// Store. The number of Regions will be len(splitKeys) + 1.
|
||||
func BootstrapWithMultiRegions(cluster *Cluster, splitKeys ...[]byte) (storeID uint64, regionIDs, peerIDs []uint64) {
|
||||
var firstRegionID, firstPeerID uint64
|
||||
storeID, firstPeerID, firstRegionID = BootstrapWithSingleStore(cluster)
|
||||
regionIDs = append([]uint64{firstRegionID}, cluster.AllocIDs(len(splitKeys))...)
|
||||
peerIDs = append([]uint64{firstPeerID}, cluster.AllocIDs(len(splitKeys))...)
|
||||
for i, k := range splitKeys {
|
||||
cluster.Split(regionIDs[i], regionIDs[i+1], k, []uint64{peerIDs[i]}, peerIDs[i])
|
||||
}
|
||||
return
|
||||
}
|
||||
|
|
@ -1,52 +0,0 @@
|
|||
// Copyright 2016 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 mocktikv
|
||||
|
||||
import "fmt"
|
||||
|
||||
// ErrLocked is returned when trying to Read/Write on a locked key. Client should
|
||||
// backoff or cleanup the lock then retry.
|
||||
type ErrLocked struct {
|
||||
Key MvccKey
|
||||
Primary []byte
|
||||
StartTS uint64
|
||||
TTL uint64
|
||||
}
|
||||
|
||||
// Error formats the lock to a string.
|
||||
func (e *ErrLocked) Error() string {
|
||||
return fmt.Sprintf("key is locked, key: %q, primary: %q, startTS: %v", e.Key, e.Primary, e.StartTS)
|
||||
}
|
||||
|
||||
// ErrRetryable suggests that client may restart the txn. e.g. write conflict.
|
||||
type ErrRetryable string
|
||||
|
||||
func (e ErrRetryable) Error() string {
|
||||
return fmt.Sprintf("retryable: %s", string(e))
|
||||
}
|
||||
|
||||
// ErrAbort means something is wrong and client should abort the txn.
|
||||
type ErrAbort string
|
||||
|
||||
func (e ErrAbort) Error() string {
|
||||
return fmt.Sprintf("abort: %s", string(e))
|
||||
}
|
||||
|
||||
// ErrAlreadyCommitted is returned specially when client tries to rollback a
|
||||
// committed lock.
|
||||
type ErrAlreadyCommitted uint64
|
||||
|
||||
func (e ErrAlreadyCommitted) Error() string {
|
||||
return fmt.Sprint("txn already committed")
|
||||
}
|
||||
|
|
@ -1,34 +0,0 @@
|
|||
// Copyright 2018 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 mocktikv
|
||||
|
||||
import pd "github.com/tikv/pd/client"
|
||||
|
||||
// NewTiKVAndPDClient creates a TiKV client and PD client from options.
|
||||
func NewTiKVAndPDClient(cluster *Cluster, mvccStore MVCCStore, path string) (*RPCClient, pd.Client, error) {
|
||||
if cluster == nil {
|
||||
cluster = NewCluster()
|
||||
BootstrapWithSingleStore(cluster)
|
||||
}
|
||||
|
||||
if mvccStore == nil {
|
||||
var err error
|
||||
mvccStore, err = NewMVCCLevelDB(path)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return NewRPCClient(cluster, mvccStore), NewPDClient(cluster), nil
|
||||
}
|
||||
|
|
@ -1,566 +0,0 @@
|
|||
// Copyright 2016 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 mocktikv
|
||||
|
||||
import (
|
||||
"math"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
)
|
||||
|
||||
func TestT(t *testing.T) {
|
||||
TestingT(t)
|
||||
}
|
||||
|
||||
// testMockTiKVSuite tests MVCCStore interface.
|
||||
// SetUpTest should set specific MVCCStore implementation.
|
||||
type testMockTiKVSuite struct {
|
||||
store MVCCStore
|
||||
}
|
||||
|
||||
type testMarshal struct{}
|
||||
|
||||
// testMVCCLevelDB is used to test MVCCLevelDB implementation.
|
||||
type testMVCCLevelDB struct {
|
||||
testMockTiKVSuite
|
||||
}
|
||||
|
||||
var (
|
||||
_ = Suite(&testMVCCLevelDB{})
|
||||
_ = Suite(testMarshal{})
|
||||
)
|
||||
|
||||
func (s *testMockTiKVSuite) SetUpTest(c *C) {
|
||||
var err error
|
||||
s.store, err = NewMVCCLevelDB("")
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func putMutations(kvpairs ...string) []*kvrpcpb.Mutation {
|
||||
var mutations []*kvrpcpb.Mutation
|
||||
for i := 0; i < len(kvpairs); i += 2 {
|
||||
mutations = append(mutations, &kvrpcpb.Mutation{
|
||||
Op: kvrpcpb.Op_Put,
|
||||
Key: []byte(kvpairs[i]),
|
||||
Value: []byte(kvpairs[i+1]),
|
||||
})
|
||||
}
|
||||
return mutations
|
||||
}
|
||||
|
||||
func lock(key, primary string, ts uint64) *kvrpcpb.LockInfo {
|
||||
return &kvrpcpb.LockInfo{
|
||||
Key: []byte(key),
|
||||
PrimaryLock: []byte(primary),
|
||||
LockVersion: ts,
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustGetNone(c *C, key string, ts uint64) {
|
||||
val, err := s.store.Get([]byte(key), ts, kvrpcpb.IsolationLevel_SI)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(val, IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustGetErr(c *C, key string, ts uint64) {
|
||||
val, err := s.store.Get([]byte(key), ts, kvrpcpb.IsolationLevel_SI)
|
||||
c.Assert(err, NotNil)
|
||||
c.Assert(val, IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustGetOK(c *C, key string, ts uint64, expect string) {
|
||||
val, err := s.store.Get([]byte(key), ts, kvrpcpb.IsolationLevel_SI)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(val), Equals, expect)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustGetRC(c *C, key string, ts uint64, expect string) {
|
||||
val, err := s.store.Get([]byte(key), ts, kvrpcpb.IsolationLevel_RC)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(val), Equals, expect)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustPutOK(c *C, key, value string, startTS, commitTS uint64) {
|
||||
errs := s.store.Prewrite(putMutations(key, value), []byte(key), startTS, 0)
|
||||
for _, err := range errs {
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
err := s.store.Commit([][]byte{[]byte(key)}, startTS, commitTS)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustDeleteOK(c *C, key string, startTS, commitTS uint64) {
|
||||
mutations := []*kvrpcpb.Mutation{
|
||||
{
|
||||
Op: kvrpcpb.Op_Del,
|
||||
Key: []byte(key),
|
||||
},
|
||||
}
|
||||
errs := s.store.Prewrite(mutations, []byte(key), startTS, 0)
|
||||
for _, err := range errs {
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
err := s.store.Commit([][]byte{[]byte(key)}, startTS, commitTS)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustScanOK(c *C, start string, limit int, ts uint64, expect ...string) {
|
||||
s.mustRangeScanOK(c, start, "", limit, ts, expect...)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustRangeScanOK(c *C, start, end string, limit int, ts uint64, expect ...string) {
|
||||
pairs := s.store.Scan([]byte(start), []byte(end), limit, ts, kvrpcpb.IsolationLevel_SI)
|
||||
c.Assert(len(pairs)*2, Equals, len(expect))
|
||||
for i := 0; i < len(pairs); i++ {
|
||||
c.Assert(pairs[i].Err, IsNil)
|
||||
c.Assert(pairs[i].Key, BytesEquals, []byte(expect[i*2]))
|
||||
c.Assert(string(pairs[i].Value), Equals, expect[i*2+1])
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustReverseScanOK(c *C, end string, limit int, ts uint64, expect ...string) {
|
||||
s.mustRangeReverseScanOK(c, "", end, limit, ts, expect...)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustRangeReverseScanOK(c *C, start, end string, limit int, ts uint64, expect ...string) {
|
||||
pairs := s.store.ReverseScan([]byte(start), []byte(end), limit, ts, kvrpcpb.IsolationLevel_SI)
|
||||
c.Assert(len(pairs)*2, Equals, len(expect))
|
||||
for i := 0; i < len(pairs); i++ {
|
||||
c.Assert(pairs[i].Err, IsNil)
|
||||
c.Assert(pairs[i].Key, BytesEquals, []byte(expect[i*2]))
|
||||
c.Assert(string(pairs[i].Value), Equals, expect[i*2+1])
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustPrewriteOK(c *C, mutations []*kvrpcpb.Mutation, primary string, startTS uint64) {
|
||||
errs := s.store.Prewrite(mutations, []byte(primary), startTS, 0)
|
||||
for _, err := range errs {
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustCommitOK(c *C, keys [][]byte, startTS, commitTS uint64) {
|
||||
err := s.store.Commit(keys, startTS, commitTS)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustCommitErr(c *C, keys [][]byte, startTS, commitTS uint64) {
|
||||
err := s.store.Commit(keys, startTS, commitTS)
|
||||
c.Assert(err, NotNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustRollbackOK(c *C, keys [][]byte, startTS uint64) {
|
||||
err := s.store.Rollback(keys, startTS)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustRollbackErr(c *C, keys [][]byte, startTS uint64) {
|
||||
err := s.store.Rollback(keys, startTS)
|
||||
c.Assert(err, NotNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustScanLock(c *C, maxTs uint64, expect []*kvrpcpb.LockInfo) {
|
||||
locks, err := s.store.ScanLock(nil, nil, maxTs)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(locks, DeepEquals, expect)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustResolveLock(c *C, startTS, commitTS uint64) {
|
||||
c.Assert(s.store.ResolveLock(nil, nil, startTS, commitTS), IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustBatchResolveLock(c *C, txnInfos map[uint64]uint64) {
|
||||
c.Assert(s.store.BatchResolveLock(nil, nil, txnInfos), IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustDeleteRange(c *C, startKey, endKey string) {
|
||||
err := s.store.DeleteRange([]byte(startKey), []byte(endKey))
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestGet(c *C) {
|
||||
s.mustGetNone(c, "x", 10)
|
||||
s.mustPutOK(c, "x", "x", 5, 10)
|
||||
s.mustGetNone(c, "x", 9)
|
||||
s.mustGetOK(c, "x", 10, "x")
|
||||
s.mustGetOK(c, "x", 11, "x")
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestGetWithLock(c *C) {
|
||||
key := "key"
|
||||
value := "value"
|
||||
s.mustPutOK(c, key, value, 5, 10)
|
||||
mutations := []*kvrpcpb.Mutation{{
|
||||
Op: kvrpcpb.Op_Lock,
|
||||
Key: []byte(key),
|
||||
},
|
||||
}
|
||||
// test with lock's type is lock
|
||||
s.mustPrewriteOK(c, mutations, key, 20)
|
||||
s.mustGetOK(c, key, 25, value)
|
||||
s.mustCommitOK(c, [][]byte{[]byte(key)}, 20, 30)
|
||||
|
||||
// test get with lock's max ts and primary key
|
||||
s.mustPrewriteOK(c, putMutations(key, "value2", "key2", "v5"), key, 40)
|
||||
s.mustGetErr(c, key, 41)
|
||||
s.mustGetErr(c, "key2", math.MaxUint64)
|
||||
s.mustGetOK(c, key, math.MaxUint64, "value")
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestDelete(c *C) {
|
||||
s.mustPutOK(c, "x", "x5-10", 5, 10)
|
||||
s.mustDeleteOK(c, "x", 15, 20)
|
||||
s.mustGetNone(c, "x", 5)
|
||||
s.mustGetNone(c, "x", 9)
|
||||
s.mustGetOK(c, "x", 10, "x5-10")
|
||||
s.mustGetOK(c, "x", 19, "x5-10")
|
||||
s.mustGetNone(c, "x", 20)
|
||||
s.mustGetNone(c, "x", 21)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestCleanupRollback(c *C) {
|
||||
s.mustPutOK(c, "secondary", "s-0", 1, 2)
|
||||
s.mustPrewriteOK(c, putMutations("primary", "p-5", "secondary", "s-5"), "primary", 5)
|
||||
s.mustGetErr(c, "secondary", 8)
|
||||
s.mustGetErr(c, "secondary", 12)
|
||||
s.mustCommitOK(c, [][]byte{[]byte("primary")}, 5, 10)
|
||||
s.mustRollbackErr(c, [][]byte{[]byte("primary")}, 5)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestReverseScan(c *C) {
|
||||
// ver10: A(10) - B(_) - C(10) - D(_) - E(10)
|
||||
s.mustPutOK(c, "A", "A10", 5, 10)
|
||||
s.mustPutOK(c, "C", "C10", 5, 10)
|
||||
s.mustPutOK(c, "E", "E10", 5, 10)
|
||||
|
||||
checkV10 := func() {
|
||||
s.mustReverseScanOK(c, "Z", 0, 10)
|
||||
s.mustReverseScanOK(c, "Z", 1, 10, "E", "E10")
|
||||
s.mustReverseScanOK(c, "Z", 2, 10, "E", "E10", "C", "C10")
|
||||
s.mustReverseScanOK(c, "Z", 3, 10, "E", "E10", "C", "C10", "A", "A10")
|
||||
s.mustReverseScanOK(c, "Z", 4, 10, "E", "E10", "C", "C10", "A", "A10")
|
||||
s.mustReverseScanOK(c, "E\x00", 3, 10, "E", "E10", "C", "C10", "A", "A10")
|
||||
s.mustReverseScanOK(c, "C\x00", 3, 10, "C", "C10", "A", "A10")
|
||||
s.mustReverseScanOK(c, "C\x00", 4, 10, "C", "C10", "A", "A10")
|
||||
s.mustReverseScanOK(c, "B", 1, 10, "A", "A10")
|
||||
s.mustRangeReverseScanOK(c, "", "E", 5, 10, "C", "C10", "A", "A10")
|
||||
s.mustRangeReverseScanOK(c, "", "C\x00", 5, 10, "C", "C10", "A", "A10")
|
||||
s.mustRangeReverseScanOK(c, "A\x00", "C", 5, 10)
|
||||
}
|
||||
checkV10()
|
||||
|
||||
// ver20: A(10) - B(20) - C(10) - D(20) - E(10)
|
||||
s.mustPutOK(c, "B", "B20", 15, 20)
|
||||
s.mustPutOK(c, "D", "D20", 15, 20)
|
||||
|
||||
checkV20 := func() {
|
||||
s.mustReverseScanOK(c, "Z", 5, 20, "E", "E10", "D", "D20", "C", "C10", "B", "B20", "A", "A10")
|
||||
s.mustReverseScanOK(c, "C\x00", 5, 20, "C", "C10", "B", "B20", "A", "A10")
|
||||
s.mustReverseScanOK(c, "A\x00", 1, 20, "A", "A10")
|
||||
s.mustRangeReverseScanOK(c, "B", "D", 5, 20, "C", "C10", "B", "B20")
|
||||
s.mustRangeReverseScanOK(c, "B", "D\x00", 5, 20, "D", "D20", "C", "C10", "B", "B20")
|
||||
s.mustRangeReverseScanOK(c, "B\x00", "D\x00", 5, 20, "D", "D20", "C", "C10")
|
||||
}
|
||||
checkV10()
|
||||
checkV20()
|
||||
|
||||
// ver30: A(_) - B(20) - C(10) - D(_) - E(10)
|
||||
s.mustDeleteOK(c, "A", 25, 30)
|
||||
s.mustDeleteOK(c, "D", 25, 30)
|
||||
|
||||
checkV30 := func() {
|
||||
s.mustReverseScanOK(c, "Z", 5, 30, "E", "E10", "C", "C10", "B", "B20")
|
||||
s.mustReverseScanOK(c, "C", 1, 30, "B", "B20")
|
||||
s.mustReverseScanOK(c, "C\x00", 5, 30, "C", "C10", "B", "B20")
|
||||
}
|
||||
checkV10()
|
||||
checkV20()
|
||||
checkV30()
|
||||
|
||||
// ver40: A(_) - B(_) - C(40) - D(40) - E(10)
|
||||
s.mustDeleteOK(c, "B", 35, 40)
|
||||
s.mustPutOK(c, "C", "C40", 35, 40)
|
||||
s.mustPutOK(c, "D", "D40", 35, 40)
|
||||
|
||||
checkV40 := func() {
|
||||
s.mustReverseScanOK(c, "Z", 5, 40, "E", "E10", "D", "D40", "C", "C40")
|
||||
s.mustReverseScanOK(c, "Z", 5, 100, "E", "E10", "D", "D40", "C", "C40")
|
||||
}
|
||||
checkV10()
|
||||
checkV20()
|
||||
checkV30()
|
||||
checkV40()
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestScan(c *C) {
|
||||
// ver10: A(10) - B(_) - C(10) - D(_) - E(10)
|
||||
s.mustPutOK(c, "A", "A10", 5, 10)
|
||||
s.mustPutOK(c, "C", "C10", 5, 10)
|
||||
s.mustPutOK(c, "E", "E10", 5, 10)
|
||||
|
||||
checkV10 := func() {
|
||||
s.mustScanOK(c, "", 0, 10)
|
||||
s.mustScanOK(c, "", 1, 10, "A", "A10")
|
||||
s.mustScanOK(c, "", 2, 10, "A", "A10", "C", "C10")
|
||||
s.mustScanOK(c, "", 3, 10, "A", "A10", "C", "C10", "E", "E10")
|
||||
s.mustScanOK(c, "", 4, 10, "A", "A10", "C", "C10", "E", "E10")
|
||||
s.mustScanOK(c, "A", 3, 10, "A", "A10", "C", "C10", "E", "E10")
|
||||
s.mustScanOK(c, "A\x00", 3, 10, "C", "C10", "E", "E10")
|
||||
s.mustScanOK(c, "C", 4, 10, "C", "C10", "E", "E10")
|
||||
s.mustScanOK(c, "F", 1, 10)
|
||||
s.mustRangeScanOK(c, "", "E", 5, 10, "A", "A10", "C", "C10")
|
||||
s.mustRangeScanOK(c, "", "C\x00", 5, 10, "A", "A10", "C", "C10")
|
||||
s.mustRangeScanOK(c, "A\x00", "C", 5, 10)
|
||||
}
|
||||
checkV10()
|
||||
|
||||
// ver20: A(10) - B(20) - C(10) - D(20) - E(10)
|
||||
s.mustPutOK(c, "B", "B20", 15, 20)
|
||||
s.mustPutOK(c, "D", "D20", 15, 20)
|
||||
|
||||
checkV20 := func() {
|
||||
s.mustScanOK(c, "", 5, 20, "A", "A10", "B", "B20", "C", "C10", "D", "D20", "E", "E10")
|
||||
s.mustScanOK(c, "C", 5, 20, "C", "C10", "D", "D20", "E", "E10")
|
||||
s.mustScanOK(c, "D\x00", 1, 20, "E", "E10")
|
||||
s.mustRangeScanOK(c, "B", "D", 5, 20, "B", "B20", "C", "C10")
|
||||
s.mustRangeScanOK(c, "B", "D\x00", 5, 20, "B", "B20", "C", "C10", "D", "D20")
|
||||
s.mustRangeScanOK(c, "B\x00", "D\x00", 5, 20, "C", "C10", "D", "D20")
|
||||
}
|
||||
checkV10()
|
||||
checkV20()
|
||||
|
||||
// ver30: A(_) - B(20) - C(10) - D(_) - E(10)
|
||||
s.mustDeleteOK(c, "A", 25, 30)
|
||||
s.mustDeleteOK(c, "D", 25, 30)
|
||||
|
||||
checkV30 := func() {
|
||||
s.mustScanOK(c, "", 5, 30, "B", "B20", "C", "C10", "E", "E10")
|
||||
s.mustScanOK(c, "A", 1, 30, "B", "B20")
|
||||
s.mustScanOK(c, "C\x00", 5, 30, "E", "E10")
|
||||
}
|
||||
checkV10()
|
||||
checkV20()
|
||||
checkV30()
|
||||
|
||||
// ver40: A(_) - B(_) - C(40) - D(40) - E(10)
|
||||
s.mustDeleteOK(c, "B", 35, 40)
|
||||
s.mustPutOK(c, "C", "C40", 35, 40)
|
||||
s.mustPutOK(c, "D", "D40", 35, 40)
|
||||
|
||||
checkV40 := func() {
|
||||
s.mustScanOK(c, "", 5, 40, "C", "C40", "D", "D40", "E", "E10")
|
||||
s.mustScanOK(c, "", 5, 100, "C", "C40", "D", "D40", "E", "E10")
|
||||
}
|
||||
checkV10()
|
||||
checkV20()
|
||||
checkV30()
|
||||
checkV40()
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestBatchGet(c *C) {
|
||||
s.mustPutOK(c, "k1", "v1", 1, 2)
|
||||
s.mustPutOK(c, "k2", "v2", 1, 2)
|
||||
s.mustPutOK(c, "k2", "v2", 3, 4)
|
||||
s.mustPutOK(c, "k3", "v3", 1, 2)
|
||||
batchKeys := [][]byte{[]byte("k1"), []byte("k2"), []byte("k3")}
|
||||
pairs := s.store.BatchGet(batchKeys, 5, kvrpcpb.IsolationLevel_SI)
|
||||
for _, pair := range pairs {
|
||||
c.Assert(pair.Err, IsNil)
|
||||
}
|
||||
c.Assert(string(pairs[0].Value), Equals, "v1")
|
||||
c.Assert(string(pairs[1].Value), Equals, "v2")
|
||||
c.Assert(string(pairs[2].Value), Equals, "v3")
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestScanLock(c *C) {
|
||||
s.mustPutOK(c, "k1", "v1", 1, 2)
|
||||
s.mustPrewriteOK(c, putMutations("p1", "v5", "s1", "v5"), "p1", 5)
|
||||
s.mustPrewriteOK(c, putMutations("p2", "v10", "s2", "v10"), "p2", 10)
|
||||
s.mustPrewriteOK(c, putMutations("p3", "v20", "s3", "v20"), "p3", 20)
|
||||
|
||||
locks, err := s.store.ScanLock([]byte("a"), []byte("r"), 12)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(locks, DeepEquals, []*kvrpcpb.LockInfo{
|
||||
lock("p1", "p1", 5),
|
||||
lock("p2", "p2", 10),
|
||||
})
|
||||
|
||||
s.mustScanLock(c, 10, []*kvrpcpb.LockInfo{
|
||||
lock("p1", "p1", 5),
|
||||
lock("p2", "p2", 10),
|
||||
lock("s1", "p1", 5),
|
||||
lock("s2", "p2", 10),
|
||||
})
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestCommitConflict(c *C) {
|
||||
// txn A want set x to A
|
||||
// txn B want set x to B
|
||||
// A prewrite.
|
||||
s.mustPrewriteOK(c, putMutations("x", "A"), "x", 5)
|
||||
// B prewrite and find A's lock.
|
||||
errs := s.store.Prewrite(putMutations("x", "B"), []byte("x"), 10, 0)
|
||||
c.Assert(errs[0], NotNil)
|
||||
// B find rollback A because A exist too long.
|
||||
s.mustRollbackOK(c, [][]byte{[]byte("x")}, 5)
|
||||
// if A commit here, it would find its lock removed, report error txn not found.
|
||||
s.mustCommitErr(c, [][]byte{[]byte("x")}, 5, 10)
|
||||
// B prewrite itself after it rollback A.
|
||||
s.mustPrewriteOK(c, putMutations("x", "B"), "x", 10)
|
||||
// if A commit here, it would find its lock replaced by others and commit fail.
|
||||
s.mustCommitErr(c, [][]byte{[]byte("x")}, 5, 20)
|
||||
// B commit success.
|
||||
s.mustCommitOK(c, [][]byte{[]byte("x")}, 10, 20)
|
||||
// if B commit again, it will success because the key already committed.
|
||||
s.mustCommitOK(c, [][]byte{[]byte("x")}, 10, 20)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestResolveLock(c *C) {
|
||||
s.mustPrewriteOK(c, putMutations("p1", "v5", "s1", "v5"), "p1", 5)
|
||||
s.mustPrewriteOK(c, putMutations("p2", "v10", "s2", "v10"), "p2", 10)
|
||||
s.mustResolveLock(c, 5, 0)
|
||||
s.mustResolveLock(c, 10, 20)
|
||||
s.mustGetNone(c, "p1", 20)
|
||||
s.mustGetNone(c, "s1", 30)
|
||||
s.mustGetOK(c, "p2", 20, "v10")
|
||||
s.mustGetOK(c, "s2", 30, "v10")
|
||||
s.mustScanLock(c, 30, nil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestBatchResolveLock(c *C) {
|
||||
s.mustPrewriteOK(c, putMutations("p1", "v11", "s1", "v11"), "p1", 11)
|
||||
s.mustPrewriteOK(c, putMutations("p2", "v12", "s2", "v12"), "p2", 12)
|
||||
s.mustPrewriteOK(c, putMutations("p3", "v13"), "p3", 13)
|
||||
s.mustPrewriteOK(c, putMutations("p4", "v14", "s3", "v14", "s4", "v14"), "p4", 14)
|
||||
s.mustPrewriteOK(c, putMutations("p5", "v15", "s5", "v15"), "p5", 15)
|
||||
txnInfos := map[uint64]uint64{
|
||||
11: 0,
|
||||
12: 22,
|
||||
13: 0,
|
||||
14: 24,
|
||||
}
|
||||
s.mustBatchResolveLock(c, txnInfos)
|
||||
s.mustGetNone(c, "p1", 20)
|
||||
s.mustGetNone(c, "p3", 30)
|
||||
s.mustGetOK(c, "p2", 30, "v12")
|
||||
s.mustGetOK(c, "s4", 30, "v14")
|
||||
s.mustScanLock(c, 30, []*kvrpcpb.LockInfo{
|
||||
lock("p5", "p5", 15),
|
||||
lock("s5", "p5", 15),
|
||||
})
|
||||
txnInfos = map[uint64]uint64{
|
||||
15: 0,
|
||||
}
|
||||
s.mustBatchResolveLock(c, txnInfos)
|
||||
s.mustScanLock(c, 30, nil)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestRollbackAndWriteConflict(c *C) {
|
||||
s.mustPutOK(c, "test", "test", 1, 3)
|
||||
|
||||
errs := s.store.Prewrite(putMutations("lock", "lock", "test", "test1"), []byte("test"), 2, 2)
|
||||
s.mustWriteWriteConflict(c, errs, 1)
|
||||
|
||||
s.mustPutOK(c, "test", "test2", 5, 8)
|
||||
|
||||
// simulate `getTxnStatus` for txn 2.
|
||||
err := s.store.Cleanup([]byte("test"), 2)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
errs = s.store.Prewrite(putMutations("test", "test3"), []byte("test"), 6, 1)
|
||||
s.mustWriteWriteConflict(c, errs, 0)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestDeleteRange(c *C) {
|
||||
for i := 1; i <= 5; i++ {
|
||||
key := string(byte(i) + byte('0'))
|
||||
value := "v" + key
|
||||
s.mustPutOK(c, key, value, uint64(1+2*i), uint64(2+2*i))
|
||||
}
|
||||
|
||||
s.mustScanOK(c, "0", 10, 20, "1", "v1", "2", "v2", "3", "v3", "4", "v4", "5", "v5")
|
||||
|
||||
s.mustDeleteRange(c, "2", "4")
|
||||
s.mustScanOK(c, "0", 10, 30, "1", "v1", "4", "v4", "5", "v5")
|
||||
|
||||
s.mustDeleteRange(c, "5", "5")
|
||||
s.mustScanOK(c, "0", 10, 40, "1", "v1", "4", "v4", "5", "v5")
|
||||
|
||||
s.mustDeleteRange(c, "41", "42")
|
||||
s.mustScanOK(c, "0", 10, 50, "1", "v1", "4", "v4", "5", "v5")
|
||||
|
||||
s.mustDeleteRange(c, "4\x00", "5\x00")
|
||||
s.mustScanOK(c, "0", 10, 60, "1", "v1", "4", "v4")
|
||||
|
||||
s.mustDeleteRange(c, "0", "9")
|
||||
s.mustScanOK(c, "0", 10, 70)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) mustWriteWriteConflict(c *C, errs []error, i int) {
|
||||
c.Assert(errs[i], NotNil)
|
||||
c.Assert(strings.Contains(errs[i].Error(), "write conflict"), IsTrue)
|
||||
}
|
||||
|
||||
func (s *testMockTiKVSuite) TestRC(c *C) {
|
||||
s.mustPutOK(c, "key", "v1", 5, 10)
|
||||
s.mustPrewriteOK(c, putMutations("key", "v2"), "key", 15)
|
||||
s.mustGetErr(c, "key", 20)
|
||||
s.mustGetRC(c, "key", 12, "v1")
|
||||
s.mustGetRC(c, "key", 20, "v1")
|
||||
}
|
||||
|
||||
func (s testMarshal) TestMarshalmvccLock(c *C) {
|
||||
l := mvccLock{
|
||||
startTS: 47,
|
||||
primary: []byte{'a', 'b', 'c'},
|
||||
value: []byte{'d', 'e'},
|
||||
op: kvrpcpb.Op_Put,
|
||||
ttl: 444,
|
||||
}
|
||||
bin, err := l.MarshalBinary()
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
var l1 mvccLock
|
||||
err = l1.UnmarshalBinary(bin)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
c.Assert(l.startTS, Equals, l1.startTS)
|
||||
c.Assert(l.op, Equals, l1.op)
|
||||
c.Assert(l.ttl, Equals, l1.ttl)
|
||||
c.Assert(string(l.primary), Equals, string(l1.primary))
|
||||
c.Assert(string(l.value), Equals, string(l1.value))
|
||||
}
|
||||
|
||||
func (s testMarshal) TestMarshalmvccValue(c *C) {
|
||||
v := mvccValue{
|
||||
valueType: typePut,
|
||||
startTS: 42,
|
||||
commitTS: 55,
|
||||
value: []byte{'d', 'e'},
|
||||
}
|
||||
bin, err := v.MarshalBinary()
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
var v1 mvccValue
|
||||
err = v1.UnmarshalBinary(bin)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
c.Assert(v.valueType, Equals, v1.valueType)
|
||||
c.Assert(v.startTS, Equals, v1.startTS)
|
||||
c.Assert(v.commitTS, Equals, v1.commitTS)
|
||||
c.Assert(string(v.value), Equals, string(v.value))
|
||||
}
|
||||
|
|
@ -1,489 +0,0 @@
|
|||
// Copyright 2016 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 mocktikv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"io"
|
||||
"math"
|
||||
"sort"
|
||||
|
||||
"github.com/google/btree"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/codec"
|
||||
)
|
||||
|
||||
type mvccValueType int
|
||||
|
||||
const (
|
||||
typePut mvccValueType = iota
|
||||
typeDelete
|
||||
typeRollback
|
||||
)
|
||||
|
||||
type mvccValue struct {
|
||||
valueType mvccValueType
|
||||
startTS uint64
|
||||
commitTS uint64
|
||||
value []byte
|
||||
}
|
||||
|
||||
type mvccLock struct {
|
||||
startTS uint64
|
||||
primary []byte
|
||||
value []byte
|
||||
op kvrpcpb.Op
|
||||
ttl uint64
|
||||
}
|
||||
|
||||
type mvccEntry struct {
|
||||
key MvccKey
|
||||
values []mvccValue
|
||||
lock *mvccLock
|
||||
}
|
||||
|
||||
// MarshalBinary implements encoding.BinaryMarshaler interface.
|
||||
func (l *mvccLock) MarshalBinary() ([]byte, error) {
|
||||
var (
|
||||
mh marshalHelper
|
||||
buf bytes.Buffer
|
||||
)
|
||||
mh.WriteNumber(&buf, l.startTS)
|
||||
mh.WriteSlice(&buf, l.primary)
|
||||
mh.WriteSlice(&buf, l.value)
|
||||
mh.WriteNumber(&buf, l.op)
|
||||
mh.WriteNumber(&buf, l.ttl)
|
||||
return buf.Bytes(), mh.err
|
||||
}
|
||||
|
||||
// UnmarshalBinary implements encoding.BinaryUnmarshaler interface.
|
||||
func (l *mvccLock) UnmarshalBinary(data []byte) error {
|
||||
var mh marshalHelper
|
||||
buf := bytes.NewBuffer(data)
|
||||
mh.ReadNumber(buf, &l.startTS)
|
||||
mh.ReadSlice(buf, &l.primary)
|
||||
mh.ReadSlice(buf, &l.value)
|
||||
mh.ReadNumber(buf, &l.op)
|
||||
mh.ReadNumber(buf, &l.ttl)
|
||||
return mh.err
|
||||
}
|
||||
|
||||
// MarshalBinary implements encoding.BinaryMarshaler interface.
|
||||
func (v mvccValue) MarshalBinary() ([]byte, error) {
|
||||
var (
|
||||
mh marshalHelper
|
||||
buf bytes.Buffer
|
||||
)
|
||||
mh.WriteNumber(&buf, int64(v.valueType))
|
||||
mh.WriteNumber(&buf, v.startTS)
|
||||
mh.WriteNumber(&buf, v.commitTS)
|
||||
mh.WriteSlice(&buf, v.value)
|
||||
return buf.Bytes(), mh.err
|
||||
}
|
||||
|
||||
// UnmarshalBinary implements encoding.BinaryUnmarshaler interface.
|
||||
func (v *mvccValue) UnmarshalBinary(data []byte) error {
|
||||
var mh marshalHelper
|
||||
buf := bytes.NewBuffer(data)
|
||||
var vt int64
|
||||
mh.ReadNumber(buf, &vt)
|
||||
v.valueType = mvccValueType(vt)
|
||||
mh.ReadNumber(buf, &v.startTS)
|
||||
mh.ReadNumber(buf, &v.commitTS)
|
||||
mh.ReadSlice(buf, &v.value)
|
||||
return mh.err
|
||||
}
|
||||
|
||||
type marshalHelper struct {
|
||||
err error
|
||||
}
|
||||
|
||||
func (mh *marshalHelper) WriteSlice(buf io.Writer, slice []byte) {
|
||||
if mh.err != nil {
|
||||
return
|
||||
}
|
||||
var tmp [binary.MaxVarintLen64]byte
|
||||
off := binary.PutUvarint(tmp[:], uint64(len(slice)))
|
||||
if err := writeFull(buf, tmp[:off]); err != nil {
|
||||
mh.err = err
|
||||
return
|
||||
}
|
||||
if err := writeFull(buf, slice); err != nil {
|
||||
mh.err = err
|
||||
}
|
||||
}
|
||||
|
||||
func (mh *marshalHelper) WriteNumber(buf io.Writer, n interface{}) {
|
||||
if mh.err != nil {
|
||||
return
|
||||
}
|
||||
err := binary.Write(buf, binary.LittleEndian, n)
|
||||
if err != nil {
|
||||
mh.err = errors.WithStack(err)
|
||||
}
|
||||
}
|
||||
|
||||
func writeFull(w io.Writer, slice []byte) error {
|
||||
written := 0
|
||||
for written < len(slice) {
|
||||
n, err := w.Write(slice[written:])
|
||||
if err != nil {
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
written += n
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mh *marshalHelper) ReadNumber(r io.Reader, n interface{}) {
|
||||
if mh.err != nil {
|
||||
return
|
||||
}
|
||||
err := binary.Read(r, binary.LittleEndian, n)
|
||||
if err != nil {
|
||||
mh.err = errors.WithStack(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (mh *marshalHelper) ReadSlice(r *bytes.Buffer, slice *[]byte) {
|
||||
if mh.err != nil {
|
||||
return
|
||||
}
|
||||
sz, err := binary.ReadUvarint(r)
|
||||
if err != nil {
|
||||
mh.err = errors.WithStack(err)
|
||||
return
|
||||
}
|
||||
const c10M = 10 * 1024 * 1024
|
||||
if sz > c10M {
|
||||
mh.err = errors.New("too large slice, maybe something wrong")
|
||||
return
|
||||
}
|
||||
data := make([]byte, sz)
|
||||
if _, err := io.ReadFull(r, data); err != nil {
|
||||
mh.err = errors.WithStack(err)
|
||||
return
|
||||
}
|
||||
*slice = data
|
||||
}
|
||||
|
||||
func newEntry(key MvccKey) *mvccEntry {
|
||||
return &mvccEntry{
|
||||
key: key,
|
||||
}
|
||||
}
|
||||
|
||||
// lockErr returns ErrLocked.
|
||||
// Note that parameter key is raw key, while key in ErrLocked is mvcc key.
|
||||
func (l *mvccLock) lockErr(key []byte) error {
|
||||
return &ErrLocked{
|
||||
Key: mvccEncode(key, lockVer),
|
||||
Primary: l.primary,
|
||||
StartTS: l.startTS,
|
||||
TTL: l.ttl,
|
||||
}
|
||||
}
|
||||
|
||||
func (l *mvccLock) check(ts uint64, key []byte) (uint64, error) {
|
||||
// ignore when ts is older than lock or lock's type is Lock.
|
||||
if l.startTS > ts || l.op == kvrpcpb.Op_Lock {
|
||||
return ts, nil
|
||||
}
|
||||
// for point get latest version.
|
||||
if ts == math.MaxUint64 && bytes.Equal(l.primary, key) {
|
||||
return l.startTS - 1, nil
|
||||
}
|
||||
return 0, l.lockErr(key)
|
||||
}
|
||||
|
||||
func (e *mvccEntry) Clone() *mvccEntry {
|
||||
var entry mvccEntry
|
||||
entry.key = append([]byte(nil), e.key...)
|
||||
for _, v := range e.values {
|
||||
entry.values = append(entry.values, mvccValue{
|
||||
valueType: v.valueType,
|
||||
startTS: v.startTS,
|
||||
commitTS: v.commitTS,
|
||||
value: append([]byte(nil), v.value...),
|
||||
})
|
||||
}
|
||||
if e.lock != nil {
|
||||
entry.lock = &mvccLock{
|
||||
startTS: e.lock.startTS,
|
||||
primary: append([]byte(nil), e.lock.primary...),
|
||||
value: append([]byte(nil), e.lock.value...),
|
||||
op: e.lock.op,
|
||||
ttl: e.lock.ttl,
|
||||
}
|
||||
}
|
||||
return &entry
|
||||
}
|
||||
|
||||
func (e *mvccEntry) Less(than btree.Item) bool {
|
||||
return bytes.Compare(e.key, than.(*mvccEntry).key) < 0
|
||||
}
|
||||
|
||||
func (e *mvccEntry) Get(ts uint64, isoLevel kvrpcpb.IsolationLevel) ([]byte, error) {
|
||||
if isoLevel == kvrpcpb.IsolationLevel_SI && e.lock != nil {
|
||||
var err error
|
||||
ts, err = e.lock.check(ts, e.key.Raw())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
for _, v := range e.values {
|
||||
if v.commitTS <= ts && v.valueType != typeRollback {
|
||||
return v.value, nil
|
||||
}
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (e *mvccEntry) Prewrite(mutation *kvrpcpb.Mutation, startTS uint64, primary []byte, ttl uint64) error {
|
||||
if len(e.values) > 0 {
|
||||
if e.values[0].commitTS >= startTS {
|
||||
return ErrRetryable("write conflict")
|
||||
}
|
||||
}
|
||||
if e.lock != nil {
|
||||
if e.lock.startTS != startTS {
|
||||
return e.lock.lockErr(e.key.Raw())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
e.lock = &mvccLock{
|
||||
startTS: startTS,
|
||||
primary: primary,
|
||||
value: mutation.Value,
|
||||
op: mutation.GetOp(),
|
||||
ttl: ttl,
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *mvccEntry) getTxnCommitInfo(startTS uint64) *mvccValue {
|
||||
for _, v := range e.values {
|
||||
if v.startTS == startTS {
|
||||
return &v
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *mvccEntry) Commit(startTS, commitTS uint64) error {
|
||||
if e.lock == nil || e.lock.startTS != startTS {
|
||||
if c := e.getTxnCommitInfo(startTS); c != nil && c.valueType != typeRollback {
|
||||
return nil
|
||||
}
|
||||
return ErrRetryable("txn not found")
|
||||
}
|
||||
if e.lock.op != kvrpcpb.Op_Lock {
|
||||
var valueType mvccValueType
|
||||
if e.lock.op == kvrpcpb.Op_Put {
|
||||
valueType = typePut
|
||||
} else {
|
||||
valueType = typeDelete
|
||||
}
|
||||
e.addValue(mvccValue{
|
||||
valueType: valueType,
|
||||
startTS: startTS,
|
||||
commitTS: commitTS,
|
||||
value: e.lock.value,
|
||||
})
|
||||
}
|
||||
e.lock = nil
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *mvccEntry) Rollback(startTS uint64) error {
|
||||
// If current transaction's lock exist.
|
||||
if e.lock != nil && e.lock.startTS == startTS {
|
||||
e.lock = nil
|
||||
e.addValue(mvccValue{
|
||||
valueType: typeRollback,
|
||||
startTS: startTS,
|
||||
commitTS: startTS,
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
// If current transaction's lock not exist.
|
||||
// If commit info of current transaction exist.
|
||||
if c := e.getTxnCommitInfo(startTS); c != nil {
|
||||
// If current transaction is already committed.
|
||||
if c.valueType != typeRollback {
|
||||
return ErrAlreadyCommitted(c.commitTS)
|
||||
}
|
||||
// If current transaction is already rollback.
|
||||
return nil
|
||||
}
|
||||
// If current transaction is not prewritted before.
|
||||
e.addValue(mvccValue{
|
||||
valueType: typeRollback,
|
||||
startTS: startTS,
|
||||
commitTS: startTS,
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *mvccEntry) addValue(v mvccValue) {
|
||||
i := sort.Search(len(e.values), func(i int) bool { return e.values[i].commitTS <= v.commitTS })
|
||||
if i >= len(e.values) {
|
||||
e.values = append(e.values, v)
|
||||
} else {
|
||||
e.values = append(e.values[:i+1], e.values[i:]...)
|
||||
e.values[i] = v
|
||||
}
|
||||
}
|
||||
|
||||
func (e *mvccEntry) containsStartTS(startTS uint64) bool {
|
||||
if e.lock != nil && e.lock.startTS == startTS {
|
||||
return true
|
||||
}
|
||||
for _, item := range e.values {
|
||||
if item.startTS == startTS {
|
||||
return true
|
||||
}
|
||||
if item.commitTS < startTS {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (e *mvccEntry) dumpMvccInfo() *kvrpcpb.MvccInfo {
|
||||
info := &kvrpcpb.MvccInfo{}
|
||||
if e.lock != nil {
|
||||
info.Lock = &kvrpcpb.MvccLock{
|
||||
Type: e.lock.op,
|
||||
StartTs: e.lock.startTS,
|
||||
Primary: e.lock.primary,
|
||||
ShortValue: e.lock.value,
|
||||
}
|
||||
}
|
||||
|
||||
info.Writes = make([]*kvrpcpb.MvccWrite, len(e.values))
|
||||
info.Values = make([]*kvrpcpb.MvccValue, len(e.values))
|
||||
|
||||
for id, item := range e.values {
|
||||
var tp kvrpcpb.Op
|
||||
switch item.valueType {
|
||||
case typePut:
|
||||
tp = kvrpcpb.Op_Put
|
||||
case typeDelete:
|
||||
tp = kvrpcpb.Op_Del
|
||||
case typeRollback:
|
||||
tp = kvrpcpb.Op_Rollback
|
||||
}
|
||||
info.Writes[id] = &kvrpcpb.MvccWrite{
|
||||
Type: tp,
|
||||
StartTs: item.startTS,
|
||||
CommitTs: item.commitTS,
|
||||
}
|
||||
|
||||
info.Values[id] = &kvrpcpb.MvccValue{
|
||||
Value: item.value,
|
||||
StartTs: item.startTS,
|
||||
}
|
||||
}
|
||||
return info
|
||||
}
|
||||
|
||||
type rawEntry struct {
|
||||
key []byte
|
||||
value []byte
|
||||
}
|
||||
|
||||
func newRawEntry(key []byte) *rawEntry {
|
||||
return &rawEntry{
|
||||
key: key,
|
||||
}
|
||||
}
|
||||
|
||||
func (e *rawEntry) Less(than btree.Item) bool {
|
||||
return bytes.Compare(e.key, than.(*rawEntry).key) < 0
|
||||
}
|
||||
|
||||
// MVCCStore is a mvcc key-value storage.
|
||||
type MVCCStore interface {
|
||||
Get(key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel) ([]byte, error)
|
||||
Scan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair
|
||||
ReverseScan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair
|
||||
BatchGet(ks [][]byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair
|
||||
Prewrite(mutations []*kvrpcpb.Mutation, primary []byte, startTS uint64, ttl uint64) []error
|
||||
Commit(keys [][]byte, startTS, commitTS uint64) error
|
||||
Rollback(keys [][]byte, startTS uint64) error
|
||||
Cleanup(key []byte, startTS uint64) error
|
||||
ScanLock(startKey, endKey []byte, maxTS uint64) ([]*kvrpcpb.LockInfo, error)
|
||||
ResolveLock(startKey, endKey []byte, startTS, commitTS uint64) error
|
||||
BatchResolveLock(startKey, endKey []byte, txnInfos map[uint64]uint64) error
|
||||
DeleteRange(startKey, endKey []byte) error
|
||||
Close() error
|
||||
}
|
||||
|
||||
// RawKV is a key-value storage. MVCCStore can be implemented upon it with timestamp encoded into key.
|
||||
type RawKV interface {
|
||||
RawGet(key []byte) []byte
|
||||
RawBatchGet(keys [][]byte) [][]byte
|
||||
RawScan(startKey, endKey []byte, limit int) []Pair
|
||||
RawPut(key, value []byte)
|
||||
RawBatchPut(keys, values [][]byte)
|
||||
RawDelete(key []byte)
|
||||
RawBatchDelete(keys [][]byte)
|
||||
RawDeleteRange(startKey, endKey []byte)
|
||||
}
|
||||
|
||||
// MVCCDebugger is for debugging.
|
||||
type MVCCDebugger interface {
|
||||
MvccGetByStartTS(startKey, endKey []byte, starTS uint64) (*kvrpcpb.MvccInfo, []byte)
|
||||
MvccGetByKey(key []byte) *kvrpcpb.MvccInfo
|
||||
}
|
||||
|
||||
// Pair is a KV pair read from MvccStore or an error if any occurs.
|
||||
type Pair struct {
|
||||
Key []byte
|
||||
Value []byte
|
||||
Err error
|
||||
}
|
||||
|
||||
func regionContains(startKey []byte, endKey []byte, key []byte) bool {
|
||||
return bytes.Compare(startKey, key) <= 0 &&
|
||||
(bytes.Compare(key, endKey) < 0 || len(endKey) == 0)
|
||||
}
|
||||
|
||||
// MvccKey is the encoded key type.
|
||||
// On TiKV, keys are encoded before they are saved into storage engine.
|
||||
type MvccKey []byte
|
||||
|
||||
// NewMvccKey encodes a key into MvccKey.
|
||||
func NewMvccKey(key []byte) MvccKey {
|
||||
if len(key) == 0 {
|
||||
return nil
|
||||
}
|
||||
return codec.EncodeBytes(key)
|
||||
}
|
||||
|
||||
// Raw decodes a MvccKey to original key.
|
||||
func (key MvccKey) Raw() []byte {
|
||||
if len(key) == 0 {
|
||||
return nil
|
||||
}
|
||||
_, k, err := codec.DecodeBytes(key)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return k
|
||||
}
|
||||
|
|
@ -1,996 +0,0 @@
|
|||
// Copyright 2017 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 mocktikv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"math"
|
||||
"sync"
|
||||
|
||||
"github.com/pingcap/goleveldb/leveldb"
|
||||
"github.com/pingcap/goleveldb/leveldb/iterator"
|
||||
"github.com/pingcap/goleveldb/leveldb/opt"
|
||||
"github.com/pingcap/goleveldb/leveldb/storage"
|
||||
"github.com/pingcap/goleveldb/leveldb/util"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/codec"
|
||||
)
|
||||
|
||||
// MVCCLevelDB implements the MVCCStore interface.
|
||||
type MVCCLevelDB struct {
|
||||
// Key layout:
|
||||
// ...
|
||||
// Key_lock -- (0)
|
||||
// Key_verMax -- (1)
|
||||
// ...
|
||||
// Key_ver+1 -- (2)
|
||||
// Key_ver -- (3)
|
||||
// Key_ver-1 -- (4)
|
||||
// ...
|
||||
// Key_0 -- (5)
|
||||
// NextKey_lock -- (6)
|
||||
// NextKey_verMax -- (7)
|
||||
// ...
|
||||
// NextKey_ver+1 -- (8)
|
||||
// NextKey_ver -- (9)
|
||||
// NextKey_ver-1 -- (10)
|
||||
// ...
|
||||
// NextKey_0 -- (11)
|
||||
// ...
|
||||
// EOF
|
||||
db *leveldb.DB
|
||||
// leveldb can not guarantee multiple operations to be atomic, for example, read
|
||||
// then write, another write may happen during it, so this lock is necessory.
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
const lockVer uint64 = math.MaxUint64
|
||||
|
||||
// ErrInvalidEncodedKey describes parsing an invalid format of EncodedKey.
|
||||
var ErrInvalidEncodedKey = errors.New("invalid encoded key")
|
||||
|
||||
// mvccEncode returns the encoded key.
|
||||
func mvccEncode(key []byte, ver uint64) []byte {
|
||||
b := codec.EncodeBytes(key)
|
||||
ret := codec.EncodeUintDesc(b, ver)
|
||||
return ret
|
||||
}
|
||||
|
||||
// mvccDecode parses the origin key and version of an encoded key, if the encoded key is a meta key,
|
||||
// just returns the origin key.
|
||||
func mvccDecode(encodedKey []byte) ([]byte, uint64, error) {
|
||||
// Skip DataPrefix
|
||||
remainBytes, key, err := codec.DecodeBytes(encodedKey)
|
||||
if err != nil {
|
||||
// should never happen
|
||||
return nil, 0, err
|
||||
}
|
||||
// if it's meta key
|
||||
if len(remainBytes) == 0 {
|
||||
return key, 0, nil
|
||||
}
|
||||
var ver uint64
|
||||
remainBytes, ver, err = codec.DecodeUintDesc(remainBytes)
|
||||
if err != nil {
|
||||
// should never happen
|
||||
return nil, 0, err
|
||||
}
|
||||
if len(remainBytes) != 0 {
|
||||
return nil, 0, ErrInvalidEncodedKey
|
||||
}
|
||||
return key, ver, nil
|
||||
}
|
||||
|
||||
// MustNewMVCCStore is used for testing, use NewMVCCLevelDB instead.
|
||||
func MustNewMVCCStore() MVCCStore {
|
||||
mvccStore, err := NewMVCCLevelDB("")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return mvccStore
|
||||
}
|
||||
|
||||
// NewMVCCLevelDB returns a new MVCCLevelDB object.
|
||||
func NewMVCCLevelDB(path string) (*MVCCLevelDB, error) {
|
||||
var (
|
||||
d *leveldb.DB
|
||||
err error
|
||||
)
|
||||
if path == "" {
|
||||
d, err = leveldb.Open(storage.NewMemStorage(), nil)
|
||||
} else {
|
||||
d, err = leveldb.OpenFile(path, &opt.Options{BlockCacheCapacity: 600 * 1024 * 1024})
|
||||
}
|
||||
|
||||
return &MVCCLevelDB{db: d}, errors.WithStack(err)
|
||||
}
|
||||
|
||||
// Iterator wraps iterator.Iterator to provide Valid() method.
|
||||
type Iterator struct {
|
||||
iterator.Iterator
|
||||
valid bool
|
||||
}
|
||||
|
||||
// Next moves the iterator to the next key/value pair.
|
||||
func (iter *Iterator) Next() {
|
||||
iter.valid = iter.Iterator.Next()
|
||||
}
|
||||
|
||||
// Valid returns whether the iterator is exhausted.
|
||||
func (iter *Iterator) Valid() bool {
|
||||
return iter.valid
|
||||
}
|
||||
|
||||
func newIterator(db *leveldb.DB, slice *util.Range) *Iterator {
|
||||
iter := &Iterator{db.NewIterator(slice, nil), true}
|
||||
iter.Next()
|
||||
return iter
|
||||
}
|
||||
|
||||
func newScanIterator(db *leveldb.DB, startKey, endKey []byte) (*Iterator, []byte, error) {
|
||||
var start, end []byte
|
||||
if len(startKey) > 0 {
|
||||
start = mvccEncode(startKey, lockVer)
|
||||
}
|
||||
if len(endKey) > 0 {
|
||||
end = mvccEncode(endKey, lockVer)
|
||||
}
|
||||
iter := newIterator(db, &util.Range{
|
||||
Start: start,
|
||||
Limit: end,
|
||||
})
|
||||
// newScanIterator must handle startKey is nil, in this case, the real startKey
|
||||
// should be change the frist key of the store.
|
||||
if len(startKey) == 0 && iter.Valid() {
|
||||
key, _, err := mvccDecode(iter.Key())
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
startKey = key
|
||||
}
|
||||
return iter, startKey, nil
|
||||
}
|
||||
|
||||
// iterDecoder tries to decode an Iterator value.
|
||||
// If current iterator value can be decoded by this decoder, store the value and call iter.Next(),
|
||||
// Otherwise current iterator is not touched and returns false.
|
||||
type iterDecoder interface {
|
||||
Decode(iter *Iterator) (bool, error)
|
||||
}
|
||||
|
||||
type lockDecoder struct {
|
||||
lock mvccLock
|
||||
expectKey []byte
|
||||
}
|
||||
|
||||
// lockDecoder decodes the lock value if current iterator is at expectKey::lock.
|
||||
func (dec *lockDecoder) Decode(iter *Iterator) (bool, error) {
|
||||
if iter.Error() != nil || !iter.Valid() {
|
||||
return false, iter.Error()
|
||||
}
|
||||
|
||||
iterKey := iter.Key()
|
||||
key, ver, err := mvccDecode(iterKey)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if !bytes.Equal(key, dec.expectKey) {
|
||||
return false, nil
|
||||
}
|
||||
if ver != lockVer {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
var lock mvccLock
|
||||
err = lock.UnmarshalBinary(iter.Value())
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
dec.lock = lock
|
||||
iter.Next()
|
||||
return true, nil
|
||||
}
|
||||
|
||||
type valueDecoder struct {
|
||||
value mvccValue
|
||||
expectKey []byte
|
||||
}
|
||||
|
||||
// valueDecoder decodes a mvcc value if iter key is expectKey.
|
||||
func (dec *valueDecoder) Decode(iter *Iterator) (bool, error) {
|
||||
if iter.Error() != nil || !iter.Valid() {
|
||||
return false, iter.Error()
|
||||
}
|
||||
|
||||
key, ver, err := mvccDecode(iter.Key())
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if !bytes.Equal(key, dec.expectKey) {
|
||||
return false, nil
|
||||
}
|
||||
if ver == lockVer {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
var value mvccValue
|
||||
err = value.UnmarshalBinary(iter.Value())
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
dec.value = value
|
||||
iter.Next()
|
||||
return true, nil
|
||||
}
|
||||
|
||||
type skipDecoder struct {
|
||||
currKey []byte
|
||||
}
|
||||
|
||||
// skipDecoder skips the iterator as long as its key is currKey, the new key would be stored.
|
||||
func (dec *skipDecoder) Decode(iter *Iterator) (bool, error) {
|
||||
if iter.Error() != nil {
|
||||
return false, iter.Error()
|
||||
}
|
||||
for iter.Valid() {
|
||||
key, _, err := mvccDecode(iter.Key())
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if !bytes.Equal(key, dec.currKey) {
|
||||
dec.currKey = key
|
||||
return true, nil
|
||||
}
|
||||
iter.Next()
|
||||
}
|
||||
return false, nil
|
||||
}
|
||||
|
||||
type mvccEntryDecoder struct {
|
||||
expectKey []byte
|
||||
// Just values and lock is valid.
|
||||
mvccEntry
|
||||
}
|
||||
|
||||
// mvccEntryDecoder decodes a mvcc entry.
|
||||
func (dec *mvccEntryDecoder) Decode(iter *Iterator) (bool, error) {
|
||||
ldec := lockDecoder{expectKey: dec.expectKey}
|
||||
ok, err := ldec.Decode(iter)
|
||||
if err != nil {
|
||||
return ok, err
|
||||
}
|
||||
if ok {
|
||||
dec.mvccEntry.lock = &ldec.lock
|
||||
}
|
||||
for iter.Valid() {
|
||||
vdec := valueDecoder{expectKey: dec.expectKey}
|
||||
ok, err = vdec.Decode(iter)
|
||||
if err != nil {
|
||||
return ok, err
|
||||
}
|
||||
if !ok {
|
||||
break
|
||||
}
|
||||
dec.mvccEntry.values = append(dec.mvccEntry.values, vdec.value)
|
||||
}
|
||||
succ := dec.mvccEntry.lock != nil || len(dec.mvccEntry.values) > 0
|
||||
return succ, nil
|
||||
}
|
||||
|
||||
// Get implements the MVCCStore interface.
|
||||
// key cannot be nil or []byte{}
|
||||
func (mvcc *MVCCLevelDB) Get(key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel) ([]byte, error) {
|
||||
mvcc.mu.RLock()
|
||||
defer mvcc.mu.RUnlock()
|
||||
|
||||
return mvcc.getValue(key, startTS, isoLevel)
|
||||
}
|
||||
|
||||
func (mvcc *MVCCLevelDB) getValue(key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel) ([]byte, error) {
|
||||
startKey := mvccEncode(key, lockVer)
|
||||
iter := newIterator(mvcc.db, &util.Range{
|
||||
Start: startKey,
|
||||
})
|
||||
defer iter.Release()
|
||||
|
||||
return getValue(iter, key, startTS, isoLevel)
|
||||
}
|
||||
|
||||
func getValue(iter *Iterator, key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel) ([]byte, error) {
|
||||
dec1 := lockDecoder{expectKey: key}
|
||||
ok, err := dec1.Decode(iter)
|
||||
if ok && isoLevel == kvrpcpb.IsolationLevel_SI {
|
||||
startTS, err = dec1.lock.check(startTS, key)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
dec2 := valueDecoder{expectKey: key}
|
||||
for iter.Valid() {
|
||||
ok, err := dec2.Decode(iter)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if !ok {
|
||||
break
|
||||
}
|
||||
|
||||
value := &dec2.value
|
||||
if value.valueType == typeRollback {
|
||||
continue
|
||||
}
|
||||
// Read the first committed value that can be seen at startTS.
|
||||
if value.commitTS <= startTS {
|
||||
if value.valueType == typeDelete {
|
||||
return nil, nil
|
||||
}
|
||||
return value.value, nil
|
||||
}
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// BatchGet implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) BatchGet(ks [][]byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair {
|
||||
mvcc.mu.RLock()
|
||||
defer mvcc.mu.RUnlock()
|
||||
|
||||
var pairs []Pair
|
||||
for _, k := range ks {
|
||||
v, err := mvcc.getValue(k, startTS, isoLevel)
|
||||
if v == nil && err == nil {
|
||||
continue
|
||||
}
|
||||
pairs = append(pairs, Pair{
|
||||
Key: k,
|
||||
Value: v,
|
||||
Err: err,
|
||||
})
|
||||
}
|
||||
return pairs
|
||||
}
|
||||
|
||||
// Scan implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) Scan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair {
|
||||
mvcc.mu.RLock()
|
||||
defer mvcc.mu.RUnlock()
|
||||
|
||||
iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey)
|
||||
defer iter.Release()
|
||||
if err != nil {
|
||||
log.Error("scan new iterator fail:", err)
|
||||
return nil
|
||||
}
|
||||
|
||||
ok := true
|
||||
var pairs []Pair
|
||||
for len(pairs) < limit && ok {
|
||||
value, err := getValue(iter, currKey, startTS, isoLevel)
|
||||
if err != nil {
|
||||
pairs = append(pairs, Pair{
|
||||
Key: currKey,
|
||||
Err: err,
|
||||
})
|
||||
}
|
||||
if value != nil {
|
||||
pairs = append(pairs, Pair{
|
||||
Key: currKey,
|
||||
Value: value,
|
||||
})
|
||||
}
|
||||
|
||||
skip := skipDecoder{currKey}
|
||||
ok, err = skip.Decode(iter)
|
||||
if err != nil {
|
||||
log.Error("seek to next key error:", err)
|
||||
break
|
||||
}
|
||||
currKey = skip.currKey
|
||||
}
|
||||
return pairs
|
||||
}
|
||||
|
||||
// ReverseScan implements the MVCCStore interface. The search range is [startKey, endKey).
|
||||
func (mvcc *MVCCLevelDB) ReverseScan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair {
|
||||
mvcc.mu.RLock()
|
||||
defer mvcc.mu.RUnlock()
|
||||
|
||||
var mvccEnd []byte
|
||||
if len(endKey) != 0 {
|
||||
mvccEnd = mvccEncode(endKey, lockVer)
|
||||
}
|
||||
iter := mvcc.db.NewIterator(&util.Range{
|
||||
Limit: mvccEnd,
|
||||
}, nil)
|
||||
defer iter.Release()
|
||||
|
||||
succ := iter.Last()
|
||||
currKey, _, err := mvccDecode(iter.Key())
|
||||
// TODO: return error.
|
||||
log.Error(err)
|
||||
helper := reverseScanHelper{
|
||||
startTS: startTS,
|
||||
isoLevel: isoLevel,
|
||||
currKey: currKey,
|
||||
}
|
||||
|
||||
for succ && len(helper.pairs) < limit {
|
||||
key, ver, err := mvccDecode(iter.Key())
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
if bytes.Compare(key, startKey) < 0 {
|
||||
break
|
||||
}
|
||||
|
||||
if !bytes.Equal(key, helper.currKey) {
|
||||
helper.finishEntry()
|
||||
helper.currKey = key
|
||||
}
|
||||
if ver == lockVer {
|
||||
var lock mvccLock
|
||||
err = lock.UnmarshalBinary(iter.Value())
|
||||
helper.entry.lock = &lock
|
||||
} else {
|
||||
var value mvccValue
|
||||
err = value.UnmarshalBinary(iter.Value())
|
||||
helper.entry.values = append(helper.entry.values, value)
|
||||
}
|
||||
if err != nil {
|
||||
log.Error("Unmarshal fail:", err)
|
||||
break
|
||||
}
|
||||
succ = iter.Prev()
|
||||
}
|
||||
if len(helper.pairs) < limit {
|
||||
helper.finishEntry()
|
||||
}
|
||||
return helper.pairs
|
||||
}
|
||||
|
||||
type reverseScanHelper struct {
|
||||
startTS uint64
|
||||
isoLevel kvrpcpb.IsolationLevel
|
||||
currKey []byte
|
||||
entry mvccEntry
|
||||
pairs []Pair
|
||||
}
|
||||
|
||||
func (helper *reverseScanHelper) finishEntry() {
|
||||
reverse(helper.entry.values)
|
||||
helper.entry.key = NewMvccKey(helper.currKey)
|
||||
val, err := helper.entry.Get(helper.startTS, helper.isoLevel)
|
||||
if len(val) != 0 || err != nil {
|
||||
helper.pairs = append(helper.pairs, Pair{
|
||||
Key: helper.currKey,
|
||||
Value: val,
|
||||
Err: err,
|
||||
})
|
||||
}
|
||||
helper.entry = mvccEntry{}
|
||||
}
|
||||
|
||||
func reverse(values []mvccValue) {
|
||||
i, j := 0, len(values)-1
|
||||
for i < j {
|
||||
values[i], values[j] = values[j], values[i]
|
||||
i++
|
||||
j--
|
||||
}
|
||||
}
|
||||
|
||||
// Prewrite implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) Prewrite(mutations []*kvrpcpb.Mutation, primary []byte, startTS uint64, ttl uint64) []error {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
anyError := false
|
||||
batch := &leveldb.Batch{}
|
||||
errs := make([]error, 0, len(mutations))
|
||||
for _, m := range mutations {
|
||||
err := prewriteMutation(mvcc.db, batch, m, startTS, primary, ttl)
|
||||
errs = append(errs, err)
|
||||
if err != nil {
|
||||
anyError = true
|
||||
}
|
||||
}
|
||||
if anyError {
|
||||
return errs
|
||||
}
|
||||
if err := mvcc.db.Write(batch, nil); err != nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
return errs
|
||||
}
|
||||
|
||||
func prewriteMutation(db *leveldb.DB, batch *leveldb.Batch, mutation *kvrpcpb.Mutation, startTS uint64, primary []byte, ttl uint64) error {
|
||||
startKey := mvccEncode(mutation.Key, lockVer)
|
||||
iter := newIterator(db, &util.Range{
|
||||
Start: startKey,
|
||||
})
|
||||
defer iter.Release()
|
||||
|
||||
dec := lockDecoder{
|
||||
expectKey: mutation.Key,
|
||||
}
|
||||
ok, err := dec.Decode(iter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if ok {
|
||||
if dec.lock.startTS != startTS {
|
||||
return dec.lock.lockErr(mutation.Key)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
dec1 := valueDecoder{
|
||||
expectKey: mutation.Key,
|
||||
}
|
||||
ok, err = dec1.Decode(iter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// Note that it's a write conflict here, even if the value is a rollback one.
|
||||
if ok && dec1.value.commitTS >= startTS {
|
||||
return ErrRetryable("write conflict")
|
||||
}
|
||||
|
||||
lock := mvccLock{
|
||||
startTS: startTS,
|
||||
primary: primary,
|
||||
value: mutation.Value,
|
||||
op: mutation.GetOp(),
|
||||
ttl: ttl,
|
||||
}
|
||||
writeKey := mvccEncode(mutation.Key, lockVer)
|
||||
writeValue, err := lock.MarshalBinary()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
batch.Put(writeKey, writeValue)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Commit implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) Commit(keys [][]byte, startTS, commitTS uint64) error {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
batch := &leveldb.Batch{}
|
||||
for _, k := range keys {
|
||||
err := commitKey(mvcc.db, batch, k, startTS, commitTS)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return mvcc.db.Write(batch, nil)
|
||||
}
|
||||
|
||||
func commitKey(db *leveldb.DB, batch *leveldb.Batch, key []byte, startTS, commitTS uint64) error {
|
||||
startKey := mvccEncode(key, lockVer)
|
||||
iter := newIterator(db, &util.Range{
|
||||
Start: startKey,
|
||||
})
|
||||
defer iter.Release()
|
||||
|
||||
dec := lockDecoder{
|
||||
expectKey: key,
|
||||
}
|
||||
ok, err := dec.Decode(iter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if !ok || dec.lock.startTS != startTS {
|
||||
// If the lock of this transaction is not found, or the lock is replaced by
|
||||
// another transaction, check commit information of this transaction.
|
||||
c, ok, err1 := getTxnCommitInfo(iter, key, startTS)
|
||||
if err1 != nil {
|
||||
return err1
|
||||
}
|
||||
if ok && c.valueType != typeRollback {
|
||||
// c.valueType != typeRollback means the transaction is already committed, do nothing.
|
||||
return nil
|
||||
}
|
||||
return ErrRetryable("txn not found")
|
||||
}
|
||||
|
||||
return commitLock(batch, dec.lock, key, startTS, commitTS)
|
||||
}
|
||||
|
||||
func commitLock(batch *leveldb.Batch, lock mvccLock, key []byte, startTS, commitTS uint64) error {
|
||||
if lock.op != kvrpcpb.Op_Lock {
|
||||
var valueType mvccValueType
|
||||
if lock.op == kvrpcpb.Op_Put {
|
||||
valueType = typePut
|
||||
} else {
|
||||
valueType = typeDelete
|
||||
}
|
||||
value := mvccValue{
|
||||
valueType: valueType,
|
||||
startTS: startTS,
|
||||
commitTS: commitTS,
|
||||
value: lock.value,
|
||||
}
|
||||
writeKey := mvccEncode(key, commitTS)
|
||||
writeValue, err := value.MarshalBinary()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
batch.Put(writeKey, writeValue)
|
||||
}
|
||||
batch.Delete(mvccEncode(key, lockVer))
|
||||
return nil
|
||||
}
|
||||
|
||||
// Rollback implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) Rollback(keys [][]byte, startTS uint64) error {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
batch := &leveldb.Batch{}
|
||||
for _, k := range keys {
|
||||
err := rollbackKey(mvcc.db, batch, k, startTS)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return mvcc.db.Write(batch, nil)
|
||||
}
|
||||
|
||||
func rollbackKey(db *leveldb.DB, batch *leveldb.Batch, key []byte, startTS uint64) error {
|
||||
startKey := mvccEncode(key, lockVer)
|
||||
iter := newIterator(db, &util.Range{
|
||||
Start: startKey,
|
||||
})
|
||||
defer iter.Release()
|
||||
|
||||
if iter.Valid() {
|
||||
dec := lockDecoder{
|
||||
expectKey: key,
|
||||
}
|
||||
ok, err := dec.Decode(iter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// If current transaction's lock exist.
|
||||
if ok && dec.lock.startTS == startTS {
|
||||
return rollbackLock(batch, dec.lock, key, startTS)
|
||||
}
|
||||
|
||||
// If current transaction's lock not exist.
|
||||
// If commit info of current transaction exist.
|
||||
c, ok, err := getTxnCommitInfo(iter, key, startTS)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if ok {
|
||||
// If current transaction is already committed.
|
||||
if c.valueType != typeRollback {
|
||||
return ErrAlreadyCommitted(c.commitTS)
|
||||
}
|
||||
// If current transaction is already rollback.
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// If current transaction is not prewritted before.
|
||||
value := mvccValue{
|
||||
valueType: typeRollback,
|
||||
startTS: startTS,
|
||||
commitTS: startTS,
|
||||
}
|
||||
writeKey := mvccEncode(key, startTS)
|
||||
writeValue, err := value.MarshalBinary()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
batch.Put(writeKey, writeValue)
|
||||
return nil
|
||||
}
|
||||
|
||||
func rollbackLock(batch *leveldb.Batch, lock mvccLock, key []byte, startTS uint64) error {
|
||||
tomb := mvccValue{
|
||||
valueType: typeRollback,
|
||||
startTS: startTS,
|
||||
commitTS: startTS,
|
||||
}
|
||||
writeKey := mvccEncode(key, startTS)
|
||||
writeValue, err := tomb.MarshalBinary()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
batch.Put(writeKey, writeValue)
|
||||
batch.Delete(mvccEncode(key, lockVer))
|
||||
return nil
|
||||
}
|
||||
|
||||
func getTxnCommitInfo(iter *Iterator, expectKey []byte, startTS uint64) (mvccValue, bool, error) {
|
||||
for iter.Valid() {
|
||||
dec := valueDecoder{
|
||||
expectKey: expectKey,
|
||||
}
|
||||
ok, err := dec.Decode(iter)
|
||||
if err != nil || !ok {
|
||||
return mvccValue{}, ok, err
|
||||
}
|
||||
|
||||
if dec.value.startTS == startTS {
|
||||
return dec.value, true, nil
|
||||
}
|
||||
}
|
||||
return mvccValue{}, false, nil
|
||||
}
|
||||
|
||||
// Cleanup implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) Cleanup(key []byte, startTS uint64) error {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
batch := &leveldb.Batch{}
|
||||
err := rollbackKey(mvcc.db, batch, key, startTS)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return mvcc.db.Write(batch, nil)
|
||||
}
|
||||
|
||||
// ScanLock implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) ScanLock(startKey, endKey []byte, maxTS uint64) ([]*kvrpcpb.LockInfo, error) {
|
||||
mvcc.mu.RLock()
|
||||
defer mvcc.mu.RUnlock()
|
||||
|
||||
iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey)
|
||||
defer iter.Release()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var locks []*kvrpcpb.LockInfo
|
||||
for iter.Valid() {
|
||||
dec := lockDecoder{expectKey: currKey}
|
||||
ok, err := dec.Decode(iter)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if ok && dec.lock.startTS <= maxTS {
|
||||
locks = append(locks, &kvrpcpb.LockInfo{
|
||||
PrimaryLock: dec.lock.primary,
|
||||
LockVersion: dec.lock.startTS,
|
||||
Key: currKey,
|
||||
})
|
||||
}
|
||||
|
||||
skip := skipDecoder{currKey: currKey}
|
||||
_, err = skip.Decode(iter)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
currKey = skip.currKey
|
||||
}
|
||||
return locks, nil
|
||||
}
|
||||
|
||||
// ResolveLock implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) ResolveLock(startKey, endKey []byte, startTS, commitTS uint64) error {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey)
|
||||
defer iter.Release()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
batch := &leveldb.Batch{}
|
||||
for iter.Valid() {
|
||||
dec := lockDecoder{expectKey: currKey}
|
||||
ok, err := dec.Decode(iter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if ok && dec.lock.startTS == startTS {
|
||||
if commitTS > 0 {
|
||||
err = commitLock(batch, dec.lock, currKey, startTS, commitTS)
|
||||
} else {
|
||||
err = rollbackLock(batch, dec.lock, currKey, startTS)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
skip := skipDecoder{currKey: currKey}
|
||||
_, err = skip.Decode(iter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
currKey = skip.currKey
|
||||
}
|
||||
return mvcc.db.Write(batch, nil)
|
||||
}
|
||||
|
||||
// BatchResolveLock implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) BatchResolveLock(startKey, endKey []byte, txnInfos map[uint64]uint64) error {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey)
|
||||
defer iter.Release()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
batch := &leveldb.Batch{}
|
||||
for iter.Valid() {
|
||||
dec := lockDecoder{expectKey: currKey}
|
||||
ok, err := dec.Decode(iter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if ok {
|
||||
if commitTS, ok := txnInfos[dec.lock.startTS]; ok {
|
||||
if commitTS > 0 {
|
||||
err = commitLock(batch, dec.lock, currKey, dec.lock.startTS, commitTS)
|
||||
} else {
|
||||
err = rollbackLock(batch, dec.lock, currKey, dec.lock.startTS)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
skip := skipDecoder{currKey: currKey}
|
||||
_, err = skip.Decode(iter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
currKey = skip.currKey
|
||||
}
|
||||
return mvcc.db.Write(batch, nil)
|
||||
}
|
||||
|
||||
// DeleteRange implements the MVCCStore interface.
|
||||
func (mvcc *MVCCLevelDB) DeleteRange(startKey, endKey []byte) error {
|
||||
return mvcc.doRawDeleteRange(codec.EncodeBytes(startKey), codec.EncodeBytes(endKey))
|
||||
}
|
||||
|
||||
// Close calls leveldb's Close to free resources.
|
||||
func (mvcc *MVCCLevelDB) Close() error {
|
||||
return mvcc.db.Close()
|
||||
}
|
||||
|
||||
// RawPut implements the RawKV interface.
|
||||
func (mvcc *MVCCLevelDB) RawPut(key, value []byte) {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
if value == nil {
|
||||
value = []byte{}
|
||||
}
|
||||
log.Error(mvcc.db.Put(key, value, nil))
|
||||
}
|
||||
|
||||
// RawBatchPut implements the RawKV interface
|
||||
func (mvcc *MVCCLevelDB) RawBatchPut(keys, values [][]byte) {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
batch := &leveldb.Batch{}
|
||||
for i, key := range keys {
|
||||
value := values[i]
|
||||
if value == nil {
|
||||
value = []byte{}
|
||||
}
|
||||
batch.Put(key, value)
|
||||
}
|
||||
log.Error(mvcc.db.Write(batch, nil))
|
||||
}
|
||||
|
||||
// RawGet implements the RawKV interface.
|
||||
func (mvcc *MVCCLevelDB) RawGet(key []byte) []byte {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
ret, err := mvcc.db.Get(key, nil)
|
||||
log.Error(err)
|
||||
return ret
|
||||
}
|
||||
|
||||
// RawBatchGet implements the RawKV interface.
|
||||
func (mvcc *MVCCLevelDB) RawBatchGet(keys [][]byte) [][]byte {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
var values [][]byte
|
||||
for _, key := range keys {
|
||||
value, err := mvcc.db.Get(key, nil)
|
||||
log.Error(err)
|
||||
values = append(values, value)
|
||||
}
|
||||
return values
|
||||
}
|
||||
|
||||
// RawDelete implements the RawKV interface.
|
||||
func (mvcc *MVCCLevelDB) RawDelete(key []byte) {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
log.Error(mvcc.db.Delete(key, nil))
|
||||
}
|
||||
|
||||
// RawBatchDelete implements the RawKV interface.
|
||||
func (mvcc *MVCCLevelDB) RawBatchDelete(keys [][]byte) {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
batch := &leveldb.Batch{}
|
||||
for _, key := range keys {
|
||||
batch.Delete(key)
|
||||
}
|
||||
log.Error(mvcc.db.Write(batch, nil))
|
||||
}
|
||||
|
||||
// RawScan implements the RawKV interface.
|
||||
func (mvcc *MVCCLevelDB) RawScan(startKey, endKey []byte, limit int) []Pair {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
iter := mvcc.db.NewIterator(&util.Range{
|
||||
Start: startKey,
|
||||
}, nil)
|
||||
|
||||
var pairs []Pair
|
||||
for iter.Next() && len(pairs) < limit {
|
||||
key := iter.Key()
|
||||
value := iter.Value()
|
||||
err := iter.Error()
|
||||
if len(endKey) > 0 && bytes.Compare(key, endKey) >= 0 {
|
||||
break
|
||||
}
|
||||
pairs = append(pairs, Pair{
|
||||
Key: append([]byte{}, key...),
|
||||
Value: append([]byte{}, value...),
|
||||
Err: err,
|
||||
})
|
||||
}
|
||||
return pairs
|
||||
}
|
||||
|
||||
// RawDeleteRange implements the RawKV interface.
|
||||
func (mvcc *MVCCLevelDB) RawDeleteRange(startKey, endKey []byte) {
|
||||
log.Error(mvcc.doRawDeleteRange(startKey, endKey))
|
||||
}
|
||||
|
||||
// doRawDeleteRange deletes all keys in a range and return the error if any.
|
||||
func (mvcc *MVCCLevelDB) doRawDeleteRange(startKey, endKey []byte) error {
|
||||
mvcc.mu.Lock()
|
||||
defer mvcc.mu.Unlock()
|
||||
|
||||
batch := &leveldb.Batch{}
|
||||
|
||||
iter := mvcc.db.NewIterator(&util.Range{
|
||||
Start: startKey,
|
||||
Limit: endKey,
|
||||
}, nil)
|
||||
for iter.Next() {
|
||||
batch.Delete(iter.Key())
|
||||
}
|
||||
|
||||
return mvcc.db.Write(batch, nil)
|
||||
}
|
||||
|
|
@ -1,159 +0,0 @@
|
|||
// Copyright 2016 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 mocktikv
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pingcap/kvproto/pkg/pdpb"
|
||||
"github.com/pkg/errors"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
||||
// Use global variables to prevent pdClients from creating duplicate timestamps.
|
||||
var tsMu = struct {
|
||||
sync.Mutex
|
||||
physicalTS int64
|
||||
logicalTS int64
|
||||
}{}
|
||||
|
||||
type pdClient struct {
|
||||
cluster *Cluster
|
||||
}
|
||||
|
||||
// NewPDClient creates a mock pd.Client that uses local timestamp and meta data
|
||||
// from a Cluster.
|
||||
func NewPDClient(cluster *Cluster) pd.Client {
|
||||
return &pdClient{
|
||||
cluster: cluster,
|
||||
}
|
||||
}
|
||||
|
||||
func (c *pdClient) GetClusterID(ctx context.Context) uint64 {
|
||||
return 1
|
||||
}
|
||||
|
||||
func (c *pdClient) GetAllMembers(ctx context.Context) ([]*pdpb.Member, error) {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) GetLeaderAddr() string {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) GetTS(context.Context) (int64, int64, error) {
|
||||
tsMu.Lock()
|
||||
defer tsMu.Unlock()
|
||||
|
||||
ts := time.Now().UnixNano() / int64(time.Millisecond)
|
||||
if tsMu.physicalTS >= ts {
|
||||
tsMu.logicalTS++
|
||||
} else {
|
||||
tsMu.physicalTS = ts
|
||||
tsMu.logicalTS = 0
|
||||
}
|
||||
return tsMu.physicalTS, tsMu.logicalTS, nil
|
||||
}
|
||||
|
||||
func (c *pdClient) GetTSAsync(ctx context.Context) pd.TSFuture {
|
||||
return &mockTSFuture{c, ctx}
|
||||
}
|
||||
|
||||
func (c *pdClient) GetLocalTS(ctx context.Context, dcLocation string) (int64, int64, error) {
|
||||
return c.GetTS(ctx)
|
||||
}
|
||||
|
||||
func (c *pdClient) GetLocalTSAsync(ctx context.Context, dcLocation string) pd.TSFuture {
|
||||
return c.GetTSAsync(ctx)
|
||||
}
|
||||
|
||||
func (c *pdClient) GetOperator(ctx context.Context, regionID uint64) (*pdpb.GetOperatorResponse, error) {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
type mockTSFuture struct {
|
||||
pdc *pdClient
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
func (m *mockTSFuture) Wait() (int64, int64, error) {
|
||||
return m.pdc.GetTS(m.ctx)
|
||||
}
|
||||
|
||||
func (c *pdClient) GetRegion(ctx context.Context, key []byte) (*pd.Region, error) {
|
||||
region, peer := c.cluster.GetRegionByKey(key)
|
||||
return &pd.Region{Meta: region, Leader: peer}, nil
|
||||
}
|
||||
|
||||
func (c *pdClient) GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string) (*pd.Region, error) {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) GetPrevRegion(ctx context.Context, key []byte) (*pd.Region, error) {
|
||||
region, peer := c.cluster.GetPrevRegionByKey(key)
|
||||
return &pd.Region{Meta: region, Leader: peer}, nil
|
||||
}
|
||||
|
||||
func (c *pdClient) GetRegionByID(ctx context.Context, regionID uint64) (*pd.Region, error) {
|
||||
region, peer := c.cluster.GetRegionByID(regionID)
|
||||
return &pd.Region{Meta: region, Leader: peer}, nil
|
||||
}
|
||||
|
||||
func (c *pdClient) ScanRegions(ctx context.Context, key, endKey []byte, limit int) ([]*pd.Region, error) {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
default:
|
||||
}
|
||||
store := c.cluster.GetStore(storeID)
|
||||
return store, nil
|
||||
}
|
||||
|
||||
func (c *pdClient) GetAllStores(ctx context.Context, opts ...pd.GetStoreOption) ([]*metapb.Store, error) {
|
||||
panic(errors.New("unimplemented"))
|
||||
}
|
||||
|
||||
func (c *pdClient) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) ScatterRegion(ctx context.Context, regionID uint64) error {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) ScatterRegionWithOption(ctx context.Context, regionID uint64, opts ...pd.RegionsOption) error {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) ScatterRegions(ctx context.Context, regionsID []uint64, opts ...pd.RegionsOption) (*pdpb.ScatterRegionResponse, error) {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) SplitRegions(ctx context.Context, splitKeys [][]byte, opts ...pd.RegionsOption) (*pdpb.SplitRegionsResponse, error) {
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
||||
func (c *pdClient) Close() {
|
||||
}
|
||||
|
|
@ -1,855 +0,0 @@
|
|||
// Copyright 2016 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 mocktikv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"io"
|
||||
"time"
|
||||
"encoding/binary"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/rpc"
|
||||
)
|
||||
|
||||
// For gofail injection.
|
||||
var errUndeterminedErr = errors.New("undetermined")
|
||||
|
||||
const requestMaxSize = 8 * 1024 * 1024
|
||||
|
||||
func checkGoContext(ctx context.Context) error {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func convertToKeyError(err error) *kvrpcpb.KeyError {
|
||||
if locked, ok := errors.Cause(err).(*ErrLocked); ok {
|
||||
return &kvrpcpb.KeyError{
|
||||
Locked: &kvrpcpb.LockInfo{
|
||||
Key: locked.Key.Raw(),
|
||||
PrimaryLock: locked.Primary,
|
||||
LockVersion: locked.StartTS,
|
||||
LockTtl: locked.TTL,
|
||||
},
|
||||
}
|
||||
}
|
||||
if retryable, ok := errors.Cause(err).(ErrRetryable); ok {
|
||||
return &kvrpcpb.KeyError{
|
||||
Retryable: retryable.Error(),
|
||||
}
|
||||
}
|
||||
return &kvrpcpb.KeyError{
|
||||
Abort: err.Error(),
|
||||
}
|
||||
}
|
||||
|
||||
func convertToKeyErrors(errs []error) []*kvrpcpb.KeyError {
|
||||
var keyErrors = make([]*kvrpcpb.KeyError, 0)
|
||||
for _, err := range errs {
|
||||
if err != nil {
|
||||
keyErrors = append(keyErrors, convertToKeyError(err))
|
||||
}
|
||||
}
|
||||
return keyErrors
|
||||
}
|
||||
|
||||
func convertToPbPairs(pairs []Pair) []*kvrpcpb.KvPair {
|
||||
kvPairs := make([]*kvrpcpb.KvPair, 0, len(pairs))
|
||||
for _, p := range pairs {
|
||||
var kvPair *kvrpcpb.KvPair
|
||||
if p.Err == nil {
|
||||
kvPair = &kvrpcpb.KvPair{
|
||||
Key: p.Key,
|
||||
Value: p.Value,
|
||||
}
|
||||
} else {
|
||||
kvPair = &kvrpcpb.KvPair{
|
||||
Error: convertToKeyError(p.Err),
|
||||
}
|
||||
}
|
||||
kvPairs = append(kvPairs, kvPair)
|
||||
}
|
||||
return kvPairs
|
||||
}
|
||||
|
||||
// rpcHandler mocks tikv's side handler behavior. In general, you may assume
|
||||
// TiKV just translate the logic from Go to Rust.
|
||||
type rpcHandler struct {
|
||||
cluster *Cluster
|
||||
mvccStore MVCCStore
|
||||
|
||||
// store id for current request
|
||||
storeID uint64
|
||||
// Used for handling normal request.
|
||||
startKey []byte
|
||||
endKey []byte
|
||||
// Used for handling coprocessor request.
|
||||
rawStartKey []byte
|
||||
rawEndKey []byte
|
||||
// Used for current request.
|
||||
isolationLevel kvrpcpb.IsolationLevel
|
||||
}
|
||||
|
||||
func (h *rpcHandler) checkRequestContext(ctx *kvrpcpb.Context) *errorpb.Error {
|
||||
ctxPeer := ctx.GetPeer()
|
||||
if ctxPeer != nil && ctxPeer.GetStoreId() != h.storeID {
|
||||
return &errorpb.Error{
|
||||
Message: *proto.String("store not match"),
|
||||
StoreNotMatch: &errorpb.StoreNotMatch{},
|
||||
}
|
||||
}
|
||||
region, leaderID := h.cluster.GetRegion(ctx.GetRegionId())
|
||||
// No region found.
|
||||
if region == nil {
|
||||
return &errorpb.Error{
|
||||
Message: *proto.String("region not found"),
|
||||
RegionNotFound: &errorpb.RegionNotFound{
|
||||
RegionId: *proto.Uint64(ctx.GetRegionId()),
|
||||
},
|
||||
}
|
||||
}
|
||||
var storePeer, leaderPeer *metapb.Peer
|
||||
for _, p := range region.Peers {
|
||||
if p.GetStoreId() == h.storeID {
|
||||
storePeer = p
|
||||
}
|
||||
if p.GetId() == leaderID {
|
||||
leaderPeer = p
|
||||
}
|
||||
}
|
||||
// The Store does not contain a Peer of the Region.
|
||||
if storePeer == nil {
|
||||
return &errorpb.Error{
|
||||
Message: *proto.String("region not found"),
|
||||
RegionNotFound: &errorpb.RegionNotFound{
|
||||
RegionId: *proto.Uint64(ctx.GetRegionId()),
|
||||
},
|
||||
}
|
||||
}
|
||||
// No leader.
|
||||
if leaderPeer == nil {
|
||||
return &errorpb.Error{
|
||||
Message: *proto.String("no leader"),
|
||||
NotLeader: &errorpb.NotLeader{
|
||||
RegionId: *proto.Uint64(ctx.GetRegionId()),
|
||||
},
|
||||
}
|
||||
}
|
||||
// The Peer on the Store is not leader.
|
||||
if storePeer.GetId() != leaderPeer.GetId() {
|
||||
return &errorpb.Error{
|
||||
Message: *proto.String("not leader"),
|
||||
NotLeader: &errorpb.NotLeader{
|
||||
RegionId: *proto.Uint64(ctx.GetRegionId()),
|
||||
Leader: leaderPeer,
|
||||
},
|
||||
}
|
||||
}
|
||||
// Region epoch does not match.
|
||||
if !proto.Equal(region.GetRegionEpoch(), ctx.GetRegionEpoch()) {
|
||||
nextRegion, _ := h.cluster.GetRegionByKey(region.GetEndKey())
|
||||
currentRegions := []*metapb.Region{region}
|
||||
if nextRegion != nil {
|
||||
currentRegions = append(currentRegions, nextRegion)
|
||||
}
|
||||
return &errorpb.Error{
|
||||
Message: *proto.String("stale epoch"),
|
||||
EpochNotMatch: &errorpb.EpochNotMatch{
|
||||
CurrentRegions: currentRegions,
|
||||
},
|
||||
}
|
||||
}
|
||||
h.startKey, h.endKey = region.StartKey, region.EndKey
|
||||
h.isolationLevel = ctx.IsolationLevel
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *rpcHandler) checkRequestSize(size int) *errorpb.Error {
|
||||
// TiKV has a limitation on raft log size.
|
||||
// mocktikv has no raft inside, so we check the request's size instead.
|
||||
if size >= requestMaxSize {
|
||||
return &errorpb.Error{
|
||||
RaftEntryTooLarge: &errorpb.RaftEntryTooLarge{},
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *rpcHandler) checkRequest(ctx *kvrpcpb.Context, size int) *errorpb.Error {
|
||||
if err := h.checkRequestContext(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
return h.checkRequestSize(size)
|
||||
}
|
||||
|
||||
func (h *rpcHandler) checkKeyInRegion(key []byte) bool {
|
||||
return regionContains(h.startKey, h.endKey, []byte(NewMvccKey(key)))
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvGet(req *kvrpcpb.GetRequest) *kvrpcpb.GetResponse {
|
||||
if !h.checkKeyInRegion(req.Key) {
|
||||
panic("KvGet: key not in region")
|
||||
}
|
||||
|
||||
val, err := h.mvccStore.Get(req.Key, req.GetVersion(), h.isolationLevel)
|
||||
if err != nil {
|
||||
return &kvrpcpb.GetResponse{
|
||||
Error: convertToKeyError(err),
|
||||
}
|
||||
}
|
||||
return &kvrpcpb.GetResponse{
|
||||
Value: val,
|
||||
}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvScan(req *kvrpcpb.ScanRequest) *kvrpcpb.ScanResponse {
|
||||
if !h.checkKeyInRegion(req.GetStartKey()) {
|
||||
panic("KvScan: startKey not in region")
|
||||
}
|
||||
endKey := h.endKey
|
||||
if len(req.EndKey) > 0 && (len(endKey) == 0 || bytes.Compare(req.EndKey, endKey) < 0) {
|
||||
endKey = req.EndKey
|
||||
}
|
||||
pairs := h.mvccStore.Scan(req.GetStartKey(), endKey, int(req.GetLimit()), req.GetVersion(), h.isolationLevel)
|
||||
return &kvrpcpb.ScanResponse{
|
||||
Pairs: convertToPbPairs(pairs),
|
||||
}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvPrewrite(req *kvrpcpb.PrewriteRequest) *kvrpcpb.PrewriteResponse {
|
||||
for _, m := range req.Mutations {
|
||||
if !h.checkKeyInRegion(m.Key) {
|
||||
panic("KvPrewrite: key not in region")
|
||||
}
|
||||
}
|
||||
errs := h.mvccStore.Prewrite(req.Mutations, req.PrimaryLock, req.GetStartVersion(), req.GetLockTtl())
|
||||
return &kvrpcpb.PrewriteResponse{
|
||||
Errors: convertToKeyErrors(errs),
|
||||
}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvCommit(req *kvrpcpb.CommitRequest) *kvrpcpb.CommitResponse {
|
||||
for _, k := range req.Keys {
|
||||
if !h.checkKeyInRegion(k) {
|
||||
panic("KvCommit: key not in region")
|
||||
}
|
||||
}
|
||||
var resp kvrpcpb.CommitResponse
|
||||
err := h.mvccStore.Commit(req.Keys, req.GetStartVersion(), req.GetCommitVersion())
|
||||
if err != nil {
|
||||
resp.Error = convertToKeyError(err)
|
||||
}
|
||||
return &resp
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvCleanup(req *kvrpcpb.CleanupRequest) *kvrpcpb.CleanupResponse {
|
||||
if !h.checkKeyInRegion(req.Key) {
|
||||
panic("KvCleanup: key not in region")
|
||||
}
|
||||
var resp kvrpcpb.CleanupResponse
|
||||
err := h.mvccStore.Cleanup(req.Key, req.GetStartVersion())
|
||||
if err != nil {
|
||||
if commitTS, ok := errors.Cause(err).(ErrAlreadyCommitted); ok {
|
||||
resp.CommitVersion = uint64(commitTS)
|
||||
} else {
|
||||
resp.Error = convertToKeyError(err)
|
||||
}
|
||||
}
|
||||
return &resp
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvBatchGet(req *kvrpcpb.BatchGetRequest) *kvrpcpb.BatchGetResponse {
|
||||
for _, k := range req.Keys {
|
||||
if !h.checkKeyInRegion(k) {
|
||||
panic("KvBatchGet: key not in region")
|
||||
}
|
||||
}
|
||||
pairs := h.mvccStore.BatchGet(req.Keys, req.GetVersion(), h.isolationLevel)
|
||||
return &kvrpcpb.BatchGetResponse{
|
||||
Pairs: convertToPbPairs(pairs),
|
||||
}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleMvccGetByKey(req *kvrpcpb.MvccGetByKeyRequest) *kvrpcpb.MvccGetByKeyResponse {
|
||||
debugger, ok := h.mvccStore.(MVCCDebugger)
|
||||
if !ok {
|
||||
return &kvrpcpb.MvccGetByKeyResponse{
|
||||
Error: "not implement",
|
||||
}
|
||||
}
|
||||
|
||||
if !h.checkKeyInRegion(req.Key) {
|
||||
panic("MvccGetByKey: key not in region")
|
||||
}
|
||||
var resp kvrpcpb.MvccGetByKeyResponse
|
||||
resp.Info = debugger.MvccGetByKey(req.Key)
|
||||
return &resp
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleMvccGetByStartTS(req *kvrpcpb.MvccGetByStartTsRequest) *kvrpcpb.MvccGetByStartTsResponse {
|
||||
debugger, ok := h.mvccStore.(MVCCDebugger)
|
||||
if !ok {
|
||||
return &kvrpcpb.MvccGetByStartTsResponse{
|
||||
Error: "not implement",
|
||||
}
|
||||
}
|
||||
var resp kvrpcpb.MvccGetByStartTsResponse
|
||||
resp.Info, resp.Key = debugger.MvccGetByStartTS(h.startKey, h.endKey, req.StartTs)
|
||||
return &resp
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvBatchRollback(req *kvrpcpb.BatchRollbackRequest) *kvrpcpb.BatchRollbackResponse {
|
||||
err := h.mvccStore.Rollback(req.Keys, req.StartVersion)
|
||||
if err != nil {
|
||||
return &kvrpcpb.BatchRollbackResponse{
|
||||
Error: convertToKeyError(err),
|
||||
}
|
||||
}
|
||||
return &kvrpcpb.BatchRollbackResponse{}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvScanLock(req *kvrpcpb.ScanLockRequest) *kvrpcpb.ScanLockResponse {
|
||||
startKey := MvccKey(h.startKey).Raw()
|
||||
endKey := MvccKey(h.endKey).Raw()
|
||||
locks, err := h.mvccStore.ScanLock(startKey, endKey, req.GetMaxVersion())
|
||||
if err != nil {
|
||||
return &kvrpcpb.ScanLockResponse{
|
||||
Error: convertToKeyError(err),
|
||||
}
|
||||
}
|
||||
return &kvrpcpb.ScanLockResponse{
|
||||
Locks: locks,
|
||||
}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvResolveLock(req *kvrpcpb.ResolveLockRequest) *kvrpcpb.ResolveLockResponse {
|
||||
startKey := MvccKey(h.startKey).Raw()
|
||||
endKey := MvccKey(h.endKey).Raw()
|
||||
err := h.mvccStore.ResolveLock(startKey, endKey, req.GetStartVersion(), req.GetCommitVersion())
|
||||
if err != nil {
|
||||
return &kvrpcpb.ResolveLockResponse{
|
||||
Error: convertToKeyError(err),
|
||||
}
|
||||
}
|
||||
return &kvrpcpb.ResolveLockResponse{}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvDeleteRange(req *kvrpcpb.DeleteRangeRequest) *kvrpcpb.DeleteRangeResponse {
|
||||
if !h.checkKeyInRegion(req.StartKey) {
|
||||
panic("KvDeleteRange: key not in region")
|
||||
}
|
||||
var resp kvrpcpb.DeleteRangeResponse
|
||||
err := h.mvccStore.DeleteRange(req.StartKey, req.EndKey)
|
||||
if err != nil {
|
||||
resp.Error = err.Error()
|
||||
}
|
||||
return &resp
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvRawGet(req *kvrpcpb.RawGetRequest) *kvrpcpb.RawGetResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
return &kvrpcpb.RawGetResponse{
|
||||
Error: "not implemented",
|
||||
}
|
||||
}
|
||||
return &kvrpcpb.RawGetResponse{
|
||||
Value: rawKV.RawGet(req.GetKey()),
|
||||
}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvRawBatchGet(req *kvrpcpb.RawBatchGetRequest) *kvrpcpb.RawBatchGetResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
// TODO should we add error ?
|
||||
return &kvrpcpb.RawBatchGetResponse{
|
||||
RegionError: &errorpb.Error{
|
||||
Message: "not implemented",
|
||||
},
|
||||
}
|
||||
}
|
||||
values := rawKV.RawBatchGet(req.Keys)
|
||||
kvPairs := make([]*kvrpcpb.KvPair, len(values))
|
||||
for i, key := range req.Keys {
|
||||
kvPairs[i] = &kvrpcpb.KvPair{
|
||||
Key: key,
|
||||
Value: values[i],
|
||||
}
|
||||
}
|
||||
return &kvrpcpb.RawBatchGetResponse{
|
||||
Pairs: kvPairs,
|
||||
}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvRawPut(req *kvrpcpb.RawPutRequest) *kvrpcpb.RawPutResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
return &kvrpcpb.RawPutResponse{
|
||||
Error: "not implemented",
|
||||
}
|
||||
}
|
||||
value := req.GetValue()
|
||||
if ttl := req.GetTtl(); ttl != 0 {
|
||||
value = append(value, make([]byte, 8)...)
|
||||
binary.LittleEndian.PutUint64(value[len(value) - 8:], ttl)
|
||||
}
|
||||
rawKV.RawPut(req.GetKey(), value)
|
||||
return &kvrpcpb.RawPutResponse{}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvRawBatchPut(req *kvrpcpb.RawBatchPutRequest) *kvrpcpb.RawBatchPutResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
return &kvrpcpb.RawBatchPutResponse{
|
||||
Error: "not implemented",
|
||||
}
|
||||
}
|
||||
keys := make([][]byte, 0, len(req.Pairs))
|
||||
values := make([][]byte, 0, len(req.Pairs))
|
||||
for _, pair := range req.Pairs {
|
||||
keys = append(keys, pair.Key)
|
||||
value := pair.Value
|
||||
if ttl := req.GetTtl(); ttl != 0 {
|
||||
value = append(value, make([]byte, 8)...)
|
||||
binary.LittleEndian.PutUint64(value[len(value) - 8:], ttl)
|
||||
}
|
||||
values = append(values, value)
|
||||
}
|
||||
rawKV.RawBatchPut(keys, values)
|
||||
return &kvrpcpb.RawBatchPutResponse{}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvRawDelete(req *kvrpcpb.RawDeleteRequest) *kvrpcpb.RawDeleteResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
return &kvrpcpb.RawDeleteResponse{
|
||||
Error: "not implemented",
|
||||
}
|
||||
}
|
||||
rawKV.RawDelete(req.GetKey())
|
||||
return &kvrpcpb.RawDeleteResponse{}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvRawBatchDelete(req *kvrpcpb.RawBatchDeleteRequest) *kvrpcpb.RawBatchDeleteResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
return &kvrpcpb.RawBatchDeleteResponse{
|
||||
Error: "not implemented",
|
||||
}
|
||||
}
|
||||
rawKV.RawBatchDelete(req.Keys)
|
||||
return &kvrpcpb.RawBatchDeleteResponse{}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvRawDeleteRange(req *kvrpcpb.RawDeleteRangeRequest) *kvrpcpb.RawDeleteRangeResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
return &kvrpcpb.RawDeleteRangeResponse{
|
||||
Error: "not implemented",
|
||||
}
|
||||
}
|
||||
rawKV.RawDeleteRange(req.GetStartKey(), req.GetEndKey())
|
||||
return &kvrpcpb.RawDeleteRangeResponse{}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleKvRawScan(req *kvrpcpb.RawScanRequest) *kvrpcpb.RawScanResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
errStr := "not implemented"
|
||||
return &kvrpcpb.RawScanResponse{
|
||||
RegionError: &errorpb.Error{
|
||||
Message: errStr,
|
||||
},
|
||||
}
|
||||
}
|
||||
endKey := h.endKey
|
||||
if len(req.EndKey) > 0 && (len(endKey) == 0 || bytes.Compare(req.EndKey, endKey) < 0) {
|
||||
endKey = req.EndKey
|
||||
}
|
||||
pairs := rawKV.RawScan(req.GetStartKey(), endKey, int(req.GetLimit()))
|
||||
if req.KeyOnly {
|
||||
//filter values when the client set key only to true.
|
||||
for i := range pairs {
|
||||
pairs[i] = Pair{
|
||||
Key: pairs[i].Key,
|
||||
Value: nil,
|
||||
Err: nil,
|
||||
}
|
||||
}
|
||||
}
|
||||
return &kvrpcpb.RawScanResponse{
|
||||
Kvs: convertToPbPairs(pairs),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
func (h *rpcHandler) handleKvRawGetKeyTTL(req *kvrpcpb.RawGetKeyTTLRequest) *kvrpcpb.RawGetKeyTTLResponse {
|
||||
rawKV, ok := h.mvccStore.(RawKV)
|
||||
if !ok {
|
||||
return &kvrpcpb.RawGetKeyTTLResponse{
|
||||
Error: "not implemented",
|
||||
}
|
||||
}
|
||||
value := rawKV.RawGet(req.GetKey())
|
||||
if value == nil {
|
||||
return &kvrpcpb.RawGetKeyTTLResponse{
|
||||
NotFound: true,
|
||||
}
|
||||
} else {
|
||||
return &kvrpcpb.RawGetKeyTTLResponse{
|
||||
Ttl: binary.LittleEndian.Uint64(value[len(value) - 8:]),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (h *rpcHandler) handleSplitRegion(req *kvrpcpb.SplitRegionRequest) *kvrpcpb.SplitRegionResponse {
|
||||
key := NewMvccKey(req.GetSplitKey())
|
||||
region, _ := h.cluster.GetRegionByKey(key)
|
||||
if bytes.Equal(region.GetStartKey(), key) {
|
||||
return &kvrpcpb.SplitRegionResponse{}
|
||||
}
|
||||
newRegionID, newPeerIDs := h.cluster.AllocID(), h.cluster.AllocIDs(len(region.Peers))
|
||||
h.cluster.SplitRaw(region.GetId(), newRegionID, key, newPeerIDs, newPeerIDs[0])
|
||||
return &kvrpcpb.SplitRegionResponse{}
|
||||
}
|
||||
|
||||
// RPCClient sends kv RPC calls to mock cluster. RPCClient mocks the behavior of
|
||||
// a rpc client at tikv's side.
|
||||
type RPCClient struct {
|
||||
Cluster *Cluster
|
||||
MvccStore MVCCStore
|
||||
streamTimeout chan *rpc.Lease
|
||||
}
|
||||
|
||||
// NewRPCClient creates an RPCClient.
|
||||
// Note that close the RPCClient may close the underlying MvccStore.
|
||||
func NewRPCClient(cluster *Cluster, mvccStore MVCCStore) *RPCClient {
|
||||
ch := make(chan *rpc.Lease)
|
||||
go rpc.CheckStreamTimeoutLoop(ch)
|
||||
return &RPCClient{
|
||||
Cluster: cluster,
|
||||
MvccStore: mvccStore,
|
||||
streamTimeout: ch,
|
||||
}
|
||||
}
|
||||
|
||||
func (c *RPCClient) getAndCheckStoreByAddr(addr string) (*metapb.Store, error) {
|
||||
store, err := c.Cluster.GetAndCheckStoreByAddr(addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if store == nil {
|
||||
return nil, errors.New("connect fail")
|
||||
}
|
||||
if store.GetState() == metapb.StoreState_Offline ||
|
||||
store.GetState() == metapb.StoreState_Tombstone {
|
||||
return nil, errors.New("connection refused")
|
||||
}
|
||||
return store, nil
|
||||
}
|
||||
|
||||
func (c *RPCClient) checkArgs(ctx context.Context, addr string) (*rpcHandler, error) {
|
||||
if err := checkGoContext(ctx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
store, err := c.getAndCheckStoreByAddr(addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
handler := &rpcHandler{
|
||||
cluster: c.Cluster,
|
||||
mvccStore: c.MvccStore,
|
||||
// set store id for current request
|
||||
storeID: store.GetId(),
|
||||
}
|
||||
return handler, nil
|
||||
}
|
||||
|
||||
// SendRequest sends a request to mock cluster.
|
||||
func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *rpc.Request, timeout time.Duration) (*rpc.Response, error) {
|
||||
// gofail: var rpcServerBusy bool
|
||||
// if rpcServerBusy {
|
||||
// return rpc.GenRegionErrorResp(req, &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}})
|
||||
// }
|
||||
handler, err := c.checkArgs(ctx, addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
reqCtx := &req.Context
|
||||
resp := &rpc.Response{}
|
||||
resp.Type = req.Type
|
||||
switch req.Type {
|
||||
case rpc.CmdGet:
|
||||
r := req.Get
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.Get = &kvrpcpb.GetResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.Get = handler.handleKvGet(r)
|
||||
case rpc.CmdScan:
|
||||
r := req.Scan
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.Scan = &kvrpcpb.ScanResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.Scan = handler.handleKvScan(r)
|
||||
|
||||
case rpc.CmdPrewrite:
|
||||
r := req.Prewrite
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.Prewrite = &kvrpcpb.PrewriteResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.Prewrite = handler.handleKvPrewrite(r)
|
||||
case rpc.CmdCommit:
|
||||
// gofail: var rpcCommitResult string
|
||||
// switch rpcCommitResult {
|
||||
// case "timeout":
|
||||
// return nil, errors.New("timeout")
|
||||
// case "notLeader":
|
||||
// return &rpc.Response{
|
||||
// Type: rpc.CmdCommit,
|
||||
// Commit: &kvrpcpb.CommitResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}},
|
||||
// }, nil
|
||||
// case "keyError":
|
||||
// return &rpc.Response{
|
||||
// Type: rpc.CmdCommit,
|
||||
// Commit: &kvrpcpb.CommitResponse{Error: &kvrpcpb.KeyError{}},
|
||||
// }, nil
|
||||
// }
|
||||
r := req.Commit
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.Commit = &kvrpcpb.CommitResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.Commit = handler.handleKvCommit(r)
|
||||
// gofail: var rpcCommitTimeout bool
|
||||
// if rpcCommitTimeout {
|
||||
// return nil, errUndeterminedErr
|
||||
// }
|
||||
case rpc.CmdCleanup:
|
||||
r := req.Cleanup
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.Cleanup = &kvrpcpb.CleanupResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.Cleanup = handler.handleKvCleanup(r)
|
||||
case rpc.CmdBatchGet:
|
||||
r := req.BatchGet
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.BatchGet = &kvrpcpb.BatchGetResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.BatchGet = handler.handleKvBatchGet(r)
|
||||
case rpc.CmdBatchRollback:
|
||||
r := req.BatchRollback
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.BatchRollback = &kvrpcpb.BatchRollbackResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.BatchRollback = handler.handleKvBatchRollback(r)
|
||||
case rpc.CmdScanLock:
|
||||
r := req.ScanLock
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.ScanLock = &kvrpcpb.ScanLockResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.ScanLock = handler.handleKvScanLock(r)
|
||||
case rpc.CmdResolveLock:
|
||||
r := req.ResolveLock
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.ResolveLock = &kvrpcpb.ResolveLockResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.ResolveLock = handler.handleKvResolveLock(r)
|
||||
case rpc.CmdGC:
|
||||
r := req.GC
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.GC = &kvrpcpb.GCResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.GC = &kvrpcpb.GCResponse{}
|
||||
case rpc.CmdDeleteRange:
|
||||
r := req.DeleteRange
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.DeleteRange = &kvrpcpb.DeleteRangeResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.DeleteRange = handler.handleKvDeleteRange(r)
|
||||
case rpc.CmdRawGet:
|
||||
r := req.RawGet
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawGet = &kvrpcpb.RawGetResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.RawGet = handler.handleKvRawGet(r)
|
||||
case rpc.CmdRawBatchGet:
|
||||
r := req.RawBatchGet
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawBatchGet = &kvrpcpb.RawBatchGetResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.RawBatchGet = handler.handleKvRawBatchGet(r)
|
||||
case rpc.CmdRawPut:
|
||||
r := req.RawPut
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawPut = &kvrpcpb.RawPutResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.RawPut = handler.handleKvRawPut(r)
|
||||
case rpc.CmdRawBatchPut:
|
||||
r := req.RawBatchPut
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawBatchPut = &kvrpcpb.RawBatchPutResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.RawBatchPut = handler.handleKvRawBatchPut(r)
|
||||
case rpc.CmdRawDelete:
|
||||
r := req.RawDelete
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawDelete = &kvrpcpb.RawDeleteResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.RawDelete = handler.handleKvRawDelete(r)
|
||||
case rpc.CmdRawBatchDelete:
|
||||
r := req.RawBatchDelete
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawBatchDelete = &kvrpcpb.RawBatchDeleteResponse{RegionError: err}
|
||||
}
|
||||
resp.RawBatchDelete = handler.handleKvRawBatchDelete(r)
|
||||
case rpc.CmdRawDeleteRange:
|
||||
r := req.RawDeleteRange
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawDeleteRange = &kvrpcpb.RawDeleteRangeResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.RawDeleteRange = handler.handleKvRawDeleteRange(r)
|
||||
case rpc.CmdRawScan:
|
||||
r := req.RawScan
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawScan = &kvrpcpb.RawScanResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.RawScan = handler.handleKvRawScan(r)
|
||||
case rpc.CmdRawGetKeyTTL:
|
||||
r := req.RawGetKeyTTL
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.RawGetKeyTTL = &kvrpcpb.RawGetKeyTTLResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.RawGetKeyTTL = handler.handleKvRawGetKeyTTL(r)
|
||||
case rpc.CmdUnsafeDestroyRange:
|
||||
panic("unimplemented")
|
||||
case rpc.CmdCop:
|
||||
// TODO: support register cop handler.
|
||||
panic("unimplemented")
|
||||
// r := req.Cop
|
||||
// if err := handler.checkRequestContext(reqCtx); err != nil {
|
||||
// resp.Cop = &coprocessor.Response{RegionError: err}
|
||||
// return resp, nil
|
||||
// }
|
||||
// handler.rawStartKey = MvccKey(handler.startKey).Raw()
|
||||
// handler.rawEndKey = MvccKey(handler.endKey).Raw()
|
||||
// var res *coprocessor.Response
|
||||
// switch r.GetTp() {
|
||||
// case kv.ReqTypeDAG:
|
||||
// res = handler.handleCopDAGRequest(r)
|
||||
// case kv.ReqTypeAnalyze:
|
||||
// res = handler.handleCopAnalyzeRequest(r)
|
||||
// case kv.ReqTypeChecksum:
|
||||
// res = handler.handleCopChecksumRequest(r)
|
||||
// default:
|
||||
// panic(fmt.Sprintf("unknown coprocessor request type: %v", r.GetTp()))
|
||||
// }
|
||||
// resp.Cop = res
|
||||
case rpc.CmdCopStream:
|
||||
// TODO: support register copStream handler.
|
||||
panic("unimplemented")
|
||||
// r := req.Cop
|
||||
// if err := handler.checkRequestContext(reqCtx); err != nil {
|
||||
// resp.CopStream = &rpc.CopStreamResponse{
|
||||
// Tikv_CoprocessorStreamClient: &mockCopStreamErrClient{Error: err},
|
||||
// Response: &coprocessor.Response{
|
||||
// RegionError: err,
|
||||
// },
|
||||
// }
|
||||
// return resp, nil
|
||||
// }
|
||||
// handler.rawStartKey = MvccKey(handler.startKey).Raw()
|
||||
// handler.rawEndKey = MvccKey(handler.endKey).Raw()
|
||||
// ctx1, cancel := context.WithCancel(ctx)
|
||||
// copStream, err := handler.handleCopStream(ctx1, r)
|
||||
// if err != nil {
|
||||
// cancel()
|
||||
// return nil, err
|
||||
// }
|
||||
|
||||
// streamResp := &rpc.CopStreamResponse{
|
||||
// Tikv_CoprocessorStreamClient: copStream,
|
||||
// }
|
||||
// streamResp.Lease.Cancel = cancel
|
||||
// streamResp.Timeout = timeout
|
||||
// c.streamTimeout <- &streamResp.Lease
|
||||
|
||||
// first, err := streamResp.Recv()
|
||||
// if err != nil {
|
||||
// return nil, err
|
||||
// }
|
||||
// streamResp.Response = first
|
||||
// resp.CopStream = streamResp
|
||||
case rpc.CmdMvccGetByKey:
|
||||
r := req.MvccGetByKey
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.MvccGetByKey = &kvrpcpb.MvccGetByKeyResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.MvccGetByKey = handler.handleMvccGetByKey(r)
|
||||
case rpc.CmdMvccGetByStartTs:
|
||||
r := req.MvccGetByStartTs
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.MvccGetByStartTS = &kvrpcpb.MvccGetByStartTsResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.MvccGetByStartTS = handler.handleMvccGetByStartTS(r)
|
||||
case rpc.CmdSplitRegion:
|
||||
r := req.SplitRegion
|
||||
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
|
||||
resp.SplitRegion = &kvrpcpb.SplitRegionResponse{RegionError: err}
|
||||
return resp, nil
|
||||
}
|
||||
resp.SplitRegion = handler.handleSplitRegion(r)
|
||||
default:
|
||||
return nil, errors.Errorf("unsupport this request type %v", req.Type)
|
||||
}
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
// Close closes the client.
|
||||
func (c *RPCClient) Close() error {
|
||||
close(c.streamTimeout)
|
||||
if raw, ok := c.MvccStore.(io.Closer); ok {
|
||||
return raw.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
@ -1,107 +0,0 @@
|
|||
// 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 httpproxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/http"
|
||||
"time"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/proxy"
|
||||
)
|
||||
|
||||
type HandlerWithConfig struct {
|
||||
http.Handler
|
||||
Config *config.Config
|
||||
}
|
||||
|
||||
// NewHTTPProxyHandlerWithConfig creates an http.Handler with the config pointer, users can set the config through it.
|
||||
func NewHTTPProxyHandlerWithConfig() HandlerWithConfig {
|
||||
h, c := newHTTPProxyHandler()
|
||||
return HandlerWithConfig{h, c}
|
||||
}
|
||||
|
||||
// NewHTTPProxyHandler creates an http.Handler that serves as a TiKV client proxy.
|
||||
func NewHTTPProxyHandler() http.Handler {
|
||||
h, _ := newHTTPProxyHandler()
|
||||
return h
|
||||
}
|
||||
|
||||
func newHTTPProxyHandler() (http.Handler, *config.Config) {
|
||||
router := mux.NewRouter()
|
||||
cd := config.Default()
|
||||
rawkv := rawkvHandler{p: proxy.NewRaw(), c: &cd}
|
||||
|
||||
router.HandleFunc("/rawkv/client/new", rawkv.handlerFunc(rawkv.New))
|
||||
router.HandleFunc("/rawkv/client/{id}/close", rawkv.handlerFunc(rawkv.Close))
|
||||
router.HandleFunc("/rawkv/client/{id}/get", rawkv.handlerFunc(rawkv.Get))
|
||||
router.HandleFunc("/rawkv/client/{id}/batch-get", rawkv.handlerFunc(rawkv.BatchGet))
|
||||
router.HandleFunc("/rawkv/client/{id}/put", rawkv.handlerFunc(rawkv.Put))
|
||||
router.HandleFunc("/rawkv/client/{id}/batch-put", rawkv.handlerFunc(rawkv.BatchPut))
|
||||
router.HandleFunc("/rawkv/client/{id}/delete", rawkv.handlerFunc(rawkv.Delete))
|
||||
router.HandleFunc("/rawkv/client/{id}/batch-delete", rawkv.handlerFunc(rawkv.BatchDelete))
|
||||
router.HandleFunc("/rawkv/client/{id}/delete-range", rawkv.handlerFunc(rawkv.DeleteRange))
|
||||
router.HandleFunc("/rawkv/client/{id}/scan", rawkv.handlerFunc(rawkv.Scan))
|
||||
router.HandleFunc("/rawkv/client/{id}/reverse-scan", rawkv.handlerFunc(rawkv.ReverseScan))
|
||||
|
||||
txnkv := txnkvHandler{p: proxy.NewTxn(), c: &cd}
|
||||
router.HandleFunc("/txnkv/client/new", txnkv.handlerFunc(txnkv.New))
|
||||
router.HandleFunc("/txnkv/client/{id}/close", txnkv.handlerFunc(txnkv.Close))
|
||||
router.HandleFunc("/txnkv/client/{id}/begin", txnkv.handlerFunc(txnkv.Begin))
|
||||
router.HandleFunc("/txnkv/client/{id}/begin-with-ts", txnkv.handlerFunc(txnkv.BeginWithTS))
|
||||
router.HandleFunc("/txnkv/client/{id}/get-ts", txnkv.handlerFunc(txnkv.GetTS))
|
||||
router.HandleFunc("/txnkv/txn/{id}/get", txnkv.handlerFunc(txnkv.TxnGet))
|
||||
router.HandleFunc("/txnkv/txn/{id}/batch-get", txnkv.handlerFunc(txnkv.TxnBatchGet))
|
||||
router.HandleFunc("/txnkv/txn/{id}/set", txnkv.handlerFunc(txnkv.TxnSet))
|
||||
router.HandleFunc("/txnkv/txn/{id}/iter", txnkv.handlerFunc(txnkv.TxnIter))
|
||||
router.HandleFunc("/txnkv/txn/{id}/iter-reverse", txnkv.handlerFunc(txnkv.TxnIterReverse))
|
||||
router.HandleFunc("/txnkv/txn/{id}/readonly", txnkv.handlerFunc(txnkv.TxnIsReadOnly))
|
||||
router.HandleFunc("/txnkv/txn/{id}/delete", txnkv.handlerFunc(txnkv.TxnDelete))
|
||||
router.HandleFunc("/txnkv/txn/{id}/commit", txnkv.handlerFunc(txnkv.TxnCommit))
|
||||
router.HandleFunc("/txnkv/txn/{id}/rollback", txnkv.handlerFunc(txnkv.TxnRollback))
|
||||
router.HandleFunc("/txnkv/txn/{id}/lock-keys", txnkv.handlerFunc(txnkv.TxnLockKeys))
|
||||
router.HandleFunc("/txnkv/txn/{id}/valid", txnkv.handlerFunc(txnkv.TxnValid))
|
||||
router.HandleFunc("/txnkv/txn/{id}/len", txnkv.handlerFunc(txnkv.TxnLen))
|
||||
router.HandleFunc("/txnkv/txn/{id}/size", txnkv.handlerFunc(txnkv.TxnSize))
|
||||
router.HandleFunc("/txnkv/iter/{id}/valid", txnkv.handlerFunc(txnkv.IterValid))
|
||||
router.HandleFunc("/txnkv/iter/{id}/key", txnkv.handlerFunc(txnkv.IterKey))
|
||||
router.HandleFunc("/txnkv/iter/{id}/value", txnkv.handlerFunc(txnkv.IterValue))
|
||||
router.HandleFunc("/txnkv/iter/{id}/next", txnkv.handlerFunc(txnkv.IterNext))
|
||||
router.HandleFunc("/txnkv/iter/{id}/close", txnkv.handlerFunc(txnkv.IterClose))
|
||||
|
||||
router.NotFoundHandler = http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
w.WriteHeader(http.StatusNotImplemented)
|
||||
w.Write([]byte("not implemented"))
|
||||
})
|
||||
|
||||
return router, &cd
|
||||
}
|
||||
|
||||
var defaultTimeout = 20 * time.Second
|
||||
|
||||
func reqContext(vars map[string]string) (context.Context, context.CancelFunc) {
|
||||
ctx := context.Background()
|
||||
if id := vars["id"]; id != "" {
|
||||
ctx = context.WithValue(ctx, proxy.UUIDKey, proxy.UUID(id))
|
||||
}
|
||||
|
||||
d, err := time.ParseDuration(vars["timeout"])
|
||||
if err != nil {
|
||||
d = defaultTimeout
|
||||
}
|
||||
|
||||
return context.WithTimeout(ctx, d)
|
||||
}
|
||||
|
|
@ -1,178 +0,0 @@
|
|||
// 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 httpproxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/proxy"
|
||||
)
|
||||
|
||||
type rawkvHandler struct {
|
||||
p proxy.RawKVProxy
|
||||
c *config.Config
|
||||
}
|
||||
|
||||
// RawRequest is the structure of a rawkv request that the http proxy accepts.
|
||||
type RawRequest struct {
|
||||
PDAddrs []string `json:"pd_addrs,omitempty"` // for new
|
||||
Key []byte `json:"key,omitempty"` // for get, put, delete
|
||||
Keys [][]byte `json:"keys,omitempty"` // for batchGet, batchPut, batchDelete
|
||||
Value []byte `json:"value,omitempty"` // for put
|
||||
Values [][]byte `json:"values,omitmepty"` // for batchPut
|
||||
StartKey []byte `json:"start_key,omitempty"` // for scan, deleteRange
|
||||
EndKey []byte `json:"end_key,omitempty"` // for scan, deleteRange
|
||||
Limit int `json:"limit,omitempty"` // for scan
|
||||
}
|
||||
|
||||
// RawResponse is the structure of a rawkv response that the http proxy sends.
|
||||
type RawResponse struct {
|
||||
ID string `json:"id,omitempty"` // for new
|
||||
Value []byte `json:"value,omitempty"` // for get
|
||||
Keys [][]byte `json:"keys,omitempty"` // for scan
|
||||
Values [][]byte `json:"values,omitempty"` // for batchGet
|
||||
}
|
||||
|
||||
func (h rawkvHandler) New(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
id, err := h.p.New(ctx, r.PDAddrs, h.getConfig())
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{ID: string(id)}, http.StatusCreated, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) Close(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
if err := h.p.Close(ctx); err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) Get(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
val, err := h.p.Get(ctx, r.Key)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{Value: val}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) BatchGet(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
vals, err := h.p.BatchGet(ctx, r.Keys)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{Values: vals}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) Put(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
err := h.p.Put(ctx, r.Key, r.Value)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) BatchPut(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
err := h.p.BatchPut(ctx, r.Keys, r.Values)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) Delete(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
err := h.p.Delete(ctx, r.Key)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) BatchDelete(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
err := h.p.BatchDelete(ctx, r.Keys)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) DeleteRange(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
err := h.p.DeleteRange(ctx, r.StartKey, r.EndKey)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) Scan(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
keys, values, err := h.p.Scan(ctx, r.StartKey, r.EndKey, r.Limit)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{Keys: keys, Values: values}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) ReverseScan(ctx context.Context, r *RawRequest) (*RawResponse, int, error) {
|
||||
keys, values, err := h.p.ReverseScan(ctx, r.StartKey, r.EndKey, r.Limit)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &RawResponse{Keys: keys, Values: values}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h rawkvHandler) handlerFunc(f func(context.Context, *RawRequest) (*RawResponse, int, error)) func(http.ResponseWriter, *http.Request) {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
data, err := ioutil.ReadAll(r.Body)
|
||||
if err != nil {
|
||||
sendError(w, err, http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
var req RawRequest
|
||||
if err = json.Unmarshal(data, &req); err != nil {
|
||||
sendError(w, err, http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
ctx, cancel := reqContext(mux.Vars(r))
|
||||
res, status, err := f(ctx, &req)
|
||||
cancel()
|
||||
if err != nil {
|
||||
sendError(w, err, status)
|
||||
return
|
||||
}
|
||||
data, err = json.Marshal(res)
|
||||
if err != nil {
|
||||
sendError(w, err, http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
w.WriteHeader(status)
|
||||
w.Write(data)
|
||||
}
|
||||
}
|
||||
|
||||
func sendError(w http.ResponseWriter, err error, status int) {
|
||||
w.WriteHeader(status)
|
||||
w.Write([]byte(err.Error()))
|
||||
}
|
||||
|
||||
func (h rawkvHandler) getConfig() config.Config {
|
||||
if h.c == nil {
|
||||
return config.Default()
|
||||
}
|
||||
return *h.c
|
||||
}
|
||||
|
|
@ -1,279 +0,0 @@
|
|||
// 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 httpproxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/proxy"
|
||||
)
|
||||
|
||||
type txnkvHandler struct {
|
||||
p proxy.TxnKVProxy
|
||||
c *config.Config
|
||||
}
|
||||
|
||||
// TxnRequest is the structure of a txnkv request that the http proxy accepts.
|
||||
type TxnRequest struct {
|
||||
PDAddrs []string `json:"pd_addrs,omitempty"` // for new
|
||||
TS uint64 `json:"ts,omitempty"` // for beginWithTS
|
||||
Key []byte `json:"key,omitempty"` // for get, set, delete, iter, iterReverse
|
||||
Value []byte `json:"value,omitempty"` // for set
|
||||
Keys [][]byte `json:"keys,omitempty"` // for batchGet, lockKeys
|
||||
UpperBound []byte `json:"upper_bound,omitempty"` // for iter
|
||||
}
|
||||
|
||||
// TxnResponse is the structure of a txnkv response that the http proxy sends.
|
||||
type TxnResponse struct {
|
||||
ID string `json:"id,omitempty"` // for new, begin, beginWithTS, iter, iterReverse
|
||||
TS uint64 `json:"ts,omitempty"` // for getTS
|
||||
Key []byte `json:"key,omitempty"` // for iterKey
|
||||
Value []byte `json:"value,omitempty"` // for get, iterValue
|
||||
Keys [][]byte `json:"keys,omitempty"` // for batchGet
|
||||
Values [][]byte `json:"values,omitempty"` // for batchGet
|
||||
IsValid bool `json:"is_valid,omitempty"` // for valid, iterValid
|
||||
IsReadOnly bool `json:"is_readonly,omitempty"` // for isReadOnly
|
||||
Size int `json:"size,omitempty"` // for size
|
||||
Length int `json:"length,omitempty"` // for length
|
||||
}
|
||||
|
||||
func (h txnkvHandler) New(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
id, err := h.p.New(ctx, r.PDAddrs, h.getConfig())
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{ID: string(id)}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) Close(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
err := h.p.Close(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) Begin(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
txnID, err := h.p.Begin(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{ID: string(txnID)}, http.StatusCreated, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) BeginWithTS(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
txnID, err := h.p.BeginWithTS(ctx, r.TS)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{ID: string(txnID)}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) GetTS(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
ts, err := h.p.GetTS(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{TS: ts}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnGet(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
val, err := h.p.TxnGet(ctx, r.Key)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{Value: val}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnBatchGet(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
m, err := h.p.TxnBatchGet(ctx, r.Keys)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
keys, values := make([][]byte, 0, len(m)), make([][]byte, 0, len(m))
|
||||
for k, v := range m {
|
||||
keys = append(keys, []byte(k))
|
||||
values = append(values, v)
|
||||
}
|
||||
return &TxnResponse{Keys: keys, Values: values}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnSet(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
err := h.p.TxnSet(ctx, r.Key, r.Value)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnIter(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
iterID, err := h.p.TxnIter(ctx, r.Key, r.UpperBound)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{ID: string(iterID)}, http.StatusCreated, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnIterReverse(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
iterID, err := h.p.TxnIterReverse(ctx, r.Key)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{ID: string(iterID)}, http.StatusCreated, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnIsReadOnly(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
readonly, err := h.p.TxnIsReadOnly(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{IsReadOnly: readonly}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnDelete(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
err := h.p.TxnDelete(ctx, r.Key)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnCommit(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
err := h.p.TxnCommit(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnRollback(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
err := h.p.TxnRollback(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnLockKeys(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
err := h.p.TxnLockKeys(ctx, r.Keys)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnValid(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
valid, err := h.p.TxnValid(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{IsValid: valid}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnLen(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
length, err := h.p.TxnLen(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{Length: length}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) TxnSize(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
size, err := h.p.TxnSize(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{Size: size}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) IterValid(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
valid, err := h.p.IterValid(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{IsValid: valid}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) IterKey(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
key, err := h.p.IterKey(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{Key: key}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) IterValue(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
val, err := h.p.IterValue(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{Value: val}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) IterNext(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
err := h.p.IterNext(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) IterClose(ctx context.Context, r *TxnRequest) (*TxnResponse, int, error) {
|
||||
err := h.p.IterClose(ctx)
|
||||
if err != nil {
|
||||
return nil, http.StatusInternalServerError, err
|
||||
}
|
||||
return &TxnResponse{}, http.StatusOK, nil
|
||||
}
|
||||
|
||||
func (h txnkvHandler) handlerFunc(f func(context.Context, *TxnRequest) (*TxnResponse, int, error)) func(http.ResponseWriter, *http.Request) {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
data, err := ioutil.ReadAll(r.Body)
|
||||
if err != nil {
|
||||
sendError(w, err, http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
var req TxnRequest
|
||||
if err = json.Unmarshal(data, &req); err != nil {
|
||||
sendError(w, err, http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
ctx, cancel := reqContext(mux.Vars(r))
|
||||
res, status, err := f(ctx, &req)
|
||||
cancel()
|
||||
if err != nil {
|
||||
sendError(w, err, status)
|
||||
return
|
||||
}
|
||||
data, err = json.Marshal(res)
|
||||
if err != nil {
|
||||
sendError(w, err, http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
w.WriteHeader(status)
|
||||
w.Write(data)
|
||||
}
|
||||
}
|
||||
|
||||
func (h txnkvHandler) getConfig() config.Config {
|
||||
if h.c == nil {
|
||||
return config.Default()
|
||||
}
|
||||
return *h.c
|
||||
}
|
||||
141
proxy/rawkv.go
141
proxy/rawkv.go
|
|
@ -1,141 +0,0 @@
|
|||
// 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 proxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/rawkv"
|
||||
)
|
||||
|
||||
// RawKVProxy implements proxy to use rawkv API.
|
||||
// It is safe to copy by value or access concurrently.
|
||||
type RawKVProxy struct {
|
||||
clients *sync.Map
|
||||
}
|
||||
|
||||
// NewRaw creates a RawKVProxy instance.
|
||||
func NewRaw() RawKVProxy {
|
||||
return RawKVProxy{
|
||||
clients: &sync.Map{},
|
||||
}
|
||||
}
|
||||
|
||||
// New creates a new client and returns the client's UUID.
|
||||
func (p RawKVProxy) New(ctx context.Context, pdAddrs []string, conf config.Config) (UUID, error) {
|
||||
client, err := rawkv.NewClient(ctx, pdAddrs, conf)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return insertWithRetry(p.clients, client), nil
|
||||
}
|
||||
|
||||
// Close releases a rawkv client.
|
||||
func (p RawKVProxy) Close(ctx context.Context) error {
|
||||
id := uuidFromContext(ctx)
|
||||
client, ok := p.clients.Load(id)
|
||||
if !ok {
|
||||
return errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
if err := client.(*rawkv.Client).Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
p.clients.Delete(id)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get queries value with the key.
|
||||
func (p RawKVProxy) Get(ctx context.Context, key []byte) ([]byte, error) {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return nil, errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).Get(ctx, key)
|
||||
}
|
||||
|
||||
// BatchGet queries values with the keys.
|
||||
func (p RawKVProxy) BatchGet(ctx context.Context, keys [][]byte) ([][]byte, error) {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return nil, errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).BatchGet(ctx, keys)
|
||||
}
|
||||
|
||||
// Put stores a key-value pair to TiKV.
|
||||
func (p RawKVProxy) Put(ctx context.Context, key, value []byte) error {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).Put(ctx, key, value)
|
||||
}
|
||||
|
||||
// BatchPut stores key-value pairs to TiKV.
|
||||
func (p RawKVProxy) BatchPut(ctx context.Context, keys, values [][]byte) error {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).BatchPut(ctx, keys, values)
|
||||
}
|
||||
|
||||
// Delete deletes a key-value pair from TiKV.
|
||||
func (p RawKVProxy) Delete(ctx context.Context, key []byte) error {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).Delete(ctx, key)
|
||||
}
|
||||
|
||||
// BatchDelete deletes key-value pairs from TiKV.
|
||||
func (p RawKVProxy) BatchDelete(ctx context.Context, keys [][]byte) error {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).BatchDelete(ctx, keys)
|
||||
}
|
||||
|
||||
// DeleteRange deletes all key-value pairs in a range from TiKV.
|
||||
func (p RawKVProxy) DeleteRange(ctx context.Context, startKey []byte, endKey []byte) error {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).DeleteRange(ctx, startKey, endKey)
|
||||
}
|
||||
|
||||
// Scan queries continuous kv pairs in range [startKey, endKey), up to limit pairs.
|
||||
func (p RawKVProxy) Scan(ctx context.Context, startKey, endKey []byte, limit int) ([][]byte, [][]byte, error) {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return nil, nil, errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).Scan(ctx, startKey, endKey, limit)
|
||||
}
|
||||
|
||||
// ReverseScan queries continuous kv pairs in range [endKey, startKey), up to limit pairs.
|
||||
// Direction is different from Scan, upper to lower.
|
||||
func (p RawKVProxy) ReverseScan(ctx context.Context, startKey, endKey []byte, limit int) ([][]byte, [][]byte, error) {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return nil, nil, errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*rawkv.Client).ReverseScan(ctx, startKey, endKey, limit)
|
||||
}
|
||||
278
proxy/txnkv.go
278
proxy/txnkv.go
|
|
@ -1,278 +0,0 @@
|
|||
// 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 proxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"unsafe"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/key"
|
||||
"github.com/tikv/client-go/txnkv"
|
||||
"github.com/tikv/client-go/txnkv/kv"
|
||||
)
|
||||
|
||||
// TxnKVProxy implements proxy to use txnkv API.
|
||||
// It is safe to copy by value or access concurrently.
|
||||
type TxnKVProxy struct {
|
||||
clients *sync.Map
|
||||
txns *sync.Map
|
||||
iterators *sync.Map
|
||||
}
|
||||
|
||||
// NewTxn creates a TxnKVProxy instance.
|
||||
func NewTxn() TxnKVProxy {
|
||||
return TxnKVProxy{
|
||||
clients: &sync.Map{},
|
||||
txns: &sync.Map{},
|
||||
iterators: &sync.Map{},
|
||||
}
|
||||
}
|
||||
|
||||
// New creates a new client and returns the client's UUID.
|
||||
func (p TxnKVProxy) New(ctx context.Context, pdAddrs []string, conf config.Config) (UUID, error) {
|
||||
client, err := txnkv.NewClient(ctx, pdAddrs, conf)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return insertWithRetry(p.clients, client), nil
|
||||
}
|
||||
|
||||
// Close releases a txnkv client.
|
||||
func (p TxnKVProxy) Close(ctx context.Context) error {
|
||||
id := uuidFromContext(ctx)
|
||||
client, ok := p.clients.Load(id)
|
||||
if !ok {
|
||||
return errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
if err := client.(*txnkv.Client).Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
p.clients.Delete(id)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Begin starts a new transaction and returns its UUID.
|
||||
func (p TxnKVProxy) Begin(ctx context.Context) (UUID, error) {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return "", errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
txn, err := client.(*txnkv.Client).Begin(ctx)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return insertWithRetry(p.txns, txn), nil
|
||||
}
|
||||
|
||||
// BeginWithTS starts a new transaction with given ts and returns its UUID.
|
||||
func (p TxnKVProxy) BeginWithTS(ctx context.Context, ts uint64) (UUID, error) {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return "", errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return insertWithRetry(p.txns, client.(*txnkv.Client).BeginWithTS(ctx, ts)), nil
|
||||
}
|
||||
|
||||
// GetTS returns a latest timestamp.
|
||||
func (p TxnKVProxy) GetTS(ctx context.Context) (uint64, error) {
|
||||
client, ok := p.clients.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return 0, errors.WithStack(ErrClientNotFound)
|
||||
}
|
||||
return client.(*txnkv.Client).GetTS(ctx)
|
||||
}
|
||||
|
||||
// TxnGet queries value for the given key from TiKV server.
|
||||
func (p TxnKVProxy) TxnGet(ctx context.Context, key []byte) ([]byte, error) {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return nil, errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
return txn.(*txnkv.Transaction).Get(ctx, key)
|
||||
}
|
||||
|
||||
// TxnBatchGet gets a batch of values from TiKV server.
|
||||
func (p TxnKVProxy) TxnBatchGet(ctx context.Context, keys [][]byte) (map[string][]byte, error) {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return nil, errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
ks := *(*[]key.Key)(unsafe.Pointer(&keys))
|
||||
return txn.(*txnkv.Transaction).BatchGet(ctx, ks)
|
||||
}
|
||||
|
||||
// TxnSet sets the value for key k as v into TiKV server.
|
||||
func (p TxnKVProxy) TxnSet(ctx context.Context, k []byte, v []byte) error {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
return txn.(*txnkv.Transaction).Set(k, v)
|
||||
}
|
||||
|
||||
// TxnIter creates an Iterator positioned on the first entry that key <= entry's
|
||||
// key and returns the Iterator's UUID.
|
||||
func (p TxnKVProxy) TxnIter(ctx context.Context, key []byte, upperBound []byte) (UUID, error) {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return "", errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
iter, err := txn.(*txnkv.Transaction).Iter(ctx, key, upperBound)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return insertWithRetry(p.iterators, iter), nil
|
||||
}
|
||||
|
||||
// TxnIterReverse creates a reversed Iterator positioned on the first entry
|
||||
// which key is less than key and returns the Iterator's UUID.
|
||||
func (p TxnKVProxy) TxnIterReverse(ctx context.Context, key []byte) (UUID, error) {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return "", errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
iter, err := txn.(*txnkv.Transaction).IterReverse(ctx, key)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return insertWithRetry(p.iterators, iter), nil
|
||||
}
|
||||
|
||||
// TxnIsReadOnly returns if there are pending key-value to commit in the transaction.
|
||||
func (p TxnKVProxy) TxnIsReadOnly(ctx context.Context) (bool, error) {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return false, errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
return txn.(*txnkv.Transaction).IsReadOnly(), nil
|
||||
}
|
||||
|
||||
// TxnDelete removes the entry for key from TiKV server.
|
||||
func (p TxnKVProxy) TxnDelete(ctx context.Context, key []byte) error {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
return txn.(*txnkv.Transaction).Delete(key)
|
||||
}
|
||||
|
||||
// TxnCommit commits the transaction operations to TiKV server.
|
||||
func (p TxnKVProxy) TxnCommit(ctx context.Context) error {
|
||||
id := uuidFromContext(ctx)
|
||||
txn, ok := p.txns.Load(id)
|
||||
if !ok {
|
||||
return errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
defer p.txns.Delete(id)
|
||||
return txn.(*txnkv.Transaction).Commit(context.Background())
|
||||
}
|
||||
|
||||
// TxnRollback undoes the transaction operations to TiKV server.
|
||||
func (p TxnKVProxy) TxnRollback(ctx context.Context) error {
|
||||
id := uuidFromContext(ctx)
|
||||
txn, ok := p.txns.Load(id)
|
||||
if !ok {
|
||||
return errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
defer p.txns.Delete(id)
|
||||
return txn.(*txnkv.Transaction).Rollback()
|
||||
}
|
||||
|
||||
// TxnLockKeys tries to lock the entries with the keys in TiKV server.
|
||||
func (p TxnKVProxy) TxnLockKeys(ctx context.Context, keys [][]byte) error {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
ks := *(*[]key.Key)(unsafe.Pointer(&keys))
|
||||
return txn.(*txnkv.Transaction).LockKeys(ks...)
|
||||
}
|
||||
|
||||
// TxnValid returns if the transaction is valid.
|
||||
func (p TxnKVProxy) TxnValid(ctx context.Context) (bool, error) {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return false, errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
return txn.(*txnkv.Transaction).Valid(), nil
|
||||
}
|
||||
|
||||
// TxnLen returns the count of key-value pairs in the transaction's memory buffer.
|
||||
func (p TxnKVProxy) TxnLen(ctx context.Context) (int, error) {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return 0, errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
return txn.(*txnkv.Transaction).Len(), nil
|
||||
}
|
||||
|
||||
// TxnSize returns the length (in bytes) of the transaction's memory buffer.
|
||||
func (p TxnKVProxy) TxnSize(ctx context.Context) (int, error) {
|
||||
txn, ok := p.txns.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return 0, errors.WithStack(ErrTxnNotFound)
|
||||
}
|
||||
return txn.(*txnkv.Transaction).Size(), nil
|
||||
}
|
||||
|
||||
// IterValid returns if the iterator is valid to use.
|
||||
func (p TxnKVProxy) IterValid(ctx context.Context) (bool, error) {
|
||||
iter, ok := p.iterators.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return false, errors.WithStack(ErrIterNotFound)
|
||||
}
|
||||
return iter.(kv.Iterator).Valid(), nil
|
||||
}
|
||||
|
||||
// IterKey returns the key which the iterator points to.
|
||||
func (p TxnKVProxy) IterKey(ctx context.Context) ([]byte, error) {
|
||||
iter, ok := p.iterators.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return nil, errors.WithStack(ErrIterNotFound)
|
||||
}
|
||||
return iter.(kv.Iterator).Key(), nil
|
||||
}
|
||||
|
||||
// IterValue returns the value which the iterator points to.
|
||||
func (p TxnKVProxy) IterValue(ctx context.Context) ([]byte, error) {
|
||||
iter, ok := p.iterators.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return nil, errors.WithStack(ErrIterNotFound)
|
||||
}
|
||||
return iter.(kv.Iterator).Value(), nil
|
||||
}
|
||||
|
||||
// IterNext moves the iterator to next entry.
|
||||
func (p TxnKVProxy) IterNext(ctx context.Context) error {
|
||||
iter, ok := p.iterators.Load(uuidFromContext(ctx))
|
||||
if !ok {
|
||||
return errors.WithStack(ErrIterNotFound)
|
||||
}
|
||||
return iter.(kv.Iterator).Next(ctx)
|
||||
}
|
||||
|
||||
// IterClose releases an iterator.
|
||||
func (p TxnKVProxy) IterClose(ctx context.Context) error {
|
||||
id := uuidFromContext(ctx)
|
||||
iter, ok := p.iterators.Load(id)
|
||||
if !ok {
|
||||
return errors.WithStack(ErrIterNotFound)
|
||||
}
|
||||
iter.(kv.Iterator).Close()
|
||||
p.iterators.Delete(id)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -1,52 +0,0 @@
|
|||
// 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 proxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"sync"
|
||||
|
||||
"github.com/google/uuid"
|
||||
)
|
||||
|
||||
// Proxy errors. Use errors.Cause() to determine error type.
|
||||
var (
|
||||
ErrClientNotFound = errors.New("client not found")
|
||||
ErrTxnNotFound = errors.New("txn not found")
|
||||
ErrIterNotFound = errors.New("iterator not found")
|
||||
)
|
||||
|
||||
type ContextKey int
|
||||
|
||||
var UUIDKey ContextKey = 1
|
||||
|
||||
// UUID is a global unique ID to identify clients, transactions, or iterators.
|
||||
type UUID string
|
||||
|
||||
func insertWithRetry(m *sync.Map, d interface{}) UUID {
|
||||
for {
|
||||
id := UUID(uuid.New().String())
|
||||
if _, hasOld := m.LoadOrStore(id, d); !hasOld {
|
||||
return id
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func uuidFromContext(ctx context.Context) UUID {
|
||||
if id := ctx.Value(UUIDKey); id != nil {
|
||||
return id.(UUID)
|
||||
}
|
||||
return ""
|
||||
}
|
||||
721
rawkv/rawkv.go
721
rawkv/rawkv.go
|
|
@ -1,721 +0,0 @@
|
|||
// Copyright 2018 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 rawkv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/locate"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"github.com/tikv/client-go/rpc"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
||||
var (
|
||||
// ErrMaxScanLimitExceeded is returned when the limit for rawkv Scan is to large.
|
||||
ErrMaxScanLimitExceeded = errors.New("limit should be less than MaxRawKVScanLimit")
|
||||
)
|
||||
|
||||
// ScanOption is used to provide additional information for scaning operation.
|
||||
type ScanOption struct {
|
||||
KeyOnly bool // if true, the result will only contains keys
|
||||
}
|
||||
|
||||
// PutOptions is used to provide additional information for put operation.
|
||||
type PutOption struct {
|
||||
TTL uint64
|
||||
}
|
||||
|
||||
func DefaultScanOption() ScanOption {
|
||||
return ScanOption{
|
||||
KeyOnly: false,
|
||||
}
|
||||
}
|
||||
|
||||
// Client is a rawkv client of TiKV server which is used as a key-value storage,
|
||||
// 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(ctx context.Context, pdAddrs []string, conf config.Config) (*Client, error) {
|
||||
pdCli, err := pd.NewClient(pdAddrs, pd.SecurityOption{
|
||||
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(ctx),
|
||||
conf: &conf,
|
||||
regionCache: locate.NewRegionCache(pdCli, &conf.RegionCache),
|
||||
pdClient: pdCli,
|
||||
rpcClient: rpc.NewRPCClient(&conf.RPC),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Close closes the client.
|
||||
func (c *Client) Close() error {
|
||||
c.pdClient.Close()
|
||||
return c.rpcClient.Close()
|
||||
}
|
||||
|
||||
// ClusterID returns the TiKV cluster ID.
|
||||
func (c *Client) ClusterID() uint64 {
|
||||
return c.clusterID
|
||||
}
|
||||
|
||||
// Get queries value with the key. When the key does not exist, it returns `nil, nil`.
|
||||
func (c *Client) Get(ctx context.Context, key []byte) ([]byte, error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("get").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdRawGet,
|
||||
RawGet: &kvrpcpb.RawGetRequest{
|
||||
Key: key,
|
||||
},
|
||||
}
|
||||
resp, _, err := c.sendReq(ctx, key, req)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
cmdResp := resp.RawGet
|
||||
if cmdResp == nil {
|
||||
return nil, errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if cmdResp.GetError() != "" {
|
||||
return nil, errors.New(cmdResp.GetError())
|
||||
}
|
||||
if len(cmdResp.Value) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
return cmdResp.Value, nil
|
||||
}
|
||||
|
||||
|
||||
// Get queries value with the key. When the key does not exist, it returns `nil, nil`.
|
||||
func (c *Client) GetKeyTTL(ctx context.Context, key []byte) (*uint64, error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("get").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdRawGetKeyTTL,
|
||||
RawGetKeyTTL: &kvrpcpb.RawGetKeyTTLRequest{
|
||||
Key: key,
|
||||
},
|
||||
}
|
||||
resp, _, err := c.sendReq(ctx, key, req)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
cmdResp := resp.RawGetKeyTTL
|
||||
if cmdResp == nil {
|
||||
return nil, errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if cmdResp.GetError() != "" {
|
||||
return nil, errors.New(cmdResp.GetError())
|
||||
}
|
||||
if cmdResp.NotFound {
|
||||
return nil, nil
|
||||
}
|
||||
return &cmdResp.Ttl, nil
|
||||
}
|
||||
|
||||
// BatchGet queries values with the keys.
|
||||
func (c *Client) BatchGet(ctx context.Context, keys [][]byte) ([][]byte, error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("batch_get").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
bo := retry.NewBackoffer(ctx, retry.RawkvMaxBackoff)
|
||||
resp, err := c.sendBatchReq(bo, keys, rpc.CmdRawBatchGet)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
cmdResp := resp.RawBatchGet
|
||||
if cmdResp == nil {
|
||||
return nil, errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
|
||||
keyToValue := make(map[string][]byte, len(keys))
|
||||
for _, pair := range cmdResp.Pairs {
|
||||
keyToValue[string(pair.Key)] = pair.Value
|
||||
}
|
||||
|
||||
values := make([][]byte, len(keys))
|
||||
for i, key := range keys {
|
||||
values[i] = keyToValue[string(key)]
|
||||
}
|
||||
return values, nil
|
||||
}
|
||||
|
||||
// Put stores a key-value pair to TiKV.
|
||||
func (c *Client) Put(ctx context.Context, key, value []byte, options ...PutOption) error {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("put").Observe(time.Since(start).Seconds()) }()
|
||||
metrics.RawkvSizeHistogram.WithLabelValues("key").Observe(float64(len(key)))
|
||||
metrics.RawkvSizeHistogram.WithLabelValues("value").Observe(float64(len(value)))
|
||||
|
||||
if len(value) == 0 {
|
||||
return errors.New("empty value is not supported")
|
||||
}
|
||||
|
||||
var ttl uint64 = 0
|
||||
if options != nil && len(options) > 0 {
|
||||
ttl = options[0].TTL
|
||||
}
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdRawPut,
|
||||
RawPut: &kvrpcpb.RawPutRequest{
|
||||
Key: key,
|
||||
Value: value,
|
||||
Ttl: ttl,
|
||||
},
|
||||
}
|
||||
resp, _, err := c.sendReq(ctx, key, req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
cmdResp := resp.RawPut
|
||||
if cmdResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if cmdResp.GetError() != "" {
|
||||
return errors.New(cmdResp.GetError())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// BatchPut stores key-value pairs to TiKV.
|
||||
func (c *Client) BatchPut(ctx context.Context, keys, values [][]byte, options ...PutOption) error {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("batch_put").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
var ttl uint64 = 0
|
||||
if options != nil && len(options) > 0 {
|
||||
ttl = options[0].TTL
|
||||
}
|
||||
|
||||
if len(keys) != len(values) {
|
||||
return errors.New("the len of keys is not equal to the len of values")
|
||||
}
|
||||
for _, value := range values {
|
||||
if len(value) == 0 {
|
||||
return errors.New("empty value is not supported")
|
||||
}
|
||||
}
|
||||
bo := retry.NewBackoffer(ctx, retry.RawkvMaxBackoff)
|
||||
return c.sendBatchPut(bo, keys, values, ttl)
|
||||
}
|
||||
|
||||
// Delete deletes a key-value pair from TiKV.
|
||||
func (c *Client) Delete(ctx context.Context, key []byte) error {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("delete").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdRawDelete,
|
||||
RawDelete: &kvrpcpb.RawDeleteRequest{
|
||||
Key: key,
|
||||
},
|
||||
}
|
||||
resp, _, err := c.sendReq(ctx, key, req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
cmdResp := resp.RawDelete
|
||||
if cmdResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if cmdResp.GetError() != "" {
|
||||
return errors.New(cmdResp.GetError())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// BatchDelete deletes key-value pairs from TiKV.
|
||||
func (c *Client) BatchDelete(ctx context.Context, keys [][]byte) error {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("batch_delete").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
bo := retry.NewBackoffer(ctx, retry.RawkvMaxBackoff)
|
||||
resp, err := c.sendBatchReq(bo, keys, rpc.CmdRawBatchDelete)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
cmdResp := resp.RawBatchDelete
|
||||
if cmdResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if cmdResp.GetError() != "" {
|
||||
return errors.New(cmdResp.GetError())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// DeleteRange deletes all key-value pairs in a range from TiKV
|
||||
func (c *Client) DeleteRange(ctx context.Context, startKey []byte, endKey []byte) error {
|
||||
start := time.Now()
|
||||
var err error
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("delete_range").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
// Process each affected region respectively
|
||||
for !bytes.Equal(startKey, endKey) {
|
||||
var resp *rpc.Response
|
||||
var actualEndKey []byte
|
||||
resp, actualEndKey, err = c.sendDeleteRangeReq(ctx, startKey, endKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
cmdResp := resp.RawDeleteRange
|
||||
if cmdResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if cmdResp.GetError() != "" {
|
||||
return errors.New(cmdResp.GetError())
|
||||
}
|
||||
startKey = actualEndKey
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Scan queries continuous kv pairs in range [startKey, endKey), up to limit pairs.
|
||||
// If endKey is empty, it means unbounded.
|
||||
// If you want to exclude the startKey or include the endKey, append a '\0' to the key. For example, to scan
|
||||
// (startKey, endKey], you can write:
|
||||
// `Scan(ctx, append(startKey, '\x00'), append(endKey, '\x00'), limit)`.
|
||||
func (c *Client) Scan(ctx context.Context, startKey, endKey []byte, limit int, options ...ScanOption) (keys [][]byte, values [][]byte, err error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("scan").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
var option ScanOption
|
||||
if options == nil || len(options) == 0 {
|
||||
option = DefaultScanOption()
|
||||
} else {
|
||||
option = options[0]
|
||||
}
|
||||
|
||||
if limit > c.conf.Raw.MaxScanLimit {
|
||||
return nil, nil, errors.WithStack(ErrMaxScanLimitExceeded)
|
||||
}
|
||||
|
||||
for len(keys) < limit && (len(endKey) == 0 || bytes.Compare(startKey, endKey) < 0) {
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdRawScan,
|
||||
RawScan: &kvrpcpb.RawScanRequest{
|
||||
StartKey: startKey,
|
||||
EndKey: endKey,
|
||||
Limit: uint32(limit - len(keys)),
|
||||
KeyOnly: option.KeyOnly,
|
||||
},
|
||||
}
|
||||
resp, loc, err := c.sendReq(ctx, startKey, req)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
cmdResp := resp.RawScan
|
||||
if cmdResp == nil {
|
||||
return nil, nil, errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
for _, pair := range cmdResp.Kvs {
|
||||
keys = append(keys, pair.Key)
|
||||
values = append(values, pair.Value)
|
||||
}
|
||||
startKey = loc.EndKey
|
||||
if len(startKey) == 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// ReverseScan queries continuous kv pairs in range [endKey, startKey), up to limit pairs.
|
||||
// Direction is different from Scan, upper to lower.
|
||||
// If endKey is empty, it means unbounded.
|
||||
// If you want to include the startKey or exclude the endKey, append a '\0' to the key. For example, to scan
|
||||
// (endKey, startKey], you can write:
|
||||
// `ReverseScan(append(startKey, '\0'), append(endKey, '\0'), limit)`.
|
||||
// It doesn't support Scanning from "", because locating the last Region is not yet implemented.
|
||||
func (c *Client) ReverseScan(ctx context.Context, startKey, endKey []byte, limit int, options ...ScanOption) (keys [][]byte, values [][]byte, err error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.RawkvCmdHistogram.WithLabelValues("reverse_scan").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
var option ScanOption
|
||||
if options == nil || len(options) == 0 {
|
||||
option = DefaultScanOption()
|
||||
} else {
|
||||
option = options[0]
|
||||
}
|
||||
|
||||
if limit > c.conf.Raw.MaxScanLimit {
|
||||
return nil, nil, errors.WithStack(ErrMaxScanLimitExceeded)
|
||||
}
|
||||
|
||||
for len(keys) < limit && bytes.Compare(startKey, endKey) > 0 {
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdRawScan,
|
||||
RawScan: &kvrpcpb.RawScanRequest{
|
||||
StartKey: startKey,
|
||||
EndKey: endKey,
|
||||
Limit: uint32(limit - len(keys)),
|
||||
Reverse: true,
|
||||
KeyOnly: option.KeyOnly,
|
||||
},
|
||||
}
|
||||
resp, loc, err := c.sendReq(ctx, startKey, req)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
cmdResp := resp.RawScan
|
||||
if cmdResp == nil {
|
||||
return nil, nil, errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
for _, pair := range cmdResp.Kvs {
|
||||
keys = append(keys, pair.Key)
|
||||
values = append(values, pair.Value)
|
||||
}
|
||||
startKey = loc.EndKey
|
||||
if len(startKey) == 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (c *Client) sendReq(ctx context.Context, key []byte, req *rpc.Request) (*rpc.Response, *locate.KeyLocation, error) {
|
||||
bo := retry.NewBackoffer(ctx, retry.RawkvMaxBackoff)
|
||||
sender := rpc.NewRegionRequestSender(c.regionCache, c.rpcClient)
|
||||
for {
|
||||
loc, err := c.regionCache.LocateKey(bo, key)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
resp, err := sender.SendReq(bo, req, loc.Region, c.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
continue
|
||||
}
|
||||
return resp, loc, nil
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Client) sendBatchReq(bo *retry.Backoffer, keys [][]byte, cmdType rpc.CmdType) (*rpc.Response, error) { // split the keys
|
||||
groups, _, err := c.regionCache.GroupKeysByRegion(bo, keys)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var batches []batch
|
||||
for regionID, groupKeys := range groups {
|
||||
batches = appendKeyBatches(batches, regionID, groupKeys, c.conf.Raw.BatchPairCount)
|
||||
}
|
||||
bo, cancel := bo.Fork()
|
||||
ches := make(chan singleBatchResp, len(batches))
|
||||
for _, batch := range batches {
|
||||
batch1 := batch
|
||||
go func() {
|
||||
singleBatchBackoffer, singleBatchCancel := bo.Fork()
|
||||
defer singleBatchCancel()
|
||||
ches <- c.doBatchReq(singleBatchBackoffer, batch1, cmdType)
|
||||
}()
|
||||
}
|
||||
|
||||
var firstError error
|
||||
var resp *rpc.Response
|
||||
switch cmdType {
|
||||
case rpc.CmdRawBatchGet:
|
||||
resp = &rpc.Response{Type: rpc.CmdRawBatchGet, RawBatchGet: &kvrpcpb.RawBatchGetResponse{}}
|
||||
case rpc.CmdRawBatchDelete:
|
||||
resp = &rpc.Response{Type: rpc.CmdRawBatchDelete, RawBatchDelete: &kvrpcpb.RawBatchDeleteResponse{}}
|
||||
}
|
||||
for i := 0; i < len(batches); i++ {
|
||||
singleResp, ok := <-ches
|
||||
if ok {
|
||||
if singleResp.err != nil {
|
||||
cancel()
|
||||
if firstError == nil {
|
||||
firstError = singleResp.err
|
||||
}
|
||||
} else if cmdType == rpc.CmdRawBatchGet {
|
||||
cmdResp := singleResp.resp.RawBatchGet
|
||||
resp.RawBatchGet.Pairs = append(resp.RawBatchGet.Pairs, cmdResp.Pairs...)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return resp, firstError
|
||||
}
|
||||
|
||||
func (c *Client) doBatchReq(bo *retry.Backoffer, batch batch, cmdType rpc.CmdType) singleBatchResp {
|
||||
var req *rpc.Request
|
||||
switch cmdType {
|
||||
case rpc.CmdRawBatchGet:
|
||||
req = &rpc.Request{
|
||||
Type: cmdType,
|
||||
RawBatchGet: &kvrpcpb.RawBatchGetRequest{
|
||||
Keys: batch.keys,
|
||||
},
|
||||
}
|
||||
case rpc.CmdRawBatchDelete:
|
||||
req = &rpc.Request{
|
||||
Type: cmdType,
|
||||
RawBatchDelete: &kvrpcpb.RawBatchDeleteRequest{
|
||||
Keys: batch.keys,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
sender := rpc.NewRegionRequestSender(c.regionCache, c.rpcClient)
|
||||
resp, err := sender.SendReq(bo, req, batch.regionID, c.conf.RPC.ReadTimeoutShort)
|
||||
|
||||
batchResp := singleBatchResp{}
|
||||
if err != nil {
|
||||
batchResp.err = err
|
||||
return batchResp
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
batchResp.err = err
|
||||
return batchResp
|
||||
}
|
||||
if regionErr != nil {
|
||||
err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
batchResp.err = err
|
||||
return batchResp
|
||||
}
|
||||
resp, err = c.sendBatchReq(bo, batch.keys, cmdType)
|
||||
batchResp.resp = resp
|
||||
batchResp.err = err
|
||||
return batchResp
|
||||
}
|
||||
|
||||
switch cmdType {
|
||||
case rpc.CmdRawBatchGet:
|
||||
batchResp.resp = resp
|
||||
case rpc.CmdRawBatchDelete:
|
||||
cmdResp := resp.RawBatchDelete
|
||||
if cmdResp == nil {
|
||||
batchResp.err = errors.WithStack(rpc.ErrBodyMissing)
|
||||
return batchResp
|
||||
}
|
||||
if cmdResp.GetError() != "" {
|
||||
batchResp.err = errors.New(cmdResp.GetError())
|
||||
return batchResp
|
||||
}
|
||||
batchResp.resp = resp
|
||||
}
|
||||
return batchResp
|
||||
}
|
||||
|
||||
// sendDeleteRangeReq sends a raw delete range request and returns the response and the actual endKey.
|
||||
// If the given range spans over more than one regions, the actual endKey is the end of the first region.
|
||||
// We can't use sendReq directly, because we need to know the end of the region before we send the request
|
||||
// TODO: Is there any better way to avoid duplicating code with func `sendReq` ?
|
||||
func (c *Client) sendDeleteRangeReq(ctx context.Context, startKey []byte, endKey []byte) (*rpc.Response, []byte, error) {
|
||||
bo := retry.NewBackoffer(ctx, retry.RawkvMaxBackoff)
|
||||
sender := rpc.NewRegionRequestSender(c.regionCache, c.rpcClient)
|
||||
for {
|
||||
loc, err := c.regionCache.LocateKey(bo, startKey)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
actualEndKey := endKey
|
||||
if len(loc.EndKey) > 0 && bytes.Compare(loc.EndKey, endKey) < 0 {
|
||||
actualEndKey = loc.EndKey
|
||||
}
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdRawDeleteRange,
|
||||
RawDeleteRange: &kvrpcpb.RawDeleteRangeRequest{
|
||||
StartKey: startKey,
|
||||
EndKey: actualEndKey,
|
||||
},
|
||||
}
|
||||
|
||||
resp, err := sender.SendReq(bo, req, loc.Region, c.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
continue
|
||||
}
|
||||
return resp, actualEndKey, nil
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Client) sendBatchPut(bo *retry.Backoffer, keys, values [][]byte, ttl uint64) error {
|
||||
keyToValue := make(map[string][]byte)
|
||||
for i, key := range keys {
|
||||
keyToValue[string(key)] = values[i]
|
||||
}
|
||||
groups, _, err := c.regionCache.GroupKeysByRegion(bo, keys)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var batches []batch
|
||||
// split the keys by size and RegionVerID
|
||||
for regionID, groupKeys := range groups {
|
||||
batches = appendBatches(batches, regionID, groupKeys, keyToValue, c.conf.Raw.MaxBatchPutSize)
|
||||
}
|
||||
bo, cancel := bo.Fork()
|
||||
ch := make(chan error, len(batches))
|
||||
for _, batch := range batches {
|
||||
batch1 := batch
|
||||
go func() {
|
||||
singleBatchBackoffer, singleBatchCancel := bo.Fork()
|
||||
defer singleBatchCancel()
|
||||
ch <- c.doBatchPut(singleBatchBackoffer, batch1, ttl)
|
||||
}()
|
||||
}
|
||||
|
||||
for i := 0; i < len(batches); i++ {
|
||||
if e := <-ch; e != nil {
|
||||
cancel()
|
||||
// catch the first error
|
||||
if err == nil {
|
||||
err = e
|
||||
}
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func appendKeyBatches(batches []batch, regionID locate.RegionVerID, groupKeys [][]byte, limit int) []batch {
|
||||
var keys [][]byte
|
||||
for start, count := 0, 0; start < len(groupKeys); start++ {
|
||||
if count > limit {
|
||||
batches = append(batches, batch{regionID: regionID, keys: keys})
|
||||
keys = make([][]byte, 0, limit)
|
||||
count = 0
|
||||
}
|
||||
keys = append(keys, groupKeys[start])
|
||||
count++
|
||||
}
|
||||
if len(keys) != 0 {
|
||||
batches = append(batches, batch{regionID: regionID, keys: keys})
|
||||
}
|
||||
return batches
|
||||
}
|
||||
|
||||
func appendBatches(batches []batch, regionID locate.RegionVerID, groupKeys [][]byte, keyToValue map[string][]byte, limit int) []batch {
|
||||
var start, size int
|
||||
var keys, values [][]byte
|
||||
for start = 0; start < len(groupKeys); start++ {
|
||||
if size >= limit {
|
||||
batches = append(batches, batch{regionID: regionID, keys: keys, values: values})
|
||||
keys = make([][]byte, 0)
|
||||
values = make([][]byte, 0)
|
||||
size = 0
|
||||
}
|
||||
key := groupKeys[start]
|
||||
value := keyToValue[string(key)]
|
||||
keys = append(keys, key)
|
||||
values = append(values, value)
|
||||
size += len(key)
|
||||
size += len(value)
|
||||
}
|
||||
if len(keys) != 0 {
|
||||
batches = append(batches, batch{regionID: regionID, keys: keys, values: values})
|
||||
}
|
||||
return batches
|
||||
}
|
||||
|
||||
func (c *Client) doBatchPut(bo *retry.Backoffer, batch batch, ttl uint64) error {
|
||||
kvPair := make([]*kvrpcpb.KvPair, 0, len(batch.keys))
|
||||
for i, key := range batch.keys {
|
||||
kvPair = append(kvPair, &kvrpcpb.KvPair{Key: key, Value: batch.values[i]})
|
||||
}
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdRawBatchPut,
|
||||
RawBatchPut: &kvrpcpb.RawBatchPutRequest{
|
||||
Pairs: kvPair,
|
||||
Ttl: ttl,
|
||||
},
|
||||
}
|
||||
|
||||
sender := rpc.NewRegionRequestSender(c.regionCache, c.rpcClient)
|
||||
resp, err := sender.SendReq(bo, req, batch.regionID, c.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// recursive call
|
||||
return c.sendBatchPut(bo, batch.keys, batch.values, ttl)
|
||||
}
|
||||
|
||||
cmdResp := resp.RawBatchPut
|
||||
if cmdResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if cmdResp.GetError() != "" {
|
||||
return errors.New(cmdResp.GetError())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type batch struct {
|
||||
regionID locate.RegionVerID
|
||||
keys [][]byte
|
||||
values [][]byte
|
||||
}
|
||||
|
||||
type singleBatchResp struct {
|
||||
resp *rpc.Response
|
||||
err error
|
||||
}
|
||||
|
|
@ -1,357 +0,0 @@
|
|||
// Copyright 2016 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 rawkv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/locate"
|
||||
"github.com/tikv/client-go/mockstore/mocktikv"
|
||||
"github.com/tikv/client-go/retry"
|
||||
)
|
||||
|
||||
func TestT(t *testing.T) {
|
||||
TestingT(t)
|
||||
}
|
||||
|
||||
type testRawKVSuite struct {
|
||||
cluster *mocktikv.Cluster
|
||||
client *Client
|
||||
bo *retry.Backoffer
|
||||
}
|
||||
|
||||
var _ = Suite(&testRawKVSuite{})
|
||||
|
||||
func (s *testRawKVSuite) SetUpTest(c *C) {
|
||||
s.cluster = mocktikv.NewCluster()
|
||||
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, &conf.RegionCache),
|
||||
pdClient: pdClient,
|
||||
rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore),
|
||||
}
|
||||
s.bo = retry.NewBackoffer(context.Background(), 5000)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) TearDownTest(c *C) {
|
||||
s.client.Close()
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustNotExist(c *C, key []byte) {
|
||||
v, err := s.client.Get(context.TODO(), key)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(v, IsNil)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustBatchNotExist(c *C, keys [][]byte) {
|
||||
values, err := s.client.BatchGet(context.TODO(), keys)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(values, NotNil)
|
||||
c.Assert(len(keys), Equals, len(values))
|
||||
for _, value := range values {
|
||||
c.Assert([]byte{}, BytesEquals, value)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustGet(c *C, key, value []byte) {
|
||||
v, err := s.client.Get(context.TODO(), key)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(v, NotNil)
|
||||
c.Assert(v, BytesEquals, value)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustBatchGet(c *C, keys, values [][]byte) {
|
||||
checkValues, err := s.client.BatchGet(context.TODO(), keys)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(checkValues, NotNil)
|
||||
c.Assert(len(keys), Equals, len(checkValues))
|
||||
for i := range keys {
|
||||
c.Check(values[i], BytesEquals, checkValues[i])
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustPut(c *C, key, value []byte) {
|
||||
err := s.client.Put(context.TODO(), key, value)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustPutTTL(c *C, key, value []byte, ttl uint64) {
|
||||
err := s.client.Put(context.TODO(), key, value, PutOption{ TTL: ttl })
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustGetKeyTTL(c *C, key []byte, expectedTtl uint64) {
|
||||
ttl, err := s.client.GetKeyTTL(context.TODO(), key)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(*ttl, Equals, expectedTtl)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustBatchPut(c *C, keys, values [][]byte) {
|
||||
err := s.client.BatchPut(context.TODO(), keys, values)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustDelete(c *C, key []byte) {
|
||||
err := s.client.Delete(context.TODO(), key)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustBatchDelete(c *C, keys [][]byte) {
|
||||
err := s.client.BatchDelete(context.TODO(), keys)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustScanKeyOnly(c *C, startKey string, limit int, expect ...string) {
|
||||
option := DefaultScanOption()
|
||||
option.KeyOnly = true
|
||||
keys, values, err := s.client.Scan(context.TODO(), []byte(startKey), nil, limit, option)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(len(keys), Equals, len(expect))
|
||||
for i := range keys {
|
||||
c.Assert(string(keys[i]), Equals, expect[i])
|
||||
c.Assert(values[i], IsNil)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustScan(c *C, startKey string, limit int, expect ...string) {
|
||||
keys, values, err := s.client.Scan(context.TODO(), []byte(startKey), nil, limit)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(len(keys)*2, Equals, len(expect))
|
||||
for i := range keys {
|
||||
c.Assert(string(keys[i]), Equals, expect[i*2])
|
||||
c.Assert(string(values[i]), Equals, expect[i*2+1])
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustScanRange(c *C, startKey string, endKey string, limit int, expect ...string) {
|
||||
keys, values, err := s.client.Scan(context.TODO(), []byte(startKey), []byte(endKey), limit)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(len(keys)*2, Equals, len(expect))
|
||||
for i := range keys {
|
||||
c.Assert(string(keys[i]), Equals, expect[i*2])
|
||||
c.Assert(string(values[i]), Equals, expect[i*2+1])
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustReverseScanKeyOnly(c *C, startKey string, limit int, expect ...string) {
|
||||
option := DefaultScanOption()
|
||||
option.KeyOnly = true
|
||||
keys, values, err := s.client.ReverseScan(context.TODO(), []byte(startKey), nil, limit, option)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(len(keys), Equals, len(expect))
|
||||
for i := range keys {
|
||||
c.Assert(string(keys[i]), Equals, expect[i])
|
||||
c.Assert(values[i], IsNil)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustReverseScan(c *C, startKey []byte, limit int, expect ...string) {
|
||||
keys, values, err := s.client.ReverseScan(context.TODO(), startKey, nil, limit)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(len(keys)*2, Equals, len(expect))
|
||||
for i := range keys {
|
||||
c.Assert(string(keys[i]), Equals, expect[i*2])
|
||||
c.Assert(string(values[i]), Equals, expect[i*2+1])
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustReverseScanRange(c *C, startKey, endKey []byte, limit int, expect ...string) {
|
||||
keys, values, err := s.client.ReverseScan(context.TODO(), startKey, endKey, limit)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(len(keys)*2, Equals, len(expect))
|
||||
for i := range keys {
|
||||
c.Assert(string(keys[i]), Equals, expect[i*2])
|
||||
c.Assert(string(values[i]), Equals, expect[i*2+1])
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) mustDeleteRange(c *C, startKey, endKey []byte, expected map[string]string) {
|
||||
err := s.client.DeleteRange(context.TODO(), startKey, endKey)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
for keyStr := range expected {
|
||||
key := []byte(keyStr)
|
||||
if bytes.Compare(startKey, key) <= 0 && bytes.Compare(key, endKey) < 0 {
|
||||
delete(expected, keyStr)
|
||||
}
|
||||
}
|
||||
|
||||
s.checkData(c, expected)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) checkData(c *C, expected map[string]string) {
|
||||
keys, values, err := s.client.Scan(context.TODO(), []byte(""), nil, len(expected)+1)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
c.Assert(len(expected), Equals, len(keys))
|
||||
for i, key := range keys {
|
||||
c.Assert(expected[string(key)], Equals, string(values[i]))
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) split(c *C, regionKey, splitKey string) error {
|
||||
loc, err := s.client.regionCache.LocateKey(s.bo, []byte(regionKey))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
newRegionID, peerID := s.cluster.AllocID(), s.cluster.AllocID()
|
||||
s.cluster.SplitRaw(loc.Region.GetID(), newRegionID, []byte(splitKey), []uint64{peerID}, peerID)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) TestSimple(c *C) {
|
||||
s.mustNotExist(c, []byte("key"))
|
||||
s.mustPut(c, []byte("key"), []byte("value"))
|
||||
s.mustGet(c, []byte("key"), []byte("value"))
|
||||
s.mustDelete(c, []byte("key"))
|
||||
s.mustNotExist(c, []byte("key"))
|
||||
err := s.client.Put(context.TODO(), []byte("key"), []byte(""))
|
||||
c.Assert(err, NotNil)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) TestTTL(c *C) {
|
||||
s.mustNotExist(c, []byte("key"))
|
||||
s.mustPutTTL(c, []byte("key"), []byte("value"), 100)
|
||||
s.mustGetKeyTTL(c, []byte("key"), 100)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) TestRawBatch(c *C) {
|
||||
|
||||
testNum := 0
|
||||
size := 0
|
||||
var testKeys [][]byte
|
||||
var testValues [][]byte
|
||||
for i := 0; size/s.client.conf.Raw.MaxBatchPutSize < 4; i++ {
|
||||
key := fmt.Sprint("key", i)
|
||||
size += len(key)
|
||||
testKeys = append(testKeys, []byte(key))
|
||||
value := fmt.Sprint("value", i)
|
||||
size += len(value)
|
||||
testValues = append(testValues, []byte(value))
|
||||
s.mustNotExist(c, []byte(key))
|
||||
testNum = i
|
||||
}
|
||||
err := s.split(c, "", fmt.Sprint("key", testNum/2))
|
||||
c.Assert(err, IsNil)
|
||||
s.mustBatchPut(c, testKeys, testValues)
|
||||
s.mustBatchGet(c, testKeys, testValues)
|
||||
s.mustBatchDelete(c, testKeys)
|
||||
s.mustBatchNotExist(c, testKeys)
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) TestSplit(c *C) {
|
||||
s.mustPut(c, []byte("k1"), []byte("v1"))
|
||||
s.mustPut(c, []byte("k3"), []byte("v3"))
|
||||
|
||||
err := s.split(c, "k", "k2")
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
s.mustGet(c, []byte("k1"), []byte("v1"))
|
||||
s.mustGet(c, []byte("k3"), []byte("v3"))
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) TestScan(c *C) {
|
||||
s.mustPut(c, []byte("k1"), []byte("v1"))
|
||||
s.mustPut(c, []byte("k3"), []byte("v3"))
|
||||
s.mustPut(c, []byte("k5"), []byte("v5"))
|
||||
s.mustPut(c, []byte("k7"), []byte("v7"))
|
||||
|
||||
check := func() {
|
||||
s.mustScan(c, "", 1, "k1", "v1")
|
||||
s.mustScan(c, "k1", 2, "k1", "v1", "k3", "v3")
|
||||
s.mustScan(c, "", 10, "k1", "v1", "k3", "v3", "k5", "v5", "k7", "v7")
|
||||
s.mustScan(c, "k2", 2, "k3", "v3", "k5", "v5")
|
||||
s.mustScan(c, "k2", 3, "k3", "v3", "k5", "v5", "k7", "v7")
|
||||
s.mustScanRange(c, "", "k1", 1)
|
||||
s.mustScanRange(c, "k1", "k3", 2, "k1", "v1")
|
||||
s.mustScanRange(c, "k1", "k5", 10, "k1", "v1", "k3", "v3")
|
||||
s.mustScanRange(c, "k1", "k5\x00", 10, "k1", "v1", "k3", "v3", "k5", "v5")
|
||||
s.mustScanRange(c, "k5\x00", "k5\x00\x00", 10)
|
||||
}
|
||||
|
||||
check()
|
||||
|
||||
err := s.split(c, "k", "k2")
|
||||
c.Assert(err, IsNil)
|
||||
check()
|
||||
|
||||
err = s.split(c, "k2", "k5")
|
||||
c.Assert(err, IsNil)
|
||||
check()
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) TestScanWithKeyOnly(c *C) {
|
||||
s.mustPut(c, []byte("k1"), []byte("v1"))
|
||||
s.mustPut(c, []byte("k3"), []byte("v3"))
|
||||
s.mustPut(c, []byte("k5"), []byte("v5"))
|
||||
s.mustPut(c, []byte("k7"), []byte("v7"))
|
||||
|
||||
check := func() {
|
||||
s.mustScanKeyOnly(c, "", 1, "k1")
|
||||
s.mustScanKeyOnly(c, "k1", 2, "k1", "k3")
|
||||
s.mustScanKeyOnly(c, "", 10, "k1", "k3", "k5", "k7")
|
||||
s.mustScanKeyOnly(c, "k2", 2, "k3", "k5")
|
||||
s.mustScanKeyOnly(c, "k2", 3, "k3", "k5", "k7")
|
||||
}
|
||||
|
||||
check()
|
||||
|
||||
err := s.split(c, "k", "k2")
|
||||
c.Assert(err, IsNil)
|
||||
check()
|
||||
|
||||
err = s.split(c, "k2", "k5")
|
||||
c.Assert(err, IsNil)
|
||||
check()
|
||||
|
||||
}
|
||||
|
||||
func (s *testRawKVSuite) TestDeleteRange(c *C) {
|
||||
// Init data
|
||||
testData := map[string]string{}
|
||||
for _, i := range []byte("abcd") {
|
||||
for j := byte('0'); j <= byte('9'); j++ {
|
||||
key := []byte{i, j}
|
||||
value := []byte{'v', i, j}
|
||||
s.mustPut(c, key, value)
|
||||
|
||||
testData[string(key)] = string(value)
|
||||
}
|
||||
}
|
||||
|
||||
err := s.split(c, "b", "b")
|
||||
c.Assert(err, IsNil)
|
||||
err = s.split(c, "c", "c")
|
||||
c.Assert(err, IsNil)
|
||||
err = s.split(c, "d", "d")
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
s.checkData(c, testData)
|
||||
s.mustDeleteRange(c, []byte("b"), []byte("c0"), testData)
|
||||
s.mustDeleteRange(c, []byte("c11"), []byte("c12"), testData)
|
||||
s.mustDeleteRange(c, []byte("d0"), []byte("d0"), testData)
|
||||
s.mustDeleteRange(c, []byte("c5"), []byte("d5"), testData)
|
||||
s.mustDeleteRange(c, []byte("a"), []byte("z"), testData)
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
259
retry/backoff.go
259
retry/backoff.go
|
|
@ -1,259 +0,0 @@
|
|||
// Copyright 2018 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 retry
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"math/rand"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
)
|
||||
|
||||
const (
|
||||
// NoJitter makes the backoff sequence strict exponential.
|
||||
NoJitter = 1 + iota
|
||||
// FullJitter applies random factors to strict exponential.
|
||||
FullJitter
|
||||
// EqualJitter is also randomized, but prevents very short sleeps.
|
||||
EqualJitter
|
||||
// DecorrJitter increases the maximum jitter based on the last random value.
|
||||
DecorrJitter
|
||||
)
|
||||
|
||||
// NewBackoffFn creates a backoff func which implements exponential backoff with
|
||||
// optional jitters.
|
||||
// See http://www.awsarchitectureblog.com/2015/03/backoff.html
|
||||
func NewBackoffFn(base, cap, jitter int) func(ctx context.Context) int {
|
||||
if base < 2 {
|
||||
// Top prevent panic in 'rand.Intn'.
|
||||
base = 2
|
||||
}
|
||||
attempts := 0
|
||||
lastSleep := base
|
||||
return func(ctx context.Context) int {
|
||||
var sleep int
|
||||
switch jitter {
|
||||
case NoJitter:
|
||||
sleep = expo(base, cap, attempts)
|
||||
case FullJitter:
|
||||
v := expo(base, cap, attempts)
|
||||
sleep = rand.Intn(v)
|
||||
case EqualJitter:
|
||||
v := expo(base, cap, attempts)
|
||||
sleep = v/2 + rand.Intn(v/2)
|
||||
case DecorrJitter:
|
||||
sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base))))
|
||||
}
|
||||
log.Debugf("backoff base %d, sleep %d", base, sleep)
|
||||
select {
|
||||
case <-time.After(time.Duration(sleep) * time.Millisecond):
|
||||
case <-ctx.Done():
|
||||
}
|
||||
|
||||
attempts++
|
||||
lastSleep = sleep
|
||||
return lastSleep
|
||||
}
|
||||
}
|
||||
|
||||
func expo(base, cap, n int) int {
|
||||
return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n))))
|
||||
}
|
||||
|
||||
// BackoffType is the retryable error type.
|
||||
type BackoffType int
|
||||
|
||||
// Back off types.
|
||||
const (
|
||||
BoTiKVRPC BackoffType = iota
|
||||
BoTxnLock
|
||||
BoTxnLockFast
|
||||
BoPDRPC
|
||||
BoRegionMiss
|
||||
BoUpdateLeader
|
||||
BoServerBusy
|
||||
)
|
||||
|
||||
func (t BackoffType) createFn() func(context.Context) int {
|
||||
switch t {
|
||||
case BoTiKVRPC:
|
||||
return NewBackoffFn(100, 2000, EqualJitter)
|
||||
case BoTxnLock:
|
||||
return NewBackoffFn(200, 3000, EqualJitter)
|
||||
case BoTxnLockFast:
|
||||
return NewBackoffFn(50, 3000, EqualJitter)
|
||||
case BoPDRPC:
|
||||
return NewBackoffFn(500, 3000, EqualJitter)
|
||||
case BoRegionMiss:
|
||||
// change base time to 2ms, because it may recover soon.
|
||||
return NewBackoffFn(2, 500, NoJitter)
|
||||
case BoUpdateLeader:
|
||||
return NewBackoffFn(1, 10, NoJitter)
|
||||
case BoServerBusy:
|
||||
return NewBackoffFn(2000, 10000, EqualJitter)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t BackoffType) String() string {
|
||||
switch t {
|
||||
case BoTiKVRPC:
|
||||
return "tikvRPC"
|
||||
case BoTxnLock:
|
||||
return "txnLock"
|
||||
case BoTxnLockFast:
|
||||
return "txnLockFast"
|
||||
case BoPDRPC:
|
||||
return "pdRPC"
|
||||
case BoRegionMiss:
|
||||
return "regionMiss"
|
||||
case BoUpdateLeader:
|
||||
return "updateLeader"
|
||||
case BoServerBusy:
|
||||
return "serverBusy"
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// Maximum total sleep time(in ms) for kv/cop commands.
|
||||
const (
|
||||
CopBuildTaskMaxBackoff = 5000
|
||||
TsoMaxBackoff = 15000
|
||||
ScannerNextMaxBackoff = 20000
|
||||
BatchGetMaxBackoff = 20000
|
||||
CopNextMaxBackoff = 20000
|
||||
GetMaxBackoff = 20000
|
||||
PrewriteMaxBackoff = 20000
|
||||
CleanupMaxBackoff = 20000
|
||||
GcOneRegionMaxBackoff = 20000
|
||||
GcResolveLockMaxBackoff = 100000
|
||||
DeleteRangeOneRegionMaxBackoff = 100000
|
||||
RawkvMaxBackoff = 20000
|
||||
SplitRegionBackoff = 20000
|
||||
)
|
||||
|
||||
// CommitMaxBackoff is max sleep time of the 'commit' command
|
||||
var CommitMaxBackoff = 41000
|
||||
|
||||
// Backoffer is a utility for retrying queries.
|
||||
type Backoffer struct {
|
||||
ctx context.Context
|
||||
|
||||
fn map[BackoffType]func(context.Context) int
|
||||
maxSleep int
|
||||
totalSleep int
|
||||
errors []error
|
||||
types []BackoffType
|
||||
}
|
||||
|
||||
// txnStartKey is a key for transaction start_ts info in context.Context.
|
||||
const txnStartKey = "_txn_start_key"
|
||||
|
||||
// NewBackoffer creates a Backoffer with maximum sleep time(in ms).
|
||||
func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer {
|
||||
return &Backoffer{
|
||||
ctx: ctx,
|
||||
maxSleep: maxSleep,
|
||||
}
|
||||
}
|
||||
|
||||
// Backoff sleeps a while base on the BackoffType and records the error message.
|
||||
// It returns a retryable error if total sleep time exceeds maxSleep.
|
||||
func (b *Backoffer) Backoff(typ BackoffType, err error) error {
|
||||
select {
|
||||
case <-b.ctx.Done():
|
||||
return err
|
||||
default:
|
||||
}
|
||||
|
||||
metrics.BackoffCounter.WithLabelValues(typ.String()).Inc()
|
||||
// Lazy initialize.
|
||||
if b.fn == nil {
|
||||
b.fn = make(map[BackoffType]func(context.Context) int)
|
||||
}
|
||||
f, ok := b.fn[typ]
|
||||
if !ok {
|
||||
f = typ.createFn()
|
||||
b.fn[typ] = f
|
||||
}
|
||||
|
||||
b.totalSleep += f(b.ctx)
|
||||
b.types = append(b.types, typ)
|
||||
|
||||
var startTs interface{}
|
||||
if ts := b.ctx.Value(txnStartKey); ts != nil {
|
||||
startTs = ts
|
||||
}
|
||||
log.Debugf("%v, retry later(totalsleep %dms, maxsleep %dms), type: %s, txn_start_ts: %v", err, b.totalSleep, b.maxSleep, typ.String(), startTs)
|
||||
|
||||
b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano)))
|
||||
if b.maxSleep > 0 && b.totalSleep >= b.maxSleep {
|
||||
errMsg := fmt.Sprintf("backoffer.maxSleep %dms is exceeded, errors:", b.maxSleep)
|
||||
for i, err := range b.errors {
|
||||
// Print only last 3 errors for non-DEBUG log levels.
|
||||
if log.GetLevel() == log.DebugLevel || i >= len(b.errors)-3 {
|
||||
errMsg += "\n" + err.Error()
|
||||
}
|
||||
}
|
||||
log.Warn(errMsg)
|
||||
// Use the first backoff type to generate a MySQL error.
|
||||
return errors.New(b.types[0].String())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *Backoffer) String() string {
|
||||
if b.totalSleep == 0 {
|
||||
return ""
|
||||
}
|
||||
return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.types)
|
||||
}
|
||||
|
||||
// Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares
|
||||
// current Backoffer's context.
|
||||
func (b *Backoffer) Clone() *Backoffer {
|
||||
return &Backoffer{
|
||||
ctx: b.ctx,
|
||||
maxSleep: b.maxSleep,
|
||||
totalSleep: b.totalSleep,
|
||||
errors: b.errors,
|
||||
}
|
||||
}
|
||||
|
||||
// Fork creates a new Backoffer which keeps current Backoffer's sleep time and errors, and holds
|
||||
// a child context of current Backoffer's context.
|
||||
func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) {
|
||||
ctx, cancel := context.WithCancel(b.ctx)
|
||||
return &Backoffer{
|
||||
ctx: ctx,
|
||||
maxSleep: b.maxSleep,
|
||||
totalSleep: b.totalSleep,
|
||||
errors: b.errors[:len(b.errors):len(b.errors)],
|
||||
}, cancel
|
||||
}
|
||||
|
||||
// GetContext returns the associated context.
|
||||
func (b *Backoffer) GetContext() context.Context {
|
||||
return b.ctx
|
||||
}
|
||||
|
||||
// TotalSleep returns the total sleep time of the backoffer.
|
||||
func (b *Backoffer) TotalSleep() time.Duration {
|
||||
return time.Duration(b.totalSleep) * time.Millisecond
|
||||
}
|
||||
667
rpc/calls.go
667
rpc/calls.go
|
|
@ -1,667 +0,0 @@
|
|||
// Copyright 2016 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 rpc
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/pingcap/kvproto/pkg/coprocessor"
|
||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/kvproto/pkg/metapb"
|
||||
"github.com/pingcap/kvproto/pkg/tikvpb"
|
||||
)
|
||||
|
||||
// CmdType represents the concrete request type in Request or response type in Response.
|
||||
type CmdType uint16
|
||||
|
||||
// CmdType values.
|
||||
const (
|
||||
CmdGet CmdType = 1 + iota
|
||||
CmdScan
|
||||
CmdPrewrite
|
||||
CmdCommit
|
||||
CmdCleanup
|
||||
CmdBatchGet
|
||||
CmdBatchRollback
|
||||
CmdScanLock
|
||||
CmdResolveLock
|
||||
CmdGC
|
||||
CmdDeleteRange
|
||||
|
||||
CmdRawGet CmdType = 256 + iota
|
||||
CmdRawBatchGet
|
||||
CmdRawPut
|
||||
CmdRawBatchPut
|
||||
CmdRawDelete
|
||||
CmdRawBatchDelete
|
||||
CmdRawDeleteRange
|
||||
CmdRawScan
|
||||
CmdRawGetKeyTTL
|
||||
|
||||
CmdUnsafeDestroyRange
|
||||
|
||||
CmdCop CmdType = 512 + iota
|
||||
CmdCopStream
|
||||
|
||||
CmdMvccGetByKey CmdType = 1024 + iota
|
||||
CmdMvccGetByStartTs
|
||||
CmdSplitRegion
|
||||
)
|
||||
|
||||
func (t CmdType) String() string {
|
||||
switch t {
|
||||
case CmdGet:
|
||||
return "Get"
|
||||
case CmdScan:
|
||||
return "Scan"
|
||||
case CmdPrewrite:
|
||||
return "Prewrite"
|
||||
case CmdCommit:
|
||||
return "Commit"
|
||||
case CmdCleanup:
|
||||
return "Cleanup"
|
||||
case CmdBatchGet:
|
||||
return "BatchGet"
|
||||
case CmdBatchRollback:
|
||||
return "BatchRollback"
|
||||
case CmdScanLock:
|
||||
return "ScanLock"
|
||||
case CmdResolveLock:
|
||||
return "ResolveLock"
|
||||
case CmdGC:
|
||||
return "GC"
|
||||
case CmdDeleteRange:
|
||||
return "DeleteRange"
|
||||
case CmdRawGet:
|
||||
return "RawGet"
|
||||
case CmdRawBatchGet:
|
||||
return "RawBatchGet"
|
||||
case CmdRawPut:
|
||||
return "RawPut"
|
||||
case CmdRawBatchPut:
|
||||
return "RawBatchPut"
|
||||
case CmdRawDelete:
|
||||
return "RawDelete"
|
||||
case CmdRawBatchDelete:
|
||||
return "RawBatchDelete"
|
||||
case CmdRawDeleteRange:
|
||||
return "RawDeleteRange"
|
||||
case CmdRawScan:
|
||||
return "RawScan"
|
||||
case CmdRawGetKeyTTL:
|
||||
return "RawGetKeyTTL"
|
||||
case CmdUnsafeDestroyRange:
|
||||
return "UnsafeDestroyRange"
|
||||
case CmdCop:
|
||||
return "Cop"
|
||||
case CmdCopStream:
|
||||
return "CopStream"
|
||||
case CmdMvccGetByKey:
|
||||
return "MvccGetByKey"
|
||||
case CmdMvccGetByStartTs:
|
||||
return "MvccGetByStartTS"
|
||||
case CmdSplitRegion:
|
||||
return "SplitRegion"
|
||||
}
|
||||
return "Unknown"
|
||||
}
|
||||
|
||||
// Request wraps all kv/coprocessor requests.
|
||||
type Request struct {
|
||||
kvrpcpb.Context
|
||||
Type CmdType
|
||||
Get *kvrpcpb.GetRequest
|
||||
Scan *kvrpcpb.ScanRequest
|
||||
Prewrite *kvrpcpb.PrewriteRequest
|
||||
Commit *kvrpcpb.CommitRequest
|
||||
Cleanup *kvrpcpb.CleanupRequest
|
||||
BatchGet *kvrpcpb.BatchGetRequest
|
||||
BatchRollback *kvrpcpb.BatchRollbackRequest
|
||||
ScanLock *kvrpcpb.ScanLockRequest
|
||||
ResolveLock *kvrpcpb.ResolveLockRequest
|
||||
GC *kvrpcpb.GCRequest
|
||||
DeleteRange *kvrpcpb.DeleteRangeRequest
|
||||
RawGet *kvrpcpb.RawGetRequest
|
||||
RawBatchGet *kvrpcpb.RawBatchGetRequest
|
||||
RawPut *kvrpcpb.RawPutRequest
|
||||
RawBatchPut *kvrpcpb.RawBatchPutRequest
|
||||
RawDelete *kvrpcpb.RawDeleteRequest
|
||||
RawBatchDelete *kvrpcpb.RawBatchDeleteRequest
|
||||
RawDeleteRange *kvrpcpb.RawDeleteRangeRequest
|
||||
RawScan *kvrpcpb.RawScanRequest
|
||||
RawGetKeyTTL *kvrpcpb.RawGetKeyTTLRequest
|
||||
UnsafeDestroyRange *kvrpcpb.UnsafeDestroyRangeRequest
|
||||
Cop *coprocessor.Request
|
||||
MvccGetByKey *kvrpcpb.MvccGetByKeyRequest
|
||||
MvccGetByStartTs *kvrpcpb.MvccGetByStartTsRequest
|
||||
SplitRegion *kvrpcpb.SplitRegionRequest
|
||||
}
|
||||
|
||||
// ToBatchCommandsRequest converts the request to an entry in BatchCommands request.
|
||||
func (req *Request) ToBatchCommandsRequest() *tikvpb.BatchCommandsRequest_Request {
|
||||
switch req.Type {
|
||||
case CmdGet:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Get{Get: req.Get}}
|
||||
case CmdScan:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Scan{Scan: req.Scan}}
|
||||
case CmdPrewrite:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Prewrite{Prewrite: req.Prewrite}}
|
||||
case CmdCommit:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Commit{Commit: req.Commit}}
|
||||
case CmdCleanup:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Cleanup{Cleanup: req.Cleanup}}
|
||||
case CmdBatchGet:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_BatchGet{BatchGet: req.BatchGet}}
|
||||
case CmdBatchRollback:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_BatchRollback{BatchRollback: req.BatchRollback}}
|
||||
case CmdScanLock:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_ScanLock{ScanLock: req.ScanLock}}
|
||||
case CmdResolveLock:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_ResolveLock{ResolveLock: req.ResolveLock}}
|
||||
case CmdGC:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_GC{GC: req.GC}}
|
||||
case CmdDeleteRange:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_DeleteRange{DeleteRange: req.DeleteRange}}
|
||||
case CmdRawGet:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawGet{RawGet: req.RawGet}}
|
||||
case CmdRawBatchGet:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawBatchGet{RawBatchGet: req.RawBatchGet}}
|
||||
case CmdRawPut:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawPut{RawPut: req.RawPut}}
|
||||
case CmdRawBatchPut:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawBatchPut{RawBatchPut: req.RawBatchPut}}
|
||||
case CmdRawDelete:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawDelete{RawDelete: req.RawDelete}}
|
||||
case CmdRawBatchDelete:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawBatchDelete{RawBatchDelete: req.RawBatchDelete}}
|
||||
case CmdRawDeleteRange:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawDeleteRange{RawDeleteRange: req.RawDeleteRange}}
|
||||
case CmdRawScan:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawScan{RawScan: req.RawScan}}
|
||||
case CmdCop:
|
||||
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Coprocessor{Coprocessor: req.Cop}}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Response wraps all kv/coprocessor responses.
|
||||
type Response struct {
|
||||
Type CmdType
|
||||
Get *kvrpcpb.GetResponse
|
||||
Scan *kvrpcpb.ScanResponse
|
||||
Prewrite *kvrpcpb.PrewriteResponse
|
||||
Commit *kvrpcpb.CommitResponse
|
||||
Cleanup *kvrpcpb.CleanupResponse
|
||||
BatchGet *kvrpcpb.BatchGetResponse
|
||||
BatchRollback *kvrpcpb.BatchRollbackResponse
|
||||
ScanLock *kvrpcpb.ScanLockResponse
|
||||
ResolveLock *kvrpcpb.ResolveLockResponse
|
||||
GC *kvrpcpb.GCResponse
|
||||
DeleteRange *kvrpcpb.DeleteRangeResponse
|
||||
RawGet *kvrpcpb.RawGetResponse
|
||||
RawBatchGet *kvrpcpb.RawBatchGetResponse
|
||||
RawPut *kvrpcpb.RawPutResponse
|
||||
RawBatchPut *kvrpcpb.RawBatchPutResponse
|
||||
RawDelete *kvrpcpb.RawDeleteResponse
|
||||
RawBatchDelete *kvrpcpb.RawBatchDeleteResponse
|
||||
RawDeleteRange *kvrpcpb.RawDeleteRangeResponse
|
||||
RawScan *kvrpcpb.RawScanResponse
|
||||
RawGetKeyTTL *kvrpcpb.RawGetKeyTTLResponse
|
||||
UnsafeDestroyRange *kvrpcpb.UnsafeDestroyRangeResponse
|
||||
Cop *coprocessor.Response
|
||||
CopStream *CopStreamResponse
|
||||
MvccGetByKey *kvrpcpb.MvccGetByKeyResponse
|
||||
MvccGetByStartTS *kvrpcpb.MvccGetByStartTsResponse
|
||||
SplitRegion *kvrpcpb.SplitRegionResponse
|
||||
}
|
||||
|
||||
// FromBatchCommandsResponse converts a BatchCommands response to Response.
|
||||
func FromBatchCommandsResponse(res *tikvpb.BatchCommandsResponse_Response) *Response {
|
||||
switch res := res.GetCmd().(type) {
|
||||
case *tikvpb.BatchCommandsResponse_Response_Get:
|
||||
return &Response{Type: CmdGet, Get: res.Get}
|
||||
case *tikvpb.BatchCommandsResponse_Response_Scan:
|
||||
return &Response{Type: CmdScan, Scan: res.Scan}
|
||||
case *tikvpb.BatchCommandsResponse_Response_Prewrite:
|
||||
return &Response{Type: CmdPrewrite, Prewrite: res.Prewrite}
|
||||
case *tikvpb.BatchCommandsResponse_Response_Commit:
|
||||
return &Response{Type: CmdCommit, Commit: res.Commit}
|
||||
case *tikvpb.BatchCommandsResponse_Response_Cleanup:
|
||||
return &Response{Type: CmdCleanup, Cleanup: res.Cleanup}
|
||||
case *tikvpb.BatchCommandsResponse_Response_BatchGet:
|
||||
return &Response{Type: CmdBatchGet, BatchGet: res.BatchGet}
|
||||
case *tikvpb.BatchCommandsResponse_Response_BatchRollback:
|
||||
return &Response{Type: CmdBatchRollback, BatchRollback: res.BatchRollback}
|
||||
case *tikvpb.BatchCommandsResponse_Response_ScanLock:
|
||||
return &Response{Type: CmdScanLock, ScanLock: res.ScanLock}
|
||||
case *tikvpb.BatchCommandsResponse_Response_ResolveLock:
|
||||
return &Response{Type: CmdResolveLock, ResolveLock: res.ResolveLock}
|
||||
case *tikvpb.BatchCommandsResponse_Response_GC:
|
||||
return &Response{Type: CmdGC, GC: res.GC}
|
||||
case *tikvpb.BatchCommandsResponse_Response_DeleteRange:
|
||||
return &Response{Type: CmdDeleteRange, DeleteRange: res.DeleteRange}
|
||||
case *tikvpb.BatchCommandsResponse_Response_RawGet:
|
||||
return &Response{Type: CmdRawGet, RawGet: res.RawGet}
|
||||
case *tikvpb.BatchCommandsResponse_Response_RawBatchGet:
|
||||
return &Response{Type: CmdRawBatchGet, RawBatchGet: res.RawBatchGet}
|
||||
case *tikvpb.BatchCommandsResponse_Response_RawPut:
|
||||
return &Response{Type: CmdRawPut, RawPut: res.RawPut}
|
||||
case *tikvpb.BatchCommandsResponse_Response_RawBatchPut:
|
||||
return &Response{Type: CmdRawBatchPut, RawBatchPut: res.RawBatchPut}
|
||||
case *tikvpb.BatchCommandsResponse_Response_RawDelete:
|
||||
return &Response{Type: CmdRawDelete, RawDelete: res.RawDelete}
|
||||
case *tikvpb.BatchCommandsResponse_Response_RawBatchDelete:
|
||||
return &Response{Type: CmdRawBatchDelete, RawBatchDelete: res.RawBatchDelete}
|
||||
case *tikvpb.BatchCommandsResponse_Response_RawDeleteRange:
|
||||
return &Response{Type: CmdRawDeleteRange, RawDeleteRange: res.RawDeleteRange}
|
||||
case *tikvpb.BatchCommandsResponse_Response_RawScan:
|
||||
return &Response{Type: CmdRawScan, RawScan: res.RawScan}
|
||||
case *tikvpb.BatchCommandsResponse_Response_Coprocessor:
|
||||
return &Response{Type: CmdCop, Cop: res.Coprocessor}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// CopStreamResponse combinates tikvpb.Tikv_CoprocessorStreamClient and the first Recv() result together.
|
||||
// In streaming API, get grpc stream client may not involve any network packet, then region error have
|
||||
// to be handled in Recv() function. This struct facilitates the error handling.
|
||||
type CopStreamResponse struct {
|
||||
tikvpb.Tikv_CoprocessorStreamClient
|
||||
*coprocessor.Response // The first result of Recv()
|
||||
Timeout time.Duration
|
||||
Lease // Shared by this object and a background goroutine.
|
||||
}
|
||||
|
||||
// SetContext set the Context field for the given req to the specified ctx.
|
||||
func SetContext(req *Request, region *metapb.Region, peer *metapb.Peer) error {
|
||||
ctx := &req.Context
|
||||
ctx.RegionId = region.Id
|
||||
ctx.RegionEpoch = region.RegionEpoch
|
||||
ctx.Peer = peer
|
||||
|
||||
switch req.Type {
|
||||
case CmdGet:
|
||||
req.Get.Context = ctx
|
||||
case CmdScan:
|
||||
req.Scan.Context = ctx
|
||||
case CmdPrewrite:
|
||||
req.Prewrite.Context = ctx
|
||||
case CmdCommit:
|
||||
req.Commit.Context = ctx
|
||||
case CmdCleanup:
|
||||
req.Cleanup.Context = ctx
|
||||
case CmdBatchGet:
|
||||
req.BatchGet.Context = ctx
|
||||
case CmdBatchRollback:
|
||||
req.BatchRollback.Context = ctx
|
||||
case CmdScanLock:
|
||||
req.ScanLock.Context = ctx
|
||||
case CmdResolveLock:
|
||||
req.ResolveLock.Context = ctx
|
||||
case CmdGC:
|
||||
req.GC.Context = ctx
|
||||
case CmdDeleteRange:
|
||||
req.DeleteRange.Context = ctx
|
||||
case CmdRawGet:
|
||||
req.RawGet.Context = ctx
|
||||
case CmdRawBatchGet:
|
||||
req.RawBatchGet.Context = ctx
|
||||
case CmdRawPut:
|
||||
req.RawPut.Context = ctx
|
||||
case CmdRawBatchPut:
|
||||
req.RawBatchPut.Context = ctx
|
||||
case CmdRawDelete:
|
||||
req.RawDelete.Context = ctx
|
||||
case CmdRawBatchDelete:
|
||||
req.RawBatchDelete.Context = ctx
|
||||
case CmdRawDeleteRange:
|
||||
req.RawDeleteRange.Context = ctx
|
||||
case CmdRawScan:
|
||||
req.RawScan.Context = ctx
|
||||
case CmdRawGetKeyTTL:
|
||||
req.RawGetKeyTTL.Context = ctx
|
||||
case CmdUnsafeDestroyRange:
|
||||
req.UnsafeDestroyRange.Context = ctx
|
||||
case CmdCop:
|
||||
req.Cop.Context = ctx
|
||||
case CmdCopStream:
|
||||
req.Cop.Context = ctx
|
||||
case CmdMvccGetByKey:
|
||||
req.MvccGetByKey.Context = ctx
|
||||
case CmdMvccGetByStartTs:
|
||||
req.MvccGetByStartTs.Context = ctx
|
||||
case CmdSplitRegion:
|
||||
req.SplitRegion.Context = ctx
|
||||
default:
|
||||
return fmt.Errorf("invalid request type %v", req.Type)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// GenRegionErrorResp returns corresponding Response with specified RegionError
|
||||
// according to the given req.
|
||||
func GenRegionErrorResp(req *Request, e *errorpb.Error) (*Response, error) {
|
||||
resp := &Response{}
|
||||
resp.Type = req.Type
|
||||
switch req.Type {
|
||||
case CmdGet:
|
||||
resp.Get = &kvrpcpb.GetResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdScan:
|
||||
resp.Scan = &kvrpcpb.ScanResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdPrewrite:
|
||||
resp.Prewrite = &kvrpcpb.PrewriteResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdCommit:
|
||||
resp.Commit = &kvrpcpb.CommitResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdCleanup:
|
||||
resp.Cleanup = &kvrpcpb.CleanupResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdBatchGet:
|
||||
resp.BatchGet = &kvrpcpb.BatchGetResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdBatchRollback:
|
||||
resp.BatchRollback = &kvrpcpb.BatchRollbackResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdScanLock:
|
||||
resp.ScanLock = &kvrpcpb.ScanLockResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdResolveLock:
|
||||
resp.ResolveLock = &kvrpcpb.ResolveLockResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdGC:
|
||||
resp.GC = &kvrpcpb.GCResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdDeleteRange:
|
||||
resp.DeleteRange = &kvrpcpb.DeleteRangeResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawGet:
|
||||
resp.RawGet = &kvrpcpb.RawGetResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawBatchGet:
|
||||
resp.RawBatchGet = &kvrpcpb.RawBatchGetResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawPut:
|
||||
resp.RawPut = &kvrpcpb.RawPutResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawBatchPut:
|
||||
resp.RawBatchPut = &kvrpcpb.RawBatchPutResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawDelete:
|
||||
resp.RawDelete = &kvrpcpb.RawDeleteResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawBatchDelete:
|
||||
resp.RawBatchDelete = &kvrpcpb.RawBatchDeleteResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawDeleteRange:
|
||||
resp.RawDeleteRange = &kvrpcpb.RawDeleteRangeResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawScan:
|
||||
resp.RawScan = &kvrpcpb.RawScanResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdRawGetKeyTTL:
|
||||
resp.RawGetKeyTTL = &kvrpcpb.RawGetKeyTTLResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdUnsafeDestroyRange:
|
||||
resp.UnsafeDestroyRange = &kvrpcpb.UnsafeDestroyRangeResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdCop:
|
||||
resp.Cop = &coprocessor.Response{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdCopStream:
|
||||
resp.CopStream = &CopStreamResponse{
|
||||
Response: &coprocessor.Response{
|
||||
RegionError: e,
|
||||
},
|
||||
}
|
||||
case CmdMvccGetByKey:
|
||||
resp.MvccGetByKey = &kvrpcpb.MvccGetByKeyResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdMvccGetByStartTs:
|
||||
resp.MvccGetByStartTS = &kvrpcpb.MvccGetByStartTsResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
case CmdSplitRegion:
|
||||
resp.SplitRegion = &kvrpcpb.SplitRegionResponse{
|
||||
RegionError: e,
|
||||
}
|
||||
default:
|
||||
return nil, fmt.Errorf("invalid request type %v", req.Type)
|
||||
}
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
// GetRegionError returns the RegionError of the underlying concrete response.
|
||||
func (resp *Response) GetRegionError() (*errorpb.Error, error) {
|
||||
var e *errorpb.Error
|
||||
switch resp.Type {
|
||||
case CmdGet:
|
||||
e = resp.Get.GetRegionError()
|
||||
case CmdScan:
|
||||
e = resp.Scan.GetRegionError()
|
||||
case CmdPrewrite:
|
||||
e = resp.Prewrite.GetRegionError()
|
||||
case CmdCommit:
|
||||
e = resp.Commit.GetRegionError()
|
||||
case CmdCleanup:
|
||||
e = resp.Cleanup.GetRegionError()
|
||||
case CmdBatchGet:
|
||||
e = resp.BatchGet.GetRegionError()
|
||||
case CmdBatchRollback:
|
||||
e = resp.BatchRollback.GetRegionError()
|
||||
case CmdScanLock:
|
||||
e = resp.ScanLock.GetRegionError()
|
||||
case CmdResolveLock:
|
||||
e = resp.ResolveLock.GetRegionError()
|
||||
case CmdGC:
|
||||
e = resp.GC.GetRegionError()
|
||||
case CmdDeleteRange:
|
||||
e = resp.DeleteRange.GetRegionError()
|
||||
case CmdRawGet:
|
||||
e = resp.RawGet.GetRegionError()
|
||||
case CmdRawBatchGet:
|
||||
e = resp.RawBatchGet.GetRegionError()
|
||||
case CmdRawPut:
|
||||
e = resp.RawPut.GetRegionError()
|
||||
case CmdRawBatchPut:
|
||||
e = resp.RawBatchPut.GetRegionError()
|
||||
case CmdRawDelete:
|
||||
e = resp.RawDelete.GetRegionError()
|
||||
case CmdRawBatchDelete:
|
||||
e = resp.RawBatchDelete.GetRegionError()
|
||||
case CmdRawDeleteRange:
|
||||
e = resp.RawDeleteRange.GetRegionError()
|
||||
case CmdRawScan:
|
||||
e = resp.RawScan.GetRegionError()
|
||||
case CmdRawGetKeyTTL:
|
||||
e = resp.RawGetKeyTTL.GetRegionError()
|
||||
case CmdUnsafeDestroyRange:
|
||||
e = resp.UnsafeDestroyRange.GetRegionError()
|
||||
case CmdCop:
|
||||
e = resp.Cop.GetRegionError()
|
||||
case CmdCopStream:
|
||||
e = resp.CopStream.Response.GetRegionError()
|
||||
case CmdMvccGetByKey:
|
||||
e = resp.MvccGetByKey.GetRegionError()
|
||||
case CmdMvccGetByStartTs:
|
||||
e = resp.MvccGetByStartTS.GetRegionError()
|
||||
case CmdSplitRegion:
|
||||
e = resp.SplitRegion.GetRegionError()
|
||||
default:
|
||||
return nil, fmt.Errorf("invalid response type %v", resp.Type)
|
||||
}
|
||||
return e, nil
|
||||
}
|
||||
|
||||
// CallRPC launches a rpc call.
|
||||
// ch is needed to implement timeout for coprocessor streaing, the stream object's
|
||||
// cancel function will be sent to the channel, together with a lease checked by a background goroutine.
|
||||
func CallRPC(ctx context.Context, client tikvpb.TikvClient, req *Request) (*Response, error) {
|
||||
resp := &Response{}
|
||||
resp.Type = req.Type
|
||||
var err error
|
||||
switch req.Type {
|
||||
case CmdGet:
|
||||
resp.Get, err = client.KvGet(ctx, req.Get)
|
||||
case CmdScan:
|
||||
resp.Scan, err = client.KvScan(ctx, req.Scan)
|
||||
case CmdPrewrite:
|
||||
resp.Prewrite, err = client.KvPrewrite(ctx, req.Prewrite)
|
||||
case CmdCommit:
|
||||
resp.Commit, err = client.KvCommit(ctx, req.Commit)
|
||||
case CmdCleanup:
|
||||
resp.Cleanup, err = client.KvCleanup(ctx, req.Cleanup)
|
||||
case CmdBatchGet:
|
||||
resp.BatchGet, err = client.KvBatchGet(ctx, req.BatchGet)
|
||||
case CmdBatchRollback:
|
||||
resp.BatchRollback, err = client.KvBatchRollback(ctx, req.BatchRollback)
|
||||
case CmdScanLock:
|
||||
resp.ScanLock, err = client.KvScanLock(ctx, req.ScanLock)
|
||||
case CmdResolveLock:
|
||||
resp.ResolveLock, err = client.KvResolveLock(ctx, req.ResolveLock)
|
||||
case CmdGC:
|
||||
resp.GC, err = client.KvGC(ctx, req.GC)
|
||||
case CmdDeleteRange:
|
||||
resp.DeleteRange, err = client.KvDeleteRange(ctx, req.DeleteRange)
|
||||
case CmdRawGet:
|
||||
resp.RawGet, err = client.RawGet(ctx, req.RawGet)
|
||||
case CmdRawBatchGet:
|
||||
resp.RawBatchGet, err = client.RawBatchGet(ctx, req.RawBatchGet)
|
||||
case CmdRawPut:
|
||||
resp.RawPut, err = client.RawPut(ctx, req.RawPut)
|
||||
case CmdRawBatchPut:
|
||||
resp.RawBatchPut, err = client.RawBatchPut(ctx, req.RawBatchPut)
|
||||
case CmdRawDelete:
|
||||
resp.RawDelete, err = client.RawDelete(ctx, req.RawDelete)
|
||||
case CmdRawBatchDelete:
|
||||
resp.RawBatchDelete, err = client.RawBatchDelete(ctx, req.RawBatchDelete)
|
||||
case CmdRawDeleteRange:
|
||||
resp.RawDeleteRange, err = client.RawDeleteRange(ctx, req.RawDeleteRange)
|
||||
case CmdRawScan:
|
||||
resp.RawScan, err = client.RawScan(ctx, req.RawScan)
|
||||
case CmdRawGetKeyTTL:
|
||||
resp.RawGetKeyTTL, err = client.RawGetKeyTTL(ctx, req.RawGetKeyTTL)
|
||||
case CmdUnsafeDestroyRange:
|
||||
resp.UnsafeDestroyRange, err = client.UnsafeDestroyRange(ctx, req.UnsafeDestroyRange)
|
||||
case CmdCop:
|
||||
resp.Cop, err = client.Coprocessor(ctx, req.Cop)
|
||||
case CmdCopStream:
|
||||
var streamClient tikvpb.Tikv_CoprocessorStreamClient
|
||||
streamClient, err = client.CoprocessorStream(ctx, req.Cop)
|
||||
resp.CopStream = &CopStreamResponse{
|
||||
Tikv_CoprocessorStreamClient: streamClient,
|
||||
}
|
||||
case CmdMvccGetByKey:
|
||||
resp.MvccGetByKey, err = client.MvccGetByKey(ctx, req.MvccGetByKey)
|
||||
case CmdMvccGetByStartTs:
|
||||
resp.MvccGetByStartTS, err = client.MvccGetByStartTs(ctx, req.MvccGetByStartTs)
|
||||
case CmdSplitRegion:
|
||||
resp.SplitRegion, err = client.SplitRegion(ctx, req.SplitRegion)
|
||||
default:
|
||||
return nil, errors.Errorf("invalid request type: %v", req.Type)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
// Lease is used to implement grpc stream timeout.
|
||||
type Lease struct {
|
||||
Cancel context.CancelFunc
|
||||
deadline int64 // A time.UnixNano value, if time.Now().UnixNano() > deadline, cancel() would be called.
|
||||
}
|
||||
|
||||
// Recv overrides the stream client Recv() function.
|
||||
func (resp *CopStreamResponse) Recv() (*coprocessor.Response, error) {
|
||||
deadline := time.Now().Add(resp.Timeout).UnixNano()
|
||||
atomic.StoreInt64(&resp.Lease.deadline, deadline)
|
||||
|
||||
ret, err := resp.Tikv_CoprocessorStreamClient.Recv()
|
||||
|
||||
atomic.StoreInt64(&resp.Lease.deadline, 0) // Stop the lease check.
|
||||
return ret, errors.WithStack(err)
|
||||
}
|
||||
|
||||
// Close closes the CopStreamResponse object.
|
||||
func (resp *CopStreamResponse) Close() {
|
||||
atomic.StoreInt64(&resp.Lease.deadline, 1)
|
||||
}
|
||||
|
||||
// CheckStreamTimeoutLoop runs periodically to check is there any stream request timeouted.
|
||||
// Lease is an object to track stream requests, call this function with "go CheckStreamTimeoutLoop()"
|
||||
func CheckStreamTimeoutLoop(ch <-chan *Lease) {
|
||||
ticker := time.NewTicker(200 * time.Millisecond)
|
||||
defer ticker.Stop()
|
||||
array := make([]*Lease, 0, 1024)
|
||||
|
||||
for {
|
||||
select {
|
||||
case item, ok := <-ch:
|
||||
if !ok {
|
||||
// This channel close means goroutine should return.
|
||||
return
|
||||
}
|
||||
array = append(array, item)
|
||||
case now := <-ticker.C:
|
||||
array = keepOnlyActive(array, now.UnixNano())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// keepOnlyActive removes completed items, call cancel function for timeout items.
|
||||
func keepOnlyActive(array []*Lease, now int64) []*Lease {
|
||||
idx := 0
|
||||
for i := 0; i < len(array); i++ {
|
||||
item := array[i]
|
||||
deadline := atomic.LoadInt64(&item.deadline)
|
||||
if deadline == 0 || deadline > now {
|
||||
array[idx] = array[i]
|
||||
idx++
|
||||
} else {
|
||||
item.Cancel()
|
||||
}
|
||||
}
|
||||
return array[:idx]
|
||||
}
|
||||
586
rpc/client.go
586
rpc/client.go
|
|
@ -1,586 +0,0 @@
|
|||
// Copyright 2016 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 rpc
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"strconv"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
|
||||
grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
|
||||
"github.com/pingcap/kvproto/pkg/coprocessor"
|
||||
"github.com/pingcap/kvproto/pkg/tikvpb"
|
||||
"github.com/pkg/errors"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"google.golang.org/grpc"
|
||||
gcodes "google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/credentials"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
gstatus "google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// Client is a client that sends RPC.
|
||||
// It should not be used after calling Close().
|
||||
type Client interface {
|
||||
// Close should release all data.
|
||||
Close() error
|
||||
// SendRequest sends Request.
|
||||
SendRequest(ctx context.Context, addr string, req *Request, timeout time.Duration) (*Response, error)
|
||||
}
|
||||
|
||||
type connArray struct {
|
||||
conf *config.RPC
|
||||
index uint32
|
||||
conns []*grpc.ClientConn
|
||||
// Bind with a background goroutine to process coprocessor streaming timeout.
|
||||
streamTimeout chan *Lease
|
||||
|
||||
// For batch commands.
|
||||
batchCommandsCh chan *batchCommandsEntry
|
||||
batchCommandsClients []*batchCommandsClient
|
||||
transportLayerLoad uint64
|
||||
}
|
||||
|
||||
type batchCommandsClient struct {
|
||||
conf *config.Batch
|
||||
conn *grpc.ClientConn
|
||||
client tikvpb.Tikv_BatchCommandsClient
|
||||
batched sync.Map
|
||||
idAlloc uint64
|
||||
transportLayerLoad *uint64
|
||||
|
||||
// Indicates the batch client is closed explicitly or not.
|
||||
closed int32
|
||||
// Protect client when re-create the streaming.
|
||||
clientLock sync.Mutex
|
||||
}
|
||||
|
||||
func (c *batchCommandsClient) isStopped() bool {
|
||||
return atomic.LoadInt32(&c.closed) != 0
|
||||
}
|
||||
|
||||
func (c *batchCommandsClient) failPendingRequests(err error) {
|
||||
c.batched.Range(func(key, value interface{}) bool {
|
||||
id, _ := key.(uint64)
|
||||
entry, _ := value.(*batchCommandsEntry)
|
||||
entry.err = err
|
||||
close(entry.res)
|
||||
c.batched.Delete(id)
|
||||
return true
|
||||
})
|
||||
}
|
||||
|
||||
func (c *batchCommandsClient) batchRecvLoop() {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
log.Errorf("batchRecvLoop %v", r)
|
||||
log.Infof("Restart batchRecvLoop")
|
||||
go c.batchRecvLoop()
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
// When `conn.Close()` is called, `client.Recv()` will return an error.
|
||||
resp, err := c.client.Recv()
|
||||
if err != nil {
|
||||
if c.isStopped() {
|
||||
return
|
||||
}
|
||||
log.Errorf("batchRecvLoop error when receive: %v", err)
|
||||
|
||||
// Hold the lock to forbid batchSendLoop using the old client.
|
||||
c.clientLock.Lock()
|
||||
c.failPendingRequests(err) // fail all pending requests.
|
||||
for { // try to re-create the streaming in the loop.
|
||||
// Re-establish a application layer stream. TCP layer is handled by gRPC.
|
||||
tikvClient := tikvpb.NewTikvClient(c.conn)
|
||||
streamClient, err := tikvClient.BatchCommands(context.TODO())
|
||||
if err == nil {
|
||||
log.Infof("batchRecvLoop re-create streaming success")
|
||||
c.client = streamClient
|
||||
break
|
||||
}
|
||||
log.Errorf("batchRecvLoop re-create streaming fail: %v", err)
|
||||
// TODO: Use a more smart backoff strategy.
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
c.clientLock.Unlock()
|
||||
continue
|
||||
}
|
||||
|
||||
responses := resp.GetResponses()
|
||||
for i, requestID := range resp.GetRequestIds() {
|
||||
value, ok := c.batched.Load(requestID)
|
||||
if !ok {
|
||||
// There shouldn't be any unknown responses because if the old entries
|
||||
// are cleaned by `failPendingRequests`, the stream must be re-created
|
||||
// so that old responses will be never received.
|
||||
panic("batchRecvLoop receives a unknown response")
|
||||
}
|
||||
entry := value.(*batchCommandsEntry)
|
||||
if atomic.LoadInt32(&entry.canceled) == 0 {
|
||||
// Put the response only if the request is not canceled.
|
||||
entry.res <- responses[i]
|
||||
}
|
||||
c.batched.Delete(requestID)
|
||||
}
|
||||
|
||||
transportLayerLoad := resp.GetTransportLayerLoad()
|
||||
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(addr string, conf *config.RPC) (*connArray, error) {
|
||||
a := &connArray{
|
||||
conf: conf,
|
||||
index: 0,
|
||||
conns: make([]*grpc.ClientConn, conf.MaxConnectionCount),
|
||||
streamTimeout: make(chan *Lease, 1024),
|
||||
batchCommandsCh: make(chan *batchCommandsEntry, conf.Batch.MaxBatchSize),
|
||||
batchCommandsClients: make([]*batchCommandsClient, 0, conf.Batch.MaxBatchSize),
|
||||
transportLayerLoad: 0,
|
||||
}
|
||||
if err := a.Init(addr); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return a, nil
|
||||
}
|
||||
|
||||
func (a *connArray) Init(addr string) error {
|
||||
opt := grpc.WithInsecure()
|
||||
if len(a.conf.Security.SSLCA) != 0 {
|
||||
tlsConfig, err := a.conf.Security.ToTLSConfig()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
opt = grpc.WithTransportCredentials(credentials.NewTLS(tlsConfig))
|
||||
}
|
||||
|
||||
unaryInterceptor := grpc_prometheus.UnaryClientInterceptor
|
||||
streamInterceptor := grpc_prometheus.StreamClientInterceptor
|
||||
if a.conf.EnableOpenTracing {
|
||||
unaryInterceptor = grpc_middleware.ChainUnaryClient(
|
||||
unaryInterceptor,
|
||||
grpc_opentracing.UnaryClientInterceptor(),
|
||||
)
|
||||
streamInterceptor = grpc_middleware.ChainStreamClient(
|
||||
streamInterceptor,
|
||||
grpc_opentracing.StreamClientInterceptor(),
|
||||
)
|
||||
}
|
||||
|
||||
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(a.conf.GrpcInitialWindowSize)),
|
||||
grpc.WithInitialConnWindowSize(int32(a.conf.GrpcInitialConnWindowSize)),
|
||||
grpc.WithUnaryInterceptor(unaryInterceptor),
|
||||
grpc.WithStreamInterceptor(streamInterceptor),
|
||||
grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(a.conf.GrpcMaxCallMsgSize)),
|
||||
grpc.WithDefaultCallOptions(grpc.MaxCallSendMsgSize(a.conf.GrpcMaxSendMsgSize)),
|
||||
grpc.WithBackoffMaxDelay(time.Second*3),
|
||||
grpc.WithKeepaliveParams(keepalive.ClientParameters{
|
||||
Time: a.conf.GrpcKeepAliveTime,
|
||||
Timeout: a.conf.GrpcKeepAliveTimeout,
|
||||
PermitWithoutStream: true,
|
||||
}),
|
||||
)
|
||||
cancel()
|
||||
if err != nil {
|
||||
// Cleanup if the initialization fails.
|
||||
a.Close()
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
a.conns[i] = conn
|
||||
|
||||
if allowBatch {
|
||||
// Initialize batch streaming clients.
|
||||
tikvClient := tikvpb.NewTikvClient(conn)
|
||||
streamClient, err := tikvClient.BatchCommands(context.TODO())
|
||||
if err != nil {
|
||||
a.Close()
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
batchClient := &batchCommandsClient{
|
||||
conf: &a.conf.Batch,
|
||||
conn: conn,
|
||||
client: streamClient,
|
||||
batched: sync.Map{},
|
||||
idAlloc: 0,
|
||||
transportLayerLoad: &a.transportLayerLoad,
|
||||
closed: 0,
|
||||
}
|
||||
a.batchCommandsClients = append(a.batchCommandsClients, batchClient)
|
||||
go batchClient.batchRecvLoop()
|
||||
}
|
||||
}
|
||||
go CheckStreamTimeoutLoop(a.streamTimeout)
|
||||
if allowBatch {
|
||||
go a.batchSendLoop()
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *connArray) Get() *grpc.ClientConn {
|
||||
next := atomic.AddUint32(&a.index, 1) % uint32(len(a.conns))
|
||||
return a.conns[next]
|
||||
}
|
||||
|
||||
func (a *connArray) Close() {
|
||||
// Close all batchRecvLoop.
|
||||
for _, c := range a.batchCommandsClients {
|
||||
// After connections are closed, `batchRecvLoop`s will check the flag.
|
||||
atomic.StoreInt32(&c.closed, 1)
|
||||
}
|
||||
close(a.batchCommandsCh)
|
||||
for i, c := range a.conns {
|
||||
if c != nil {
|
||||
c.Close()
|
||||
a.conns[i] = nil
|
||||
}
|
||||
}
|
||||
close(a.streamTimeout)
|
||||
}
|
||||
|
||||
type batchCommandsEntry struct {
|
||||
req *tikvpb.BatchCommandsRequest_Request
|
||||
res chan *tikvpb.BatchCommandsResponse_Response
|
||||
|
||||
// Indicated the request is canceled or not.
|
||||
canceled int32
|
||||
err error
|
||||
}
|
||||
|
||||
// fetchAllPendingRequests fetches all pending requests from the channel.
|
||||
func fetchAllPendingRequests(
|
||||
ch chan *batchCommandsEntry,
|
||||
maxBatchSize int,
|
||||
entries *[]*batchCommandsEntry,
|
||||
requests *[]*tikvpb.BatchCommandsRequest_Request,
|
||||
) {
|
||||
// Block on the first element.
|
||||
headEntry := <-ch
|
||||
if headEntry == nil {
|
||||
return
|
||||
}
|
||||
*entries = append(*entries, headEntry)
|
||||
*requests = append(*requests, headEntry.req)
|
||||
|
||||
// This loop is for trying best to collect more requests.
|
||||
for len(*entries) < maxBatchSize {
|
||||
select {
|
||||
case entry := <-ch:
|
||||
if entry == nil {
|
||||
return
|
||||
}
|
||||
*entries = append(*entries, entry)
|
||||
*requests = append(*requests, entry.req)
|
||||
default:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// fetchMorePendingRequests fetches more pending requests from the channel.
|
||||
func fetchMorePendingRequests(
|
||||
ch chan *batchCommandsEntry,
|
||||
maxBatchSize int,
|
||||
batchWaitSize int,
|
||||
maxWaitTime time.Duration,
|
||||
entries *[]*batchCommandsEntry,
|
||||
requests *[]*tikvpb.BatchCommandsRequest_Request,
|
||||
) {
|
||||
waitStart := time.Now()
|
||||
|
||||
// Try to collect `batchWaitSize` requests, or wait `maxWaitTime`.
|
||||
after := time.NewTimer(maxWaitTime)
|
||||
for len(*entries) < batchWaitSize {
|
||||
select {
|
||||
case entry := <-ch:
|
||||
if entry == nil {
|
||||
return
|
||||
}
|
||||
*entries = append(*entries, entry)
|
||||
*requests = append(*requests, entry.req)
|
||||
case waitEnd := <-after.C:
|
||||
metrics.BatchWaitDuration.Observe(float64(waitEnd.Sub(waitStart)))
|
||||
return
|
||||
}
|
||||
}
|
||||
after.Stop()
|
||||
|
||||
// Do an additional non-block try.
|
||||
for len(*entries) < maxBatchSize {
|
||||
select {
|
||||
case entry := <-ch:
|
||||
if entry == nil {
|
||||
return
|
||||
}
|
||||
*entries = append(*entries, entry)
|
||||
*requests = append(*requests, entry.req)
|
||||
default:
|
||||
metrics.BatchWaitDuration.Observe(float64(time.Since(waitStart)))
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (a *connArray) batchSendLoop() {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
log.Errorf("batchSendLoop %v", r)
|
||||
log.Infof("Restart batchSendLoop")
|
||||
go a.batchSendLoop()
|
||||
}
|
||||
}()
|
||||
|
||||
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.conns))
|
||||
batchCommandsClient := a.batchCommandsClients[next]
|
||||
|
||||
entries = entries[:0]
|
||||
requests = requests[:0]
|
||||
requestIDs = requestIDs[:0]
|
||||
|
||||
metrics.PendingBatchRequests.Set(float64(len(a.batchCommandsCh)))
|
||||
fetchAllPendingRequests(a.batchCommandsCh, int(conf.MaxBatchSize), &entries, &requests)
|
||||
|
||||
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) >= conf.OverloadThreshold {
|
||||
fetchMorePendingRequests(
|
||||
a.batchCommandsCh, int(conf.MaxBatchSize), int(conf.MaxWaitSize),
|
||||
conf.MaxWaitTime, &entries, &requests,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
length := len(requests)
|
||||
if uint(length) == 0 {
|
||||
// The batch command channel is closed.
|
||||
return
|
||||
}
|
||||
maxBatchID := atomic.AddUint64(&batchCommandsClient.idAlloc, uint64(length))
|
||||
for i := 0; i < length; i++ {
|
||||
requestID := uint64(i) + maxBatchID - uint64(length)
|
||||
requestIDs = append(requestIDs, requestID)
|
||||
}
|
||||
|
||||
request := &tikvpb.BatchCommandsRequest{
|
||||
Requests: requests,
|
||||
RequestIds: requestIDs,
|
||||
}
|
||||
|
||||
// Use the lock to protect the stream client won't be replaced by RecvLoop,
|
||||
// and new added request won't be removed by `failPendingRequests`.
|
||||
batchCommandsClient.clientLock.Lock()
|
||||
for i, requestID := range request.RequestIds {
|
||||
batchCommandsClient.batched.Store(requestID, entries[i])
|
||||
}
|
||||
err := batchCommandsClient.client.Send(request)
|
||||
batchCommandsClient.clientLock.Unlock()
|
||||
if err != nil {
|
||||
log.Errorf("batch commands send error: %v", err)
|
||||
batchCommandsClient.failPendingRequests(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// rpcClient is RPC client struct.
|
||||
// TODO: Add flow control between RPC clients in TiDB ond RPC servers in TiKV.
|
||||
// Since we use shared client connection to communicate to the same TiKV, it's possible
|
||||
// that there are too many concurrent requests which overload the service of TiKV.
|
||||
// TODO: Implement background cleanup. It adds a background goroutine to periodically check
|
||||
// whether there is any connection is idle and then close and remove these idle connections.
|
||||
type rpcClient struct {
|
||||
sync.RWMutex
|
||||
isClosed bool
|
||||
conns map[string]*connArray
|
||||
conf *config.RPC
|
||||
}
|
||||
|
||||
// NewRPCClient manages connections and rpc calls with tikv-servers.
|
||||
func NewRPCClient(conf *config.RPC) Client {
|
||||
return &rpcClient{
|
||||
conns: make(map[string]*connArray),
|
||||
conf: conf,
|
||||
}
|
||||
}
|
||||
|
||||
func (c *rpcClient) getConnArray(addr string) (*connArray, error) {
|
||||
c.RLock()
|
||||
if c.isClosed {
|
||||
c.RUnlock()
|
||||
return nil, errors.Errorf("rpcClient is closed")
|
||||
}
|
||||
array, ok := c.conns[addr]
|
||||
c.RUnlock()
|
||||
if !ok {
|
||||
var err error
|
||||
array, err = c.createConnArray(addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return array, nil
|
||||
}
|
||||
|
||||
func (c *rpcClient) createConnArray(addr string) (*connArray, error) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
array, ok := c.conns[addr]
|
||||
if !ok {
|
||||
var err error
|
||||
array, err = newConnArray(addr, c.conf)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
c.conns[addr] = array
|
||||
}
|
||||
return array, nil
|
||||
}
|
||||
|
||||
func (c *rpcClient) closeConns() {
|
||||
c.Lock()
|
||||
if !c.isClosed {
|
||||
c.isClosed = true
|
||||
// close all connections
|
||||
for _, array := range c.conns {
|
||||
array.Close()
|
||||
}
|
||||
}
|
||||
c.Unlock()
|
||||
}
|
||||
|
||||
func sendBatchRequest(
|
||||
ctx context.Context,
|
||||
addr string,
|
||||
connArray *connArray,
|
||||
req *tikvpb.BatchCommandsRequest_Request,
|
||||
timeout time.Duration,
|
||||
) (*Response, error) {
|
||||
entry := &batchCommandsEntry{
|
||||
req: req,
|
||||
res: make(chan *tikvpb.BatchCommandsResponse_Response, 1),
|
||||
canceled: 0,
|
||||
err: nil,
|
||||
}
|
||||
ctx1, cancel := context.WithTimeout(ctx, timeout)
|
||||
defer cancel()
|
||||
|
||||
select {
|
||||
case connArray.batchCommandsCh <- entry:
|
||||
case <-ctx1.Done():
|
||||
log.Warnf("SendRequest to %s is timeout", addr)
|
||||
return nil, errors.WithStack(gstatus.Error(gcodes.DeadlineExceeded, "Canceled or timeout"))
|
||||
}
|
||||
|
||||
select {
|
||||
case res, ok := <-entry.res:
|
||||
if !ok {
|
||||
return nil, errors.WithStack(entry.err)
|
||||
}
|
||||
return FromBatchCommandsResponse(res), nil
|
||||
case <-ctx1.Done():
|
||||
atomic.StoreInt32(&entry.canceled, 1)
|
||||
log.Warnf("SendRequest to %s is canceled", addr)
|
||||
return nil, errors.WithStack(gstatus.Error(gcodes.DeadlineExceeded, "Canceled or timeout"))
|
||||
}
|
||||
}
|
||||
|
||||
// SendRequest sends a Request to server and receives Response.
|
||||
func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *Request, timeout time.Duration) (*Response, error) {
|
||||
start := time.Now()
|
||||
reqType := req.Type.String()
|
||||
storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10)
|
||||
defer func() {
|
||||
metrics.SendReqHistogram.WithLabelValues(reqType, storeID).Observe(time.Since(start).Seconds())
|
||||
}()
|
||||
|
||||
connArray, err := c.getConnArray(addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if c.conf.Batch.MaxBatchSize > 0 {
|
||||
if batchReq := req.ToBatchCommandsRequest(); batchReq != nil {
|
||||
return sendBatchRequest(ctx, addr, connArray, batchReq, timeout)
|
||||
}
|
||||
}
|
||||
|
||||
client := tikvpb.NewTikvClient(connArray.Get())
|
||||
|
||||
if req.Type != CmdCopStream {
|
||||
ctx1, cancel := context.WithTimeout(ctx, timeout)
|
||||
defer cancel()
|
||||
return CallRPC(ctx1, client, req)
|
||||
}
|
||||
|
||||
// Coprocessor streaming request.
|
||||
// Use context to support timeout for grpc streaming client.
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
resp, err := CallRPC(ctx1, client, req)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Put the lease object to the timeout channel, so it would be checked periodically.
|
||||
copStream := resp.CopStream
|
||||
copStream.Timeout = timeout
|
||||
copStream.Lease.Cancel = cancel
|
||||
connArray.streamTimeout <- &copStream.Lease
|
||||
|
||||
// Read the first streaming response to get CopStreamResponse.
|
||||
// This can make error handling much easier, because SendReq() retry on
|
||||
// region error automatically.
|
||||
var first *coprocessor.Response
|
||||
first, err = copStream.Recv()
|
||||
if err != nil {
|
||||
if errors.Cause(err) != io.EOF {
|
||||
return nil, err
|
||||
}
|
||||
log.Debug("copstream returns nothing for the request.")
|
||||
}
|
||||
copStream.Response = first
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
func (c *rpcClient) Close() error {
|
||||
c.closeConns()
|
||||
return nil
|
||||
}
|
||||
|
|
@ -1,250 +0,0 @@
|
|||
// Copyright 2018 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 rpc
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/kvproto/pkg/errorpb"
|
||||
"github.com/pkg/errors"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/locate"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// ErrBodyMissing response body is missing error
|
||||
var ErrBodyMissing = errors.New("response body is missing")
|
||||
|
||||
// RegionRequestSender sends KV/Cop requests to tikv server. It handles network
|
||||
// errors and some region errors internally.
|
||||
//
|
||||
// Typically, a KV/Cop request is bind to a region, all keys that are involved
|
||||
// in the request should be located in the region.
|
||||
// The sending process begins with looking for the address of leader store's
|
||||
// address of the target region from cache, and the request is then sent to the
|
||||
// destination tikv server over TCP connection.
|
||||
// If region is updated, can be caused by leader transfer, region split, region
|
||||
// merge, or region balance, tikv server may not able to process request and
|
||||
// send back a RegionError.
|
||||
// RegionRequestSender takes care of errors that does not relevant to region
|
||||
// range, such as 'I/O timeout', 'NotLeader', and 'ServerIsBusy'. For other
|
||||
// errors, since region range have changed, the request may need to split, so we
|
||||
// simply return the error to caller.
|
||||
type RegionRequestSender struct {
|
||||
regionCache *locate.RegionCache
|
||||
client Client
|
||||
storeAddr string
|
||||
rpcError error
|
||||
}
|
||||
|
||||
// NewRegionRequestSender creates a new sender.
|
||||
func NewRegionRequestSender(regionCache *locate.RegionCache, client Client) *RegionRequestSender {
|
||||
return &RegionRequestSender{
|
||||
regionCache: regionCache,
|
||||
client: client,
|
||||
}
|
||||
}
|
||||
|
||||
// RPCError returns an error if an RPC error is encountered during request.
|
||||
func (s *RegionRequestSender) RPCError() error {
|
||||
return s.rpcError
|
||||
}
|
||||
|
||||
// SendReq sends a request to tikv server.
|
||||
func (s *RegionRequestSender) SendReq(bo *retry.Backoffer, req *Request, regionID locate.RegionVerID, timeout time.Duration) (*Response, error) {
|
||||
|
||||
// gofail: var tikvStoreSendReqResult string
|
||||
// switch tikvStoreSendReqResult {
|
||||
// case "timeout":
|
||||
// return nil, errors.New("timeout")
|
||||
// case "GCNotLeader":
|
||||
// if req.Type == CmdGC {
|
||||
// return &Response{
|
||||
// Type: CmdGC,
|
||||
// GC: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}},
|
||||
// }, nil
|
||||
// }
|
||||
// case "GCServerIsBusy":
|
||||
// if req.Type == CmdGC {
|
||||
// return &Response{
|
||||
// Type: CmdGC,
|
||||
// GC: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}}},
|
||||
// }, nil
|
||||
// }
|
||||
// }
|
||||
|
||||
for {
|
||||
ctx, err := s.regionCache.GetRPCContext(bo, regionID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if ctx == nil {
|
||||
// If the region is not found in cache, it must be out
|
||||
// of date and already be cleaned up. We can skip the
|
||||
// RPC by returning RegionError directly.
|
||||
|
||||
// TODO: Change the returned error to something like "region missing in cache",
|
||||
// and handle this error like StaleEpoch, which means to re-split the request and retry.
|
||||
return GenRegionErrorResp(req, &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}})
|
||||
}
|
||||
|
||||
s.storeAddr = ctx.Addr
|
||||
resp, retry, err := s.sendReqToRegion(bo, ctx, req, timeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if retry {
|
||||
continue
|
||||
}
|
||||
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if regionErr != nil {
|
||||
retry, err := s.onRegionError(bo, ctx, regionErr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if retry {
|
||||
continue
|
||||
}
|
||||
}
|
||||
return resp, nil
|
||||
}
|
||||
}
|
||||
|
||||
func (s *RegionRequestSender) sendReqToRegion(bo *retry.Backoffer, ctx *locate.RPCContext, req *Request, timeout time.Duration) (resp *Response, retry bool, err error) {
|
||||
if e := SetContext(req, ctx.Meta, ctx.Peer); e != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
resp, err = s.client.SendRequest(bo.GetContext(), ctx.Addr, req, timeout)
|
||||
if err != nil {
|
||||
s.rpcError = err
|
||||
if e := s.onSendFail(bo, ctx, err); e != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
return nil, true, nil
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *locate.RPCContext, err error) error {
|
||||
// If it failed because the context is cancelled by ourself, don't retry.
|
||||
if errors.Cause(err) == context.Canceled {
|
||||
return err
|
||||
}
|
||||
code := codes.Unknown
|
||||
if s, ok := status.FromError(errors.Cause(err)); ok {
|
||||
code = s.Code()
|
||||
}
|
||||
if code == codes.Canceled {
|
||||
select {
|
||||
case <-bo.GetContext().Done():
|
||||
return err
|
||||
default:
|
||||
// If we don't cancel, but the error code is Canceled, it must be from grpc remote.
|
||||
// This may happen when tikv is killed and exiting.
|
||||
// Backoff and retry in this case.
|
||||
log.Warn("receive a grpc cancel signal from remote:", err)
|
||||
}
|
||||
}
|
||||
|
||||
s.regionCache.DropStoreOnSendRequestFail(ctx, err)
|
||||
|
||||
// Retry on send request failure when it's not canceled.
|
||||
// When a store is not available, the leader of related region should be elected quickly.
|
||||
// TODO: the number of retry time should be limited:since region may be unavailable
|
||||
// when some unrecoverable disaster happened.
|
||||
return bo.Backoff(retry.BoTiKVRPC, errors.Errorf("send tikv request error: %v, ctx: %v, try next peer later", err, ctx))
|
||||
}
|
||||
|
||||
func regionErrorToLabel(e *errorpb.Error) string {
|
||||
if e.GetNotLeader() != nil {
|
||||
return "not_leader"
|
||||
} else if e.GetRegionNotFound() != nil {
|
||||
return "region_not_found"
|
||||
} else if e.GetKeyNotInRegion() != nil {
|
||||
return "key_not_in_region"
|
||||
} else if e.GetEpochNotMatch() != nil {
|
||||
return "epoch_not_match"
|
||||
} else if e.GetServerIsBusy() != nil {
|
||||
return "server_is_busy"
|
||||
} else if e.GetStaleCommand() != nil {
|
||||
return "stale_command"
|
||||
} else if e.GetStoreNotMatch() != nil {
|
||||
return "store_not_match"
|
||||
}
|
||||
return "unknown"
|
||||
}
|
||||
|
||||
func (s *RegionRequestSender) onRegionError(bo *retry.Backoffer, ctx *locate.RPCContext, regionErr *errorpb.Error) (retryable bool, err error) {
|
||||
metrics.RegionErrorCounter.WithLabelValues(regionErrorToLabel(regionErr)).Inc()
|
||||
if notLeader := regionErr.GetNotLeader(); notLeader != nil {
|
||||
// Retry if error is `NotLeader`.
|
||||
log.Debugf("tikv reports `NotLeader`: %s, ctx: %v, retry later", notLeader, ctx)
|
||||
s.regionCache.UpdateLeader(ctx.Region, notLeader.GetLeader().GetStoreId())
|
||||
|
||||
var boType retry.BackoffType
|
||||
if notLeader.GetLeader() != nil {
|
||||
boType = retry.BoUpdateLeader
|
||||
} else {
|
||||
boType = retry.BoRegionMiss
|
||||
}
|
||||
|
||||
if err = bo.Backoff(boType, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
return true, nil
|
||||
}
|
||||
|
||||
if storeNotMatch := regionErr.GetStoreNotMatch(); storeNotMatch != nil {
|
||||
// store not match
|
||||
log.Warnf("tikv reports `StoreNotMatch`: %s, ctx: %v, retry later", storeNotMatch, ctx)
|
||||
s.regionCache.ClearStoreByID(ctx.GetStoreID())
|
||||
return true, nil
|
||||
}
|
||||
|
||||
if epochNotMatch := regionErr.GetEpochNotMatch(); epochNotMatch != nil {
|
||||
log.Debugf("tikv reports `StaleEpoch`, ctx: %v, retry later", ctx)
|
||||
err = s.regionCache.OnRegionStale(ctx, epochNotMatch.CurrentRegions)
|
||||
return false, err
|
||||
}
|
||||
if regionErr.GetServerIsBusy() != nil {
|
||||
log.Warnf("tikv reports `ServerIsBusy`, reason: %s, ctx: %v, retry later", regionErr.GetServerIsBusy().GetReason(), ctx)
|
||||
err = bo.Backoff(retry.BoServerBusy, errors.Errorf("server is busy, ctx: %v", ctx))
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
if regionErr.GetStaleCommand() != nil {
|
||||
log.Debugf("tikv reports `StaleCommand`, ctx: %v", ctx)
|
||||
return true, nil
|
||||
}
|
||||
if regionErr.GetRaftEntryTooLarge() != nil {
|
||||
log.Warnf("tikv reports `RaftEntryTooLarge`, ctx: %v", ctx)
|
||||
return false, errors.New(regionErr.String())
|
||||
}
|
||||
// For other errors, we only drop cache here.
|
||||
// Because caller may need to re-split the request.
|
||||
log.Debugf("tikv reports region error: %s, ctx: %v", regionErr, ctx)
|
||||
s.regionCache.DropRegion(ctx.Region)
|
||||
return false, nil
|
||||
}
|
||||
|
|
@ -1,62 +0,0 @@
|
|||
// 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 txnkv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"github.com/tikv/client-go/txnkv/store"
|
||||
)
|
||||
|
||||
// Client is a transactional client of TiKV server.
|
||||
type Client struct {
|
||||
tikvStore *store.TiKVStore
|
||||
}
|
||||
|
||||
// NewClient creates a client with PD addresses.
|
||||
func NewClient(ctx context.Context, pdAddrs []string, config config.Config) (*Client, error) {
|
||||
tikvStore, err := store.NewStore(ctx, pdAddrs, config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Client{
|
||||
tikvStore: tikvStore,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Close stop the client.
|
||||
func (c *Client) Close() error {
|
||||
return c.tikvStore.Close()
|
||||
}
|
||||
|
||||
// Begin creates a transaction for read/write.
|
||||
func (c *Client) Begin(ctx context.Context) (*Transaction, error) {
|
||||
ts, err := c.GetTS(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return c.BeginWithTS(ctx, ts), nil
|
||||
}
|
||||
|
||||
// BeginWithTS creates a transaction which is normally readonly.
|
||||
func (c *Client) BeginWithTS(ctx context.Context, ts uint64) *Transaction {
|
||||
return newTransaction(c.tikvStore, ts)
|
||||
}
|
||||
|
||||
// GetTS returns a latest timestamp.
|
||||
func (c *Client) GetTS(ctx context.Context) (uint64, error) {
|
||||
return c.tikvStore.GetTimestampWithRetry(retry.NewBackoffer(ctx, retry.TsoMaxBackoff))
|
||||
}
|
||||
|
|
@ -1,108 +0,0 @@
|
|||
// Copyright 2015 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 kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/key"
|
||||
)
|
||||
|
||||
// BufferStore wraps a Retriever for read and a MemBuffer for buffered write.
|
||||
// Common usage pattern:
|
||||
// bs := NewBufferStore(r) // use BufferStore to wrap a Retriever
|
||||
// // ...
|
||||
// // read/write on bs
|
||||
// // ...
|
||||
// bs.SaveTo(m) // save above operations to a Mutator
|
||||
type BufferStore struct {
|
||||
MemBuffer
|
||||
r Retriever
|
||||
}
|
||||
|
||||
// NewBufferStore creates a BufferStore using r for read.
|
||||
func NewBufferStore(r Retriever, conf *config.Txn) *BufferStore {
|
||||
return &BufferStore{
|
||||
r: r,
|
||||
MemBuffer: &lazyMemBuffer{conf: conf},
|
||||
}
|
||||
}
|
||||
|
||||
// Reset resets s.MemBuffer.
|
||||
func (s *BufferStore) Reset() {
|
||||
s.MemBuffer.Reset()
|
||||
}
|
||||
|
||||
// SetCap sets the MemBuffer capability.
|
||||
func (s *BufferStore) SetCap(cap int) {
|
||||
s.MemBuffer.SetCap(cap)
|
||||
}
|
||||
|
||||
// Get implements the Retriever interface.
|
||||
func (s *BufferStore) Get(ctx context.Context, k key.Key) ([]byte, error) {
|
||||
val, err := s.MemBuffer.Get(ctx, k)
|
||||
if IsErrNotFound(err) {
|
||||
val, err = s.r.Get(ctx, k)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(val) == 0 {
|
||||
return nil, ErrNotExist
|
||||
}
|
||||
return val, nil
|
||||
}
|
||||
|
||||
// Iter implements the Retriever interface.
|
||||
func (s *BufferStore) Iter(ctx context.Context, k key.Key, upperBound key.Key) (Iterator, error) {
|
||||
bufferIt, err := s.MemBuffer.Iter(ctx, k, upperBound)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
retrieverIt, err := s.r.Iter(ctx, k, upperBound)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewUnionIter(ctx, bufferIt, retrieverIt, false)
|
||||
}
|
||||
|
||||
// IterReverse implements the Retriever interface.
|
||||
func (s *BufferStore) IterReverse(ctx context.Context, k key.Key) (Iterator, error) {
|
||||
bufferIt, err := s.MemBuffer.IterReverse(ctx, k)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
retrieverIt, err := s.r.IterReverse(ctx, k)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewUnionIter(ctx, bufferIt, retrieverIt, true)
|
||||
}
|
||||
|
||||
// WalkBuffer iterates all buffered kv pairs.
|
||||
func (s *BufferStore) WalkBuffer(f func(k key.Key, v []byte) error) error {
|
||||
return WalkMemBuffer(s.MemBuffer, f)
|
||||
}
|
||||
|
||||
// SaveTo saves all buffered kv pairs into a Mutator.
|
||||
func (s *BufferStore) SaveTo(m Mutator) error {
|
||||
err := s.WalkBuffer(func(k key.Key, v []byte) error {
|
||||
if len(v) == 0 {
|
||||
return m.Delete(k)
|
||||
}
|
||||
return m.Set(k, v)
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
|
@ -1,73 +0,0 @@
|
|||
// Copyright 2016 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 kv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/key"
|
||||
)
|
||||
|
||||
func TestT(t *testing.T) {
|
||||
TestingT(t)
|
||||
}
|
||||
|
||||
type testBufferStoreSuite struct{}
|
||||
|
||||
var _ = Suite(testBufferStoreSuite{})
|
||||
|
||||
func (s testBufferStoreSuite) TestGetSet(c *C) {
|
||||
conf := config.DefaultTxn()
|
||||
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(&conf, 0)}, &conf)
|
||||
key := key.Key("key")
|
||||
_, err := bs.Get(context.TODO(), key)
|
||||
c.Check(err, NotNil)
|
||||
|
||||
err = bs.Set(key, []byte("value"))
|
||||
c.Check(err, IsNil)
|
||||
|
||||
value, err := bs.Get(context.TODO(), key)
|
||||
c.Check(err, IsNil)
|
||||
c.Check(bytes.Compare(value, []byte("value")), Equals, 0)
|
||||
}
|
||||
|
||||
func (s testBufferStoreSuite) TestSaveTo(c *C) {
|
||||
conf := config.DefaultTxn()
|
||||
bs := NewBufferStore(&mockSnapshot{NewMemDbBuffer(&conf, 0)}, &conf)
|
||||
var buf bytes.Buffer
|
||||
for i := 0; i < 10; i++ {
|
||||
fmt.Fprint(&buf, i)
|
||||
err := bs.Set(buf.Bytes(), buf.Bytes())
|
||||
c.Check(err, IsNil)
|
||||
buf.Reset()
|
||||
}
|
||||
bs.Set(key.Key("novalue"), nil)
|
||||
|
||||
mutator := NewMemDbBuffer(&conf, 0)
|
||||
err := bs.SaveTo(mutator)
|
||||
c.Check(err, IsNil)
|
||||
|
||||
iter, err := mutator.Iter(context.TODO(), nil, nil)
|
||||
c.Check(err, IsNil)
|
||||
for iter.Valid() {
|
||||
cmp := bytes.Compare(iter.Key(), iter.Value())
|
||||
c.Check(cmp, Equals, 0)
|
||||
iter.Next(context.TODO())
|
||||
}
|
||||
}
|
||||
|
|
@ -1,38 +0,0 @@
|
|||
// 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 kv
|
||||
|
||||
import (
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
var (
|
||||
// ErrNotExist is used when try to get an entry with an unexist key from KV store.
|
||||
ErrNotExist = errors.New("key not exist")
|
||||
// ErrCannotSetNilValue is the error when sets an empty value.
|
||||
ErrCannotSetNilValue = errors.New("can not set nil value")
|
||||
// ErrTxnTooLarge is the error when transaction is too large, lock time reached the maximum value.
|
||||
ErrTxnTooLarge = errors.New("transaction is too large")
|
||||
// ErrEntryTooLarge is the error when a key value entry is too large.
|
||||
ErrEntryTooLarge = errors.New("entry is too large")
|
||||
// ErrKeyExists returns when key is already exist.
|
||||
ErrKeyExists = errors.New("key already exist")
|
||||
// ErrInvalidTxn is the error that using a transaction after calling Commit or Rollback.
|
||||
ErrInvalidTxn = errors.New("invalid transaction")
|
||||
)
|
||||
|
||||
// IsErrNotFound checks if err is a kind of NotFound error.
|
||||
func IsErrNotFound(err error) bool {
|
||||
return errors.Cause(err) == ErrNotExist
|
||||
}
|
||||
128
txnkv/kv/kv.go
128
txnkv/kv/kv.go
|
|
@ -1,128 +0,0 @@
|
|||
// 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 kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/tikv/client-go/key"
|
||||
)
|
||||
|
||||
// Priority value for transaction priority.
|
||||
const (
|
||||
PriorityNormal = iota
|
||||
PriorityLow
|
||||
PriorityHigh
|
||||
)
|
||||
|
||||
// IsoLevel is the transaction's isolation level.
|
||||
type IsoLevel int
|
||||
|
||||
const (
|
||||
// SI stands for 'snapshot isolation'.
|
||||
SI IsoLevel = iota
|
||||
// RC stands for 'read committed'.
|
||||
RC
|
||||
)
|
||||
|
||||
// Retriever is the interface wraps the basic Get and Seek methods.
|
||||
type Retriever interface {
|
||||
// Get gets the value for key k from kv store.
|
||||
// If corresponding kv pair does not exist, it returns nil and ErrNotExist.
|
||||
Get(ctx context.Context, k key.Key) ([]byte, error)
|
||||
// Iter creates an Iterator positioned on the first entry that k <= entry's key.
|
||||
// If such entry is not found, it returns an invalid Iterator with no error.
|
||||
// It yields only keys that < upperBound. If upperBound is nil, it means the upperBound is unbounded.
|
||||
// The Iterator must be closed after use.
|
||||
Iter(ctx context.Context, k key.Key, upperBound key.Key) (Iterator, error)
|
||||
|
||||
// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k.
|
||||
// The returned iterator will iterate from greater key to smaller key.
|
||||
// If k is nil, the returned iterator will be positioned at the last key.
|
||||
// TODO: Add lower bound limit
|
||||
IterReverse(ctx context.Context, k key.Key) (Iterator, error)
|
||||
}
|
||||
|
||||
// Mutator is the interface wraps the basic Set and Delete methods.
|
||||
type Mutator interface {
|
||||
// Set sets the value for key k as v into kv store.
|
||||
// v must NOT be nil or empty, otherwise it returns ErrCannotSetNilValue.
|
||||
Set(k key.Key, v []byte) error
|
||||
// Delete removes the entry for key k from kv store.
|
||||
Delete(k key.Key) error
|
||||
}
|
||||
|
||||
// RetrieverMutator is the interface that groups Retriever and Mutator interfaces.
|
||||
type RetrieverMutator interface {
|
||||
Retriever
|
||||
Mutator
|
||||
}
|
||||
|
||||
// MemBuffer is an in-memory kv collection, can be used to buffer write operations.
|
||||
type MemBuffer interface {
|
||||
RetrieverMutator
|
||||
// Size returns sum of keys and values length.
|
||||
Size() int
|
||||
// Len returns the number of entries in the DB.
|
||||
Len() int
|
||||
// Reset cleanup the MemBuffer
|
||||
Reset()
|
||||
// SetCap sets the MemBuffer capability, to reduce memory allocations.
|
||||
// Please call it before you use the MemBuffer, otherwise it will not works.
|
||||
SetCap(cap int)
|
||||
}
|
||||
|
||||
// Snapshot defines the interface for the snapshot fetched from KV store.
|
||||
type Snapshot interface {
|
||||
Retriever
|
||||
// BatchGet gets a batch of values from snapshot.
|
||||
BatchGet(ctx context.Context, keys []key.Key) (map[string][]byte, error)
|
||||
// SetPriority snapshot set the priority
|
||||
SetPriority(priority int)
|
||||
}
|
||||
|
||||
// Iterator is the interface for a iterator on KV store.
|
||||
type Iterator interface {
|
||||
Valid() bool
|
||||
Key() key.Key
|
||||
Value() []byte
|
||||
Next(context.Context) error
|
||||
Close()
|
||||
}
|
||||
|
||||
// Transaction options
|
||||
const (
|
||||
// PresumeKeyNotExists indicates that when dealing with a Get operation but failing to read data from cache,
|
||||
// we presume that the key does not exist in Store. The actual existence will be checked before the
|
||||
// transaction's commit.
|
||||
// This option is an optimization for frequent checks during a transaction, e.g. batch inserts.
|
||||
PresumeKeyNotExists Option = iota + 1
|
||||
// PresumeKeyNotExistsError is the option key for error.
|
||||
// When PresumeKeyNotExists is set and condition is not match, should throw the error.
|
||||
PresumeKeyNotExistsError
|
||||
// BinlogInfo contains the binlog data and client.
|
||||
BinlogInfo
|
||||
// SchemaChecker is used for checking schema-validity.
|
||||
SchemaChecker
|
||||
// IsolationLevel sets isolation level for current transaction. The default level is SI.
|
||||
IsolationLevel
|
||||
// Priority marks the priority of this transaction.
|
||||
Priority
|
||||
// NotFillCache makes this request do not touch the LRU cache of the underlying storage.
|
||||
NotFillCache
|
||||
// SyncLog decides whether the WAL(write-ahead log) of this request should be synchronized.
|
||||
SyncLog
|
||||
// KeyOnly retrieve only keys, it can be used in scan now.
|
||||
KeyOnly
|
||||
)
|
||||
|
|
@ -1,289 +0,0 @@
|
|||
// Copyright 2015 PingCAP, Inc.
|
||||
//
|
||||
// Copyright 2015 Wenbin Xiao
|
||||
//
|
||||
// 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 kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"testing"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/tikv/client-go/config"
|
||||
)
|
||||
|
||||
const (
|
||||
startIndex = 0
|
||||
testCount = 2
|
||||
indexStep = 2
|
||||
)
|
||||
|
||||
var _ = Suite(&testKVSuite{})
|
||||
|
||||
type testKVSuite struct {
|
||||
bs []MemBuffer
|
||||
}
|
||||
|
||||
func (s *testKVSuite) SetUpSuite(c *C) {
|
||||
conf := config.DefaultTxn()
|
||||
s.bs = make([]MemBuffer, 1)
|
||||
s.bs[0] = NewMemDbBuffer(&conf, 0)
|
||||
}
|
||||
|
||||
func (s *testKVSuite) ResetMembuffers() {
|
||||
conf := config.DefaultTxn()
|
||||
s.bs[0] = NewMemDbBuffer(&conf, 0)
|
||||
}
|
||||
|
||||
func insertData(c *C, buffer MemBuffer) {
|
||||
for i := startIndex; i < testCount; i++ {
|
||||
val := encodeInt(i * indexStep)
|
||||
err := buffer.Set(val, val)
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
}
|
||||
|
||||
func encodeInt(n int) []byte {
|
||||
return []byte(fmt.Sprintf("%010d", n))
|
||||
}
|
||||
|
||||
func decodeInt(s []byte) int {
|
||||
var n int
|
||||
fmt.Sscanf(string(s), "%010d", &n)
|
||||
return n
|
||||
}
|
||||
|
||||
func valToStr(c *C, iter Iterator) string {
|
||||
val := iter.Value()
|
||||
return string(val)
|
||||
}
|
||||
|
||||
func checkNewIterator(c *C, buffer MemBuffer) {
|
||||
for i := startIndex; i < testCount; i++ {
|
||||
val := encodeInt(i * indexStep)
|
||||
iter, err := buffer.Iter(context.TODO(), val, nil)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert([]byte(iter.Key()), BytesEquals, val)
|
||||
c.Assert(decodeInt([]byte(valToStr(c, iter))), Equals, i*indexStep)
|
||||
iter.Close()
|
||||
}
|
||||
|
||||
// Test iterator Next()
|
||||
for i := startIndex; i < testCount-1; i++ {
|
||||
val := encodeInt(i * indexStep)
|
||||
iter, err := buffer.Iter(context.TODO(), val, nil)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert([]byte(iter.Key()), BytesEquals, val)
|
||||
c.Assert(valToStr(c, iter), Equals, string(val))
|
||||
|
||||
err = iter.Next(context.TODO())
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(iter.Valid(), IsTrue)
|
||||
|
||||
val = encodeInt((i + 1) * indexStep)
|
||||
c.Assert([]byte(iter.Key()), BytesEquals, val)
|
||||
c.Assert(valToStr(c, iter), Equals, string(val))
|
||||
iter.Close()
|
||||
}
|
||||
|
||||
// Non exist and beyond maximum seek test
|
||||
iter, err := buffer.Iter(context.TODO(), encodeInt(testCount*indexStep), nil)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(iter.Valid(), IsFalse)
|
||||
|
||||
// Non exist but between existing keys seek test,
|
||||
// it returns the smallest key that larger than the one we are seeking
|
||||
inBetween := encodeInt((testCount-1)*indexStep - 1)
|
||||
last := encodeInt((testCount - 1) * indexStep)
|
||||
iter, err = buffer.Iter(context.TODO(), inBetween, nil)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(iter.Valid(), IsTrue)
|
||||
c.Assert([]byte(iter.Key()), Not(BytesEquals), inBetween)
|
||||
c.Assert([]byte(iter.Key()), BytesEquals, last)
|
||||
iter.Close()
|
||||
}
|
||||
|
||||
func mustGet(c *C, buffer MemBuffer) {
|
||||
for i := startIndex; i < testCount; i++ {
|
||||
s := encodeInt(i * indexStep)
|
||||
val, err := buffer.Get(context.TODO(), s)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(val), Equals, string(s))
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testKVSuite) TestGetSet(c *C) {
|
||||
for _, buffer := range s.bs {
|
||||
insertData(c, buffer)
|
||||
mustGet(c, buffer)
|
||||
}
|
||||
s.ResetMembuffers()
|
||||
}
|
||||
|
||||
func (s *testKVSuite) TestNewIterator(c *C) {
|
||||
for _, buffer := range s.bs {
|
||||
// should be invalid
|
||||
iter, err := buffer.Iter(context.TODO(), nil, nil)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(iter.Valid(), IsFalse)
|
||||
|
||||
insertData(c, buffer)
|
||||
checkNewIterator(c, buffer)
|
||||
}
|
||||
s.ResetMembuffers()
|
||||
}
|
||||
|
||||
func (s *testKVSuite) TestBasicNewIterator(c *C) {
|
||||
for _, buffer := range s.bs {
|
||||
it, err := buffer.Iter(context.TODO(), []byte("2"), nil)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(it.Valid(), IsFalse)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testKVSuite) TestNewIteratorMin(c *C) {
|
||||
kvs := []struct {
|
||||
key string
|
||||
value string
|
||||
}{
|
||||
{"DATA_test_main_db_tbl_tbl_test_record__00000000000000000001", "lock-version"},
|
||||
{"DATA_test_main_db_tbl_tbl_test_record__00000000000000000001_0002", "1"},
|
||||
{"DATA_test_main_db_tbl_tbl_test_record__00000000000000000001_0003", "hello"},
|
||||
{"DATA_test_main_db_tbl_tbl_test_record__00000000000000000002", "lock-version"},
|
||||
{"DATA_test_main_db_tbl_tbl_test_record__00000000000000000002_0002", "2"},
|
||||
{"DATA_test_main_db_tbl_tbl_test_record__00000000000000000002_0003", "hello"},
|
||||
}
|
||||
for _, buffer := range s.bs {
|
||||
for _, kv := range kvs {
|
||||
buffer.Set([]byte(kv.key), []byte(kv.value))
|
||||
}
|
||||
|
||||
cnt := 0
|
||||
it, err := buffer.Iter(context.TODO(), nil, nil)
|
||||
c.Assert(err, IsNil)
|
||||
for it.Valid() {
|
||||
cnt++
|
||||
it.Next(context.TODO())
|
||||
}
|
||||
c.Assert(cnt, Equals, 6)
|
||||
|
||||
it, err = buffer.Iter(context.TODO(), []byte("DATA_test_main_db_tbl_tbl_test_record__00000000000000000000"), nil)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(it.Key()), Equals, "DATA_test_main_db_tbl_tbl_test_record__00000000000000000001")
|
||||
}
|
||||
s.ResetMembuffers()
|
||||
}
|
||||
|
||||
func (s *testKVSuite) TestBufferLimit(c *C) {
|
||||
conf := config.DefaultTxn()
|
||||
buffer := NewMemDbBuffer(&conf, 0).(*memDbBuffer)
|
||||
buffer.bufferSizeLimit = 1000
|
||||
buffer.entrySizeLimit = 500
|
||||
|
||||
err := buffer.Set([]byte("x"), make([]byte, 500))
|
||||
c.Assert(err, NotNil) // entry size limit
|
||||
|
||||
err = buffer.Set([]byte("x"), make([]byte, 499))
|
||||
c.Assert(err, IsNil)
|
||||
err = buffer.Set([]byte("yz"), make([]byte, 499))
|
||||
c.Assert(err, NotNil) // buffer size limit
|
||||
|
||||
buffer = NewMemDbBuffer(&conf, 0).(*memDbBuffer)
|
||||
buffer.bufferLenLimit = 10
|
||||
for i := 0; i < 10; i++ {
|
||||
err = buffer.Set([]byte{byte(i)}, []byte{byte(i)})
|
||||
c.Assert(err, IsNil)
|
||||
}
|
||||
err = buffer.Set([]byte("x"), []byte("y"))
|
||||
c.Assert(err, NotNil) // buffer len limit
|
||||
}
|
||||
|
||||
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(&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(&conf, 0)
|
||||
benchmarkSetGet(b, buffer, data)
|
||||
b.ReportAllocs()
|
||||
}
|
||||
|
||||
func BenchmarkMemDbIter(b *testing.B) {
|
||||
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(&conf, 0)
|
||||
}
|
||||
b.ReportAllocs()
|
||||
}
|
||||
|
||||
func shuffle(slc [][]byte) {
|
||||
N := len(slc)
|
||||
for i := 0; i < N; i++ {
|
||||
// choose index uniformly in [i, N-1]
|
||||
r := i + rand.Intn(N-i)
|
||||
slc[r], slc[i] = slc[i], slc[r]
|
||||
}
|
||||
}
|
||||
func benchmarkSetGet(b *testing.B, buffer MemBuffer, data [][]byte) {
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
for _, k := range data {
|
||||
buffer.Set(k, k)
|
||||
}
|
||||
for _, k := range data {
|
||||
buffer.Get(context.TODO(), k)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func benchIterator(b *testing.B, buffer MemBuffer) {
|
||||
for k := 0; k < opCnt; k++ {
|
||||
buffer.Set(encodeInt(k), encodeInt(k))
|
||||
}
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
iter, err := buffer.Iter(context.TODO(), nil, nil)
|
||||
if err != nil {
|
||||
b.Error(err)
|
||||
}
|
||||
for iter.Valid() {
|
||||
iter.Next(context.TODO())
|
||||
}
|
||||
iter.Close()
|
||||
}
|
||||
}
|
||||
|
|
@ -1,182 +0,0 @@
|
|||
// Copyright 2015 PingCAP, Inc.
|
||||
//
|
||||
// Copyright 2015 Wenbin Xiao
|
||||
//
|
||||
// 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 kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"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"
|
||||
)
|
||||
|
||||
// memDbBuffer implements the MemBuffer interface.
|
||||
type memDbBuffer struct {
|
||||
db *memdb.DB
|
||||
entrySizeLimit int
|
||||
bufferLenLimit int
|
||||
bufferSizeLimit int
|
||||
}
|
||||
|
||||
type memDbIter struct {
|
||||
iter iterator.Iterator
|
||||
reverse bool
|
||||
}
|
||||
|
||||
// NewMemDbBuffer creates a new memDbBuffer.
|
||||
func NewMemDbBuffer(conf *config.Txn, cap int) MemBuffer {
|
||||
if cap <= 0 {
|
||||
cap = conf.DefaultMembufCap
|
||||
}
|
||||
return &memDbBuffer{
|
||||
db: memdb.New(comparer.DefaultComparer, cap),
|
||||
entrySizeLimit: conf.EntrySizeLimit,
|
||||
bufferLenLimit: conf.EntryCountLimit,
|
||||
bufferSizeLimit: conf.TotalSizeLimit,
|
||||
}
|
||||
}
|
||||
|
||||
// Iter creates an Iterator.
|
||||
func (m *memDbBuffer) Iter(ctx context.Context, k key.Key, upperBound key.Key) (Iterator, error) {
|
||||
i := &memDbIter{iter: m.db.NewIterator(&util.Range{Start: []byte(k), Limit: []byte(upperBound)}), reverse: false}
|
||||
|
||||
err := i.Next(ctx)
|
||||
if err != nil {
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
|
||||
func (m *memDbBuffer) SetCap(cap int) {
|
||||
|
||||
}
|
||||
|
||||
func (m *memDbBuffer) IterReverse(ctx context.Context, k key.Key) (Iterator, error) {
|
||||
var i *memDbIter
|
||||
if k == nil {
|
||||
i = &memDbIter{iter: m.db.NewIterator(&util.Range{}), reverse: true}
|
||||
} else {
|
||||
i = &memDbIter{iter: m.db.NewIterator(&util.Range{Limit: []byte(k)}), reverse: true}
|
||||
}
|
||||
i.iter.Last()
|
||||
return i, nil
|
||||
}
|
||||
|
||||
// Get returns the value associated with key.
|
||||
func (m *memDbBuffer) Get(ctx context.Context, k key.Key) ([]byte, error) {
|
||||
v, err := m.db.Get(k)
|
||||
if err == leveldb.ErrNotFound {
|
||||
return nil, ErrNotExist
|
||||
}
|
||||
return v, nil
|
||||
}
|
||||
|
||||
// Set associates key with value.
|
||||
func (m *memDbBuffer) Set(k key.Key, v []byte) error {
|
||||
if len(v) == 0 {
|
||||
return errors.WithStack(ErrCannotSetNilValue)
|
||||
}
|
||||
if len(k)+len(v) > m.entrySizeLimit {
|
||||
return errors.WithMessage(ErrEntryTooLarge, fmt.Sprintf("entry too large, size: %d", len(k)+len(v)))
|
||||
}
|
||||
|
||||
err := m.db.Put(k, v)
|
||||
if m.Size() > m.bufferSizeLimit {
|
||||
return errors.WithMessage(ErrTxnTooLarge, fmt.Sprintf("transaction too large, size:%d", m.Size()))
|
||||
}
|
||||
if m.Len() > int(m.bufferLenLimit) {
|
||||
return errors.WithMessage(ErrTxnTooLarge, fmt.Sprintf("transaction too large, size:%d", m.Size()))
|
||||
}
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
|
||||
// Delete removes the entry from buffer with provided key.
|
||||
func (m *memDbBuffer) Delete(k key.Key) error {
|
||||
err := m.db.Put(k, nil)
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
|
||||
// Size returns sum of keys and values length.
|
||||
func (m *memDbBuffer) Size() int {
|
||||
return m.db.Size()
|
||||
}
|
||||
|
||||
// Len returns the number of entries in the DB.
|
||||
func (m *memDbBuffer) Len() int {
|
||||
return m.db.Len()
|
||||
}
|
||||
|
||||
// Reset cleanup the MemBuffer.
|
||||
func (m *memDbBuffer) Reset() {
|
||||
m.db.Reset()
|
||||
}
|
||||
|
||||
// Next implements the Iterator Next.
|
||||
func (i *memDbIter) Next(context.Context) error {
|
||||
if i.reverse {
|
||||
i.iter.Prev()
|
||||
} else {
|
||||
i.iter.Next()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Valid implements the Iterator Valid.
|
||||
func (i *memDbIter) Valid() bool {
|
||||
return i.iter.Valid()
|
||||
}
|
||||
|
||||
// Key implements the Iterator Key.
|
||||
func (i *memDbIter) Key() key.Key {
|
||||
return i.iter.Key()
|
||||
}
|
||||
|
||||
// Value implements the Iterator Value.
|
||||
func (i *memDbIter) Value() []byte {
|
||||
return i.iter.Value()
|
||||
}
|
||||
|
||||
// Close Implements the Iterator Close.
|
||||
func (i *memDbIter) Close() {
|
||||
i.iter.Release()
|
||||
}
|
||||
|
||||
// WalkMemBuffer iterates all buffered kv pairs in memBuf
|
||||
func WalkMemBuffer(memBuf MemBuffer, f func(k key.Key, v []byte) error) error {
|
||||
iter, err := memBuf.Iter(context.Background(), nil, nil)
|
||||
if err != nil {
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
|
||||
defer iter.Close()
|
||||
for iter.Valid() {
|
||||
if err = f(iter.Key(), iter.Value()); err != nil {
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
err = iter.Next(context.Background())
|
||||
if err != nil {
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
@ -1,55 +0,0 @@
|
|||
// Copyright 2016 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 kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/tikv/client-go/key"
|
||||
)
|
||||
|
||||
type mockSnapshot struct {
|
||||
store MemBuffer
|
||||
}
|
||||
|
||||
func (s *mockSnapshot) Get(ctx context.Context, k key.Key) ([]byte, error) {
|
||||
return s.store.Get(ctx, k)
|
||||
}
|
||||
|
||||
func (s *mockSnapshot) SetPriority(priority int) {
|
||||
|
||||
}
|
||||
|
||||
func (s *mockSnapshot) BatchGet(ctx context.Context, keys []key.Key) (map[string][]byte, error) {
|
||||
m := make(map[string][]byte)
|
||||
for _, k := range keys {
|
||||
v, err := s.store.Get(ctx, k)
|
||||
if IsErrNotFound(err) {
|
||||
continue
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
m[string(k)] = v
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (s *mockSnapshot) Iter(ctx context.Context, k key.Key, upperBound key.Key) (Iterator, error) {
|
||||
return s.store.Iter(ctx, k, upperBound)
|
||||
}
|
||||
|
||||
func (s *mockSnapshot) IterReverse(ctx context.Context, k key.Key) (Iterator, error) {
|
||||
return s.store.IterReverse(ctx, k)
|
||||
}
|
||||
|
|
@ -1,185 +0,0 @@
|
|||
// Copyright 2015 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 kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/key"
|
||||
)
|
||||
|
||||
// UnionIter is the iterator on an UnionStore.
|
||||
type UnionIter struct {
|
||||
dirtyIt Iterator
|
||||
snapshotIt Iterator
|
||||
|
||||
dirtyValid bool
|
||||
snapshotValid bool
|
||||
|
||||
curIsDirty bool
|
||||
isValid bool
|
||||
reverse bool
|
||||
}
|
||||
|
||||
// NewUnionIter returns a union iterator for BufferStore.
|
||||
func NewUnionIter(ctx context.Context, dirtyIt Iterator, snapshotIt Iterator, reverse bool) (*UnionIter, error) {
|
||||
it := &UnionIter{
|
||||
dirtyIt: dirtyIt,
|
||||
snapshotIt: snapshotIt,
|
||||
dirtyValid: dirtyIt.Valid(),
|
||||
snapshotValid: snapshotIt.Valid(),
|
||||
reverse: reverse,
|
||||
}
|
||||
err := it.updateCur(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return it, nil
|
||||
}
|
||||
|
||||
// dirtyNext makes iter.dirtyIt go and update valid status.
|
||||
func (iter *UnionIter) dirtyNext(ctx context.Context) error {
|
||||
err := iter.dirtyIt.Next(ctx)
|
||||
iter.dirtyValid = iter.dirtyIt.Valid()
|
||||
return err
|
||||
}
|
||||
|
||||
// snapshotNext makes iter.snapshotIt go and update valid status.
|
||||
func (iter *UnionIter) snapshotNext(ctx context.Context) error {
|
||||
err := iter.snapshotIt.Next(ctx)
|
||||
iter.snapshotValid = iter.snapshotIt.Valid()
|
||||
return err
|
||||
}
|
||||
|
||||
func (iter *UnionIter) updateCur(ctx context.Context) error {
|
||||
iter.isValid = true
|
||||
for {
|
||||
if !iter.dirtyValid && !iter.snapshotValid {
|
||||
iter.isValid = false
|
||||
break
|
||||
}
|
||||
|
||||
if !iter.dirtyValid {
|
||||
iter.curIsDirty = false
|
||||
break
|
||||
}
|
||||
|
||||
if !iter.snapshotValid {
|
||||
iter.curIsDirty = true
|
||||
// if delete it
|
||||
if len(iter.dirtyIt.Value()) == 0 {
|
||||
if err := iter.dirtyNext(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
// both valid
|
||||
if iter.snapshotValid && iter.dirtyValid {
|
||||
snapshotKey := iter.snapshotIt.Key()
|
||||
dirtyKey := iter.dirtyIt.Key()
|
||||
cmp := dirtyKey.Cmp(snapshotKey)
|
||||
if iter.reverse {
|
||||
cmp = -cmp
|
||||
}
|
||||
// if equal, means both have value
|
||||
if cmp == 0 {
|
||||
if len(iter.dirtyIt.Value()) == 0 {
|
||||
// snapshot has a record, but txn says we have deleted it
|
||||
// just go next
|
||||
if err := iter.dirtyNext(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := iter.snapshotNext(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
if err := iter.snapshotNext(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
iter.curIsDirty = true
|
||||
break
|
||||
} else if cmp > 0 {
|
||||
// record from snapshot comes first
|
||||
iter.curIsDirty = false
|
||||
break
|
||||
} else {
|
||||
// record from dirty comes first
|
||||
if len(iter.dirtyIt.Value()) == 0 {
|
||||
log.Warnf("[kv] delete a record not exists? k = %q", iter.dirtyIt.Key())
|
||||
// jump over this deletion
|
||||
if err := iter.dirtyNext(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
iter.curIsDirty = true
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Next implements the Iterator Next interface.
|
||||
func (iter *UnionIter) Next(ctx context.Context) error {
|
||||
var err error
|
||||
if !iter.curIsDirty {
|
||||
err = iter.snapshotNext(ctx)
|
||||
} else {
|
||||
err = iter.dirtyNext(ctx)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return iter.updateCur(ctx)
|
||||
}
|
||||
|
||||
// Value implements the Iterator Value interface.
|
||||
// Multi columns
|
||||
func (iter *UnionIter) Value() []byte {
|
||||
if !iter.curIsDirty {
|
||||
return iter.snapshotIt.Value()
|
||||
}
|
||||
return iter.dirtyIt.Value()
|
||||
}
|
||||
|
||||
// Key implements the Iterator Key interface.
|
||||
func (iter *UnionIter) Key() key.Key {
|
||||
if !iter.curIsDirty {
|
||||
return iter.snapshotIt.Key()
|
||||
}
|
||||
return iter.dirtyIt.Key()
|
||||
}
|
||||
|
||||
// Valid implements the Iterator Valid interface.
|
||||
func (iter *UnionIter) Valid() bool {
|
||||
return iter.isValid
|
||||
}
|
||||
|
||||
// Close implements the Iterator Close interface.
|
||||
func (iter *UnionIter) Close() {
|
||||
if iter.snapshotIt != nil {
|
||||
iter.snapshotIt.Close()
|
||||
iter.snapshotIt = nil
|
||||
}
|
||||
if iter.dirtyIt != nil {
|
||||
iter.dirtyIt.Close()
|
||||
iter.dirtyIt = nil
|
||||
}
|
||||
}
|
||||
|
|
@ -1,252 +0,0 @@
|
|||
// Copyright 2015 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 kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/key"
|
||||
)
|
||||
|
||||
// UnionStore is a store that wraps a snapshot for read and a BufferStore for buffered write.
|
||||
// Also, it provides some transaction related utilities.
|
||||
type UnionStore interface {
|
||||
MemBuffer
|
||||
// Returns related condition pair
|
||||
LookupConditionPair(k key.Key) *conditionPair
|
||||
// WalkBuffer iterates all buffered kv pairs.
|
||||
WalkBuffer(f func(k key.Key, v []byte) error) error
|
||||
// SetOption sets an option with a value, when val is nil, uses the default
|
||||
// value of this option.
|
||||
SetOption(opt Option, val interface{})
|
||||
// DelOption deletes an option.
|
||||
DelOption(opt Option)
|
||||
// GetOption gets an option.
|
||||
GetOption(opt Option) interface{}
|
||||
// GetMemBuffer return the MemBuffer binding to this UnionStore.
|
||||
GetMemBuffer() MemBuffer
|
||||
}
|
||||
|
||||
// Option is used for customizing kv store's behaviors during a transaction.
|
||||
type Option int
|
||||
|
||||
// Options is an interface of a set of options. Each option is associated with a value.
|
||||
type Options interface {
|
||||
// Get gets an option value.
|
||||
Get(opt Option) (v interface{}, ok bool)
|
||||
}
|
||||
|
||||
// conditionPair is used to store lazy check condition.
|
||||
// If condition not match (value is not equal as expected one), returns err.
|
||||
type conditionPair struct {
|
||||
key key.Key
|
||||
value []byte
|
||||
err error
|
||||
}
|
||||
|
||||
func (c *conditionPair) ShouldNotExist() bool {
|
||||
return len(c.value) == 0
|
||||
}
|
||||
|
||||
func (c *conditionPair) Err() error {
|
||||
return c.err
|
||||
}
|
||||
|
||||
// unionStore is an in-memory Store which contains a buffer for write and a
|
||||
// snapshot for read.
|
||||
type unionStore struct {
|
||||
*BufferStore
|
||||
snapshot Snapshot // for read
|
||||
lazyConditionPairs map[string]*conditionPair // for delay check
|
||||
opts options
|
||||
}
|
||||
|
||||
// NewUnionStore builds a new UnionStore.
|
||||
func NewUnionStore(conf *config.Txn, snapshot Snapshot) UnionStore {
|
||||
return &unionStore{
|
||||
BufferStore: NewBufferStore(snapshot, conf),
|
||||
snapshot: snapshot,
|
||||
lazyConditionPairs: make(map[string]*conditionPair),
|
||||
opts: make(map[Option]interface{}),
|
||||
}
|
||||
}
|
||||
|
||||
// invalidIterator implements Iterator interface.
|
||||
// It is used for read-only transaction which has no data written, the iterator is always invalid.
|
||||
type invalidIterator struct{}
|
||||
|
||||
func (it invalidIterator) Valid() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
func (it invalidIterator) Next(context.Context) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it invalidIterator) Key() key.Key {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it invalidIterator) Value() []byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it invalidIterator) Close() {}
|
||||
|
||||
// lazyMemBuffer wraps a MemBuffer which is to be initialized when it is modified.
|
||||
type lazyMemBuffer struct {
|
||||
mb MemBuffer
|
||||
cap int
|
||||
conf *config.Txn
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) Get(ctx context.Context, k key.Key) ([]byte, error) {
|
||||
if lmb.mb == nil {
|
||||
return nil, ErrNotExist
|
||||
}
|
||||
|
||||
return lmb.mb.Get(ctx, k)
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) Set(key key.Key, value []byte) error {
|
||||
if lmb.mb == nil {
|
||||
lmb.mb = NewMemDbBuffer(lmb.conf, lmb.cap)
|
||||
}
|
||||
|
||||
return lmb.mb.Set(key, value)
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) Delete(k key.Key) error {
|
||||
if lmb.mb == nil {
|
||||
lmb.mb = NewMemDbBuffer(lmb.conf, lmb.cap)
|
||||
}
|
||||
|
||||
return lmb.mb.Delete(k)
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) Iter(ctx context.Context, k key.Key, upperBound key.Key) (Iterator, error) {
|
||||
if lmb.mb == nil {
|
||||
return invalidIterator{}, nil
|
||||
}
|
||||
return lmb.mb.Iter(ctx, k, upperBound)
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) IterReverse(ctx context.Context, k key.Key) (Iterator, error) {
|
||||
if lmb.mb == nil {
|
||||
return invalidIterator{}, nil
|
||||
}
|
||||
return lmb.mb.IterReverse(ctx, k)
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) Size() int {
|
||||
if lmb.mb == nil {
|
||||
return 0
|
||||
}
|
||||
return lmb.mb.Size()
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) Len() int {
|
||||
if lmb.mb == nil {
|
||||
return 0
|
||||
}
|
||||
return lmb.mb.Len()
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) Reset() {
|
||||
if lmb.mb != nil {
|
||||
lmb.mb.Reset()
|
||||
}
|
||||
}
|
||||
|
||||
func (lmb *lazyMemBuffer) SetCap(cap int) {
|
||||
lmb.cap = cap
|
||||
}
|
||||
|
||||
// Get implements the Retriever interface.
|
||||
func (us *unionStore) Get(ctx context.Context, k key.Key) ([]byte, error) {
|
||||
v, err := us.MemBuffer.Get(ctx, k)
|
||||
if IsErrNotFound(err) {
|
||||
if _, ok := us.opts.Get(PresumeKeyNotExists); ok {
|
||||
e, ok := us.opts.Get(PresumeKeyNotExistsError)
|
||||
if ok && e != nil {
|
||||
us.markLazyConditionPair(k, nil, e.(error))
|
||||
} else {
|
||||
us.markLazyConditionPair(k, nil, ErrKeyExists)
|
||||
}
|
||||
return nil, ErrNotExist
|
||||
}
|
||||
v, err = us.BufferStore.r.Get(ctx, k)
|
||||
}
|
||||
if err != nil {
|
||||
return v, err
|
||||
}
|
||||
if len(v) == 0 {
|
||||
return nil, ErrNotExist
|
||||
}
|
||||
return v, nil
|
||||
}
|
||||
|
||||
// markLazyConditionPair marks a kv pair for later check.
|
||||
// If condition not match, should return e as error.
|
||||
func (us *unionStore) markLazyConditionPair(k key.Key, v []byte, e error) {
|
||||
us.lazyConditionPairs[string(k)] = &conditionPair{
|
||||
key: k.Clone(),
|
||||
value: v,
|
||||
err: e,
|
||||
}
|
||||
}
|
||||
|
||||
func (us *unionStore) LookupConditionPair(k key.Key) *conditionPair {
|
||||
if c, ok := us.lazyConditionPairs[string(k)]; ok {
|
||||
return c
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetOption implements the UnionStore SetOption interface.
|
||||
func (us *unionStore) SetOption(opt Option, val interface{}) {
|
||||
us.opts[opt] = val
|
||||
}
|
||||
|
||||
// DelOption implements the UnionStore DelOption interface.
|
||||
func (us *unionStore) DelOption(opt Option) {
|
||||
delete(us.opts, opt)
|
||||
}
|
||||
|
||||
// GetOption implements the UnionStore GetOption interface.
|
||||
func (us *unionStore) GetOption(opt Option) interface{} {
|
||||
return us.opts[opt]
|
||||
}
|
||||
|
||||
// GetMemBuffer return the MemBuffer binding to this UnionStore.
|
||||
func (us *unionStore) GetMemBuffer() MemBuffer {
|
||||
return us.BufferStore.MemBuffer
|
||||
}
|
||||
|
||||
// SetCap sets membuffer capability.
|
||||
func (us *unionStore) SetCap(cap int) {
|
||||
us.BufferStore.SetCap(cap)
|
||||
}
|
||||
|
||||
func (us *unionStore) Reset() {
|
||||
us.BufferStore.Reset()
|
||||
}
|
||||
|
||||
type options map[Option]interface{}
|
||||
|
||||
func (opts options) Get(opt Option) (interface{}, bool) {
|
||||
v, ok := opts[opt]
|
||||
return v, ok
|
||||
}
|
||||
|
|
@ -1,146 +0,0 @@
|
|||
// Copyright 2015 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 kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/config"
|
||||
)
|
||||
|
||||
var _ = Suite(&testUnionStoreSuite{})
|
||||
|
||||
type testUnionStoreSuite struct {
|
||||
store MemBuffer
|
||||
us UnionStore
|
||||
}
|
||||
|
||||
func (s *testUnionStoreSuite) SetUpTest(c *C) {
|
||||
conf := config.DefaultTxn()
|
||||
s.store = NewMemDbBuffer(&conf, 0)
|
||||
s.us = NewUnionStore(&conf, &mockSnapshot{s.store})
|
||||
}
|
||||
|
||||
func (s *testUnionStoreSuite) TestGetSet(c *C) {
|
||||
s.store.Set([]byte("1"), []byte("1"))
|
||||
v, err := s.us.Get(context.TODO(), []byte("1"))
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(v, BytesEquals, []byte("1"))
|
||||
s.us.Set([]byte("1"), []byte("2"))
|
||||
v, err = s.us.Get(context.TODO(), []byte("1"))
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(v, BytesEquals, []byte("2"))
|
||||
}
|
||||
|
||||
func (s *testUnionStoreSuite) TestDelete(c *C) {
|
||||
s.store.Set([]byte("1"), []byte("1"))
|
||||
err := s.us.Delete([]byte("1"))
|
||||
c.Assert(err, IsNil)
|
||||
_, err = s.us.Get(context.TODO(), []byte("1"))
|
||||
c.Assert(IsErrNotFound(err), IsTrue)
|
||||
|
||||
s.us.Set([]byte("1"), []byte("2"))
|
||||
v, err := s.us.Get(context.TODO(), []byte("1"))
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(v, BytesEquals, []byte("2"))
|
||||
}
|
||||
|
||||
func (s *testUnionStoreSuite) TestSeek(c *C) {
|
||||
s.store.Set([]byte("1"), []byte("1"))
|
||||
s.store.Set([]byte("2"), []byte("2"))
|
||||
s.store.Set([]byte("3"), []byte("3"))
|
||||
|
||||
iter, err := s.us.Iter(context.TODO(), nil, nil)
|
||||
c.Assert(err, IsNil)
|
||||
checkIterator(c, iter, [][]byte{[]byte("1"), []byte("2"), []byte("3")}, [][]byte{[]byte("1"), []byte("2"), []byte("3")})
|
||||
|
||||
iter, err = s.us.Iter(context.TODO(), []byte("2"), nil)
|
||||
c.Assert(err, IsNil)
|
||||
checkIterator(c, iter, [][]byte{[]byte("2"), []byte("3")}, [][]byte{[]byte("2"), []byte("3")})
|
||||
|
||||
s.us.Set([]byte("4"), []byte("4"))
|
||||
iter, err = s.us.Iter(context.TODO(), []byte("2"), nil)
|
||||
c.Assert(err, IsNil)
|
||||
checkIterator(c, iter, [][]byte{[]byte("2"), []byte("3"), []byte("4")}, [][]byte{[]byte("2"), []byte("3"), []byte("4")})
|
||||
|
||||
s.us.Delete([]byte("3"))
|
||||
iter, err = s.us.Iter(context.TODO(), []byte("2"), nil)
|
||||
c.Assert(err, IsNil)
|
||||
checkIterator(c, iter, [][]byte{[]byte("2"), []byte("4")}, [][]byte{[]byte("2"), []byte("4")})
|
||||
}
|
||||
|
||||
func (s *testUnionStoreSuite) TestIterReverse(c *C) {
|
||||
|
||||
s.store.Set([]byte("1"), []byte("1"))
|
||||
s.store.Set([]byte("2"), []byte("2"))
|
||||
s.store.Set([]byte("3"), []byte("3"))
|
||||
|
||||
iter, err := s.us.IterReverse(context.TODO(), nil)
|
||||
c.Assert(err, IsNil)
|
||||
checkIterator(c, iter, [][]byte{[]byte("3"), []byte("2"), []byte("1")}, [][]byte{[]byte("3"), []byte("2"), []byte("1")})
|
||||
|
||||
iter, err = s.us.IterReverse(context.TODO(), []byte("3"))
|
||||
c.Assert(err, IsNil)
|
||||
checkIterator(c, iter, [][]byte{[]byte("2"), []byte("1")}, [][]byte{[]byte("2"), []byte("1")})
|
||||
|
||||
s.us.Set([]byte("0"), []byte("0"))
|
||||
iter, err = s.us.IterReverse(context.TODO(), []byte("3"))
|
||||
c.Assert(err, IsNil)
|
||||
checkIterator(c, iter, [][]byte{[]byte("2"), []byte("1"), []byte("0")}, [][]byte{[]byte("2"), []byte("1"), []byte("0")})
|
||||
|
||||
s.us.Delete([]byte("1"))
|
||||
iter, err = s.us.IterReverse(context.TODO(), []byte("3"))
|
||||
c.Assert(err, IsNil)
|
||||
checkIterator(c, iter, [][]byte{[]byte("2"), []byte("0")}, [][]byte{[]byte("2"), []byte("0")})
|
||||
}
|
||||
|
||||
func (s *testUnionStoreSuite) TestLazyConditionCheck(c *C) {
|
||||
|
||||
s.store.Set([]byte("1"), []byte("1"))
|
||||
s.store.Set([]byte("2"), []byte("2"))
|
||||
|
||||
v, err := s.us.Get(context.TODO(), []byte("1"))
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(v, BytesEquals, []byte("1"))
|
||||
|
||||
s.us.SetOption(PresumeKeyNotExists, nil)
|
||||
s.us.SetOption(PresumeKeyNotExistsError, ErrNotExist)
|
||||
_, err = s.us.Get(context.TODO(), []byte("2"))
|
||||
c.Assert(errors.Cause(err) == ErrNotExist, IsTrue, Commentf("err %v", err))
|
||||
|
||||
condionPair1 := s.us.LookupConditionPair([]byte("1"))
|
||||
c.Assert(condionPair1, IsNil)
|
||||
|
||||
condionPair2 := s.us.LookupConditionPair([]byte("2"))
|
||||
c.Assert(condionPair2, NotNil)
|
||||
c.Assert(condionPair2.ShouldNotExist(), IsTrue)
|
||||
|
||||
err2 := s.us.LookupConditionPair([]byte("2")).Err()
|
||||
c.Assert(errors.Cause(err) == ErrNotExist, IsTrue, Commentf("err %v", err2))
|
||||
}
|
||||
|
||||
func checkIterator(c *C, iter Iterator, keys [][]byte, values [][]byte) {
|
||||
defer iter.Close()
|
||||
c.Assert(len(keys), Equals, len(values))
|
||||
for i, k := range keys {
|
||||
v := values[i]
|
||||
c.Assert(iter.Valid(), IsTrue)
|
||||
c.Assert([]byte(iter.Key()), BytesEquals, k)
|
||||
c.Assert(iter.Value(), BytesEquals, v)
|
||||
c.Assert(iter.Next(context.TODO()), IsNil)
|
||||
}
|
||||
c.Assert(iter.Valid(), IsFalse)
|
||||
}
|
||||
|
|
@ -1,306 +0,0 @@
|
|||
// Copyright 2018 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 latch
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"math/bits"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/spaolacci/murmur3"
|
||||
"github.com/tikv/client-go/config"
|
||||
)
|
||||
|
||||
type node struct {
|
||||
slotID int
|
||||
key []byte
|
||||
maxCommitTS uint64
|
||||
value *Lock
|
||||
|
||||
next *node
|
||||
}
|
||||
|
||||
// latch stores a key's waiting transactions information.
|
||||
type latch struct {
|
||||
conf *config.Latch
|
||||
queue *node
|
||||
count int
|
||||
waiting []*Lock
|
||||
sync.Mutex
|
||||
}
|
||||
|
||||
// Lock is the locks' information required for a transaction.
|
||||
type Lock struct {
|
||||
keys [][]byte
|
||||
// requiredSlots represents required slots.
|
||||
// The slot IDs of the latches(keys) that a startTS must acquire before being able to processed.
|
||||
requiredSlots []int
|
||||
// acquiredCount represents the number of latches that the transaction has acquired.
|
||||
// For status is stale, it includes the latch whose front is current lock already.
|
||||
acquiredCount int
|
||||
// startTS represents current transaction's.
|
||||
startTS uint64
|
||||
// commitTS represents current transaction's.
|
||||
commitTS uint64
|
||||
|
||||
wg sync.WaitGroup
|
||||
isStale bool
|
||||
}
|
||||
|
||||
// acquireResult is the result type for acquire()
|
||||
type acquireResult int32
|
||||
|
||||
const (
|
||||
// acquireSuccess is a type constant for acquireResult.
|
||||
// which means acquired success
|
||||
acquireSuccess acquireResult = iota
|
||||
// acquireLocked is a type constant for acquireResult
|
||||
// which means still locked by other Lock.
|
||||
acquireLocked
|
||||
// acquireStale is a type constant for acquireResult
|
||||
// which means current Lock's startTS is stale.
|
||||
acquireStale
|
||||
)
|
||||
|
||||
// IsStale returns whether the status is stale.
|
||||
func (l *Lock) IsStale() bool {
|
||||
return l.isStale
|
||||
}
|
||||
|
||||
func (l *Lock) isLocked() bool {
|
||||
return !l.isStale && l.acquiredCount != len(l.requiredSlots)
|
||||
}
|
||||
|
||||
// SetCommitTS sets the lock's commitTS.
|
||||
func (l *Lock) SetCommitTS(commitTS uint64) {
|
||||
l.commitTS = commitTS
|
||||
}
|
||||
|
||||
// Latches which are used for concurrency control.
|
||||
// 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
|
||||
}
|
||||
|
||||
type bytesSlice [][]byte
|
||||
|
||||
func (s bytesSlice) Len() int {
|
||||
return len(s)
|
||||
}
|
||||
|
||||
func (s bytesSlice) Swap(i, j int) {
|
||||
s[i], s[j] = s[j], s[i]
|
||||
}
|
||||
|
||||
func (s bytesSlice) Less(i, j int) bool {
|
||||
return bytes.Compare(s[i], s[j]) < 0
|
||||
}
|
||||
|
||||
// NewLatches create a Latches with fixed length,
|
||||
// the size will be rounded up to the power of 2.
|
||||
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,
|
||||
}
|
||||
}
|
||||
|
||||
// genLock generates Lock for the transaction with startTS and keys.
|
||||
func (latches *Latches) genLock(startTS uint64, keys [][]byte) *Lock {
|
||||
sort.Sort(bytesSlice(keys))
|
||||
return &Lock{
|
||||
keys: keys,
|
||||
requiredSlots: latches.genSlotIDs(keys),
|
||||
acquiredCount: 0,
|
||||
startTS: startTS,
|
||||
}
|
||||
}
|
||||
|
||||
func (latches *Latches) genSlotIDs(keys [][]byte) []int {
|
||||
slots := make([]int, 0, len(keys))
|
||||
for _, key := range keys {
|
||||
slots = append(slots, latches.slotID(key))
|
||||
}
|
||||
return slots
|
||||
}
|
||||
|
||||
// slotID return slotID for current key.
|
||||
func (latches *Latches) slotID(key []byte) int {
|
||||
return int(murmur3.Sum32(key)) & (len(latches.slots) - 1)
|
||||
}
|
||||
|
||||
// acquire tries to acquire the lock for a transaction.
|
||||
func (latches *Latches) acquire(lock *Lock) acquireResult {
|
||||
if lock.IsStale() {
|
||||
return acquireStale
|
||||
}
|
||||
for lock.acquiredCount < len(lock.requiredSlots) {
|
||||
status := latches.acquireSlot(lock)
|
||||
if status != acquireSuccess {
|
||||
return status
|
||||
}
|
||||
}
|
||||
return acquireSuccess
|
||||
}
|
||||
|
||||
// release releases all latches owned by the `lock` and returns the wakeup list.
|
||||
// Preconditions: the caller must ensure the transaction's status is not locked.
|
||||
func (latches *Latches) release(lock *Lock, wakeupList []*Lock) []*Lock {
|
||||
wakeupList = wakeupList[:0]
|
||||
for lock.acquiredCount > 0 {
|
||||
if nextLock := latches.releaseSlot(lock); nextLock != nil {
|
||||
wakeupList = append(wakeupList, nextLock)
|
||||
}
|
||||
}
|
||||
return wakeupList
|
||||
}
|
||||
|
||||
func (latches *Latches) releaseSlot(lock *Lock) (nextLock *Lock) {
|
||||
key := lock.keys[lock.acquiredCount-1]
|
||||
slotID := lock.requiredSlots[lock.acquiredCount-1]
|
||||
latch := &latches.slots[slotID]
|
||||
lock.acquiredCount--
|
||||
latch.Lock()
|
||||
defer latch.Unlock()
|
||||
|
||||
find := findNode(latch.queue, key)
|
||||
if find.value != lock {
|
||||
panic("releaseSlot wrong")
|
||||
}
|
||||
if lock.commitTS > find.maxCommitTS {
|
||||
find.maxCommitTS = lock.commitTS
|
||||
}
|
||||
find.value = nil
|
||||
// Make a copy of the key, so latch does not reference the transaction's memory.
|
||||
// If we do not do it, transaction memory can't be recycle by GC and there will
|
||||
// be a leak.
|
||||
copyKey := make([]byte, len(find.key))
|
||||
copy(copyKey, find.key)
|
||||
find.key = copyKey
|
||||
if len(latch.waiting) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
var idx int
|
||||
for idx = 0; idx < len(latch.waiting); idx++ {
|
||||
waiting := latch.waiting[idx]
|
||||
if bytes.Equal(waiting.keys[waiting.acquiredCount], key) {
|
||||
break
|
||||
}
|
||||
}
|
||||
// Wake up the first one in waiting queue.
|
||||
if idx < len(latch.waiting) {
|
||||
nextLock = latch.waiting[idx]
|
||||
// Delete element latch.waiting[idx] from the array.
|
||||
copy(latch.waiting[idx:], latch.waiting[idx+1:])
|
||||
latch.waiting[len(latch.waiting)-1] = nil
|
||||
latch.waiting = latch.waiting[:len(latch.waiting)-1]
|
||||
|
||||
if find.maxCommitTS > nextLock.startTS {
|
||||
find.value = nextLock
|
||||
nextLock.acquiredCount++
|
||||
nextLock.isStale = true
|
||||
}
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (latches *Latches) acquireSlot(lock *Lock) acquireResult {
|
||||
key := lock.keys[lock.acquiredCount]
|
||||
slotID := lock.requiredSlots[lock.acquiredCount]
|
||||
latch := &latches.slots[slotID]
|
||||
latch.Lock()
|
||||
defer latch.Unlock()
|
||||
|
||||
// Try to recycle to limit the memory usage.
|
||||
if latch.count >= latches.conf.ListCount {
|
||||
latch.recycle(lock.startTS, latches.conf.ExpireDuration)
|
||||
}
|
||||
|
||||
find := findNode(latch.queue, key)
|
||||
if find == nil {
|
||||
tmp := &node{
|
||||
slotID: slotID,
|
||||
key: key,
|
||||
value: lock,
|
||||
}
|
||||
tmp.next = latch.queue
|
||||
latch.queue = tmp
|
||||
latch.count++
|
||||
|
||||
lock.acquiredCount++
|
||||
return acquireSuccess
|
||||
}
|
||||
|
||||
if find.maxCommitTS > lock.startTS {
|
||||
lock.isStale = true
|
||||
return acquireStale
|
||||
}
|
||||
|
||||
if find.value == nil {
|
||||
find.value = lock
|
||||
lock.acquiredCount++
|
||||
return acquireSuccess
|
||||
}
|
||||
|
||||
// Push the current transaction into waitingQueue.
|
||||
latch.waiting = append(latch.waiting, lock)
|
||||
return acquireLocked
|
||||
}
|
||||
|
||||
// recycle is not thread safe, the latch should acquire its lock before executing this function.
|
||||
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) >= expireDuration && curr.value == nil {
|
||||
l.count--
|
||||
prev.next = curr.next
|
||||
total++
|
||||
} else {
|
||||
prev = curr
|
||||
}
|
||||
}
|
||||
l.queue = fakeHead.next
|
||||
return total
|
||||
}
|
||||
|
||||
func (latches *Latches) recycle(currentTS uint64) {
|
||||
total := 0
|
||||
for i := 0; i < len(latches.slots); i++ {
|
||||
latch := &latches.slots[i]
|
||||
latch.Lock()
|
||||
total += latch.recycle(currentTS, latches.conf.ExpireDuration)
|
||||
latch.Unlock()
|
||||
}
|
||||
log.Debugf("recycle run at %v, recycle count = %d...\n", time.Now(), total)
|
||||
}
|
||||
|
||||
func findNode(list *node, key []byte) *node {
|
||||
for n := list; n != nil; n = n.next {
|
||||
if bytes.Equal(n.key, key) {
|
||||
return n
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
@ -1,157 +0,0 @@
|
|||
// Copyright 2018 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 latch
|
||||
|
||||
import (
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/txnkv/oracle"
|
||||
)
|
||||
|
||||
func TestT(t *testing.T) {
|
||||
TestingT(t)
|
||||
}
|
||||
|
||||
var _ = Suite(&testLatchSuite{})
|
||||
|
||||
var baseTso uint64
|
||||
|
||||
type testLatchSuite struct {
|
||||
latches *Latches
|
||||
}
|
||||
|
||||
func (s *testLatchSuite) SetUpTest(c *C) {
|
||||
conf := config.DefaultLatch()
|
||||
conf.Capacity = 256
|
||||
s.latches = NewLatches(&conf)
|
||||
}
|
||||
|
||||
func (s *testLatchSuite) newLock(keys [][]byte) (startTS uint64, lock *Lock) {
|
||||
startTS = getTso()
|
||||
lock = s.latches.genLock(startTS, keys)
|
||||
return
|
||||
}
|
||||
|
||||
func getTso() uint64 {
|
||||
return atomic.AddUint64(&baseTso, uint64(1))
|
||||
}
|
||||
|
||||
func (s *testLatchSuite) TestWakeUp(c *C) {
|
||||
keysA := [][]byte{
|
||||
[]byte("a"), []byte("b"), []byte("c")}
|
||||
_, lockA := s.newLock(keysA)
|
||||
|
||||
keysB := [][]byte{[]byte("d"), []byte("e"), []byte("a"), []byte("c")}
|
||||
startTSB, lockB := s.newLock(keysB)
|
||||
|
||||
// A acquire lock success.
|
||||
result := s.latches.acquire(lockA)
|
||||
c.Assert(result, Equals, acquireSuccess)
|
||||
|
||||
// B acquire lock failed.
|
||||
result = s.latches.acquire(lockB)
|
||||
c.Assert(result, Equals, acquireLocked)
|
||||
|
||||
// A release lock, and get wakeup list.
|
||||
commitTSA := getTso()
|
||||
wakeupList := make([]*Lock, 0)
|
||||
lockA.SetCommitTS(commitTSA)
|
||||
wakeupList = s.latches.release(lockA, wakeupList)
|
||||
c.Assert(wakeupList[0].startTS, Equals, startTSB)
|
||||
|
||||
// B acquire failed since startTSB has stale for some keys.
|
||||
result = s.latches.acquire(lockB)
|
||||
c.Assert(result, Equals, acquireStale)
|
||||
|
||||
// B release lock since it received a stale.
|
||||
wakeupList = s.latches.release(lockB, wakeupList)
|
||||
c.Assert(wakeupList, HasLen, 0)
|
||||
|
||||
// B restart:get a new startTS.
|
||||
startTSB = getTso()
|
||||
lockB = s.latches.genLock(startTSB, keysB)
|
||||
result = s.latches.acquire(lockB)
|
||||
c.Assert(result, Equals, acquireSuccess)
|
||||
}
|
||||
|
||||
func (s *testLatchSuite) TestFirstAcquireFailedWithStale(c *C) {
|
||||
keys := [][]byte{
|
||||
[]byte("a"), []byte("b"), []byte("c")}
|
||||
_, lockA := s.newLock(keys)
|
||||
startTSB, lockB := s.newLock(keys)
|
||||
// acquire lockA success
|
||||
result := s.latches.acquire(lockA)
|
||||
c.Assert(result, Equals, acquireSuccess)
|
||||
// release lockA
|
||||
commitTSA := getTso()
|
||||
wakeupList := make([]*Lock, 0)
|
||||
lockA.SetCommitTS(commitTSA)
|
||||
s.latches.release(lockA, wakeupList)
|
||||
|
||||
c.Assert(commitTSA, Greater, startTSB)
|
||||
// acquire lockB first time, should be failed with stale since commitTSA > startTSB
|
||||
result = s.latches.acquire(lockB)
|
||||
c.Assert(result, Equals, acquireStale)
|
||||
s.latches.release(lockB, wakeupList)
|
||||
}
|
||||
|
||||
func (s *testLatchSuite) TestRecycle(c *C) {
|
||||
conf := config.DefaultLatch()
|
||||
conf.Capacity = 8
|
||||
latches := NewLatches(&conf)
|
||||
now := time.Now()
|
||||
startTS := oracle.ComposeTS(oracle.GetPhysical(now), 0)
|
||||
lock := latches.genLock(startTS, [][]byte{
|
||||
[]byte("a"), []byte("b"),
|
||||
})
|
||||
lock1 := latches.genLock(startTS, [][]byte{
|
||||
[]byte("b"), []byte("c"),
|
||||
})
|
||||
c.Assert(latches.acquire(lock), Equals, acquireSuccess)
|
||||
c.Assert(latches.acquire(lock1), Equals, acquireLocked)
|
||||
lock.SetCommitTS(startTS + 1)
|
||||
var wakeupList []*Lock
|
||||
latches.release(lock, wakeupList)
|
||||
// Release lock will grant latch to lock1 automatically,
|
||||
// so release lock1 is called here.
|
||||
latches.release(lock1, wakeupList)
|
||||
|
||||
lock2 := latches.genLock(startTS+3, [][]byte{
|
||||
[]byte("b"), []byte("c"),
|
||||
})
|
||||
c.Assert(latches.acquire(lock2), Equals, acquireSuccess)
|
||||
wakeupList = wakeupList[:0]
|
||||
latches.release(lock2, wakeupList)
|
||||
|
||||
allEmpty := true
|
||||
for i := 0; i < len(latches.slots); i++ {
|
||||
latch := &latches.slots[i]
|
||||
if latch.queue != nil {
|
||||
allEmpty = false
|
||||
}
|
||||
}
|
||||
c.Assert(allEmpty, IsFalse)
|
||||
|
||||
currentTS := oracle.ComposeTS(oracle.GetPhysical(now.Add(conf.ExpireDuration)), 3)
|
||||
latches.recycle(currentTS)
|
||||
|
||||
for i := 0; i < len(latches.slots); i++ {
|
||||
latch := &latches.slots[i]
|
||||
c.Assert(latch.queue, IsNil)
|
||||
}
|
||||
}
|
||||
|
|
@ -1,116 +0,0 @@
|
|||
// Copyright 2018 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 latch
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/txnkv/oracle"
|
||||
)
|
||||
|
||||
// LatchesScheduler is used to schedule latches for transactions.
|
||||
type LatchesScheduler struct {
|
||||
conf *config.Latch
|
||||
latches *Latches
|
||||
unlockCh chan *Lock
|
||||
closed bool
|
||||
lastRecycleTime uint64
|
||||
sync.RWMutex
|
||||
}
|
||||
|
||||
// NewScheduler create the LatchesScheduler.
|
||||
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,
|
||||
}
|
||||
go scheduler.run()
|
||||
return scheduler
|
||||
}
|
||||
|
||||
func (scheduler *LatchesScheduler) run() {
|
||||
var counter int
|
||||
wakeupList := make([]*Lock, 0)
|
||||
for lock := range scheduler.unlockCh {
|
||||
wakeupList = scheduler.latches.release(lock, wakeupList)
|
||||
if len(wakeupList) > 0 {
|
||||
scheduler.wakeup(wakeupList)
|
||||
}
|
||||
|
||||
if lock.commitTS > lock.startTS {
|
||||
currentTS := lock.commitTS
|
||||
elapsed := tsoSub(currentTS, scheduler.lastRecycleTime)
|
||||
if elapsed > scheduler.conf.CheckInterval || counter > scheduler.conf.CheckCounter {
|
||||
go scheduler.latches.recycle(lock.commitTS)
|
||||
scheduler.lastRecycleTime = currentTS
|
||||
counter = 0
|
||||
}
|
||||
}
|
||||
counter++
|
||||
}
|
||||
}
|
||||
|
||||
func (scheduler *LatchesScheduler) wakeup(wakeupList []*Lock) {
|
||||
for _, lock := range wakeupList {
|
||||
if scheduler.latches.acquire(lock) != acquireLocked {
|
||||
lock.wg.Done()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Close closes LatchesScheduler.
|
||||
func (scheduler *LatchesScheduler) Close() {
|
||||
scheduler.RWMutex.Lock()
|
||||
defer scheduler.RWMutex.Unlock()
|
||||
if !scheduler.closed {
|
||||
close(scheduler.unlockCh)
|
||||
scheduler.closed = true
|
||||
}
|
||||
}
|
||||
|
||||
// Lock acquire the lock for transaction with startTS and keys. The caller goroutine
|
||||
// would be blocked if the lock can't be obtained now. When this function returns,
|
||||
// the lock state would be either success or stale(call lock.IsStale)
|
||||
func (scheduler *LatchesScheduler) Lock(startTS uint64, keys [][]byte) *Lock {
|
||||
lock := scheduler.latches.genLock(startTS, keys)
|
||||
lock.wg.Add(1)
|
||||
if scheduler.latches.acquire(lock) == acquireLocked {
|
||||
lock.wg.Wait()
|
||||
}
|
||||
if lock.isLocked() {
|
||||
panic("should never run here")
|
||||
}
|
||||
return lock
|
||||
}
|
||||
|
||||
// UnLock unlocks a lock.
|
||||
func (scheduler *LatchesScheduler) UnLock(lock *Lock) {
|
||||
scheduler.RLock()
|
||||
defer scheduler.RUnlock()
|
||||
if !scheduler.closed {
|
||||
scheduler.unlockCh <- lock
|
||||
}
|
||||
}
|
||||
|
||||
func tsoSub(ts1, ts2 uint64) time.Duration {
|
||||
t1 := oracle.GetTimeFromTS(ts1)
|
||||
t2 := oracle.GetTimeFromTS(ts2)
|
||||
return t1.Sub(t2)
|
||||
}
|
||||
|
|
@ -1,97 +0,0 @@
|
|||
// Copyright 2018 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 latch
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"math/rand"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/tikv/client-go/config"
|
||||
)
|
||||
|
||||
var _ = Suite(&testSchedulerSuite{})
|
||||
|
||||
type testSchedulerSuite struct {
|
||||
}
|
||||
|
||||
func (s *testSchedulerSuite) SetUpTest(c *C) {
|
||||
}
|
||||
|
||||
func (s *testSchedulerSuite) TestWithConcurrency(c *C) {
|
||||
conf := config.DefaultLatch()
|
||||
conf.Capacity = 7
|
||||
sched := NewScheduler(&conf)
|
||||
defer sched.Close()
|
||||
rand.Seed(time.Now().Unix())
|
||||
|
||||
ch := make(chan [][]byte, 100)
|
||||
const workerCount = 10
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(workerCount)
|
||||
for i := 0; i < workerCount; i++ {
|
||||
go func(ch <-chan [][]byte, wg *sync.WaitGroup) {
|
||||
for txn := range ch {
|
||||
lock := sched.Lock(getTso(), txn)
|
||||
if lock.IsStale() {
|
||||
// Should restart the transaction or return error
|
||||
} else {
|
||||
lock.SetCommitTS(getTso())
|
||||
// Do 2pc
|
||||
}
|
||||
sched.UnLock(lock)
|
||||
}
|
||||
wg.Done()
|
||||
}(ch, &wg)
|
||||
}
|
||||
|
||||
for i := 0; i < 999; i++ {
|
||||
ch <- generate()
|
||||
}
|
||||
close(ch)
|
||||
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
// generate generates something like:
|
||||
// {[]byte("a"), []byte("b"), []byte("c")}
|
||||
// {[]byte("a"), []byte("d"), []byte("e"), []byte("f")}
|
||||
// {[]byte("e"), []byte("f"), []byte("g"), []byte("h")}
|
||||
// The data should not repeat in the sequence.
|
||||
func generate() [][]byte {
|
||||
table := []byte{'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h'}
|
||||
ret := make([][]byte, 0, 5)
|
||||
chance := []int{100, 60, 40, 20}
|
||||
for i := 0; i < len(chance); i++ {
|
||||
needMore := rand.Intn(100) < chance[i]
|
||||
if needMore {
|
||||
randBytes := []byte{table[rand.Intn(len(table))]}
|
||||
if !contains(randBytes, ret) {
|
||||
ret = append(ret, randBytes)
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func contains(x []byte, set [][]byte) bool {
|
||||
for _, y := range set {
|
||||
if bytes.Equal(x, y) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
|
@ -1,60 +0,0 @@
|
|||
// Copyright 2016 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 oracle
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
)
|
||||
|
||||
// Oracle is the interface that provides strictly ascending timestamps.
|
||||
type Oracle interface {
|
||||
GetTimestamp(ctx context.Context) (uint64, error)
|
||||
GetTimestampAsync(ctx context.Context) Future
|
||||
IsExpired(lockTimestamp uint64, TTL uint64) bool
|
||||
Close()
|
||||
}
|
||||
|
||||
// Future is a future which promises to return a timestamp.
|
||||
type Future interface {
|
||||
Wait() (uint64, error)
|
||||
}
|
||||
|
||||
const physicalShiftBits = 18
|
||||
|
||||
// ComposeTS creates a ts from physical and logical parts.
|
||||
func ComposeTS(physical, logical int64) uint64 {
|
||||
return uint64((physical << physicalShiftBits) + logical)
|
||||
}
|
||||
|
||||
// ExtractPhysical returns a ts's physical part.
|
||||
func ExtractPhysical(ts uint64) int64 {
|
||||
return int64(ts >> physicalShiftBits)
|
||||
}
|
||||
|
||||
// GetPhysical returns physical from an instant time with millisecond precision.
|
||||
func GetPhysical(t time.Time) int64 {
|
||||
return t.UnixNano() / int64(time.Millisecond)
|
||||
}
|
||||
|
||||
// EncodeTSO encodes a millisecond into tso.
|
||||
func EncodeTSO(ts int64) uint64 {
|
||||
return uint64(ts) << physicalShiftBits
|
||||
}
|
||||
|
||||
// GetTimeFromTS extracts time.Time from a timestamp.
|
||||
func GetTimeFromTS(ts uint64) time.Time {
|
||||
ms := ExtractPhysical(ts)
|
||||
return time.Unix(ms/1e3, (ms%1e3)*1e6)
|
||||
}
|
||||
|
|
@ -1,72 +0,0 @@
|
|||
// Copyright 2016 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 oracles
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/tikv/client-go/txnkv/oracle"
|
||||
)
|
||||
|
||||
var _ oracle.Oracle = &localOracle{}
|
||||
|
||||
type localOracle struct {
|
||||
sync.Mutex
|
||||
lastTimeStampTS uint64
|
||||
n uint64
|
||||
}
|
||||
|
||||
// NewLocalOracle creates an Oracle that uses local time as data source.
|
||||
func NewLocalOracle() oracle.Oracle {
|
||||
return &localOracle{}
|
||||
}
|
||||
|
||||
func (l *localOracle) IsExpired(lockTS uint64, TTL uint64) bool {
|
||||
return oracle.GetPhysical(time.Now()) >= oracle.ExtractPhysical(lockTS)+int64(TTL)
|
||||
}
|
||||
|
||||
func (l *localOracle) GetTimestamp(context.Context) (uint64, error) {
|
||||
l.Lock()
|
||||
defer l.Unlock()
|
||||
physical := oracle.GetPhysical(time.Now())
|
||||
ts := oracle.ComposeTS(physical, 0)
|
||||
if l.lastTimeStampTS == ts {
|
||||
l.n++
|
||||
return ts + l.n, nil
|
||||
}
|
||||
l.lastTimeStampTS = ts
|
||||
l.n = 0
|
||||
return ts, nil
|
||||
}
|
||||
|
||||
func (l *localOracle) GetTimestampAsync(ctx context.Context) oracle.Future {
|
||||
return &future{
|
||||
ctx: ctx,
|
||||
l: l,
|
||||
}
|
||||
}
|
||||
|
||||
type future struct {
|
||||
ctx context.Context
|
||||
l *localOracle
|
||||
}
|
||||
|
||||
func (f *future) Wait() (uint64, error) {
|
||||
return f.l.GetTimestamp(f.ctx)
|
||||
}
|
||||
|
||||
func (l *localOracle) Close() {
|
||||
}
|
||||
|
|
@ -1,52 +0,0 @@
|
|||
// Copyright 2016 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 oracles
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestLocalOracle(t *testing.T) {
|
||||
l := NewLocalOracle()
|
||||
defer l.Close()
|
||||
m := map[uint64]struct{}{}
|
||||
for i := 0; i < 100000; i++ {
|
||||
ts, err := l.GetTimestamp(context.Background())
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
m[ts] = struct{}{}
|
||||
}
|
||||
|
||||
if len(m) != 100000 {
|
||||
t.Error("generated same ts")
|
||||
}
|
||||
}
|
||||
|
||||
func TestIsExpired(t *testing.T) {
|
||||
o := NewLocalOracle()
|
||||
defer o.Close()
|
||||
ts, _ := o.GetTimestamp(context.Background())
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
expire := o.IsExpired(uint64(ts), 40)
|
||||
if !expire {
|
||||
t.Error("should expired")
|
||||
}
|
||||
expire = o.IsExpired(uint64(ts), 200)
|
||||
if expire {
|
||||
t.Error("should not expired")
|
||||
}
|
||||
}
|
||||
|
|
@ -1,140 +0,0 @@
|
|||
// Copyright 2016 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 oracles
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"github.com/tikv/client-go/txnkv/oracle"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
||||
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/tikv/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 `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, conf.OracleUpdateInterval)
|
||||
// Initialize lastTS by Get.
|
||||
_, err := o.GetTimestamp(ctx)
|
||||
if err != nil {
|
||||
o.Close()
|
||||
return nil, err
|
||||
}
|
||||
return o, nil
|
||||
}
|
||||
|
||||
// IsExpired returns whether lockTS+TTL is expired, both are ms. It uses `lastTS`
|
||||
// to compare, may return false negative result temporarily.
|
||||
func (o *pdOracle) IsExpired(lockTS, TTL uint64) bool {
|
||||
lastTS := atomic.LoadUint64(&o.lastTS)
|
||||
return oracle.ExtractPhysical(lastTS) >= oracle.ExtractPhysical(lockTS)+int64(TTL)
|
||||
}
|
||||
|
||||
// GetTimestamp gets a new increasing time.
|
||||
func (o *pdOracle) GetTimestamp(ctx context.Context) (uint64, error) {
|
||||
ts, err := o.getTimestamp(ctx)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
o.setLastTS(ts)
|
||||
return ts, nil
|
||||
}
|
||||
|
||||
type tsFuture struct {
|
||||
pd.TSFuture
|
||||
o *pdOracle
|
||||
}
|
||||
|
||||
// Wait implements the oracle.Future interface.
|
||||
func (f *tsFuture) Wait() (uint64, error) {
|
||||
now := time.Now()
|
||||
physical, logical, err := f.TSFuture.Wait()
|
||||
metrics.TSFutureWaitDuration.Observe(time.Since(now).Seconds())
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
ts := oracle.ComposeTS(physical, logical)
|
||||
f.o.setLastTS(ts)
|
||||
return ts, nil
|
||||
}
|
||||
|
||||
func (o *pdOracle) GetTimestampAsync(ctx context.Context) oracle.Future {
|
||||
ts := o.c.GetTSAsync(ctx)
|
||||
return &tsFuture{ts, o}
|
||||
}
|
||||
|
||||
func (o *pdOracle) getTimestamp(ctx context.Context) (uint64, error) {
|
||||
now := time.Now()
|
||||
physical, logical, err := o.c.GetTS(ctx)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
dist := time.Since(now)
|
||||
if dist > o.conf.TsoSlowThreshold {
|
||||
log.Warnf("get timestamp too slow: %s", dist)
|
||||
}
|
||||
return oracle.ComposeTS(physical, logical), nil
|
||||
}
|
||||
|
||||
func (o *pdOracle) setLastTS(ts uint64) {
|
||||
lastTS := atomic.LoadUint64(&o.lastTS)
|
||||
if ts > lastTS {
|
||||
atomic.CompareAndSwapUint64(&o.lastTS, lastTS, ts)
|
||||
}
|
||||
}
|
||||
|
||||
func (o *pdOracle) updateTS(ctx context.Context, interval time.Duration) {
|
||||
ticker := time.NewTicker(interval)
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
ts, err := o.getTimestamp(ctx)
|
||||
if err != nil {
|
||||
log.Errorf("updateTS error: %v", err)
|
||||
break
|
||||
}
|
||||
o.setLastTS(ts)
|
||||
case <-o.quit:
|
||||
ticker.Stop()
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (o *pdOracle) Close() {
|
||||
close(o.quit)
|
||||
}
|
||||
|
|
@ -1,39 +0,0 @@
|
|||
// 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 store
|
||||
|
||||
import "github.com/tikv/client-go/locate"
|
||||
|
||||
// batchKeys is a batch of keys in the same region.
|
||||
type batchKeys struct {
|
||||
region locate.RegionVerID
|
||||
keys [][]byte
|
||||
}
|
||||
|
||||
// appendBatchBySize appends keys to []batchKeys. It may split the keys to make
|
||||
// sure each batch's size does not exceed the limit.
|
||||
func appendBatchBySize(b []batchKeys, region locate.RegionVerID, keys [][]byte, sizeFn func([]byte) int, limit int) []batchKeys {
|
||||
var start, end int
|
||||
for start = 0; start < len(keys); start = end {
|
||||
var size int
|
||||
for end = start; end < len(keys) && size < limit; end++ {
|
||||
size += sizeFn(keys[end])
|
||||
}
|
||||
b = append(b, batchKeys{
|
||||
region: region,
|
||||
keys: keys[start:end],
|
||||
})
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
|
@ -1,106 +0,0 @@
|
|||
// Copyright 2018 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 store
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
)
|
||||
|
||||
// ExecDetails contains execution detail information.
|
||||
type ExecDetails struct {
|
||||
CalleeAddress string
|
||||
ProcessTime time.Duration
|
||||
WaitTime time.Duration
|
||||
BackoffTime time.Duration
|
||||
RequestCount int
|
||||
TotalKeys int64
|
||||
ProcessedKeys int64
|
||||
CommitDetail *CommitDetails
|
||||
}
|
||||
|
||||
// CommitDetails contains commit detail information.
|
||||
type CommitDetails struct {
|
||||
GetCommitTsTime time.Duration
|
||||
PrewriteTime time.Duration
|
||||
CommitTime time.Duration
|
||||
LocalLatchTime time.Duration
|
||||
TotalBackoffTime time.Duration
|
||||
ResolveLockTime int64
|
||||
WriteKeys int
|
||||
WriteSize int
|
||||
PrewriteRegionNum int32
|
||||
TxnRetry int
|
||||
}
|
||||
|
||||
// String implements the fmt.Stringer interface.
|
||||
func (d ExecDetails) String() string {
|
||||
parts := make([]string, 0, 6)
|
||||
if d.ProcessTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("process_time:%vs", d.ProcessTime.Seconds()))
|
||||
}
|
||||
if d.WaitTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("wait_time:%vs", d.WaitTime.Seconds()))
|
||||
}
|
||||
if d.BackoffTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("backoff_time:%vs", d.BackoffTime.Seconds()))
|
||||
}
|
||||
if d.RequestCount > 0 {
|
||||
parts = append(parts, fmt.Sprintf("request_count:%d", d.RequestCount))
|
||||
}
|
||||
if d.TotalKeys > 0 {
|
||||
parts = append(parts, fmt.Sprintf("total_keys:%d", d.TotalKeys))
|
||||
}
|
||||
if d.ProcessedKeys > 0 {
|
||||
parts = append(parts, fmt.Sprintf("processed_keys:%d", d.ProcessedKeys))
|
||||
}
|
||||
commitDetails := d.CommitDetail
|
||||
if commitDetails != nil {
|
||||
if commitDetails.PrewriteTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("prewrite_time:%vs", commitDetails.PrewriteTime.Seconds()))
|
||||
}
|
||||
if commitDetails.CommitTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("commit_time:%vs", commitDetails.CommitTime.Seconds()))
|
||||
}
|
||||
if commitDetails.GetCommitTsTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("get_commit_ts_time:%vs", commitDetails.GetCommitTsTime.Seconds()))
|
||||
}
|
||||
if commitDetails.TotalBackoffTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("total_backoff_time:%vs", commitDetails.TotalBackoffTime.Seconds()))
|
||||
}
|
||||
resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLockTime)
|
||||
if resolveLockTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("resolve_lock_time:%vs", time.Duration(resolveLockTime).Seconds()))
|
||||
}
|
||||
if commitDetails.LocalLatchTime > 0 {
|
||||
parts = append(parts, fmt.Sprintf("local_latch_wait_time:%vs", commitDetails.LocalLatchTime.Seconds()))
|
||||
}
|
||||
if commitDetails.WriteKeys > 0 {
|
||||
parts = append(parts, fmt.Sprintf("write_keys:%d", commitDetails.WriteKeys))
|
||||
}
|
||||
if commitDetails.WriteSize > 0 {
|
||||
parts = append(parts, fmt.Sprintf("write_size:%d", commitDetails.WriteSize))
|
||||
}
|
||||
prewriteRegionNum := atomic.LoadInt32(&commitDetails.PrewriteRegionNum)
|
||||
if prewriteRegionNum > 0 {
|
||||
parts = append(parts, fmt.Sprintf("prewrite_region:%d", prewriteRegionNum))
|
||||
}
|
||||
if commitDetails.TxnRetry > 0 {
|
||||
parts = append(parts, fmt.Sprintf("txn_retry:%d", commitDetails.TxnRetry))
|
||||
}
|
||||
}
|
||||
return strings.Join(parts, " ")
|
||||
}
|
||||
|
|
@ -1,124 +0,0 @@
|
|||
// Copyright 2018 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 store
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"github.com/tikv/client-go/rpc"
|
||||
)
|
||||
|
||||
// DeleteRangeTask is used to delete all keys in a range. After
|
||||
// performing DeleteRange, it keeps how many ranges it affects and
|
||||
// if the task was canceled or not.
|
||||
type DeleteRangeTask struct {
|
||||
completedRegions int
|
||||
canceled bool
|
||||
store *TiKVStore
|
||||
ctx context.Context
|
||||
startKey []byte
|
||||
endKey []byte
|
||||
}
|
||||
|
||||
// NewDeleteRangeTask creates a DeleteRangeTask. Deleting will not be performed right away.
|
||||
// WARNING: Currently, this API may leave some waste key-value pairs uncleaned in TiKV. Be careful while using it.
|
||||
func NewDeleteRangeTask(ctx context.Context, store *TiKVStore, startKey []byte, endKey []byte) *DeleteRangeTask {
|
||||
return &DeleteRangeTask{
|
||||
completedRegions: 0,
|
||||
canceled: false,
|
||||
store: store,
|
||||
ctx: ctx,
|
||||
startKey: startKey,
|
||||
endKey: endKey,
|
||||
}
|
||||
}
|
||||
|
||||
// Execute performs the delete range operation.
|
||||
func (t *DeleteRangeTask) Execute() error {
|
||||
conf := t.store.GetConfig()
|
||||
|
||||
startKey, rangeEndKey := t.startKey, t.endKey
|
||||
for {
|
||||
select {
|
||||
case <-t.ctx.Done():
|
||||
t.canceled = true
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
bo := retry.NewBackoffer(t.ctx, retry.DeleteRangeOneRegionMaxBackoff)
|
||||
loc, err := t.store.GetRegionCache().LocateKey(bo, startKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Delete to the end of the region, except if it's the last region overlapping the range
|
||||
endKey := loc.EndKey
|
||||
// If it is the last region
|
||||
if loc.Contains(rangeEndKey) {
|
||||
endKey = rangeEndKey
|
||||
}
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdDeleteRange,
|
||||
DeleteRange: &kvrpcpb.DeleteRangeRequest{
|
||||
StartKey: startKey,
|
||||
EndKey: endKey,
|
||||
},
|
||||
}
|
||||
|
||||
resp, err := t.store.SendReq(bo, req, loc.Region, conf.RPC.ReadTimeoutMedium)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
deleteRangeResp := resp.DeleteRange
|
||||
if deleteRangeResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if err := deleteRangeResp.GetError(); err != "" {
|
||||
return errors.Errorf("unexpected delete range err: %v", err)
|
||||
}
|
||||
t.completedRegions++
|
||||
if bytes.Equal(endKey, rangeEndKey) {
|
||||
break
|
||||
}
|
||||
startKey = endKey
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// CompletedRegions returns the number of regions that are affected by this delete range task
|
||||
func (t *DeleteRangeTask) CompletedRegions() int {
|
||||
return t.completedRegions
|
||||
}
|
||||
|
||||
// IsCanceled returns true if the delete range operation was canceled on the half way
|
||||
func (t *DeleteRangeTask) IsCanceled() bool {
|
||||
return t.canceled
|
||||
}
|
||||
|
|
@ -1,47 +0,0 @@
|
|||
// 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 store
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/tikv/client-go/key"
|
||||
)
|
||||
|
||||
// TxnRetryableMark is used to direct user to restart a transaction.
|
||||
// TiDB decides whether to retry transaction by checking if error message contains
|
||||
// string "try again later" literally. The common usage is `errors.Annotate(err, TxnRetryableMark)`.
|
||||
// Note that it should be only used if i) the error occurs inside a transaction
|
||||
// and ii) the error is not totally unexpected and hopefully will recover soon.
|
||||
const TxnRetryableMark = "[try again later]"
|
||||
|
||||
var (
|
||||
// ErrResultUndetermined means that the commit status is unknown.
|
||||
ErrResultUndetermined = errors.New("result undetermined")
|
||||
// ErrNotImplemented returns when a function is not implemented yet.
|
||||
ErrNotImplemented = errors.New("not implemented")
|
||||
// ErrPDServerTimeout is the error that PD does not repond in time.
|
||||
ErrPDServerTimeout = errors.New("PD server timeout")
|
||||
// ErrStartTSFallBehind is the error a transaction runs too long and data
|
||||
// loaded from TiKV may out of date because of GC.
|
||||
ErrStartTSFallBehind = errors.New("StartTS may fall behind safePoint")
|
||||
)
|
||||
|
||||
// ErrKeyAlreadyExist is the error that a key exists in TiKV when it should not.
|
||||
type ErrKeyAlreadyExist key.Key
|
||||
|
||||
func (e ErrKeyAlreadyExist) Error() string {
|
||||
return fmt.Sprintf("key already exists: %q", key.Key(e))
|
||||
}
|
||||
|
|
@ -1,368 +0,0 @@
|
|||
// Copyright 2016 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 store
|
||||
|
||||
import (
|
||||
"container/list"
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"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/locate"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"github.com/tikv/client-go/rpc"
|
||||
)
|
||||
|
||||
// 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).
|
||||
resolved map[uint64]TxnStatus
|
||||
recentResolved *list.List
|
||||
}
|
||||
}
|
||||
|
||||
func newLockResolver(store *TiKVStore) *LockResolver {
|
||||
r := &LockResolver{
|
||||
store: store,
|
||||
conf: store.GetConfig(),
|
||||
}
|
||||
r.mu.resolved = make(map[uint64]TxnStatus)
|
||||
r.mu.recentResolved = list.New()
|
||||
return r
|
||||
}
|
||||
|
||||
// NewLockResolver is exported for other pkg to use, suppress unused warning.
|
||||
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(ctx context.Context, etcdAddrs []string, conf config.Config) (*LockResolver, error) {
|
||||
s, err := NewStore(ctx, etcdAddrs, conf)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return s.GetLockResolver(), nil
|
||||
}
|
||||
|
||||
// TxnStatus represents a txn's final status. It should be Commit or Rollback.
|
||||
type TxnStatus uint64
|
||||
|
||||
// IsCommitted returns true if the txn's final status is Commit.
|
||||
func (s TxnStatus) IsCommitted() bool { return s > 0 }
|
||||
|
||||
// CommitTS returns the txn's commitTS. It is valid iff `IsCommitted` is true.
|
||||
func (s TxnStatus) CommitTS() uint64 { return uint64(s) }
|
||||
|
||||
// Lock represents a lock from tikv server.
|
||||
type Lock struct {
|
||||
Key []byte
|
||||
Primary []byte
|
||||
TxnID uint64
|
||||
TTL uint64
|
||||
}
|
||||
|
||||
// NewLock creates a new *Lock.
|
||||
func NewLock(l *kvrpcpb.LockInfo, defaultTTL uint64) *Lock {
|
||||
ttl := l.GetLockTtl()
|
||||
if ttl == 0 {
|
||||
ttl = defaultTTL
|
||||
}
|
||||
return &Lock{
|
||||
Key: l.GetKey(),
|
||||
Primary: l.GetPrimaryLock(),
|
||||
TxnID: l.GetLockVersion(),
|
||||
TTL: ttl,
|
||||
}
|
||||
}
|
||||
|
||||
func (lr *LockResolver) saveResolved(txnID uint64, status TxnStatus) {
|
||||
lr.mu.Lock()
|
||||
defer lr.mu.Unlock()
|
||||
|
||||
if _, ok := lr.mu.resolved[txnID]; ok {
|
||||
return
|
||||
}
|
||||
lr.mu.resolved[txnID] = status
|
||||
lr.mu.recentResolved.PushBack(txnID)
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
||||
func (lr *LockResolver) getResolved(txnID uint64) (TxnStatus, bool) {
|
||||
lr.mu.RLock()
|
||||
defer lr.mu.RUnlock()
|
||||
|
||||
s, ok := lr.mu.resolved[txnID]
|
||||
return s, ok
|
||||
}
|
||||
|
||||
// BatchResolveLocks resolve locks in a batch
|
||||
func (lr *LockResolver) BatchResolveLocks(bo *retry.Backoffer, locks []*Lock, loc locate.RegionVerID) (bool, error) {
|
||||
if len(locks) == 0 {
|
||||
return true, nil
|
||||
}
|
||||
|
||||
metrics.LockResolverCounter.WithLabelValues("batch_resolve").Inc()
|
||||
|
||||
var expiredLocks []*Lock
|
||||
for _, l := range locks {
|
||||
if lr.store.GetOracle().IsExpired(l.TxnID, l.TTL) {
|
||||
metrics.LockResolverCounter.WithLabelValues("expired").Inc()
|
||||
expiredLocks = append(expiredLocks, l)
|
||||
} else {
|
||||
metrics.LockResolverCounter.WithLabelValues("not_expired").Inc()
|
||||
}
|
||||
}
|
||||
if len(expiredLocks) != len(locks) {
|
||||
log.Errorf("BatchResolveLocks: get %d Locks, but only %d are expired, maybe safe point is wrong!", len(locks), len(expiredLocks))
|
||||
return false, nil
|
||||
}
|
||||
|
||||
startTime := time.Now()
|
||||
txnInfos := make(map[uint64]uint64)
|
||||
for _, l := range expiredLocks {
|
||||
if _, ok := txnInfos[l.TxnID]; ok {
|
||||
continue
|
||||
}
|
||||
|
||||
status, err := lr.getTxnStatus(bo, l.TxnID, l.Primary)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
txnInfos[l.TxnID] = uint64(status)
|
||||
}
|
||||
log.Infof("BatchResolveLocks: it took %v to lookup %v txn status", time.Since(startTime), len(txnInfos))
|
||||
|
||||
var listTxnInfos []*kvrpcpb.TxnInfo
|
||||
for txnID, status := range txnInfos {
|
||||
listTxnInfos = append(listTxnInfos, &kvrpcpb.TxnInfo{
|
||||
Txn: txnID,
|
||||
Status: status,
|
||||
})
|
||||
}
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdResolveLock,
|
||||
ResolveLock: &kvrpcpb.ResolveLockRequest{
|
||||
TxnInfos: listTxnInfos,
|
||||
},
|
||||
}
|
||||
startTime = time.Now()
|
||||
resp, err := lr.store.SendReq(bo, req, loc, lr.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
return false, nil
|
||||
}
|
||||
|
||||
cmdResp := resp.ResolveLock
|
||||
if cmdResp == nil {
|
||||
return false, errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if keyErr := cmdResp.GetError(); keyErr != nil {
|
||||
return false, errors.Errorf("unexpected resolve err: %s", keyErr)
|
||||
}
|
||||
|
||||
log.Infof("BatchResolveLocks: it took %v to resolve %v locks in a batch.", time.Since(startTime), len(expiredLocks))
|
||||
return true, nil
|
||||
}
|
||||
|
||||
// ResolveLocks tries to resolve Locks. The resolving process is in 3 steps:
|
||||
// 1) Use the `lockTTL` to pick up all expired locks. Only locks that are too
|
||||
// old are considered orphan locks and will be handled later. If all locks
|
||||
// are expired then all locks will be resolved so the returned `ok` will be
|
||||
// true, otherwise caller should sleep a while before retry.
|
||||
// 2) For each lock, query the primary key to get txn(which left the lock)'s
|
||||
// commit status.
|
||||
// 3) Send `ResolveLock` cmd to the lock's region to resolve all locks belong to
|
||||
// the same transaction.
|
||||
func (lr *LockResolver) ResolveLocks(bo *retry.Backoffer, locks []*Lock) (ok bool, err error) {
|
||||
if len(locks) == 0 {
|
||||
return true, nil
|
||||
}
|
||||
|
||||
metrics.LockResolverCounter.WithLabelValues("resolve").Inc()
|
||||
|
||||
var expiredLocks []*Lock
|
||||
for _, l := range locks {
|
||||
if lr.store.GetOracle().IsExpired(l.TxnID, l.TTL) {
|
||||
metrics.LockResolverCounter.WithLabelValues("expired").Inc()
|
||||
expiredLocks = append(expiredLocks, l)
|
||||
} else {
|
||||
metrics.LockResolverCounter.WithLabelValues("not_expired").Inc()
|
||||
}
|
||||
}
|
||||
if len(expiredLocks) == 0 {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
// TxnID -> []Region, record resolved Regions.
|
||||
// TODO: Maybe put it in LockResolver and share by all txns.
|
||||
cleanTxns := make(map[uint64]map[locate.RegionVerID]struct{})
|
||||
for _, l := range expiredLocks {
|
||||
status, err := lr.getTxnStatus(bo, l.TxnID, l.Primary)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
cleanRegions := cleanTxns[l.TxnID]
|
||||
if cleanRegions == nil {
|
||||
cleanRegions = make(map[locate.RegionVerID]struct{})
|
||||
cleanTxns[l.TxnID] = cleanRegions
|
||||
}
|
||||
|
||||
err = lr.resolveLock(bo, l, status, cleanRegions)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
}
|
||||
return len(expiredLocks) == len(locks), nil
|
||||
}
|
||||
|
||||
// GetTxnStatus queries tikv-server for a txn's status (commit/rollback).
|
||||
// If the primary key is still locked, it will launch a Rollback to abort it.
|
||||
// To avoid unnecessarily aborting too many txns, it is wiser to wait a few
|
||||
// seconds before calling it after Prewrite.
|
||||
func (lr *LockResolver) GetTxnStatus(ctx context.Context, txnID uint64, primary []byte) (TxnStatus, error) {
|
||||
bo := retry.NewBackoffer(ctx, retry.CleanupMaxBackoff)
|
||||
return lr.getTxnStatus(bo, txnID, primary)
|
||||
}
|
||||
|
||||
func (lr *LockResolver) getTxnStatus(bo *retry.Backoffer, txnID uint64, primary []byte) (TxnStatus, error) {
|
||||
if s, ok := lr.getResolved(txnID); ok {
|
||||
return s, nil
|
||||
}
|
||||
|
||||
metrics.LockResolverCounter.WithLabelValues("query_txn_status").Inc()
|
||||
|
||||
var status TxnStatus
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdCleanup,
|
||||
Cleanup: &kvrpcpb.CleanupRequest{
|
||||
Key: primary,
|
||||
StartVersion: txnID,
|
||||
},
|
||||
}
|
||||
for {
|
||||
loc, err := lr.store.GetRegionCache().LocateKey(bo, primary)
|
||||
if err != nil {
|
||||
return status, err
|
||||
}
|
||||
resp, err := lr.store.SendReq(bo, req, loc.Region, lr.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return status, err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return status, err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return status, err
|
||||
}
|
||||
continue
|
||||
}
|
||||
cmdResp := resp.Cleanup
|
||||
if cmdResp == nil {
|
||||
return status, errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if keyErr := cmdResp.GetError(); keyErr != nil {
|
||||
err = errors.Errorf("unexpected cleanup err: %s, tid: %v", keyErr, txnID)
|
||||
log.Error(err)
|
||||
return status, err
|
||||
}
|
||||
if cmdResp.CommitVersion != 0 {
|
||||
status = TxnStatus(cmdResp.GetCommitVersion())
|
||||
metrics.LockResolverCounter.WithLabelValues("query_txn_status_committed").Inc()
|
||||
} else {
|
||||
metrics.LockResolverCounter.WithLabelValues("query_txn_status_rolled_back").Inc()
|
||||
}
|
||||
lr.saveResolved(txnID, status)
|
||||
return status, nil
|
||||
}
|
||||
}
|
||||
|
||||
func (lr *LockResolver) resolveLock(bo *retry.Backoffer, l *Lock, status TxnStatus, cleanRegions map[locate.RegionVerID]struct{}) error {
|
||||
metrics.LockResolverCounter.WithLabelValues("query_resolve_locks").Inc()
|
||||
for {
|
||||
loc, err := lr.store.GetRegionCache().LocateKey(bo, l.Key)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if _, ok := cleanRegions[loc.Region]; ok {
|
||||
return nil
|
||||
}
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdResolveLock,
|
||||
ResolveLock: &kvrpcpb.ResolveLockRequest{
|
||||
StartVersion: l.TxnID,
|
||||
},
|
||||
}
|
||||
if status.IsCommitted() {
|
||||
req.ResolveLock.CommitVersion = status.CommitTS()
|
||||
}
|
||||
resp, err := lr.store.SendReq(bo, req, loc.Region, lr.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
cmdResp := resp.ResolveLock
|
||||
if cmdResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
if keyErr := cmdResp.GetError(); keyErr != nil {
|
||||
err = errors.Errorf("unexpected resolve err: %s, lock: %v", keyErr, l)
|
||||
log.Error(err)
|
||||
return err
|
||||
}
|
||||
cleanRegions[loc.Region] = struct{}{}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
|
@ -1,146 +0,0 @@
|
|||
// Copyright 2017 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 store
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
|
||||
"github.com/pkg/errors"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
// SafePointKV is used for a seamingless integration for mockTest and runtime.
|
||||
type SafePointKV interface {
|
||||
Put(k string, v string) error
|
||||
Get(k string) (string, error)
|
||||
}
|
||||
|
||||
// MockSafePointKV implements SafePointKV at mock test
|
||||
type MockSafePointKV struct {
|
||||
store map[string]string
|
||||
mockLock sync.RWMutex
|
||||
}
|
||||
|
||||
// NewMockSafePointKV creates an instance of MockSafePointKV
|
||||
func NewMockSafePointKV() *MockSafePointKV {
|
||||
return &MockSafePointKV{
|
||||
store: make(map[string]string),
|
||||
}
|
||||
}
|
||||
|
||||
// Put implements the Put method for SafePointKV
|
||||
func (w *MockSafePointKV) Put(k string, v string) error {
|
||||
w.mockLock.Lock()
|
||||
defer w.mockLock.Unlock()
|
||||
w.store[k] = v
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get implements the Get method for SafePointKV
|
||||
func (w *MockSafePointKV) Get(k string) (string, error) {
|
||||
w.mockLock.RLock()
|
||||
defer w.mockLock.RUnlock()
|
||||
elem := w.store[k]
|
||||
return elem, nil
|
||||
}
|
||||
|
||||
// EtcdSafePointKV implements SafePointKV at runtime
|
||||
type EtcdSafePointKV struct {
|
||||
cli *clientv3.Client
|
||||
}
|
||||
|
||||
// NewEtcdSafePointKV creates an instance of EtcdSafePointKV
|
||||
func NewEtcdSafePointKV(addrs []string, tlsConfig *tls.Config) (*EtcdSafePointKV, error) {
|
||||
etcdCli, err := createEtcdKV(addrs, tlsConfig)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &EtcdSafePointKV{cli: etcdCli}, nil
|
||||
}
|
||||
|
||||
func createEtcdKV(addrs []string, tlsConfig *tls.Config) (*clientv3.Client, error) {
|
||||
cli, err := clientv3.New(clientv3.Config{
|
||||
Endpoints: addrs,
|
||||
DialTimeout: 5 * time.Second,
|
||||
DialOptions: []grpc.DialOption{
|
||||
grpc.WithUnaryInterceptor(grpc_prometheus.UnaryClientInterceptor),
|
||||
grpc.WithStreamInterceptor(grpc_prometheus.StreamClientInterceptor),
|
||||
},
|
||||
TLS: tlsConfig,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
return cli, nil
|
||||
}
|
||||
|
||||
// Put implements the Put method for SafePointKV
|
||||
func (w *EtcdSafePointKV) Put(k string, v string) error {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
|
||||
_, err := w.cli.Put(ctx, k, v)
|
||||
cancel()
|
||||
if err != nil {
|
||||
return errors.WithStack(err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get implements the Get method for SafePointKV
|
||||
func (w *EtcdSafePointKV) Get(k string) (string, error) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
|
||||
resp, err := w.cli.Get(ctx, k)
|
||||
cancel()
|
||||
if err != nil {
|
||||
return "", errors.WithStack(err)
|
||||
}
|
||||
if len(resp.Kvs) > 0 {
|
||||
return string(resp.Kvs[0].Value), nil
|
||||
}
|
||||
return "", nil
|
||||
}
|
||||
|
||||
func saveSafePoint(kv SafePointKV, key string, t uint64) error {
|
||||
s := strconv.FormatUint(t, 10)
|
||||
err := kv.Put(key, s)
|
||||
if err != nil {
|
||||
log.Error("save safepoint failed:", err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func loadSafePoint(kv SafePointKV, key string) (uint64, error) {
|
||||
str, err := kv.Get(key)
|
||||
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
if str == "" {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
t, err := strconv.ParseUint(str, 10, 64)
|
||||
if err != nil {
|
||||
return 0, errors.WithStack(err)
|
||||
}
|
||||
return t, nil
|
||||
}
|
||||
|
|
@ -1,237 +0,0 @@
|
|||
// Copyright 2016 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 store
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
|
||||
pb "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"
|
||||
"github.com/tikv/client-go/txnkv/kv"
|
||||
)
|
||||
|
||||
// Scanner support tikv scan
|
||||
type Scanner struct {
|
||||
snapshot *TiKVSnapshot
|
||||
conf *config.Config
|
||||
batchSize int
|
||||
valid bool
|
||||
cache []*pb.KvPair
|
||||
idx int
|
||||
nextStartKey []byte
|
||||
endKey []byte
|
||||
eof bool
|
||||
}
|
||||
|
||||
func newScanner(ctx context.Context, snapshot *TiKVSnapshot, startKey []byte, endKey []byte, batchSize int) (*Scanner, error) {
|
||||
// It must be > 1. Otherwise scanner won't skipFirst.
|
||||
if batchSize <= 1 {
|
||||
batchSize = snapshot.conf.Txn.ScanBatchSize
|
||||
}
|
||||
scanner := &Scanner{
|
||||
snapshot: snapshot,
|
||||
conf: snapshot.conf,
|
||||
batchSize: batchSize,
|
||||
valid: true,
|
||||
nextStartKey: startKey,
|
||||
endKey: endKey,
|
||||
}
|
||||
err := scanner.Next(ctx)
|
||||
if kv.IsErrNotFound(err) {
|
||||
return scanner, nil
|
||||
}
|
||||
return scanner, err
|
||||
}
|
||||
|
||||
// Valid return valid.
|
||||
func (s *Scanner) Valid() bool {
|
||||
return s.valid
|
||||
}
|
||||
|
||||
// Key return key.
|
||||
func (s *Scanner) Key() key.Key {
|
||||
if s.valid {
|
||||
return s.cache[s.idx].Key
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Value return value.
|
||||
func (s *Scanner) Value() []byte {
|
||||
if s.valid {
|
||||
return s.cache[s.idx].Value
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Next return next element.
|
||||
func (s *Scanner) Next(ctx context.Context) error {
|
||||
bo := retry.NewBackoffer(ctx, retry.ScannerNextMaxBackoff)
|
||||
if !s.valid {
|
||||
return errors.New("scanner iterator is invalid")
|
||||
}
|
||||
for {
|
||||
s.idx++
|
||||
if s.idx >= len(s.cache) {
|
||||
if s.eof {
|
||||
s.Close()
|
||||
return nil
|
||||
}
|
||||
err := s.getData(bo)
|
||||
if err != nil {
|
||||
s.Close()
|
||||
return err
|
||||
}
|
||||
if s.idx >= len(s.cache) {
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
current := s.cache[s.idx]
|
||||
if len(s.endKey) > 0 && key.Key(current.Key).Cmp(key.Key(s.endKey)) >= 0 {
|
||||
s.eof = true
|
||||
s.Close()
|
||||
return nil
|
||||
}
|
||||
// Try to resolve the lock
|
||||
if current.GetError() != nil {
|
||||
// 'current' would be modified if the lock being resolved
|
||||
if err := s.resolveCurrentLock(bo, current); err != nil {
|
||||
s.Close()
|
||||
return err
|
||||
}
|
||||
|
||||
// The check here does not violate the KeyOnly semantic, because current's value
|
||||
// is filled by resolveCurrentLock which fetches the value by snapshot.get, so an empty
|
||||
// value stands for NotExist
|
||||
if len(current.Value) == 0 {
|
||||
continue
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// Close close iterator.
|
||||
func (s *Scanner) Close() {
|
||||
s.valid = false
|
||||
}
|
||||
|
||||
func (s *Scanner) startTS() uint64 {
|
||||
return s.snapshot.ts
|
||||
}
|
||||
|
||||
func (s *Scanner) resolveCurrentLock(bo *retry.Backoffer, current *pb.KvPair) error {
|
||||
val, err := s.snapshot.get(bo, key.Key(current.Key))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
current.Error = nil
|
||||
current.Value = val
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Scanner) getData(bo *retry.Backoffer) error {
|
||||
log.Debugf("txn getData nextStartKey[%q], txn %d", s.nextStartKey, s.startTS())
|
||||
sender := rpc.NewRegionRequestSender(s.snapshot.store.GetRegionCache(), s.snapshot.store.GetRPCClient())
|
||||
|
||||
for {
|
||||
loc, err := s.snapshot.store.regionCache.LocateKey(bo, s.nextStartKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
reqEndKey := s.endKey
|
||||
if len(reqEndKey) > 0 && len(loc.EndKey) > 0 && bytes.Compare(loc.EndKey, reqEndKey) < 0 {
|
||||
reqEndKey = loc.EndKey
|
||||
}
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdScan,
|
||||
Scan: &pb.ScanRequest{
|
||||
StartKey: s.nextStartKey,
|
||||
EndKey: reqEndKey,
|
||||
Limit: uint32(s.batchSize),
|
||||
Version: s.startTS(),
|
||||
KeyOnly: s.snapshot.KeyOnly,
|
||||
},
|
||||
Context: pb.Context{
|
||||
Priority: s.snapshot.Priority,
|
||||
NotFillCache: s.snapshot.NotFillCache,
|
||||
},
|
||||
}
|
||||
resp, err := sender.SendReq(bo, req, loc.Region, s.conf.RPC.ReadTimeoutMedium)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
log.Debugf("scanner getData failed: %s", regionErr)
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
cmdScanResp := resp.Scan
|
||||
if cmdScanResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
|
||||
err = s.snapshot.store.CheckVisibility(s.startTS())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
kvPairs := cmdScanResp.Pairs
|
||||
// Check if kvPair contains error, it should be a Lock.
|
||||
for _, pair := range kvPairs {
|
||||
if keyErr := pair.GetError(); keyErr != nil {
|
||||
lock, err := extractLockFromKeyErr(keyErr, s.conf.Txn.DefaultLockTTL)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pair.Key = lock.Key
|
||||
}
|
||||
}
|
||||
|
||||
s.cache, s.idx = kvPairs, 0
|
||||
if len(kvPairs) < s.batchSize {
|
||||
// No more data in current Region. Next getData() starts
|
||||
// from current Region's endKey.
|
||||
s.nextStartKey = loc.EndKey
|
||||
if len(loc.EndKey) == 0 || (len(s.endKey) > 0 && key.Key(s.nextStartKey).Cmp(key.Key(s.endKey)) >= 0) {
|
||||
// Current Region is the last one.
|
||||
s.eof = true
|
||||
}
|
||||
return nil
|
||||
}
|
||||
// next getData() starts from the last key in kvPairs (but skip
|
||||
// it by appending a '\x00' to the key). Note that next getData()
|
||||
// may get an empty response if the Region in fact does not have
|
||||
// more data.
|
||||
lastKey := kvPairs[len(kvPairs)-1].GetKey()
|
||||
s.nextStartKey = key.Key(lastKey).Next()
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
|
@ -1,311 +0,0 @@
|
|||
// Copyright 2015 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 store
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"unsafe"
|
||||
|
||||
pb "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/metrics"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"github.com/tikv/client-go/rpc"
|
||||
"github.com/tikv/client-go/txnkv/kv"
|
||||
)
|
||||
|
||||
// TiKVSnapshot supports read from TiKV.
|
||||
type TiKVSnapshot struct {
|
||||
store *TiKVStore
|
||||
ts uint64
|
||||
conf *config.Config
|
||||
|
||||
Priority pb.CommandPri
|
||||
NotFillCache bool
|
||||
SyncLog bool
|
||||
KeyOnly bool
|
||||
}
|
||||
|
||||
func newTiKVSnapshot(store *TiKVStore, ts uint64) *TiKVSnapshot {
|
||||
metrics.SnapshotCounter.Inc()
|
||||
|
||||
return &TiKVSnapshot{
|
||||
store: store,
|
||||
ts: ts,
|
||||
conf: store.GetConfig(),
|
||||
Priority: pb.CommandPri_Normal,
|
||||
}
|
||||
}
|
||||
|
||||
// BatchGet gets all the keys' value from kv-server and returns a map contains key/value pairs.
|
||||
// The map will not contain nonexistent keys.
|
||||
func (s *TiKVSnapshot) BatchGet(ctx context.Context, keys []key.Key) (map[string][]byte, error) {
|
||||
m := make(map[string][]byte)
|
||||
if len(keys) == 0 {
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// We want [][]byte instead of []key.Key, use some magic to save memory.
|
||||
bytesKeys := *(*[][]byte)(unsafe.Pointer(&keys))
|
||||
bo := retry.NewBackoffer(ctx, retry.BatchGetMaxBackoff)
|
||||
|
||||
// Create a map to collect key-values from region servers.
|
||||
var mu sync.Mutex
|
||||
err := s.batchGetKeysByRegions(bo, bytesKeys, func(k, v []byte) {
|
||||
if len(v) == 0 {
|
||||
return
|
||||
}
|
||||
mu.Lock()
|
||||
m[string(k)] = v
|
||||
mu.Unlock()
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = s.store.CheckVisibility(s.ts)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (s *TiKVSnapshot) batchGetKeysByRegions(bo *retry.Backoffer, keys [][]byte, collectF func(k, v []byte)) error {
|
||||
groups, _, err := s.store.regionCache.GroupKeysByRegion(bo, keys)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
metrics.TxnRegionsNumHistogram.WithLabelValues("snapshot").Observe(float64(len(groups)))
|
||||
|
||||
var batches []batchKeys
|
||||
for id, g := range groups {
|
||||
batches = appendBatchBySize(batches, id, g, func([]byte) int { return 1 }, s.conf.Txn.BatchGetSize)
|
||||
}
|
||||
|
||||
if len(batches) == 0 {
|
||||
return nil
|
||||
}
|
||||
if len(batches) == 1 {
|
||||
return s.batchGetSingleRegion(bo, batches[0], collectF)
|
||||
}
|
||||
ch := make(chan error)
|
||||
for _, batch1 := range batches {
|
||||
batch := batch1
|
||||
go func() {
|
||||
backoffer, cancel := bo.Fork()
|
||||
defer cancel()
|
||||
ch <- s.batchGetSingleRegion(backoffer, batch, collectF)
|
||||
}()
|
||||
}
|
||||
for i := 0; i < len(batches); i++ {
|
||||
if e := <-ch; e != nil {
|
||||
log.Debugf("snapshot batchGet failed: %v, tid: %d", e, s.ts)
|
||||
err = e
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (s *TiKVSnapshot) batchGetSingleRegion(bo *retry.Backoffer, batch batchKeys, collectF func(k, v []byte)) error {
|
||||
sender := rpc.NewRegionRequestSender(s.store.GetRegionCache(), s.store.GetRPCClient())
|
||||
|
||||
pending := batch.keys
|
||||
for {
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdBatchGet,
|
||||
BatchGet: &pb.BatchGetRequest{
|
||||
Keys: pending,
|
||||
Version: s.ts,
|
||||
},
|
||||
Context: pb.Context{
|
||||
Priority: s.Priority,
|
||||
NotFillCache: s.NotFillCache,
|
||||
},
|
||||
}
|
||||
resp, err := sender.SendReq(bo, req, batch.region, s.conf.RPC.ReadTimeoutMedium)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return s.batchGetKeysByRegions(bo, pending, collectF)
|
||||
}
|
||||
batchGetResp := resp.BatchGet
|
||||
if batchGetResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
var (
|
||||
lockedKeys [][]byte
|
||||
locks []*Lock
|
||||
)
|
||||
for _, pair := range batchGetResp.Pairs {
|
||||
keyErr := pair.GetError()
|
||||
if keyErr == nil {
|
||||
collectF(pair.GetKey(), pair.GetValue())
|
||||
continue
|
||||
}
|
||||
lock, err := extractLockFromKeyErr(keyErr, s.conf.Txn.DefaultLockTTL)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
lockedKeys = append(lockedKeys, lock.Key)
|
||||
locks = append(locks, lock)
|
||||
}
|
||||
if len(lockedKeys) > 0 {
|
||||
ok, err := s.store.lockResolver.ResolveLocks(bo, locks)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if !ok {
|
||||
err = bo.Backoff(retry.BoTxnLockFast, errors.Errorf("batchGet lockedKeys: %d", len(lockedKeys)))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
pending = lockedKeys
|
||||
continue
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// Get gets the value for key k from snapshot.
|
||||
func (s *TiKVSnapshot) Get(ctx context.Context, k key.Key) ([]byte, error) {
|
||||
val, err := s.get(retry.NewBackoffer(ctx, retry.GetMaxBackoff), k)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(val) == 0 {
|
||||
return nil, kv.ErrNotExist
|
||||
}
|
||||
return val, nil
|
||||
}
|
||||
|
||||
func (s *TiKVSnapshot) get(bo *retry.Backoffer, k key.Key) ([]byte, error) {
|
||||
sender := rpc.NewRegionRequestSender(s.store.GetRegionCache(), s.store.GetRPCClient())
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdGet,
|
||||
Get: &pb.GetRequest{
|
||||
Key: k,
|
||||
Version: s.ts,
|
||||
},
|
||||
Context: pb.Context{
|
||||
Priority: s.Priority,
|
||||
NotFillCache: s.NotFillCache,
|
||||
},
|
||||
}
|
||||
for {
|
||||
loc, err := s.store.regionCache.LocateKey(bo, k)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
resp, err := sender.SendReq(bo, req, loc.Region, s.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
continue
|
||||
}
|
||||
cmdGetResp := resp.Get
|
||||
if cmdGetResp == nil {
|
||||
return nil, errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
val := cmdGetResp.GetValue()
|
||||
if keyErr := cmdGetResp.GetError(); keyErr != nil {
|
||||
lock, err := extractLockFromKeyErr(keyErr, s.conf.Txn.DefaultLockTTL)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
ok, err := s.store.lockResolver.ResolveLocks(bo, []*Lock{lock})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if !ok {
|
||||
err = bo.Backoff(retry.BoTxnLockFast, errors.New(keyErr.String()))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
continue
|
||||
}
|
||||
return val, nil
|
||||
}
|
||||
}
|
||||
|
||||
// Iter returns a list of key-value pair after `k`.
|
||||
func (s *TiKVSnapshot) Iter(ctx context.Context, k key.Key, upperBound key.Key) (kv.Iterator, error) {
|
||||
scanner, err := newScanner(ctx, s, k, upperBound, s.conf.Txn.ScanBatchSize)
|
||||
return scanner, err
|
||||
}
|
||||
|
||||
// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k.
|
||||
func (s *TiKVSnapshot) IterReverse(ctx context.Context, k key.Key) (kv.Iterator, error) {
|
||||
return nil, ErrNotImplemented
|
||||
}
|
||||
|
||||
// SetPriority sets the priority of read requests.
|
||||
func (s *TiKVSnapshot) SetPriority(priority int) {
|
||||
s.Priority = pb.CommandPri(priority)
|
||||
}
|
||||
|
||||
func extractLockFromKeyErr(keyErr *pb.KeyError, defaultTTL uint64) (*Lock, error) {
|
||||
if locked := keyErr.GetLocked(); locked != nil {
|
||||
return NewLock(locked, defaultTTL), nil
|
||||
}
|
||||
if keyErr.Conflict != nil {
|
||||
err := errors.New(conflictToString(keyErr.Conflict))
|
||||
return nil, errors.WithMessage(err, TxnRetryableMark)
|
||||
}
|
||||
if keyErr.Retryable != "" {
|
||||
err := errors.Errorf("tikv restarts txn: %s", keyErr.GetRetryable())
|
||||
log.Debug(err)
|
||||
return nil, errors.WithMessage(err, TxnRetryableMark)
|
||||
}
|
||||
if keyErr.Abort != "" {
|
||||
err := errors.Errorf("tikv aborts txn: %s", keyErr.GetAbort())
|
||||
log.Warn(err)
|
||||
return nil, err
|
||||
}
|
||||
return nil, errors.Errorf("unexpected KeyError: %s", keyErr.String())
|
||||
}
|
||||
|
||||
func conflictToString(conflict *pb.WriteConflict) string {
|
||||
var buf bytes.Buffer
|
||||
fmt.Fprintf(&buf, "WriteConflict: startTS=%d, conflictTS=%d, key=%q, primary=%q", conflict.StartTs, conflict.ConflictTs, conflict.Key, conflict.Primary)
|
||||
return buf.String()
|
||||
}
|
||||
|
|
@ -1,69 +0,0 @@
|
|||
// Copyright 2017 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 store
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/key"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"github.com/tikv/client-go/rpc"
|
||||
)
|
||||
|
||||
// SplitRegion splits the region contains splitKey into 2 regions: [start,
|
||||
// splitKey) and [splitKey, end).
|
||||
func SplitRegion(ctx context.Context, store *TiKVStore, splitKey key.Key) error {
|
||||
log.Infof("start split_region at %q", splitKey)
|
||||
bo := retry.NewBackoffer(ctx, retry.SplitRegionBackoff)
|
||||
sender := rpc.NewRegionRequestSender(store.GetRegionCache(), store.GetRPCClient())
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdSplitRegion,
|
||||
SplitRegion: &kvrpcpb.SplitRegionRequest{
|
||||
SplitKey: splitKey,
|
||||
},
|
||||
}
|
||||
req.Context.Priority = kvrpcpb.CommandPri_Normal
|
||||
conf := store.GetConfig()
|
||||
for {
|
||||
loc, err := store.GetRegionCache().LocateKey(bo, splitKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if bytes.Equal(splitKey, loc.StartKey) {
|
||||
log.Infof("skip split_region region at %q", splitKey)
|
||||
return nil
|
||||
}
|
||||
res, err := sender.SendReq(bo, req, loc.Region, conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := res.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
log.Infof("split_region at %q complete, new regions: %v, %v", splitKey, res.SplitRegion.GetLeft(), res.SplitRegion.GetRight())
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
|
@ -1,229 +0,0 @@
|
|||
// 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 store
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/tikv/client-go/config"
|
||||
"github.com/tikv/client-go/locate"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"github.com/tikv/client-go/rpc"
|
||||
"github.com/tikv/client-go/txnkv/latch"
|
||||
"github.com/tikv/client-go/txnkv/oracle"
|
||||
"github.com/tikv/client-go/txnkv/oracle/oracles"
|
||||
pd "github.com/tikv/pd/client"
|
||||
)
|
||||
|
||||
// TiKVStore contains methods to interact with a TiKV cluster.
|
||||
type TiKVStore struct {
|
||||
conf *config.Config
|
||||
clusterID uint64
|
||||
uuid string
|
||||
oracle oracle.Oracle
|
||||
client rpc.Client
|
||||
pdClient pd.Client
|
||||
regionCache *locate.RegionCache
|
||||
lockResolver *LockResolver
|
||||
txnLatches *latch.LatchesScheduler
|
||||
etcdAddrs []string
|
||||
tlsConfig *tls.Config
|
||||
|
||||
spkv SafePointKV
|
||||
safePoint uint64
|
||||
spTime time.Time
|
||||
spMutex sync.RWMutex // this is used to update safePoint and spTime
|
||||
closed chan struct{} // this is used to nofity when the store is closed
|
||||
}
|
||||
|
||||
// NewStore creates a TiKVStore instance.
|
||||
func NewStore(ctx context.Context, pdAddrs []string, conf config.Config) (*TiKVStore, error) {
|
||||
pdCli, err := pd.NewClient(pdAddrs, pd.SecurityOption{
|
||||
CAPath: conf.RPC.Security.SSLCA,
|
||||
CertPath: conf.RPC.Security.SSLCert,
|
||||
KeyPath: conf.RPC.Security.SSLKey,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
pdClient := &locate.CodecPDClient{Client: pdCli}
|
||||
|
||||
oracle, err := oracles.NewPdOracle(pdCli, &conf.Txn)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tlsConfig, err := conf.RPC.Security.ToTLSConfig()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
spkv, err := NewEtcdSafePointKV(pdAddrs, tlsConfig)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
clusterID := pdCli.GetClusterID(ctx)
|
||||
|
||||
store := &TiKVStore{
|
||||
conf: &conf,
|
||||
clusterID: clusterID,
|
||||
uuid: fmt.Sprintf("tikv-%d", clusterID),
|
||||
oracle: oracle,
|
||||
client: rpc.NewRPCClient(&conf.RPC),
|
||||
pdClient: pdClient,
|
||||
regionCache: locate.NewRegionCache(pdClient, &conf.RegionCache),
|
||||
etcdAddrs: pdAddrs,
|
||||
tlsConfig: tlsConfig,
|
||||
spkv: spkv,
|
||||
spTime: time.Now(),
|
||||
closed: make(chan struct{}),
|
||||
}
|
||||
|
||||
store.lockResolver = newLockResolver(store)
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
// GetOracle returns the oracle instance.
|
||||
func (s *TiKVStore) GetOracle() oracle.Oracle {
|
||||
return s.oracle
|
||||
}
|
||||
|
||||
// GetRegionCache returns the region cache instance.
|
||||
func (s *TiKVStore) GetRegionCache() *locate.RegionCache {
|
||||
return s.regionCache
|
||||
}
|
||||
|
||||
// GetRPCClient returns the rpc client instance.
|
||||
func (s *TiKVStore) GetRPCClient() rpc.Client {
|
||||
return s.client
|
||||
}
|
||||
|
||||
// GetTxnLatches returns the latch scheduler instance.
|
||||
func (s *TiKVStore) GetTxnLatches() *latch.LatchesScheduler {
|
||||
return s.txnLatches
|
||||
}
|
||||
|
||||
// GetSnapshot creates a snapshot for read.
|
||||
func (s *TiKVStore) GetSnapshot(ts uint64) *TiKVSnapshot {
|
||||
return newTiKVSnapshot(s, ts)
|
||||
}
|
||||
|
||||
// SendReq sends a request to TiKV server.
|
||||
func (s *TiKVStore) SendReq(bo *retry.Backoffer, req *rpc.Request, regionID locate.RegionVerID, timeout time.Duration) (*rpc.Response, error) {
|
||||
sender := rpc.NewRegionRequestSender(s.regionCache, s.client)
|
||||
return sender.SendReq(bo, req, regionID, timeout)
|
||||
}
|
||||
|
||||
// Closed returns a channel that will be closed when TiKVStore is closed.
|
||||
func (s *TiKVStore) Closed() <-chan struct{} {
|
||||
return s.closed
|
||||
}
|
||||
|
||||
// Close stops the TiKVStore instance and releases resources.
|
||||
func (s *TiKVStore) Close() error {
|
||||
s.oracle.Close()
|
||||
s.pdClient.Close()
|
||||
|
||||
close(s.closed)
|
||||
if err := s.client.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if s.txnLatches != nil {
|
||||
s.txnLatches.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetTimestampWithRetry queries PD for a new timestamp.
|
||||
func (s *TiKVStore) GetTimestampWithRetry(bo *retry.Backoffer) (uint64, error) {
|
||||
for {
|
||||
startTS, err := s.oracle.GetTimestamp(bo.GetContext())
|
||||
if err == nil {
|
||||
return startTS, nil
|
||||
}
|
||||
err = bo.Backoff(retry.BoPDRPC, errors.Errorf("get timestamp failed: %v", err))
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *TiKVStore) runSafePointChecker() {
|
||||
d := s.conf.Txn.GcSafePointUpdateInterval
|
||||
for {
|
||||
select {
|
||||
case spCachedTime := <-time.After(d):
|
||||
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 = s.conf.Txn.GcSafePointUpdateInterval
|
||||
} else {
|
||||
metrics.LoadSafepointCounter.WithLabelValues("fail").Inc()
|
||||
log.Errorf("fail to load safepoint from pd: %v", err)
|
||||
d = s.conf.Txn.GcSafePointQuickRepeatInterval
|
||||
}
|
||||
case <-s.Closed():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// CheckVisibility checks if it is safe to read using startTS (the startTS should
|
||||
// be greater than current GC safepoint).
|
||||
func (s *TiKVStore) CheckVisibility(startTS uint64) error {
|
||||
s.spMutex.RLock()
|
||||
cachedSafePoint := s.safePoint
|
||||
cachedTime := s.spTime
|
||||
s.spMutex.RUnlock()
|
||||
diff := time.Since(cachedTime)
|
||||
|
||||
if diff > (s.conf.Txn.GcSafePointCacheInterval - s.conf.Txn.GcCPUTimeInaccuracyBound) {
|
||||
return errors.WithStack(ErrPDServerTimeout)
|
||||
}
|
||||
|
||||
if startTS < cachedSafePoint {
|
||||
return errors.WithStack(ErrStartTSFallBehind)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
@ -1,596 +0,0 @@
|
|||
// Copyright 2016 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 store
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"math"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
pb "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/metrics"
|
||||
"github.com/tikv/client-go/retry"
|
||||
"github.com/tikv/client-go/rpc"
|
||||
"github.com/tikv/client-go/txnkv/kv"
|
||||
)
|
||||
|
||||
type commitAction int
|
||||
|
||||
const (
|
||||
actionPrewrite commitAction = 1
|
||||
actionCommit commitAction = 2
|
||||
actionCleanup commitAction = 3
|
||||
)
|
||||
|
||||
func (ca commitAction) String() string {
|
||||
switch ca {
|
||||
case actionPrewrite:
|
||||
return "prewrite"
|
||||
case actionCommit:
|
||||
return "commit"
|
||||
case actionCleanup:
|
||||
return "cleanup"
|
||||
}
|
||||
return "unknown"
|
||||
}
|
||||
|
||||
// MetricsTag returns detail tag for metrics.
|
||||
func (ca commitAction) MetricsTag() string {
|
||||
return "2pc_" + ca.String()
|
||||
}
|
||||
|
||||
// TxnCommitter executes a two-phase commit protocol.
|
||||
type TxnCommitter struct {
|
||||
Priority pb.CommandPri
|
||||
SyncLog bool
|
||||
ConnID uint64 // ConnID is used for log.
|
||||
|
||||
store *TiKVStore
|
||||
conf *config.Config
|
||||
startTS uint64
|
||||
keys [][]byte
|
||||
mutations map[string]*pb.Mutation
|
||||
lockTTL uint64
|
||||
commitTS uint64
|
||||
mu struct {
|
||||
sync.RWMutex
|
||||
committed bool
|
||||
undeterminedErr error // undeterminedErr saves the rpc error we encounter when commit primary key.
|
||||
}
|
||||
cleanWg sync.WaitGroup
|
||||
// maxTxnTimeUse represents 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
|
||||
detail CommitDetails
|
||||
}
|
||||
|
||||
// NewTxnCommitter creates a TxnCommitter.
|
||||
func NewTxnCommitter(store *TiKVStore, startTS uint64, startTime time.Time, mutations map[string]*pb.Mutation) (*TxnCommitter, error) {
|
||||
var (
|
||||
keys [][]byte
|
||||
size int
|
||||
putCnt int
|
||||
delCnt int
|
||||
lockCnt int
|
||||
)
|
||||
|
||||
conf := store.GetConfig()
|
||||
for key, mut := range mutations {
|
||||
switch mut.Op {
|
||||
case pb.Op_Put, pb.Op_Insert:
|
||||
putCnt++
|
||||
case pb.Op_Del:
|
||||
delCnt++
|
||||
case pb.Op_Lock:
|
||||
lockCnt++
|
||||
}
|
||||
keys = append(keys, []byte(key))
|
||||
entrySize := len(mut.Key) + len(mut.Value)
|
||||
if entrySize > conf.Txn.EntrySizeLimit {
|
||||
return nil, kv.ErrEntryTooLarge
|
||||
}
|
||||
size += entrySize
|
||||
}
|
||||
|
||||
if putCnt == 0 && delCnt == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
if len(keys) > int(conf.Txn.EntryCountLimit) || size > conf.Txn.TotalSizeLimit {
|
||||
return nil, kv.ErrTxnTooLarge
|
||||
}
|
||||
|
||||
// Convert from sec to ms
|
||||
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(conf, startTime, size),
|
||||
maxTxnTimeUse: maxTxnTimeUse,
|
||||
detail: CommitDetails{WriteSize: size, WriteKeys: len(keys)},
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) primary() []byte {
|
||||
return c.keys[0]
|
||||
}
|
||||
|
||||
const bytesPerMiB = 1024 * 1024
|
||||
|
||||
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 := conf.Txn.DefaultLockTTL
|
||||
if txnSize >= conf.Txn.CommitBatchSize {
|
||||
sizeMiB := float64(txnSize) / bytesPerMiB
|
||||
lockTTL = uint64(float64(conf.Txn.TTLFactor) * math.Sqrt(sizeMiB))
|
||||
if lockTTL < conf.Txn.DefaultLockTTL {
|
||||
lockTTL = conf.Txn.DefaultLockTTL
|
||||
}
|
||||
if lockTTL > conf.Txn.MaxLockTTL {
|
||||
lockTTL = conf.Txn.MaxLockTTL
|
||||
}
|
||||
}
|
||||
|
||||
// Increase lockTTL by the transaction's read time.
|
||||
// When resolving a lock, we compare current ts and startTS+lockTTL to decide whether to clean up. If a txn
|
||||
// takes a long time to read, increasing its TTL will help to prevent it from been aborted soon after prewrite.
|
||||
elapsed := time.Since(startTime) / time.Millisecond
|
||||
return lockTTL + uint64(elapsed)
|
||||
}
|
||||
|
||||
// doActionOnKeys groups keys into primary batch and secondary batches, if primary batch exists in the key,
|
||||
// it does action on primary batch first, then on secondary batches. If action is commit, secondary batches
|
||||
// is done in background goroutine.
|
||||
func (c *TxnCommitter) doActionOnKeys(bo *retry.Backoffer, action commitAction, keys [][]byte) error {
|
||||
if len(keys) == 0 {
|
||||
return nil
|
||||
}
|
||||
groups, firstRegion, err := c.store.GetRegionCache().GroupKeysByRegion(bo, keys)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
metrics.TxnRegionsNumHistogram.WithLabelValues(action.MetricsTag()).Observe(float64(len(groups)))
|
||||
|
||||
var batches []batchKeys
|
||||
var sizeFunc = c.keySize
|
||||
if action == actionPrewrite {
|
||||
sizeFunc = c.keyValueSize
|
||||
atomic.AddInt32(&c.detail.PrewriteRegionNum, int32(len(groups)))
|
||||
}
|
||||
// Make sure the group that contains primary key goes first.
|
||||
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, commitBatchSize)
|
||||
}
|
||||
|
||||
firstIsPrimary := bytes.Equal(keys[0], c.primary())
|
||||
if firstIsPrimary && (action == actionCommit || action == actionCleanup) {
|
||||
// primary should be committed/cleanup first
|
||||
err = c.doActionOnBatches(bo, action, batches[:1])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
batches = batches[1:]
|
||||
}
|
||||
if action == actionCommit {
|
||||
// Commit secondary batches in background goroutine to reduce latency.
|
||||
// The backoffer instance is created outside of the goroutine to avoid
|
||||
// potencial data race in unit test since `CommitMaxBackoff` will be updated
|
||||
// by test suites.
|
||||
secondaryBo := retry.NewBackoffer(context.Background(), retry.CommitMaxBackoff)
|
||||
go func() {
|
||||
e := c.doActionOnBatches(secondaryBo, action, batches)
|
||||
if e != nil {
|
||||
log.Debugf("con:%d 2PC async doActionOnBatches %s err: %v", c.ConnID, action, e)
|
||||
metrics.SecondaryLockCleanupFailureCounter.WithLabelValues("commit").Inc()
|
||||
}
|
||||
}()
|
||||
} else {
|
||||
err = c.doActionOnBatches(bo, action, batches)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// doActionOnBatches does action to batches in parallel.
|
||||
func (c *TxnCommitter) doActionOnBatches(bo *retry.Backoffer, action commitAction, batches []batchKeys) error {
|
||||
if len(batches) == 0 {
|
||||
return nil
|
||||
}
|
||||
var singleBatchActionFunc func(bo *retry.Backoffer, batch batchKeys) error
|
||||
switch action {
|
||||
case actionPrewrite:
|
||||
singleBatchActionFunc = c.prewriteSingleBatch
|
||||
case actionCommit:
|
||||
singleBatchActionFunc = c.commitSingleBatch
|
||||
case actionCleanup:
|
||||
singleBatchActionFunc = c.cleanupSingleBatch
|
||||
}
|
||||
if len(batches) == 1 {
|
||||
e := singleBatchActionFunc(bo, batches[0])
|
||||
if e != nil {
|
||||
log.Debugf("con:%d 2PC doActionOnBatches %s failed: %v, tid: %d", c.ConnID, action, e, c.startTS)
|
||||
}
|
||||
return e
|
||||
}
|
||||
|
||||
// For prewrite, stop sending other requests after receiving first error.
|
||||
backoffer := bo
|
||||
var cancel context.CancelFunc
|
||||
if action == actionPrewrite {
|
||||
backoffer, cancel = bo.Fork()
|
||||
defer cancel()
|
||||
}
|
||||
|
||||
// Concurrently do the work for each batch.
|
||||
ch := make(chan error, len(batches))
|
||||
for _, batch1 := range batches {
|
||||
batch := batch1
|
||||
go func() {
|
||||
if action == actionCommit {
|
||||
// Because the secondary batches of the commit actions are implemented to be
|
||||
// committed asynchronously in background goroutines, we should not
|
||||
// fork a child context and call cancel() while the foreground goroutine exits.
|
||||
// Otherwise the background goroutines will be canceled exceptionally.
|
||||
// Here we makes a new clone of the original backoffer for this goroutine
|
||||
// exclusively to avoid the data race when using the same backoffer
|
||||
// in concurrent goroutines.
|
||||
singleBatchBackoffer := backoffer.Clone()
|
||||
ch <- singleBatchActionFunc(singleBatchBackoffer, batch)
|
||||
} else {
|
||||
singleBatchBackoffer, singleBatchCancel := backoffer.Fork()
|
||||
defer singleBatchCancel()
|
||||
ch <- singleBatchActionFunc(singleBatchBackoffer, batch)
|
||||
}
|
||||
}()
|
||||
}
|
||||
var err error
|
||||
for i := 0; i < len(batches); i++ {
|
||||
if e := <-ch; e != nil {
|
||||
log.Debugf("con:%d 2PC doActionOnBatches %s failed: %v, tid: %d", c.ConnID, action, e, c.startTS)
|
||||
// Cancel other requests and return the first error.
|
||||
if cancel != nil {
|
||||
log.Debugf("con:%d 2PC doActionOnBatches %s to cancel other actions, tid: %d", c.ConnID, action, c.startTS)
|
||||
cancel()
|
||||
}
|
||||
if err == nil {
|
||||
err = e
|
||||
}
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) keyValueSize(key []byte) int {
|
||||
size := len(key)
|
||||
if mutation := c.mutations[string(key)]; mutation != nil {
|
||||
size += len(mutation.Value)
|
||||
}
|
||||
return size
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) keySize(key []byte) int {
|
||||
return len(key)
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) prewriteSingleBatch(bo *retry.Backoffer, batch batchKeys) error {
|
||||
mutations := make([]*pb.Mutation, len(batch.keys))
|
||||
for i, k := range batch.keys {
|
||||
mutations[i] = c.mutations[string(k)]
|
||||
}
|
||||
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdPrewrite,
|
||||
Prewrite: &pb.PrewriteRequest{
|
||||
Mutations: mutations,
|
||||
PrimaryLock: c.primary(),
|
||||
StartVersion: c.startTS,
|
||||
LockTtl: c.lockTTL,
|
||||
},
|
||||
Context: pb.Context{
|
||||
Priority: c.Priority,
|
||||
SyncLog: c.SyncLog,
|
||||
},
|
||||
}
|
||||
for {
|
||||
resp, err := c.store.SendReq(bo, req, batch.region, c.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return c.prewriteKeys(bo, batch.keys)
|
||||
}
|
||||
prewriteResp := resp.Prewrite
|
||||
if prewriteResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
keyErrs := prewriteResp.GetErrors()
|
||||
if len(keyErrs) == 0 {
|
||||
return nil
|
||||
}
|
||||
var locks []*Lock
|
||||
for _, keyErr := range keyErrs {
|
||||
// Check already exists error
|
||||
if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil {
|
||||
return errors.WithStack(ErrKeyAlreadyExist(alreadyExist.GetKey()))
|
||||
}
|
||||
|
||||
// Extract lock from key error
|
||||
lock, err1 := extractLockFromKeyErr(keyErr, c.conf.Txn.DefaultLockTTL)
|
||||
if err1 != nil {
|
||||
return err1
|
||||
}
|
||||
log.Debugf("con:%d 2PC prewrite encounters lock: %v", c.ConnID, lock)
|
||||
locks = append(locks, lock)
|
||||
}
|
||||
start := time.Now()
|
||||
ok, err := c.store.GetLockResolver().ResolveLocks(bo, locks)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
atomic.AddInt64(&c.detail.ResolveLockTime, int64(time.Since(start)))
|
||||
if !ok {
|
||||
err = bo.Backoff(retry.BoTxnLock, errors.Errorf("2PC prewrite lockedKeys: %d", len(locks)))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) setUndeterminedErr(err error) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
c.mu.undeterminedErr = err
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) getUndeterminedErr() error {
|
||||
c.mu.RLock()
|
||||
defer c.mu.RUnlock()
|
||||
return c.mu.undeterminedErr
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) commitSingleBatch(bo *retry.Backoffer, batch batchKeys) error {
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdCommit,
|
||||
Commit: &pb.CommitRequest{
|
||||
StartVersion: c.startTS,
|
||||
Keys: batch.keys,
|
||||
CommitVersion: c.commitTS,
|
||||
},
|
||||
Context: pb.Context{
|
||||
Priority: c.Priority,
|
||||
SyncLog: c.SyncLog,
|
||||
},
|
||||
}
|
||||
req.Context.Priority = c.Priority
|
||||
|
||||
sender := rpc.NewRegionRequestSender(c.store.GetRegionCache(), c.store.GetRPCClient())
|
||||
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.
|
||||
// Under this circumstance, we can not declare the commit is complete (may lead to data lost), nor can we throw
|
||||
// an error (may lead to the duplicated key error when upper level restarts the transaction). Currently the best
|
||||
// solution is to populate this error and let upper layer drop the connection to the corresponding mysql client.
|
||||
isPrimary := bytes.Equal(batch.keys[0], c.primary())
|
||||
if isPrimary && sender.RPCError() != nil {
|
||||
c.setUndeterminedErr(sender.RPCError())
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// re-split keys and commit again.
|
||||
return c.commitKeys(bo, batch.keys)
|
||||
}
|
||||
commitResp := resp.Commit
|
||||
if commitResp == nil {
|
||||
return errors.WithStack(rpc.ErrBodyMissing)
|
||||
}
|
||||
// Here we can make sure tikv has processed the commit primary key request. So
|
||||
// we can clean undetermined error.
|
||||
if isPrimary {
|
||||
c.setUndeterminedErr(nil)
|
||||
}
|
||||
if keyErr := commitResp.GetError(); keyErr != nil {
|
||||
c.mu.RLock()
|
||||
defer c.mu.RUnlock()
|
||||
err = errors.Errorf("con:%d 2PC commit failed: %v", c.ConnID, keyErr.String())
|
||||
if c.mu.committed {
|
||||
// No secondary key could be rolled back after it's primary key is committed.
|
||||
// There must be a serious bug somewhere.
|
||||
log.Errorf("2PC failed commit key after primary key committed: %v, tid: %d", err, c.startTS)
|
||||
return err
|
||||
}
|
||||
// The transaction maybe rolled back by concurrent transactions.
|
||||
log.Debugf("2PC failed commit primary key: %v, retry later, tid: %d", err, c.startTS)
|
||||
return errors.WithMessage(err, TxnRetryableMark)
|
||||
}
|
||||
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
// Group that contains primary key is always the first.
|
||||
// We mark transaction's status committed when we receive the first success response.
|
||||
c.mu.committed = true
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) cleanupSingleBatch(bo *retry.Backoffer, batch batchKeys) error {
|
||||
req := &rpc.Request{
|
||||
Type: rpc.CmdBatchRollback,
|
||||
BatchRollback: &pb.BatchRollbackRequest{
|
||||
Keys: batch.keys,
|
||||
StartVersion: c.startTS,
|
||||
},
|
||||
Context: pb.Context{
|
||||
Priority: c.Priority,
|
||||
SyncLog: c.SyncLog,
|
||||
},
|
||||
}
|
||||
resp, err := c.store.SendReq(bo, req, batch.region, c.conf.RPC.ReadTimeoutShort)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
regionErr, err := resp.GetRegionError()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if regionErr != nil {
|
||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return c.cleanupKeys(bo, batch.keys)
|
||||
}
|
||||
if keyErr := resp.BatchRollback.GetError(); keyErr != nil {
|
||||
err = errors.Errorf("con:%d 2PC cleanup failed: %s", c.ConnID, keyErr)
|
||||
log.Debugf("2PC failed cleanup key: %v, tid: %d", err, c.startTS)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) prewriteKeys(bo *retry.Backoffer, keys [][]byte) error {
|
||||
return c.doActionOnKeys(bo, actionPrewrite, keys)
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) commitKeys(bo *retry.Backoffer, keys [][]byte) error {
|
||||
return c.doActionOnKeys(bo, actionCommit, keys)
|
||||
}
|
||||
|
||||
func (c *TxnCommitter) cleanupKeys(bo *retry.Backoffer, keys [][]byte) error {
|
||||
return c.doActionOnKeys(bo, actionCleanup, keys)
|
||||
}
|
||||
|
||||
// Execute executes the two-phase commit protocol.
|
||||
func (c *TxnCommitter) Execute(ctx context.Context) error {
|
||||
defer func() {
|
||||
// Always clean up all written keys if the txn does not commit.
|
||||
c.mu.RLock()
|
||||
committed := c.mu.committed
|
||||
undetermined := c.mu.undeterminedErr != nil
|
||||
c.mu.RUnlock()
|
||||
if !committed && !undetermined {
|
||||
c.cleanWg.Add(1)
|
||||
go func() {
|
||||
err := c.cleanupKeys(retry.NewBackoffer(context.Background(), retry.CleanupMaxBackoff), c.keys)
|
||||
if err != nil {
|
||||
metrics.SecondaryLockCleanupFailureCounter.WithLabelValues("rollback").Inc()
|
||||
log.Infof("con:%d 2PC cleanup err: %v, tid: %d", c.ConnID, err, c.startTS)
|
||||
} else {
|
||||
log.Infof("con:%d 2PC clean up done, tid: %d", c.ConnID, c.startTS)
|
||||
}
|
||||
c.cleanWg.Done()
|
||||
}()
|
||||
}
|
||||
}()
|
||||
|
||||
prewriteBo := retry.NewBackoffer(ctx, retry.PrewriteMaxBackoff)
|
||||
start := time.Now()
|
||||
err := c.prewriteKeys(prewriteBo, c.keys)
|
||||
c.detail.PrewriteTime = time.Since(start)
|
||||
c.detail.TotalBackoffTime += prewriteBo.TotalSleep()
|
||||
|
||||
if err != nil {
|
||||
log.Debugf("con:%d 2PC failed on prewrite: %v, tid: %d", c.ConnID, err, c.startTS)
|
||||
return err
|
||||
}
|
||||
|
||||
start = time.Now()
|
||||
commitTS, err := c.store.GetTimestampWithRetry(retry.NewBackoffer(ctx, retry.TsoMaxBackoff))
|
||||
if err != nil {
|
||||
log.Warnf("con:%d 2PC get commitTS failed: %v, tid: %d", c.ConnID, err, c.startTS)
|
||||
return err
|
||||
}
|
||||
c.detail.GetCommitTsTime = time.Since(start)
|
||||
|
||||
// check commitTS
|
||||
if commitTS <= c.startTS {
|
||||
err = errors.Errorf("con:%d Invalid transaction tso with start_ts=%v while commit_ts=%v",
|
||||
c.ConnID, c.startTS, commitTS)
|
||||
log.Error(err)
|
||||
return err
|
||||
}
|
||||
c.commitTS = commitTS
|
||||
|
||||
if c.store.GetOracle().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.WithMessage(err, TxnRetryableMark)
|
||||
}
|
||||
|
||||
start = time.Now()
|
||||
commitBo := retry.NewBackoffer(ctx, retry.CommitMaxBackoff)
|
||||
err = c.commitKeys(commitBo, c.keys)
|
||||
c.detail.CommitTime = time.Since(start)
|
||||
c.detail.TotalBackoffTime += commitBo.TotalSleep()
|
||||
if err != nil {
|
||||
if undeterminedErr := c.getUndeterminedErr(); undeterminedErr != nil {
|
||||
log.Warnf("con:%d 2PC commit result undetermined, err: %v, rpcErr: %v, tid: %v", c.ConnID, err, undeterminedErr, c.startTS)
|
||||
log.Error(err)
|
||||
err = errors.WithStack(ErrResultUndetermined)
|
||||
}
|
||||
if !c.mu.committed {
|
||||
log.Debugf("con:%d 2PC failed on commit: %v, tid: %d", c.ConnID, err, c.startTS)
|
||||
return err
|
||||
}
|
||||
log.Debugf("con:%d 2PC succeed with error: %v, tid: %d", c.ConnID, err, c.startTS)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetKeys returns all keys of the committer.
|
||||
func (c *TxnCommitter) GetKeys() [][]byte {
|
||||
return c.keys
|
||||
}
|
||||
|
||||
// GetCommitTS returns the commit timestamp of the transaction.
|
||||
func (c *TxnCommitter) GetCommitTS() uint64 {
|
||||
return c.commitTS
|
||||
}
|
||||
303
txnkv/txn.go
303
txnkv/txn.go
|
|
@ -1,303 +0,0 @@
|
|||
// 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 txnkv
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/common/log"
|
||||
"github.com/tikv/client-go/key"
|
||||
"github.com/tikv/client-go/metrics"
|
||||
"github.com/tikv/client-go/txnkv/kv"
|
||||
"github.com/tikv/client-go/txnkv/store"
|
||||
)
|
||||
|
||||
// Transaction is a key-value transaction.
|
||||
type Transaction struct {
|
||||
tikvStore *store.TiKVStore
|
||||
snapshot *store.TiKVSnapshot
|
||||
us kv.UnionStore
|
||||
|
||||
startTS uint64
|
||||
startTime time.Time // Monotonic timestamp for recording txn time consuming.
|
||||
commitTS uint64
|
||||
valid bool
|
||||
lockKeys [][]byte
|
||||
}
|
||||
|
||||
func newTransaction(tikvStore *store.TiKVStore, ts uint64) *Transaction {
|
||||
metrics.TxnCounter.Inc()
|
||||
|
||||
snapshot := tikvStore.GetSnapshot(ts)
|
||||
us := kv.NewUnionStore(&tikvStore.GetConfig().Txn, snapshot)
|
||||
return &Transaction{
|
||||
tikvStore: tikvStore,
|
||||
snapshot: snapshot,
|
||||
us: us,
|
||||
|
||||
startTS: ts,
|
||||
startTime: time.Now(),
|
||||
valid: true,
|
||||
}
|
||||
}
|
||||
|
||||
// Get implements transaction interface.
|
||||
// kv.IsErrNotFound can be used to check the error is a not found error.
|
||||
func (txn *Transaction) Get(ctx context.Context, k key.Key) ([]byte, error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.TxnCmdHistogram.WithLabelValues("get").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
ret, err := txn.us.Get(ctx, k)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = txn.tikvStore.CheckVisibility(txn.startTS)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
// BatchGet gets a batch of values from TiKV server.
|
||||
func (txn *Transaction) BatchGet(ctx context.Context, keys []key.Key) (map[string][]byte, error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.TxnCmdHistogram.WithLabelValues("batch_get").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
if txn.IsReadOnly() {
|
||||
return txn.snapshot.BatchGet(ctx, keys)
|
||||
}
|
||||
bufferValues := make([][]byte, len(keys))
|
||||
shrinkKeys := make([]key.Key, 0, len(keys))
|
||||
for i, key := range keys {
|
||||
val, err := txn.us.GetMemBuffer().Get(ctx, key)
|
||||
if kv.IsErrNotFound(err) {
|
||||
shrinkKeys = append(shrinkKeys, key)
|
||||
continue
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(val) != 0 {
|
||||
bufferValues[i] = val
|
||||
}
|
||||
}
|
||||
storageValues, err := txn.snapshot.BatchGet(ctx, shrinkKeys)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for i, key := range keys {
|
||||
if bufferValues[i] == nil {
|
||||
continue
|
||||
}
|
||||
storageValues[string(key)] = bufferValues[i]
|
||||
}
|
||||
return storageValues, nil
|
||||
}
|
||||
|
||||
// Set sets the value for key k as v into kv store.
|
||||
func (txn *Transaction) Set(k key.Key, v []byte) error {
|
||||
start := time.Now()
|
||||
defer func() { metrics.TxnCmdHistogram.WithLabelValues("set").Observe(time.Since(start).Seconds()) }()
|
||||
return txn.us.Set(k, v)
|
||||
}
|
||||
|
||||
func (txn *Transaction) String() string {
|
||||
return fmt.Sprintf("txn-%d", txn.startTS)
|
||||
}
|
||||
|
||||
// Iter creates an Iterator positioned on the first entry that k <= entry's key.
|
||||
// If such entry is not found, it returns an invalid Iterator with no error.
|
||||
// It yields only keys that < upperBound. If upperBound is nil, it means the upperBound is unbounded.
|
||||
// The Iterator must be closed after use.
|
||||
func (txn *Transaction) Iter(ctx context.Context, k key.Key, upperBound key.Key) (kv.Iterator, error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.TxnCmdHistogram.WithLabelValues("iter").Observe(time.Since(start).Seconds()) }()
|
||||
|
||||
return txn.us.Iter(ctx, k, upperBound)
|
||||
}
|
||||
|
||||
// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k.
|
||||
func (txn *Transaction) IterReverse(ctx context.Context, k key.Key) (kv.Iterator, error) {
|
||||
start := time.Now()
|
||||
defer func() { metrics.TxnCmdHistogram.WithLabelValues("iter_reverse").Observe(time.Since(start).Seconds()) }()
|
||||
return txn.us.IterReverse(ctx, k)
|
||||
}
|
||||
|
||||
// IsReadOnly returns if there are pending key-value to commit in the transaction.
|
||||
func (txn *Transaction) IsReadOnly() bool {
|
||||
return txn.us.GetMemBuffer().Len() == 0 && len(txn.lockKeys) == 0
|
||||
}
|
||||
|
||||
// Delete removes the entry for key k from kv store.
|
||||
func (txn *Transaction) Delete(k key.Key) error {
|
||||
start := time.Now()
|
||||
defer func() { metrics.TxnCmdHistogram.WithLabelValues("delete").Observe(time.Since(start).Seconds()) }()
|
||||
return txn.us.Delete(k)
|
||||
}
|
||||
|
||||
// SetOption sets an option with a value, when val is nil, uses the default
|
||||
// value of this option.
|
||||
func (txn *Transaction) SetOption(opt kv.Option, val interface{}) {
|
||||
txn.us.SetOption(opt, val)
|
||||
switch opt {
|
||||
case kv.Priority:
|
||||
txn.snapshot.SetPriority(val.(int))
|
||||
case kv.NotFillCache:
|
||||
txn.snapshot.NotFillCache = val.(bool)
|
||||
case kv.SyncLog:
|
||||
txn.snapshot.SyncLog = val.(bool)
|
||||
case kv.KeyOnly:
|
||||
txn.snapshot.KeyOnly = val.(bool)
|
||||
}
|
||||
}
|
||||
|
||||
// DelOption deletes an option.
|
||||
func (txn *Transaction) DelOption(opt kv.Option) {
|
||||
txn.us.DelOption(opt)
|
||||
}
|
||||
|
||||
func (txn *Transaction) close() {
|
||||
txn.valid = false
|
||||
}
|
||||
|
||||
// Commit commits the transaction operations to KV store.
|
||||
func (txn *Transaction) Commit(ctx context.Context) error {
|
||||
if !txn.valid {
|
||||
return kv.ErrInvalidTxn
|
||||
}
|
||||
defer txn.close()
|
||||
|
||||
// gofail: var mockCommitError bool
|
||||
// if mockCommitError && kv.IsMockCommitErrorEnable() {
|
||||
// kv.MockCommitErrorDisable()
|
||||
// return errors.New("mock commit error")
|
||||
// }
|
||||
|
||||
start := time.Now()
|
||||
defer func() {
|
||||
metrics.TxnCmdHistogram.WithLabelValues("commit").Observe(time.Since(start).Seconds())
|
||||
metrics.TxnHistogram.Observe(time.Since(txn.startTime).Seconds())
|
||||
}()
|
||||
|
||||
mutations := make(map[string]*kvrpcpb.Mutation)
|
||||
err := txn.us.WalkBuffer(func(k key.Key, v []byte) error {
|
||||
op := kvrpcpb.Op_Put
|
||||
if c := txn.us.LookupConditionPair(k); c != nil && c.ShouldNotExist() {
|
||||
op = kvrpcpb.Op_Insert
|
||||
}
|
||||
if len(v) == 0 {
|
||||
op = kvrpcpb.Op_Del
|
||||
}
|
||||
mutations[string(k)] = &kvrpcpb.Mutation{
|
||||
Op: op,
|
||||
Key: k,
|
||||
Value: v,
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, lockKey := range txn.lockKeys {
|
||||
if _, ok := mutations[string(lockKey)]; !ok {
|
||||
mutations[string(lockKey)] = &kvrpcpb.Mutation{
|
||||
Op: kvrpcpb.Op_Lock,
|
||||
Key: lockKey,
|
||||
}
|
||||
}
|
||||
}
|
||||
if len(mutations) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
committer, err := store.NewTxnCommitter(txn.tikvStore, txn.startTS, txn.startTime, mutations)
|
||||
if err != nil || committer == nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// latches disabled
|
||||
if txn.tikvStore.GetTxnLatches() == nil {
|
||||
err = committer.Execute(ctx)
|
||||
log.Debug("[kv]", txn.startTS, " txnLatches disabled, 2pc directly:", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// latches enabled
|
||||
// for transactions which need to acquire latches
|
||||
start = time.Now()
|
||||
lock := txn.tikvStore.GetTxnLatches().Lock(txn.startTS, committer.GetKeys())
|
||||
localLatchTime := time.Since(start)
|
||||
if localLatchTime > 0 {
|
||||
metrics.LocalLatchWaitTimeHistogram.Observe(localLatchTime.Seconds())
|
||||
}
|
||||
defer txn.tikvStore.GetTxnLatches().UnLock(lock)
|
||||
if lock.IsStale() {
|
||||
err = errors.Errorf("startTS %d is stale", txn.startTS)
|
||||
return errors.WithMessage(err, store.TxnRetryableMark)
|
||||
}
|
||||
err = committer.Execute(ctx)
|
||||
if err == nil {
|
||||
lock.SetCommitTS(committer.GetCommitTS())
|
||||
}
|
||||
log.Debug("[kv]", txn.startTS, " txnLatches enabled while txn retryable:", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Rollback undoes the transaction operations to KV store.
|
||||
func (txn *Transaction) Rollback() error {
|
||||
if !txn.valid {
|
||||
return kv.ErrInvalidTxn
|
||||
}
|
||||
start := time.Now()
|
||||
defer func() {
|
||||
metrics.TxnCmdHistogram.WithLabelValues("rollback").Observe(time.Since(start).Seconds())
|
||||
metrics.TxnHistogram.Observe(time.Since(txn.startTime).Seconds())
|
||||
}()
|
||||
txn.close()
|
||||
log.Debugf("[kv] Rollback txn %d", txn.startTS)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// LockKeys tries to lock the entries with the keys in KV store.
|
||||
func (txn *Transaction) LockKeys(keys ...key.Key) error {
|
||||
start := time.Now()
|
||||
defer func() { metrics.TxnCmdHistogram.WithLabelValues("lock_keys").Observe(time.Since(start).Seconds()) }()
|
||||
for _, key := range keys {
|
||||
txn.lockKeys = append(txn.lockKeys, key)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Valid returns if the transaction is valid.
|
||||
// A transaction becomes invalid after commit or rollback.
|
||||
func (txn *Transaction) Valid() bool {
|
||||
return txn.valid
|
||||
}
|
||||
|
||||
// Len returns the count of key-value pairs in the transaction's memory buffer.
|
||||
func (txn *Transaction) Len() int {
|
||||
return txn.us.Len()
|
||||
}
|
||||
|
||||
// Size returns the length (in bytes) of the transaction's memory buffer.
|
||||
func (txn *Transaction) Size() int {
|
||||
return txn.us.Size()
|
||||
}
|
||||
Loading…
Reference in New Issue