Skip to content

Commit

Permalink
Merge branch 'master' into api-version
Browse files Browse the repository at this point in the history
  • Loading branch information
rleungx committed May 18, 2023
2 parents 86833c8 + 3fa9c36 commit 88762e4
Show file tree
Hide file tree
Showing 33 changed files with 42 additions and 42 deletions.
2 changes: 1 addition & 1 deletion cmd/pd-server/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import (
"github.com/tikv/pd/pkg/autoscaling"
"github.com/tikv/pd/pkg/dashboard"
"github.com/tikv/pd/pkg/errs"
resource_manager "github.com/tikv/pd/pkg/mcs/resource_manager/server"
resource_manager "github.com/tikv/pd/pkg/mcs/resourcemanager/server"
tso "github.com/tikv/pd/pkg/mcs/tso/server"
"github.com/tikv/pd/pkg/schedule/schedulers"
"github.com/tikv/pd/pkg/swaggerserver"
Expand Down
File renamed without changes.
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
package install

import (
ms_server "github.com/tikv/pd/pkg/mcs/meta_storage/server"
ms_server "github.com/tikv/pd/pkg/mcs/metastorage/server"
"github.com/tikv/pd/pkg/mcs/registry"
)

Expand Down
File renamed without changes.
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (
"github.com/gin-gonic/gin"
"github.com/joho/godotenv"
rmpb "github.com/pingcap/kvproto/pkg/resource_manager"
rmserver "github.com/tikv/pd/pkg/mcs/resource_manager/server"
rmserver "github.com/tikv/pd/pkg/mcs/resourcemanager/server"
"github.com/tikv/pd/pkg/mcs/utils"
"github.com/tikv/pd/pkg/utils/apiutil"
"github.com/tikv/pd/pkg/utils/apiutil/multiservicesapi"
Expand Down
File renamed without changes.
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,10 @@ package install

import (
"github.com/tikv/pd/pkg/mcs/registry"
rm_server "github.com/tikv/pd/pkg/mcs/resource_manager/server"
rm_server "github.com/tikv/pd/pkg/mcs/resourcemanager/server"

// init API group
_ "github.com/tikv/pd/pkg/mcs/resource_manager/server/apis/v1"
_ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/apis/v1"
)

func init() {
Expand Down
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
18 changes: 9 additions & 9 deletions pkg/storage/endpoint/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,11 @@ import (
type MetaStorage interface {
LoadMeta(meta *metapb.Cluster) (bool, error)
SaveMeta(meta *metapb.Cluster) error
LoadStore(storeID uint64, store *metapb.Store) (bool, error)
SaveStore(store *metapb.Store) error
LoadStoreMeta(storeID uint64, store *metapb.Store) (bool, error)
SaveStoreMeta(store *metapb.Store) error
SaveStoreWeight(storeID uint64, leader, region float64) error
LoadStores(f func(store *core.StoreInfo)) error
DeleteStore(store *metapb.Store) error
DeleteStoreMeta(store *metapb.Store) error
RegionStorage
}

Expand Down Expand Up @@ -71,13 +71,13 @@ func (se *StorageEndpoint) SaveMeta(meta *metapb.Cluster) error {
return se.saveProto(clusterPath, meta)
}

// LoadStore loads one store from storage.
func (se *StorageEndpoint) LoadStore(storeID uint64, store *metapb.Store) (bool, error) {
// LoadStoreMeta loads one store from storage.
func (se *StorageEndpoint) LoadStoreMeta(storeID uint64, store *metapb.Store) (bool, error) {
return se.loadProto(StorePath(storeID), store)
}

// SaveStore saves one store to storage.
func (se *StorageEndpoint) SaveStore(store *metapb.Store) error {
// SaveStoreMeta saves one store to storage.
func (se *StorageEndpoint) SaveStoreMeta(store *metapb.Store) error {
return se.saveProto(StorePath(store.GetId()), store)
}

Expand Down Expand Up @@ -146,8 +146,8 @@ func (se *StorageEndpoint) loadFloatWithDefaultValue(path string, def float64) (
return val, nil
}

// DeleteStore deletes one store from storage.
func (se *StorageEndpoint) DeleteStore(store *metapb.Store) error {
// DeleteStoreMeta deletes one store from storage.
func (se *StorageEndpoint) DeleteStoreMeta(store *metapb.Store) error {
return se.Remove(StorePath(store.GetId()))
}

Expand Down
8 changes: 4 additions & 4 deletions pkg/storage/storage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,12 +52,12 @@ func TestBasic(t *testing.T) {
re.Equal(meta, newMeta)

store := &metapb.Store{Id: 123}
ok, err = storage.LoadStore(123, store)
ok, err = storage.LoadStoreMeta(123, store)
re.False(ok)
re.NoError(err)
re.NoError(storage.SaveStore(store))
re.NoError(storage.SaveStoreMeta(store))
newStore := &metapb.Store{}
ok, err = storage.LoadStore(123, newStore)
ok, err = storage.LoadStoreMeta(123, newStore)
re.True(ok)
re.NoError(err)
re.Equal(store, newStore)
Expand Down Expand Up @@ -87,7 +87,7 @@ func mustSaveStores(re *require.Assertions, s Storage, n int) []*metapb.Store {
}

for _, store := range stores {
re.NoError(s.SaveStore(store))
re.NoError(s.SaveStoreMeta(store))
}

return stores
Expand Down
8 changes: 4 additions & 4 deletions server/cluster/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ import (
"github.com/tikv/pd/pkg/utils/typeutil"
"github.com/tikv/pd/pkg/versioninfo"
"github.com/tikv/pd/server/config"
syncer "github.com/tikv/pd/server/region_syncer"
syncer "github.com/tikv/pd/server/regionsyncer"
"github.com/tikv/pd/server/replication"
"go.etcd.io/etcd/clientv3"
"go.uber.org/zap"
Expand Down Expand Up @@ -892,7 +892,7 @@ func (c *RaftCluster) HandleStoreHeartbeat(heartbeat *pdpb.StoreHeartbeatRequest
zap.Uint64("available", newStore.GetAvailable()))
}
if newStore.NeedPersist() && c.storage != nil {
if err := c.storage.SaveStore(newStore.GetMeta()); err != nil {
if err := c.storage.SaveStoreMeta(newStore.GetMeta()); err != nil {
log.Error("failed to persist store", zap.Uint64("store-id", storeID), errs.ZapError(err))
} else {
newStore = newStore.Clone(core.SetLastPersistTime(nowTime))
Expand Down Expand Up @@ -1714,7 +1714,7 @@ func (c *RaftCluster) SetStoreWeight(storeID uint64, leaderWeight, regionWeight

func (c *RaftCluster) putStoreLocked(store *core.StoreInfo) error {
if c.storage != nil {
if err := c.storage.SaveStore(store.GetMeta()); err != nil {
if err := c.storage.SaveStoreMeta(store.GetMeta()); err != nil {
return err
}
}
Expand Down Expand Up @@ -2058,7 +2058,7 @@ func (c *RaftCluster) RemoveTombStoneRecords() error {
// deleteStore deletes the store from the cluster. it's concurrent safe.
func (c *RaftCluster) deleteStore(store *core.StoreInfo) error {
if c.storage != nil {
if err := c.storage.DeleteStore(store.GetMeta()); err != nil {
if err := c.storage.DeleteStoreMeta(store.GetMeta()); err != nil {
return err
}
}
Expand Down
2 changes: 1 addition & 1 deletion server/cluster/cluster_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ func TestStoreHeartbeat(t *testing.T) {

for i, store := range stores {
tmp := &metapb.Store{}
ok, err := cluster.storage.LoadStore(store.GetID(), tmp)
ok, err := cluster.storage.LoadStoreMeta(store.GetID(), tmp)
re.True(ok)
re.NoError(err)
re.Equal(storeMetasAfterHeartbeat[i], tmp)
Expand Down
2 changes: 1 addition & 1 deletion server/config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import (
"github.com/spf13/pflag"
"github.com/tikv/pd/pkg/core/storelimit"
"github.com/tikv/pd/pkg/errs"
rm "github.com/tikv/pd/pkg/mcs/resource_manager/server"
rm "github.com/tikv/pd/pkg/mcs/resourcemanager/server"
"github.com/tikv/pd/pkg/utils/configutil"
"github.com/tikv/pd/pkg/utils/grpcutil"
"github.com/tikv/pd/pkg/utils/metricutil"
Expand Down
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
File renamed without changes.
10 changes: 5 additions & 5 deletions server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,11 +48,11 @@ import (
"github.com/tikv/pd/pkg/errs"
"github.com/tikv/pd/pkg/id"
"github.com/tikv/pd/pkg/keyspace"
ms_server "github.com/tikv/pd/pkg/mcs/meta_storage/server"
ms_server "github.com/tikv/pd/pkg/mcs/metastorage/server"
"github.com/tikv/pd/pkg/mcs/registry"
rm_server "github.com/tikv/pd/pkg/mcs/resource_manager/server"
_ "github.com/tikv/pd/pkg/mcs/resource_manager/server/apis/v1" // init API group
_ "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" // init tso API group
rm_server "github.com/tikv/pd/pkg/mcs/resourcemanager/server"
_ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/apis/v1" // init API group
_ "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" // init tso API group
mcs "github.com/tikv/pd/pkg/mcs/utils"
"github.com/tikv/pd/pkg/member"
"github.com/tikv/pd/pkg/ratelimit"
Expand All @@ -75,7 +75,7 @@ import (
"github.com/tikv/pd/server/cluster"
"github.com/tikv/pd/server/config"
"github.com/tikv/pd/server/gc"
syncer "github.com/tikv/pd/server/region_syncer"
syncer "github.com/tikv/pd/server/regionsyncer"
"go.etcd.io/etcd/clientv3"
"go.etcd.io/etcd/embed"
"go.etcd.io/etcd/mvcc/mvccpb"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,14 +30,14 @@ import (
"github.com/stretchr/testify/suite"
pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/resource_group/controller"
"github.com/tikv/pd/pkg/mcs/resource_manager/server"
"github.com/tikv/pd/pkg/mcs/resourcemanager/server"
"github.com/tikv/pd/pkg/utils/testutil"
"github.com/tikv/pd/tests"
"go.uber.org/goleak"

// Register Service
_ "github.com/tikv/pd/pkg/mcs/registry"
_ "github.com/tikv/pd/pkg/mcs/resource_manager/server/install"
_ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/install"
)

func TestMain(m *testing.M) {
Expand All @@ -61,7 +61,7 @@ func (suite *resourceManagerClientTestSuite) SetupSuite() {
var err error
re := suite.Require()

re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/resource_manager/server/enableDegradedMode", `return(true)`))
re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/resourcemanager/server/enableDegradedMode", `return(true)`))

suite.ctx, suite.clean = context.WithCancel(context.Background())

Expand Down Expand Up @@ -146,7 +146,7 @@ func (suite *resourceManagerClientTestSuite) TearDownSuite() {
suite.client.Close()
suite.cluster.Destroy()
suite.clean()
re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/mcs/resource_manager/server/enableDegradedMode"))
re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/mcs/resourcemanager/server/enableDegradedMode"))
}

func (suite *resourceManagerClientTestSuite) TearDownTest() {
Expand Down Expand Up @@ -599,7 +599,7 @@ func (suite *resourceManagerClientTestSuite) TestAcquireTokenBucket() {
Requests: make([]*rmpb.TokenBucketRequest, 0),
TargetRequestPeriodMs: uint64(time.Second * 10 / time.Millisecond),
}
re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/resource_manager/server/fastPersist", `return(true)`))
re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/resourcemanager/server/fastPersist", `return(true)`))
suite.resignAndWaitLeader()
groups = append(groups, &rmpb.ResourceGroup{Name: "test3"})
for i := 0; i < 3; i++ {
Expand Down Expand Up @@ -647,7 +647,7 @@ func (suite *resourceManagerClientTestSuite) TestAcquireTokenBucket() {
re.NoError(err)
checkFunc(gresp, groups[0])
}
re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/mcs/resource_manager/server/fastPersist"))
re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/mcs/resourcemanager/server/fastPersist"))
}

func (suite *resourceManagerClientTestSuite) TestBasicResourceGroupCURD() {
Expand Down Expand Up @@ -933,7 +933,7 @@ func (suite *resourceManagerClientTestSuite) TestResourceManagerClientDegradedMo
WriteCostPerByte: 1,
CPUMsCost: 1,
}
re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/resource_manager/server/acquireFailed", `return(true)`))
re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/resourcemanager/server/acquireFailed", `return(true)`))
re.NoError(failpoint.Enable("github.com/tikv/pd/client/resource_group/controller/degradedModeRU", "return(true)"))
controller, _ := controller.NewResourceGroupController(suite.ctx, 1, cli, cfg)
controller.Start(suite.ctx)
Expand All @@ -959,7 +959,7 @@ func (suite *resourceManagerClientTestSuite) TestResourceManagerClientDegradedMo
// we can not check `inDegradedMode` because of data race.
re.True(endTime.Before(beginTime.Add(time.Second)))
controller.Stop()
re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/mcs/resource_manager/server/acquireFailed"))
re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/mcs/resourcemanager/server/acquireFailed"))
re.NoError(failpoint.Disable("github.com/tikv/pd/client/resource_group/controller/degradedModeRU"))
}

Expand Down Expand Up @@ -997,7 +997,7 @@ func (suite *resourceManagerClientTestSuite) TestLoadRequestUnitConfig() {
re.Equal(expectedConfig.WriteBaseCost, config.WriteBaseCost)
re.Equal(expectedConfig.WriteBytesCost, config.WriteBytesCost)
re.Equal(expectedConfig.CPUMsCost, config.CPUMsCost)
// refer github.com/tikv/pd/pkg/mcs/resource_manager/server/enableDegradedMode, check with 1s.
// refer github.com/tikv/pd/pkg/mcs/resourcemanager/server/enableDegradedMode, check with 1s.
re.Equal(time.Second, config.DegradedModeWaitDuration)
}

Expand Down
2 changes: 1 addition & 1 deletion tests/integrations/mcs/testutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (
"github.com/stretchr/testify/require"
pd "github.com/tikv/pd/client"
bs "github.com/tikv/pd/pkg/basicserver"
rm "github.com/tikv/pd/pkg/mcs/resource_manager/server"
rm "github.com/tikv/pd/pkg/mcs/resourcemanager/server"
tso "github.com/tikv/pd/pkg/mcs/tso/server"
"github.com/tikv/pd/pkg/utils/logutil"
"github.com/tikv/pd/pkg/utils/testutil"
Expand Down
6 changes: 3 additions & 3 deletions tests/server/cluster/cluster_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ import (
"github.com/tikv/pd/server"
"github.com/tikv/pd/server/cluster"
"github.com/tikv/pd/server/config"
syncer "github.com/tikv/pd/server/region_syncer"
syncer "github.com/tikv/pd/server/regionsyncer"
"github.com/tikv/pd/tests"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
Expand Down Expand Up @@ -831,7 +831,7 @@ func TestLoadClusterInfo(t *testing.T) {
}

for _, store := range stores {
re.NoError(testStorage.SaveStore(store))
re.NoError(testStorage.SaveStoreMeta(store))
}

regions := make([]*metapb.Region, 0, n)
Expand Down Expand Up @@ -1450,7 +1450,7 @@ func TestTransferLeaderBack(t *testing.T) {
}

for _, store := range stores {
re.NoError(storage.SaveStore(store))
re.NoError(storage.SaveStoreMeta(store))
}
rc, err = rc.LoadClusterInfo()
re.NoError(err)
Expand Down

0 comments on commit 88762e4

Please sign in to comment.