// Copyright 2020 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, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. package executor import ( "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/syncutil" ) // ShardCount controls the shard maps within the concurrent map var ShardCount = 320 // A "thread" safe map of type string:Anything. // To avoid lock bottlenecks this map is dived to several (ShardCount) map shards. type concurrentMap []*concurrentMapShared // A "thread" safe string to anything map. type concurrentMapShared struct { items map[uint64]*entry syncutil.RWMutex // Read Write mutex, guards access to internal map. bInMap int64 // indicate there are 2^bInMap buckets in items } // newConcurrentMap creates a new concurrent map. func newConcurrentMap() concurrentMap { m := make(concurrentMap, ShardCount) for i := 0; i < ShardCount; i++ { m[i] = &concurrentMapShared{items: make(map[uint64]*entry), bInMap: 0} } return m } // getShard returns shard under given key func (m concurrentMap) getShard(hashKey uint64) *concurrentMapShared { return m[hashKey%uint64(ShardCount)] } // Insert inserts a value in a shard safely func (m concurrentMap) Insert(key uint64, value *entry) (memDelta int64) { shard := m.getShard(key) shard.Lock() oldValue := shard.items[key] value.next = oldValue shard.items[key] = value if len(shard.items) > (1<