You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 

304 lines
7.5 KiB

// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package latch
import (
"bytes"
"math/bits"
"sort"
"sync"
"time"
"github.com/cznic/mathutil"
"github.com/pingcap/tidb/util/logutil"
"github.com/spaolacci/murmur3"
"go.uber.org/zap"
)
type node struct {
slotID int
key []byte
maxCommitTS uint64
value *Lock
next *node
}
// latch stores a key's waiting transactions information.
type latch struct {
queue *node
count int
waiting []*Lock
sync.Mutex
}
// Lock is the locks' information required for a transaction.
type Lock struct {
keys [][]byte
// requiredSlots represents required slots.
// The slot IDs of the latches(keys) that a startTS must acquire before being able to processed.
requiredSlots []int
// acquiredCount represents the number of latches that the transaction has acquired.
// For status is stale, it include the latch whose front is current lock already.
acquiredCount int
// startTS represents current transaction's.
startTS uint64
// commitTS represents current transaction's.
commitTS uint64
wg sync.WaitGroup
isStale bool
}
// acquireResult is the result type for acquire()
type acquireResult int32
const (
// acquireSuccess is a type constant for acquireResult.
// which means acquired success
acquireSuccess acquireResult = iota
// acquireLocked is a type constant for acquireResult
// which means still locked by other Lock.
acquireLocked
// acquireStale is a type constant for acquireResult
// which means current Lock's startTS is stale.
acquireStale
)
// IsStale returns whether the status is stale.
func (l *Lock) IsStale() bool {
return l.isStale
}
func (l *Lock) isLocked() bool {
return !l.isStale && l.acquiredCount != len(l.requiredSlots)
}
// SetCommitTS sets the lock's commitTS.
func (l *Lock) SetCommitTS(commitTS uint64) {
l.commitTS = commitTS
}
// Latches which are used for concurrency control.
// Each latch is indexed by a slot's ID, hence the term latch and slot are used in interchangeable,
// but conceptually a latch is a queue, and a slot is an index to the queue
type Latches struct {
slots []latch
}
type bytesSlice [][]byte
func (s bytesSlice) Len() int {
return len(s)
}
func (s bytesSlice) Swap(i, j int) {
s[i], s[j] = s[j], s[i]
}
func (s bytesSlice) Less(i, j int) bool {
return bytes.Compare(s[i], s[j]) < 0
}
// NewLatches create a Latches with fixed length,
// the size will be rounded up to the power of 2.
func NewLatches(size uint) *Latches {
powerOfTwoSize := 1 << uint32(bits.Len32(uint32(size-1)))
slots := make([]latch, powerOfTwoSize)
return &Latches{
slots: slots,
}
}
// genLock generates Lock for the transaction with startTS and keys.
func (latches *Latches) genLock(startTS uint64, keys [][]byte) *Lock {
sort.Sort(bytesSlice(keys))
return &Lock{
keys: keys,
requiredSlots: latches.genSlotIDs(keys),
acquiredCount: 0,
startTS: startTS,
}
}
func (latches *Latches) genSlotIDs(keys [][]byte) []int {
slots := make([]int, 0, len(keys))
for _, key := range keys {
slots = append(slots, latches.slotID(key))
}
return slots
}
// slotID return slotID for current key.
func (latches *Latches) slotID(key []byte) int {
return int(murmur3.Sum32(key)) & (len(latches.slots) - 1)
}
// acquire tries to acquire the lock for a transaction.
func (latches *Latches) acquire(lock *Lock) acquireResult {
if lock.IsStale() {
return acquireStale
}
for lock.acquiredCount < len(lock.requiredSlots) {
status := latches.acquireSlot(lock)
if status != acquireSuccess {
return status
}
}
return acquireSuccess
}
// release releases all latches owned by the `lock` and returns the wakeup list.
// Preconditions: the caller must ensure the transaction's status is not locked.
func (latches *Latches) release(lock *Lock, wakeupList []*Lock) []*Lock {
wakeupList = wakeupList[:0]
for lock.acquiredCount > 0 {
if nextLock := latches.releaseSlot(lock); nextLock != nil {
wakeupList = append(wakeupList, nextLock)
}
}
return wakeupList
}
func (latches *Latches) releaseSlot(lock *Lock) (nextLock *Lock) {
key := lock.keys[lock.acquiredCount-1]
slotID := lock.requiredSlots[lock.acquiredCount-1]
latch := &latches.slots[slotID]
lock.acquiredCount--
latch.Lock()
defer latch.Unlock()
find := findNode(latch.queue, key)
if find.value != lock {
panic("releaseSlot wrong")
}
find.maxCommitTS = mathutil.MaxUint64(find.maxCommitTS, lock.commitTS)
find.value = nil
// Make a copy of the key, so latch does not reference the transaction's memory.
// If we do not do it, transaction memory can't be recycle by GC and there will
// be a leak.
copyKey := make([]byte, len(find.key))
copy(copyKey, find.key)
find.key = copyKey
if len(latch.waiting) == 0 {
return nil
}
var idx int
for idx = 0; idx < len(latch.waiting); idx++ {
waiting := latch.waiting[idx]
if bytes.Equal(waiting.keys[waiting.acquiredCount], key) {
break
}
}
// Wake up the first one in waiting queue.
if idx < len(latch.waiting) {
nextLock = latch.waiting[idx]
// Delete element latch.waiting[idx] from the array.
copy(latch.waiting[idx:], latch.waiting[idx+1:])
latch.waiting[len(latch.waiting)-1] = nil
latch.waiting = latch.waiting[:len(latch.waiting)-1]
if find.maxCommitTS > nextLock.startTS {
find.value = nextLock
nextLock.acquiredCount++
nextLock.isStale = true
}
}
return
}
func (latches *Latches) acquireSlot(lock *Lock) acquireResult {
key := lock.keys[lock.acquiredCount]
slotID := lock.requiredSlots[lock.acquiredCount]
latch := &latches.slots[slotID]
latch.Lock()
defer latch.Unlock()
// Try to recycle to limit the memory usage.
if latch.count >= latchListCount {
latch.recycle(lock.startTS)
}
find := findNode(latch.queue, key)
if find == nil {
tmp := &node{
slotID: slotID,
key: key,
value: lock,
}
tmp.next = latch.queue
latch.queue = tmp
latch.count++
lock.acquiredCount++
return acquireSuccess
}
if find.maxCommitTS > lock.startTS {
lock.isStale = true
return acquireStale
}
if find.value == nil {
find.value = lock
lock.acquiredCount++
return acquireSuccess
}
// Push the current transaction into waitingQueue.
latch.waiting = append(latch.waiting, lock)
return acquireLocked
}
// recycle is not thread safe, the latch should acquire its lock before executing this function.
func (l *latch) recycle(currentTS uint64) int {
total := 0
fakeHead := node{next: l.queue}
prev := &fakeHead
for curr := prev.next; curr != nil; curr = curr.next {
if tsoSub(currentTS, curr.maxCommitTS) >= expireDuration && curr.value == nil {
l.count--
prev.next = curr.next
total++
} else {
prev = curr
}
}
l.queue = fakeHead.next
return total
}
func (latches *Latches) recycle(currentTS uint64) {
total := 0
for i := 0; i < len(latches.slots); i++ {
latch := &latches.slots[i]
latch.Lock()
total += latch.recycle(currentTS)
latch.Unlock()
}
logutil.BgLogger().Debug("recycle",
zap.Time("start at", time.Now()),
zap.Int("count", total))
}
func findNode(list *node, key []byte) *node {
for n := list; n != nil; n = n.next {
if bytes.Equal(n.key, key) {
return n
}
}
return nil
}