Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
41 commits
Select commit Hold shift + click to select a range
ec98c72
save work
okJiang Dec 11, 2024
1d87074
Merge branch 'master' of github.com:tikv/pd into key-path-3
okJiang Dec 12, 2024
ae0ecb0
save work
okJiang Dec 12, 2024
46daf9d
update resource_group and keyspace_group
okJiang Dec 16, 2024
2cc8896
update safe point
okJiang Dec 16, 2024
2416b53
update config and rule
okJiang Dec 16, 2024
357bbb4
remove rootPath from test file
okJiang Dec 16, 2024
987316f
fix lint
okJiang Dec 16, 2024
4cd4bc2
Merge branch 'master' of github.com:tikv/pd into key-path-3
okJiang Jan 22, 2025
885ae70
fix regexp
okJiang Jan 22, 2025
6a3c636
fix lint
okJiang Jan 22, 2025
9fe44ee
fix
okJiang Jan 22, 2025
8de091e
fix
okJiang Jan 22, 2025
8c088d7
fix prefix
okJiang Jan 22, 2025
6820f69
fix ut
okJiang Jan 22, 2025
55ab136
fix ut
okJiang Jan 23, 2025
a37c6d7
fix watch
okJiang Jan 26, 2025
c10de29
fix prefix
okJiang Feb 7, 2025
7484010
fix rule
okJiang Feb 7, 2025
9217632
fix resource group
okJiang Feb 7, 2025
344dd00
fix keyspace
okJiang Feb 8, 2025
50f40b7
remove useless key
okJiang Feb 8, 2025
957904f
Merge branch 'master' of github.com:tikv/pd into key-path-3
okJiang Feb 8, 2025
9a3439f
Merge branch 'master' of github.com:tikv/pd into key-path-3
okJiang Feb 19, 2025
0cf837a
Merge branch 'master' of github.com:tikv/pd into key-path-3
okJiang Feb 20, 2025
f656b20
fix comment
okJiang Feb 20, 2025
dab7bb9
fix lint
okJiang Feb 20, 2025
40a384a
fix ut
okJiang Feb 20, 2025
9f145bf
remove useless code
okJiang Feb 20, 2025
2090dd1
add log
okJiang Feb 20, 2025
8dc31e2
fix lint
okJiang Feb 20, 2025
483e7e7
remove tiflash from realclustertest
okJiang Feb 21, 2025
35f30a8
Merge branch 'key-path-3' of github.com:okJiang/pd into key-path-3
okJiang Feb 21, 2025
9c00dcd
Merge branch 'master' of github.com:tikv/pd into key-path-3
okJiang Feb 24, 2025
7bb7351
fix comment
okJiang Feb 25, 2025
6f124eb
update ut and add comment
okJiang Feb 25, 2025
6d422c4
fix comment
okJiang Feb 25, 2025
30d74bb
Merge branch 'master' into key-path-3
ti-chi-bot[bot] Feb 26, 2025
ee022de
Merge branch 'master' of github.com:tikv/pd into key-path-3
okJiang Feb 26, 2025
a4a9fa1
Merge branch 'master' into key-path-3
ti-chi-bot[bot] Feb 26, 2025
b4e33a5
Merge branch 'master' into key-path-3
ti-chi-bot[bot] Feb 26, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 1 addition & 3 deletions client/servicediscovery/tso_service_discovery.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,12 +41,10 @@ import (
)

