diff --git a/pkg/mock/mockcluster/mockcluster.go b/pkg/mock/mockcluster/mockcluster.go index 017da7db475..93326fd46fc 100644 --- a/pkg/mock/mockcluster/mockcluster.go +++ b/pkg/mock/mockcluster/mockcluster.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/pd/v4/server/schedule/placement" "github.com/pingcap/pd/v4/server/schedule/storelimit" "github.com/pingcap/pd/v4/server/statistics" + "github.com/pingcap/pd/v4/server/versioninfo" "go.uber.org/zap" ) @@ -636,6 +637,11 @@ func (mc *Cluster) SetStoreLabel(storeID uint64, labels map[string]string) { mc.PutStore(newStore) } +// IsFeatureSupported checks if the feature is supported by current cluster. +func (mc *Cluster) IsFeatureSupported(versioninfo.Feature) bool { + return true +} + // AddSuspectRegions mock method func (mc *Cluster) AddSuspectRegions(ids ...uint64) { for _, id := range ids { diff --git a/server/api/config_test.go b/server/api/config_test.go index b339bda04b6..9bc7af99391 100644 --- a/server/api/config_test.go +++ b/server/api/config_test.go @@ -22,8 +22,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/pd/v4/pkg/typeutil" "github.com/pingcap/pd/v4/server" - "github.com/pingcap/pd/v4/server/cluster" "github.com/pingcap/pd/v4/server/config" + "github.com/pingcap/pd/v4/server/versioninfo" ) var _ = Suite(&testConfigSuite{}) @@ -107,7 +107,7 @@ func (s *testConfigSuite) TestConfigAll(c *C) { cfg.Log.Level = "warn" cfg.ReplicationMode.DRAutoSync.LabelKey = "foobar" cfg.ReplicationMode.ReplicationMode = "dr-auto-sync" - v, err := cluster.ParseVersion("v4.0.0-beta") + v, err := versioninfo.ParseVersion("v4.0.0-beta") c.Assert(err, IsNil) cfg.ClusterVersion = *v c.Assert(newCfg1, DeepEquals, cfg) diff --git a/server/api/operator_test.go b/server/api/operator_test.go index d41f5a452d9..c182b43d882 100644 --- a/server/api/operator_test.go +++ b/server/api/operator_test.go @@ -24,9 +24,9 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/pd/v4/server" - "github.com/pingcap/pd/v4/server/cluster" "github.com/pingcap/pd/v4/server/config" "github.com/pingcap/pd/v4/server/core" + "github.com/pingcap/pd/v4/server/versioninfo" ) var _ = Suite(&testOperatorSuite{}) @@ -143,7 +143,7 @@ func mustPutStore(c *C, svr *server.Server, id uint64, state metapb.StoreState, Address: fmt.Sprintf("tikv%d", id), State: state, Labels: labels, - Version: (*cluster.MinSupportedVersion(cluster.Version2_0)).String(), + Version: versioninfo.MinSupportedVersion(versioninfo.Version2_0).String(), }, }) c.Assert(err, IsNil) diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 500ad263cec..b5b175be901 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -45,6 +45,7 @@ import ( "github.com/pingcap/pd/v4/server/schedule/placement" "github.com/pingcap/pd/v4/server/schedule/storelimit" "github.com/pingcap/pd/v4/server/statistics" + "github.com/pingcap/pd/v4/server/versioninfo" "github.com/pkg/errors" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" @@ -901,12 +902,12 @@ func (c *RaftCluster) PutStore(store *metapb.Store, force bool) error { return errors.Errorf("invalid put store %v", store) } - v, err := ParseVersion(store.GetVersion()) + v, err := versioninfo.ParseVersion(store.GetVersion()) if err != nil { return errors.Errorf("invalid put store %v, error: %s", store, err) } clusterVersion := *c.opt.GetClusterVersion() - if !IsCompatible(clusterVersion, *v) { + if !versioninfo.IsCompatible(clusterVersion, *v) { return errors.Errorf("version should compatible with version %s, got %s", clusterVersion, v) } @@ -1289,30 +1290,26 @@ func (c *RaftCluster) AllocID() (uint64, error) { func (c *RaftCluster) OnStoreVersionChange() { c.RLock() defer c.RUnlock() - var ( - minVersion *semver.Version - clusterVersion *semver.Version - ) - + var minVersion *semver.Version stores := c.GetStores() for _, s := range stores { if s.IsTombstone() { continue } - v := MustParseVersion(s.GetVersion()) + v := versioninfo.MustParseVersion(s.GetVersion()) if minVersion == nil || v.LessThan(*minVersion) { minVersion = v } } - clusterVersion = c.opt.GetClusterVersion() + clusterVersion := c.opt.GetClusterVersion() // If the cluster version of PD is less than the minimum version of all stores, // it will update the cluster version. failpoint.Inject("versionChangeConcurrency", func() { time.Sleep(500 * time.Millisecond) }) - if (*clusterVersion).LessThan(*minVersion) { + if minVersion != nil && clusterVersion.LessThan(*minVersion) { if !c.opt.CASClusterVersion(clusterVersion, minVersion) { log.Error("cluster version changed by API at the same time") } @@ -1331,12 +1328,18 @@ func (c *RaftCluster) changedRegionNotifier() <-chan *core.RegionInfo { } // IsFeatureSupported checks if the feature is supported by current cluster. -func (c *RaftCluster) IsFeatureSupported(f Feature) bool { +func (c *RaftCluster) IsFeatureSupported(f versioninfo.Feature) bool { c.RLock() defer c.RUnlock() clusterVersion := *c.opt.GetClusterVersion() - minSupportVersion := *MinSupportedVersion(f) - return !clusterVersion.LessThan(minSupportVersion) + minSupportVersion := *versioninfo.MinSupportedVersion(f) + // For features before version 5.0 (such as BatchSplit), strict version checks are performed according to the + // original logic. But according to Semantic Versioning, specify a version MAJOR.MINOR.PATCH, PATCH is used when you + // make backwards compatible bug fixes. In version 5.0 and later, we need to strictly comply. + if versioninfo.IsCompatible(minSupportVersion, *versioninfo.MinSupportedVersion(versioninfo.Version4_0)) { + return !clusterVersion.LessThan(minSupportVersion) + } + return versioninfo.IsCompatible(minSupportVersion, clusterVersion) } // GetConfig gets config from cluster. diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index 2cccdbf7690..1c0087fdfb3 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/pd/v4/server/id" "github.com/pingcap/pd/v4/server/kv" "github.com/pingcap/pd/v4/server/schedule/opt" + "github.com/pingcap/pd/v4/server/versioninfo" ) func Test(t *testing.T) { @@ -663,7 +664,7 @@ func newTestScheduleConfig() (*config.ScheduleConfig, *config.PersistOptions, er return nil, nil, err } opt := config.NewPersistOptions(cfg) - opt.SetClusterVersion(MinSupportedVersion(Version2_0)) + opt.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version2_0)) return &cfg.Schedule, opt, nil } diff --git a/server/cluster/cluster_worker.go b/server/cluster/cluster_worker.go index 3e916921c04..8e50420fecb 100644 --- a/server/cluster/cluster_worker.go +++ b/server/cluster/cluster_worker.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/pd/v4/server/core" "github.com/pingcap/pd/v4/server/schedule" "github.com/pingcap/pd/v4/server/schedulers" + "github.com/pingcap/pd/v4/server/versioninfo" "github.com/pkg/errors" "go.uber.org/zap" ) @@ -66,7 +67,7 @@ func (c *RaftCluster) HandleAskSplit(request *pdpb.AskSplitRequest) (*pdpb.AskSp } } - if c.IsFeatureSupported(RegionMerge) { + if c.IsFeatureSupported(versioninfo.RegionMerge) { // Disable merge for the 2 regions in a period of time. c.GetMergeChecker().RecordRegionSplit([]uint64{reqRegion.GetId(), newRegionID}) } @@ -132,7 +133,7 @@ func (c *RaftCluster) HandleAskBatchSplit(request *pdpb.AskBatchSplitRequest) (* } recordRegions = append(recordRegions, reqRegion.GetId()) - if c.IsFeatureSupported(RegionMerge) { + if c.IsFeatureSupported(versioninfo.RegionMerge) { // Disable merge the regions in a period of time. c.GetMergeChecker().RecordRegionSplit(recordRegions) } diff --git a/server/grpc_service.go b/server/grpc_service.go index 77812d5e013..4bcf0f4e699 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/pd/v4/server/cluster" "github.com/pingcap/pd/v4/server/core" + "github.com/pingcap/pd/v4/server/versioninfo" "github.com/pkg/errors" "go.uber.org/zap" "google.golang.org/grpc/codes" @@ -551,7 +552,7 @@ func (s *Server) AskBatchSplit(ctx context.Context, request *pdpb.AskBatchSplitR return &pdpb.AskBatchSplitResponse{Header: s.notBootstrappedHeader()}, nil } - if !rc.IsFeatureSupported(cluster.BatchSplit) { + if !rc.IsFeatureSupported(versioninfo.BatchSplit) { return &pdpb.AskBatchSplitResponse{Header: s.incompatibleVersion("batch_split")}, nil } if request.GetRegion() == nil { diff --git a/server/schedule/opt/opts.go b/server/schedule/opt/opts.go index ca6ab5a2ade..5f2f00a6b99 100644 --- a/server/schedule/opt/opts.go +++ b/server/schedule/opt/opts.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/pd/v4/server/schedule/placement" "github.com/pingcap/pd/v4/server/schedule/storelimit" "github.com/pingcap/pd/v4/server/statistics" + "github.com/pingcap/pd/v4/server/versioninfo" ) // Options for schedulers. @@ -89,6 +90,7 @@ type Cluster interface { AllocID() (uint64, error) FitRegion(*core.RegionInfo) *placement.RegionFit RemoveScheduler(name string) error + IsFeatureSupported(f versioninfo.Feature) bool AddSuspectRegions(ids ...uint64) } diff --git a/server/server.go b/server/server.go index b6af4df34e4..ff71942c863 100644 --- a/server/server.go +++ b/server/server.go @@ -913,7 +913,7 @@ func (s *Server) GetLabelProperty() config.LabelPropertyConfig { // SetClusterVersion sets the version of cluster. func (s *Server) SetClusterVersion(v string) error { - version, err := cluster.ParseVersion(v) + version, err := versioninfo.ParseVersion(v) if err != nil { return err } diff --git a/server/util.go b/server/util.go index 79a0799ded6..f1d8bf64477 100644 --- a/server/util.go +++ b/server/util.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/log" "github.com/pingcap/pd/v4/pkg/etcdutil" "github.com/pingcap/pd/v4/pkg/typeutil" - "github.com/pingcap/pd/v4/server/cluster" "github.com/pingcap/pd/v4/server/config" "github.com/pingcap/pd/v4/server/versioninfo" "github.com/pkg/errors" @@ -69,9 +68,9 @@ func PrintConfigCheckMsg(cfg *config.Config) { // CheckPDVersion checks if PD needs to be upgraded. func CheckPDVersion(opt *config.PersistOptions) { - pdVersion := *cluster.MinSupportedVersion(cluster.Base) + pdVersion := versioninfo.MinSupportedVersion(versioninfo.Base) if versioninfo.PDReleaseVersion != "None" { - pdVersion = *cluster.MustParseVersion(versioninfo.PDReleaseVersion) + pdVersion = versioninfo.MustParseVersion(versioninfo.PDReleaseVersion) } clusterVersion := *opt.GetClusterVersion() log.Info("load cluster version", zap.Stringer("cluster-version", clusterVersion)) diff --git a/server/cluster/version.go b/server/versioninfo/feature.go similarity index 59% rename from server/cluster/version.go rename to server/versioninfo/feature.go index 9ec745083a3..cbece548b03 100644 --- a/server/cluster/version.go +++ b/server/versioninfo/feature.go @@ -1,4 +1,4 @@ -// Copyright 2018 PingCAP, Inc. +// 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. @@ -11,12 +11,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -package cluster +package versioninfo import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/log" - "github.com/pkg/errors" "go.uber.org/zap" ) @@ -37,13 +36,22 @@ const ( // BatchSplit can speed up the region split. // and PD will response the BatchSplit request. BatchSplit + Version3_0 + Version4_0 + Version5_0 + // JointConsensus can support safe conf change across data center. + JointConsensus ) var featuresDict = map[Feature]string{ - Base: "1.0.0", - Version2_0: "2.0.0", - RegionMerge: "2.0.0", - BatchSplit: "2.1.0-rc.1", + Base: "1.0.0", + Version2_0: "2.0.0", + RegionMerge: "2.0.0", + BatchSplit: "2.1.0-rc.1", + Version3_0: "3.0.0", + Version4_0: "4.0.0", + Version5_0: "5.0.0", + JointConsensus: "5.0.0", } // MinSupportedVersion returns the minimum support version for the specified feature. @@ -55,33 +63,3 @@ func MinSupportedVersion(v Feature) *semver.Version { version := MustParseVersion(target) return version } - -// ParseVersion wraps semver.NewVersion and handles compatibility issues. -func ParseVersion(v string) (*semver.Version, error) { - // for compatibility with old version which not support `version` mechanism. - if v == "" { - return semver.New(featuresDict[Base]), nil - } - if v[0] == 'v' { - v = v[1:] - } - ver, err := semver.NewVersion(v) - return ver, errors.WithStack(err) -} - -// MustParseVersion wraps ParseVersion and will panic if error is not nil. -func MustParseVersion(v string) *semver.Version { - ver, err := ParseVersion(v) - if err != nil { - log.Fatal("version string is illegal", zap.Error(err)) - } - return ver -} - -// IsCompatible checks if the clusterVersion is compatible with the specified version. -func IsCompatible(clusterVersion, v semver.Version) bool { - if clusterVersion.LessThan(v) { - return true - } - return clusterVersion.Major == v.Major && clusterVersion.Minor == v.Minor -} diff --git a/server/versioninfo/versioninfo.go b/server/versioninfo/versioninfo.go index 71b4a2e3f6e..e92811b0f89 100644 --- a/server/versioninfo/versioninfo.go +++ b/server/versioninfo/versioninfo.go @@ -13,6 +13,13 @@ package versioninfo +import ( + "github.com/coreos/go-semver/semver" + "github.com/pingcap/log" + "github.com/pkg/errors" + "go.uber.org/zap" +) + const ( // CommunityEdition is the default edition for building. CommunityEdition = "Community" @@ -26,3 +33,33 @@ var ( PDGitBranch = "None" PDEdition = CommunityEdition ) + +// ParseVersion wraps semver.NewVersion and handles compatibility issues. +func ParseVersion(v string) (*semver.Version, error) { + // for compatibility with old version which not support `version` mechanism. + if v == "" { + return semver.New(featuresDict[Base]), nil + } + if v[0] == 'v' { + v = v[1:] + } + ver, err := semver.NewVersion(v) + return ver, errors.WithStack(err) +} + +// MustParseVersion wraps ParseVersion and will panic if error is not nil. +func MustParseVersion(v string) *semver.Version { + ver, err := ParseVersion(v) + if err != nil { + log.Fatal("version string is illegal", zap.Error(err)) + } + return ver +} + +// IsCompatible checks if the version a is compatible with the version b. +func IsCompatible(a, b semver.Version) bool { + if a.LessThan(b) { + return true + } + return a.Major == b.Major && a.Minor == b.Minor +} diff --git a/tests/pdctl/helper.go b/tests/pdctl/helper.go index 37ddf69cbd6..79dc174351a 100644 --- a/tests/pdctl/helper.go +++ b/tests/pdctl/helper.go @@ -28,8 +28,8 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/pd/v4/server" "github.com/pingcap/pd/v4/server/api" - "github.com/pingcap/pd/v4/server/cluster" "github.com/pingcap/pd/v4/server/core" + "github.com/pingcap/pd/v4/server/versioninfo" "github.com/pingcap/pd/v4/tests" "github.com/pingcap/pd/v4/tools/pd-ctl/pdctl" "github.com/pingcap/pd/v4/tools/pd-ctl/pdctl/command" @@ -118,7 +118,7 @@ func MustPutStore(c *check.C, svr *server.Server, id uint64, state metapb.StoreS Address: fmt.Sprintf("tikv%d", id), State: state, Labels: labels, - Version: (*cluster.MinSupportedVersion(cluster.Version2_0)).String(), + Version: versioninfo.MinSupportedVersion(versioninfo.Version2_0).String(), }, }) c.Assert(err, check.IsNil)