├── proto ├── _tools │ └── .gitignore ├── tools.json ├── proto │ ├── coprocessor.proto │ ├── errorpb.proto │ ├── metapb.proto │ ├── linkkvpb.proto │ ├── raft_serverpb.proto │ ├── raft_cmdpb.proto │ └── eraftpb.proto ├── generate_go.sh └── include │ ├── google │ └── protobuf │ │ ├── source_context.proto │ │ ├── empty.proto │ │ ├── struct.proto │ │ ├── wrappers.proto │ │ ├── duration.proto │ │ └── any.proto │ └── gogoproto │ └── gogo.proto ├── errors ├── .travis.yml ├── .gitignore ├── appveyor.yml ├── group.go ├── LICENSE ├── README.md ├── bench_test.go ├── juju_adaptor.go └── example_test.go ├── kv ├── raftstore │ ├── message │ │ ├── raft_router.go │ │ ├── callback.go │ │ └── msg.go │ ├── util │ │ ├── test_util.go │ │ ├── error_test.go │ │ └── error.go │ ├── bootstrap_test.go │ ├── snap │ │ └── snap_builder.go │ ├── cmd_resp.go │ ├── raft_worker.go │ ├── runner │ │ ├── raftlog_gc.go │ │ ├── split_checker.go │ │ └── scheduler_task.go │ ├── router.go │ ├── meta │ │ ├── values.go │ │ └── keys.go │ ├── ticker.go │ └── bootstrap.go ├── util │ ├── engine_util │ │ ├── doc.go │ │ ├── engines.go │ │ ├── write_batch.go │ │ ├── util.go │ │ ├── cf_iterator.go │ │ └── engine_util_test.go │ ├── worker │ │ └── worker.go │ ├── file.go │ └── codec │ │ └── codec.go ├── storage │ ├── storage.go │ ├── standalone_storage │ │ └── standalone_storage.go │ ├── modify.go │ ├── raft_storage │ │ ├── resolver.go │ │ ├── region_reader.go │ │ ├── transport.go │ │ ├── raft_client.go │ │ └── snap_runner.go │ └── mem_storage.go ├── transaction │ ├── latches │ │ ├── latches_test.go │ │ └── latches.go │ └── mvcc │ │ ├── scanner.go │ │ ├── write.go │ │ ├── lock.go │ │ └── transaction.go ├── test_raftstore │ ├── filter.go │ ├── utils.go │ └── node.go ├── main.go ├── config │ └── config.go ├── coprocessor │ ├── topn.go │ └── rowcodec │ │ ├── common.go │ │ └── decoder.go └── server │ └── server.go ├── LICENSE ├── go.mod └── raft ├── log.go ├── util.go └── rawnode.go /proto/_tools/.gitignore: -------------------------------------------------------------------------------- 1 | /bin/ 2 | /pkg/ 3 | /manifest.json 4 | -------------------------------------------------------------------------------- /errors/.travis.yml: -------------------------------------------------------------------------------- 1 | language: go 2 | go_import_path: github.com/pkg/errors 3 | go: 4 | - 1.4.x 5 | - 1.5.x 6 | - 1.6.x 7 | - 1.7.x 8 | - 1.8.x 9 | - 1.9.x 10 | - 1.10.x 11 | - tip 12 | 13 | script: 14 | - go test -v ./... 15 | -------------------------------------------------------------------------------- /kv/raftstore/message/raft_router.go: -------------------------------------------------------------------------------- 1 | package message 2 | 3 | import ( 4 | "miniLinkDB/proto/pkg/raft_cmdpb" 5 | "miniLinkDB/proto/pkg/raft_serverpb" 6 | ) 7 | 8 | type RaftRouter interface { 9 | Send(regionID uint64, msg Msg) error 10 | SendRaftMessage(msg *raft_serverpb.RaftMessage) error 11 | SendRaftCommand(req *raft_cmdpb.RaftCmdRequest, cb *Callback) error 12 | } -------------------------------------------------------------------------------- /errors/.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled Object files, Static and Dynamic libs (Shared Objects) 2 | *.o 3 | *.a 4 | *.so 5 | 6 | # Folders 7 | _obj 8 | _test 9 | 10 | # Architecture specific extensions/prefixes 11 | *.[568vq] 12 | [568vq].out 13 | 14 | *.cgo1.go 15 | *.cgo2.c 16 | _cgo_defun.c 17 | _cgo_gotypes.go 18 | _cgo_export.* 19 | 20 | _testmain.go 21 | 22 | *.exe 23 | *.test 24 | *.prof 25 | -------------------------------------------------------------------------------- /proto/tools.json: -------------------------------------------------------------------------------- 1 | { 2 | "Tools": [ 3 | { 4 | "Repository": "github.com/gogo/protobuf/protoc-gen-gofast", 5 | "Commit": "636bf0302bc95575d69441b25a2603156ffdddf1" 6 | }, 7 | { 8 | "Repository": "golang.org/x/tools/cmd/goimports", 9 | "Commit": "04b5d21e00f1f47bd824a6ade581e7189bacde87" 10 | } 11 | ], 12 | "RetoolVersion": "1.3.7" 13 | } 14 | -------------------------------------------------------------------------------- /kv/util/engine_util/doc.go: -------------------------------------------------------------------------------- 1 | package engine_util 2 | 3 | /** 4 | 引擎是用于在本地存储键值对的低级系统(没有分发或任何事务支持等). 5 | 该软件包包含用于与此类引擎进行交互的代码。 CF的意思是“列家庭”. 6 | 列族描述参考: https://github.comfacebookrocksdbwikiColumn-Families(专门针对RocksDB,但一般概念是通用的). 7 | 简而言之,列族是关键名称空间. 8 | 通常将多个列族实现为几乎独立的数据库. 9 | 可以使跨列族的写入成为原子的,而对于单独的数据库则无法做到 10 | engine_util包括以下软件包: 11 | engines:一种数据结构,用于保持unistore所需的引擎 12 | write_batch:用于批量写入单个原子“事务”的代码 13 | cf_iterator:在badge中遍历整个列族的代码 14 | */ 15 | -------------------------------------------------------------------------------- /kv/storage/storage.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "miniLinkDB/kv/util/engine_util" 5 | "miniLinkDB/proto/pkg/kvrpcpb" 6 | ) 7 | 8 | // Storage 代表 LinkDB的面向内部的服务器部分 处理来自其它服务器的发送和接收 9 | 10 | type Storage interface { 11 | Start() error 12 | Stop() error 13 | Write(ctx *kvrpcpb.Context, batch []Modify) error 14 | Reader(ctx *kvrpcpb.Context) (StorageReader, error) 15 | } 16 | 17 | type StorageReader interface { 18 | // if key doesn't exist, return nil for the value 19 | GetCF(cf string, key []byte) ([]byte, error) 20 | IterCF(cf string) engine_util.DBIterator 21 | } 22 | -------------------------------------------------------------------------------- /kv/storage/standalone_storage/standalone_storage.go: -------------------------------------------------------------------------------- 1 | package standalone_storage 2 | 3 | import ( 4 | "miniLinkDB/kv/config" 5 | "miniLinkDB/kv/storage" 6 | "miniLinkDB/proto/pkg/kvrpcpb" 7 | ) 8 | 9 | type StandAloneStorage struct { 10 | } 11 | 12 | func NewStandAloneStorage(conf *config.Config) *StandAloneStorage { 13 | return nil 14 | } 15 | 16 | func (s *StandAloneStorage) Start() error { 17 | return nil 18 | } 19 | 20 | func (s *StandAloneStorage) Stop() error { 21 | return nil 22 | } 23 | 24 | func (s *StandAloneStorage) Reader(ctx *kvrpcpb.Context) (storage.StorageReader, error) { 25 | return nil, nil 26 | } 27 | 28 | func (s *StandAloneStorage) Write(ctx *kvrpcpb.Context, batch []storage.Modify) error { 29 | return nil 30 | } 31 | -------------------------------------------------------------------------------- /errors/appveyor.yml: -------------------------------------------------------------------------------- 1 | version: build-{build}.{branch} 2 | 3 | clone_folder: C:\gopath\src\github.com\pkg\errors 4 | shallow_clone: true # for startup speed 5 | 6 | environment: 7 | GOPATH: C:\gopath 8 | 9 | platform: 10 | - x64 11 | 12 | # http://www.appveyor.com/docs/installed-software 13 | install: 14 | # some helpful output for debugging builds 15 | - go version 16 | - go env 17 | # pre-installed MinGW at C:\MinGW is 32bit only 18 | # but MSYS2 at C:\msys64 has mingw64 19 | - set PATH=C:\msys64\mingw64\bin;%PATH% 20 | - gcc --version 21 | - g++ --version 22 | 23 | build_script: 24 | - go install -v ./... 25 | 26 | test_script: 27 | - set PATH=C:\gopath\bin;%PATH% 28 | - go test -v ./... 29 | 30 | #artifacts: 31 | # - path: '%GOPATH%\bin\*.exe' 32 | deploy: off 33 | -------------------------------------------------------------------------------- /kv/transaction/latches/latches_test.go: -------------------------------------------------------------------------------- 1 | package latches 2 | 3 | import ( 4 | "github.com/stretchr/testify/assert" 5 | "sync" 6 | "testing" 7 | ) 8 | 9 | func TestAcquireLatches(t *testing.T) { 10 | l := Latches{ 11 | latchMap: make(map[string]*sync.WaitGroup), 12 | } 13 | 14 | // Acquiring a new latch is ok. 15 | wg := l.AcquireLatches([][]byte{{}, {3}, {3, 0, 42}}) 16 | assert.Nil(t, wg) 17 | 18 | // Can only acquire once. 19 | wg = l.AcquireLatches([][]byte{{}}) 20 | assert.NotNil(t, wg) 21 | wg = l.AcquireLatches([][]byte{{3, 0, 42}}) 22 | assert.NotNil(t, wg) 23 | 24 | // Release then acquire is ok. 25 | l.ReleaseLatches([][]byte{{3}, {3, 0, 43}}) 26 | wg = l.AcquireLatches([][]byte{{3}}) 27 | assert.Nil(t, wg) 28 | wg = l.AcquireLatches([][]byte{{3, 0, 42}}) 29 | assert.NotNil(t, wg) 30 | } -------------------------------------------------------------------------------- /kv/storage/modify.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | // modify是对基础存储单元的单个alter 4 | 5 | type Modify struct { 6 | Data interface{} 7 | } 8 | 9 | type Put struct { 10 | Key []byte 11 | Value []byte 12 | Cf string 13 | } 14 | 15 | type Delete struct { 16 | Key []byte 17 | Cf string 18 | } 19 | 20 | func (m *Modify) Key() []byte { 21 | switch m.Data.(type) { 22 | case Put: 23 | return m.Data.(Put).Key 24 | case Delete: 25 | return m.Data.(Delete).Key 26 | } 27 | return nil 28 | } 29 | 30 | func (m *Modify) Value() []byte { 31 | if putData, ok := m.Data.(Put); ok { 32 | return putData.Value 33 | } 34 | 35 | return nil 36 | } 37 | 38 | func (m *Modify) Cf() string { 39 | switch m.Data.(type) { 40 | case Put: 41 | return m.Data.(Put).Cf 42 | case Delete: 43 | return m.Data.(Delete).Cf 44 | } 45 | 46 | return "" 47 | } 48 | -------------------------------------------------------------------------------- /kv/transaction/mvcc/scanner.go: -------------------------------------------------------------------------------- 1 | package mvcc 2 | 3 | // Scanner is used for reading multiple sequential key/value pairs from the storage layer. It is aware of the implementation 4 | // of the storage layer and returns results suitable for users. 5 | // Invariant: either the scanner is finished and cannot be used, or it is ready to return a value immediately. 6 | type Scanner struct { 7 | 8 | } 9 | 10 | // NewScanner creates a new scanner ready to read from the snapshot in txn. 11 | func NewScanner(startKey []byte, txn *MvccTxn) *Scanner { 12 | 13 | return nil 14 | } 15 | 16 | func (scan *Scanner) Close() { 17 | 18 | } 19 | 20 | // Next returns the next key/value pair from the scanner. If the scanner is exhausted, then it will return `nil, nil, nil`. 21 | func (scan *Scanner) Next() ([]byte, []byte, error) { 22 | 23 | return nil, nil, nil 24 | } -------------------------------------------------------------------------------- /proto/proto/coprocessor.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package coprocessor; 3 | 4 | import "miniLinkDB/proto/pkg/proto/errorpb.proto"; 5 | import "errorpb.proto"; 6 | import "kvrpcpb.proto"; 7 | import "gogoproto/gogo.proto"; 8 | 9 | option (gogoproto.marshaler_all) = true; 10 | option (gogoproto.sizer_all) = true; 11 | option (gogoproto.unmarshaler_all) = true; 12 | 13 | 14 | // [start, end) 15 | message KeyRange { 16 | bytes start = 1; 17 | bytes end = 2; 18 | } 19 | 20 | message Request { 21 | kvrpcpb.Context context = 1; 22 | int64 tp = 2; 23 | bytes data = 3; 24 | uint64 start_ts = 7; 25 | repeated KeyRange ranges = 4; 26 | } 27 | 28 | message Response { 29 | bytes data = 1; 30 | errorpb.Error region_error = 2; 31 | kvrpcpb.LockInfo locked = 3; 32 | string other_error = 4; 33 | KeyRange range = 5; 34 | } 35 | 36 | -------------------------------------------------------------------------------- /kv/raftstore/message/callback.go: -------------------------------------------------------------------------------- 1 | package message 2 | 3 | import ( 4 | "github.com/Connor1996/badger" 5 | "miniLinkDB/proto/pkg/raft_cmdpb" 6 | "time" 7 | ) 8 | 9 | type Callback struct { 10 | Resp *raft_cmdpb.RaftCmdResponse 11 | Txn *badger.Txn 12 | done chan struct{} 13 | } 14 | 15 | func (cb *Callback) Done(resp *raft_cmdpb.RaftCmdResponse) { 16 | if cb == nil { 17 | return 18 | } 19 | if resp != nil { 20 | cb.Resp = resp 21 | } 22 | cb.done <- struct{}{} 23 | } 24 | 25 | func (cb *Callback) WaitResp() *raft_cmdpb.RaftCmdResponse { 26 | select { 27 | case <-cb.done: 28 | return cb.Resp 29 | } 30 | } 31 | 32 | func (cb *Callback) WaitRespWithTimeout(timeout time.Duration) *raft_cmdpb.RaftCmdResponse { 33 | select { 34 | case <-cb.done: 35 | return cb.Resp 36 | case <-time.After(timeout): 37 | return cb.Resp 38 | } 39 | } 40 | 41 | func NewCallback() *Callback { 42 | done := make(chan struct{}, 1) 43 | cb := &Callback{done: done} 44 | return cb 45 | } 46 | -------------------------------------------------------------------------------- /kv/test_raftstore/filter.go: -------------------------------------------------------------------------------- 1 | package test_raftstore 2 | 3 | import ( 4 | "math/rand" 5 | rspb "miniLinkDB/proto/pkg/raft_serverpb" 6 | ) 7 | 8 | type Filter interface { 9 | Before(msgs *rspb.RaftMessage) bool 10 | After() 11 | } 12 | 13 | type PartitionFilter struct { 14 | s1 []uint64 15 | s2 []uint64 16 | } 17 | 18 | func (f *PartitionFilter) Before(msg *rspb.RaftMessage) bool { 19 | inS1 := false 20 | inS2 := false 21 | for _, storeID := range f.s1 { 22 | if msg.FromPeer.StoreId == storeID || msg.ToPeer.StoreId == storeID { 23 | inS1 = true 24 | break 25 | } 26 | } 27 | for _, storeID := range f.s2 { 28 | if msg.FromPeer.StoreId == storeID || msg.ToPeer.StoreId == storeID { 29 | inS2 = true 30 | break 31 | } 32 | } 33 | return !(inS1 && inS2) 34 | } 35 | 36 | func (f *PartitionFilter) After() { 37 | 38 | } 39 | 40 | type DropFilter struct { 41 | 42 | } 43 | 44 | func (f *DropFilter) Before(msg *rspb.RaftMessage) bool { 45 | return (rand.Int() % 1000) > 100 46 | } 47 | 48 | func (f *DropFilter) After() { 49 | 50 | } -------------------------------------------------------------------------------- /kv/raftstore/util/test_util.go: -------------------------------------------------------------------------------- 1 | package util 2 | 3 | import ( 4 | "github.com/Connor1996/badger" 5 | "io/ioutil" 6 | "miniLinkDB/kv/util/engine_util" 7 | ) 8 | 9 | func NewTestEngines() *engine_util.Engines { 10 | engines := new(engine_util.Engines) 11 | var err error 12 | engines.KvPath, err = ioutil.TempDir("","linkkv_kv") 13 | if err != nil { 14 | panic("create kv dir failed") 15 | } 16 | kvOpts := badger.DefaultOptions 17 | kvOpts.Dir = engines.KvPath 18 | kvOpts.ValueDir = engines.KvPath 19 | kvOpts.ValueThreshold = 256 20 | engines.Kv, err = badger.Open(kvOpts) 21 | if err != nil { 22 | panic("open kv db failed") 23 | } 24 | engines.RaftPath, err = ioutil.TempDir("", "linkkv_raft") 25 | if err != nil { 26 | panic("create raft dir failed") 27 | } 28 | raftOpts := badger.DefaultOptions 29 | raftOpts.Dir = engines.RaftPath 30 | raftOpts.ValueDir = engines.RaftPath 31 | raftOpts.ValueThreshold = 256 32 | engines.Raft, err = badger.Open(raftOpts) 33 | if err != nil { 34 | panic("open raft db failed") 35 | } 36 | return engines 37 | } -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2021 李佳 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /proto/proto/errorpb.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package errorpb; 3 | 4 | import "metapb.proto"; 5 | import "gogoproto/gogo.proto"; 6 | 7 | option (gogoproto.marshaler_all) = true; 8 | option (gogoproto.sizer_all) = true; 9 | option (gogoproto.unmarshaler_all) = true; 10 | 11 | message NotLeader { 12 | uint64 region_id = 1; 13 | metapb.Peer leader = 2; 14 | } 15 | 16 | message StoreNotMatch { 17 | uint64 request_store_id = 1; 18 | uint64 actual_store_id = 2; 19 | } 20 | 21 | message RegionNotFound { 22 | uint64 region_id = 1; 23 | } 24 | 25 | message KeyNotInRegion { 26 | bytes key = 1; 27 | uint64 region_id = 2; 28 | bytes start_key = 3; 29 | bytes end_key = 4; 30 | } 31 | 32 | message EpochNotMatch { 33 | repeated metapb.Region current_regions = 1; 34 | } 35 | 36 | message StaleCommand { 37 | } 38 | 39 | message Error { 40 | reserved "stale_epoch"; 41 | 42 | string message = 1; 43 | NotLeader not_leader = 2; 44 | RegionNotFound region_not_found = 3; 45 | KeyNotInRegion key_not_in_region = 4; 46 | EpochNotMatch epoch_not_match = 5; 47 | StaleCommand stale_command = 7; 48 | StoreNotMatch store_not_match = 8; 49 | } -------------------------------------------------------------------------------- /kv/util/worker/worker.go: -------------------------------------------------------------------------------- 1 | package worker 2 | 3 | import "sync" 4 | 5 | type TaskStop struct { 6 | } 7 | 8 | type Task interface { 9 | 10 | } 11 | 12 | type Worker struct { 13 | name string 14 | sender chan<- Task 15 | receiver <- chan Task 16 | closeCh chan struct{} 17 | wg *sync.WaitGroup 18 | } 19 | 20 | type TaskHandler interface { 21 | Handle(t Task) 22 | } 23 | 24 | type Starter interface { 25 | Start() 26 | } 27 | 28 | func (w *Worker) Start(handler TaskHandler) { 29 | w.wg.Add(1) 30 | go func() { 31 | defer w.wg.Done() 32 | if s, ok := handler.(Starter); ok { 33 | s.Start() 34 | } 35 | for { 36 | Task := <-w.receiver 37 | if _, ok := Task.(TaskStop); ok { 38 | return 39 | } 40 | handler.Handle(Task) 41 | } 42 | }() 43 | } 44 | 45 | func (w *Worker) Sender() chan<- Task { 46 | return w.sender 47 | } 48 | 49 | func (w *Worker) Stop() { 50 | w.sender <- TaskStop{} 51 | } 52 | 53 | const defaultWorkerCapacity = 128 54 | 55 | func NewWorker(name string, wg *sync.WaitGroup) *Worker { 56 | ch := make(chan Task, defaultWorkerCapacity) 57 | return &Worker { 58 | sender: (chan<- Task)(ch), 59 | receiver: (<-chan Task)(ch), 60 | name: name, 61 | wg: wg, 62 | } 63 | } -------------------------------------------------------------------------------- /kv/util/file.go: -------------------------------------------------------------------------------- 1 | package util 2 | 3 | import ( 4 | "hash/crc32" 5 | "io" 6 | "miniLinkDB/errors" 7 | "os" 8 | ) 9 | 10 | func GetFileSize(path string) (uint64, error) { 11 | fi, err := os.Stat(path) 12 | if err != nil { 13 | return 0, errors.WithStack(err) 14 | } 15 | return uint64(fi.Size()), nil 16 | } 17 | 18 | func FileExists(path string) bool { 19 | fi, err := os.Stat(path) 20 | if err != nil { 21 | return false 22 | } 23 | return !fi.IsDir() 24 | } 25 | 26 | func DirExists(path string) bool { 27 | fi, err := os.Stat(path) 28 | if err != nil { 29 | return false 30 | } 31 | return fi.IsDir() 32 | } 33 | 34 | func DeleteFileIfExists(path string) (bool, error) { 35 | err := os.Remove(path) 36 | if os.IsNotExist(err) { 37 | return false, nil 38 | } else if err != nil { 39 | return false, errors.WithStack(err) 40 | } 41 | return true, nil 42 | } 43 | 44 | // CalcCRC32计算给定文件的CRC32校验和 45 | func CalcCRC32(path string) (uint32, error) { 46 | digest := crc32.NewIEEE() 47 | f, err := os.Open(path) 48 | if err != nil { 49 | return 0, errors.WithStack(err) 50 | } 51 | _, err = io.Copy(digest, f) 52 | if err != nil { 53 | return 0, errors.WithStack(err) 54 | } 55 | return digest.Sum32(), nil 56 | } -------------------------------------------------------------------------------- /errors/group.go: -------------------------------------------------------------------------------- 1 | package errors 2 | 3 | // ErrorGroup is an interface for multiple errors that are not a chain. 4 | // This happens for example when executing multiple operations in parallel. 5 | type ErrorGroup interface { 6 | Errors() []error 7 | } 8 | 9 | // Errors uses the ErrorGroup interface to return a slice of errors. 10 | // If the ErrorGroup interface is not implemented it returns an array containing just the given error. 11 | func Errors(err error) []error { 12 | if eg, ok := err.(ErrorGroup); ok { 13 | return eg.Errors() 14 | } 15 | return []error{err} 16 | } 17 | 18 | // WalkDeep does a depth-first traversal of all errors. 19 | // Any ErrorGroup is traversed (after going deep). 20 | // The visitor function can return true to end the traversal early 21 | // In that case, WalkDeep will return true, otherwise false. 22 | func WalkDeep(err error, visitor func(err error) bool) bool { 23 | // Go deep 24 | unErr := err 25 | for unErr != nil { 26 | if done := visitor(unErr); done { 27 | return true 28 | } 29 | unErr = Unwrap(unErr) 30 | } 31 | 32 | // Go wide 33 | if group, ok := err.(ErrorGroup); ok { 34 | for _, err := range group.Errors() { 35 | if early := WalkDeep(err, visitor); early { 36 | return true 37 | } 38 | } 39 | } 40 | 41 | return false 42 | } 43 | -------------------------------------------------------------------------------- /proto/proto/metapb.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package metapb; 3 | 4 | import "gogoproto/gogo.proto"; 5 | 6 | option (gogoproto.marshaler_all) = true; 7 | option (gogoproto.sizer_all) = true; 8 | option (gogoproto.unmarshaler_all) = true; 9 | 10 | message Cluster { 11 | uint64 id = 1; 12 | // max peer count for a region. 13 | // scheduler will do the auto-balance if region peer count mismatches. 14 | uint32 max_peer_count = 2; 15 | // more attributes...... 16 | } 17 | 18 | enum StoreState { 19 | Up = 0; 20 | Offline = 1; 21 | Tombstone = 2; 22 | } 23 | 24 | message Store { 25 | uint64 id = 1; 26 | // Address to handle client requests (kv, cop, etc.) 27 | string address = 2; 28 | StoreState state = 3; 29 | } 30 | 31 | message RegionEpoch { 32 | // Conf change version, auto increment when add or remove peer 33 | uint64 conf_ver = 1; 34 | // Region version, auto increment when split or merge 35 | uint64 version = 2; 36 | } 37 | 38 | message Region { 39 | uint64 id = 1; 40 | // Region key range [start_key, end_key). 41 | bytes start_key = 2; 42 | bytes end_key = 3; 43 | RegionEpoch region_epoch = 4; 44 | repeated Peer peers = 5; 45 | } 46 | 47 | message Peer { 48 | uint64 id = 1; 49 | uint64 store_id = 2; 50 | } 51 | -------------------------------------------------------------------------------- /kv/raftstore/bootstrap_test.go: -------------------------------------------------------------------------------- 1 | package raftstore 2 | 3 | import ( 4 | "github.com/stretchr/testify/require" 5 | "miniLinkDB/kv/raftstore/meta" 6 | "miniLinkDB/kv/raftstore/util" 7 | "miniLinkDB/kv/util/engine_util" 8 | "miniLinkDB/proto/pkg/metapb" 9 | "testing" 10 | ) 11 | 12 | func TestBootstrapStore(t *testing.T) { 13 | engines := util.NewTestEngines() 14 | defer engines.Destroy() 15 | require.Nil(t, BootstrapStore(engines, 1, 1)) 16 | require.NotNil(t, BootstrapStore(engines, 1,1)) 17 | _, err := PrepareBootstrap(engines, 1, 1,1) 18 | require.Nil(t, err) 19 | region := new(metapb.Region) 20 | require.Nil(t, engine_util.GetMeta(engines.Kv, meta.PrepareBootstrapKey, region)) 21 | _, err = meta.GetRegionLocalState(engines.Kv, 1) 22 | require.Nil(t, err) 23 | _, err = meta.GetApplyState(engines.Kv, 1) 24 | require.Nil(t, err) 25 | _, err = meta.GetRaftLocalState(engines.Raft, 1) 26 | require.Nil(t, err) 27 | 28 | require.Nil(t, ClearPrepareBootstrapState(engines)) 29 | require.Nil(t, ClearPrepareBootstrap(engines, 1)) 30 | empty, err := isRangeEmpty(engines.Kv, meta.RegionMetaPrefixKey(1), meta.RegionMetaPrefixKey(2)) 31 | require.Nil(t, err) 32 | require.True(t, empty) 33 | 34 | empty, err = isRangeEmpty(engines.Kv, meta.RegionRaftPrefixKey(1), meta.RegionRaftPrefixKey(2)) 35 | require.Nil(t, err) 36 | require.True(t, empty) 37 | } -------------------------------------------------------------------------------- /errors/LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2015, Dave Cheney 2 | All rights reserved. 3 | 4 | Redistribution and use in source and binary forms, with or without 5 | modification, are permitted provided that the following conditions are met: 6 | 7 | * Redistributions of source code must retain the above copyright notice, this 8 | list of conditions and the following disclaimer. 9 | 10 | * Redistributions in binary form must reproduce the above copyright notice, 11 | this list of conditions and the following disclaimer in the documentation 12 | and/or other materials provided with the distribution. 13 | 14 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 15 | AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 16 | IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE 17 | DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE 18 | FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL 19 | DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR 20 | SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 21 | CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, 22 | OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 23 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 24 | -------------------------------------------------------------------------------- /kv/raftstore/snap/snap_builder.go: -------------------------------------------------------------------------------- 1 | package snap 2 | 3 | import ( 4 | "github.com/Connor1996/badger" 5 | "github.com/Connor1996/badger/y" 6 | "miniLinkDB/kv/util/engine_util" 7 | "miniLinkDB/proto/pkg/metapb" 8 | ) 9 | 10 | type snapBuilder struct { 11 | region *metapb.Region 12 | txn *badger.Txn 13 | cfFiles []*CFFile 14 | kvCount int 15 | size int 16 | } 17 | 18 | func newSnapBuilder(cfFiles []*CFFile, dbSnap *badger.Txn, region *metapb.Region) *snapBuilder { 19 | return &snapBuilder{ 20 | region: region, 21 | cfFiles: cfFiles, 22 | txn: dbSnap, 23 | } 24 | } 25 | 26 | func (b *snapBuilder) build() error { 27 | defer b.txn.Discard() 28 | startKey, endKey := b.region.StartKey, b.region.EndKey 29 | 30 | for _, file := range b.cfFiles { 31 | cf := file.CF 32 | sstWriter := file.SstWriter 33 | 34 | it := engine_util.NewCFIterator(cf, b.txn) 35 | for it.Seek(startKey); it.Valid(); it.Next() { 36 | item := it.Item() 37 | key := item.Key() 38 | if engine_util.ExceedEndKey(key, endKey) { 39 | break 40 | } 41 | value, err := item.Value() 42 | if err != nil { 43 | return err 44 | } 45 | cfKey := engine_util.KeyWithCF(cf, key) 46 | if err := sstWriter.Add(cfKey, y.ValueStruct{ 47 | Value: value, 48 | }); err != nil { 49 | return err 50 | } 51 | file.KVCount++ 52 | file.Size += uint64(len(cfKey) + len(value)) 53 | } 54 | it.Close() 55 | b.kvCount += file.KVCount 56 | b.size += int(file.Size) 57 | } 58 | return nil 59 | } -------------------------------------------------------------------------------- /kv/raftstore/cmd_resp.go: -------------------------------------------------------------------------------- 1 | package raftstore 2 | 3 | import ( 4 | "miniLinkDB/kv/raftstore/util" 5 | "miniLinkDB/proto/pkg/errorpb" 6 | "miniLinkDB/proto/pkg/raft_cmdpb" 7 | ) 8 | 9 | func ensureRespHeader(resp *raft_cmdpb.RaftCmdResponse) { 10 | header := resp.GetHeader() 11 | if header == nil { 12 | resp.Header = &raft_cmdpb.RaftResponseHeader{} 13 | } 14 | } 15 | 16 | func BindRespTerm(resp *raft_cmdpb.RaftCmdResponse, term uint64) { 17 | if term == 0 { 18 | return 19 | } 20 | ensureRespHeader(resp) 21 | resp.Header.CurrentTerm = term 22 | } 23 | 24 | func BindRespError(resp *raft_cmdpb.RaftCmdResponse, err error) { 25 | ensureRespHeader(resp) 26 | resp.Header.Error = util.RaftstoreErrToPbError(err) 27 | } 28 | 29 | func ErrResp(err error) *raft_cmdpb.RaftCmdResponse { 30 | resp := &raft_cmdpb.RaftCmdResponse{Header: &raft_cmdpb.RaftResponseHeader{}} 31 | BindRespError(resp, err) 32 | return resp 33 | } 34 | 35 | func ErrRespWithTerm(err error, term uint64) *raft_cmdpb.RaftCmdResponse { 36 | resp := ErrResp(err) 37 | BindRespTerm(resp, term) 38 | return resp 39 | } 40 | 41 | func ErrRespStaleCommand(term uint64) *raft_cmdpb.RaftCmdResponse { 42 | return ErrRespWithTerm(new(util.ErrStaleCommand), term) 43 | } 44 | 45 | func ErrRespRegionNotFound(regionID uint64) *raft_cmdpb.RaftCmdResponse { 46 | return &raft_cmdpb.RaftCmdResponse{ 47 | Header: &raft_cmdpb.RaftResponseHeader{ 48 | Error: &errorpb.Error{ 49 | Message: "region is not found", 50 | RegionNotFound: &errorpb.RegionNotFound{ 51 | RegionId: regionID, 52 | }, 53 | }, 54 | }, 55 | } 56 | } 57 | 58 | func newCmdResp() *raft_cmdpb.RaftCmdResponse { 59 | return &raft_cmdpb.RaftCmdResponse{ 60 | Header: &raft_cmdpb.RaftResponseHeader{}, 61 | } 62 | } -------------------------------------------------------------------------------- /kv/util/engine_util/engines.go: -------------------------------------------------------------------------------- 1 | package engine_util 2 | 3 | import ( 4 | "github.com/Connor1996/badger" 5 | "miniLinkDB/log" 6 | "os" 7 | ) 8 | 9 | // 引擎会保留对unistore使用的引擎的引用和数据,所有引擎都是badge键值数据库 “路径”字段是存储数据的文件系统路径 10 | type Engines struct { 11 | // 12 | Kv *badger.DB 13 | KvPath string 14 | // Metadata used used by Raft 15 | Raft *badger.DB 16 | RaftPath string 17 | } 18 | 19 | func NewEngines(kvEngine, raftEngine *badger.DB, kvPath, raftPath string) *Engines { 20 | return &Engines{ 21 | Kv: kvEngine, 22 | KvPath: kvPath, 23 | Raft: raftEngine, 24 | RaftPath: raftPath, 25 | } 26 | } 27 | 28 | func (en *Engines) WriteKV(wb *WriteBatch) error { 29 | return wb.WriteToDB(en.Kv) 30 | } 31 | 32 | func (en *Engines) WriteRaft(wb *WriteBatch) error { 33 | return wb.WriteToDB(en.Raft) 34 | } 35 | 36 | func (en *Engines) Close() error { 37 | if err := en.Kv.Close(); err != nil { 38 | return err 39 | } 40 | if err := en.Raft.Close(); err != nil { 41 | return err 42 | } 43 | return nil 44 | } 45 | 46 | 47 | func (en *Engines) Destroy() error { 48 | if err := en.Close(); err != nil { 49 | return err 50 | } 51 | if err := os.RemoveAll(en.KvPath); err != nil { 52 | return err 53 | } 54 | if err := os.RemoveAll(en.RaftPath); err != nil { 55 | return err 56 | } 57 | return nil 58 | } 59 | 60 | // CreateDB在子路径的磁盘上创建一个新的Badger DB 61 | func CreateDB(path string, raft bool) *badger.DB { 62 | opts := badger.DefaultOptions 63 | if raft { 64 | opts.ValueThreshold = 0 65 | } 66 | 67 | opts.Dir = path 68 | opts.ValueDir = opts.Dir 69 | if err := os.MkdirAll(opts.Dir, os.ModePerm); err != nil { 70 | log.Fatal(err) 71 | } 72 | db, err := badger.Open(opts) 73 | if err != nil { 74 | log.Fatal(err) 75 | } 76 | return db 77 | } 78 | -------------------------------------------------------------------------------- /kv/transaction/mvcc/write.go: -------------------------------------------------------------------------------- 1 | package mvcc 2 | 3 | import ( 4 | "encoding/binary" 5 | "fmt" 6 | "miniLinkDB/proto/pkg/kvrpcpb" 7 | ) 8 | 9 | // Write is a representation of a committed write to backing storage. 10 | // A serialized version is stored in the "write" CF of our engine when a write is committed. That allows MvccTxn to find 11 | // the status of a key at a given timestamp. 12 | type Write struct { 13 | StartTS uint64 14 | Kind WriteKind 15 | } 16 | 17 | func (wr *Write) ToBytes() []byte { 18 | buf := append([]byte{byte(wr.Kind)}, 0, 0, 0, 0, 0, 0, 0, 0) 19 | binary.BigEndian.PutUint64(buf[1:], wr.StartTS) 20 | return buf 21 | } 22 | 23 | func ParseWrite(value []byte) (*Write, error) { 24 | if value == nil { 25 | return nil, nil 26 | } 27 | if len(value) != 9 { 28 | return nil, fmt.Errorf("mvcc/write/ParseWrite: value is incorrect length, expected 9, found %d", len(value)) 29 | } 30 | kind := value[0] 31 | startTs := binary.BigEndian.Uint64(value[1:]) 32 | 33 | return &Write{startTs, WriteKind(kind)}, nil 34 | } 35 | 36 | type WriteKind int 37 | 38 | const ( 39 | WriteKindPut WriteKind = 1 40 | WriteKindDelete WriteKind = 2 41 | WriteKindRollback WriteKind = 3 42 | ) 43 | 44 | func (wk WriteKind) ToProto() kvrpcpb.Op { 45 | switch wk { 46 | case WriteKindPut: 47 | return kvrpcpb.Op_Put 48 | case WriteKindDelete: 49 | return kvrpcpb.Op_Del 50 | case WriteKindRollback: 51 | return kvrpcpb.Op_Rollback 52 | } 53 | return -1 54 | } 55 | 56 | func WriteKindFromProto(op kvrpcpb.Op) WriteKind { 57 | switch op { 58 | case kvrpcpb.Op_Put: 59 | return WriteKindPut 60 | case kvrpcpb.Op_Del: 61 | return WriteKindDelete 62 | case kvrpcpb.Op_Rollback: 63 | return WriteKindRollback 64 | default: 65 | panic("unsupported type") 66 | } 67 | } -------------------------------------------------------------------------------- /kv/storage/raft_storage/resolver.go: -------------------------------------------------------------------------------- 1 | package raft_storage 2 | 3 | import ( 4 | "context" 5 | "miniLinkDB/errors" 6 | "time" 7 | 8 | "miniLinkDB/kv/raftstore/scheduler_client" 9 | "miniLinkDB/kv/util/worker" 10 | "miniLinkDB/proto/pkg/metapb" 11 | ) 12 | 13 | // 句柄会将t的storeID解析为应处理t的LinkKV节点的地址 14 | // 然后使用该地址调用t的回调 15 | func (r *resolverRunner) Handle(t worker.Task) { 16 | data := t.(*resolveAddrTask) 17 | data.callback(r.getAddr(data.storeID)) 18 | } 19 | 20 | 21 | const storeAddressRefreshSeconds = 60 22 | 23 | type storeAddr struct { 24 | addr string 25 | lastUpdate time.Time 26 | } 27 | 28 | type resolverRunner struct { 29 | schedulerClient scheduler_client.Client 30 | storeAddrs map[uint64]storeAddr 31 | } 32 | 33 | 34 | type resolveAddrTask struct { 35 | storeID uint64 36 | callback func(addr string, err error) 37 | } 38 | 39 | 40 | func newResolverRunner(schedulerClient scheduler_client.Client) *resolverRunner { 41 | return &resolverRunner{ 42 | schedulerClient: schedulerClient, 43 | storeAddrs: make(map[uint64]storeAddr), 44 | } 45 | } 46 | 47 | func (r *resolverRunner) getAddr(id uint64) (string, error) { 48 | if sa, ok := r.storeAddrs[id]; ok { 49 | if time.Since(sa.lastUpdate).Seconds() < storeAddressRefreshSeconds { 50 | return sa.addr, nil 51 | } 52 | } 53 | 54 | store, err := r.schedulerClient.GetStore(context.TODO(),id) 55 | if err != nil { 56 | return "", err 57 | } 58 | if store.GetState() == metapb.StoreState_Tombstone { 59 | return "", errors.Errorf("store %d has been removed", id) 60 | } 61 | 62 | addr := store.GetAddress() 63 | if addr == "" { 64 | return "", errors.Errorf("invalid empty address for store %d", id) 65 | } 66 | r.storeAddrs[id] = storeAddr{ 67 | addr: addr, 68 | lastUpdate: time.Now(), 69 | } 70 | return addr, nil 71 | } -------------------------------------------------------------------------------- /proto/proto/linkkvpb.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package linkdbpb; 3 | 4 | import "kvrpcpb.proto"; 5 | import "raft_serverpb.proto"; 6 | import "coprocessor.proto"; 7 | 8 | import "gogoproto/gogo.proto"; 9 | 10 | option (gogoproto.sizer_all) = true; 11 | option (gogoproto.marshaler_all) = true; 12 | option (gogoproto.unmarshaler_all) = true; 13 | 14 | // Serve as a distributed kv database. See the request and response definitions in 15 | // kvrpcpb.proto 16 | service LinkKv { 17 | // KV commands with mvcc/txn supported. 18 | rpc KvGet(kvrpcpb.GetRequest) returns (kvrpcpb.GetResponse) {} 19 | rpc KvScan(kvrpcpb.ScanRequest) returns (kvrpcpb.ScanResponse) {} 20 | rpc KvPrewrite(kvrpcpb.PrewriteRequest) returns (kvrpcpb.PrewriteResponse) {} 21 | rpc KvCommit(kvrpcpb.CommitRequest) returns (kvrpcpb.CommitResponse) {} 22 | rpc KvCheckTxnStatus(kvrpcpb.CheckTxnStatusRequest) returns (kvrpcpb.CheckTxnStatusResponse) {} 23 | rpc KvBatchRollback(kvrpcpb.BatchRollbackRequest) returns (kvrpcpb.BatchRollbackResponse) {} 24 | rpc KvResolveLock(kvrpcpb.ResolveLockRequest) returns (kvrpcpb.ResolveLockResponse) {} 25 | 26 | // RawKV commands. 27 | rpc RawGet(kvrpcpb.RawGetRequest) returns (kvrpcpb.RawGetResponse) {} 28 | rpc RawPut(kvrpcpb.RawPutRequest) returns (kvrpcpb.RawPutResponse) {} 29 | rpc RawDelete(kvrpcpb.RawDeleteRequest) returns (kvrpcpb.RawDeleteResponse) {} 30 | rpc RawScan(kvrpcpb.RawScanRequest) returns (kvrpcpb.RawScanResponse) {} 31 | 32 | // Raft commands (linkkv <-> linkkv). 33 | rpc Raft(stream raft_serverpb.RaftMessage) returns (raft_serverpb.Done) {} 34 | rpc Snapshot(stream raft_serverpb.SnapshotChunk) returns (raft_serverpb.Done) {} 35 | 36 | // Coprocessor 37 | rpc Coprocessor(coprocessor.Request) returns (coprocessor.Response) {} 38 | } 39 | -------------------------------------------------------------------------------- /kv/raftstore/raft_worker.go: -------------------------------------------------------------------------------- 1 | package raftstore 2 | 3 | import ( 4 | "miniLinkDB/kv/raftstore/message" 5 | "sync" 6 | ) 7 | 8 | // raftWorker is responsible for run raft commands and apply raft logs. 9 | type raftWorker struct { 10 | pr *router 11 | 12 | raftCh chan message.Msg 13 | ctx *GlobalContext 14 | 15 | close <- chan struct{} 16 | } 17 | 18 | func newRaftWorker(ctx *GlobalContext, pm *router) *raftWorker { 19 | return &raftWorker{ 20 | raftCh: pm.peerSender, 21 | ctx: ctx, 22 | pr: pm, 23 | } 24 | } 25 | 26 | 27 | // run runs raft commands. 28 | // On each loop, raft commands are batched by channel buffer. 29 | // After commands are handled, we collect apply messages by peers, make a applyBatch, send it to apply channel. 30 | func (rw *raftWorker) run(closeCh <-chan struct{}, wg *sync.WaitGroup) { 31 | defer wg.Done() 32 | var msgs []message.Msg 33 | for { 34 | msgs = msgs[:0] 35 | select { 36 | case <-closeCh: 37 | return 38 | case msg := <-rw.raftCh: 39 | msgs = append(msgs, msg) 40 | } 41 | pending := len(rw.raftCh) 42 | for i := 0; i < pending; i++ { 43 | msgs = append(msgs, <-rw.raftCh) 44 | } 45 | peerStateMap := make(map[uint64]*peerState) 46 | for _, msg := range msgs { 47 | peerState := rw.getPeerState(peerStateMap, msg.RegionID) 48 | if peerState == nil { 49 | continue 50 | } 51 | newPeerMsgHandler(peerState.peer, rw.ctx).HandleMsg(msg) 52 | } 53 | for _, peerState := range peerStateMap { 54 | newPeerMsgHandler(peerState.peer, rw.ctx).HandleRaftReady() 55 | } 56 | } 57 | } 58 | 59 | 60 | func (rw *raftWorker) getPeerState(peersMap map[uint64]*peerState, regionID uint64) *peerState { 61 | peer, ok := peersMap[regionID] 62 | if !ok { 63 | peer = rw.pr.get(regionID) 64 | if peer == nil { 65 | return nil 66 | } 67 | peersMap[regionID] = peer 68 | } 69 | return peer 70 | } -------------------------------------------------------------------------------- /kv/raftstore/util/error_test.go: -------------------------------------------------------------------------------- 1 | package util 2 | 3 | import ( 4 | "github.com/stretchr/testify/assert" 5 | "github.com/stretchr/testify/require" 6 | "miniLinkDB/proto/pkg/metapb" 7 | "testing" 8 | ) 9 | 10 | func TestRaftstoreErrToPbError(t *testing.T) { 11 | regionId := uint64(1) 12 | notLeader := &ErrNotLeader{RegionId: regionId, Leader: nil} 13 | pbErr := RaftstoreErrToPbError(notLeader) 14 | require.NotNil(t, pbErr.NotLeader) 15 | assert.Equal(t, regionId, pbErr.NotLeader.RegionId) 16 | 17 | regionNotFound := &ErrRegionNotFound{RegionId: regionId} 18 | pbErr = RaftstoreErrToPbError(regionNotFound) 19 | require.NotNil(t, pbErr.RegionNotFound) 20 | assert.Equal(t, regionId, pbErr.RegionNotFound.RegionId) 21 | 22 | region := &metapb.Region{Id: regionId, StartKey: []byte{0}, EndKey: []byte{1}} 23 | 24 | keyNotInRegion := &ErrKeyNotInRegion{Key:[]byte{2},Region: region} 25 | pbErr = RaftstoreErrToPbError(keyNotInRegion) 26 | require.NotNil(t, []byte{0}, pbErr.KeyNotInRegion) 27 | assert.Equal(t, []byte{0}, pbErr.KeyNotInRegion.StartKey) 28 | assert.Equal(t, []byte{1}, pbErr.KeyNotInRegion.EndKey) 29 | assert.Equal(t, []byte{2}, pbErr.KeyNotInRegion.Key) 30 | 31 | epochNotMatch := &ErrEpochNotMatch{Regions: []*metapb.Region{region}} 32 | pbErr = RaftstoreErrToPbError(epochNotMatch) 33 | require.NotNil(t, pbErr.EpochNotMatch) 34 | assert.Equal(t, []*metapb.Region{region}, pbErr.EpochNotMatch.CurrentRegions) 35 | 36 | staleCommand := &ErrStaleCommand{} 37 | pbErr = RaftstoreErrToPbError(staleCommand) 38 | require.NotNil(t, pbErr.StaleCommand) 39 | 40 | requestStoreId, actualStoreId := uint64(1), uint64(2) 41 | storeNotMatch := &ErrStoreNotMatch{RequestStoreId: requestStoreId, ActualStoreId: actualStoreId} 42 | pbErr = RaftstoreErrToPbError(storeNotMatch) 43 | require.NotNil(t, pbErr.StoreNotMatch) 44 | assert.Equal(t, requestStoreId, pbErr.StoreNotMatch.RequestStoreId) 45 | assert.Equal(t, actualStoreId, pbErr.StoreNotMatch.ActualStoreId) 46 | } 47 | -------------------------------------------------------------------------------- /go.mod: -------------------------------------------------------------------------------- 1 | module miniLinkDB 2 | 3 | require ( 4 | github.com/Connor1996/badger v1.5.1-0.20210202034640-5ff470f827f8 5 | github.com/gogo/protobuf v1.3.1 6 | github.com/golang/protobuf v1.3.4 7 | github.com/petar/GoLLRB v0.0.0-20190514000832-33fb24c13b99 8 | github.com/pingcap-incubator/tinykv v0.0.0-20210223122022-25dec1a21fff 9 | golang.org/x/net v0.0.0-20200226121028-0de0cce0169b 10 | google.golang.org/grpc v1.25.1 11 | github.com/BurntSushi/toml v0.3.1 12 | github.com/Connor1996/badger v1.5.1-0.20210202034640-5ff470f827f8 13 | github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f 14 | github.com/docker/go-units v0.4.0 15 | github.com/gogo/protobuf v1.3.1 16 | github.com/golang/protobuf v1.3.4 17 | github.com/google/btree v1.0.0 18 | github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5 19 | github.com/juju/testing v0.0.0-20200510222523-6c8c298c77a0 // indirect 20 | github.com/opentracing/opentracing-go v1.0.2 21 | github.com/petar/GoLLRB v0.0.0-20190514000832-33fb24c13b99 22 | github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 23 | github.com/pingcap/errcode v0.0.0-20180921232412-a1a7271709d9 24 | github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011 25 | github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd 26 | github.com/pingcap/parser v3.1.2+incompatible 27 | github.com/pingcap/tidb v1.1.0-beta.0.20200309111804-d8264d47f760 28 | github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 29 | github.com/pkg/errors v0.8.1 30 | github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect 31 | github.com/shirou/gopsutil v2.19.10+incompatible 32 | github.com/sirupsen/logrus v1.2.0 33 | github.com/stretchr/testify v1.4.0 34 | github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8 35 | go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738 36 | go.uber.org/zap v1.14.0 37 | golang.org/x/net v0.0.0-20200226121028-0de0cce0169b 38 | golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e // indirect 39 | google.golang.org/grpc v1.25.1 40 | gopkg.in/natefinch/lumberjack.v2 v2.0.0 41 | gopkg.in/yaml.v2 v2.2.4 // indirect 42 | ) 43 | 44 | go 1.16 45 | 46 | -------------------------------------------------------------------------------- /kv/raftstore/message/msg.go: -------------------------------------------------------------------------------- 1 | package message 2 | 3 | import ( 4 | "miniLinkDB/kv/raftstore/snap" 5 | "miniLinkDB/proto/pkg/metapb" 6 | "miniLinkDB/proto/pkg/raft_cmdpb" 7 | ) 8 | 9 | type MsgType int64 10 | 11 | const ( 12 | // just a placeholder 13 | MsgTypeNull MsgType = 0 14 | // message to start the ticker of peer 15 | MsgTypeStart MsgType = 1 16 | // message of base tick to drive the ticker 17 | MsgTypeTick MsgType = 2 18 | 19 | // message wraps a raft message that should be forwarded to Raft module 20 | // the raft message is from peer on other store 21 | MsgTypeRaftMessage MsgType = 3 22 | // message warps a raft command that maybe a read/write request or admin request 23 | // the raft command should be proposed to Raft module 24 | MsgTypeRaftCmd MsgType = 4 25 | // 触发分割区域的消息 它首先要求调度程序分配新的分割区域的ID 然后使用split admin命令调度MsyTypeRaftCmd 26 | MsgTypeSplitRegion MsgType = 5 27 | // message to update region approximate size 28 | // it is sent by split checker 29 | MsgTypeRegionApproximateSize MsgType = 6 30 | 31 | // message 以触发gc生成的快照 32 | MsgTypeGcSnap MsgType = 7 33 | 34 | // message wraps a raft message to the peer not existing on the Store. 35 | // It is due to region split or add peer conf change 36 | MsgTypeStoreRaftMessage MsgType = 101 37 | 38 | // message of store base tick to drive the store ticker, including store heartbeat 39 | MsgTypeStoreTick MsgType = 106 40 | 41 | // message to start the ticker of store 42 | MsgTypeStoreStart MsgType = 107 43 | ) 44 | 45 | type Msg struct { 46 | Type MsgType 47 | RegionID uint64 48 | Data interface{} 49 | } 50 | 51 | func NewMsg(tp MsgType, data interface{}) Msg { 52 | return Msg{Type: tp, Data: data} 53 | } 54 | 55 | func NewPeerMsg(tp MsgType, regionID uint64, data interface{}) Msg { 56 | return Msg{Type: tp, RegionID: regionID, Data: data} 57 | } 58 | 59 | type MsgGCSnap struct { 60 | Snaps []snap.SnapKeyWithSending 61 | } 62 | 63 | type MsgRaftCmd struct { 64 | Request *raft_cmdpb.RaftCmdRequest 65 | Callback *Callback 66 | } 67 | 68 | type MsgSplitRegion struct { 69 | RegionEpoch *metapb.RegionEpoch 70 | SplitKey []byte 71 | Callback *Callback 72 | } 73 | -------------------------------------------------------------------------------- /proto/generate_go.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # This script generates Go representations of Protobuf protocols. It will generate Go code in the pkg subdirectory 4 | # for every protocol in the proto subdirectory. It uses protoc, the protobuf compiler, which must be installed. 5 | 6 | set -ex 7 | 8 | push () { 9 | pushd $1 >/dev/null 2>&1 10 | } 11 | 12 | pop () { 13 | popd $1 >/dev/null 2>&1 14 | } 15 | 16 | cmd_exists () { 17 | which "$1" 1>/dev/null 2>&1 18 | } 19 | 20 | PROGRAM=$(basename "$0") 21 | 22 | if [ -z $GOPATH ]; then 23 | printf "Error: the environment variable GOPATH is not set, please set it before running %s\n" $PROGRAM > /dev/stderr 24 | exit 1 25 | fi 26 | 27 | GO_PREFIX_PATH=github.com/pingcap-incubator/linkkv/proto/pkg 28 | export PATH=$(pwd)/_tools/bin:$GOPATH/bin:$PATH 29 | 30 | echo "install tools..." 31 | GO111MODULE=off go get github.com/twitchtv/retool 32 | # Ensure we're using the right versions of our tools (see tools.json). 33 | GO111MODULE=off retool -base-dir=$(pwd) sync || exit 1 34 | 35 | function collect() { 36 | file=$(basename $1) 37 | base_name=$(basename $file ".proto") 38 | mkdir -p ../pkg/$base_name 39 | if [ -z $GO_OUT_M ]; then 40 | GO_OUT_M="M$file=$GO_PREFIX_PATH/$base_name" 41 | else 42 | GO_OUT_M="$GO_OUT_M,M$file=$GO_PREFIX_PATH/$base_name" 43 | fi 44 | } 45 | 46 | # shellcheck disable=SC2045 47 | for file in `ls *.proto` 48 | do 49 | collect $file 50 | done 51 | 52 | echo "generate go code..." 53 | ret=0 54 | 55 | function gen() { 56 | base_name=$(basename $1 ".proto") 57 | protoc -I.:../include --gofast_out=plugins=grpc,$GO_OUT_M:../pkg/$base_name $1 || ret=$? 58 | cd ../pkg/$base_name 59 | sed -i.bak -E 's/import _ \"gogoproto\"//g' *.pb.go 60 | sed -i.bak -E 's/import fmt \"fmt\"//g' *.pb.go 61 | sed -i.bak -E 's/import io \"io\"//g' *.pb.go 62 | sed -i.bak -E 's/import math \"math\"//g' *.pb.go 63 | sed -i.bak -E 's/import _ \".*rustproto\"//' *.pb.go 64 | rm -f *.bak 65 | goimports -w *.pb.go 66 | cd ../../proto 67 | } 68 | 69 | for file in `ls *.proto` 70 | do 71 | gen $file 72 | done 73 | exit $ret 74 | -------------------------------------------------------------------------------- /kv/storage/raft_storage/region_reader.go: -------------------------------------------------------------------------------- 1 | package raft_storage 2 | 3 | import ( 4 | "github.com/Connor1996/badger" 5 | "miniLinkDB/kv/raftstore/util" 6 | "miniLinkDB/kv/util/engine_util" 7 | "miniLinkDB/proto/pkg/metapb" 8 | ) 9 | 10 | type RegionReader struct { 11 | txn *badger.Txn 12 | region *metapb.Region 13 | } 14 | 15 | func NewRegionReader(txn *badger.Txn, region metapb.Region) *RegionReader { 16 | return &RegionReader{ 17 | txn: txn, 18 | region: ®ion, 19 | } 20 | } 21 | 22 | func (r *RegionReader) GetCF(cf string, key []byte) ([]byte, error) { 23 | if err := util.CheckKeyInRegion(key, r.region); err != nil { 24 | return nil, err 25 | } 26 | val, err := engine_util.GetCFFromTxn(r.txn, cf, key) 27 | if err == badger.ErrKeyNotFound { 28 | return nil, nil 29 | } 30 | return val, err 31 | } 32 | 33 | func (r *RegionReader) IterCF(cf string) engine_util.DBIterator { 34 | return NewRegionIterator(engine_util.NewCFIterator(cf, r.txn), r.region) 35 | } 36 | 37 | func (r *RegionReader) Close() { 38 | r.txn.Discard() 39 | } 40 | 41 | // RegionIterator包装一个db迭代器,并且只允许它在区域中进行迭代。它的行为就像基础db仅包含一个区域。 42 | type RegionIterator struct { 43 | iter *engine_util.BadgerIterator 44 | region *metapb.Region 45 | } 46 | 47 | func NewRegionIterator(iter *engine_util.BadgerIterator, region *metapb.Region) *RegionIterator { 48 | return &RegionIterator{ 49 | iter: iter, 50 | region: region, 51 | } 52 | } 53 | 54 | func (it *RegionIterator) Item() engine_util.DBItem { 55 | return it.iter.Item() 56 | } 57 | 58 | func (it *RegionIterator) Valid() bool { 59 | if !it.iter.Valid() || engine_util.ExceedEndKey(it.iter.Item().Key(), it.region.EndKey) { 60 | return false 61 | } 62 | return true 63 | } 64 | 65 | func (it *RegionIterator) ValidForPrefix(prefix []byte) bool { 66 | if !it.iter.ValidForPrefix(prefix) || engine_util.ExceedEndKey(it.iter.Item().Key(), it.region.EndKey) { 67 | return false 68 | } 69 | return true 70 | } 71 | 72 | func (it *RegionIterator) Close() { 73 | it.iter.Close() 74 | } 75 | 76 | func (it *RegionIterator) Next() { 77 | it.iter.Next() 78 | } 79 | 80 | func (it *RegionIterator) Seek(key []byte) { 81 | if err := util.CheckKeyInRegion(key, it.region); err != nil { 82 | panic(err) 83 | } 84 | it.iter.Seek(key) 85 | } 86 | 87 | func (it *RegionIterator) Rewind() { 88 | it.iter.Rewind() 89 | } -------------------------------------------------------------------------------- /kv/raftstore/util/error.go: -------------------------------------------------------------------------------- 1 | package util 2 | 3 | import ( 4 | "fmt" 5 | "miniLinkDB/errors" 6 | 7 | "miniLinkDB/proto/pkg/errorpb" 8 | "miniLinkDB/proto/pkg/metapb" 9 | ) 10 | 11 | type ErrNotLeader struct { 12 | RegionId uint64 13 | Leader *metapb.Peer 14 | } 15 | 16 | func (e *ErrNotLeader) Error() string { 17 | return fmt.Sprintf("region %v is not leader", e.RegionId) 18 | } 19 | 20 | type ErrRegionNotFound struct { 21 | RegionId uint64 22 | } 23 | 24 | func (e *ErrRegionNotFound) Error() string { 25 | return fmt.Sprintf("region %v is not found", e.RegionId) 26 | } 27 | 28 | type ErrKeyNotInRegion struct { 29 | Key []byte 30 | Region *metapb.Region 31 | } 32 | 33 | func (e *ErrKeyNotInRegion) Error() string { 34 | return fmt.Sprintf("key %v is not in region %v", e.Key, e.Region) 35 | } 36 | 37 | type ErrEpochNotMatch struct { 38 | Message string 39 | Regions []*metapb.Region 40 | } 41 | 42 | func (e *ErrEpochNotMatch) Error() string { 43 | return fmt.Sprintf("epoch not match, error msg %v, regions %v", e.Message, e.Regions) 44 | } 45 | 46 | type ErrStaleCommand struct{} 47 | 48 | func (e *ErrStaleCommand) Error() string { 49 | return fmt.Sprintf("stale command") 50 | } 51 | 52 | type ErrStoreNotMatch struct { 53 | RequestStoreId uint64 54 | ActualStoreId uint64 55 | } 56 | 57 | func (e *ErrStoreNotMatch) Error() string { 58 | return fmt.Sprintf("store not match, request store id is %v, but actual store id is %v", e.RequestStoreId, e.ActualStoreId) 59 | } 60 | 61 | 62 | func RaftstoreErrToPbError(e error) *errorpb.Error { 63 | ret := new(errorpb.Error) 64 | switch err := errors.Cause(e).(type) { 65 | case *ErrNotLeader: 66 | ret.NotLeader = &errorpb.NotLeader{RegionId: err.RegionId, Leader: err.Leader} 67 | case *ErrRegionNotFound: 68 | ret.RegionNotFound = &errorpb.RegionNotFound{RegionId: err.RegionId} 69 | case *ErrKeyNotInRegion: 70 | ret.KeyNotInRegion = &errorpb.KeyNotInRegion{Key: err.Key, RegionId: err.Region.Id, 71 | StartKey: err.Region.StartKey, EndKey: err.Region.EndKey} 72 | case *ErrEpochNotMatch: 73 | ret.EpochNotMatch = &errorpb.EpochNotMatch{CurrentRegions: err.Regions} 74 | case *ErrStaleCommand: 75 | ret.StaleCommand = &errorpb.StaleCommand{} 76 | case *ErrStoreNotMatch: 77 | ret.StoreNotMatch = &errorpb.StoreNotMatch{RequestStoreId: err.RequestStoreId, ActualStoreId: err.ActualStoreId} 78 | default: 79 | ret.Message = e.Error() 80 | 81 | } 82 | return ret 83 | } -------------------------------------------------------------------------------- /proto/include/google/protobuf/source_context.proto: -------------------------------------------------------------------------------- 1 | // Protocol Buffers - Google's data interchange format 2 | // Copyright 2008 Google Inc. All rights reserved. 3 | // https://developers.google.com/protocol-buffers/ 4 | // 5 | // Redistribution and use in source and binary forms, with or without 6 | // modification, are permitted provided that the following conditions are 7 | // met: 8 | // 9 | // * Redistributions of source code must retain the above copyright 10 | // notice, this list of conditions and the following disclaimer. 11 | // * Redistributions in binary form must reproduce the above 12 | // copyright notice, this list of conditions and the following disclaimer 13 | // in the documentation and/or other materials provided with the 14 | // distribution. 15 | // * Neither the name of Google Inc. nor the names of its 16 | // contributors may be used to endorse or promote products derived from 17 | // this software without specific prior written permission. 18 | // 19 | // THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 20 | // "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 21 | // LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 22 | // A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 23 | // OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 24 | // SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 25 | // LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 26 | // DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 27 | // THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 28 | // (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | // OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | 31 | syntax = "proto3"; 32 | 33 | package google.protobuf; 34 | 35 | option csharp_namespace = "Google.Protobuf.WellKnownTypes"; 36 | option java_package = "com.google.protobuf"; 37 | option java_outer_classname = "SourceContextProto"; 38 | option java_multiple_files = true; 39 | option objc_class_prefix = "GPB"; 40 | option go_package = "types"; 41 | 42 | // `SourceContext` represents information about the source of a 43 | // protobuf element, like the file in which it is defined. 44 | message SourceContext { 45 | // The path-qualified name of the .proto file that contained the associated 46 | // protobuf element. For example: `"google/protobuf/source_context.proto"`. 47 | string file_name = 1; 48 | } 49 | -------------------------------------------------------------------------------- /errors/README.md: -------------------------------------------------------------------------------- 1 | # errors [![Travis-CI](https://travis-ci.org/pkg/errors.svg)](https://travis-ci.org/pkg/errors) [![AppVeyor](https://ci.appveyor.com/api/projects/status/b98mptawhudj53ep/branch/master?svg=true)](https://ci.appveyor.com/project/davecheney/errors/branch/master) [![GoDoc](https://godoc.org/github.com/pkg/errors?status.svg)](http://godoc.org/github.com/pkg/errors) [![Report card](https://goreportcard.com/badge/github.com/pkg/errors)](https://goreportcard.com/report/github.com/pkg/errors) [![Sourcegraph](https://sourcegraph.com/github.com/pkg/errors/-/badge.svg)](https://sourcegraph.com/github.com/pkg/errors?badge) 2 | 3 | Package errors provides simple error handling primitives. 4 | 5 | `go get github.com/pkg/errors` 6 | 7 | The traditional error handling idiom in Go is roughly akin to 8 | ```go 9 | if err != nil { 10 | return err 11 | } 12 | ``` 13 | which applied recursively up the call stack results in error reports without context or debugging information. The errors package allows programmers to add context to the failure path in their code in a way that does not destroy the original value of the error. 14 | 15 | ## Adding context to an error 16 | 17 | The errors.Wrap function returns a new error that adds context to the original error. For example 18 | ```go 19 | _, err := ioutil.ReadAll(r) 20 | if err != nil { 21 | return errors.Wrap(err, "read failed") 22 | } 23 | ``` 24 | ## Retrieving the cause of an error 25 | 26 | Using `errors.Wrap` constructs a stack of errors, adding context to the preceding error. Depending on the nature of the error it may be necessary to reverse the operation of errors.Wrap to retrieve the original error for inspection. Any error value which implements this interface can be inspected by `errors.Cause`. 27 | ```go 28 | type causer interface { 29 | Cause() error 30 | } 31 | ``` 32 | `errors.Cause` will recursively retrieve the topmost error which does not implement `causer`, which is assumed to be the original cause. For example: 33 | ```go 34 | switch err := errors.Cause(err).(type) { 35 | case *MyError: 36 | // handle specifically 37 | default: 38 | // unknown error 39 | } 40 | ``` 41 | 42 | [Read the package documentation for more information](https://godoc.org/github.com/pkg/errors). 43 | 44 | ## Contributing 45 | 46 | We welcome pull requests, bug fixes and issue reports. With that said, the bar for adding new symbols to this package is intentionally set high. 47 | 48 | Before proposing a change, please discuss your change by raising an issue. 49 | 50 | ## License 51 | 52 | BSD-2-Clause 53 | -------------------------------------------------------------------------------- /errors/bench_test.go: -------------------------------------------------------------------------------- 1 | // +build go1.7 2 | 3 | package errors 4 | 5 | import ( 6 | "fmt" 7 | "testing" 8 | 9 | stderrors "errors" 10 | ) 11 | 12 | func noErrors(at, depth int) error { 13 | if at >= depth { 14 | return stderrors.New("no error") 15 | } 16 | return noErrors(at+1, depth) 17 | } 18 | 19 | func yesErrors(at, depth int) error { 20 | if at >= depth { 21 | return New("ye error") 22 | } 23 | return yesErrors(at+1, depth) 24 | } 25 | 26 | // GlobalE is an exported global to store the result of benchmark results, 27 | // preventing the compiler from optimising the benchmark functions away. 28 | var GlobalE interface{} 29 | 30 | func BenchmarkErrors(b *testing.B) { 31 | type run struct { 32 | stack int 33 | std bool 34 | } 35 | runs := []run{ 36 | {10, false}, 37 | {10, true}, 38 | {100, false}, 39 | {100, true}, 40 | {1000, false}, 41 | {1000, true}, 42 | } 43 | for _, r := range runs { 44 | part := "pkg/errors" 45 | if r.std { 46 | part = "errors" 47 | } 48 | name := fmt.Sprintf("%s-stack-%d", part, r.stack) 49 | b.Run(name, func(b *testing.B) { 50 | var err error 51 | f := yesErrors 52 | if r.std { 53 | f = noErrors 54 | } 55 | b.ReportAllocs() 56 | for i := 0; i < b.N; i++ { 57 | err = f(0, r.stack) 58 | } 59 | b.StopTimer() 60 | GlobalE = err 61 | }) 62 | } 63 | } 64 | 65 | func BenchmarkStackFormatting(b *testing.B) { 66 | type run struct { 67 | stack int 68 | format string 69 | } 70 | runs := []run{ 71 | {10, "%s"}, 72 | {10, "%v"}, 73 | {10, "%+v"}, 74 | {30, "%s"}, 75 | {30, "%v"}, 76 | {30, "%+v"}, 77 | {60, "%s"}, 78 | {60, "%v"}, 79 | {60, "%+v"}, 80 | } 81 | 82 | var stackStr string 83 | for _, r := range runs { 84 | name := fmt.Sprintf("%s-stack-%d", r.format, r.stack) 85 | b.Run(name, func(b *testing.B) { 86 | err := yesErrors(0, r.stack) 87 | b.ReportAllocs() 88 | b.ResetTimer() 89 | for i := 0; i < b.N; i++ { 90 | stackStr = fmt.Sprintf(r.format, err) 91 | } 92 | b.StopTimer() 93 | }) 94 | } 95 | 96 | for _, r := range runs { 97 | name := fmt.Sprintf("%s-stacktrace-%d", r.format, r.stack) 98 | b.Run(name, func(b *testing.B) { 99 | err := yesErrors(0, r.stack) 100 | st := err.(*fundamental).stack.StackTrace() 101 | b.ReportAllocs() 102 | b.ResetTimer() 103 | for i := 0; i < b.N; i++ { 104 | stackStr = fmt.Sprintf(r.format, st) 105 | } 106 | b.StopTimer() 107 | }) 108 | } 109 | GlobalE = stackStr 110 | } 111 | -------------------------------------------------------------------------------- /proto/include/google/protobuf/empty.proto: -------------------------------------------------------------------------------- 1 | // Protocol Buffers - Google's data interchange format 2 | // Copyright 2008 Google Inc. All rights reserved. 3 | // https://developers.google.com/protocol-buffers/ 4 | // 5 | // Redistribution and use in source and binary forms, with or without 6 | // modification, are permitted provided that the following conditions are 7 | // met: 8 | // 9 | // * Redistributions of source code must retain the above copyright 10 | // notice, this list of conditions and the following disclaimer. 11 | // * Redistributions in binary form must reproduce the above 12 | // copyright notice, this list of conditions and the following disclaimer 13 | // in the documentation and/or other materials provided with the 14 | // distribution. 15 | // * Neither the name of Google Inc. nor the names of its 16 | // contributors may be used to endorse or promote products derived from 17 | // this software without specific prior written permission. 18 | // 19 | // THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 20 | // "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 21 | // LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 22 | // A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 23 | // OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 24 | // SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 25 | // LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 26 | // DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 27 | // THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 28 | // (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | // OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | 31 | syntax = "proto3"; 32 | 33 | package google.protobuf; 34 | 35 | option csharp_namespace = "Google.Protobuf.WellKnownTypes"; 36 | option go_package = "types"; 37 | option java_package = "com.google.protobuf"; 38 | option java_outer_classname = "EmptyProto"; 39 | option java_multiple_files = true; 40 | option objc_class_prefix = "GPB"; 41 | option cc_enable_arenas = true; 42 | 43 | // A generic empty message that you can re-use to avoid defining duplicated 44 | // empty messages in your APIs. A typical example is to use it as the request 45 | // or the response type of an API method. For instance: 46 | // 47 | // service Foo { 48 | // rpc Bar(google.protobuf.Empty) returns (google.protobuf.Empty); 49 | // } 50 | // 51 | // The JSON representation for `Empty` is empty JSON object `{}`. 52 | message Empty {} 53 | -------------------------------------------------------------------------------- /kv/util/codec/codec.go: -------------------------------------------------------------------------------- 1 | package codec 2 | 3 | import ( 4 | "fmt" 5 | "miniLinkDB/errors" 6 | ) 7 | 8 | const ( 9 | encGroupSize = 8 10 | encMarker = byte(0xFF) 11 | encPad = byte(0x0) 12 | ) 13 | 14 | var pads = make([]byte, encGroupSize) 15 | 16 | // EncodeBytes 保证编码值按升序进行比较, 17 | // 使用以下规则进行编码: 18 | // [group1][marker1]...[groupN][markerN] 19 | // group is 8 bytes slice which is padding with 0. 20 | // marker is `0xFF - padding 0 count` 21 | // For example: 22 | // [] -> [0, 0, 0, 0, 0, 0, 0, 0, 247] 23 | // [1, 2, 3] -> [1, 2, 3, 0, 0, 0, 0, 0, 250] 24 | // [1, 2, 3, 0] -> [1, 2, 3, 0, 0, 0, 0, 0, 251] 25 | // [1, 2, 3, 4, 5, 6, 7, 8] -> [1, 2, 3, 4, 5, 6, 7, 8, 255, 0, 0, 0, 0, 0, 0, 0, 0, 247] 26 | // Refer: https://github.com/facebook/mysql-5.6/wiki/MyRocks-record-format#memcomparable-format 27 | func EncodeBytes(data []byte) []byte { 28 | // 分配更多空间以避免不必要的切片增长. 29 | // Assume that the byte slice size is about `(len(data) / encGroupSize + 1) * (encGroupSize + 1)` bytes, 30 | // that is `(len(data) / 8 + 1) * 9` in our implement. 31 | dLen := len(data) 32 | result := make([]byte, 0, (dLen/encGroupSize+1)*(encGroupSize+1)+8) 33 | for idx := 0; idx <= dLen; idx += encGroupSize { 34 | remain := dLen - idx 35 | padCount := 0 36 | if remain >= encGroupSize { 37 | result = append(result, data[idx:idx+encGroupSize]...) 38 | } else { 39 | padCount = encGroupSize - remain 40 | result = append(result, data[idx:]...) 41 | result = append(result, pads[:padCount]...) 42 | } 43 | 44 | marker := encMarker - byte(padCount) 45 | result = append(result, marker) 46 | } 47 | return result 48 | } 49 | 50 | 51 | // DecodeBytes 解码之前由EncodeBytes编码的字节, 52 | // returns the leftover bytes and decoded value if no error. 53 | func DecodeBytes(b []byte) ([]byte, []byte, error) { 54 | data := make([]byte, 0, len(b)) 55 | for { 56 | if len(b) < encGroupSize+1 { 57 | return nil, nil, fmt.Errorf("insufficient bytes to decode value: %d", len(b)) 58 | } 59 | groupBytes := b[:encGroupSize+1] 60 | group := groupBytes[:encGroupSize] 61 | marker := groupBytes[encGroupSize] 62 | 63 | padCount := encMarker - marker 64 | if padCount > encGroupSize { 65 | return nil, nil, errors.Errorf("invalid marker byte, group bytes %q", groupBytes) 66 | } 67 | 68 | realGroupSize := encGroupSize - padCount 69 | data = append(data, group[:realGroupSize]...) 70 | b = b[encGroupSize+1:] 71 | 72 | if padCount != 0 { 73 | // Check validity of padding bytes. 74 | for _, v := range group[realGroupSize:] { 75 | if v != encPad { 76 | return nil, nil, errors.Errorf("invalid padding byte, group bytes %q", groupBytes) 77 | } 78 | } 79 | break 80 | } 81 | } 82 | return b, data, nil 83 | } -------------------------------------------------------------------------------- /kv/raftstore/runner/raftlog_gc.go: -------------------------------------------------------------------------------- 1 | package runner 2 | 3 | import ( 4 | "github.com/Connor1996/badger" 5 | "miniLinkDB/kv/raftstore/meta" 6 | "miniLinkDB/kv/util/engine_util" 7 | "miniLinkDB/kv/util/worker" 8 | "miniLinkDB/log" 9 | ) 10 | 11 | type RaftLogGCTask struct { 12 | RaftEngine *badger.DB 13 | RegionID uint64 14 | StartIdx uint64 15 | EndIdx uint64 16 | } 17 | 18 | type raftLogGcTaskRes uint64 19 | 20 | type raftLogGCTaskHandler struct { 21 | taskResCh chan<- raftLogGcTaskRes 22 | } 23 | 24 | func NewRaftLogGCTaskHandler() *raftLogGCTaskHandler { 25 | return &raftLogGCTaskHandler{} 26 | } 27 | 28 | // gcRaftLog does the GC job and returns the count of logs collected.' 29 | func (r *raftLogGCTaskHandler) gcRaftLog(raftDb *badger.DB, regionId, startIdx, endIdx uint64) (uint64, error) { 30 | // Find the raft log idx range needed to be gc. 31 | firstIdx := startIdx 32 | if firstIdx == 0 { 33 | firstIdx = endIdx 34 | err := raftDb.View(func(txn *badger.Txn) error { 35 | startKey := meta.RaftLogKey(regionId, 0) 36 | ite := txn.NewIterator(badger.DefaultIteratorOptions) 37 | defer ite.Close() 38 | if ite.Seek(startKey); ite.Valid() { 39 | var err error 40 | if firstIdx, err = meta.RaftLogIndex(ite.Item().Key()); err != nil { 41 | return err 42 | } 43 | } 44 | return nil 45 | }) 46 | if err != nil { 47 | return 0, err 48 | } 49 | } 50 | 51 | if firstIdx >= endIdx { 52 | log.Infof("no need to gc, [regionId: %d]", regionId) 53 | return 0, nil 54 | } 55 | 56 | raftWb := engine_util.WriteBatch{} 57 | for idx := firstIdx; idx < endIdx; idx += 1 { 58 | key := meta.RaftLogKey(regionId, idx) 59 | raftWb.DeleteMeta(key) 60 | } 61 | if raftWb.Len() != 0 { 62 | if err := raftWb.WriteToDB(raftDb); err != nil { 63 | return 0, err 64 | } 65 | } 66 | return endIdx - firstIdx, nil 67 | } 68 | 69 | func (r *raftLogGCTaskHandler) reportCollected(collected uint64) { 70 | if r.taskResCh == nil { 71 | return 72 | } 73 | r.taskResCh <- raftLogGcTaskRes(collected) 74 | } 75 | 76 | func (r *raftLogGCTaskHandler) Handle(t worker.Task) { 77 | logGcTask, ok := t.(*RaftLogGCTask) 78 | if !ok { 79 | log.Error("unsupported worker.Task: %+v", t) 80 | return 81 | } 82 | log.Debugf("execute gc log. [regionId: %d, endIndex: %d]", logGcTask.RegionID, logGcTask.EndIdx) 83 | collected, err := r.gcRaftLog(logGcTask.RaftEngine, logGcTask.RegionID, logGcTask.StartIdx, logGcTask.EndIdx) 84 | if err != nil { 85 | log.Errorf("failed to gc. [regionId: %d, collected: %d, err: %v]", logGcTask.RegionID, collected, err) 86 | } else { 87 | log.Debugf("collected log entries. [regionId: %d, entryCount: %d]", logGcTask.RegionID, collected) 88 | } 89 | r.reportCollected(collected) 90 | } 91 | -------------------------------------------------------------------------------- /kv/util/engine_util/write_batch.go: -------------------------------------------------------------------------------- 1 | package engine_util 2 | 3 | import ( 4 | "github.com/Connor1996/badger" 5 | "github.com/golang/protobuf/proto" 6 | "miniLinkDB/errors" 7 | ) 8 | 9 | type WriteBatch struct { 10 | entries []*badger.Entry 11 | size int 12 | safePoint int 13 | safePointSize int 14 | safePointUndo int 15 | } 16 | 17 | const ( 18 | CfDefault string = "default" 19 | CfWrite string = "write" 20 | CfLock string = "lock" 21 | ) 22 | 23 | var CFs [3]string = [3]string{CfDefault, CfWrite, CfLock} 24 | 25 | func (wb *WriteBatch) Len() int { 26 | return len(wb.entries) 27 | } 28 | 29 | func (wb *WriteBatch) SetCF(cf string, key, val []byte) { 30 | wb.entries = append(wb.entries, &badger.Entry{ 31 | Key: KeyWithCF(cf, key), 32 | Value: val, 33 | }) 34 | wb.size += len(key) + len(val) 35 | } 36 | 37 | func (wb *WriteBatch) DeleteMeta(key []byte) { 38 | wb.entries = append(wb.entries, &badger.Entry{ 39 | Key: key, 40 | }) 41 | wb.size += len(key) 42 | } 43 | 44 | func (wb *WriteBatch) DeleteCF(cf string, key []byte) { 45 | wb.entries = append(wb.entries, &badger.Entry{ 46 | Key: KeyWithCF(cf, key), 47 | }) 48 | wb.size += len(key) 49 | } 50 | 51 | func (wb *WriteBatch) SetMeta(key []byte, msg proto.Message) error { 52 | val, err := proto.Marshal(msg) 53 | if err != nil { 54 | return errors.WithStack(err) 55 | } 56 | wb.entries = append(wb.entries, &badger.Entry{ 57 | Key: key, 58 | Value: val, 59 | }) 60 | wb.size += len(key) + len(val) 61 | return nil 62 | } 63 | 64 | func (wb *WriteBatch) SetSafePoint() { 65 | wb.safePoint = len(wb.entries) 66 | wb.safePointSize = wb.size 67 | } 68 | 69 | func (wb *WriteBatch) RollbackToSafePoint() { 70 | wb.entries = wb.entries[:wb.safePoint] 71 | wb.size = wb.safePointSize 72 | } 73 | 74 | func (wb *WriteBatch) WriteToDB(db *badger.DB) error { 75 | if len(wb.entries) > 0 { 76 | err := db.Update(func(txn *badger.Txn) error { 77 | for _, entry := range wb.entries { 78 | var err1 error 79 | if len(entry.Value) == 0 { 80 | err1 = txn.Delete(entry.Key) 81 | } else { 82 | err1 = txn.SetEntry(entry) 83 | } 84 | if err1 != nil { 85 | return err1 86 | } 87 | } 88 | return nil 89 | }) 90 | if err != nil { 91 | return errors.WithStack(err) 92 | } 93 | } 94 | return nil 95 | } 96 | 97 | func (wb *WriteBatch) MustWriteToDB(db *badger.DB) { 98 | err := wb.WriteToDB(db) 99 | if err != nil { 100 | panic(err) 101 | } 102 | } 103 | 104 | func (wb *WriteBatch) Reset() { 105 | wb.entries = wb.entries[:0] 106 | wb.size = 0 107 | wb.safePoint = 0 108 | wb.safePointSize = 0 109 | wb.safePointUndo = 0 110 | } 111 | -------------------------------------------------------------------------------- /kv/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "google.golang.org/grpc" 6 | "google.golang.org/grpc/keepalive" 7 | 8 | "miniLinkDB/kv/config" 9 | "miniLinkDB/kv/server" 10 | "miniLinkDB/kv/storage" 11 | "miniLinkDB/kv/storage/raft_storage" 12 | "miniLinkDB/kv/storage/standalone_storage" 13 | "miniLinkDB/log" 14 | 15 | "miniLinkDB/proto/pkg/linkkvpb" 16 | "net" 17 | "os" 18 | "os/signal" 19 | "strings" 20 | "syscall" 21 | "time" 22 | 23 | _ "net/http/pprof" 24 | 25 | ) 26 | 27 | var ( 28 | schedulerAddr = flag.String("scheduler", "", "scheduler address") 29 | storeAddr = flag.String("addr", "", "store address") 30 | dbPath = flag.String("path", "", "directory path of db") 31 | logLevel = flag.String("loglevel", "", "the level of log") 32 | ) 33 | 34 | func main() { 35 | flag.Parse() 36 | conf := config.NewDefaultConfig() 37 | if *schedulerAddr != "" { 38 | conf.SchedulerAddr = *schedulerAddr 39 | } 40 | if *storeAddr != "" { 41 | conf.StoreAddr = *storeAddr 42 | } 43 | if *dbPath != "" { 44 | conf.DBPath = *dbPath 45 | } 46 | if *logLevel != "" { 47 | conf.LogLevel = *logLevel 48 | } 49 | 50 | log.SetLevelByString(conf.LogLevel) 51 | log.SetFlags(log.Ldate | log.Ltime | log.Lmicroseconds | log.Lshortfile) 52 | log.Infof("Server started with conf %+v", conf) 53 | 54 | var storage storage.Storage 55 | if conf.Raft { 56 | storage = raft_storage.NewRaftStorage(conf) 57 | } else { 58 | storage = standalone_storage.NewStandAloneStorage(conf) 59 | } 60 | 61 | if err := storage.Start(); err != nil { 62 | log.Fatal(err) 63 | } 64 | server := server.NewServer(storage) 65 | 66 | var alivePolicy = keepalive.EnforcementPolicy{ 67 | MinTime: 2 * time.Second, 68 | PermitWithoutStream: true, 69 | } 70 | 71 | grpcServer := grpc.NewServer( 72 | grpc.KeepaliveEnforcementPolicy(alivePolicy), 73 | grpc.InitialWindowSize(1<<30), 74 | grpc.InitialConnWindowSize(1<<30), 75 | grpc.MaxRecvMsgSize(10 * 1024 * 1024), 76 | ) 77 | linkkvpb.RegisterLinkKvServer(grpcServer, server) 78 | listenAddr := conf.StoreAddr[strings.IndexByte(conf.StoreAddr,':'):] 79 | l, err := net.Listen("tcp", listenAddr) 80 | if err != nil { 81 | log.Fatal(err) 82 | } 83 | handleSignal(grpcServer) 84 | 85 | err = grpcServer.Serve(l) 86 | if err != nil { 87 | log.Fatal(err) 88 | } 89 | log.Info("Server stopped.") 90 | 91 | } 92 | 93 | func handleSignal(grpcServer *grpc.Server) { 94 | sigCh := make(chan os.Signal, 1) 95 | signal.Notify(sigCh, 96 | syscall.SIGHUP, 97 | syscall.SIGINT, 98 | syscall.SIGTERM, 99 | syscall.SIGQUIT) 100 | go func() { 101 | sig := <- sigCh 102 | log.Infof("Got signal [%s] to exit.", sig) 103 | grpcServer.Stop() 104 | }() 105 | } -------------------------------------------------------------------------------- /proto/proto/raft_serverpb.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package raft_serverpb; 3 | 4 | import "eraftpb.proto"; 5 | import "metapb.proto"; 6 | 7 | // The message sent between Raft peer, it wraps the raft meessage with some meta information. 8 | message RaftMessage { 9 | uint64 region_id = 1; 10 | metapb.Peer from_peer = 2; 11 | metapb.Peer to_peer = 3; 12 | eraftpb.Message message = 4; 13 | metapb.RegionEpoch region_epoch = 5; 14 | // true means to_peer is a tombstone peer and it should remove itself. 15 | bool is_tombstone = 6; 16 | // Region key range [start_key, end_key). (Used in 3B) 17 | bytes start_key = 7; 18 | bytes end_key = 8; 19 | } 20 | 21 | // Used to store the persistent state for Raft, including the hard state for raft and the last index of the raft log. 22 | message RaftLocalState { 23 | eraftpb.HardState hard_state = 1; 24 | uint64 last_index = 2; 25 | uint64 last_term = 3; 26 | } 27 | 28 | // Used to store the persistent state for Raft state machine. 29 | message RaftApplyState { 30 | // Record the applied index of the state machine to make sure 31 | // not apply any index twice after restart. 32 | uint64 applied_index = 1; 33 | // Record the index and term of the last raft log that have been truncated. (Used in 2C) 34 | RaftTruncatedState truncated_state = 2; 35 | } 36 | 37 | // The truncated state for Raft log compaction. 38 | message RaftTruncatedState { 39 | uint64 index = 1; 40 | uint64 term = 2; 41 | } 42 | 43 | // Used to store Region information and the corresponding Peer state on this Store. 44 | message RegionLocalState { 45 | PeerState state = 1; 46 | metapb.Region region = 2; 47 | } 48 | 49 | // Normal indicates that this Peer is normal; 50 | // Tombstone shows that this Peer has been removed from Region and cannot join in Raft Group. 51 | enum PeerState { 52 | Normal = 0; 53 | Tombstone = 2; 54 | } 55 | 56 | // The persistent identification for Store. 57 | // It used to recover the store id after restart. 58 | message StoreIdent { 59 | uint64 cluster_id = 1; 60 | uint64 store_id = 2; 61 | } 62 | 63 | // Snapshot sending and reciveing related messages. 64 | // Not included in the course scope. 65 | message KeyValue { 66 | bytes key = 1; 67 | bytes value = 2; 68 | } 69 | 70 | message RaftSnapshotData { 71 | metapb.Region region = 1; 72 | uint64 file_size = 2; 73 | repeated KeyValue data = 3; 74 | SnapshotMeta meta = 5; 75 | } 76 | 77 | message SnapshotCFFile { 78 | string cf = 1; 79 | uint64 size = 2; 80 | uint32 checksum = 3; 81 | } 82 | 83 | message SnapshotMeta { 84 | repeated SnapshotCFFile cf_files = 1; 85 | } 86 | 87 | message SnapshotChunk { 88 | RaftMessage message = 1; 89 | bytes data = 2; 90 | } 91 | 92 | message Done {} 93 | 94 | -------------------------------------------------------------------------------- /kv/transaction/mvcc/lock.go: -------------------------------------------------------------------------------- 1 | package mvcc 2 | 3 | import ( 4 | "bytes" 5 | "encoding/binary" 6 | "fmt" 7 | "reflect" 8 | 9 | "miniLinkDB/kv/util/engine_util" 10 | "miniLinkDB/proto/pkg/kvrpcpb" 11 | ) 12 | 13 | const TsMax uint64 = ^uint64(0) 14 | 15 | type Lock struct { 16 | Primary []byte 17 | Ts uint64 18 | Ttl uint64 19 | Kind WriteKind 20 | } 21 | 22 | type KlPair struct { 23 | Key []byte 24 | Lock *Lock 25 | } 26 | 27 | func (lock *Lock) Info(key []byte) *kvrpcpb.LockInfo { 28 | info := kvrpcpb.LockInfo{} 29 | info.Key = key 30 | info.LockVersion = lock.Ts 31 | info.PrimaryLock = lock.Primary 32 | info.LockTtl = lock.Ttl 33 | return &info 34 | } 35 | 36 | func (lock *Lock) ToBytes() []byte { 37 | buf := append(lock.Primary, byte(lock.Kind)) 38 | buf = append(buf, make([]byte, 16)...) 39 | binary.BigEndian.PutUint64(buf[len(lock.Primary)+1:], lock.Ts) 40 | binary.BigEndian.PutUint64(buf[len(lock.Primary)+9:], lock.Ttl) 41 | return buf 42 | } 43 | 44 | // ParseLock attempts to parse a byte string into a Lock object. 45 | func ParseLock(input []byte) (*Lock, error) { 46 | if len(input) <= 16 { 47 | return nil, fmt.Errorf("mvcc: error parsing lock, not enough input, found %d bytes", len(input)) 48 | } 49 | 50 | primaryLen := len(input) - 17 51 | primary := input[:primaryLen] 52 | kind := WriteKind(input[primaryLen]) 53 | ts := binary.BigEndian.Uint64(input[primaryLen +1:]) 54 | ttl := binary.BigEndian.Uint64(input[primaryLen +9:]) 55 | 56 | return &Lock{Primary: primary, Ts :ts, Ttl: ttl, Kind: kind}, nil 57 | } 58 | 59 | // IsLockedFor checks if lock locks key at txnStartTs. 60 | func (lock *Lock) IsLockedFor(key []byte, txnStartTs uint64, resp interface{}) bool { 61 | if lock == nil { 62 | return false 63 | } 64 | if txnStartTs == TsMax && bytes.Compare(key, lock.Primary) != 0 { 65 | return false 66 | } 67 | 68 | if lock.Ts <= txnStartTs { 69 | err := &kvrpcpb.KeyError{Locked: lock.Info(key) } 70 | respValue := reflect.ValueOf(resp) 71 | reflect.Indirect(respValue).FieldByName("Error").Set(reflect.ValueOf(err)) 72 | return true 73 | } 74 | return false 75 | } 76 | 77 | 78 | // AllLocksForTxn returns all locks for the current transaction. 79 | func AllLocksForTxn(txn *MvccTxn) ([]KlPair, error) { 80 | var result []KlPair 81 | iter := txn.Reader.IterCF(engine_util.CfLock) 82 | defer iter.Close() 83 | 84 | for ; iter.Valid(); iter.Next() { 85 | item := iter.Item() 86 | val, err := item.Value() 87 | if err != nil { 88 | return nil, err 89 | } 90 | lock, err := ParseLock(val) 91 | if err != nil { 92 | return nil, err 93 | } 94 | if lock.Ts == txn.StartTS { 95 | result = append(result, KlPair{item.Key(), lock}) 96 | } 97 | } 98 | return result, nil 99 | } 100 | -------------------------------------------------------------------------------- /kv/raftstore/router.go: -------------------------------------------------------------------------------- 1 | package raftstore 2 | 3 | import ( 4 | "miniLinkDB/errors" 5 | 6 | "miniLinkDB/kv/raftstore/message" 7 | "miniLinkDB/proto/pkg/raft_cmdpb" 8 | "miniLinkDB/proto/pkg/raft_serverpb" 9 | "sync" 10 | "sync/atomic" 11 | ) 12 | // peerState contains the peer states that needs to run raft command and apply command. 13 | type peerState struct { 14 | closed uint32 15 | peer *peer 16 | } 17 | 18 | // router routes a message to a peer. 19 | type router struct { 20 | peers sync.Map // regionID -> peerState 21 | peerSender chan message.Msg 22 | storeSender chan <- message.Msg 23 | } 24 | 25 | func newRouter(storeSender chan<- message.Msg) *router { 26 | pm := &router{ 27 | peerSender: make(chan message.Msg, 40960), 28 | storeSender: storeSender, 29 | } 30 | return pm 31 | } 32 | 33 | func (pr *router) get(regionID uint64) *peerState { 34 | v, ok := pr.peers.Load(regionID) 35 | if ok { 36 | return v.(*peerState) 37 | } 38 | return nil 39 | } 40 | 41 | func (pr *router) register(peer *peer) { 42 | id := peer.regionId 43 | newPeer := &peerState { 44 | peer: peer, 45 | } 46 | pr.peers.Store(id, newPeer) 47 | } 48 | 49 | func (pr *router) close(regionID uint64) { 50 | v, ok := pr.peers.Load(regionID) 51 | if ok { 52 | ps := v.(*peerState) 53 | atomic.StoreUint32(&ps.closed, 1) 54 | pr.peers.Delete(regionID) 55 | } 56 | } 57 | 58 | func (pr *router) send(regionID uint64, msg message.Msg) error { 59 | msg.RegionID = regionID 60 | p := pr.get(regionID) 61 | if p== nil || atomic.LoadUint32(&p.closed) ==1 { 62 | return errPeerNotFound 63 | } 64 | pr.peerSender <- msg 65 | return nil 66 | } 67 | 68 | func (pr *router) sendStore(msg message.Msg) { 69 | pr.storeSender <- msg 70 | } 71 | 72 | var errPeerNotFound = errors.New("peer not found") 73 | 74 | type RaftstoreRouter struct { 75 | router *router 76 | } 77 | 78 | func NewRaftstoreRouter(router *router) *RaftstoreRouter { 79 | return &RaftstoreRouter{router: router} 80 | } 81 | 82 | func (r *RaftstoreRouter) Send(regionID uint64, msg message.Msg) error { 83 | return r.router.send(regionID, msg) 84 | } 85 | 86 | func (r *RaftstoreRouter) SendRaftMessage(msg *raft_serverpb.RaftMessage) error { 87 | regionID := msg.RegionId 88 | if r.router.send(regionID, message.NewPeerMsg(message.MsgTypeRaftMessage, regionID, msg)) != nil { 89 | r.router.sendStore(message.NewPeerMsg(message.MsgTypeStoreRaftMessage, regionID, msg)) 90 | } 91 | return nil 92 | } 93 | 94 | func (r *RaftstoreRouter) SendRaftCommand(req *raft_cmdpb.RaftCmdRequest, cb *message.Callback) error { 95 | cmd := &message.MsgRaftCmd{ 96 | Request: req, 97 | Callback: cb, 98 | } 99 | regionID := req.Header.RegionId 100 | return r.router.send(regionID, message.NewPeerMsg(message.MsgTypeRaftCmd,regionID, cmd)) 101 | } -------------------------------------------------------------------------------- /kv/transaction/latches/latches.go: -------------------------------------------------------------------------------- 1 | package latches 2 | 3 | 4 | import ( 5 | "miniLinkDB/kv/transaction/mvcc" 6 | "sync" 7 | ) 8 | 9 | // Latching provides atomicity of LinkKV commands. This should not be confused with SQL transactions which provide atomicity 10 | // for multiple LinkKV commands. For example, consider two commit commands, these write to multiple keys/CFs so if they race, 11 | // then it is possible for inconsistent data to be written. By latching the keys each command might write, we ensure that the 12 | // two commands will not race to write the same keys. 13 | // 14 | // A latch is a per-key lock. There is only one latch per user key, not one per CF or one for each encoded key. Latches are 15 | // only needed for writing. Only one thread can hold a latch at a time and all keys that a command might write must be locked 16 | // at once. 17 | 18 | // Latching is implemented using a single map which maps keys to a Go WaitGroup. Access to this map is guarded by a mutex 19 | // to ensure that latching is atomic and consistent. Since the mutex is a global lock, it would cause intolerable contention 20 | // in a real system. 21 | 22 | type Latches struct { 23 | 24 | latchMap map[string]*sync.WaitGroup 25 | 26 | latchGuard sync.Mutex 27 | 28 | Validation func(txn *mvcc.MvccTxn, keys [][]byte) 29 | } 30 | 31 | func NewLatches() *Latches { 32 | l := new(Latches) 33 | l.latchMap = make(map[string]*sync.WaitGroup) 34 | return l 35 | } 36 | 37 | func (l *Latches) AcquireLatches(keysToLatch [][]byte) *sync.WaitGroup { 38 | l.latchGuard.Lock() 39 | defer l.latchGuard.Unlock() 40 | 41 | for _, key := range keysToLatch { 42 | if latchWg, ok := l.latchMap[string(key)]; ok { 43 | return latchWg 44 | } 45 | } 46 | 47 | // All Latches are available, lock them all with a new wait group. 48 | wg := new(sync.WaitGroup) 49 | wg.Add(1) 50 | for _, key := range keysToLatch { 51 | l.latchMap[string(key)] = wg 52 | } 53 | return nil 54 | } 55 | 56 | func (l *Latches) ReleaseLatches(keysToUnlatch [][]byte) { 57 | l.latchGuard.Lock() 58 | defer l.latchGuard.Unlock() 59 | 60 | first := true 61 | for _, key := range keysToUnlatch { 62 | if first { 63 | wg := l.latchMap[string(key)] 64 | wg.Done() 65 | first = false 66 | } 67 | delete(l.latchMap, string(key)) 68 | } 69 | } 70 | 71 | // WaitForLatches attempts to lock all keys in keysToLatch using AcquireLatches. If a latch ia already locked, then = 72 | // WaitForLatches will wait for it to become unlocked then try again. Therefore WaitForLatches may block for an unbounded 73 | // length of time. 74 | func (l *Latches) WaitForLatches(keysToLatch [][]byte) { 75 | for { 76 | wg := l.AcquireLatches(keysToLatch) 77 | if wg == nil { 78 | return 79 | } 80 | wg.Wait() 81 | } 82 | } 83 | 84 | // Validate calls the function in Validation, if it exists. 85 | func (l *Latches) Validate(txn *mvcc.MvccTxn, latched [][]byte) { 86 | if l.Validation != nil { 87 | l.Validation(txn, latched) 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /kv/util/engine_util/util.go: -------------------------------------------------------------------------------- 1 | package engine_util 2 | 3 | import ( 4 | "bytes" 5 | "github.com/Connor1996/badger" 6 | "github.com/golang/protobuf/proto" 7 | ) 8 | 9 | func KeyWithCF(cf string, key []byte) []byte { 10 | return append([]byte(cf+"_"), key...) 11 | } 12 | 13 | func GetCF(db *badger.DB, cf string, key []byte) (val []byte, err error) { 14 | err = db.View(func(txn *badger.Txn) error { 15 | val, err = GetCFFromTxn(txn, cf, key) 16 | return err 17 | }) 18 | return 19 | } 20 | 21 | func GetCFFromTxn(txn *badger.Txn, cf string, key []byte) (val []byte, err error) { 22 | item, err := txn.Get(KeyWithCF(cf, key)) 23 | if err != nil { 24 | return nil, err 25 | } 26 | val, err = item.ValueCopy(val) 27 | return 28 | } 29 | 30 | func PutCF(engine *badger.DB, cf string, key []byte, val []byte) error { 31 | return engine.Update(func(txn *badger.Txn) error { 32 | return txn.Set(KeyWithCF(cf, key), val) 33 | }) 34 | } 35 | 36 | func GetMeta(engine *badger.DB, key []byte, msg proto.Message) error { 37 | var val []byte 38 | err := engine.View(func(txn *badger.Txn) error { 39 | item, err := txn.Get(key) 40 | if err != nil { 41 | return err 42 | } 43 | val, err = item.Value() 44 | return err 45 | }) 46 | if err != nil { 47 | return err 48 | } 49 | return proto.Unmarshal(val, msg) 50 | } 51 | 52 | func GetMetaFromTxn(txn *badger.Txn, key []byte, msg proto.Message) error { 53 | item, err := txn.Get(key) 54 | if err != nil { 55 | return err 56 | } 57 | val, err := item.Value() 58 | if err != nil { 59 | return err 60 | } 61 | return proto.Unmarshal(val, msg) 62 | } 63 | 64 | func PutMeta(engine *badger.DB, key []byte, msg proto.Message) error { 65 | val, err := proto.Marshal(msg) 66 | if err != nil { 67 | return err 68 | } 69 | return engine.Update(func(txn *badger.Txn) error { 70 | return txn.Set(key, val) 71 | }) 72 | } 73 | 74 | func DeleteCF(engine *badger.DB, cf string, key []byte) error { 75 | return engine.Update(func(txn *badger.Txn) error { 76 | return txn.Delete(KeyWithCF(cf, key)) 77 | }) 78 | } 79 | 80 | func DeleteRange(db *badger.DB, startKey, endKey []byte) error { 81 | batch := new(WriteBatch) 82 | txn := db.NewTransaction(false) 83 | defer txn.Discard() 84 | for _, cf := range CFs { 85 | deleteRangeCF(txn, batch, cf, startKey, endKey) 86 | } 87 | 88 | return batch.WriteToDB(db) 89 | } 90 | 91 | func deleteRangeCF(txn *badger.Txn, batch *WriteBatch, cf string, startKey, endKey []byte) { 92 | it := NewCFIterator(cf, txn) 93 | for it.Seek(startKey); it.Valid(); it.Next() { 94 | item := it.Item() 95 | key := item.KeyCopy(nil) 96 | if ExceedEndKey(key, endKey) { 97 | break 98 | } 99 | batch.DeleteCF(cf, key) 100 | } 101 | defer it.Close() 102 | } 103 | 104 | func ExceedEndKey(current, endKey []byte) bool { 105 | if len(endKey) == 0 { 106 | return false 107 | } 108 | return bytes.Compare(current, endKey) >= 0 109 | } 110 | -------------------------------------------------------------------------------- /raft/log.go: -------------------------------------------------------------------------------- 1 | // Copyright 2015 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package raft 16 | 17 | import pb "miniLinkDB/proto/pkg/eraftpb" 18 | 19 | // RaftLog manage the log entries, its struct look like: 20 | // 21 | // snapshot/first.....applied....committed....stabled.....last 22 | // --------|------------------------------------------------| 23 | // log entries 24 | // 25 | // for simplify the RaftLog implement should manage all log entries 26 | // that not truncated 27 | type RaftLog struct { 28 | // storage contains all stable entries since the last snapshot. 29 | storage Storage 30 | 31 | // committed is the highest log position that is known to be in 32 | // stable storage on a quorum of nodes. 33 | committed uint64 34 | 35 | // applied is the highest log position that the application has 36 | // been instructed to apply to its state machine. 37 | // Invariant: applied <= committed 38 | applied uint64 39 | 40 | // log entries with index <= stabled are persisted to storage. 41 | // It is used to record the logs that are not persisted by storage yet. 42 | // Everytime handling `Ready`, the unstabled logs will be included. 43 | stabled uint64 44 | 45 | // all entries that have not yet compact. 46 | entries []pb.Entry 47 | 48 | // the incoming unstable snapshot, if any. 49 | // (Used in 2C) 50 | pendingSnapshot *pb.Snapshot 51 | 52 | // Your Data Here (2A). 53 | } 54 | 55 | // newLog returns log using the given storage. It recovers the log 56 | // to the state that it just commits and applies the latest snapshot. 57 | func newLog(storage Storage) *RaftLog { 58 | 59 | return nil 60 | } 61 | 62 | // We need to compact the log entries in some point of time like 63 | // storage compact stabled log entries prevent the log entries 64 | // grow unlimitedly in memory 65 | func (l *RaftLog) maybeCompact() { 66 | 67 | } 68 | 69 | // unstableEntries return all the unstable entries 70 | func (l *RaftLog) unstableEntries() []pb.Entry { 71 | 72 | return nil 73 | } 74 | 75 | // nextEnts returns all the committed but not applied entries 76 | func (l *RaftLog) nextEnts() (ents []pb.Entry) { 77 | 78 | return nil 79 | } 80 | 81 | // LastIndex return the last index of the log entries 82 | func (l *RaftLog) LastIndex() uint64 { 83 | 84 | return 0 85 | } 86 | 87 | // Term return the term of the entry in the given index 88 | func (l *RaftLog) Term(i uint64) (uint64, error) { 89 | 90 | return 0, nil 91 | } 92 | -------------------------------------------------------------------------------- /kv/util/engine_util/cf_iterator.go: -------------------------------------------------------------------------------- 1 | package engine_util 2 | 3 | import "github.com/Connor1996/badger" 4 | 5 | type CFItem struct { 6 | item *badger.Item 7 | prefixLen int 8 | } 9 | 10 | func (i *CFItem) String() string { 11 | return i.item.String() 12 | } 13 | 14 | func (i *CFItem) Key() []byte { 15 | return i.item.Key()[i.prefixLen:] 16 | } 17 | 18 | func (i *CFItem) KeyCopy(dst []byte) []byte { 19 | return i.item.KeyCopy(dst)[i.prefixLen:] 20 | } 21 | 22 | func (i *CFItem) Version() uint64 { 23 | return i.item.Version() 24 | } 25 | 26 | func (i *CFItem) IsEmpty() bool { 27 | return i.item.IsEmpty() 28 | } 29 | 30 | func (i *CFItem) Value() ([]byte, error) { 31 | return i.item.Value() 32 | } 33 | 34 | func (i *CFItem) ValueSize() int { 35 | return i.item.ValueSize() 36 | } 37 | 38 | func (i *CFItem) ValueCopy(dst []byte) ([]byte, error) { 39 | return i.item.ValueCopy(dst) 40 | } 41 | 42 | func (i *CFItem) IsDeleted() bool { 43 | return i.item.IsDeleted() 44 | } 45 | 46 | func (i *CFItem) EstimatedSize() int64 { 47 | return i.item.EstimatedSize() 48 | } 49 | 50 | func (i *CFItem) UserMeta() []byte { 51 | return i.item.UserMeta() 52 | } 53 | 54 | type BadgerIterator struct { 55 | iter *badger.Iterator 56 | prefix string 57 | } 58 | 59 | func NewCFIterator(cf string, txn *badger.Txn) *BadgerIterator { 60 | return &BadgerIterator{ 61 | iter: txn.NewIterator(badger.DefaultIteratorOptions), 62 | prefix: cf + "_", 63 | } 64 | } 65 | 66 | func (it *BadgerIterator) Item() DBItem { 67 | return &CFItem{ 68 | item: it.iter.Item(), 69 | prefixLen: len(it.prefix), 70 | } 71 | } 72 | 73 | func (it *BadgerIterator) Valid() bool { 74 | return it.iter.ValidForPrefix([]byte(it.prefix)) 75 | } 76 | 77 | func (it *BadgerIterator) ValidForPrefix(prefix []byte) bool { 78 | return it.iter.ValidForPrefix(append(prefix, []byte(it.prefix)...)) 79 | } 80 | 81 | func (it *BadgerIterator) Close() { 82 | it.iter.Close() 83 | } 84 | 85 | func (it *BadgerIterator) Next() { 86 | it.iter.Next() 87 | } 88 | 89 | func (it *BadgerIterator) Seek(key []byte) { 90 | it.iter.Seek(append([]byte(it.prefix), key...)) 91 | } 92 | 93 | func (it *BadgerIterator) Rewind() { 94 | it.iter.Rewind() 95 | } 96 | 97 | type DBIterator interface { 98 | // Item返回当前键值对的指针 99 | Item() DBItem 100 | // 迭代完成后 有效返回false 101 | Valid() bool 102 | // 将迭代器前进一个,始终在Next之后检查it.Valid,以确保您可以访问有效的it.Item() 103 | Next() 104 | // Seek将寻求提供的密钥(如果存在)。如果不存在,它将寻求大于提供的下一个最小密钥。 105 | Seek([]byte) 106 | // Close the iterator 107 | Close() 108 | } 109 | 110 | type DBItem interface { 111 | // Key returns the key. 112 | Key() []byte 113 | // KeyCopy返回该项目的密钥的副本,并将其写入dst slice。如果通过nil或dst的容量不足,则将分配并返回一个新的片。 114 | KeyCopy(dst []byte) []byte 115 | // Value retrieves the value of the item. 116 | Value() ([]byte, error) 117 | // ValueSize返回value 的大小 118 | ValueSize() int 119 | // ValueCopy 从值日志中返回该项目的值的副本,并将其写入dst slice。如果通过nil或dst的容量不足,则将分配并返回一个新的片 120 | ValueCopy(dst []byte) ([]byte, error) 121 | } 122 | 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | 131 | 132 | 133 | -------------------------------------------------------------------------------- /kv/config/config.go: -------------------------------------------------------------------------------- 1 | package config 2 | 3 | import ( 4 | "fmt" 5 | "miniLinkDB/log" 6 | "time" 7 | ) 8 | 9 | type Config struct { 10 | StoreAddr string 11 | Raft bool 12 | SchedulerAddr string 13 | LogLevel string 14 | 15 | DBPath string // 用于存储数据的目录 16 | 17 | // raft_base_tick_interval是基本刻度间隔(毫秒) 18 | RaftBaseTickInterval time.Duration 19 | RaftHeartbeatTicks int 20 | RaftElectionTimeoutTicks int 21 | 22 | // gc不必要的筏日志的时间间隔(毫秒) 23 | RaftLogGCTickInterval time.Duration 24 | // 当条目计数超过此值时,将强制触发gc 25 | RaftLogGcCountLimit uint64 26 | // 检查区域是否需要分割的间隔(毫秒) 27 | SplitRegionCheckTickInterval time.Duration 28 | // 删除陈旧节点之前的延迟时间 29 | SchedulerHeartbeatTickInterval time.Duration 30 | SchedulerStoreHeartbeatTickInterval time.Duration 31 | 32 | // 当区域[a,e) 的大小满足regionMaxSize时,它将被分为几个区域[a,b],[b,c),[c,d),[d,e), 33 | // 并且[a,b), [b,c),[c,d) 的大小将为regionSplitSize(可能更大一些)。 34 | RegionMaxSize uint64 35 | RegionSplitSize uint64 36 | } 37 | 38 | func (c *Config) Validate() error { 39 | if c.RaftHeartbeatTicks == 0 { 40 | return fmt.Errorf("heart tick must greater than 0") 41 | } 42 | 43 | if c.RaftElectionTimeoutTicks != 10 { 44 | log.Warnf("Election timeout ticks needs to be same across all the cluster, otherwise it may lead to inconsistency.") 45 | } 46 | 47 | if c.RaftElectionTimeoutTicks <= c.RaftHeartbeatTicks { 48 | return fmt.Errorf("选举刻度必须大于心跳刻度"); 49 | } 50 | return nil 51 | } 52 | 53 | const ( 54 | KB uint64 = 1024 55 | MB uint64 = 2024 * 1024 56 | ) 57 | 58 | func NewDefaultConfig() *Config { 59 | return &Config{ 60 | SchedulerAddr: "127.0.0.1:2379", 61 | StoreAddr: "127.0.0.1:20160", 62 | LogLevel: "info", 63 | Raft: true, 64 | RaftBaseTickInterval: 1 * time.Second, 65 | RaftHeartbeatTicks: 2, 66 | RaftElectionTimeoutTicks: 10, 67 | RaftLogGCTickInterval: 10 * time.Second, 68 | // Assume the average size of entries is 1k. 69 | RaftLogGcCountLimit: 128000, 70 | SplitRegionCheckTickInterval: 10 * time.Second, 71 | SchedulerHeartbeatTickInterval: 100 * time.Millisecond, 72 | SchedulerStoreHeartbeatTickInterval: 10 * time.Second, 73 | RegionMaxSize: 144 * MB, 74 | RegionSplitSize: 96 * MB, 75 | DBPath: "/tmp/badger", 76 | } 77 | } 78 | 79 | func NewTestConfig() *Config { 80 | return &Config{ 81 | LogLevel: "info", 82 | Raft: true, 83 | RaftBaseTickInterval: 50 * time.Millisecond, 84 | RaftHeartbeatTicks: 2, 85 | RaftElectionTimeoutTicks: 10, 86 | RaftLogGCTickInterval: 50 * time.Millisecond, 87 | // Assume the average size of entries is 1k. 88 | RaftLogGcCountLimit: 128000, 89 | SplitRegionCheckTickInterval: 100 * time.Millisecond, 90 | SchedulerHeartbeatTickInterval: 100 * time.Millisecond, 91 | SchedulerStoreHeartbeatTickInterval: 500 * time.Millisecond, 92 | RegionMaxSize: 144 * MB, 93 | RegionSplitSize: 96 * MB, 94 | DBPath: "/tmp/badger", 95 | } 96 | } -------------------------------------------------------------------------------- /kv/coprocessor/topn.go: -------------------------------------------------------------------------------- 1 | package coprocessor 2 | 3 | import ( 4 | "container/heap" 5 | 6 | "github.com/juju/errors" 7 | "github.com/pingcap/tidb/sessionctx/stmtctx" 8 | "github.com/pingcap/tidb/types" 9 | tipb "github.com/pingcap/tipb/go-tipb" 10 | ) 11 | 12 | type sortRow struct { 13 | key []types.Datum 14 | data [][]byte 15 | } 16 | 17 | // topNSorter implements sort.Interface. When all rows have been processed, the topNSorter will sort the whole data in heap. 18 | type topNSorter struct { 19 | orderByItems []*tipb.ByItem 20 | rows []*sortRow 21 | err error 22 | sc *stmtctx.StatementContext 23 | } 24 | 25 | func (t *topNSorter) Len() int { 26 | return len(t.rows) 27 | } 28 | 29 | func (t *topNSorter) Swap(i, j int) { 30 | t.rows[i], t.rows[j] = t.rows[j], t.rows[i] 31 | } 32 | 33 | func (t *topNSorter) Less(i, j int) bool { 34 | for index, by := range t.orderByItems { 35 | v1 := t.rows[i].key[index] 36 | v2 := t.rows[j].key[index] 37 | 38 | ret, err := v1.CompareDatum(t.sc, &v2) 39 | if err != nil { 40 | t.err = errors.Trace(err) 41 | return true 42 | } 43 | 44 | if by.Desc { 45 | ret = -ret 46 | } 47 | 48 | if ret < 0 { 49 | return true 50 | } else if ret > 0 { 51 | return false 52 | } 53 | } 54 | 55 | return false 56 | } 57 | 58 | // topNHeap holds the top n elements using heap structure. It implements heap.Interface. 59 | // When we insert a row, topNHeap will check if the row can become one of the top n element or not. 60 | type topNHeap struct { 61 | topNSorter 62 | 63 | // totalCount is equal to the limit count, which means the max size of heap. 64 | totalCount int 65 | // heapSize means the current size of this heap. 66 | heapSize int 67 | } 68 | 69 | func (t *topNHeap) len() int { 70 | return t.heapSize 71 | } 72 | 73 | func (t *topNHeap) Push(x interface{}) { 74 | t.rows = append(t.rows, x.(*sortRow)) 75 | t.heapSize++ 76 | } 77 | 78 | func (t *topNHeap) Pop() interface{} { 79 | return nil 80 | } 81 | 82 | func (t *topNHeap) Less(i, j int) bool { 83 | for index, by := range t.orderByItems { 84 | v1 := t.rows[i].key[index] 85 | v2 := t.rows[j].key[index] 86 | 87 | ret, err := v1.CompareDatum(t.sc, &v2) 88 | if err != nil { 89 | t.err = errors.Trace(err) 90 | return true 91 | } 92 | 93 | if by.Desc { 94 | ret = -ret 95 | } 96 | 97 | if ret > 0 { 98 | return true 99 | } else if ret < 0 { 100 | return false 101 | } 102 | } 103 | 104 | return false 105 | } 106 | 107 | // tryToAddRow tries to add a row to heap. 108 | // When this row is not less than any rows in heap, it will never become the top n element. 109 | // Then this function returns false. 110 | func (t *topNHeap) tryToAddRow(row *sortRow) bool { 111 | success := false 112 | if t.heapSize == t.totalCount { 113 | t.rows = append(t.rows, row) 114 | // 当此行小于顶部元素时,它将替换它并调整堆结构 115 | if t.Less(0, t.heapSize) { 116 | t.Swap(0, t.heapSize) 117 | heap.Fix(t, 0) 118 | success = true 119 | } 120 | t.rows = t.rows[:t.heapSize] 121 | } else { 122 | heap.Push(t, row) 123 | success = true 124 | } 125 | return success 126 | } -------------------------------------------------------------------------------- /kv/storage/raft_storage/transport.go: -------------------------------------------------------------------------------- 1 | package raft_storage 2 | 3 | import ( 4 | 5 | "miniLinkDB/kv/raftstore/message" 6 | "miniLinkDB/kv/util/worker" 7 | "miniLinkDB/log" 8 | "miniLinkDB/proto/pkg/raft_serverpb" 9 | "sync" 10 | ) 11 | 12 | type ServerTransport struct { 13 | raftClient *RaftClient 14 | raftRouter message.RaftRouter 15 | resolverScheduler chan<- worker.Task 16 | snapScheduler chan <- worker.Task 17 | resolving sync.Map 18 | } 19 | 20 | func NewServerTransport(raftClient *RaftClient, snapScheduler chan<- worker.Task, raftRouter message.RaftRouter, resolverScheduler chan <- worker.Task) *ServerTransport { 21 | return &ServerTransport{ 22 | raftClient: raftClient, 23 | raftRouter: raftRouter, 24 | resolverScheduler: resolverScheduler, 25 | snapScheduler: snapScheduler, 26 | } 27 | } 28 | 29 | func (t *ServerTransport) Send(msg *raft_serverpb.RaftMessage) error { 30 | storeID := msg.GetToPeer().GetStoreId() 31 | t.SendStore(storeID, msg) 32 | return nil 33 | } 34 | 35 | 36 | func (t *ServerTransport) SendStore(storeID uint64, msg *raft_serverpb.RaftMessage) { 37 | addr := t.raftClient.GetAddr(storeID) 38 | if addr != "" { 39 | t.WriteData(storeID, addr, msg) 40 | return 41 | } 42 | if _, ok := t.resolving.Load(storeID); ok { 43 | log.Debugf("store address is being resolved, msg dropped. storeID: %v, msg: %s", storeID, msg) 44 | return 45 | } 46 | log.Debug("begin to resolve store address. storeID: %v", storeID) 47 | t.resolving.Store(storeID, struct{}{}) 48 | t.Resolve(storeID, msg) 49 | } 50 | 51 | 52 | func (t *ServerTransport) Resolve(storeID uint64, msg *raft_serverpb.RaftMessage) { 53 | callback := func(addr string, err error) { 54 | // clear resolving 55 | t.resolving.Delete(storeID) 56 | if err != nil { 57 | log.Errorf("resolve store address failed. storeID: %v, err: %v", storeID, err) 58 | return 59 | } 60 | t.raftClient.InsertAddr(storeID, addr) 61 | t.WriteData(storeID, addr, msg) 62 | t.raftClient.Flush() 63 | } 64 | t.resolverScheduler <- &resolveAddrTask{ 65 | storeID: storeID, 66 | callback: callback, 67 | } 68 | } 69 | 70 | func (t *ServerTransport) WriteData(storeID uint64, addr string, msg *raft_serverpb.RaftMessage) { 71 | if msg.GetMessage().GetSnapshot() != nil { 72 | t.SendSnapshotSock(addr, msg) 73 | return 74 | } 75 | if err := t.raftClient.Send(storeID, addr, msg); err != nil { 76 | log.Errorf("send raft msg err. err: %v", err) 77 | } 78 | } 79 | 80 | func (t *ServerTransport) SendSnapshotSock(addr string, msg *raft_serverpb.RaftMessage) { 81 | callback := func(err error) { 82 | regionID := msg.GetRegionId() 83 | toPeerID := msg.GetToPeer().GetId() 84 | toStoreID := msg.GetToPeer().GetStoreId() 85 | log.Debugf("send snapshot. toPeerID: %v, toStoreID: %v, regionID: %v, status: %v", toPeerID, toStoreID, regionID, err) 86 | } 87 | 88 | t.snapScheduler <- &sendSnapTask{ 89 | addr: addr, 90 | msg: msg, 91 | callback: callback, 92 | } 93 | } 94 | 95 | func (t *ServerTransport) Flush() { 96 | t.raftClient.Flush() 97 | } 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | -------------------------------------------------------------------------------- /kv/util/engine_util/engine_util_test.go: -------------------------------------------------------------------------------- 1 | package engine_util 2 | 3 | import ( 4 | "bytes" 5 | "github.com/Connor1996/badger" 6 | "github.com/stretchr/testify/require" 7 | "io/ioutil" 8 | "testing" 9 | ) 10 | 11 | func TestEngineUtil(t *testing.T) { 12 | dir, err := ioutil.TempDir("", "engine_util") 13 | opts := badger.DefaultOptions 14 | opts.Dir = dir 15 | opts.ValueDir = dir 16 | db, err := badger.Open(opts) 17 | require.Nil(t, err) 18 | 19 | batch := new(WriteBatch) 20 | batch.SetCF(CfDefault, []byte("a"), []byte("a1")) 21 | batch.SetCF(CfDefault, []byte("b"), []byte("b1")) 22 | batch.SetCF(CfDefault, []byte("c"), []byte("c1")) 23 | batch.SetCF(CfDefault, []byte("d"), []byte("d1")) 24 | batch.SetCF(CfWrite, []byte("a"), []byte("a2")) 25 | batch.SetCF(CfWrite, []byte("b"), []byte("b2")) 26 | batch.SetCF(CfWrite, []byte("d"), []byte("d2")) 27 | batch.SetCF(CfLock, []byte("a"), []byte("a3")) 28 | batch.SetCF(CfLock, []byte("c"), []byte("c3")) 29 | batch.SetCF(CfDefault, []byte("e"), []byte("e1")) 30 | batch.DeleteCF(CfDefault, []byte("e")) 31 | err = batch.WriteToDB(db) 32 | require.Nil(t, err) 33 | 34 | _, err = GetCF(db, CfDefault, []byte("e")) 35 | require.Equal(t, err, badger.ErrKeyNotFound) 36 | 37 | err = PutCF(db, CfDefault, []byte("e"), []byte("e2")) 38 | require.Nil(t, err) 39 | val, _ := GetCF(db, CfDefault, []byte("e")) 40 | require.Equal(t, val, []byte("e2")) 41 | err = DeleteCF(db, CfDefault, []byte("e")) 42 | require.Nil(t, err) 43 | _, err = GetCF(db, CfDefault, []byte("e")) 44 | require.Equal(t, err, badger.ErrKeyNotFound) 45 | 46 | txn := db.NewTransaction(false) 47 | defer txn.Discard() 48 | defaultIter := NewCFIterator(CfDefault, txn) 49 | defaultIter.Seek([]byte("a")) 50 | item := defaultIter.Item() 51 | require.True(t, bytes.Equal(item.Key(), []byte("a"))) 52 | val, _ = item.Value() 53 | require.True(t, bytes.Equal(val, []byte("a1"))) 54 | defaultIter.Next() 55 | item = defaultIter.Item() 56 | require.True(t, bytes.Equal(item.Key(), []byte("b"))) 57 | val, _ = item.Value() 58 | require.True(t, bytes.Equal(val, []byte("b1"))) 59 | defaultIter.Next() 60 | item = defaultIter.Item() 61 | require.True(t, bytes.Equal(item.Key(), []byte("c"))) 62 | val, _ = item.Value() 63 | require.True(t, bytes.Equal(val, []byte("c1"))) 64 | defaultIter.Next() 65 | item = defaultIter.Item() 66 | require.True(t, bytes.Equal(item.Key(), []byte("d"))) 67 | val, _ = item.Value() 68 | require.True(t, bytes.Equal(val, []byte("d1"))) 69 | defaultIter.Next() 70 | require.False(t, defaultIter.Valid()) 71 | defaultIter.Close() 72 | 73 | writeIter := NewCFIterator(CfWrite, txn) 74 | writeIter.Seek([]byte("b")) 75 | item = writeIter.Item() 76 | require.True(t, bytes.Equal(item.Key(), []byte("b"))) 77 | val, _ = item.Value() 78 | require.True(t, bytes.Equal(val, []byte("b2"))) 79 | writeIter.Next() 80 | item = writeIter.Item() 81 | require.True(t, bytes.Equal(item.Key(), []byte("d"))) 82 | val, _ = item.Value() 83 | require.True(t, bytes.Equal(val, []byte("d2"))) 84 | writeIter.Next() 85 | require.False(t, writeIter.Valid()) 86 | writeIter.Close() 87 | 88 | lockIter := NewCFIterator(CfLock, txn) 89 | lockIter.Seek([]byte("d")) 90 | require.False(t, lockIter.Valid()) 91 | lockIter.Close() 92 | } 93 | 94 | -------------------------------------------------------------------------------- /kv/storage/raft_storage/raft_client.go: -------------------------------------------------------------------------------- 1 | package raft_storage 2 | 3 | import "C" 4 | import ( 5 | "context" 6 | "google.golang.org/grpc" 7 | "google.golang.org/grpc/keepalive" 8 | "sync" 9 | "time" 10 | 11 | "miniLinkDB/kv/config" 12 | "miniLinkDB/log" 13 | "miniLinkDB/proto/pkg/linkkvpb" 14 | "miniLinkDB/proto/pkg/raft_serverpb" 15 | ) 16 | 17 | type raftConn struct { 18 | streamMu sync.Mutex 19 | stream linkkvpb.LinkKv_RaftClient 20 | ctx context.Context 21 | cancel context.CancelFunc 22 | } 23 | 24 | func newRaftConn(addr string, cfg *config.Config) (*raftConn, error) { 25 | cc, err := grpc.Dial(addr, grpc.WithInsecure(), 26 | grpc.WithInitialWindowSize(2*1024*1024), 27 | grpc.WithKeepaliveParams(keepalive.ClientParameters{ 28 | Time: 3 * time.Second, 29 | Timeout: 60 * time.Second, 30 | PermitWithoutStream: true, 31 | })) 32 | if err != nil { 33 | return nil, err 34 | } 35 | ctx, cancel := context.WithCancel(context.Background()) 36 | stream, err := linkkvpb.NewLinkKvClient(cc).Raft(ctx) 37 | if err != nil { 38 | cancel() 39 | return nil, err 40 | } 41 | return &raftConn{ 42 | stream: stream, 43 | ctx: ctx, 44 | cancel: cancel, 45 | }, nil 46 | } 47 | 48 | func (c *raftConn) Stop() { 49 | c.cancel() 50 | } 51 | 52 | func (c *raftConn) Send(msg *raft_serverpb.RaftMessage) error { 53 | c.streamMu.Lock() 54 | defer c.streamMu.Unlock() 55 | return c.stream.Send(msg) 56 | } 57 | 58 | type RaftClient struct { 59 | config *config.Config 60 | sync.RWMutex 61 | conns map[string]*raftConn 62 | addrs map[uint64]string 63 | } 64 | 65 | func newRaftClient(config *config.Config) *RaftClient { 66 | return &RaftClient{ 67 | config: config, 68 | conns: make(map[string]*raftConn), 69 | addrs: make(map[uint64]string), 70 | } 71 | } 72 | 73 | func (c *RaftClient) getConn(addr string, regionId uint64) (*raftConn, error) { 74 | c.RLock() 75 | conn, ok := c.conns[addr] 76 | if ok { 77 | c.RUnlock() 78 | return conn, nil 79 | } 80 | c.RUnlock() 81 | newConn, err := newRaftConn(addr, c.config) 82 | if err != nil { 83 | return nil, err 84 | } 85 | c.Lock() 86 | defer c.Unlock() 87 | if conn, ok := c.conns[addr]; ok { 88 | newConn.Stop() 89 | return conn, nil 90 | } 91 | c.conns[addr] = newConn 92 | return newConn, nil 93 | } 94 | 95 | func (c *RaftClient) Send(storeID uint64, addr string, msg *raft_serverpb.RaftMessage) error { 96 | conn, err := c.getConn(addr, msg.GetRegionId()) 97 | if err != nil { 98 | return err 99 | } 100 | err = conn.Send(msg) 101 | if err == nil { 102 | return nil 103 | } 104 | 105 | log.Error("raft client failed to send") 106 | c.Lock() 107 | defer c.Unlock() 108 | conn.Stop() 109 | delete(c.conns, addr) 110 | if oldAddr, ok := c.addrs[storeID]; ok && oldAddr == addr { 111 | delete(c.addrs, storeID) 112 | } 113 | return err 114 | } 115 | 116 | func (c *RaftClient) GetAddr(storeID uint64) string { 117 | c.RLock() 118 | defer c.RUnlock() 119 | v, _ := c.addrs[storeID] 120 | return v 121 | } 122 | 123 | func (c *RaftClient) InsertAddr(storeID uint64, addr string) { 124 | c.Lock() 125 | defer c.Unlock() 126 | c.addrs[storeID] = addr 127 | } 128 | 129 | func (c *RaftClient) Flush() { 130 | // Not support BufferHint 131 | } -------------------------------------------------------------------------------- /kv/raftstore/meta/values.go: -------------------------------------------------------------------------------- 1 | package meta 2 | 3 | import ( 4 | "github.com/Connor1996/badger" 5 | "miniLinkDB/kv/util/engine_util" 6 | "miniLinkDB/proto/pkg/eraftpb" 7 | "miniLinkDB/proto/pkg/metapb" 8 | rspb "miniLinkDB/proto/pkg/raft_serverpb" 9 | ) 10 | 11 | func GetRegionLocalState(db *badger.DB, regionId uint64) (*rspb.RegionLocalState, error) { 12 | regionLocalState := new(rspb.RegionLocalState) 13 | if err := engine_util.GetMeta(db, RegionStateKey(regionId), regionLocalState); err != nil { 14 | return regionLocalState, err 15 | } 16 | return regionLocalState, nil 17 | } 18 | 19 | func GetRaftLocalState(db *badger.DB, regionId uint64) (*rspb.RaftLocalState, error) { 20 | raftLocalState := new(rspb.RaftLocalState) 21 | if err := engine_util.GetMeta(db, RaftStateKey(regionId), raftLocalState); err != nil { 22 | return raftLocalState, err 23 | } 24 | return raftLocalState, nil 25 | } 26 | 27 | func GetApplyState(db *badger.DB, regionId uint64) (*rspb.RaftApplyState, error) { 28 | applyState := new(rspb.RaftApplyState) 29 | if err := engine_util.GetMeta(db, ApplyStateKey(regionId), applyState); err != nil { 30 | return nil, err 31 | } 32 | return applyState, nil 33 | } 34 | 35 | func GetRaftEntry(db *badger.DB, regionId, idx uint64) (*eraftpb.Entry, error) { 36 | entry := new(eraftpb.Entry) 37 | if err := engine_util.GetMeta(db, RaftLogKey(regionId, idx), entry); err != nil { 38 | return nil, err 39 | } 40 | return entry, nil 41 | } 42 | 43 | const ( 44 | // When we create a region peer, 应该初始化其log termindex> 0 45 | // so that we can force the follower peer to sync the snapshot first 46 | RaftInitLogTerm = 5 47 | RaftInitLogIndex = 5 48 | ) 49 | 50 | func InitRaftLocalState(raftEngine *badger.DB, region *metapb.Region) (*rspb.RaftLocalState, error) { 51 | raftState, err := GetRaftLocalState(raftEngine, region.Id) 52 | if err != nil && err != badger.ErrKeyNotFound { 53 | return nil, err 54 | } 55 | if err == badger.ErrKeyNotFound { 56 | raftState = new(rspb.RaftLocalState) 57 | raftState.HardState = new(eraftpb.HardState) 58 | if len(region.Peers)>0 { 59 | // new split region 60 | raftState.LastIndex = RaftInitLogIndex 61 | raftState.LastTerm = RaftInitLogTerm 62 | raftState.HardState.Term = RaftInitLogTerm 63 | raftState.HardState.Commit = RaftInitLogIndex 64 | err = engine_util.PutMeta(raftEngine, RaftStateKey(region.Id), raftState) 65 | if err != nil { 66 | return raftState, err 67 | } 68 | } 69 | } 70 | return raftState, nil 71 | } 72 | 73 | func InitApplyState(kvEngine *badger.DB, region *metapb.Region) (*rspb.RaftApplyState, error) { 74 | applyState, err := GetApplyState(kvEngine, region.Id) 75 | if err != nil && err != badger.ErrKeyNotFound { 76 | return nil, err 77 | } 78 | if err == badger.ErrKeyNotFound { 79 | applyState = new(rspb.RaftApplyState) 80 | applyState.TruncatedState = new(rspb.RaftTruncatedState) 81 | if len(region.Peers) >0 { 82 | applyState.AppliedIndex = RaftInitLogIndex 83 | applyState.TruncatedState.Index = RaftInitLogIndex 84 | applyState.TruncatedState.Term = RaftInitLogTerm 85 | } 86 | err = engine_util.PutMeta(kvEngine, ApplyStateKey(region.Id), applyState) 87 | if err != nil { 88 | return applyState, err 89 | } 90 | } 91 | return applyState, nil 92 | } 93 | 94 | func WriteRegionState(kvWB *engine_util.WriteBatch, region *metapb.Region, state rspb.PeerState) { 95 | regionState := new(rspb.RegionLocalState) 96 | regionState.State = state 97 | regionState.Region = region 98 | kvWB.SetMeta(RegionStateKey(region.Id), regionState) 99 | } -------------------------------------------------------------------------------- /raft/util.go: -------------------------------------------------------------------------------- 1 | // Copyright 2015 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package raft 16 | 17 | import ( 18 | "fmt" 19 | "io" 20 | "io/ioutil" 21 | "os" 22 | "os/exec" 23 | "sort" 24 | "strings" 25 | 26 | pb "miniLinkDB/proto/pkg/eraftpb" 27 | ) 28 | 29 | func min(a, b uint64) uint64 { 30 | if a > b { 31 | return b 32 | } 33 | return a 34 | } 35 | 36 | func max(a, b uint64) uint64 { 37 | if a > b { 38 | return a 39 | } 40 | return b 41 | } 42 | 43 | // IsEmptyHardState returns true if the given HardState is empty. 44 | func IsEmptyHardState(st pb.HardState) bool { 45 | return isHardStateEqual(st, pb.HardState{}) 46 | } 47 | 48 | // IsEmptySnap returns true if the given Snapshot is empty. 49 | func IsEmptySnap(sp *pb.Snapshot) bool { 50 | if sp == nil || sp.Metadata == nil { 51 | return true 52 | } 53 | return sp.Metadata.Index == 0 54 | } 55 | 56 | func mustTerm(term uint64, err error) uint64 { 57 | if err != nil { 58 | panic(err) 59 | } 60 | return term 61 | } 62 | 63 | func nodes(r *Raft) []uint64 { 64 | nodes := make([]uint64, 0, len(r.Prs)) 65 | for id := range r.Prs { 66 | nodes = append(nodes, id) 67 | } 68 | sort.Sort(uint64Slice(nodes)) 69 | return nodes 70 | } 71 | 72 | func diffu(a, b string) string { 73 | if a == b { 74 | return "" 75 | } 76 | aname, bname := mustTemp("base", a), mustTemp("other", b) 77 | defer os.Remove(aname) 78 | defer os.Remove(bname) 79 | cmd := exec.Command("diff", "-u", aname, bname) 80 | buf, err := cmd.CombinedOutput() 81 | if err != nil { 82 | if _, ok := err.(*exec.ExitError); ok { 83 | // do nothing 84 | return string(buf) 85 | } 86 | panic(err) 87 | } 88 | return string(buf) 89 | } 90 | 91 | func mustTemp(pre, body string) string { 92 | f, err := ioutil.TempFile("", pre) 93 | if err != nil { 94 | panic(err) 95 | } 96 | _, err = io.Copy(f, strings.NewReader(body)) 97 | if err != nil { 98 | panic(err) 99 | } 100 | f.Close() 101 | return f.Name() 102 | } 103 | 104 | func ltoa(l *RaftLog) string { 105 | s := fmt.Sprintf("committed: %d\n", l.committed) 106 | s += fmt.Sprintf("applied: %d\n", l.applied) 107 | for i, e := range l.entries { 108 | s += fmt.Sprintf("#%d: %+v\n", i, e) 109 | } 110 | return s 111 | } 112 | 113 | type uint64Slice []uint64 114 | 115 | func (p uint64Slice) Len() int { return len(p) } 116 | func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] } 117 | func (p uint64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } 118 | 119 | func IsLocalMsg(msgt pb.MessageType) bool { 120 | return msgt == pb.MessageType_MsgHup || msgt == pb.MessageType_MsgBeat 121 | } 122 | 123 | func IsResponseMsg(msgt pb.MessageType) bool { 124 | return msgt == pb.MessageType_MsgAppendResponse || msgt == pb.MessageType_MsgRequestVoteResponse || msgt == pb.MessageType_MsgHeartbeatResponse 125 | } 126 | 127 | func isHardStateEqual(a, b pb.HardState) bool { 128 | return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit 129 | } 130 | -------------------------------------------------------------------------------- /kv/raftstore/ticker.go: -------------------------------------------------------------------------------- 1 | package raftstore 2 | 3 | import ( 4 | "miniLinkDB/kv/config" 5 | "miniLinkDB/kv/raftstore/message" 6 | "time" 7 | ) 8 | 9 | type ticker struct { 10 | regionID uint64 11 | tick int64 12 | schedules []tickSchedule 13 | } 14 | 15 | type tickSchedule struct { 16 | runAt int64 17 | interval int64 18 | } 19 | 20 | func newTicker(regionID uint64, cfg *config.Config) *ticker { 21 | baseInterval := cfg.RaftBaseTickInterval 22 | t := &ticker{ 23 | regionID: regionID, 24 | schedules: make([]tickSchedule, 6), 25 | } 26 | t.schedules[int(PeerTickRaft)].interval = 1 27 | t.schedules[int(PeerTickRaftLogGC)].interval = int64(cfg.RaftLogGCTickInterval / baseInterval) 28 | t.schedules[int(PeerTickSplitRegionCheck)].interval = int64(cfg.SplitRegionCheckTickInterval / baseInterval) 29 | t.schedules[int(PeerTickSchedulerHeartbeat)].interval = int64(cfg.SchedulerHeartbeatTickInterval / baseInterval) 30 | return t 31 | } 32 | 33 | const SnapMgrGcTickInterval = 1 * time.Minute 34 | 35 | func newStoreTicker(cfg *config.Config) *ticker { 36 | baseInterval := cfg.RaftBaseTickInterval 37 | t := &ticker{ 38 | schedules: make([]tickSchedule, 4), 39 | } 40 | t.schedules[int(StoreTickSchedulerStoreHeartbeat)].interval = int64(cfg.SchedulerStoreHeartbeatTickInterval / baseInterval) 41 | t.schedules[int(StoreTickSnapGC)].interval = int64(SnapMgrGcTickInterval / baseInterval) 42 | return t 43 | } 44 | 45 | // tickClock should be called when peerMsgHandler received tick message. 46 | func (t *ticker) tickClock() { 47 | t.tick++ 48 | } 49 | 50 | // schedule arrange the next run for the PeerTick. 51 | func (t *ticker) schedule(tp PeerTick) { 52 | sched := &t.schedules[int(tp)] 53 | if sched.interval <= 0 { 54 | sched.runAt = -1 55 | return 56 | } 57 | sched.runAt = t.tick + sched.interval 58 | } 59 | 60 | // isOnTick checks if the PeerTick should run. 61 | func (t *ticker) isOnTick(tp PeerTick) bool { 62 | sched := &t.schedules[int(tp)] 63 | return sched.runAt == t.tick 64 | } 65 | 66 | func (t *ticker) isOnStoreTick(tp StoreTick) bool { 67 | sched := &t.schedules[int(tp)] 68 | return sched.runAt == t.tick 69 | } 70 | 71 | func (t *ticker) scheduleStore(tp StoreTick) { 72 | sched := &t.schedules[int(tp)] 73 | if sched.interval <= 0 { 74 | sched.runAt = -1 75 | return 76 | } 77 | sched.runAt = t.tick + sched.interval 78 | } 79 | 80 | type tickDriver struct { 81 | baseTickInterval time.Duration 82 | newRegionCh chan uint64 83 | regions map[uint64]struct{} 84 | router *router 85 | storeTicker *ticker 86 | } 87 | 88 | func newTickDriver(baseTickInterval time.Duration, router *router, storeTicker *ticker) *tickDriver { 89 | return &tickDriver{ 90 | baseTickInterval: baseTickInterval, 91 | newRegionCh: make(chan uint64), 92 | regions: make(map[uint64]struct{}), 93 | router: router, 94 | storeTicker: storeTicker, 95 | } 96 | } 97 | 98 | func (r *tickDriver) run() { 99 | timer := time.Tick(r.baseTickInterval) 100 | for { 101 | select { 102 | case <-timer: 103 | for regionID := range r.regions { 104 | if r.router.send(regionID, message.NewPeerMsg(message.MsgTypeTick, regionID, nil)) != nil { 105 | delete(r.regions, regionID) 106 | } 107 | } 108 | r.tickStore() 109 | case regionID, ok := <-r.newRegionCh: 110 | if !ok { 111 | return 112 | } 113 | r.regions[regionID] = struct{}{} 114 | } 115 | } 116 | } 117 | 118 | func (r *tickDriver) stop() { 119 | close(r.newRegionCh) 120 | } 121 | 122 | func (r *tickDriver) tickStore() { 123 | r.storeTicker.tickClock() 124 | for i := range r.storeTicker.schedules { 125 | if r.storeTicker.isOnStoreTick(StoreTick(i)) { 126 | r.router.sendStore(message.NewMsg(message.MsgTypeStoreTick, StoreTick(i))) 127 | } 128 | } 129 | } -------------------------------------------------------------------------------- /proto/proto/raft_cmdpb.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package raft_cmdpb; 3 | 4 | import "metapb.proto"; 5 | import "errorpb.proto"; 6 | import "eraftpb.proto"; 7 | 8 | message GetRequest { 9 | string cf = 1; 10 | bytes key = 2; 11 | } 12 | 13 | message GetResponse { 14 | bytes value = 1; 15 | } 16 | 17 | message PutRequest { 18 | string cf = 1; 19 | bytes key = 2; 20 | bytes value = 3; 21 | } 22 | 23 | message PutResponse {} 24 | 25 | message DeleteRequest { 26 | string cf = 1; 27 | bytes key = 2; 28 | } 29 | 30 | message DeleteResponse {} 31 | 32 | message SnapRequest {} 33 | 34 | message SnapResponse { 35 | metapb.Region region = 1; 36 | } 37 | 38 | enum CmdType { 39 | Invalid = 0; 40 | Get = 1; 41 | Put = 3; 42 | Delete = 4; 43 | Snap = 5; 44 | } 45 | 46 | message Request { 47 | CmdType cmd_type = 1; 48 | GetRequest get = 2; 49 | PutRequest put = 4; 50 | DeleteRequest delete = 5; 51 | SnapRequest snap = 6; 52 | } 53 | 54 | message Response { 55 | CmdType cmd_type = 1; 56 | GetResponse get = 2; 57 | PutResponse put = 4; 58 | DeleteResponse delete = 5; 59 | SnapResponse snap = 6; 60 | } 61 | 62 | message ChangePeerRequest { 63 | // This can be only called in internal Raftstore now. 64 | eraftpb.ConfChangeType change_type = 1; 65 | metapb.Peer peer = 2; 66 | } 67 | 68 | message ChangePeerResponse { 69 | metapb.Region region = 1; 70 | } 71 | 72 | message SplitRequest { 73 | // This can be only called in internal Raftstore now. 74 | // The split_key has to exist in the splitting region. 75 | bytes split_key = 1; 76 | // We split the region into two. The first uses the origin 77 | // parent region id, and the second uses the new_region_id. 78 | // We must guarantee that the new_region_id is global unique. 79 | uint64 new_region_id = 2; 80 | // The peer ids for the new split region. 81 | repeated uint64 new_peer_ids = 3; 82 | } 83 | 84 | message SplitResponse { 85 | // SplitResponse contains the region where specific keys have split into. 86 | repeated metapb.Region regions = 1; 87 | } 88 | 89 | message CompactLogRequest { 90 | uint64 compact_index = 1; 91 | uint64 compact_term = 2; 92 | } 93 | 94 | message CompactLogResponse {} 95 | 96 | message TransferLeaderRequest { 97 | metapb.Peer peer = 1; 98 | } 99 | 100 | message TransferLeaderResponse {} 101 | 102 | enum AdminCmdType { 103 | InvalidAdmin = 0; 104 | ChangePeer = 1; 105 | CompactLog = 3; 106 | TransferLeader = 4; 107 | Split = 10; 108 | } 109 | 110 | message AdminRequest { 111 | AdminCmdType cmd_type = 1; 112 | ChangePeerRequest change_peer = 2; 113 | CompactLogRequest compact_log = 4; 114 | TransferLeaderRequest transfer_leader = 5; 115 | SplitRequest split = 10; 116 | } 117 | 118 | message AdminResponse { 119 | AdminCmdType cmd_type = 1; 120 | ChangePeerResponse change_peer = 2; 121 | CompactLogResponse compact_log = 4; 122 | TransferLeaderResponse transfer_leader = 5; 123 | SplitResponse split = 10; 124 | } 125 | 126 | message RaftRequestHeader { 127 | uint64 region_id = 1; 128 | metapb.Peer peer = 2; 129 | metapb.RegionEpoch region_epoch = 4; 130 | uint64 term = 5; 131 | } 132 | 133 | message RaftResponseHeader { 134 | errorpb.Error error = 1; 135 | bytes uuid = 2; 136 | uint64 current_term = 3; 137 | } 138 | 139 | message RaftCmdRequest { 140 | RaftRequestHeader header = 1; 141 | // We can't enclose normal requests and administrator request 142 | // at same time. 143 | repeated Request requests = 2; 144 | AdminRequest admin_request = 3; 145 | } 146 | 147 | message RaftCmdResponse { 148 | RaftResponseHeader header = 1; 149 | repeated Response responses = 2; 150 | AdminResponse admin_response = 3; 151 | } 152 | -------------------------------------------------------------------------------- /kv/raftstore/meta/keys.go: -------------------------------------------------------------------------------- 1 | package meta 2 | 3 | import ( 4 | "bytes" 5 | "encoding/binary" 6 | "miniLinkDB/errors" 7 | ) 8 | 9 | const ( 10 | // local is in (0x01, 0x02) 11 | LocalPrefix byte = 0x01 12 | // We save two types region data in DB, for raft and other meta data. 13 | // When the store starts, we should iterate all region meta data to 14 | // construct peer, no need to travel large raft data, so we separate them 15 | // with different prefixes. 16 | RegionRaftPrefix byte = 0x02 17 | RegionMetaPrefix byte = 0x03 18 | RegionRaftPrefixLen = 11 // REGION_RAFT_PREFIX_KEY + region_id + suffix 19 | RegionRaftLogLen = 19 // REGION_RAFT_PREFIX_KEY + region_id + suffix + index 20 | // Following are the suffix after the local prefix. 21 | // For region id 22 | RaftLogSuffix byte = 0x01 23 | RaftStateSuffix byte = 0x02 24 | ApplyStateSuffix byte = 0x03 25 | 26 | RegionStateSuffix byte = 0x01 27 | ) 28 | 29 | var ( 30 | MinKey = []byte{} 31 | MaxKey = []byte{255} 32 | LocalMinKey = []byte{LocalPrefix} 33 | LocalMaxKey = []byte{LocalPrefix + 1} 34 | RegionMetaMinKey = []byte{LocalPrefix, RegionMetaPrefix} 35 | RegionMetaMaxKey = []byte{LocalPrefix, RegionMetaPrefix + 1} 36 | 37 | // 后面的键都是本地键,因此第一个字节必须为0x01 38 | PrepareBootstrapKey = []byte{LocalPrefix, 0x01} 39 | StoreIdentKey = []byte{LocalPrefix, 0x02} 40 | ) 41 | 42 | func makeRegionPrefix(regionID uint64, suffix byte) []byte { 43 | key := make([]byte, 11) 44 | key[0] = LocalPrefix 45 | key[1] = RegionRaftPrefix 46 | binary.BigEndian.PutUint64(key[2:], regionID) 47 | key[10] = suffix 48 | return key 49 | } 50 | 51 | func makeRegionKey(regionID uint64, suffix byte, subID uint64) []byte { 52 | key := make([]byte, 19) 53 | key[0] = LocalPrefix 54 | key[1] = RegionRaftPrefix 55 | binary.BigEndian.PutUint64(key[2:], regionID) 56 | key[10] = suffix 57 | binary.BigEndian.PutUint64(key[11:], subID) 58 | return key 59 | } 60 | 61 | func RegionRaftPrefixKey(regionID uint64) []byte { 62 | key := make([]byte, 10) 63 | key[0] = LocalPrefix 64 | key[1] = RegionRaftPrefix 65 | binary.BigEndian.PutUint64(key[2:], regionID) 66 | return key 67 | } 68 | 69 | func RaftLogKey(regionID, index uint64) []byte { 70 | return makeRegionKey(regionID, RaftLogSuffix, index) 71 | } 72 | 73 | func RaftStateKey(regionID uint64) []byte { 74 | return makeRegionPrefix(regionID, RaftStateSuffix) 75 | } 76 | 77 | func ApplyStateKey(regionID uint64) []byte { 78 | return makeRegionPrefix(regionID, ApplyStateSuffix) 79 | } 80 | 81 | func IsRaftStateKey(key []byte) bool { 82 | return len(key) == 11 && key[0] == LocalPrefix && key[1] == RegionRaftPrefix 83 | } 84 | 85 | func DecodeRegionMetaKey(key []byte) (uint64, byte, error) { 86 | if len(RegionMetaMinKey) +8+1 != len(key) { 87 | return 0,0, errors.Errorf("invalid region meta key length for key %v", key) 88 | } 89 | if !bytes.HasPrefix(key, RegionMetaMinKey) { 90 | return 0,0, errors.Errorf("invalid region meta key prefix for key %v", key) 91 | } 92 | regionID := binary.BigEndian.Uint64(key[len(RegionMetaMinKey):]) 93 | return regionID, key[len(key)-1], nil 94 | } 95 | 96 | func RegionMetaPrefixKey(regionID uint64) []byte { 97 | key := make([]byte, 10) 98 | key[0] = LocalPrefix 99 | key[1] = RegionMetaPrefix 100 | binary.BigEndian.PutUint64(key[2:], regionID) 101 | return key 102 | } 103 | 104 | func RegionStateKey(regionID uint64) []byte { 105 | key := make([]byte, 11) 106 | key[0] = LocalPrefix 107 | key[1] = RegionMetaPrefix 108 | binary.BigEndian.PutUint64(key[2:], regionID) 109 | key[10] = RegionStateSuffix 110 | return key 111 | } 112 | /// RaftLogIndex 从raft_log_key生成的raft日志键中获取日志索引 113 | func RaftLogIndex(key []byte) (uint64, error) { 114 | if len(key) != RegionRaftLogLen { 115 | return 0, errors.Errorf("key %v is not a valid raft log key", key) 116 | } 117 | return binary.BigEndian.Uint64(key[RegionRaftLogLen-8:]), nil 118 | } -------------------------------------------------------------------------------- /errors/juju_adaptor.go: -------------------------------------------------------------------------------- 1 | package errors 2 | 3 | import ( 4 | "fmt" 5 | "strings" 6 | ) 7 | 8 | // ==================== juju adaptor start ======================== 9 | 10 | // Trace just calls AddStack. 11 | func Trace(err error) error { 12 | if err == nil { 13 | return nil 14 | } 15 | return AddStack(err) 16 | } 17 | 18 | // Annotate adds a message and ensures there is a stack trace. 19 | func Annotate(err error, message string) error { 20 | if err == nil { 21 | return nil 22 | } 23 | hasStack := HasStack(err) 24 | err = &withMessage{ 25 | cause: err, 26 | msg: message, 27 | causeHasStack: hasStack, 28 | } 29 | if hasStack { 30 | return err 31 | } 32 | return &withStack{ 33 | err, 34 | callers(), 35 | } 36 | } 37 | 38 | // Annotatef adds a message and ensures there is a stack trace. 39 | func Annotatef(err error, format string, args ...interface{}) error { 40 | if err == nil { 41 | return nil 42 | } 43 | hasStack := HasStack(err) 44 | err = &withMessage{ 45 | cause: err, 46 | msg: fmt.Sprintf(format, args...), 47 | causeHasStack: hasStack, 48 | } 49 | if hasStack { 50 | return err 51 | } 52 | return &withStack{ 53 | err, 54 | callers(), 55 | } 56 | } 57 | 58 | var emptyStack stack 59 | 60 | // NewNoStackError creates error without error stack 61 | // later duplicate trace will no longer generate Stack too. 62 | func NewNoStackError(msg string) error { 63 | return &fundamental{ 64 | msg: msg, 65 | stack: &emptyStack, 66 | } 67 | } 68 | 69 | // SuspendStack suspends stack generate for error. 70 | func SuspendStack(err error) error { 71 | if err == nil { 72 | return err 73 | } 74 | cleared := clearStack(err) 75 | if cleared { 76 | return err 77 | } 78 | return &withStack{ 79 | err, 80 | &emptyStack, 81 | } 82 | } 83 | 84 | func clearStack(err error) (cleared bool) { 85 | switch typedErr := err.(type) { 86 | case *withMessage: 87 | return clearStack(typedErr.Cause()) 88 | case *fundamental: 89 | typedErr.stack = &emptyStack 90 | return true 91 | case *withStack: 92 | typedErr.stack = &emptyStack 93 | clearStack(typedErr.Cause()) 94 | return true 95 | default: 96 | return false 97 | } 98 | } 99 | 100 | // ErrorStack will format a stack trace if it is available, otherwise it will be Error() 101 | // If the error is nil, the empty string is returned 102 | // Note that this just calls fmt.Sprintf("%+v", err) 103 | func ErrorStack(err error) string { 104 | if err == nil { 105 | return "" 106 | } 107 | return fmt.Sprintf("%+v", err) 108 | } 109 | 110 | // IsNotFound reports whether err was not found error. 111 | func IsNotFound(err error) bool { 112 | return strings.Contains(err.Error(), "not found") 113 | } 114 | 115 | // NotFoundf represents an error with not found message. 116 | func NotFoundf(format string, args ...interface{}) error { 117 | return Errorf(format+" not found", args...) 118 | } 119 | 120 | // BadRequestf represents an error with bad request message. 121 | func BadRequestf(format string, args ...interface{}) error { 122 | return Errorf(format+" bad request", args...) 123 | } 124 | 125 | // NotSupportedf represents an error with not supported message. 126 | func NotSupportedf(format string, args ...interface{}) error { 127 | return Errorf(format+" not supported", args...) 128 | } 129 | 130 | // NotValidf represents an error with not valid message. 131 | func NotValidf(format string, args ...interface{}) error { 132 | return Errorf(format+" not valid", args...) 133 | } 134 | 135 | // IsAlreadyExists reports whether err was already exists error. 136 | func IsAlreadyExists(err error) bool { 137 | return strings.Contains(err.Error(), "already exists") 138 | } 139 | 140 | // AlreadyExistsf represents an error with already exists message. 141 | func AlreadyExistsf(format string, args ...interface{}) error { 142 | return Errorf(format+" already exists", args...) 143 | } 144 | 145 | // ==================== juju adaptor end ======================== 146 | -------------------------------------------------------------------------------- /kv/raftstore/runner/split_checker.go: -------------------------------------------------------------------------------- 1 | package runner 2 | 3 | import ( 4 | "encoding/hex" 5 | "github.com/Connor1996/badger" 6 | 7 | "miniLinkDB/kv/config" 8 | "miniLinkDB/kv/raftstore/message" 9 | "miniLinkDB/kv/raftstore/util" 10 | "miniLinkDB/kv/util/codec" 11 | "miniLinkDB/kv/util/engine_util" 12 | "miniLinkDB/kv/util/worker" 13 | "miniLinkDB/log" 14 | "miniLinkDB/proto/pkg/metapb" 15 | ) 16 | 17 | type SplitCheckTask struct { 18 | Region *metapb.Region 19 | } 20 | 21 | type splitCheckHandler struct { 22 | engine *badger.DB 23 | router message.RaftRouter 24 | checker *sizeSplitChecker 25 | } 26 | 27 | func NewSplitCheckHandler(engine *badger.DB, router message.RaftRouter, conf *config.Config) *splitCheckHandler { 28 | runner := &splitCheckHandler{ 29 | engine: engine, 30 | router: router, 31 | checker: newSizeSplitChecker(conf.RegionMaxSize, conf.RegionSplitSize), 32 | } 33 | return runner 34 | } 35 | 36 | /// run checks a region with split checkers to produce split keys and generates split admin command. 37 | func (r *splitCheckHandler) Handle(t worker.Task) { 38 | spCheckTask, ok := t.(*SplitCheckTask) 39 | if !ok { 40 | log.Error("unsupported worker.Task: %+v", t) 41 | return 42 | } 43 | region := spCheckTask.Region 44 | regionId := region.Id 45 | log.Debugf("executing split check worker.Task: [regionId: %d, startKey: %s, endKey: %s]", regionId, 46 | hex.EncodeToString(region.StartKey), hex.EncodeToString(region.EndKey)) 47 | key := r.splitCheck(regionId, region.StartKey, region.EndKey) 48 | if key != nil { 49 | _, userKey, err := codec.DecodeBytes(key) 50 | if err == nil { 51 | key = codec.EncodeBytes(userKey) 52 | } 53 | msg := message.Msg{ 54 | Type: message.MsgTypeSplitRegion, 55 | RegionID: regionId, 56 | Data: &message.MsgSplitRegion{ 57 | RegionEpoch: region.GetRegionEpoch(), 58 | SplitKey: key, 59 | }, 60 | } 61 | err = r.router.Send(regionId, msg) 62 | if err != nil { 63 | log.Warnf("failed to send check result: [regionId: %d, err: %v]", regionId, err) 64 | } 65 | } else { 66 | log.Debugf("no need to send, split key not found: [regionId: %v]", regionId) 67 | } 68 | } 69 | 70 | 71 | /// SplitCheck gets the split keys by scanning the range. 72 | func (r *splitCheckHandler) splitCheck(regionID uint64, startKey, endKey []byte) []byte { 73 | txn := r.engine.NewTransaction(false) 74 | defer txn.Discard() 75 | 76 | r.checker.reset() 77 | it := engine_util.NewCFIterator(engine_util.CfDefault, txn) 78 | defer it.Close() 79 | for it.Seek(startKey); it.Valid(); it.Next() { 80 | item := it.Item() 81 | key := item.Key() 82 | if engine_util.ExceedEndKey(key, endKey) { 83 | 84 | r.router.Send(regionID, message.Msg{ 85 | Type: message.MsgTypeRegionApproximateSize, 86 | Data: r.checker.currentSize, 87 | }) 88 | break 89 | } 90 | if r.checker.onKv(key, item) { 91 | break 92 | } 93 | } 94 | return r.checker.getSplitKey() 95 | } 96 | 97 | type sizeSplitChecker struct { 98 | maxSize uint64 99 | splitSize uint64 100 | 101 | currentSize uint64 102 | splitKey []byte 103 | } 104 | 105 | func newSizeSplitChecker(maxSize, splitSize uint64) *sizeSplitChecker { 106 | return &sizeSplitChecker{ 107 | maxSize: maxSize, 108 | splitSize: splitSize, 109 | } 110 | } 111 | 112 | func (checker *sizeSplitChecker) reset() { 113 | checker.currentSize = 0 114 | checker.splitKey = nil 115 | } 116 | 117 | func (checker *sizeSplitChecker) onKv(key []byte, item engine_util.DBItem) bool { 118 | valueSize := uint64(item.ValueSize()) 119 | size := uint64(len(key)) + valueSize 120 | checker.currentSize += size 121 | if checker.currentSize > checker.splitSize && checker.splitKey == nil { 122 | checker.splitKey = util.SafeCopy(key) 123 | } 124 | return checker.currentSize > checker.maxSize 125 | } 126 | 127 | func (checker *sizeSplitChecker) getSplitKey() []byte { 128 | if checker.currentSize < checker.maxSize { 129 | checker.splitKey = nil 130 | } 131 | return checker.splitKey 132 | } -------------------------------------------------------------------------------- /kv/server/server.go: -------------------------------------------------------------------------------- 1 | package server 2 | 3 | import ( 4 | "context" 5 | "miniLinkDB/kv/coprocessor" 6 | "miniLinkDB/kv/storage" 7 | "miniLinkDB/kv/storage/raft_storage" 8 | "miniLinkDB/kv/transaction/latches" 9 | coppb "miniLinkDB/proto/pkg/coprocessor" 10 | "miniLinkDB/proto/pkg/kvrpcpb" 11 | "miniLinkDB/proto/pkg/linkkvpb" 12 | 13 | "github.com/pingcap/tidb/kv" 14 | ) 15 | 16 | var _ linkkvpb.LinkKvServer = new(Server) 17 | 18 | // Server is a LinkKV server, it 'faces outwards', sending and receiving messages from clients such as SQL. 19 | type Server struct { 20 | storage storage.Storage 21 | 22 | // (Used in 4A/4B) 23 | Latches *latches.Latches 24 | 25 | // coprocessor API handler, out of course scope 26 | copHandler *coprocessor.CopHandler 27 | } 28 | 29 | func NewServer(storage storage.Storage) *Server { 30 | return &Server{ 31 | storage: storage, 32 | Latches: latches.NewLatches(), 33 | } 34 | } 35 | 36 | // The below functions are Server's gRPC API (implements LinkKvServer). 37 | 38 | // Raw API. 39 | func (server *Server) RawGet(_ context.Context, req *kvrpcpb.RawGetRequest) (*kvrpcpb.RawGetResponse, error) { 40 | 41 | return nil, nil 42 | } 43 | 44 | func (server *Server) RawPut(_ context.Context, req *kvrpcpb.RawPutRequest) (*kvrpcpb.RawPutResponse, error) { 45 | 46 | return nil, nil 47 | } 48 | 49 | func (server *Server) RawDelete(_ context.Context, req *kvrpcpb.RawDeleteRequest) (*kvrpcpb.RawDeleteResponse, error) { 50 | 51 | return nil, nil 52 | } 53 | 54 | func (server *Server) RawScan(_ context.Context, req *kvrpcpb.RawScanRequest) (*kvrpcpb.RawScanResponse, error) { 55 | 56 | return nil, nil 57 | } 58 | 59 | // Raft commands (linkkv <-> linkkv) 60 | // Only used for RaftStorage, so trivially forward it. 61 | func (server *Server) Raft(stream linkkvpb.LinkKv_RaftServer) error { 62 | return server.storage.(*raft_storage.RaftStorage).Raft(stream) 63 | } 64 | 65 | // Snapshot stream (linkkv <-> linkkv) 66 | // Only used for RaftStorage, so trivially forward it. 67 | func (server *Server) Snapshot(stream linkkvpb.LinkKv_SnapshotServer) error { 68 | return server.storage.(*raft_storage.RaftStorage).Snapshot(stream) 69 | } 70 | 71 | // Transactional API. 72 | func (server *Server) KvGet(_ context.Context, req *kvrpcpb.GetRequest) (*kvrpcpb.GetResponse, error) { 73 | 74 | return nil, nil 75 | } 76 | 77 | func (server *Server) KvPrewrite(_ context.Context, req *kvrpcpb.PrewriteRequest) (*kvrpcpb.PrewriteResponse, error) { 78 | 79 | return nil, nil 80 | } 81 | 82 | func (server *Server) KvCommit(_ context.Context, req *kvrpcpb.CommitRequest) (*kvrpcpb.CommitResponse, error) { 83 | 84 | return nil, nil 85 | } 86 | 87 | func (server *Server) KvScan(_ context.Context, req *kvrpcpb.ScanRequest) (*kvrpcpb.ScanResponse, error) { 88 | 89 | return nil, nil 90 | } 91 | 92 | func (server *Server) KvCheckTxnStatus(_ context.Context, req *kvrpcpb.CheckTxnStatusRequest) (*kvrpcpb.CheckTxnStatusResponse, error) { 93 | 94 | return nil, nil 95 | } 96 | 97 | func (server *Server) KvBatchRollback(_ context.Context, req *kvrpcpb.BatchRollbackRequest) (*kvrpcpb.BatchRollbackResponse, error) { 98 | 99 | return nil, nil 100 | } 101 | 102 | func (server *Server) KvResolveLock(_ context.Context, req *kvrpcpb.ResolveLockRequest) (*kvrpcpb.ResolveLockResponse, error) { 103 | 104 | return nil, nil 105 | } 106 | 107 | // SQL push down commands. 108 | func (server *Server) Coprocessor(_ context.Context, req *coppb.Request) (*coppb.Response, error) { 109 | resp := new(coppb.Response) 110 | reader, err := server.storage.Reader(req.Context) 111 | if err != nil { 112 | if regionErr, ok := err.(*raft_storage.RegionError); ok { 113 | resp.RegionError = regionErr.RequestErr 114 | return resp, nil 115 | } 116 | return nil, err 117 | } 118 | switch req.Tp { 119 | case kv.ReqTypeDAG: 120 | return server.copHandler.HandleCopDAGRequest(reader, req), nil 121 | case kv.ReqTypeAnalyze: 122 | return server.copHandler.HandleCopAnalyzeRequest(reader, req), nil 123 | } 124 | return nil, nil 125 | } 126 | -------------------------------------------------------------------------------- /proto/include/google/protobuf/struct.proto: -------------------------------------------------------------------------------- 1 | // Protocol Buffers - Google's data interchange format 2 | // Copyright 2008 Google Inc. All rights reserved. 3 | // https://developers.google.com/protocol-buffers/ 4 | // 5 | // Redistribution and use in source and binary forms, with or without 6 | // modification, are permitted provided that the following conditions are 7 | // met: 8 | // 9 | // * Redistributions of source code must retain the above copyright 10 | // notice, this list of conditions and the following disclaimer. 11 | // * Redistributions in binary form must reproduce the above 12 | // copyright notice, this list of conditions and the following disclaimer 13 | // in the documentation and/or other materials provided with the 14 | // distribution. 15 | // * Neither the name of Google Inc. nor the names of its 16 | // contributors may be used to endorse or promote products derived from 17 | // this software without specific prior written permission. 18 | // 19 | // THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 20 | // "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 21 | // LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 22 | // A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 23 | // OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 24 | // SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 25 | // LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 26 | // DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 27 | // THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 28 | // (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | // OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | 31 | syntax = "proto3"; 32 | 33 | package google.protobuf; 34 | 35 | option csharp_namespace = "Google.Protobuf.WellKnownTypes"; 36 | option cc_enable_arenas = true; 37 | option go_package = "types"; 38 | option java_package = "com.google.protobuf"; 39 | option java_outer_classname = "StructProto"; 40 | option java_multiple_files = true; 41 | option objc_class_prefix = "GPB"; 42 | 43 | 44 | // `Struct` represents a structured data value, consisting of fields 45 | // which map to dynamically typed values. In some languages, `Struct` 46 | // might be supported by a native representation. For example, in 47 | // scripting languages like JS a struct is represented as an 48 | // object. The details of that representation are described together 49 | // with the proto support for the language. 50 | // 51 | // The JSON representation for `Struct` is JSON object. 52 | message Struct { 53 | // Unordered map of dynamically typed values. 54 | map fields = 1; 55 | } 56 | 57 | // `Value` represents a dynamically typed value which can be either 58 | // null, a number, a string, a boolean, a recursive struct value, or a 59 | // list of values. A producer of value is expected to set one of that 60 | // variants, absence of any variant indicates an error. 61 | // 62 | // The JSON representation for `Value` is JSON value. 63 | message Value { 64 | // The kind of value. 65 | oneof kind { 66 | // Represents a null value. 67 | NullValue null_value = 1; 68 | // Represents a double value. 69 | double number_value = 2; 70 | // Represents a string value. 71 | string string_value = 3; 72 | // Represents a boolean value. 73 | bool bool_value = 4; 74 | // Represents a structured value. 75 | Struct struct_value = 5; 76 | // Represents a repeated `Value`. 77 | ListValue list_value = 6; 78 | } 79 | } 80 | 81 | // `NullValue` is a singleton enumeration to represent the null value for the 82 | // `Value` type union. 83 | // 84 | // The JSON representation for `NullValue` is JSON `null`. 85 | enum NullValue { 86 | // Null value. 87 | NULL_VALUE = 0; 88 | } 89 | 90 | // `ListValue` is a wrapper around a repeated field of values. 91 | // 92 | // The JSON representation for `ListValue` is JSON array. 93 | message ListValue { 94 | // Repeated field of dynamically typed values. 95 | repeated Value values = 1; 96 | } 97 | -------------------------------------------------------------------------------- /kv/transaction/mvcc/transaction.go: -------------------------------------------------------------------------------- 1 | package mvcc 2 | 3 | import ( 4 | "encoding/binary" 5 | "miniLinkDB/kv/storage" 6 | "miniLinkDB/kv/util/codec" 7 | 8 | "miniLinkDB/scheduler/pkg/tsoutil" 9 | 10 | "miniLinkDB/proto/pkg/kvrpcpb" 11 | ) 12 | 13 | type KeyError struct { 14 | kvrpcpb.KeyError 15 | } 16 | 17 | func (ke *KeyError) Error() string { 18 | return ke.String() 19 | } 20 | 21 | // MvccTxn groups together writes as part of a single transaction. It also provides an abstraction over low-level 22 | // storage, lowering the concepts of timestamps, writes, and locks into plain keys and values. 23 | type MvccTxn struct { 24 | StartTS uint64 25 | Reader storage.StorageReader 26 | writes []storage.Modify 27 | } 28 | 29 | func NewMvccTxn(reader storage.StorageReader, startTs uint64) *MvccTxn { 30 | return &MvccTxn{ 31 | Reader: reader, 32 | StartTS: startTs, 33 | } 34 | } 35 | 36 | // Writes returns all changes added to this transaction. 37 | func (txn *MvccTxn) Writes() []storage.Modify { 38 | return txn.writes 39 | } 40 | 41 | // PutWrite records a write at key and ts. 42 | func (txn *MvccTxn) PutWrite(key []byte, ts uint64, write *Write) { 43 | 44 | } 45 | 46 | // GetLock returns a lock if key is locked. It will return (nil, nil) if there is no lock on key, and (nil, err) 47 | // if an error occurs during lookup. 48 | func (txn *MvccTxn) GetLock(key []byte) (*Lock, error) { 49 | 50 | return nil, nil 51 | } 52 | 53 | // PutLock adds a key/lock to this transaction. 54 | func (txn *MvccTxn) PutLock(key []byte, lock *Lock) { 55 | 56 | } 57 | 58 | // DeleteLock adds a delete lock to this transaction. 59 | func (txn *MvccTxn) DeleteLock(key []byte) { 60 | 61 | } 62 | 63 | // GetValue finds the value for key, valid at the start timestamp of this transaction. 64 | // I.e., the most recent value committed before the start of this transaction. 65 | func (txn *MvccTxn) GetValue(key []byte) ([]byte, error) { 66 | 67 | return nil, nil 68 | } 69 | 70 | // PutValue adds a key/value write to this transaction. 71 | func (txn *MvccTxn) PutValue(key []byte, value []byte) { 72 | 73 | } 74 | 75 | // DeleteValue removes a key/value pair in this transaction. 76 | func (txn *MvccTxn) DeleteValue(key []byte) { 77 | 78 | } 79 | 80 | 81 | // CurrentWrite searches for a write with this transaction's start timestamp. It returns a Write from the DB and that 82 | // write's commit timestamp, or an error. 83 | func (txn *MvccTxn) CurrentWrite(key []byte) (*Write, uint64, error) { 84 | 85 | return nil, 0, nil 86 | } 87 | 88 | // MostRecentWrite finds the most recent write with the given key. It returns a Write from the DB and that 89 | // write's commit timestamp, or an error. 90 | func (txn *MvccTxn) MostRecentWrite(key []byte) (*Write, uint64, error) { 91 | return nil, 0, nil 92 | } 93 | 94 | 95 | // EncodeKey encodes a user key and appends an encoded timestamp to a key. Keys and timestamps are encoded so that 96 | // timestamped keys are sorted first by key (ascending), then by timestamp (descending). The encoding is based on 97 | // https://github.com/facebook/mysql-5.6/wiki/MyRocks-record-format#memcomparable-format. 98 | func EncodeKey(key []byte, ts uint64) []byte { 99 | encodedKey := codec.EncodeBytes(key) 100 | newKey := append(encodedKey, make([]byte, 8)...) 101 | binary.BigEndian.PutUint64(newKey[len(encodedKey):], ^ts) 102 | return newKey 103 | } 104 | 105 | // DecodeUserKey takes a key + timestamp and returns the key part. 106 | func DecodeUserKey(key []byte) []byte { 107 | _, userKey, err := codec.DecodeBytes(key) 108 | if err != nil { 109 | panic(err) 110 | } 111 | return userKey 112 | } 113 | 114 | // decodeTimestamp takes a key + timestamp and returns the timestamp part. 115 | func decodeTimestamp(key []byte) uint64 { 116 | left, _, err := codec.DecodeBytes(key) 117 | if err != nil { 118 | panic(err) 119 | } 120 | return ^binary.BigEndian.Uint64(left) 121 | } 122 | 123 | // PhysicalTime returns the physical time part of the timestamp. 124 | func PhysicalTime(ts uint64) uint64 { 125 | return ts >> tsoutil.PhysicalShiftBits 126 | } -------------------------------------------------------------------------------- /proto/include/google/protobuf/wrappers.proto: -------------------------------------------------------------------------------- 1 | // Protocol Buffers - Google's data interchange format 2 | // Copyright 2008 Google Inc. All rights reserved. 3 | // https://developers.google.com/protocol-buffers/ 4 | // 5 | // Redistribution and use in source and binary forms, with or without 6 | // modification, are permitted provided that the following conditions are 7 | // met: 8 | // 9 | // * Redistributions of source code must retain the above copyright 10 | // notice, this list of conditions and the following disclaimer. 11 | // * Redistributions in binary form must reproduce the above 12 | // copyright notice, this list of conditions and the following disclaimer 13 | // in the documentation and/or other materials provided with the 14 | // distribution. 15 | // * Neither the name of Google Inc. nor the names of its 16 | // contributors may be used to endorse or promote products derived from 17 | // this software without specific prior written permission. 18 | // 19 | // THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 20 | // "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 21 | // LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 22 | // A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 23 | // OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 24 | // SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 25 | // LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 26 | // DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 27 | // THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 28 | // (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | // OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | 31 | // Wrappers for primitive (non-message) types. These types are useful 32 | // for embedding primitives in the `google.protobuf.Any` type and for places 33 | // where we need to distinguish between the absence of a primitive 34 | // typed field and its default value. 35 | 36 | syntax = "proto3"; 37 | 38 | package google.protobuf; 39 | 40 | option csharp_namespace = "Google.Protobuf.WellKnownTypes"; 41 | option cc_enable_arenas = true; 42 | option go_package = "types"; 43 | option java_package = "com.google.protobuf"; 44 | option java_outer_classname = "WrappersProto"; 45 | option java_multiple_files = true; 46 | option objc_class_prefix = "GPB"; 47 | 48 | // Wrapper message for `double`. 49 | // 50 | // The JSON representation for `DoubleValue` is JSON number. 51 | message DoubleValue { 52 | // The double value. 53 | double value = 1; 54 | } 55 | 56 | // Wrapper message for `float`. 57 | // 58 | // The JSON representation for `FloatValue` is JSON number. 59 | message FloatValue { 60 | // The float value. 61 | float value = 1; 62 | } 63 | 64 | // Wrapper message for `int64`. 65 | // 66 | // The JSON representation for `Int64Value` is JSON string. 67 | message Int64Value { 68 | // The int64 value. 69 | int64 value = 1; 70 | } 71 | 72 | // Wrapper message for `uint64`. 73 | // 74 | // The JSON representation for `UInt64Value` is JSON string. 75 | message UInt64Value { 76 | // The uint64 value. 77 | uint64 value = 1; 78 | } 79 | 80 | // Wrapper message for `int32`. 81 | // 82 | // The JSON representation for `Int32Value` is JSON number. 83 | message Int32Value { 84 | // The int32 value. 85 | int32 value = 1; 86 | } 87 | 88 | // Wrapper message for `uint32`. 89 | // 90 | // The JSON representation for `UInt32Value` is JSON number. 91 | message UInt32Value { 92 | // The uint32 value. 93 | uint32 value = 1; 94 | } 95 | 96 | // Wrapper message for `bool`. 97 | // 98 | // The JSON representation for `BoolValue` is JSON `true` and `false`. 99 | message BoolValue { 100 | // The bool value. 101 | bool value = 1; 102 | } 103 | 104 | // Wrapper message for `string`. 105 | // 106 | // The JSON representation for `StringValue` is JSON string. 107 | message StringValue { 108 | // The string value. 109 | string value = 1; 110 | } 111 | 112 | // Wrapper message for `bytes`. 113 | // 114 | // The JSON representation for `BytesValue` is JSON string. 115 | message BytesValue { 116 | // The bytes value. 117 | bytes value = 1; 118 | } 119 | -------------------------------------------------------------------------------- /proto/proto/eraftpb.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package eraftpb; 3 | 4 | enum EntryType { 5 | EntryNormal = 0; 6 | EntryConfChange = 1; 7 | } 8 | 9 | // The entry is a type of change that needs to be applied. It contains two data fields. 10 | // While the fields are built into the model; their usage is determined by the entry_type. 11 | // 12 | // For normal entries, the data field should contain the data change that should be applied. 13 | // The context field can be used for any contextual data that might be relevant to the 14 | // application of the data. 15 | // 16 | // For configuration changes, the data will contain the ConfChange message and the 17 | // context will provide anything needed to assist the configuration change. The context 18 | // is for the user to set and use in this case. 19 | message Entry { 20 | EntryType entry_type = 1; 21 | uint64 term = 2; 22 | uint64 index = 3; 23 | bytes data = 4; 24 | } 25 | 26 | // SnapshotMetadata contains the log index and term of the last log applied to this 27 | // Snapshot, along with the membership information of the time the last log applied. 28 | message SnapshotMetadata { 29 | ConfState conf_state = 1; 30 | uint64 index = 2; 31 | uint64 term = 3; 32 | } 33 | 34 | message Snapshot { 35 | bytes data = 1; 36 | SnapshotMetadata metadata = 2; 37 | } 38 | 39 | // Some MessageType defined here are local messages which not come from the network, but should 40 | // also use the Step method to handle 41 | enum MessageType { 42 | // 'MessageType_MsgHup' is a local message used for election. If an election timeout happened, 43 | // the node should pass 'MessageType_MsgHup' to its Step method and start a new election. 44 | MsgHup = 0; 45 | // 'MessageType_MsgBeat' is a local message that signals the leader to send a heartbeat 46 | // of the 'MessageType_MsgHeartbeat' type to its followers. 47 | MsgBeat = 1; 48 | // 'MessageType_MsgPropose' is a local message that proposes to append data to the leader's log entries. 49 | MsgPropose = 2; 50 | // 'MessageType_MsgAppend' contains log entries to replicate. 51 | MsgAppend = 3; 52 | // 'MessageType_MsgAppendResponse' is response to log replication request('MessageType_MsgAppend'). 53 | MsgAppendResponse = 4; 54 | // 'MessageType_MsgRequestVote' requests votes for election. 55 | MsgRequestVote = 5; 56 | // 'MessageType_MsgRequestVoteResponse' contains responses from voting request. 57 | MsgRequestVoteResponse = 6; 58 | // 'MessageType_MsgSnapshot' requests to install a snapshot message. 59 | MsgSnapshot = 7; 60 | // 'MessageType_MsgHeartbeat' sends heartbeat from leader to its followers. 61 | MsgHeartbeat = 8; 62 | // 'MessageType_MsgHeartbeatResponse' is a response to 'MessageType_MsgHeartbeat'. 63 | MsgHeartbeatResponse = 9; 64 | // 'MessageType_MsgTransferLeader' requests the leader to transfer its leadership. 65 | MsgTransferLeader = 11; 66 | // 'MessageType_MsgTimeoutNow' send from the leader to the leadership transfer target, to let 67 | // the transfer target timeout immediately and start a new election. 68 | MsgTimeoutNow = 12; 69 | } 70 | 71 | message Message { 72 | MessageType msg_type = 1; 73 | uint64 to = 2; 74 | uint64 from = 3; 75 | uint64 term = 4; 76 | uint64 log_term = 5; 77 | uint64 index = 6; 78 | repeated Entry entries = 7; 79 | uint64 commit = 8; 80 | Snapshot snapshot = 9; 81 | bool reject = 10; 82 | } 83 | 84 | // HardState contains the state of a node need to be peristed, including the current term, commit index 85 | // and the vote record 86 | message HardState { 87 | uint64 term = 1; 88 | uint64 vote = 2; 89 | uint64 commit = 3; 90 | } 91 | 92 | // ConfState contains the current membership information of the raft group 93 | message ConfState { 94 | // all node id 95 | repeated uint64 nodes = 1; 96 | } 97 | 98 | enum ConfChangeType { 99 | AddNode = 0; 100 | RemoveNode = 1; 101 | } 102 | 103 | // ConfChange is the data that attach on entry with EntryConfChange type 104 | message ConfChange { 105 | ConfChangeType change_type = 1; 106 | // node will be add/remove 107 | uint64 node_id = 2; 108 | bytes context = 3; 109 | } 110 | -------------------------------------------------------------------------------- /kv/test_raftstore/utils.go: -------------------------------------------------------------------------------- 1 | package test_raftstore 2 | 3 | import ( 4 | "bytes" 5 | "encoding/hex" 6 | "fmt" 7 | "time" 8 | 9 | "github.com/Connor1996/badger" 10 | 11 | 12 | "miniLinkDB/kv/config" 13 | "miniLinkDB/kv/util/engine_util" 14 | "miniLinkDB/log" 15 | "miniLinkDB/proto/pkg/metapb" 16 | "miniLinkDB/proto/pkg/raft_cmdpb" 17 | ) 18 | 19 | func SleepMS(ms int64) { 20 | time.Sleep(time.Duration(ms) * time.Millisecond) 21 | } 22 | 23 | func NewPeer(storeID, peerID uint64) *metapb.Peer { 24 | peer := &metapb.Peer{ 25 | StoreId: storeID, 26 | Id: peerID, 27 | } 28 | return peer 29 | } 30 | 31 | func NewBaseRequest(regionID uint64, epoch *metapb.RegionEpoch) raft_cmdpb.RaftCmdRequest { 32 | req := raft_cmdpb.RaftCmdRequest{} 33 | req.Header = &raft_cmdpb.RaftRequestHeader{RegionId: regionID, RegionEpoch: epoch} 34 | return req 35 | } 36 | 37 | func NewRequest(regionID uint64, epoch *metapb.RegionEpoch, requests []*raft_cmdpb.Request) raft_cmdpb.RaftCmdRequest { 38 | req := NewBaseRequest(regionID, epoch) 39 | req.Requests = requests 40 | return req 41 | } 42 | 43 | func NewAdminRequest(regionID uint64, epoch *metapb.RegionEpoch, request *raft_cmdpb.AdminRequest) *raft_cmdpb.RaftCmdRequest { 44 | req := NewBaseRequest(regionID, epoch) 45 | req.AdminRequest = request 46 | return &req 47 | } 48 | 49 | func NewPutCfCmd(cf string, key, value []byte) *raft_cmdpb.Request { 50 | cmd := &raft_cmdpb.Request{} 51 | cmd.CmdType = raft_cmdpb.CmdType_Put 52 | cmd.Put = &raft_cmdpb.PutRequest{Key: key, Value: value, Cf: cf} 53 | return cmd 54 | } 55 | 56 | func NewGetCfCmd(cf string, key []byte) *raft_cmdpb.Request { 57 | get := &raft_cmdpb.GetRequest{ 58 | Cf: cf, 59 | Key: key, 60 | } 61 | cmd := &raft_cmdpb.Request{ 62 | CmdType: raft_cmdpb.CmdType_Get, 63 | Get: get, 64 | } 65 | return cmd 66 | } 67 | 68 | func NewDeleteCfCmd(cf string, key []byte) *raft_cmdpb.Request { 69 | delete := &raft_cmdpb.DeleteRequest{ 70 | Cf: cf, 71 | Key: key, 72 | } 73 | cmd := &raft_cmdpb.Request{ 74 | CmdType: raft_cmdpb.CmdType_Delete, 75 | Delete: delete, 76 | } 77 | return cmd 78 | } 79 | 80 | func NewSnapCmd() *raft_cmdpb.Request { 81 | cmd := &raft_cmdpb.Request{ 82 | CmdType: raft_cmdpb.CmdType_Snap, 83 | Snap: &raft_cmdpb.SnapRequest{}, 84 | } 85 | return cmd 86 | } 87 | 88 | func NewTransferLeaderCmd(peer *metapb.Peer) *raft_cmdpb.AdminRequest { 89 | transferLeader := raft_cmdpb.TransferLeaderRequest{Peer: peer} 90 | cmd := &raft_cmdpb.AdminRequest{ 91 | CmdType: raft_cmdpb.AdminCmdType_TransferLeader, 92 | TransferLeader: &transferLeader, 93 | } 94 | return cmd 95 | } 96 | 97 | func MustGetCf(engine *engine_util.Engines, cf string, key []byte, value []byte) { 98 | for i := 0; i < 300; i++ { 99 | val, err := engine_util.GetCF(engine.Kv, cf, key) 100 | if err == nil && (value == nil || bytes.Compare(val, value) == 0) { 101 | return 102 | } 103 | SleepMS(20) 104 | } 105 | panic(fmt.Sprintf("can't get value %s for key %s", hex.EncodeToString(value), hex.EncodeToString(key))) 106 | } 107 | 108 | func MustGetCfEqual(engine *engine_util.Engines, cf string, key []byte, value []byte) { 109 | MustGetCf(engine, cf, key, value) 110 | } 111 | 112 | func MustGetEqual(engine *engine_util.Engines, key []byte, value []byte) { 113 | MustGetCf(engine, engine_util.CfDefault, key, value) 114 | } 115 | 116 | func MustGetCfNone(engine *engine_util.Engines, cf string, key []byte) { 117 | var val []byte 118 | var err error 119 | for i := 0; i < 300; i++ { 120 | val, err = engine_util.GetCF(engine.Kv, cf, key) 121 | if err == badger.ErrKeyNotFound { 122 | return 123 | } 124 | SleepMS(20) 125 | } 126 | panic(fmt.Sprintf("get value %s for key %s", hex.EncodeToString(val), hex.EncodeToString(key))) 127 | } 128 | 129 | func MustGetNone(engine *engine_util.Engines, key []byte) { 130 | MustGetCfNone(engine, engine_util.CfDefault, key) 131 | } 132 | 133 | func NewTestCluster(count int, cfg *config.Config) *Cluster { 134 | log.SetLevelByString(cfg.LogLevel) 135 | log.SetFlags(log.Ldate | log.Ltime | log.Lmicroseconds | log.Lshortfile) 136 | schedulerClient := NewMockSchedulerClient(0, uint64(count)+1) 137 | simulator := NewNodeSimulator(schedulerClient) 138 | return NewCluster(count, schedulerClient, simulator, cfg) 139 | } 140 | -------------------------------------------------------------------------------- /kv/raftstore/bootstrap.go: -------------------------------------------------------------------------------- 1 | package raftstore 2 | 3 | import ( 4 | "bytes" 5 | 6 | "github.com/Connor1996/badger" 7 | "miniLinkDB/errors" 8 | "miniLinkDB/kv/raftstore/meta" 9 | "miniLinkDB/kv/util/engine_util" 10 | "miniLinkDB/proto/pkg/eraftpb" 11 | "miniLinkDB/proto/pkg/metapb" 12 | rspb "miniLinkDB/proto/pkg/raft_serverpb" 13 | ) 14 | 15 | const ( 16 | InitEpochVer uint64 = 1 17 | InitEpochConfVer uint64 = 1 18 | ) 19 | 20 | func isRangeEmpty(engine *badger.DB, startKey, endKey []byte) (bool, error) { 21 | var hasData bool 22 | err := engine.View(func(txn *badger.Txn) error { 23 | it := txn.NewIterator(badger.DefaultIteratorOptions) 24 | defer it.Close() 25 | it.Seek(startKey) 26 | if it.Valid() { 27 | item := it.Item() 28 | if bytes.Compare(item.Key(), endKey) <0 { 29 | hasData = true 30 | } 31 | } 32 | return nil 33 | }) 34 | if err != nil { 35 | return false, errors.WithStack(err) 36 | } 37 | return !hasData, err 38 | } 39 | 40 | func BootstrapStore(engines *engine_util.Engines, clusterID, storeID uint64) error { 41 | ident := new(rspb.StoreIdent) 42 | empty, err := isRangeEmpty(engines.Kv, meta.MinKey, meta.MaxKey) 43 | if err != nil { 44 | return err 45 | } 46 | if !empty { 47 | return errors.New("kv store is not empty and has already had data.") 48 | } 49 | empty, err = isRangeEmpty(engines.Raft, meta.MinKey, meta.MaxKey) 50 | if err != nil { 51 | return err 52 | } 53 | if !empty { 54 | return errors.New("raft store is not empty and has already had data.") 55 | } 56 | ident.ClusterId = clusterID 57 | ident.StoreId = storeID 58 | err = engine_util.PutMeta(engines.Kv, meta.StoreIdentKey, ident) 59 | if err != nil { 60 | return err 61 | } 62 | return nil 63 | } 64 | 65 | func PrepareBootstrap(engines *engine_util.Engines, storeID, regionID, peerID uint64) (*metapb.Region, error) { 66 | region := &metapb.Region{ 67 | Id: regionID, 68 | StartKey: []byte{}, 69 | EndKey: []byte{}, 70 | RegionEpoch: &metapb.RegionEpoch{ 71 | Version: InitEpochVer, 72 | ConfVer: InitEpochConfVer, 73 | }, 74 | Peers: []*metapb.Peer{ 75 | { 76 | Id: peerID, 77 | StoreId: storeID, 78 | }, 79 | }, 80 | } 81 | err := PrepareBootstrapCluster(engines, region) 82 | if err != nil { 83 | return nil, err 84 | } 85 | return region, nil 86 | } 87 | 88 | func PrepareBootstrapCluster(engines *engine_util.Engines, region *metapb.Region) error { 89 | state := new(rspb.RegionLocalState) 90 | state.Region = region 91 | kvWB := new(engine_util.WriteBatch) 92 | kvWB.SetMeta(meta.PrepareBootstrapKey, state) 93 | kvWB.SetMeta(meta.RegionStateKey(region.Id), state) 94 | writeInitialApplyState(kvWB, region.Id) 95 | err := engines.WriteKV(kvWB) 96 | if err != nil { 97 | return err 98 | } 99 | raftWB :=new(engine_util.WriteBatch) 100 | writeInitialRaftState(raftWB, region.Id) 101 | err = engines.WriteRaft(raftWB) 102 | if err != nil { 103 | return err 104 | } 105 | return nil 106 | } 107 | 108 | func writeInitialApplyState(kvWB *engine_util.WriteBatch, regionID uint64) { 109 | applyState := &rspb.RaftApplyState{ 110 | AppliedIndex: meta.RaftInitLogIndex, 111 | TruncatedState: &rspb.RaftTruncatedState{ 112 | Index: meta.RaftInitLogIndex, 113 | Term: meta.RaftInitLogTerm, 114 | }, 115 | } 116 | kvWB.SetMeta(meta.ApplyStateKey(regionID), applyState) 117 | } 118 | 119 | func writeInitialRaftState(raftWB *engine_util.WriteBatch, regionID uint64) { 120 | raftState := &rspb.RaftLocalState{ 121 | HardState: &eraftpb.HardState{ 122 | Term: meta.RaftInitLogTerm, 123 | Commit: meta.RaftInitLogIndex, 124 | }, 125 | LastIndex: meta.RaftInitLogIndex, 126 | } 127 | raftWB.SetMeta(meta.RaftStateKey(regionID), raftState) 128 | } 129 | 130 | func ClearPrepareBootstrap(engines *engine_util.Engines, regionID uint64) error { 131 | err := engines.Raft.Update(func(txn *badger.Txn) error { 132 | return txn.Delete(meta.RaftStateKey(regionID)) 133 | }) 134 | if err != nil { 135 | return errors.WithStack(err) 136 | } 137 | wb := new(engine_util.WriteBatch) 138 | wb.DeleteMeta(meta.PrepareBootstrapKey) 139 | // should clear raft initial state too. 140 | wb.DeleteMeta(meta.RegionStateKey(regionID)) 141 | wb.DeleteMeta(meta.ApplyStateKey(regionID)) 142 | err = engines.WriteKV(wb) 143 | if err != nil { 144 | return err 145 | } 146 | return nil 147 | } 148 | 149 | func ClearPrepareBootstrapState(engines *engine_util.Engines) error { 150 | err := engines.Kv.Update(func(txn *badger.Txn) error { 151 | return txn.Delete(meta.PrepareBootstrapKey) 152 | }) 153 | return errors.WithStack(err) 154 | } -------------------------------------------------------------------------------- /kv/raftstore/runner/scheduler_task.go: -------------------------------------------------------------------------------- 1 | package runner 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/Connor1996/badger" 7 | "github.com/shirou/gopsutil/disk" 8 | "miniLinkDB/kv/raftstore/message" 9 | "miniLinkDB/kv/raftstore/scheduler_client" 10 | 11 | "miniLinkDB/kv/util/worker" 12 | "miniLinkDB/log" 13 | "miniLinkDB/proto/pkg/metapb" 14 | "miniLinkDB/proto/pkg/raft_cmdpb" 15 | "miniLinkDB/proto/pkg/schedulerpb" 16 | ) 17 | 18 | type SchedulerAskSplitTask struct { 19 | Region *metapb.Region 20 | SplitKey []byte 21 | Peer *metapb.Peer 22 | Callback *message.Callback 23 | } 24 | 25 | type SchedulerRegionHeartbeatTask struct { 26 | Region *metapb.Region 27 | Peer *metapb.Peer 28 | PendingPeers []*metapb.Peer 29 | ApproximateSize *uint64 30 | } 31 | 32 | type SchedulerStoreHeartbeatTask struct { 33 | Stats *schedulerpb.StoreStats 34 | Engine *badger.DB 35 | Path string 36 | } 37 | 38 | type SchedulerTaskHandler struct { 39 | storeID uint64 40 | SchedulerClient scheduler_client.Client 41 | router message.RaftRouter 42 | } 43 | 44 | func NewSchedulerTaskHandler(storeID uint64, SchedulerClient scheduler_client.Client, router message.RaftRouter) *SchedulerTaskHandler { 45 | return &SchedulerTaskHandler{ 46 | storeID: storeID, 47 | SchedulerClient: SchedulerClient, 48 | router: router, 49 | } 50 | } 51 | 52 | func (r *SchedulerTaskHandler) Handle(t worker.Task) { 53 | switch t.(type) { 54 | case *SchedulerAskSplitTask: 55 | r.onAskSplit(t.(*SchedulerAskSplitTask)) 56 | case *SchedulerRegionHeartbeatTask: 57 | r.onHeartbeat(t.(*SchedulerRegionHeartbeatTask)) 58 | case *SchedulerStoreHeartbeatTask: 59 | r.onStoreHeartbeat(t.(*SchedulerStoreHeartbeatTask)) 60 | default: 61 | log.Error("unsupported worker.Task: %+v", t) 62 | } 63 | } 64 | 65 | func (r *SchedulerTaskHandler) Start() { 66 | r.SchedulerClient.SetRegionHeartbeatResponseHandler(r.storeID, r.onRegionHeartbeatResponse) 67 | } 68 | 69 | func (r *SchedulerTaskHandler) onRegionHeartbeatResponse(resp *schedulerpb.RegionHeartbeatResponse) { 70 | if changePeer := resp.GetChangePeer(); changePeer != nil { 71 | r.sendAdminRequest(resp.RegionId, resp.RegionEpoch, resp.TargetPeer, &raft_cmdpb.AdminRequest{ 72 | CmdType: raft_cmdpb.AdminCmdType_ChangePeer, 73 | ChangePeer: &raft_cmdpb.ChangePeerRequest{ 74 | ChangeType: changePeer.ChangeType, 75 | Peer: changePeer.Peer, 76 | }, 77 | }, message.NewCallback()) 78 | } else if transferLeader := resp.GetTransferLeader(); transferLeader != nil { 79 | r.sendAdminRequest(resp.RegionId, resp.RegionEpoch, resp.TargetPeer, &raft_cmdpb.AdminRequest{ 80 | CmdType: raft_cmdpb.AdminCmdType_TransferLeader, 81 | TransferLeader: &raft_cmdpb.TransferLeaderRequest{ 82 | Peer: transferLeader.Peer, 83 | }, 84 | }, message.NewCallback()) 85 | } 86 | } 87 | 88 | func (r *SchedulerTaskHandler) onAskSplit(t *SchedulerAskSplitTask) { 89 | resp, err := r.SchedulerClient.AskSplit(context.TODO(), t.Region) 90 | if err != nil { 91 | log.Error(err) 92 | return 93 | } 94 | 95 | aq := &raft_cmdpb.AdminRequest{ 96 | CmdType: raft_cmdpb.AdminCmdType_Split, 97 | Split: &raft_cmdpb.SplitRequest{ 98 | SplitKey: t.SplitKey, 99 | NewRegionId: resp.NewRegionId, 100 | NewPeerIds: resp.NewPeerIds, 101 | }, 102 | } 103 | r.sendAdminRequest(t.Region.GetId(), t.Region.GetRegionEpoch(), t.Peer, aq, t.Callback) 104 | } 105 | 106 | func (r *SchedulerTaskHandler) onHeartbeat(t *SchedulerRegionHeartbeatTask) { 107 | var size int64 108 | if t.ApproximateSize != nil { 109 | size = int64(*t.ApproximateSize) 110 | } 111 | req := &schedulerpb.RegionHeartbeatRequest{ 112 | Region: t.Region, 113 | Leader: t.Peer, 114 | PendingPeers: t.PendingPeers, 115 | ApproximateSize: uint64(size), 116 | } 117 | r.SchedulerClient.RegionHeartbeat(req) 118 | } 119 | 120 | func (r *SchedulerTaskHandler) onStoreHeartbeat(t *SchedulerStoreHeartbeatTask) { 121 | diskStat, err := disk.Usage(t.Path) 122 | if err != nil { 123 | log.Error(err) 124 | return 125 | } 126 | 127 | capacity := diskStat.Total 128 | lsmSize, vlogSize := t.Engine.Size() 129 | usedSize := t.Stats.UsedSize + uint64(lsmSize) + uint64(vlogSize) // t.Stats.UsedSize contains size of snapshot files. 130 | available := uint64(0) 131 | if capacity > usedSize { 132 | available = capacity - usedSize 133 | } 134 | 135 | t.Stats.Capacity = capacity 136 | t.Stats.UsedSize = usedSize 137 | t.Stats.Available = available 138 | 139 | r.SchedulerClient.StoreHeartbeat(context.TODO(), t.Stats) 140 | } 141 | 142 | func (r *SchedulerTaskHandler) sendAdminRequest(regionID uint64, epoch *metapb.RegionEpoch, peer *metapb.Peer, req *raft_cmdpb.AdminRequest, callback *message.Callback) { 143 | cmd := &raft_cmdpb.RaftCmdRequest{ 144 | Header : &raft_cmdpb.RaftRequestHeader{ 145 | RegionId: regionID, 146 | Peer: peer, 147 | RegionEpoch: epoch, 148 | }, 149 | AdminRequest: req, 150 | } 151 | r.router.SendRaftCommand(cmd, callback) 152 | } -------------------------------------------------------------------------------- /proto/include/gogoproto/gogo.proto: -------------------------------------------------------------------------------- 1 | // Protocol Buffers for Go with Gadgets 2 | // 3 | // Copyright (c) 2013, The GoGo Authors. All rights reserved. 4 | // http://github.com/gogo/protobuf 5 | // 6 | // Redistribution and use in source and binary forms, with or without 7 | // modification, are permitted provided that the following conditions are 8 | // met: 9 | // 10 | // * Redistributions of source code must retain the above copyright 11 | // notice, this list of conditions and the following disclaimer. 12 | // * Redistributions in binary form must reproduce the above 13 | // copyright notice, this list of conditions and the following disclaimer 14 | // in the documentation and/or other materials provided with the 15 | // distribution. 16 | // 17 | // THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 18 | // "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 19 | // LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 20 | // A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 21 | // OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 22 | // SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 23 | // LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 24 | // DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 25 | // THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 26 | // (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 27 | // OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 28 | 29 | syntax = "proto2"; 30 | package gogoproto; 31 | 32 | import "google/protobuf/descriptor.proto"; 33 | 34 | option java_package = "com.google.protobuf"; 35 | option java_outer_classname = "GoGoProtos"; 36 | option go_package = "github.com/gogo/protobuf/gogoproto"; 37 | 38 | extend google.protobuf.EnumOptions { 39 | optional bool goproto_enum_prefix = 62001; 40 | optional bool goproto_enum_stringer = 62021; 41 | optional bool enum_stringer = 62022; 42 | optional string enum_customname = 62023; 43 | optional bool enumdecl = 62024; 44 | } 45 | 46 | extend google.protobuf.EnumValueOptions { 47 | optional string enumvalue_customname = 66001; 48 | } 49 | 50 | extend google.protobuf.FileOptions { 51 | optional bool goproto_getters_all = 63001; 52 | optional bool goproto_enum_prefix_all = 63002; 53 | optional bool goproto_stringer_all = 63003; 54 | optional bool verbose_equal_all = 63004; 55 | optional bool face_all = 63005; 56 | optional bool gostring_all = 63006; 57 | optional bool populate_all = 63007; 58 | optional bool stringer_all = 63008; 59 | optional bool onlyone_all = 63009; 60 | 61 | optional bool equal_all = 63013; 62 | optional bool description_all = 63014; 63 | optional bool testgen_all = 63015; 64 | optional bool benchgen_all = 63016; 65 | optional bool marshaler_all = 63017; 66 | optional bool unmarshaler_all = 63018; 67 | optional bool stable_marshaler_all = 63019; 68 | 69 | optional bool sizer_all = 63020; 70 | 71 | optional bool goproto_enum_stringer_all = 63021; 72 | optional bool enum_stringer_all = 63022; 73 | 74 | optional bool unsafe_marshaler_all = 63023; 75 | optional bool unsafe_unmarshaler_all = 63024; 76 | 77 | optional bool goproto_extensions_map_all = 63025; 78 | optional bool goproto_unrecognized_all = 63026; 79 | optional bool gogoproto_import = 63027; 80 | optional bool protosizer_all = 63028; 81 | optional bool compare_all = 63029; 82 | optional bool typedecl_all = 63030; 83 | optional bool enumdecl_all = 63031; 84 | 85 | optional bool goproto_registration = 63032; 86 | optional bool messagename_all = 63033; 87 | } 88 | 89 | extend google.protobuf.MessageOptions { 90 | optional bool goproto_getters = 64001; 91 | optional bool goproto_stringer = 64003; 92 | optional bool verbose_equal = 64004; 93 | optional bool face = 64005; 94 | optional bool gostring = 64006; 95 | optional bool populate = 64007; 96 | optional bool stringer = 67008; 97 | optional bool onlyone = 64009; 98 | 99 | optional bool equal = 64013; 100 | optional bool description = 64014; 101 | optional bool testgen = 64015; 102 | optional bool benchgen = 64016; 103 | optional bool marshaler = 64017; 104 | optional bool unmarshaler = 64018; 105 | optional bool stable_marshaler = 64019; 106 | 107 | optional bool sizer = 64020; 108 | 109 | optional bool unsafe_marshaler = 64023; 110 | optional bool unsafe_unmarshaler = 64024; 111 | 112 | optional bool goproto_extensions_map = 64025; 113 | optional bool goproto_unrecognized = 64026; 114 | 115 | optional bool protosizer = 64028; 116 | optional bool compare = 64029; 117 | 118 | optional bool typedecl = 64030; 119 | 120 | optional bool messagename = 64033; 121 | } 122 | 123 | extend google.protobuf.FieldOptions { 124 | optional bool nullable = 65001; 125 | optional bool embed = 65002; 126 | optional string customtype = 65003; 127 | optional string customname = 65004; 128 | optional string jsontag = 65005; 129 | optional string moretags = 65006; 130 | optional string casttype = 65007; 131 | optional string castkey = 65008; 132 | optional string castvalue = 65009; 133 | 134 | optional bool stdtime = 65010; 135 | optional bool stdduration = 65011; 136 | } 137 | -------------------------------------------------------------------------------- /proto/include/google/protobuf/duration.proto: -------------------------------------------------------------------------------- 1 | // Protocol Buffers - Google's data interchange format 2 | // Copyright 2008 Google Inc. All rights reserved. 3 | // https://developers.google.com/protocol-buffers/ 4 | // 5 | // Redistribution and use in source and binary forms, with or without 6 | // modification, are permitted provided that the following conditions are 7 | // met: 8 | // 9 | // * Redistributions of source code must retain the above copyright 10 | // notice, this list of conditions and the following disclaimer. 11 | // * Redistributions in binary form must reproduce the above 12 | // copyright notice, this list of conditions and the following disclaimer 13 | // in the documentation and/or other materials provided with the 14 | // distribution. 15 | // * Neither the name of Google Inc. nor the names of its 16 | // contributors may be used to endorse or promote products derived from 17 | // this software without specific prior written permission. 18 | // 19 | // THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 20 | // "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 21 | // LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 22 | // A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 23 | // OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 24 | // SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 25 | // LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 26 | // DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 27 | // THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 28 | // (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | // OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | 31 | syntax = "proto3"; 32 | 33 | package google.protobuf; 34 | 35 | option csharp_namespace = "Google.Protobuf.WellKnownTypes"; 36 | option cc_enable_arenas = true; 37 | option go_package = "types"; 38 | option java_package = "com.google.protobuf"; 39 | option java_outer_classname = "DurationProto"; 40 | option java_multiple_files = true; 41 | option objc_class_prefix = "GPB"; 42 | 43 | // A Duration represents a signed, fixed-length span of time represented 44 | // as a count of seconds and fractions of seconds at nanosecond 45 | // resolution. It is independent of any calendar and concepts like "day" 46 | // or "month". It is related to Timestamp in that the difference between 47 | // two Timestamp values is a Duration and it can be added or subtracted 48 | // from a Timestamp. Range is approximately +-10,000 years. 49 | // 50 | // # Examples 51 | // 52 | // Example 1: Compute Duration from two Timestamps in pseudo code. 53 | // 54 | // Timestamp start = ...; 55 | // Timestamp end = ...; 56 | // Duration duration = ...; 57 | // 58 | // duration.seconds = end.seconds - start.seconds; 59 | // duration.nanos = end.nanos - start.nanos; 60 | // 61 | // if (duration.seconds < 0 && duration.nanos > 0) { 62 | // duration.seconds += 1; 63 | // duration.nanos -= 1000000000; 64 | // } else if (durations.seconds > 0 && duration.nanos < 0) { 65 | // duration.seconds -= 1; 66 | // duration.nanos += 1000000000; 67 | // } 68 | // 69 | // Example 2: Compute Timestamp from Timestamp + Duration in pseudo code. 70 | // 71 | // Timestamp start = ...; 72 | // Duration duration = ...; 73 | // Timestamp end = ...; 74 | // 75 | // end.seconds = start.seconds + duration.seconds; 76 | // end.nanos = start.nanos + duration.nanos; 77 | // 78 | // if (end.nanos < 0) { 79 | // end.seconds -= 1; 80 | // end.nanos += 1000000000; 81 | // } else if (end.nanos >= 1000000000) { 82 | // end.seconds += 1; 83 | // end.nanos -= 1000000000; 84 | // } 85 | // 86 | // Example 3: Compute Duration from datetime.timedelta in Python. 87 | // 88 | // td = datetime.timedelta(days=3, minutes=10) 89 | // duration = Duration() 90 | // duration.FromTimedelta(td) 91 | // 92 | // # JSON Mapping 93 | // 94 | // In JSON format, the Duration type is encoded as a string rather than an 95 | // object, where the string ends in the suffix "s" (indicating seconds) and 96 | // is preceded by the number of seconds, with nanoseconds expressed as 97 | // fractional seconds. For example, 3 seconds with 0 nanoseconds should be 98 | // encoded in JSON format as "3s", while 3 seconds and 1 nanosecond should 99 | // be expressed in JSON format as "3.000000001s", and 3 seconds and 1 100 | // microsecond should be expressed in JSON format as "3.000001s". 101 | // 102 | // 103 | message Duration { 104 | 105 | // Signed seconds of the span of time. Must be from -315,576,000,000 106 | // to +315,576,000,000 inclusive. Note: these bounds are computed from: 107 | // 60 sec/min * 60 min/hr * 24 hr/day * 365.25 days/year * 10000 years 108 | int64 seconds = 1; 109 | 110 | // Signed fractions of a second at nanosecond resolution of the span 111 | // of time. Durations less than one second are represented with a 0 112 | // `seconds` field and a positive or negative `nanos` field. For durations 113 | // of one second or more, a non-zero value for the `nanos` field must be 114 | // of the same sign as the `seconds` field. Must be from -999,999,999 115 | // to +999,999,999 inclusive. 116 | int32 nanos = 2; 117 | } 118 | -------------------------------------------------------------------------------- /kv/storage/raft_storage/snap_runner.go: -------------------------------------------------------------------------------- 1 | package raft_storage 2 | 3 | import ( 4 | 5 | 6 | "bytes" 7 | "context" 8 | "io" 9 | "time" 10 | "google.golang.org/grpc" 11 | "google.golang.org/grpc/keepalive" 12 | "miniLinkDB/errors" 13 | 14 | "miniLinkDB/kv/raftstore/message" 15 | "miniLinkDB/kv/raftstore/snap" 16 | "miniLinkDB/kv/util/worker" 17 | 18 | _ "math" 19 | "miniLinkDB/kv/config" 20 | "miniLinkDB/log" 21 | "miniLinkDB/proto/pkg/linkkvpb" 22 | "miniLinkDB/proto/pkg/raft_serverpb" 23 | ) 24 | 25 | type sendSnapTask struct { 26 | addr string 27 | msg *raft_serverpb.RaftMessage 28 | callback func(err error) 29 | } 30 | 31 | type recvSnapTask struct { 32 | stream linkkvpb.LinkKv_SnapshotServer 33 | callback func(err error) 34 | } 35 | 36 | type snapRunner struct { 37 | config *config.Config 38 | snapManager *snap.SnapManager 39 | router message.RaftRouter 40 | } 41 | 42 | func newSnapRunner(snapManager *snap.SnapManager, config *config.Config, router message.RaftRouter) *snapRunner { 43 | return &snapRunner{ 44 | config: config, 45 | snapManager: snapManager, 46 | router: router, 47 | } 48 | } 49 | 50 | func (r *snapRunner) Handle(t worker.Task) { 51 | switch t.(type) { 52 | case *sendSnapTask: 53 | r.send(t.(*sendSnapTask)) 54 | case *recvSnapTask: 55 | r.recv(t.(*recvSnapTask)) 56 | } 57 | } 58 | 59 | func (r *snapRunner) send(t *sendSnapTask) { 60 | t.callback(r.sendSnap(t.addr, t.msg)) 61 | } 62 | 63 | const snapChunkLen = 1024 * 1024 64 | 65 | func (r *snapRunner) sendSnap(addr string, msg *raft_serverpb.RaftMessage) error { 66 | start := time.Now() 67 | msgSnap := msg.GetMessage().GetSnapshot() 68 | snapKey, err := snap.SnapKeyFromSnap(msgSnap) 69 | if err != nil { 70 | return err 71 | } 72 | 73 | r.snapManager.Register(snapKey, snap.SnapEntrySending) 74 | defer r.snapManager.Deregister(snapKey, snap.SnapEntrySending) 75 | 76 | snap, err := r.snapManager.GetSnapshotForSending(snapKey) 77 | if err != nil { 78 | return err 79 | } 80 | if !snap.Exists() { 81 | return errors.Errorf("missing snap file: %v", snap.Path()) 82 | } 83 | 84 | cc, err := grpc.Dial(addr, grpc.WithInsecure(), 85 | grpc.WithInitialWindowSize(2*1024*1024), 86 | grpc.WithKeepaliveParams(keepalive.ClientParameters{ 87 | Time: 3 * time.Second, 88 | Timeout: 60 * time.Second, 89 | })) 90 | if err != nil { 91 | return err 92 | } 93 | 94 | client := linkkvpb.NewLinkKvClient(cc) 95 | stream, err := client.Snapshot(context.TODO()) 96 | if err != nil { 97 | return err 98 | } 99 | err = stream.Send(&raft_serverpb.SnapshotChunk{Message : msg}) 100 | if err != nil { 101 | return err 102 | } 103 | 104 | buf := make([]byte, snapChunkLen) 105 | for remain := snap.TotalSize(); remain > 0; remain -= uint64(len(buf)) { 106 | if remain < uint64(len(buf)) { 107 | buf = buf[:remain] 108 | } 109 | _, err := io.ReadFull(snap, buf) 110 | if err != nil { 111 | return errors.Errorf("failed to read snapshot chunk: %v", err) 112 | } 113 | err = stream.Send(&raft_serverpb.SnapshotChunk{Data: buf}) 114 | if err != nil { 115 | return err 116 | } 117 | } 118 | _, err = stream.CloseAndRecv() 119 | if err != nil { 120 | return err 121 | } 122 | log.Infof("sent snapshot. regionID: %v, snapKey: %v, size: %v, duration: %s", snapKey.RegionID, snapKey, snap.TotalSize(), time.Since(start)) 123 | return nil 124 | } 125 | 126 | func (r *snapRunner) recv(t *recvSnapTask) { 127 | msg, err := r.recvSnap(t.stream) 128 | if err == nil { 129 | r.router.SendRaftMessage(msg) 130 | } 131 | t.callback(err) 132 | } 133 | 134 | func (r *snapRunner) recvSnap(stream linkkvpb.LinkKv_SnapshotServer) (*raft_serverpb.RaftMessage, error) { 135 | head, err := stream.Recv() 136 | if err != nil { 137 | return nil, err 138 | } 139 | if head.GetMessage() == nil { 140 | return nil, errors.New("no raft message in the first chunk") 141 | } 142 | message := head.GetMessage().GetMessage() 143 | snapKey, err := snap.SnapKeyFromSnap(message.GetSnapshot()) 144 | if err != nil { 145 | return nil, errors.Errorf("failed to create snap key: %v", err) 146 | } 147 | 148 | data := message.GetSnapshot().GetData() 149 | snapshot, err := r.snapManager.GetSnapshotForReceiving(snapKey, data) 150 | if err != nil { 151 | return nil, errors.Errorf("%v failed to create snapshot file: %v", snapKey, err) 152 | } 153 | if snapshot.Exists() { 154 | log.Infof("snapshot file already exists, skip receiving. snapKey: %v, file: %v", snapKey, snapshot.Path()) 155 | stream.SendAndClose(&raft_serverpb.Done{}) 156 | return head.GetMessage(), nil 157 | } 158 | r.snapManager.Register(snapKey, snap.SnapEntryReceiving) 159 | defer r.snapManager.Deregister(snapKey, snap.SnapEntryReceiving) 160 | 161 | for { 162 | chunk, err := stream.Recv() 163 | if err != nil { 164 | if err == io.EOF { 165 | break 166 | } 167 | return nil, err 168 | } 169 | data := chunk.GetData() 170 | if len(data) == 0 { 171 | return nil, errors.Errorf("%v receive chunk with empty data", snapKey) 172 | } 173 | _, err = bytes.NewReader(data).WriteTo(snapshot) 174 | if err != nil { 175 | return nil, errors.Errorf("%v failed to write snapshot file %v: %v", snapKey, snapshot.Path(), err) 176 | } 177 | } 178 | 179 | err = snapshot.Save() 180 | if err != nil { 181 | return nil, err 182 | } 183 | 184 | stream.SendAndClose(&raft_serverpb.Done{}) 185 | return head.GetMessage(), nil 186 | } 187 | -------------------------------------------------------------------------------- /kv/test_raftstore/node.go: -------------------------------------------------------------------------------- 1 | package test_raftstore 2 | 3 | import ( 4 | 5 | "context" 6 | "errors" 7 | "fmt" 8 | "io" 9 | "sync" 10 | "time" 11 | "github.com/Connor1996/badger" 12 | "miniLinkDB/kv/raftstore" 13 | "miniLinkDB/kv/raftstore/message" 14 | "miniLinkDB/kv/raftstore/scheduler_client" 15 | "miniLinkDB/kv/raftstore/snap" 16 | "miniLinkDB/kv/util/engine_util" 17 | "miniLinkDB/log" 18 | "miniLinkDB/proto/pkg/raft_cmdpb" 19 | "miniLinkDB/proto/pkg/raft_serverpb" 20 | "miniLinkDB/proto/pkg/eraftpb" 21 | "miniLinkDB/kv/config" 22 | 23 | ) 24 | 25 | type MockTransport struct { 26 | sync.RWMutex 27 | 28 | filters []Filter 29 | routers map[uint64]message.RaftRouter 30 | snapMgrs map[uint64]*snap.SnapManager 31 | } 32 | 33 | func NewMockTransport() *MockTransport { 34 | return &MockTransport{ 35 | routers: make(map[uint64]message.RaftRouter), 36 | snapMgrs: make(map[uint64]*snap.SnapManager), 37 | } 38 | } 39 | 40 | func (t *MockTransport) AddStore(storeID uint64, raftRouter message.RaftRouter, snapMgr *snap.SnapManager) { 41 | t.Lock() 42 | defer t.Unlock() 43 | 44 | t.routers[storeID] = raftRouter 45 | t.snapMgrs[storeID] = snapMgr 46 | } 47 | 48 | func (t *MockTransport) RemoveStore(storeID uint64) { 49 | t.Lock() 50 | defer t.Unlock() 51 | 52 | delete(t.routers, storeID) 53 | delete(t.snapMgrs, storeID) 54 | } 55 | 56 | func (t *MockTransport) AddFilter(filter Filter) { 57 | t.Lock() 58 | defer t.Unlock() 59 | 60 | t.filters = append(t.filters, filter) 61 | } 62 | 63 | func (t *MockTransport) ClearFilters() { 64 | t.Lock() 65 | defer t.Unlock() 66 | 67 | t.filters = nil 68 | } 69 | 70 | func (t *MockTransport) Send(msg *raft_serverpb.RaftMessage) error { 71 | t.RLock() 72 | defer t.RUnlock() 73 | 74 | for _, filter := range t.filters { 75 | if !filter.Before(msg) { 76 | return errors.New(fmt.Sprintf("message %+v is dropped", msg)) 77 | } 78 | } 79 | 80 | fromStore := msg.GetFromPeer().GetStoreId() 81 | toStore := msg.GetToPeer().GetStoreId() 82 | 83 | isSnapshot := msg.GetMessage().GetMsgType() == eraftpb.MessageType_MsgSnapshot 84 | if isSnapshot { 85 | snapshot := msg.Message.Snapshot 86 | key, err := snap.SnapKeyFromSnap(snapshot) 87 | if err != nil { 88 | return err 89 | } 90 | 91 | fromSnapMgr, found := t.snapMgrs[fromStore] 92 | if !found { 93 | return errors.New(fmt.Sprintf("store %d is closed", fromStore)) 94 | } 95 | fromSnapMgr.Register(key, snap.SnapEntrySending) 96 | fromSnap, err := fromSnapMgr.GetSnapshotForSending(key) 97 | if err != nil { 98 | return err 99 | } 100 | 101 | toSnapMgr, found := t.snapMgrs[toStore] 102 | if !found { 103 | return errors.New(fmt.Sprintf("store %d is closed", toStore)) 104 | } 105 | toSnapMgr.Register(key, snap.SnapEntryReceiving) 106 | toSnap, err := toSnapMgr.GetSnapshotForReceiving(key, snapshot.GetData()) 107 | if err != nil { 108 | return err 109 | } 110 | 111 | io.Copy(toSnap, fromSnap) 112 | toSnap.Save() 113 | 114 | toSnapMgr.Deregister(key, snap.SnapEntryReceiving) 115 | fromSnapMgr.Deregister(key, snap.SnapEntrySending) 116 | } 117 | 118 | router, found := t.routers[toStore] 119 | if !found { 120 | return errors.New(fmt.Sprintf("store %d is closed", toStore)) 121 | } 122 | router.SendRaftMessage(msg) 123 | 124 | for _, filter := range t.filters { 125 | filter.After() 126 | } 127 | 128 | return nil 129 | } 130 | 131 | type NodeSimulator struct { 132 | sync.RWMutex 133 | 134 | trans *MockTransport 135 | schedulerClient scheduler_client.Client 136 | nodes map[uint64]*raftstore.Node 137 | } 138 | 139 | func NewNodeSimulator(schedulerClient scheduler_client.Client) *NodeSimulator { 140 | trans := NewMockTransport() 141 | return &NodeSimulator{ 142 | trans: trans, 143 | schedulerClient: schedulerClient, 144 | nodes: make(map[uint64]*raftstore.Node), 145 | } 146 | } 147 | 148 | func (c *NodeSimulator) RunStore(cfg *config.Config, engine *engine_util.Engines, ctx context.Context) error { 149 | c.Lock() 150 | defer c.Unlock() 151 | 152 | raftRouter, raftSystem := raftstore.CreateRaftstore(cfg) 153 | snapManager := snap.NewSnapManager(cfg.DBPath + "/snap") 154 | node := raftstore.NewNode(raftSystem, cfg, c.schedulerClient) 155 | 156 | err := node.Start(ctx, engine, c.trans, snapManager) 157 | if err != nil { 158 | return err 159 | } 160 | 161 | storeID := node.GetStoreID() 162 | c.nodes[storeID] = node 163 | c.trans.AddStore(storeID, raftRouter, snapManager) 164 | 165 | return nil 166 | } 167 | 168 | func (c *NodeSimulator) StopStore(storeID uint64) { 169 | c.Lock() 170 | defer c.Unlock() 171 | 172 | node := c.nodes[storeID] 173 | if node == nil { 174 | panic(fmt.Sprintf("Can not find store %d", storeID)) 175 | } 176 | node.Stop() 177 | delete(c.nodes, storeID) 178 | c.trans.RemoveStore(storeID) 179 | } 180 | 181 | func (c *NodeSimulator) AddFilter(filter Filter) { 182 | c.Lock() 183 | defer c.Unlock() 184 | c.trans.AddFilter(filter) 185 | } 186 | 187 | func (c *NodeSimulator) ClearFilters() { 188 | c.Lock() 189 | defer c.Unlock() 190 | c.trans.ClearFilters() 191 | } 192 | 193 | func (c *NodeSimulator) GetStoreIds() []uint64 { 194 | c.RLock() 195 | defer c.RUnlock() 196 | storeIDs := make([]uint64, 0, len(c.nodes)) 197 | for storeID := range c.nodes { 198 | storeIDs = append(storeIDs, storeID) 199 | } 200 | return storeIDs 201 | } 202 | 203 | func (c *NodeSimulator) CallCommandOnStore(storeID uint64, request *raft_cmdpb.RaftCmdRequest, timeout time.Duration) (*raft_cmdpb.RaftCmdResponse, *badger.Txn) { 204 | c.RLock() 205 | router := c.trans.routers[storeID] 206 | if router == nil { 207 | log.Fatalf("Can not find node %d", storeID) 208 | } 209 | c.RUnlock() 210 | 211 | cb := message.NewCallback() 212 | err := router.SendRaftCommand(request, cb) 213 | if err != nil { 214 | return nil, nil 215 | } 216 | 217 | resp := cb.WaitRespWithTimeout(timeout) 218 | return resp, cb.Txn 219 | } 220 | -------------------------------------------------------------------------------- /raft/rawnode.go: -------------------------------------------------------------------------------- 1 | // Copyright 2015 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package raft 16 | 17 | import ( 18 | "errors" 19 | 20 | pb "miniLinkDB/proto/pkg/eraftpb" 21 | ) 22 | 23 | // ErrStepLocalMsg is returned when try to step a local raft message 24 | var ErrStepLocalMsg = errors.New("raft: cannot step raft local message") 25 | 26 | // ErrStepPeerNotFound is returned when try to step a response message 27 | // but there is no peer found in raft.Prs for that node. 28 | var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found") 29 | 30 | // SoftState provides state that is volatile and does not need to be persisted to the WAL. 31 | type SoftState struct { 32 | Lead uint64 33 | RaftState StateType 34 | } 35 | 36 | // Ready encapsulates the entries and messages that are ready to read, 37 | // be saved to stable storage, committed or sent to other peers. 38 | // All fields in Ready are read-only. 39 | type Ready struct { 40 | // The current volatile state of a Node. 41 | // SoftState will be nil if there is no update. 42 | // It is not required to consume or store SoftState. 43 | *SoftState 44 | 45 | // The current state of a Node to be saved to stable storage BEFORE 46 | // Messages are sent. 47 | // HardState will be equal to empty state if there is no update. 48 | pb.HardState 49 | 50 | // Entries specifies entries to be saved to stable storage BEFORE 51 | // Messages are sent. 52 | Entries []pb.Entry 53 | 54 | // Snapshot specifies the snapshot to be saved to stable storage. 55 | Snapshot pb.Snapshot 56 | 57 | // CommittedEntries specifies entries to be committed to a 58 | // store/state-machine. These have previously been committed to stable 59 | // store. 60 | CommittedEntries []pb.Entry 61 | 62 | // Messages specifies outbound messages to be sent AFTER Entries are 63 | // committed to stable storage. 64 | // If it contains a MessageType_MsgSnapshot message, the application MUST report back to raft 65 | // when the snapshot has been received or has failed by calling ReportSnapshot. 66 | Messages []pb.Message 67 | } 68 | 69 | // RawNode is a wrapper of Raft. 70 | type RawNode struct { 71 | Raft *Raft 72 | // Your Data Here (2A). 73 | } 74 | 75 | // NewRawNode returns a new RawNode given configuration and a list of raft peers. 76 | func NewRawNode(config *Config) (*RawNode, error) { 77 | 78 | return nil, nil 79 | } 80 | 81 | // Tick advances the internal logical clock by a single tick. 82 | func (rn *RawNode) Tick() { 83 | rn.Raft.tick() 84 | } 85 | 86 | // Campaign causes this RawNode to transition to candidate state. 87 | func (rn *RawNode) Campaign() error { 88 | return rn.Raft.Step(pb.Message{ 89 | MsgType: pb.MessageType_MsgHup, 90 | }) 91 | } 92 | 93 | // Propose proposes data be appended to the raft log. 94 | func (rn *RawNode) Propose(data []byte) error { 95 | ent := pb.Entry{Data: data} 96 | return rn.Raft.Step(pb.Message{ 97 | MsgType: pb.MessageType_MsgPropose, 98 | From: rn.Raft.id, 99 | Entries: []*pb.Entry{&ent}}) 100 | } 101 | 102 | // ProposeConfChange proposes a config change. 103 | func (rn *RawNode) ProposeConfChange(cc pb.ConfChange) error { 104 | data, err := cc.Marshal() 105 | if err != nil { 106 | return err 107 | } 108 | ent := pb.Entry{EntryType: pb.EntryType_EntryConfChange, Data: data} 109 | return rn.Raft.Step(pb.Message{ 110 | MsgType: pb.MessageType_MsgPropose, 111 | Entries: []*pb.Entry{&ent}, 112 | }) 113 | } 114 | 115 | // ApplyConfChange applies a config change to the local node. 116 | func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState { 117 | if cc.NodeId == None { 118 | return &pb.ConfState{Nodes: nodes(rn.Raft)} 119 | } 120 | switch cc.ChangeType { 121 | case pb.ConfChangeType_AddNode: 122 | rn.Raft.addNode(cc.NodeId) 123 | case pb.ConfChangeType_RemoveNode: 124 | rn.Raft.removeNode(cc.NodeId) 125 | default: 126 | panic("unexpected conf type") 127 | } 128 | return &pb.ConfState{Nodes: nodes(rn.Raft)} 129 | } 130 | 131 | // Step advances the state machine using the given message. 132 | func (rn *RawNode) Step(m pb.Message) error { 133 | // ignore unexpected local messages receiving over network 134 | if IsLocalMsg(m.MsgType) { 135 | return ErrStepLocalMsg 136 | } 137 | if pr := rn.Raft.Prs[m.From]; pr != nil || !IsResponseMsg(m.MsgType) { 138 | return rn.Raft.Step(m) 139 | } 140 | return ErrStepPeerNotFound 141 | } 142 | 143 | // Ready returns the current point-in-time state of this RawNode. 144 | func (rn *RawNode) Ready() Ready { 145 | 146 | return Ready{} 147 | } 148 | 149 | // HasReady called when RawNode user need to check if any Ready pending. 150 | func (rn *RawNode) HasReady() bool { 151 | 152 | return false 153 | } 154 | 155 | // Advance notifies the RawNode that the application has applied and saved progress in the 156 | // last Ready results. 157 | func (rn *RawNode) Advance(rd Ready) { 158 | 159 | } 160 | 161 | // GetProgress return the the Progress of this node and its peers, if this 162 | // node is leader. 163 | func (rn *RawNode) GetProgress() map[uint64]Progress { 164 | prs := make(map[uint64]Progress) 165 | if rn.Raft.State == StateLeader { 166 | for id, p := range rn.Raft.Prs { 167 | prs[id] = *p 168 | } 169 | } 170 | return prs 171 | } 172 | 173 | // TransferLeader tries to transfer leadership to the given transferee. 174 | func (rn *RawNode) TransferLeader(transferee uint64) { 175 | _ = rn.Raft.Step(pb.Message{MsgType: pb.MessageType_MsgTransferLeader, From: transferee}) 176 | } 177 | -------------------------------------------------------------------------------- /kv/coprocessor/rowcodec/common.go: -------------------------------------------------------------------------------- 1 | package rowcodec 2 | 3 | import ( 4 | "encoding/binary" 5 | "fmt" 6 | "github.com/juju/errors" 7 | "reflect" 8 | "strings" 9 | "unsafe" 10 | ) 11 | 12 | // CodecVer is the constant number that represent the new row format. 13 | const CodecVer = 128 14 | 15 | var invalidCodecVer = errors.New("invalid codec version") 16 | 17 | // First byte in the encoded value which specifies the encoding type. 18 | const ( 19 | NilFalg byte = 0 20 | BytesFlag byte = 1 21 | CompactBytesFlag byte = 2 22 | IntFlag byte = 3 23 | UintFlag byte = 4 24 | VarintFlag byte = 8 25 | VaruintFlag byte = 9 26 | ) 27 | 28 | 29 | // row is the struct type used to access the a row. 30 | type row struct { 31 | // small: colID []byte, offsets []uint16, optimized for most cases. 32 | // large: colID []uint32, offsets []uint32. 33 | large bool 34 | numNotNullCols uint16 35 | numNullCols uint16 36 | colIDs []byte 37 | 38 | offsets []uint16 39 | data []byte 40 | 41 | // for large row 42 | colIDs32 []uint32 43 | offsets32 []uint32 44 | } 45 | // String implements the strings.Stringer interface. 46 | func (r row) String() string { 47 | var colValStrs []string 48 | for i := 0; i < int(r.numNotNullCols); i++ { 49 | var colID, offStart, offEnd int64 50 | if r.large { 51 | colID = int64(r.colIDs32[i]) 52 | if i != 0 { 53 | offStart = int64(r.offsets32[i-1]) 54 | } 55 | offEnd = int64(r.offsets32[i]) 56 | } else { 57 | colID = int64(r.colIDs[i]) 58 | if i != 0 { 59 | offStart = int64(r.offsets[i-1]) 60 | } 61 | offEnd = int64(r.offsets[i]) 62 | } 63 | colValData := r.data[offStart:offEnd] 64 | colValStr := fmt.Sprintf("(%d:%v)", colID, colValData) 65 | colValStrs = append(colValStrs, colValStr) 66 | } 67 | return strings.Join(colValStrs, ",") 68 | } 69 | 70 | func (r *row) getData(i int) []byte { 71 | var start, end uint32 72 | if r.large { 73 | if i > 0 { 74 | start = r.offsets32[i-1] 75 | } 76 | end = r.offsets32[i] 77 | } else { 78 | if i > 0 { 79 | start = uint32(r.offsets[i-1]) 80 | } 81 | end = uint32(r.offsets[i]) 82 | } 83 | return r.data[start:end] 84 | } 85 | 86 | func (r *row) setRowData(rowData []byte) error { 87 | if rowData[0] != CodecVer { 88 | return invalidCodecVer 89 | } 90 | r.large = rowData[1]&1 > 0 91 | r.numNotNullCols = binary.LittleEndian.Uint16(rowData[2:]) 92 | r.numNullCols = binary.LittleEndian.Uint16(rowData[4:]) 93 | cursor := 6 94 | if r.large { 95 | colIDsLen := int(r.numNotNullCols+r.numNullCols) * 4 96 | r.colIDs32 = bytesToU32Slice(rowData[cursor : cursor+colIDsLen]) 97 | cursor += colIDsLen 98 | offsetsLen := int(r.numNotNullCols) * 4 99 | r.offsets32 = bytesToU32Slice(rowData[cursor : cursor+offsetsLen]) 100 | cursor += offsetsLen 101 | } else { 102 | colIDsLen := int(r.numNotNullCols + r.numNullCols) 103 | r.colIDs = rowData[cursor : cursor+colIDsLen] 104 | cursor += colIDsLen 105 | offsetsLen := int(r.numNotNullCols) * 2 106 | r.offsets = bytes2U16Slice(rowData[cursor : cursor+offsetsLen]) 107 | cursor += offsetsLen 108 | } 109 | r.data = rowData[cursor:] 110 | return nil 111 | } 112 | 113 | func bytesToU32Slice(b []byte) []uint32 { 114 | if len(b) == 0 { 115 | return nil 116 | } 117 | var u32s []uint32 118 | hdr := (*reflect.SliceHeader)(unsafe.Pointer(&u32s)) 119 | hdr.Len = len(b) / 4 120 | hdr.Cap = hdr.Len 121 | hdr.Data = uintptr(unsafe.Pointer(&b[0])) 122 | return u32s 123 | } 124 | 125 | func bytes2U16Slice(b []byte) []uint16 { 126 | if len(b) == 0 { 127 | return nil 128 | } 129 | var u16s []uint16 130 | hdr := (*reflect.SliceHeader)(unsafe.Pointer(&u16s)) 131 | hdr.Len = len(b) / 2 132 | hdr.Cap = hdr.Len 133 | hdr.Data = uintptr(unsafe.Pointer(&b[0])) 134 | return u16s 135 | } 136 | 137 | 138 | func u16SliceToBytes(u16s []uint16) []byte { 139 | if len(u16s) == 0 { 140 | return nil 141 | } 142 | var b []byte 143 | hdr := (*reflect.SliceHeader)(unsafe.Pointer(&b)) 144 | hdr.Len = len(u16s) * 2 145 | hdr.Cap = hdr.Len 146 | hdr.Data = uintptr(unsafe.Pointer(&u16s[0])) 147 | return b 148 | } 149 | 150 | func u32SliceToBytes(u32s []uint32) []byte { 151 | if len(u32s) == 0 { 152 | return nil 153 | } 154 | var b []byte 155 | hdr := (*reflect.SliceHeader)(unsafe.Pointer(&b)) 156 | hdr.Len = len(u32s) * 4 157 | hdr.Cap = hdr.Len 158 | hdr.Data = uintptr(unsafe.Pointer(&u32s[0])) 159 | return b 160 | } 161 | 162 | func encodeInt(buf []byte, iVal int64) []byte { 163 | var tmp [8]byte 164 | if int64(int8(iVal)) == iVal { 165 | buf = append(buf, byte(iVal)) 166 | } else if int64(int16(iVal)) == iVal { 167 | binary.LittleEndian.PutUint16(tmp[:], uint16(iVal)) 168 | buf = append(buf, tmp[:2]...) 169 | } else if int64(int32(iVal)) == iVal { 170 | binary.LittleEndian.PutUint32(tmp[:], uint32(iVal)) 171 | buf = append(buf, tmp[:4]...) 172 | } else { 173 | binary.LittleEndian.PutUint64(tmp[:], uint64(iVal)) 174 | buf = append(buf, tmp[:8]...) 175 | } 176 | return buf 177 | } 178 | 179 | func decodeInt(val []byte) int64 { 180 | switch len(val) { 181 | case 1: 182 | return int64(int8(val[0])) 183 | case 2: 184 | return int64(int16(binary.LittleEndian.Uint16(val))) 185 | case 4: 186 | return int64(int32(binary.LittleEndian.Uint32(val))) 187 | default: 188 | return int64(binary.LittleEndian.Uint64(val)) 189 | } 190 | } 191 | 192 | func encodeUint(buf []byte, uVal uint64) []byte { 193 | var tmp [8]byte 194 | if uint64(uint8(uVal)) == uVal { 195 | buf = append(buf, byte(uVal)) 196 | } else if uint64(uint16(uVal)) == uVal { 197 | binary.LittleEndian.PutUint16(tmp[:], uint16(uVal)) 198 | buf = append(buf, tmp[:2]...) 199 | } else if uint64(uint32(uVal)) == uVal { 200 | binary.LittleEndian.PutUint32(tmp[:], uint32(uVal)) 201 | buf = append(buf, tmp[:4]...) 202 | } else { 203 | binary.LittleEndian.PutUint64(tmp[:], uint64(uVal)) 204 | buf = append(buf, tmp[:8]...) 205 | } 206 | return buf 207 | } 208 | 209 | func decodeUint(val []byte) uint64 { 210 | switch len(val) { 211 | case 1: 212 | return uint64(val[0]) 213 | case 2: 214 | return uint64(binary.LittleEndian.Uint16(val)) 215 | case 4: 216 | return uint64(binary.LittleEndian.Uint32(val)) 217 | default: 218 | return binary.LittleEndian.Uint64(val) 219 | } 220 | } 221 | -------------------------------------------------------------------------------- /kv/storage/mem_storage.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "bytes" 5 | "fmt" 6 | 7 | "github.com/Connor1996/badger/y" 8 | "github.com/petar/GoLLRB/llrb" 9 | "miniLinkDB/kv/util/engine_util" 10 | "miniLinkDB/proto/pkg/kvrpcpb" 11 | ) 12 | 13 | // MemStorage used test memory storage engine,data not write to disk, not sent to disk 14 | 15 | type MemStorage struct { 16 | CfDefault *llrb.LLRB 17 | CfLock *llrb.LLRB 18 | CfWrite *llrb.LLRB 19 | } 20 | 21 | func NewMemStorage() *MemStorage { 22 | return &MemStorage{ 23 | CfDefault: llrb.New(), 24 | CfLock: llrb.New(), 25 | CfWrite: llrb.New(), 26 | } 27 | } 28 | 29 | func (s *MemStorage) Start() error { 30 | return nil 31 | } 32 | 33 | func (s *MemStorage) Stop() error { 34 | return nil 35 | } 36 | 37 | func (s *MemStorage) Reader(ctx *kvrpcpb.Context) (StorageReader, error) { 38 | return &memReader{s, 0}, nil 39 | } 40 | 41 | func (s *MemStorage) Write(ctx *kvrpcpb.Context, batch []Modify) error { 42 | for _, m := range batch { 43 | switch data := m.Data.(type) { 44 | case Put: 45 | item := memItem{data.Key, data.Value, false} 46 | switch data.Cf { 47 | case engine_util.CfDefault: 48 | s.CfDefault.ReplaceOrInsert(item) 49 | case engine_util.CfLock: 50 | s.CfLock.ReplaceOrInsert(item) 51 | case engine_util.CfWrite: 52 | s.CfWrite.ReplaceOrInsert(item) 53 | } 54 | case Delete: 55 | item := memItem{key: data.Key} 56 | switch data.Cf { 57 | case engine_util.CfDefault: 58 | s.CfDefault.Delete(item) 59 | case engine_util.CfLock: 60 | s.CfLock.Delete(item) 61 | case engine_util.CfWrite: 62 | s.CfWrite.Delete(item) 63 | } 64 | } 65 | } 66 | 67 | return nil 68 | } 69 | 70 | func (s *MemStorage) Get(cf string, key []byte) []byte { 71 | item := memItem{key: key} 72 | var result llrb.Item 73 | switch cf { 74 | case engine_util.CfDefault: 75 | result = s.CfDefault.Get(item) 76 | case engine_util.CfLock: 77 | result = s.CfLock.Get(item) 78 | case engine_util.CfWrite: 79 | result = s.CfWrite.Get(item) 80 | } 81 | 82 | if result == nil { 83 | return nil 84 | } 85 | 86 | return result.(memItem).value 87 | } 88 | 89 | func (s *MemStorage) Set(cf string, key []byte, value []byte) { 90 | item := memItem{key, value, true} 91 | switch cf { 92 | case engine_util.CfDefault: 93 | s.CfDefault.ReplaceOrInsert(item) 94 | case engine_util.CfLock: 95 | s.CfLock.ReplaceOrInsert(item) 96 | case engine_util.CfWrite: 97 | s.CfWrite.ReplaceOrInsert(item) 98 | } 99 | } 100 | 101 | func (s *MemStorage) HasChanged(cf string, key []byte) bool { 102 | item := memItem{key: key} 103 | var result llrb.Item 104 | switch cf { 105 | case engine_util.CfDefault: 106 | result = s.CfDefault.Get(item) 107 | case engine_util.CfLock: 108 | result = s.CfLock.Get(item) 109 | case engine_util.CfWrite: 110 | result = s.CfWrite.Get(item) 111 | } 112 | if result == nil { 113 | return true 114 | } 115 | 116 | return !result.(memItem).fresh 117 | } 118 | 119 | 120 | 121 | func (s *MemStorage) Len(cf string) int { 122 | switch cf { 123 | case engine_util.CfDefault: 124 | return s.CfDefault.Len() 125 | case engine_util.CfLock: 126 | return s.CfLock.Len() 127 | case engine_util.CfWrite: 128 | return s.CfWrite.Len() 129 | } 130 | 131 | return -1 132 | } 133 | 134 | 135 | 136 | 137 | // memReader 是从 MemStorage 读取的 StorageReader 138 | type memReader struct { 139 | inner *MemStorage 140 | iterCount int 141 | } 142 | 143 | func (mr *memReader) GetCF(cf string, key []byte) ([]byte, error) { 144 | item := memItem{key: key} 145 | var result llrb.Item 146 | switch cf { 147 | case engine_util.CfDefault: 148 | result = mr.inner.CfDefault.Get(item) 149 | case engine_util.CfLock: 150 | result = mr.inner.CfLock.Get(item) 151 | case engine_util.CfWrite: 152 | result = mr.inner.CfWrite.Get(item) 153 | default: 154 | return nil, fmt.Errorf("mem-server: bad CF #{cf}") 155 | } 156 | 157 | if result == nil { 158 | return nil, nil 159 | } 160 | 161 | return result.(memItem).value, nil 162 | } 163 | 164 | func (mr *memReader) IterCF(cf string) engine_util.DBIterator { 165 | var data *llrb.LLRB 166 | switch cf { 167 | case engine_util.CfDefault: 168 | data = mr.inner.CfDefault 169 | case engine_util.CfLock: 170 | data = mr.inner.CfLock 171 | case engine_util.CfWrite: 172 | data = mr.inner.CfWrite 173 | default: 174 | return nil 175 | } 176 | 177 | mr.iterCount += 1 178 | min := data.Min() 179 | if min == nil { 180 | return &memIter{data, memItem{}, mr} 181 | } 182 | return &memIter{data, min.(memItem), mr} 183 | } 184 | 185 | 186 | func (r *memReader) Close() { 187 | if r.iterCount > 0 { 188 | panic("Unclosed iterator") 189 | } 190 | } 191 | 192 | type memIter struct { 193 | data *llrb.LLRB 194 | item memItem 195 | reader *memReader 196 | } 197 | 198 | func (it *memIter) Item() engine_util.DBItem { 199 | return it.item 200 | } 201 | 202 | func (it *memIter) Valid() bool { 203 | return it.item.key != nil 204 | } 205 | 206 | func (it *memIter) Next() { 207 | first := true 208 | oldItem := it.item 209 | it.item = memItem{} 210 | it.data.AscendGreaterOrEqual(oldItem, func(item llrb.Item) bool { 211 | // 212 | if first { 213 | first = false 214 | return true 215 | } 216 | 217 | it.item = item.(memItem) 218 | return false 219 | }) 220 | } 221 | 222 | func (it *memIter) Seek(key []byte) { 223 | it.item = memItem{} 224 | it.data.AscendGreaterOrEqual(memItem{key: key}, func(item llrb.Item) bool { 225 | it.item = item.(memItem) 226 | 227 | return false 228 | }) 229 | 230 | } 231 | 232 | func (it *memIter) Close() { 233 | it.reader.iterCount -= 1 234 | } 235 | 236 | 237 | type memItem struct { 238 | key []byte 239 | value []byte 240 | fresh bool 241 | } 242 | 243 | func (it memItem) Key() []byte { 244 | return it.key 245 | } 246 | 247 | func (it memItem) KeyCopy(dst []byte) []byte { 248 | return y.SafeCopy(dst, it.key) 249 | } 250 | 251 | 252 | func (it memItem) Value() ([]byte, error) { 253 | return it.value, nil 254 | } 255 | 256 | func (it memItem) ValueSize() int { 257 | return len(it.value) 258 | } 259 | 260 | func (it memItem) ValueCopy(dst []byte) ([]byte, error) { 261 | return y.SafeCopy(dst, it.value), nil 262 | } 263 | 264 | func (it memItem) Less(than llrb.Item) bool { 265 | other := than.(memItem) 266 | return bytes.Compare(it.key, other.key) < 0 267 | } -------------------------------------------------------------------------------- /kv/coprocessor/rowcodec/decoder.go: -------------------------------------------------------------------------------- 1 | package rowcodec 2 | 3 | import ( 4 | "github.com/juju/errors" 5 | "github.com/pingcap/parser/mysql" 6 | "github.com/pingcap/tidb/types" 7 | "github.com/pingcap/tidb/util/chunk" 8 | "github.com/pingcap/tidb/util/codec" 9 | "math" 10 | "time" 11 | ) 12 | 13 | type Decoder struct { 14 | row 15 | requestColIDs []int64 16 | handleColID int64 17 | requestTypes []*types.FieldType 18 | origDefaults [][]byte 19 | loc *time.Location 20 | } 21 | 22 | 23 | 24 | // NewDecoder creates a NewDecoder. 25 | // requestColIDs is the columnIDs to decode. tps is the field types for request columns. 26 | // origDefault is the original default value in old format, if the column ID is not found in the row, 27 | // the origDefault will be used. 28 | func NewDecoder(requestColIDs []int64, handleColID int64, tps []*types.FieldType, origDefaults [][]byte, 29 | loc *time.Location) (*Decoder, error) { 30 | xOrigDefaultVals := make([][]byte, len(origDefaults)) 31 | for i := 0; i < len(origDefaults); i++ { 32 | if len(origDefaults[i]) == 0 { 33 | continue 34 | } 35 | xDefaultVal, err := convertDefaultValue(origDefaults[i]) 36 | if err != nil { 37 | return nil, err 38 | } 39 | xOrigDefaultVals[i] = xDefaultVal 40 | } 41 | return &Decoder{ 42 | requestColIDs: requestColIDs, 43 | handleColID: handleColID, 44 | requestTypes: tps, 45 | origDefaults: xOrigDefaultVals, 46 | loc: loc, 47 | }, nil 48 | } 49 | 50 | 51 | func convertDefaultValue(defaultVal []byte) (colVal []byte, err error) { 52 | var d types.Datum 53 | _, d, err = codec.DecodeOne(defaultVal) 54 | if err != nil { 55 | return 56 | } 57 | switch d.Kind() { 58 | case types.KindNull: 59 | return nil, nil 60 | case types.KindInt64: 61 | return encodeInt(nil, d.GetInt64()), nil 62 | case types.KindUint64: 63 | return encodeUint(nil, d.GetUint64()), nil 64 | case types.KindString, types.KindBytes: 65 | return d.GetBytes(), nil 66 | case types.KindFloat32: 67 | return encodeUint(nil, uint64(math.Float32bits(d.GetFloat32()))), nil 68 | case types.KindFloat64: 69 | return encodeUint(nil, math.Float64bits(d.GetFloat64())), nil 70 | default: 71 | return defaultVal[1:], nil 72 | } 73 | } 74 | 75 | // Decode decodes a row to chunk. 76 | func (decoder *Decoder) Decode(rowData []byte, handle int64, chk *chunk.Chunk) error { 77 | 78 | err := decoder.setRowData(rowData) 79 | if err != nil { 80 | return err 81 | } 82 | for colIdx, colID := range decoder.requestColIDs { 83 | if colID == decoder.handleColID { 84 | chk.AppendInt64(colIdx, handle) 85 | continue 86 | } 87 | // Search the column in not-null columns array. 88 | i, j := 0, int(decoder.numNotNullCols) 89 | var found bool 90 | for i < j { 91 | h := int(uint(i+j) >> 1) // avoid overflow when computing h 92 | // i ≤ h < j 93 | var v int64 94 | if decoder.large { 95 | v = int64(decoder.colIDs32[h]) 96 | } else { 97 | v = int64(decoder.colIDs[h]) 98 | } 99 | if v < colID { 100 | i = h + 1 101 | } else if v > colID { 102 | j = h 103 | } else { 104 | found = true 105 | colData := decoder.getData(h) 106 | err := decoder.decodeColData(colIdx, colData, chk) 107 | if err != nil { 108 | return err 109 | } 110 | break 111 | } 112 | } 113 | if found { 114 | continue 115 | } 116 | defaultVal := decoder.origDefaults[colIdx] 117 | if decoder.isNull(colID, defaultVal) { 118 | chk.AppendNull(colIdx) 119 | } else { 120 | err := decoder.decodeColData(colIdx, defaultVal, chk) 121 | if err != nil { 122 | return err 123 | } 124 | } 125 | } 126 | return nil 127 | } 128 | 129 | // ColumnIsNull returns if the column value is null. Mainly used for count column aggregation. 130 | func (decoder *Decoder) ColumnIsNull(rowData []byte, colID int64, defaultVal []byte) (bool, error) { 131 | err := decoder.setRowData(rowData) 132 | if err != nil { 133 | return false, err 134 | } 135 | // Search the column in not-null columns array. 136 | i, j := 0, int(decoder.numNotNullCols) 137 | for i < j { 138 | h := int(uint(i+j) >> 1) // avoid overflow when computing h 139 | // i ≤ h < j 140 | var v int64 141 | if decoder.large { 142 | v = int64(decoder.colIDs32[h]) 143 | } else { 144 | v = int64(decoder.colIDs[h]) 145 | } 146 | if v < colID { 147 | i = h + 1 148 | } else if v > colID { 149 | j = h 150 | } else { 151 | return false, nil 152 | } 153 | } 154 | return decoder.isNull(colID, defaultVal), nil 155 | } 156 | 157 | func (decoder *Decoder) isNull(colID int64, defaultVal []byte) bool { 158 | // Search the column in null columns array. 159 | i, j := int(decoder.numNotNullCols), int(decoder.numNotNullCols+decoder.numNullCols) 160 | for i < j { 161 | h := int(uint(i+j) >> 1) // avoid overflow when computing h 162 | // i ≤ h < j 163 | var v int64 164 | if decoder.large { 165 | v = int64(decoder.colIDs32[h]) 166 | } else { 167 | v = int64(decoder.colIDs[h]) 168 | } 169 | if v < colID { 170 | i = h + 1 171 | } else if v > colID { 172 | j = h 173 | } else { 174 | return true 175 | } 176 | } 177 | return defaultVal == nil 178 | } 179 | 180 | func (decoder *Decoder) decodeColData(colIdx int, colData []byte, chk *chunk.Chunk) error { 181 | ft := decoder.requestTypes[colIdx] 182 | switch ft.Tp { 183 | case mysql.TypeLonglong, mysql.TypeLong, mysql.TypeInt24, mysql.TypeShort, mysql.TypeTiny, mysql.TypeYear: 184 | if mysql.HasUnsignedFlag(ft.Flag) { 185 | chk.AppendUint64(colIdx, decodeUint(colData)) 186 | } else { 187 | chk.AppendInt64(colIdx, decodeInt(colData)) 188 | } 189 | case mysql.TypeFloat: 190 | _, fVal, err := codec.DecodeFloat(colData) 191 | if err != nil { 192 | return err 193 | } 194 | chk.AppendFloat32(colIdx, float32(fVal)) 195 | case mysql.TypeDouble: 196 | _, fVal, err := codec.DecodeFloat(colData) 197 | if err != nil { 198 | return err 199 | } 200 | chk.AppendFloat64(colIdx, fVal) 201 | case mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeString, 202 | mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: 203 | chk.AppendBytes(colIdx, colData) 204 | case mysql.TypeBit: 205 | byteSize := (ft.Flen + 7) >> 3 206 | chk.AppendBytes(colIdx, types.NewBinaryLiteralFromUint(decodeUint(colData), byteSize)) 207 | default: 208 | return errors.Errorf("unknown type %d", ft.Tp) 209 | } 210 | return nil 211 | } -------------------------------------------------------------------------------- /errors/example_test.go: -------------------------------------------------------------------------------- 1 | package errors_test 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/pkg/errors" 7 | ) 8 | 9 | func ExampleNew() { 10 | err := errors.New("whoops") 11 | fmt.Println(err) 12 | 13 | // Output: whoops 14 | } 15 | 16 | func ExampleNew_printf() { 17 | err := errors.New("whoops") 18 | fmt.Printf("%+v", err) 19 | 20 | // Example output: 21 | // whoops 22 | // github.com/pkg/errors_test.ExampleNew_printf 23 | // /home/dfc/src/github.com/pkg/errors/example_test.go:17 24 | // testing.runExample 25 | // /home/dfc/go/src/testing/example.go:114 26 | // testing.RunExamples 27 | // /home/dfc/go/src/testing/example.go:38 28 | // testing.(*M).Run 29 | // /home/dfc/go/src/testing/testing.go:744 30 | // main.main 31 | // /github.com/pkg/errors/_test/_testmain.go:106 32 | // runtime.main 33 | // /home/dfc/go/src/runtime/proc.go:183 34 | // runtime.goexit 35 | // /home/dfc/go/src/runtime/asm_amd64.s:2059 36 | } 37 | 38 | func ExampleWithMessage() { 39 | cause := errors.New("whoops") 40 | err := errors.WithMessage(cause, "oh noes") 41 | fmt.Println(err) 42 | 43 | // Output: oh noes: whoops 44 | } 45 | 46 | func ExampleWithStack() { 47 | cause := errors.New("whoops") 48 | err := errors.WithStack(cause) 49 | fmt.Println(err) 50 | 51 | // Output: whoops 52 | } 53 | 54 | func ExampleWithStack_printf() { 55 | cause := errors.New("whoops") 56 | err := errors.WithStack(cause) 57 | fmt.Printf("%+v", err) 58 | 59 | // Example Output: 60 | // whoops 61 | // github.com/pkg/errors_test.ExampleWithStack_printf 62 | // /home/fabstu/go/src/github.com/pkg/errors/example_test.go:55 63 | // testing.runExample 64 | // /usr/lib/go/src/testing/example.go:114 65 | // testing.RunExamples 66 | // /usr/lib/go/src/testing/example.go:38 67 | // testing.(*M).Run 68 | // /usr/lib/go/src/testing/testing.go:744 69 | // main.main 70 | // github.com/pkg/errors/_test/_testmain.go:106 71 | // runtime.main 72 | // /usr/lib/go/src/runtime/proc.go:183 73 | // runtime.goexit 74 | // /usr/lib/go/src/runtime/asm_amd64.s:2086 75 | // github.com/pkg/errors_test.ExampleWithStack_printf 76 | // /home/fabstu/go/src/github.com/pkg/errors/example_test.go:56 77 | // testing.runExample 78 | // /usr/lib/go/src/testing/example.go:114 79 | // testing.RunExamples 80 | // /usr/lib/go/src/testing/example.go:38 81 | // testing.(*M).Run 82 | // /usr/lib/go/src/testing/testing.go:744 83 | // main.main 84 | // github.com/pkg/errors/_test/_testmain.go:106 85 | // runtime.main 86 | // /usr/lib/go/src/runtime/proc.go:183 87 | // runtime.goexit 88 | // /usr/lib/go/src/runtime/asm_amd64.s:2086 89 | } 90 | 91 | func ExampleWrap() { 92 | cause := errors.New("whoops") 93 | err := errors.Wrap(cause, "oh noes") 94 | fmt.Println(err) 95 | 96 | // Output: oh noes: whoops 97 | } 98 | 99 | func fn() error { 100 | e1 := errors.New("error") 101 | e2 := errors.Wrap(e1, "inner") 102 | e3 := errors.Wrap(e2, "middle") 103 | return errors.Wrap(e3, "outer") 104 | } 105 | 106 | func ExampleCause() { 107 | err := fn() 108 | fmt.Println(err) 109 | fmt.Println(errors.Cause(err)) 110 | 111 | // Output: outer: middle: inner: error 112 | // error 113 | } 114 | 115 | func ExampleWrap_extended() { 116 | err := fn() 117 | fmt.Printf("%+v\n", err) 118 | 119 | // Example output: 120 | // error 121 | // github.com/pkg/errors_test.fn 122 | // /home/dfc/src/github.com/pkg/errors/example_test.go:47 123 | // github.com/pkg/errors_test.ExampleCause_printf 124 | // /home/dfc/src/github.com/pkg/errors/example_test.go:63 125 | // testing.runExample 126 | // /home/dfc/go/src/testing/example.go:114 127 | // testing.RunExamples 128 | // /home/dfc/go/src/testing/example.go:38 129 | // testing.(*M).Run 130 | // /home/dfc/go/src/testing/testing.go:744 131 | // main.main 132 | // /github.com/pkg/errors/_test/_testmain.go:104 133 | // runtime.main 134 | // /home/dfc/go/src/runtime/proc.go:183 135 | // runtime.goexit 136 | // /home/dfc/go/src/runtime/asm_amd64.s:2059 137 | // github.com/pkg/errors_test.fn 138 | // /home/dfc/src/github.com/pkg/errors/example_test.go:48: inner 139 | // github.com/pkg/errors_test.fn 140 | // /home/dfc/src/github.com/pkg/errors/example_test.go:49: middle 141 | // github.com/pkg/errors_test.fn 142 | // /home/dfc/src/github.com/pkg/errors/example_test.go:50: outer 143 | } 144 | 145 | func ExampleWrapf() { 146 | cause := errors.New("whoops") 147 | err := errors.Wrapf(cause, "oh noes #%d", 2) 148 | fmt.Println(err) 149 | 150 | // Output: oh noes #2: whoops 151 | } 152 | 153 | func ExampleErrorf_extended() { 154 | err := errors.Errorf("whoops: %s", "foo") 155 | fmt.Printf("%+v", err) 156 | 157 | // Example output: 158 | // whoops: foo 159 | // github.com/pkg/errors_test.ExampleErrorf 160 | // /home/dfc/src/github.com/pkg/errors/example_test.go:101 161 | // testing.runExample 162 | // /home/dfc/go/src/testing/example.go:114 163 | // testing.RunExamples 164 | // /home/dfc/go/src/testing/example.go:38 165 | // testing.(*M).Run 166 | // /home/dfc/go/src/testing/testing.go:744 167 | // main.main 168 | // /github.com/pkg/errors/_test/_testmain.go:102 169 | // runtime.main 170 | // /home/dfc/go/src/runtime/proc.go:183 171 | // runtime.goexit 172 | // /home/dfc/go/src/runtime/asm_amd64.s:2059 173 | } 174 | 175 | func Example_stackTrace() { 176 | type stackTracer interface { 177 | StackTrace() errors.StackTrace 178 | } 179 | 180 | err, ok := errors.Cause(fn()).(stackTracer) 181 | if !ok { 182 | panic("oops, err does not implement stackTracer") 183 | } 184 | 185 | st := err.StackTrace() 186 | fmt.Printf("%+v", st[0:2]) // top two frames 187 | 188 | // Example output: 189 | // github.com/pkg/errors_test.fn 190 | // /home/dfc/src/github.com/pkg/errors/example_test.go:47 191 | // github.com/pkg/errors_test.Example_stackTrace 192 | // /home/dfc/src/github.com/pkg/errors/example_test.go:127 193 | } 194 | 195 | func ExampleCause_printf() { 196 | err := errors.Wrap(func() error { 197 | return func() error { 198 | return errors.Errorf("hello %s", fmt.Sprintf("world")) 199 | }() 200 | }(), "failed") 201 | 202 | fmt.Printf("%v", err) 203 | 204 | // Output: failed: hello world 205 | } 206 | -------------------------------------------------------------------------------- /proto/include/google/protobuf/any.proto: -------------------------------------------------------------------------------- 1 | // Protocol Buffers - Google's data interchange format 2 | // Copyright 2008 Google Inc. All rights reserved. 3 | // https://developers.google.com/protocol-buffers/ 4 | // 5 | // Redistribution and use in source and binary forms, with or without 6 | // modification, are permitted provided that the following conditions are 7 | // met: 8 | // 9 | // * Redistributions of source code must retain the above copyright 10 | // notice, this list of conditions and the following disclaimer. 11 | // * Redistributions in binary form must reproduce the above 12 | // copyright notice, this list of conditions and the following disclaimer 13 | // in the documentation and/or other materials provided with the 14 | // distribution. 15 | // * Neither the name of Google Inc. nor the names of its 16 | // contributors may be used to endorse or promote products derived from 17 | // this software without specific prior written permission. 18 | // 19 | // THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 20 | // "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 21 | // LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 22 | // A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 23 | // OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 24 | // SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 25 | // LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 26 | // DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 27 | // THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 28 | // (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | // OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | 31 | syntax = "proto3"; 32 | 33 | package google.protobuf; 34 | 35 | option csharp_namespace = "Google.Protobuf.WellKnownTypes"; 36 | option go_package = "types"; 37 | option java_package = "com.google.protobuf"; 38 | option java_outer_classname = "AnyProto"; 39 | option java_multiple_files = true; 40 | option objc_class_prefix = "GPB"; 41 | 42 | // `Any` contains an arbitrary serialized protocol buffer message along with a 43 | // URL that describes the type of the serialized message. 44 | // 45 | // Protobuf library provides support to pack/unpack Any values in the form 46 | // of utility functions or additional generated methods of the Any type. 47 | // 48 | // Example 1: Pack and unpack a message in C++. 49 | // 50 | // Foo foo = ...; 51 | // Any any; 52 | // any.PackFrom(foo); 53 | // ... 54 | // if (any.UnpackTo(&foo)) { 55 | // ... 56 | // } 57 | // 58 | // Example 2: Pack and unpack a message in Java. 59 | // 60 | // Foo foo = ...; 61 | // Any any = Any.pack(foo); 62 | // ... 63 | // if (any.is(Foo.class)) { 64 | // foo = any.unpack(Foo.class); 65 | // } 66 | // 67 | // Example 3: Pack and unpack a message in Python. 68 | // 69 | // foo = Foo(...) 70 | // any = Any() 71 | // any.Pack(foo) 72 | // ... 73 | // if any.Is(Foo.DESCRIPTOR): 74 | // any.Unpack(foo) 75 | // ... 76 | // 77 | // Example 4: Pack and unpack a message in Go 78 | // 79 | // foo := &pb.Foo{...} 80 | // any, err := ptypes.MarshalAny(foo) 81 | // ... 82 | // foo := &pb.Foo{} 83 | // if err := ptypes.UnmarshalAny(any, foo); err != nil { 84 | // ... 85 | // } 86 | // 87 | // The pack methods provided by protobuf library will by default use 88 | // 'type.googleapis.com/full.type.name' as the type URL and the unpack 89 | // methods only use the fully qualified type name after the last '/' 90 | // in the type URL, for example "foo.bar.com/x/y.z" will yield type 91 | // name "y.z". 92 | // 93 | // 94 | // JSON 95 | // ==== 96 | // The JSON representation of an `Any` value uses the regular 97 | // representation of the deserialized, embedded message, with an 98 | // additional field `@type` which contains the type URL. Example: 99 | // 100 | // package google.profile; 101 | // message Person { 102 | // string first_name = 1; 103 | // string last_name = 2; 104 | // } 105 | // 106 | // { 107 | // "@type": "type.googleapis.com/google.profile.Person", 108 | // "firstName": , 109 | // "lastName": 110 | // } 111 | // 112 | // If the embedded message type is well-known and has a custom JSON 113 | // representation, that representation will be embedded adding a field 114 | // `value` which holds the custom JSON in addition to the `@type` 115 | // field. Example (for message [google.protobuf.Duration][]): 116 | // 117 | // { 118 | // "@type": "type.googleapis.com/google.protobuf.Duration", 119 | // "value": "1.212s" 120 | // } 121 | // 122 | message Any { 123 | // A URL/resource name that uniquely identifies the type of the serialized 124 | // protocol buffer message. The last segment of the URL's path must represent 125 | // the fully qualified name of the type (as in 126 | // `path/google.protobuf.Duration`). The name should be in a canonical form 127 | // (e.g., leading "." is not accepted). 128 | // 129 | // In practice, teams usually precompile into the binary all types that they 130 | // expect it to use in the context of Any. However, for URLs which use the 131 | // scheme `http`, `https`, or no scheme, one can optionally set up a type 132 | // server that maps type URLs to message definitions as follows: 133 | // 134 | // * If no scheme is provided, `https` is assumed. 135 | // * An HTTP GET on the URL must yield a [google.protobuf.Type][] 136 | // value in binary format, or produce an error. 137 | // * Applications are allowed to cache lookup results based on the 138 | // URL, or have them precompiled into a binary to avoid any 139 | // lookup. Therefore, binary compatibility needs to be preserved 140 | // on changes to types. (Use versioned type names to manage 141 | // breaking changes.) 142 | // 143 | // Note: this functionality is not currently available in the official 144 | // protobuf release, and it is not used for type URLs beginning with 145 | // type.googleapis.com. 146 | // 147 | // Schemes other than `http`, `https` (or the empty scheme) might be 148 | // used with implementation specific semantics. 149 | // 150 | string type_url = 1; 151 | 152 | // Must be a valid serialized protocol buffer of the above specified type. 153 | bytes value = 2; 154 | } 155 | --------------------------------------------------------------------------------