From 8fbae906ceca34238a383ba61b4bb1cac9699319 Mon Sep 17 00:00:00 2001 From: bufferflies <1045931706@qq.com> Date: Thu, 14 Jul 2022 15:33:42 +0800 Subject: [PATCH] remove 1<<20 Signed-off-by: bufferflies <1045931706@qq.com> --- pkg/mock/mockcluster/mockcluster.go | 24 ++++++++-------- server/cluster/coordinator_test.go | 5 ++-- server/core/region.go | 3 +- server/core/store.go | 17 ++++++------ server/core/store_test.go | 41 ++++++++++++++-------------- server/core/test_util.go | 3 +- server/schedule/range_cluster.go | 5 ++-- tests/server/cluster/cluster_test.go | 5 ++-- 8 files changed, 54 insertions(+), 49 deletions(-) diff --git a/pkg/mock/mockcluster/mockcluster.go b/pkg/mock/mockcluster/mockcluster.go index f7eb252b31bd..76f26cce98da 100644 --- a/pkg/mock/mockcluster/mockcluster.go +++ b/pkg/mock/mockcluster/mockcluster.go @@ -20,6 +20,7 @@ import ( "strconv" "time" + "github.com/docker/go-units" "github.com/gogo/protobuf/proto" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" @@ -39,9 +40,8 @@ import ( ) const ( - defaultStoreCapacity = 100 * (1 << 30) // 100GiB - defaultRegionSize = 96 * (1 << 20) // 96MiB - mb = 1 << 20 // 1MiB + defaultStoreCapacity = 100 * units.GiB // 100GiB + defaultRegionSize = 96 * units.MiB // 96MiB ) // Cluster is used to mock a cluster for test purpose. @@ -261,7 +261,7 @@ func (mc *Cluster) AddLeaderStore(storeID uint64, leaderCount int, leaderSizes . if len(leaderSizes) != 0 { leaderSize = leaderSizes[0] } else { - leaderSize = int64(leaderCount) * defaultRegionSize / mb + leaderSize = int64(leaderCount) * defaultRegionSize / units.MiB } store := core.NewStoreInfo( @@ -291,7 +291,7 @@ func (mc *Cluster) AddRegionStore(storeID uint64, regionCount int) { }}, core.SetStoreStats(stats), core.SetRegionCount(regionCount), - core.SetRegionSize(int64(regionCount)*defaultRegionSize/mb), + core.SetRegionSize(int64(regionCount)*defaultRegionSize/units.MiB), core.SetLastHeartbeatTS(time.Now()), ) mc.SetStoreLimit(storeID, storelimit.AddPeer, 60) @@ -329,7 +329,7 @@ func (mc *Cluster) AddLabelsStore(storeID uint64, regionCount int, labels map[st }, core.SetStoreStats(stats), core.SetRegionCount(regionCount), - core.SetRegionSize(int64(regionCount)*defaultRegionSize/mb), + core.SetRegionSize(int64(regionCount)*defaultRegionSize/units.MiB), core.SetLastHeartbeatTS(time.Now()), ) mc.SetStoreLimit(storeID, storelimit.AddPeer, 60) @@ -340,7 +340,7 @@ func (mc *Cluster) AddLabelsStore(storeID uint64, regionCount int, labels map[st // AddLeaderRegion adds region with specified leader and followers. func (mc *Cluster) AddLeaderRegion(regionID uint64, leaderStoreID uint64, otherPeerStoreIDs ...uint64) *core.RegionInfo { origin := mc.newMockRegionInfo(regionID, leaderStoreID, otherPeerStoreIDs...) - region := origin.Clone(core.SetApproximateSize(defaultRegionSize/mb), core.SetApproximateKeys(10)) + region := origin.Clone(core.SetApproximateSize(defaultRegionSize/units.MiB), core.SetApproximateKeys(10)) mc.PutRegion(region) return region } @@ -348,7 +348,7 @@ func (mc *Cluster) AddLeaderRegion(regionID uint64, leaderStoreID uint64, otherP // AddRegionWithLearner adds region with specified leader, followers and learners. func (mc *Cluster) AddRegionWithLearner(regionID uint64, leaderStoreID uint64, followerStoreIDs, learnerStoreIDs []uint64) *core.RegionInfo { origin := mc.MockRegionInfo(regionID, leaderStoreID, followerStoreIDs, learnerStoreIDs, nil) - region := origin.Clone(core.SetApproximateSize(defaultRegionSize/mb), core.SetApproximateKeys(10)) + region := origin.Clone(core.SetApproximateSize(defaultRegionSize/units.MiB), core.SetApproximateKeys(10)) mc.PutRegion(region) return region } @@ -520,7 +520,7 @@ func (mc *Cluster) UpdateLeaderCount(storeID uint64, leaderCount int) { store := mc.GetStore(storeID) newStore := store.Clone( core.SetLeaderCount(leaderCount), - core.SetLeaderSize(int64(leaderCount)*defaultRegionSize/mb), + core.SetLeaderSize(int64(leaderCount)*defaultRegionSize/units.MiB), ) mc.PutStore(newStore) } @@ -530,7 +530,7 @@ func (mc *Cluster) UpdateRegionCount(storeID uint64, regionCount int) { store := mc.GetStore(storeID) newStore := store.Clone( core.SetRegionCount(regionCount), - core.SetRegionSize(int64(regionCount)*defaultRegionSize/mb), + core.SetRegionSize(int64(regionCount)*defaultRegionSize/units.MiB), ) mc.PutStore(newStore) } @@ -647,8 +647,8 @@ func (mc *Cluster) UpdateStoreStatus(id uint64) { store := mc.Stores.GetStore(id) stats := &pdpb.StoreStats{} stats.Capacity = defaultStoreCapacity - stats.Available = stats.Capacity - uint64(store.GetRegionSize()*mb) - stats.UsedSize = uint64(store.GetRegionSize() * mb) + stats.Available = stats.Capacity - uint64(store.GetRegionSize()*units.MiB) + stats.UsedSize = uint64(store.GetRegionSize() * units.MiB) newStore := store.Clone( core.SetStoreStats(stats), core.SetLeaderCount(leaderCount), diff --git a/server/cluster/coordinator_test.go b/server/cluster/coordinator_test.go index 01dfa63e6efc..2412dd26f329 100644 --- a/server/cluster/coordinator_test.go +++ b/server/cluster/coordinator_test.go @@ -22,6 +22,7 @@ import ( "testing" "time" + "github.com/docker/go-units" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" @@ -63,8 +64,8 @@ func (c *testCluster) addRegionStore(storeID uint64, regionCount int, regionSize } stats := &pdpb.StoreStats{} - stats.Capacity = 100 * (1 << 30) - stats.UsedSize = regionSize * (1 << 20) + stats.Capacity = 100 * units.GiB + stats.UsedSize = regionSize * units.MiB stats.Available = stats.Capacity - stats.UsedSize newStore := core.NewStoreInfo(&metapb.Store{Id: storeID}, core.SetStoreStats(stats), diff --git a/server/core/region.go b/server/core/region.go index 8114a0e411db..d11f8e00bc48 100644 --- a/server/core/region.go +++ b/server/core/region.go @@ -24,6 +24,7 @@ import ( "sync/atomic" "unsafe" + "github.com/docker/go-units" "github.com/gogo/protobuf/proto" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" @@ -130,7 +131,7 @@ func RegionFromHeartbeat(heartbeat *pdpb.RegionHeartbeatRequest, opts ...RegionC // Convert unit to MB. // If region isn't empty and less than 1MB, use 1MB instead. // The size of empty region will be correct by the previous RegionInfo. - regionSize := heartbeat.GetApproximateSize() / (1 << 20) + regionSize := heartbeat.GetApproximateSize() / units.MiB if heartbeat.GetApproximateSize() > 0 && regionSize < EmptyRegionApproximateSize { regionSize = EmptyRegionApproximateSize } diff --git a/server/core/store.go b/server/core/store.go index f7cb7dd92fa5..22e59b355b0b 100644 --- a/server/core/store.go +++ b/server/core/store.go @@ -19,6 +19,7 @@ import ( "strings" "time" + "github.com/docker/go-units" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/errs" @@ -29,10 +30,8 @@ import ( const ( // Interval to save store meta (including heartbeat ts) to etcd. storePersistInterval = 5 * time.Minute - mb = 1 << 20 // megabyte - gb = 1 << 30 // 1GB size - initialMaxRegionCounts = 30 // exclude storage Threshold Filter when region less than 30 - initialMinSpace = 1 << 33 // 2^33=8GB + initialMaxRegionCounts = 30 // exclude storage Threshold Filter when region less than 30 + initialMinSpace = 8 * units.GiB // 2^33=8GB slowStoreThreshold = 80 // EngineKey is the label key used to indicate engine. @@ -335,9 +334,9 @@ func (s *StoreInfo) RegionScore(version string, highSpaceRatio, lowSpaceRatio fl func (s *StoreInfo) regionScoreV1(highSpaceRatio, lowSpaceRatio float64, delta int64) float64 { var score float64 var amplification float64 - available := float64(s.GetAvailable()) / mb - used := float64(s.GetUsedSize()) / mb - capacity := float64(s.GetCapacity()) / mb + available := float64(s.GetAvailable()) / units.MiB + used := float64(s.GetUsedSize()) / units.MiB + capacity := float64(s.GetCapacity()) / units.MiB if s.GetRegionSize() == 0 || used == 0 { amplification = 1 @@ -378,8 +377,8 @@ func (s *StoreInfo) regionScoreV1(highSpaceRatio, lowSpaceRatio float64, delta i } func (s *StoreInfo) regionScoreV2(delta int64, lowSpaceRatio float64) float64 { - A := float64(s.GetAvgAvailable()) / gb - C := float64(s.GetCapacity()) / gb + A := float64(s.GetAvgAvailable()) / units.GiB + C := float64(s.GetCapacity()) / units.GiB R := float64(s.GetRegionSize() + delta) if R < 0 { R = float64(s.GetRegionSize()) diff --git a/server/core/store_test.go b/server/core/store_test.go index e93d0e80eaf1..2e0de9b06a12 100644 --- a/server/core/store_test.go +++ b/server/core/store_test.go @@ -20,6 +20,7 @@ import ( "testing" "time" + "github.com/docker/go-units" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" @@ -113,8 +114,8 @@ func BenchmarkStoreClone(b *testing.B) { func TestRegionScore(t *testing.T) { re := require.New(t) stats := &pdpb.StoreStats{} - stats.Capacity = 512 * (1 << 20) // 512 MB - stats.Available = 100 * (1 << 20) // 100 MB + stats.Capacity = 512 * units.MiB // 512 MB + stats.Available = 100 * units.MiB // 100 MB stats.UsedSize = 0 store := NewStoreInfo( @@ -146,41 +147,41 @@ func TestLowSpaceScoreV2(t *testing.T) { bigger *StoreInfo small *StoreInfo }{{ - // store1 and store2 has same store available ratio and store1 less 50gb - bigger: NewStoreInfoWithAvailable(1, 20*gb, 100*gb, 1.4), - small: NewStoreInfoWithAvailable(2, 200*gb, 1000*gb, 1.4), + // store1 and store2 has same store available ratio and store1 less 50units.GiB + bigger: NewStoreInfoWithAvailable(1, 20*units.GiB, 100*units.GiB, 1.4), + small: NewStoreInfoWithAvailable(2, 200*units.GiB, 1000*units.GiB, 1.4), }, { - // store1 and store2 has same available space and less than 50gb - bigger: NewStoreInfoWithAvailable(1, 10*gb, 1000*gb, 1.4), - small: NewStoreInfoWithAvailable(2, 10*gb, 100*gb, 1.4), + // store1 and store2 has same available space and less than 50units.GiB + bigger: NewStoreInfoWithAvailable(1, 10*units.GiB, 1000*units.GiB, 1.4), + small: NewStoreInfoWithAvailable(2, 10*units.GiB, 100*units.GiB, 1.4), }, { // store1 and store2 has same available ratio less than 0.2 - bigger: NewStoreInfoWithAvailable(1, 20*gb, 1000*gb, 1.4), - small: NewStoreInfoWithAvailable(2, 10*gb, 500*gb, 1.4), + bigger: NewStoreInfoWithAvailable(1, 20*units.GiB, 1000*units.GiB, 1.4), + small: NewStoreInfoWithAvailable(2, 10*units.GiB, 500*units.GiB, 1.4), }, { // store1 and store2 has same available ratio // but the store1 ratio less than store2 ((50-10)/50=0.8<(200-100)/200=0.5) - bigger: NewStoreInfoWithAvailable(1, 10*gb, 100*gb, 1.4), - small: NewStoreInfoWithAvailable(2, 100*gb, 1000*gb, 1.4), + bigger: NewStoreInfoWithAvailable(1, 10*units.GiB, 100*units.GiB, 1.4), + small: NewStoreInfoWithAvailable(2, 100*units.GiB, 1000*units.GiB, 1.4), }, { // store1 and store2 has same usedSize and capacity // but the bigger's amp is bigger - bigger: NewStoreInfoWithAvailable(1, 10*gb, 100*gb, 1.5), - small: NewStoreInfoWithAvailable(2, 10*gb, 100*gb, 1.4), + bigger: NewStoreInfoWithAvailable(1, 10*units.GiB, 100*units.GiB, 1.5), + small: NewStoreInfoWithAvailable(2, 10*units.GiB, 100*units.GiB, 1.4), }, { // store1 and store2 has same capacity and regionSizeļ¼ˆ40g) // but store1 has less available space size - bigger: NewStoreInfoWithAvailable(1, 60*gb, 100*gb, 1), - small: NewStoreInfoWithAvailable(2, 80*gb, 100*gb, 2), + bigger: NewStoreInfoWithAvailable(1, 60*units.GiB, 100*units.GiB, 1), + small: NewStoreInfoWithAvailable(2, 80*units.GiB, 100*units.GiB, 2), }, { // store1 and store2 has same capacity and store2 (40g) has twice usedSize than store1 (20g) // but store1 has higher amp, so store1(60g) has more regionSize (40g) - bigger: NewStoreInfoWithAvailable(1, 80*gb, 100*gb, 3), - small: NewStoreInfoWithAvailable(2, 60*gb, 100*gb, 1), + bigger: NewStoreInfoWithAvailable(1, 80*units.GiB, 100*units.GiB, 3), + small: NewStoreInfoWithAvailable(2, 60*units.GiB, 100*units.GiB, 1), }, { // store1's capacity is less than store2's capacity, but store2 has more available space, - bigger: NewStoreInfoWithAvailable(1, 2*gb, 100*gb, 3), - small: NewStoreInfoWithAvailable(2, 100*gb, 10*1000*gb, 3), + bigger: NewStoreInfoWithAvailable(1, 2*units.GiB, 100*units.GiB, 3), + small: NewStoreInfoWithAvailable(2, 100*units.GiB, 10*1000*units.GiB, 3), }} for _, v := range testdata { score1 := v.bigger.regionScoreV2(0, 0.8) diff --git a/server/core/test_util.go b/server/core/test_util.go index 78e3c10508cd..b7bf021eb7f2 100644 --- a/server/core/test_util.go +++ b/server/core/test_util.go @@ -19,6 +19,7 @@ import ( "math/rand" "time" + "github.com/docker/go-units" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" ) @@ -90,7 +91,7 @@ func NewStoreInfoWithAvailable(id, available, capacity uint64, amp float64) *Sto stats.Capacity = capacity stats.Available = available usedSize := capacity - available - regionSize := (float64(usedSize) * amp) / mb + regionSize := (float64(usedSize) * amp) / units.MiB store := NewStoreInfo( &metapb.Store{ Id: id, diff --git a/server/schedule/range_cluster.go b/server/schedule/range_cluster.go index d2073fb61202..efffa7779793 100644 --- a/server/schedule/range_cluster.go +++ b/server/schedule/range_cluster.go @@ -15,6 +15,7 @@ package schedule import ( + "github.com/docker/go-units" "github.com/tikv/pd/server/core" ) @@ -41,7 +42,7 @@ func GenRangeCluster(cluster Cluster, startKey, endKey []byte) *RangeCluster { func (r *RangeCluster) updateStoreInfo(s *core.StoreInfo) *core.StoreInfo { id := s.GetID() - used := float64(s.GetUsedSize()) / (1 << 20) + used := float64(s.GetUsedSize()) / units.MiB if used == 0 { return s } @@ -52,7 +53,7 @@ func (r *RangeCluster) updateStoreInfo(s *core.StoreInfo) *core.StoreInfo { regionSize := r.subCluster.GetStoreRegionSize(id) pendingPeerCount := r.subCluster.GetStorePendingPeerCount(id) newStats := s.CloneStoreStats() - newStats.UsedSize = uint64(float64(regionSize)/amplification) * (1 << 20) + newStats.UsedSize = uint64(float64(regionSize)/amplification) * units.MiB newStats.Available = s.GetCapacity() - newStats.UsedSize newStore := s.Clone( core.SetNewStoreStats(newStats), // it means to use instant value directly diff --git a/tests/server/cluster/cluster_test.go b/tests/server/cluster/cluster_test.go index b16099ddda05..3600de7491ac 100644 --- a/tests/server/cluster/cluster_test.go +++ b/tests/server/cluster/cluster_test.go @@ -24,6 +24,7 @@ import ( "time" "github.com/coreos/go-semver/semver" + "github.com/docker/go-units" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" @@ -585,8 +586,8 @@ func TestConcurrentHandleRegion(t *testing.T) { Header: testutil.NewRequestHeader(clusterID), Stats: &pdpb.StoreStats{ StoreId: store.GetId(), - Capacity: 1000 * (1 << 20), - Available: 1000 * (1 << 20), + Capacity: 1000 * units.MiB, + Available: 1000 * units.MiB, }, } grpcServer := &server.GrpcServer{Server: leaderServer.GetServer()}