diff --git a/store/mockstore/mocktikv/cluster.go b/store/mockstore/mocktikv/cluster.go index 0bd27f3d68..3b1b726383 100644 --- a/store/mockstore/mocktikv/cluster.go +++ b/store/mockstore/mocktikv/cluster.go @@ -336,7 +336,7 @@ func (c *Cluster) splitRange(mvccStore MVCCStore, start, end MvccKey, count int) c.createNewRegions(regionPairs, start, end) } -// getPairsGroupByRegions groups the key value pairs into splitted regions. +// getEntriesGroupByRegions groups the key value pairs into splitted regions. func (c *Cluster) getEntriesGroupByRegions(mvccStore MVCCStore, start, end MvccKey, count int) [][]Pair { startTS := uint64(math.MaxUint64) limit := int(math.MaxInt32) diff --git a/store/mockstore/mocktikv/mvcc_leveldb.go b/store/mockstore/mocktikv/mvcc_leveldb.go index 824ee2d911..85739251aa 100644 --- a/store/mockstore/mocktikv/mvcc_leveldb.go +++ b/store/mockstore/mocktikv/mvcc_leveldb.go @@ -52,7 +52,10 @@ type MVCCLevelDB struct { // NextKey_0 -- (11) // ... // EOF + + // db represents leveldb db *leveldb.DB + // mu used for lock // leveldb can not guarantee multiple operations to be atomic, for example, read // then write, another write may happen during it, so this lock is necessory. mu sync.RWMutex @@ -177,7 +180,7 @@ type lockDecoder struct { expectKey []byte } -// lockDecoder decodes the lock value if current iterator is at expectKey::lock. +// Decode decodes the lock value if current iterator is at expectKey::lock. func (dec *lockDecoder) Decode(iter *Iterator) (bool, error) { if iter.Error() != nil || !iter.Valid() { return false, iter.Error() @@ -210,7 +213,7 @@ type valueDecoder struct { expectKey []byte } -// valueDecoder decodes a mvcc value if iter key is expectKey. +// Decode decodes a mvcc value if iter key is expectKey. func (dec *valueDecoder) Decode(iter *Iterator) (bool, error) { if iter.Error() != nil || !iter.Valid() { return false, iter.Error() @@ -241,7 +244,7 @@ type skipDecoder struct { currKey []byte } -// skipDecoder skips the iterator as long as its key is currKey, the new key would be stored. +// Decode skips the iterator as long as its key is currKey, the new key would be stored. func (dec *skipDecoder) Decode(iter *Iterator) (bool, error) { if iter.Error() != nil { return false, iter.Error() @@ -262,11 +265,11 @@ func (dec *skipDecoder) Decode(iter *Iterator) (bool, error) { type mvccEntryDecoder struct { expectKey []byte - // Just values and lock is valid. + // mvccEntry represents values and lock is valid. mvccEntry } -// mvccEntryDecoder decodes a mvcc entry. +// Decode decodes a mvcc entry. func (dec *mvccEntryDecoder) Decode(iter *Iterator) (bool, error) { ldec := lockDecoder{expectKey: dec.expectKey} ok, err := ldec.Decode(iter) diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index 413e58d00c..c5a82f3f02 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -101,15 +101,15 @@ type rpcHandler struct { cluster *Cluster mvccStore MVCCStore - // store id for current request + // storeID stores id for current request storeID uint64 - // Used for handling normal request. + // startKey is used for handling normal request. startKey []byte endKey []byte - // Used for handling coprocessor request. + // rawStartKey is used for handling coprocessor request. rawStartKey []byte rawEndKey []byte - // Used for current request. + // isolationLevel is used for current request. isolationLevel kvrpcpb.IsolationLevel } diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 81316aa9d5..063cfcc0b0 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -78,7 +78,7 @@ type twoPhaseCommitter struct { syncLog bool connID uint64 // connID is used for log. cleanWg sync.WaitGroup - // The max time a Txn may use (in ms) from its startTS to commitTS. + // maxTxnTimeUse represents max time a Txn may use (in ms) from its startTS to commitTS. // We use it to guarantee GC worker will not influence any active txn. The value // should be less than GC life time. maxTxnTimeUse uint64 diff --git a/store/tikv/2pc_slow_test.go b/store/tikv/2pc_slow_test.go index d3dd407952..bf96a24485 100644 --- a/store/tikv/2pc_slow_test.go +++ b/store/tikv/2pc_slow_test.go @@ -17,6 +17,7 @@ package tikv import . "github.com/pingcap/check" +// TestCommitMultipleRegions tests commit multiple regions. // The test takes too long under the race detector. func (s *testCommitterSuite) TestCommitMultipleRegions(c *C) { m := make(map[string]string) diff --git a/store/tikv/client.go b/store/tikv/client.go index 84fb589251..52a3d9e916 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -74,10 +74,10 @@ type Client interface { type connArray struct { index uint32 v []*grpc.ClientConn - // Bind with a background goroutine to process coprocessor streaming timeout. + // streamTimeout binds with a background goroutine to process coprocessor streaming timeout. streamTimeout chan *tikvrpc.Lease - // For batch commands. + // batchCommandsCh used for batch commands. batchCommandsCh chan *batchCommandsEntry batchCommandsClients []*batchCommandsClient tikvTransportLayerLoad uint64 @@ -90,9 +90,9 @@ type batchCommandsClient struct { idAlloc uint64 tikvTransportLayerLoad *uint64 - // Indicates the batch client is closed explicitly or not. + // closed indicates the batch client is closed explicitly or not. closed int32 - // Protect client when re-create the streaming. + // clientLock protects client when re-create the streaming. clientLock sync.Mutex } @@ -302,7 +302,7 @@ type batchCommandsEntry struct { req *tikvpb.BatchCommandsRequest_Request res chan *tikvpb.BatchCommandsResponse_Response - // Indicated the request is canceled or not. + // canceled indicated the request is canceled or not. canceled int32 err error } diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index aed70ef1cd..5d878a2509 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -214,7 +214,7 @@ func (r *copRanges) toPBRanges() []*coprocessor.KeyRange { return ranges } -// Split ranges into (left, right) by key. +// split ranges into (left, right) by key. func (r *copRanges) split(key []byte) (*copRanges, *copRanges) { n := sort.Search(r.len(), func(i int) bool { cur := r.at(i) @@ -354,6 +354,7 @@ type copIterator struct { req *kv.Request concurrency int finishCh chan struct{} + // closed represents when the Close is called. // There are two cases we need to close the `finishCh` channel, one is when context is done, the other one is // when the Close is called. we use atomic.CompareAndSwap `closed` to to make sure the channel is not closed twice. closed uint32 diff --git a/store/tikv/delete_range_test.go b/store/tikv/delete_range_test.go index 49be7fbbcc..b0467f35b1 100644 --- a/store/tikv/delete_range_test.go +++ b/store/tikv/delete_range_test.go @@ -99,7 +99,7 @@ func deleteRangeFromMap(m map[string]string, startKey []byte, endKey []byte) { } } -// testDeleteRangeOnce does delete range on both the map and the storage, and assert they are equal after deleting +// mustDeleteRange does delete range on both the map and the storage, and assert they are equal after deleting func (s *testDeleteRangeSuite) mustDeleteRange(c *C, startKey []byte, endKey []byte, expected map[string]string) { s.deleteRange(c, startKey, endKey) deleteRangeFromMap(expected, startKey, endKey) diff --git a/store/tikv/gcworker/gc_worker.go b/store/tikv/gcworker/gc_worker.go index 5dcfe97f1f..e7a143520b 100644 --- a/store/tikv/gcworker/gc_worker.go +++ b/store/tikv/gcworker/gc_worker.go @@ -230,7 +230,7 @@ func (w *GCWorker) storeIsBootstrapped() bool { return ver > notBootstrappedVer } -// Leader of GC worker checks if it should start a GC job every tick. +// leaderTick of GC worker checks if it should start a GC job every tick. func (w *GCWorker) leaderTick(ctx context.Context) error { if w.gcIsRunning { log.Infof("[gc worker] leaderTick on %s: there's already a gc job running. skipped.", w.uuid) @@ -437,7 +437,7 @@ func (w *GCWorker) runGCJob(ctx context.Context, safePoint uint64) { w.done <- nil } -// `deleteRanges` processes all delete range records whose ts < safePoint in table `gc_delete_range` +// deleteRanges processes all delete range records whose ts < safePoint in table `gc_delete_range` func (w *GCWorker) deleteRanges(ctx context.Context, safePoint uint64) error { metrics.GCWorkerCounter.WithLabelValues("delete_range").Inc() @@ -470,7 +470,7 @@ func (w *GCWorker) deleteRanges(ctx context.Context, safePoint uint64) error { return nil } -// `redoDeleteRanges` checks all deleted ranges whose ts is at least `lifetime + 24h` ago. See TiKV RFC #2. +// redoDeleteRanges checks all deleted ranges whose ts is at least `lifetime + 24h` ago. See TiKV RFC #2. func (w *GCWorker) redoDeleteRanges(ctx context.Context, safePoint uint64) error { metrics.GCWorkerCounter.WithLabelValues("redo_delete_range").Inc() @@ -729,7 +729,7 @@ type gcTaskWorker struct { store tikv.Storage taskCh chan *gcTask wg *sync.WaitGroup - // use atomic to read and set + // successRegions and failedRegions use atomic to read and set. successRegions *int32 failedRegions *int32 } @@ -801,6 +801,7 @@ func (w *gcTaskWorker) doGCForRange(startKey []byte, endKey []byte, safePoint ui return nil } +// doGCForRegion used for gc for region. // these two errors should not return together, for more, see the func 'doGC' func (w *gcTaskWorker) doGCForRegion(bo *tikv.Backoffer, safePoint uint64, region tikv.RegionVerID) (*errorpb.Error, error) { req := &tikvrpc.Request{ diff --git a/store/tikv/isolation_test.go b/store/tikv/isolation_test.go index 9e3e5ee330..5944980f62 100644 --- a/store/tikv/isolation_test.go +++ b/store/tikv/isolation_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/kv" ) +// testIsolationSuite represents test isolation suite. // The test suite takes too long under the race detector. type testIsolationSuite struct { OneByOneSuite diff --git a/store/tikv/latch/latch.go b/store/tikv/latch/latch.go index 6a919e1eba..c2a309c355 100644 --- a/store/tikv/latch/latch.go +++ b/store/tikv/latch/latch.go @@ -45,14 +45,15 @@ type latch struct { // Lock is the locks' information required for a transaction. type Lock struct { keys [][]byte + // requiredSlots represents required slots. // The slot IDs of the latches(keys) that a startTS must acquire before being able to processed. requiredSlots []int - // The number of latches that the transaction has acquired. For status is stale, it include the - // latch whose front is current lock already. + // acquiredCount represents the number of latches that the transaction has acquired. + // For status is stale, it include the latch whose front is current lock already. acquiredCount int - // Current transaction's startTS. + // startTS represents current transaction's. startTS uint64 - // Current transaction's commitTS. + // commitTS represents current transaction's. commitTS uint64 wg sync.WaitGroup diff --git a/store/tikv/pd_codec.go b/store/tikv/pd_codec.go index daabcf54ea..4553d20ac1 100644 --- a/store/tikv/pd_codec.go +++ b/store/tikv/pd_codec.go @@ -40,7 +40,7 @@ func (c *codecPDClient) GetPrevRegion(ctx context.Context, key []byte) (*metapb. return processRegionResult(region, peer, err) } -// GetRegion encodes the key before send requests to pd-server and decodes the +// GetRegionByID encodes the key before send requests to pd-server and decodes the // returned StartKey && EndKey from pd-server. func (c *codecPDClient) GetRegionByID(ctx context.Context, regionID uint64) (*metapb.Region, *metapb.Peer, error) { region, peer, err := c.Client.GetRegionByID(ctx, regionID) diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index a9a79e37e8..4f03b29b4b 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -159,7 +159,7 @@ func (c *cancelContextClient) SendRequest(ctx context.Context, addr string, req // mockTikvGrpcServer mock a tikv gprc server for testing. type mockTikvGrpcServer struct{} -// KV commands with mvcc/txn supported. +// KvGet commands with mvcc/txn supported. func (s *mockTikvGrpcServer) KvGet(context.Context, *kvrpcpb.GetRequest) (*kvrpcpb.GetResponse, error) { return nil, errors.New("unreachable") }