const (
msServiceRootPath = "/ms"
tsoServiceName = "tso"
// tsoSvcDiscoveryFormat defines the key prefix for keyspace group primary election.
// The entire key is in the format of "/ms/<cluster-id>/tso/<group-id>/primary".
// The <group-id> is 5 digits integer with leading zeros.
tsoSvcDiscoveryFormat = msServiceRootPath + "/%d/" + tsoServiceName + "/%05d/primary"
tsoSvcDiscoveryFormat = "/ms/%d/tso/%05d/primary"
// initRetryInterval is the rpc retry interval during the initialization phase.
initRetryInterval = time.Second
// tsoQueryRetryMaxTimes is the max retry times for querying TSO.
Expand Down
2 changes: 1 addition & 1 deletion pkg/keyspace/tso_keyspace_group.go
Original file line number Diff line number Diff line change
Expand Up @@ -219,7 +219,7 @@ func (m *GroupManager) allocNodesToAllKeyspaceGroups(ctx context.Context) {
}

func (m *GroupManager) initTSONodesWatcher(client *clientv3.Client) {
tsoServiceKey := keypath.TSOPath()
tsoServiceKey := keypath.ServicePath(constant.TSOServiceName)

putFn := func(kv *mvccpb.KeyValue) error {
s := &discovery.ServiceRegistryEntry{}
Expand Down
13 changes: 3 additions & 10 deletions pkg/mcs/metastorage/server/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,12 @@ import (
"github.com/pingcap/log"

bs "github.com/tikv/pd/pkg/basicserver"
"github.com/tikv/pd/pkg/storage/endpoint"
"github.com/tikv/pd/pkg/storage/kv"
)

// Manager is the manager of resource group.
// Manager is the manager of meta storage.
type Manager struct {
srv bs.Server
client *clientv3.Client
storage *endpoint.StorageEndpoint
srv bs.Server
client *clientv3.Client
}

// NewManager returns a new Manager.
Expand All @@ -38,10 +35,6 @@ func NewManager(srv bs.Server) *Manager {
// The first initialization after the server is started.
srv.AddStartCallback(func() {
log.Info("meta storage starts to initialize", zap.String("name", srv.Name()))
m.storage = endpoint.NewStorageEndpoint(
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is unused.

kv.NewEtcdKVBase(srv.GetClient(), "meta_storage"),
nil,
)
m.client = srv.GetClient()
m.srv = srv
})
Expand Down
2 changes: 1 addition & 1 deletion pkg/mcs/resourcemanager/server/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ func NewManager[T ConfigProvider](srv bs.Server) *Manager {
srv.AddStartCallback(func() {
log.Info("resource group manager starts to initialize", zap.String("name", srv.Name()))
m.storage = endpoint.NewStorageEndpoint(
kv.NewEtcdKVBase(srv.GetClient(), "resource_group"),
kv.NewEtcdKVBase(srv.GetClient()),
nil,
)
m.srv = srv
Expand Down
5 changes: 2 additions & 3 deletions pkg/mcs/scheduling/server/config/watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,10 +186,9 @@ func (cw *Watcher) initializeTTLConfigWatcher() error {
}

func (cw *Watcher) initializeSchedulerConfigWatcher() error {
prefixToTrim := cw.schedulerConfigPathPrefix + "/"
putFn := func(kv *mvccpb.KeyValue) error {
key := string(kv.Key)
name := strings.TrimPrefix(key, prefixToTrim)
name := strings.TrimPrefix(key, cw.schedulerConfigPathPrefix)
log.Info("update scheduler config", zap.String("name", name),
zap.String("value", string(kv.Value)))
err := cw.storage.SaveSchedulerConfig(name, kv.Value)
Expand All @@ -210,7 +209,7 @@ func (cw *Watcher) initializeSchedulerConfigWatcher() error {
key := string(kv.Key)
log.Info("remove scheduler config", zap.String("key", key))
return cw.storage.RemoveSchedulerConfig(
strings.TrimPrefix(key, prefixToTrim),
strings.TrimPrefix(key, cw.schedulerConfigPathPrefix),
)
}
cw.schedulerConfigWatcher = etcdutil.NewLoopWatcher(
Expand Down
7 changes: 3 additions & 4 deletions pkg/mcs/scheduling/server/rule/watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ func (rw *Watcher) initializeRuleWatcher() error {
key := string(kv.Key)
if strings.HasPrefix(key, rw.rulesPathPrefix) {
log.Info("delete placement rule", zap.String("key", key))
ruleJSON, err := rw.ruleStorage.LoadRule(strings.TrimPrefix(key, rw.rulesPathPrefix+"/"))
ruleJSON, err := rw.ruleStorage.LoadRule(strings.TrimPrefix(key, rw.rulesPathPrefix))
if err != nil {
return err
}
Expand All @@ -174,7 +174,7 @@ func (rw *Watcher) initializeRuleWatcher() error {
return err
} else if strings.HasPrefix(key, rw.ruleGroupPathPrefix) {
log.Info("delete placement rule group", zap.String("key", key))
trimmedKey := strings.TrimPrefix(key, rw.ruleGroupPathPrefix+"/")
trimmedKey := strings.TrimPrefix(key, rw.ruleGroupPathPrefix)
// Try to add the rule group change to the patch.
rw.patch.DeleteGroup(trimmedKey)
// Update the suspect key ranges
Expand Down Expand Up @@ -211,7 +211,6 @@ func (rw *Watcher) initializeRuleWatcher() error {
}

func (rw *Watcher) initializeRegionLabelWatcher() error {
prefixToTrim := rw.regionLabelPathPrefix + "/"
// TODO: use txn in region labeler.
preEventsFn := func([]*clientv3.Event) error {
// It will be locked until the postEventsFn is finished.
Expand All @@ -229,7 +228,7 @@ func (rw *Watcher) initializeRegionLabelWatcher() error {
deleteFn := func(kv *mvccpb.KeyValue) error {
key := string(kv.Key)
log.Info("delete region label rule", zap.String("key", key))
return rw.regionLabeler.DeleteLabelRuleLocked(strings.TrimPrefix(key, prefixToTrim))
return rw.regionLabeler.DeleteLabelRuleLocked(strings.TrimPrefix(key, rw.regionLabelPathPrefix))
}
postEventsFn := func([]*clientv3.Event) error {
defer rw.regionLabeler.Unlock()
Expand Down
2 changes: 1 addition & 1 deletion pkg/mcs/scheduling/server/rule/watcher_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ func prepare(t require.TestingT) (context.Context, *clientv3.Client, func()) {
}
value, err := json.Marshal(rule)
re.NoError(err)
key := keypath.RegionLabelPathPrefix() + "/" + rule.ID
key := keypath.RegionLabelKeyPath(rule.ID)
_, err = clientv3.NewKV(client).Put(ctx, key, string(value))
re.NoError(err)
}
Expand Down
4 changes: 1 addition & 3 deletions pkg/mcs/tso/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -346,11 +346,9 @@ func (s *Server) startServer() (err error) {

// Initialize the TSO service.
s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.Context())
legacySvcRootPath := keypath.LegacyRootPath()
tsoSvcRootPath := keypath.TSOSvcRootPath()
s.keyspaceGroupManager = tso.NewKeyspaceGroupManager(
s.serverLoopCtx, s.serviceID, s.GetClient(), s.GetHTTPClient(),
s.cfg.AdvertiseListenAddr, legacySvcRootPath, tsoSvcRootPath, s.cfg)
s.cfg.AdvertiseListenAddr, s.cfg)
if err := s.keyspaceGroupManager.Initialize(); err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/schedule/labeler/labeler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ func TestTxnWithEtcd(t *testing.T) {
re := require.New(t)
_, client, clean := etcdutil.NewTestEtcdCluster(t, 1)
defer clean()
store := storage.NewStorageWithEtcdBackend(client, "")
store := storage.NewStorageWithEtcdBackend(client)
labeler, err := NewRegionLabeler(context.Background(), store, time.Millisecond*10)
re.NoError(err)
// test patch rules in batch
Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/endpoint/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ var _ ConfigStorage = (*StorageEndpoint)(nil)

// LoadConfig loads config from keypath.Config then unmarshal it to cfg.
func (se *StorageEndpoint) LoadConfig(cfg any) (bool, error) {
value, err := se.Load(keypath.Config)
value, err := se.Load(keypath.ConfigPath())
if err != nil || value == "" {
return false, err
}
Expand All @@ -53,12 +53,12 @@ func (se *StorageEndpoint) LoadConfig(cfg any) (bool, error) {

// SaveConfig stores marshallable cfg to the keypath.Config.
func (se *StorageEndpoint) SaveConfig(cfg any) error {
return se.saveJSON(keypath.Config, cfg)
return se.saveJSON(keypath.ConfigPath(), cfg)
}

// LoadAllSchedulerConfigs loads all schedulers' config.
func (se *StorageEndpoint) LoadAllSchedulerConfigs() (keys, values []string, err error) {
prefix := keypath.CustomSchedulerConfigPath + "/"
prefix := keypath.SchedulerConfigPathPrefix()
keys, values, err = se.LoadRange(prefix, clientv3.GetPrefixRangeEnd(prefix), MinKVRangeLimit)
for i, key := range keys {
keys[i] = strings.TrimPrefix(key, prefix)
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/endpoint/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,13 +65,13 @@ const (
// LoadMeta loads cluster meta from the storage. This method will only
// be used by the PD server, so we should only implement it for the etcd storage.
func (se *StorageEndpoint) LoadMeta(meta *metapb.Cluster) (bool, error) {
return se.loadProto(keypath.ClusterPath, meta)
return se.loadProto(keypath.ClusterPath(), meta)
}

// SaveMeta save cluster meta to the storage. This method will only
// be used by the PD server, so we should only implement it for the etcd storage.
func (se *StorageEndpoint) SaveMeta(meta *metapb.Cluster) error {
return se.saveProto(keypath.ClusterPath, meta)
return se.saveProto(keypath.ClusterPath(), meta)
}

// LoadStoreMeta loads one store from storage.
Expand Down
16 changes: 8 additions & 8 deletions pkg/storage/endpoint/resource_group.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,40 +36,40 @@

// SaveResourceGroupSetting stores a resource group to storage.
func (se *StorageEndpoint) SaveResourceGroupSetting(name string, msg proto.Message) error {
return se.saveProto(keypath.ResourceGroupSettingKeyPath(name), msg)
return se.saveProto(keypath.ResourceGroupSettingPath(name), msg)
}

// DeleteResourceGroupSetting removes a resource group from storage.
func (se *StorageEndpoint) DeleteResourceGroupSetting(name string) error {
return se.Remove(keypath.ResourceGroupSettingKeyPath(name))
return se.Remove(keypath.ResourceGroupSettingPath(name))
}

// LoadResourceGroupSettings loads all resource groups from storage.
func (se *StorageEndpoint) LoadResourceGroupSettings(f func(k, v string)) error {
return se.loadRangeByPrefix(keypath.ResourceGroupSettingsPath+"/", f)
return se.loadRangeByPrefix(keypath.ResourceGroupSettingPrefix(), f)
}

// SaveResourceGroupStates stores a resource group to storage.
func (se *StorageEndpoint) SaveResourceGroupStates(name string, obj any) error {
return se.saveJSON(keypath.ResourceGroupStateKeyPath(name), obj)
return se.saveJSON(keypath.ResourceGroupStatePath(name), obj)
}

// DeleteResourceGroupStates removes a resource group from storage.
func (se *StorageEndpoint) DeleteResourceGroupStates(name string) error {
return se.Remove(keypath.ResourceGroupStateKeyPath(name))
return se.Remove(keypath.ResourceGroupStatePath(name))

Check warning on line 59 in pkg/storage/endpoint/resource_group.go

View check run for this annotation

Codecov / codecov/patch

pkg/storage/endpoint/resource_group.go#L59

Added line #L59 was not covered by tests
}

// LoadResourceGroupStates loads all resource groups from storage.
func (se *StorageEndpoint) LoadResourceGroupStates(f func(k, v string)) error {
return se.loadRangeByPrefix(keypath.ResourceGroupStatesPath+"/", f)
return se.loadRangeByPrefix(keypath.ResourceGroupStatePrefix(), f)
}

// SaveControllerConfig stores the resource controller config to storage.
func (se *StorageEndpoint) SaveControllerConfig(config any) error {
return se.saveJSON(keypath.ControllerConfigPath, config)
return se.saveJSON(keypath.ControllerConfigPath(), config)
}

// LoadControllerConfig loads the resource controller config from storage.
func (se *StorageEndpoint) LoadControllerConfig() (string, error) {
return se.Load(keypath.ControllerConfigPath)
return se.Load(keypath.ControllerConfigPath())
}
6 changes: 3 additions & 3 deletions pkg/storage/endpoint/rule.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func (*StorageEndpoint) DeleteRule(txn kv.Txn, ruleKey string) error {

// LoadRuleGroups loads all rule groups from storage.
func (se *StorageEndpoint) LoadRuleGroups(f func(k, v string)) error {
return se.loadRangeByPrefix(keypath.RuleGroupPath+"/", f)
return se.loadRangeByPrefix(keypath.RuleGroupPathPrefix(), f)
}

// SaveRuleGroup stores a rule group config to storage.
Expand All @@ -71,7 +71,7 @@ func (*StorageEndpoint) DeleteRuleGroup(txn kv.Txn, groupID string) error {

// LoadRegionRules loads region rules from storage.
func (se *StorageEndpoint) LoadRegionRules(f func(k, v string)) error {
return se.loadRangeByPrefix(keypath.RegionLabelPath+"/", f)
return se.loadRangeByPrefix(keypath.RegionLabelPathPrefix(), f)
}

// SaveRegionRule saves a region rule to the storage.
Expand All @@ -91,5 +91,5 @@ func (se *StorageEndpoint) LoadRule(ruleKey string) (string, error) {

// LoadRules loads placement rules from storage.
func (se *StorageEndpoint) LoadRules(f func(k, v string)) error {
return se.loadRangeByPrefix(keypath.RulesPath+"/", f)
return se.loadRangeByPrefix(keypath.RulesPathPrefix(), f)
}
4 changes: 2 additions & 2 deletions pkg/storage/endpoint/service_middleware.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ var _ ServiceMiddlewareStorage = (*StorageEndpoint)(nil)

// LoadServiceMiddlewareConfig loads service middleware config from ServiceMiddlewarePath then unmarshal it to cfg.
func (se *StorageEndpoint) LoadServiceMiddlewareConfig(cfg any) (bool, error) {
value, err := se.Load(keypath.ServiceMiddlewarePath)
value, err := se.Load(keypath.ServiceMiddlewarePath())
if err != nil || value == "" {
return false, err
}
Expand All @@ -44,5 +44,5 @@ func (se *StorageEndpoint) LoadServiceMiddlewareConfig(cfg any) (bool, error) {

// SaveServiceMiddlewareConfig stores marshallable cfg to the ServiceMiddlewarePath.
func (se *StorageEndpoint) SaveServiceMiddlewareConfig(cfg any) error {
return se.saveJSON(keypath.ServiceMiddlewarePath, cfg)
return se.saveJSON(keypath.ServiceMiddlewarePath(), cfg)
}
18 changes: 9 additions & 9 deletions pkg/storage/endpoint/tso.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,8 @@
// TSOStorage is the interface for timestamp storage.
type TSOStorage interface {
LoadTimestamp(prefix string) (time.Time, error)
SaveTimestamp(key string, ts time.Time) error
DeleteTimestamp(key string) error
SaveTimestamp(groupID uint32, ts time.Time) error
DeleteTimestamp(groupID uint32) error
}

var _ TSOStorage = (*StorageEndpoint)(nil)
Expand All @@ -55,7 +55,7 @@
maxTSWindow := typeutil.ZeroTime
for i, key := range keys {
key := strings.TrimSpace(key)
if !strings.HasSuffix(key, keypath.TimestampKey) {
if !strings.HasSuffix(key, "timestamp") {
continue
}
tsWindow, err := typeutil.ParseTimestamp([]byte(values[i]))
Expand All @@ -71,9 +71,9 @@
}

// SaveTimestamp saves the timestamp to the storage.
func (se *StorageEndpoint) SaveTimestamp(key string, ts time.Time) error {
func (se *StorageEndpoint) SaveTimestamp(groupID uint32, ts time.Time) error {
return se.RunInTxn(context.Background(), func(txn kv.Txn) error {
value, err := txn.Load(key)
value, err := txn.Load(keypath.TimestampPath(groupID))
if err != nil {
return err
}
Expand All @@ -82,21 +82,21 @@
if value != "" {
previousTS, err = typeutil.ParseTimestamp([]byte(value))
if err != nil {
log.Error("parse timestamp failed", zap.String("key", key), zap.String("value", value), zap.Error(err))
log.Error("parse timestamp failed", zap.Uint32("group-id", groupID), zap.String("value", value), zap.Error(err))

Check warning on line 85 in pkg/storage/endpoint/tso.go

View check run for this annotation

Codecov / codecov/patch

pkg/storage/endpoint/tso.go#L85

Added line #L85 was not covered by tests
return err
}
}
if previousTS != typeutil.ZeroTime && typeutil.SubRealTimeByWallClock(ts, previousTS) <= 0 {
return errors.Errorf("saving timestamp %d is less than or equal to the previous one %d", ts.UnixNano(), previousTS.UnixNano())
}
data := typeutil.Uint64ToBytes(uint64(ts.UnixNano()))
return txn.Save(key, string(data))
return txn.Save(keypath.TimestampPath(groupID), string(data))
})
}

// DeleteTimestamp deletes the timestamp from the storage.
func (se *StorageEndpoint) DeleteTimestamp(key string) error {
func (se *StorageEndpoint) DeleteTimestamp(groupID uint32) error {
return se.RunInTxn(context.Background(), func(txn kv.Txn) error {
return txn.Remove(key)
return txn.Remove(keypath.TimestampPath(groupID))
})
}
4 changes: 2 additions & 2 deletions pkg/storage/etcd_backend.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,10 @@ type etcdBackend struct {
}

// newEtcdBackend is used to create a new etcd backend.
func newEtcdBackend(client *clientv3.Client, rootPath string) *etcdBackend {
func newEtcdBackend(client *clientv3.Client) *etcdBackend {
return &etcdBackend{
endpoint.NewStorageEndpoint(
kv.NewEtcdKVBase(client, rootPath),
kv.NewEtcdKVBase(client),
nil,
),
}
Expand Down
11 changes: 6 additions & 5 deletions pkg/storage/keyspace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,9 +147,10 @@ func makeTestKeyspaces() []*keyspacepb.KeyspaceMeta {
// TestEncodeSpaceID test spaceID encoding.
func TestEncodeSpaceID(t *testing.T) {
re := require.New(t)
re.Equal("keyspaces/meta/00000000", keypath.KeyspaceMetaPath(0))
re.Equal("keyspaces/meta/16777215", keypath.KeyspaceMetaPath(1<<24-1))
re.Equal("keyspaces/meta/00000100", keypath.KeyspaceMetaPath(100))
re.Equal("keyspaces/meta/00000011", keypath.KeyspaceMetaPath(11))
re.Equal("keyspaces/meta/00000010", keypath.KeyspaceMetaPath(10))
prefix := keypath.KeyspaceMetaPrefix()
re.Equal(prefix+"00000000", keypath.KeyspaceMetaPath(0))
re.Equal(prefix+"16777215", keypath.KeyspaceMetaPath(1<<24-1))
re.Equal(prefix+"00000100", keypath.KeyspaceMetaPath(100))
re.Equal(prefix+"00000011", keypath.KeyspaceMetaPath(11))
re.Equal(prefix+"00000010", keypath.KeyspaceMetaPath(10))
}
Loading