From 16a399edf95db541ce7b8e3851bac2ede8f7877a Mon Sep 17 00:00:00 2001 From: rambohe Date: Thu, 10 Nov 2022 10:51:13 +0800 Subject: [PATCH 01/20] add yurthub leader election and coordinator framework (#1035) * improve health checker for adapting coordinator * add yurthub leader election and coordinator framework --- cmd/yurthub/app/config/config.go | 3 + cmd/yurthub/app/options/options.go | 42 +++++ cmd/yurthub/app/start.go | 21 ++- pkg/yurthub/poolcoordinator/coordinator.go | 89 +++++++++++ .../poolcoordinator/leader_election.go | 143 ++++++++++++++++++ 5 files changed, 287 insertions(+), 11 deletions(-) create mode 100644 pkg/yurthub/poolcoordinator/coordinator.go create mode 100644 pkg/yurthub/poolcoordinator/leader_election.go diff --git a/cmd/yurthub/app/config/config.go b/cmd/yurthub/app/config/config.go index f333c411c7d..984cb3c224c 100644 --- a/cmd/yurthub/app/config/config.go +++ b/cmd/yurthub/app/config/config.go @@ -23,6 +23,8 @@ import ( "strings" "time" + componentbaseconfig "k8s.io/component-base/config" + corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/fields" @@ -89,6 +91,7 @@ type YurtHubConfiguration struct { YurtHubProxyServerServing *apiserver.DeprecatedInsecureServingInfo YurtHubDummyProxyServerServing *apiserver.DeprecatedInsecureServingInfo YurtHubSecureProxyServerServing *apiserver.SecureServingInfo + LeaderElection componentbaseconfig.LeaderElectionConfiguration } // Complete converts *options.YurtHubOptions to *YurtHubConfiguration diff --git a/cmd/yurthub/app/options/options.go b/cmd/yurthub/app/options/options.go index 94075e70d6b..508a7245a98 100644 --- a/cmd/yurthub/app/options/options.go +++ b/cmd/yurthub/app/options/options.go @@ -23,7 +23,10 @@ import ( "time" "github.com/spf13/pflag" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/client-go/kubernetes" + "k8s.io/client-go/tools/leaderelection/resourcelock" + componentbaseconfig "k8s.io/component-base/config" "k8s.io/klog/v2" utilnet "k8s.io/utils/net" @@ -76,6 +79,7 @@ type YurtHubOptions struct { CACertHashes []string UnsafeSkipCAVerification bool ClientForTest kubernetes.Interface + LeaderElection componentbaseconfig.LeaderElectionConfiguration } // NewYurtHubOptions creates a new YurtHubOptions with a default config. @@ -109,6 +113,15 @@ func NewYurtHubOptions() *YurtHubOptions { MinRequestTimeout: time.Second * 1800, CACertHashes: make([]string, 0), UnsafeSkipCAVerification: true, + LeaderElection: componentbaseconfig.LeaderElectionConfiguration{ + LeaderElect: true, + LeaseDuration: metav1.Duration{Duration: 15 * time.Second}, + RenewDeadline: metav1.Duration{Duration: 10 * time.Second}, + RetryPeriod: metav1.Duration{Duration: 2 * time.Second}, + ResourceLock: resourcelock.LeasesResourceLock, + ResourceName: projectinfo.GetHubName(), + ResourceNamespace: "kube-system", + }, } return o } @@ -182,6 +195,35 @@ func (o *YurtHubOptions) AddFlags(fs *pflag.FlagSet) { fs.DurationVar(&o.MinRequestTimeout, "min-request-timeout", o.MinRequestTimeout, "An optional field indicating at least how long a proxy handler must keep a request open before timing it out. Currently only honored by the local watch request handler(use request parameter timeoutSeconds firstly), which picks a randomized value above this number as the connection timeout, to spread out load.") fs.StringSliceVar(&o.CACertHashes, "discovery-token-ca-cert-hash", o.CACertHashes, "For token-based discovery, validate that the root CA public key matches this hash (format: \":\").") fs.BoolVar(&o.UnsafeSkipCAVerification, "discovery-token-unsafe-skip-ca-verification", o.UnsafeSkipCAVerification, "For token-based discovery, allow joining without --discovery-token-ca-cert-hash pinning.") + bindFlags(&o.LeaderElection, fs) +} + +// bindFlags binds the LeaderElectionConfiguration struct fields to a flagset +func bindFlags(l *componentbaseconfig.LeaderElectionConfiguration, fs *pflag.FlagSet) { + fs.BoolVar(&l.LeaderElect, "leader-elect", l.LeaderElect, ""+ + "Start a leader election client and gain leadership based on pool coordinator") + fs.DurationVar(&l.LeaseDuration.Duration, "leader-elect-lease-duration", l.LeaseDuration.Duration, ""+ + "The duration that non-leader candidates will wait after observing a leadership "+ + "renewal until attempting to acquire leadership of a led but unrenewed leader "+ + "slot. This is effectively the maximum duration that a leader can be stopped "+ + "before it is replaced by another candidate. This is only applicable if leader "+ + "election is enabled.") + fs.DurationVar(&l.RenewDeadline.Duration, "leader-elect-renew-deadline", l.RenewDeadline.Duration, ""+ + "The interval between attempts by the acting master to renew a leadership slot "+ + "before it stops leading. This must be less than or equal to the lease duration. "+ + "This is only applicable if leader election is enabled.") + fs.DurationVar(&l.RetryPeriod.Duration, "leader-elect-retry-period", l.RetryPeriod.Duration, ""+ + "The duration the clients should wait between attempting acquisition and renewal "+ + "of a leadership. This is only applicable if leader election is enabled.") + fs.StringVar(&l.ResourceLock, "leader-elect-resource-lock", l.ResourceLock, ""+ + "The type of resource object that is used for locking during "+ + "leader election. Supported options are `leases` (default), `endpoints` and `configmaps`.") + fs.StringVar(&l.ResourceName, "leader-elect-resource-name", l.ResourceName, ""+ + "The name of resource object that is used for locking during "+ + "leader election.") + fs.StringVar(&l.ResourceNamespace, "leader-elect-resource-namespace", l.ResourceNamespace, ""+ + "The namespace of resource object that is used for locking during "+ + "leader election.") } // verifyDummyIP verify the specified ip is valid or not and set the default ip if empty diff --git a/cmd/yurthub/app/start.go b/cmd/yurthub/app/start.go index ccfb7d02155..548999b1082 100644 --- a/cmd/yurthub/app/start.go +++ b/cmd/yurthub/app/start.go @@ -184,17 +184,16 @@ func createHealthCheckerClient(heartbeatTimeoutSeconds int, remoteServers []*url healthCheckerClientsForCloud[remoteServers[i].String()] = c } - // comment the following code temporarily - //cfg := &rest.Config{ - // Host: coordinatorServer.String(), - // Transport: tp.CurrentTransport(), - // Timeout: time.Duration(heartbeatTimeoutSeconds) * time.Second, - //} - //c, err := kubernetes.NewForConfig(cfg) - //if err != nil { - // return healthCheckerClientsForCloud, healthCheckerClientForCoordinator, err - //} - //healthCheckerClientForCoordinator = c + cfg := &rest.Config{ + Host: coordinatorServer.String(), + Transport: tp.CurrentTransport(), + Timeout: time.Duration(heartbeatTimeoutSeconds) * time.Second, + } + c, err := kubernetes.NewForConfig(cfg) + if err != nil { + return healthCheckerClientsForCloud, healthCheckerClientForCoordinator, err + } + healthCheckerClientForCoordinator = c return healthCheckerClientsForCloud, healthCheckerClientForCoordinator, nil } diff --git a/pkg/yurthub/poolcoordinator/coordinator.go b/pkg/yurthub/poolcoordinator/coordinator.go new file mode 100644 index 00000000000..6afdc8ffd84 --- /dev/null +++ b/pkg/yurthub/poolcoordinator/coordinator.go @@ -0,0 +1,89 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 poolcoordinator + +import ( + "k8s.io/klog/v2" + + "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" +) + +type Coordinator struct { + coordinatorHealthChecker healthchecker.HealthChecker + hubElector *HubElector + informerStarted bool +} + +func NewCoordinator(coordinatorHealthChecker healthchecker.HealthChecker, elector *HubElector, stopCh <-chan struct{}) *Coordinator { + return &Coordinator{ + coordinatorHealthChecker: coordinatorHealthChecker, + hubElector: elector, + } +} + +func (coordinator *Coordinator) Run(stopCh <-chan struct{}) { + for { + select { + case <-stopCh: + klog.Infof("exit normally in coordinator loop.") + if coordinator.informerStarted { + // stop shared informer + // + coordinator.informerStarted = false + } + return + case electorStatus, ok := <-coordinator.hubElector.StatusChan(): + if !ok { + return + } + + if electorStatus != PendingHub && !coordinator.cacheIsUploaded() { + // upload local cache, and make sure yurthub pod is the last resource uploaded + } + + if electorStatus == LeaderHub { + if !coordinator.informerStarted { + coordinator.informerStarted = true + // start shared informer for pool-scope data + // make sure + + // start shared informer for lease delegating + // + } + break + } + + if electorStatus == FollowerHub { + if coordinator.informerStarted { + // stop shared informer + // + coordinator.informerStarted = false + } + } + } + } +} + +func (coordinator *Coordinator) cacheIsUploaded() bool { + // check yurthub pod is uploaded + return true +} + +func (coordinator *Coordinator) IsReady() bool { + + return true +} diff --git a/pkg/yurthub/poolcoordinator/leader_election.go b/pkg/yurthub/poolcoordinator/leader_election.go new file mode 100644 index 00000000000..ed9eae6e620 --- /dev/null +++ b/pkg/yurthub/poolcoordinator/leader_election.go @@ -0,0 +1,143 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 poolcoordinator + +import ( + "context" + "time" + + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/tools/leaderelection" + "k8s.io/client-go/tools/leaderelection/resourcelock" + "k8s.io/klog/v2" + + "github.com/openyurtio/openyurt/cmd/yurthub/app/config" + "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" +) + +const ( + InitHub int32 = iota // 0 + LeaderHub + FollowerHub + PendingHub +) + +type HubElector struct { + coordinatorClient kubernetes.Interface + coordinatorHealthChecker healthchecker.HealthChecker + cloudAPIServerHealthChecker healthchecker.HealthChecker + electorStatus chan int32 + le *leaderelection.LeaderElector + inElecting bool +} + +func NewHubElector(cfg *config.YurtHubConfiguration, coordinatorClient kubernetes.Interface, cloudAPIServerHealthyChecker healthchecker.HealthChecker, stopCh <-chan struct{}) (*HubElector, error) { + coordinatorHealthyChecker, err := healthchecker.NewCoordinatorHealthChecker(cfg, coordinatorClient, cloudAPIServerHealthyChecker, stopCh) + if err != nil { + return nil, err + } + + he := &HubElector{ + coordinatorClient: coordinatorClient, + coordinatorHealthChecker: coordinatorHealthyChecker, + cloudAPIServerHealthChecker: cloudAPIServerHealthyChecker, + electorStatus: make(chan int32), + } + + rl, err := resourcelock.New(cfg.LeaderElection.ResourceLock, + cfg.LeaderElection.ResourceNamespace, + cfg.LeaderElection.ResourceName, + coordinatorClient.CoreV1(), + coordinatorClient.CoordinationV1(), + resourcelock.ResourceLockConfig{Identity: cfg.NodeName}) + if err != nil { + return nil, err + } + + le, err := leaderelection.NewLeaderElector(leaderelection.LeaderElectionConfig{ + Lock: rl, + LeaseDuration: cfg.LeaderElection.LeaseDuration.Duration, + RenewDeadline: cfg.LeaderElection.RenewDeadline.Duration, + RetryPeriod: cfg.LeaderElection.RetryPeriod.Duration, + Callbacks: leaderelection.LeaderCallbacks{ + OnStartedLeading: func(ctx context.Context) { + klog.Infof("yurthub of %s became lease", cfg.NodeName) + he.electorStatus <- LeaderHub + }, + OnStoppedLeading: func() { + + }, + }, + }) + if err != nil { + return nil, err + } + he.le = le + he.electorStatus <- PendingHub + + return he, nil +} + +func (he *HubElector) Run(stopCh <-chan struct{}) { + intervalTicker := time.NewTicker(5 * time.Second) + defer intervalTicker.Stop() + defer close(he.electorStatus) + + var ctx context.Context + var cancel context.CancelFunc + for { + select { + case <-stopCh: + klog.Infof("exit normally in leader election loop.") + + if cancel != nil { + cancel() + he.inElecting = false + } + return + case <-intervalTicker.C: + if !he.coordinatorHealthChecker.IsHealthy() { + if he.inElecting && cancel != nil { + cancel() + he.inElecting = false + he.electorStatus <- PendingHub + } + break + } + + if !he.cloudAPIServerHealthChecker.IsHealthy() { + if he.inElecting && cancel != nil { + cancel() + he.inElecting = false + he.electorStatus <- FollowerHub + } + break + } + + if !he.inElecting { + he.electorStatus <- FollowerHub + ctx, cancel = context.WithCancel(context.TODO()) + go he.le.Run(ctx) + he.inElecting = true + } + } + } +} + +func (he *HubElector) StatusChan() chan int32 { + return he.electorStatus +} From 3e694686da0a1e724f7e2fdd0bdbbca15e8e0aee Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Wed, 7 Dec 2022 19:36:33 +0800 Subject: [PATCH 02/20] pool-coordinator implementation of yurthub (#1073) * pool-coordinator implementation of yurthub Signed-off-by: Congrool --- cmd/yurthub/app/config/config.go | 22 +- cmd/yurthub/app/options/options.go | 83 +-- cmd/yurthub/app/start.go | 87 ++- cmd/yurthub/yurthub.go | 2 +- go.mod | 5 +- pkg/yurthub/cachemanager/cache_manager.go | 27 +- .../cachemanager/cache_manager_test.go | 56 ++ pkg/yurthub/cachemanager/storage_wrapper.go | 5 + pkg/yurthub/healthchecker/health_checker.go | 6 +- .../healthchecker/health_checker_test.go | 2 +- .../poolcoordinator/constants/constants.go | 39 ++ pkg/yurthub/poolcoordinator/coordinator.go | 611 ++++++++++++++++-- pkg/yurthub/poolcoordinator/informer_lease.go | 178 +++++ .../poolcoordinator/leader_election.go | 19 +- pkg/yurthub/proxy/local/local.go | 24 +- pkg/yurthub/proxy/local/local_test.go | 14 +- pkg/yurthub/proxy/pool/pool.go | 256 ++++++++ pkg/yurthub/proxy/proxy.go | 168 ++++- pkg/yurthub/proxy/remote/loadbalancer.go | 276 +++++++- pkg/yurthub/proxy/remote/loadbalancer_test.go | 93 ++- pkg/yurthub/proxy/{remote => util}/remote.go | 126 +--- pkg/yurthub/proxy/util/util.go | 68 ++ pkg/yurthub/storage/disk/storage.go | 2 +- pkg/yurthub/storage/etcd/etcd_suite_test.go | 58 ++ pkg/yurthub/storage/etcd/key.go | 78 +++ pkg/yurthub/storage/etcd/key_test.go | 103 +++ pkg/yurthub/storage/etcd/keycache.go | 280 ++++++++ pkg/yurthub/storage/etcd/keycache_test.go | 311 +++++++++ pkg/yurthub/storage/etcd/storage.go | 500 ++++++++++++++ pkg/yurthub/storage/etcd/storage_test.go | 548 ++++++++++++++++ pkg/yurthub/storage/store.go | 5 +- pkg/yurthub/storage/utils/validate.go | 6 +- pkg/yurthub/util/util.go | 93 +++ 33 files changed, 3813 insertions(+), 338 deletions(-) create mode 100644 pkg/yurthub/poolcoordinator/constants/constants.go create mode 100644 pkg/yurthub/poolcoordinator/informer_lease.go create mode 100644 pkg/yurthub/proxy/pool/pool.go rename pkg/yurthub/proxy/{remote => util}/remote.go (50%) create mode 100644 pkg/yurthub/storage/etcd/etcd_suite_test.go create mode 100644 pkg/yurthub/storage/etcd/key.go create mode 100644 pkg/yurthub/storage/etcd/key_test.go create mode 100644 pkg/yurthub/storage/etcd/keycache.go create mode 100644 pkg/yurthub/storage/etcd/keycache_test.go create mode 100644 pkg/yurthub/storage/etcd/storage.go create mode 100644 pkg/yurthub/storage/etcd/storage_test.go diff --git a/cmd/yurthub/app/config/config.go b/cmd/yurthub/app/config/config.go index 984cb3c224c..f4b20c50120 100644 --- a/cmd/yurthub/app/config/config.go +++ b/cmd/yurthub/app/config/config.go @@ -23,8 +23,6 @@ import ( "strings" "time" - componentbaseconfig "k8s.io/component-base/config" - corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/fields" @@ -40,6 +38,7 @@ import ( core "k8s.io/client-go/testing" "k8s.io/client-go/tools/cache" "k8s.io/client-go/tools/clientcmd" + componentbaseconfig "k8s.io/component-base/config" "k8s.io/klog/v2" "github.com/openyurtio/openyurt/cmd/yurthub/app/options" @@ -91,6 +90,17 @@ type YurtHubConfiguration struct { YurtHubProxyServerServing *apiserver.DeprecatedInsecureServingInfo YurtHubDummyProxyServerServing *apiserver.DeprecatedInsecureServingInfo YurtHubSecureProxyServerServing *apiserver.SecureServingInfo + YurtHubProxyServerAddr string + DiskCachePath string + CoordinatorPKIDir string + EnableCoordinator bool + CoordinatorServerURL *url.URL + CoordinatorStoragePrefix string + CoordinatorStorageAddr string // ip:port + CoordinatorStorageCaFile string + CoordinatorStorageCertFile string + CoordinatorStorageKeyFile string + CoordinatorClient kubernetes.Interface LeaderElection componentbaseconfig.LeaderElectionConfiguration } @@ -148,6 +158,14 @@ func Complete(options *options.YurtHubOptions) (*YurtHubConfiguration, error) { FilterManager: filterManager, MinRequestTimeout: options.MinRequestTimeout, TenantNs: tenantNs, + YurtHubProxyServerAddr: fmt.Sprintf("http://%s:%d", options.YurtHubProxyHost, options.YurtHubProxyPort), + DiskCachePath: options.DiskCachePath, + CoordinatorPKIDir: filepath.Join(options.RootDir, "poolcoordinator"), + EnableCoordinator: options.EnableCoordinator, + CoordinatorServerURL: coordinatorServerURL, + CoordinatorStoragePrefix: options.CoordinatorStoragePrefix, + CoordinatorStorageAddr: options.CoordinatorStorageAddr, + LeaderElection: options.LeaderElection, } certMgr, err := createCertManager(options, us) diff --git a/cmd/yurthub/app/options/options.go b/cmd/yurthub/app/options/options.go index 508a7245a98..e7d4fa16ade 100644 --- a/cmd/yurthub/app/options/options.go +++ b/cmd/yurthub/app/options/options.go @@ -44,42 +44,47 @@ const ( // YurtHubOptions is the main settings for the yurthub type YurtHubOptions struct { - ServerAddr string - YurtHubHost string // YurtHub server host (e.g.: expose metrics API) - YurtHubProxyHost string // YurtHub proxy server host - YurtHubPort int - YurtHubProxyPort int - YurtHubProxySecurePort int - GCFrequency int - YurtHubCertOrganizations []string - NodeName string - NodePoolName string - LBMode string - HeartbeatFailedRetry int - HeartbeatHealthyThreshold int - HeartbeatTimeoutSeconds int - HeartbeatIntervalSeconds int - MaxRequestInFlight int - JoinToken string - RootDir string - Version bool - EnableProfiling bool - EnableDummyIf bool - EnableIptables bool - HubAgentDummyIfIP string - HubAgentDummyIfName string - DiskCachePath string - AccessServerThroughHub bool - EnableResourceFilter bool - DisabledResourceFilters []string - WorkingMode string - KubeletHealthGracePeriod time.Duration - EnableNodePool bool - MinRequestTimeout time.Duration - CACertHashes []string - UnsafeSkipCAVerification bool - ClientForTest kubernetes.Interface - LeaderElection componentbaseconfig.LeaderElectionConfiguration + ServerAddr string + YurtHubHost string // YurtHub server host (e.g.: expose metrics API) + YurtHubProxyHost string // YurtHub proxy server host + YurtHubPort int + YurtHubProxyPort int + YurtHubProxySecurePort int + GCFrequency int + YurtHubCertOrganizations []string + NodeName string + NodePoolName string + LBMode string + HeartbeatFailedRetry int + HeartbeatHealthyThreshold int + HeartbeatTimeoutSeconds int + HeartbeatIntervalSeconds int + MaxRequestInFlight int + JoinToken string + RootDir string + Version bool + EnableProfiling bool + EnableDummyIf bool + EnableIptables bool + HubAgentDummyIfIP string + HubAgentDummyIfName string + DiskCachePath string + AccessServerThroughHub bool + EnableResourceFilter bool + DisabledResourceFilters []string + WorkingMode string + KubeletHealthGracePeriod time.Duration + EnableNodePool bool + MinRequestTimeout time.Duration + CACertHashes []string + UnsafeSkipCAVerification bool + ClientForTest kubernetes.Interface + CoordinatorStoragePrefix string + CoordinatorStorageAddr string + CoordinatorStorageCaFile string + CoordinatorStorageCertFile string + CoordinatorStorageKeyFile string + LeaderElection componentbaseconfig.LeaderElectionConfiguration } // NewYurtHubOptions creates a new YurtHubOptions with a default config. @@ -113,6 +118,7 @@ func NewYurtHubOptions() *YurtHubOptions { MinRequestTimeout: time.Second * 1800, CACertHashes: make([]string, 0), UnsafeSkipCAVerification: true, + CoordinatorStoragePrefix: "/registry", LeaderElection: componentbaseconfig.LeaderElectionConfiguration{ LeaderElect: true, LeaseDuration: metav1.Duration{Duration: 15 * time.Second}, @@ -195,6 +201,11 @@ func (o *YurtHubOptions) AddFlags(fs *pflag.FlagSet) { fs.DurationVar(&o.MinRequestTimeout, "min-request-timeout", o.MinRequestTimeout, "An optional field indicating at least how long a proxy handler must keep a request open before timing it out. Currently only honored by the local watch request handler(use request parameter timeoutSeconds firstly), which picks a randomized value above this number as the connection timeout, to spread out load.") fs.StringSliceVar(&o.CACertHashes, "discovery-token-ca-cert-hash", o.CACertHashes, "For token-based discovery, validate that the root CA public key matches this hash (format: \":\").") fs.BoolVar(&o.UnsafeSkipCAVerification, "discovery-token-unsafe-skip-ca-verification", o.UnsafeSkipCAVerification, "For token-based discovery, allow joining without --discovery-token-ca-cert-hash pinning.") + fs.StringVar(&o.CoordinatorStoragePrefix, "coordinator-storage-prefix", o.CoordinatorStoragePrefix, "Pool-Coordinator etcd storage prefix, same as etcd-prefix of Kube-APIServer") + fs.StringVar(&o.CoordinatorStorageAddr, "coordinator-storage-addr", o.CoordinatorStorageAddr, "Address of Pool-Coordinator etcd, in the format ip:port") + fs.StringVar(&o.CoordinatorStorageCaFile, "coordinator-storage-ca", o.CoordinatorStorageCaFile, "CA file path to communicate with Pool-Coordinator etcd") + fs.StringVar(&o.CoordinatorStorageCertFile, "coordinator-storage-cert", o.CoordinatorStorageCertFile, "Cert file path to communicate with Pool-Coordinator etcd") + fs.StringVar(&o.CoordinatorStorageKeyFile, "coordinator-storage-key", o.CoordinatorStorageKeyFile, "Key file path to communicate with Pool-Coordinator etcd") bindFlags(&o.LeaderElection, fs) } diff --git a/cmd/yurthub/app/start.go b/cmd/yurthub/app/start.go index 548999b1082..00b97b1baff 100644 --- a/cmd/yurthub/app/start.go +++ b/cmd/yurthub/app/start.go @@ -17,12 +17,14 @@ limitations under the License. package app import ( + "context" "fmt" "net/url" "time" "github.com/spf13/cobra" "github.com/spf13/pflag" + "k8s.io/apimachinery/pkg/util/wait" "k8s.io/client-go/kubernetes" "k8s.io/client-go/rest" "k8s.io/klog/v2" @@ -34,6 +36,7 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/gc" "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" hubrest "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/rest" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator" "github.com/openyurtio/openyurt/pkg/yurthub/proxy" "github.com/openyurtio/openyurt/pkg/yurthub/server" "github.com/openyurtio/openyurt/pkg/yurthub/tenant" @@ -42,7 +45,7 @@ import ( ) // NewCmdStartYurtHub creates a *cobra.Command object with default parameters -func NewCmdStartYurtHub(stopCh <-chan struct{}) *cobra.Command { +func NewCmdStartYurtHub(ctx context.Context) *cobra.Command { yurtHubOptions := options.NewYurtHubOptions() cmd := &cobra.Command{ @@ -69,7 +72,7 @@ func NewCmdStartYurtHub(stopCh <-chan struct{}) *cobra.Command { } klog.Infof("%s cfg: %#+v", projectinfo.GetHubName(), yurtHubCfg) - if err := Run(yurtHubCfg, stopCh); err != nil { + if err := Run(ctx, yurtHubCfg); err != nil { klog.Fatalf("run %s failed, %v", projectinfo.GetHubName(), err) } }, @@ -80,40 +83,47 @@ func NewCmdStartYurtHub(stopCh <-chan struct{}) *cobra.Command { } // Run runs the YurtHubConfiguration. This should never exit -func Run(cfg *config.YurtHubConfiguration, stopCh <-chan struct{}) error { +func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { defer cfg.CertManager.Stop() trace := 1 klog.Infof("%d. new transport manager", trace) - transportManager, err := transport.NewTransportManager(cfg.CertManager, stopCh) + transportManager, err := transport.NewTransportManager(cfg.CertManager, ctx.Done()) if err != nil { return fmt.Errorf("could not new transport manager, %w", err) } trace++ klog.Infof("%d. prepare for health checker clients", trace) - healthCheckerClientsForCloud, _, err := createHealthCheckerClient(cfg.HeartbeatTimeoutSeconds, cfg.RemoteServers, cfg.CoordinatorServer, transportManager) + cloudClients, coordinatorClient, err := createClients(cfg.HeartbeatTimeoutSeconds, cfg.RemoteServers, cfg.CoordinatorServer, transportManager) if err != nil { return fmt.Errorf("failed to create health checker clients, %w", err) } trace++ - var healthChecker healthchecker.MultipleBackendsHealthChecker + var cloudHealthChecker healthchecker.MultipleBackendsHealthChecker + var coordinatorHealthChecker healthchecker.HealthChecker if cfg.WorkingMode == util.WorkingModeEdge { - klog.Infof("%d. create health checker for remote servers ", trace) - healthChecker, err = healthchecker.NewCloudAPIServerHealthChecker(cfg, healthCheckerClientsForCloud, stopCh) + klog.Infof("%d. create health checkers for remote servers and pool coordinator", trace) + cloudHealthChecker, err = healthchecker.NewCloudAPIServerHealthChecker(cfg, cloudClients, ctx.Done()) if err != nil { - return fmt.Errorf("could not new health checker, %w", err) + return fmt.Errorf("could not new cloud health checker, %w", err) } + coordinatorHealthChecker, err = healthchecker.NewCoordinatorHealthChecker(cfg, coordinatorClient, cloudHealthChecker, ctx.Done()) + if err != nil { + return fmt.Errorf("failed to create coordinator health checker, %v", err) + } + } else { klog.Infof("%d. disable health checker for node %s because it is a cloud node", trace, cfg.NodeName) - // In cloud mode, health checker is not needed. - // This fake checker will always report that the remote server is healthy. - healthChecker = healthchecker.NewFakeChecker(true, make(map[string]int)) + // In cloud mode, cloud health checker and pool coordinator health checker are not needed. + // This fake checker will always report that the cloud is healthy and pool coordinator is unhealthy. + cloudHealthChecker = healthchecker.NewFakeChecker(true, make(map[string]int)) + coordinatorHealthChecker = healthchecker.NewFakeChecker(false, make(map[string]int)) } trace++ klog.Infof("%d. new restConfig manager", trace) - restConfigMgr, err := hubrest.NewRestConfigManager(cfg.CertManager, healthChecker) + restConfigMgr, err := hubrest.NewRestConfigManager(cfg.CertManager, cloudHealthChecker) if err != nil { return fmt.Errorf("could not new restConfig manager, %w", err) } @@ -130,7 +140,7 @@ func Run(cfg *config.YurtHubConfiguration, stopCh <-chan struct{}) error { if cfg.WorkingMode == util.WorkingModeEdge { klog.Infof("%d. new gc manager for node %s, and gc frequency is a random time between %d min and %d min", trace, cfg.NodeName, cfg.GCFrequency, 3*cfg.GCFrequency) - gcMgr, err := gc.NewGCManager(cfg, restConfigMgr, stopCh) + gcMgr, err := gc.NewGCManager(cfg, restConfigMgr, ctx.Done()) if err != nil { return fmt.Errorf("could not new gc manager, %w", err) } @@ -141,36 +151,55 @@ func Run(cfg *config.YurtHubConfiguration, stopCh <-chan struct{}) error { trace++ klog.Infof("%d. new tenant sa manager", trace) - tenantMgr := tenant.New(cfg.TenantNs, cfg.SharedFactory, stopCh) + tenantMgr := tenant.New(cfg.TenantNs, cfg.SharedFactory, ctx.Done()) + trace++ + + klog.Infof("%d. create yurthub elector", trace) + elector, err := poolcoordinator.NewHubElector(cfg, coordinatorClient, coordinatorHealthChecker, cloudHealthChecker, ctx.Done()) + if err != nil { + klog.Errorf("failed to create hub elector, %v", err) + } + elector.Run(ctx.Done()) + trace++ + + // TODO: cloud client load balance + klog.Infof("%d. create coordinator", trace) + coordinator, err := poolcoordinator.NewCoordinator(ctx, cfg, restConfigMgr, transportManager, elector) + if err != nil { + klog.Errorf("failed to create coordinator, %v", err) + } + coordinator.Run() trace++ klog.Infof("%d. new reverse proxy handler for remote servers", trace) - yurtProxyHandler, err := proxy.NewYurtReverseProxyHandler(cfg, cacheMgr, transportManager, healthChecker, tenantMgr, stopCh) + yurtProxyHandler, err := proxy.NewYurtReverseProxyHandler(cfg, cacheMgr, transportManager, coordinator, cloudHealthChecker, coordinatorHealthChecker, tenantMgr, ctx.Done()) if err != nil { return fmt.Errorf("could not create reverse proxy handler, %w", err) } trace++ if cfg.NetworkMgr != nil { - cfg.NetworkMgr.Run(stopCh) + cfg.NetworkMgr.Run(ctx.Done()) } // start shared informers before start hub server - cfg.SharedFactory.Start(stopCh) - cfg.YurtSharedFactory.Start(stopCh) + cfg.SharedFactory.Start(ctx.Done()) + cfg.YurtSharedFactory.Start(ctx.Done()) klog.Infof("%d. new %s server and begin to serve", trace, projectinfo.GetHubName()) - if err := server.RunYurtHubServers(cfg, yurtProxyHandler, restConfigMgr, stopCh); err != nil { + if err := server.RunYurtHubServers(cfg, yurtProxyHandler, restConfigMgr, ctx.Done()); err != nil { return fmt.Errorf("could not run hub servers, %w", err) } - <-stopCh + <-ctx.Done() klog.Infof("hub agent exited") return nil } -func createHealthCheckerClient(heartbeatTimeoutSeconds int, remoteServers []*url.URL, coordinatorServer *url.URL, tp transport.Interface) (map[string]kubernetes.Interface, kubernetes.Interface, error) { - var healthCheckerClientForCoordinator kubernetes.Interface - healthCheckerClientsForCloud := make(map[string]kubernetes.Interface) +// createClients will create clients for all cloud APIServer and client for pool coordinator +// It will return a map, mapping cloud APIServer URL to its client, and a pool coordinator client +func createClients(heartbeatTimeoutSeconds int, remoteServers []*url.URL, coordinatorServer *url.URL, tp transport.Interface) (map[string]kubernetes.Interface, kubernetes.Interface, error) { + var coordinatorClient kubernetes.Interface + cloudClients := make(map[string]kubernetes.Interface) for i := range remoteServers { restConf := &rest.Config{ Host: remoteServers[i].String(), @@ -179,9 +208,9 @@ func createHealthCheckerClient(heartbeatTimeoutSeconds int, remoteServers []*url } c, err := kubernetes.NewForConfig(restConf) if err != nil { - return healthCheckerClientsForCloud, healthCheckerClientForCoordinator, err + return cloudClients, coordinatorClient, err } - healthCheckerClientsForCloud[remoteServers[i].String()] = c + cloudClients[remoteServers[i].String()] = c } cfg := &rest.Config{ @@ -191,9 +220,9 @@ func createHealthCheckerClient(heartbeatTimeoutSeconds int, remoteServers []*url } c, err := kubernetes.NewForConfig(cfg) if err != nil { - return healthCheckerClientsForCloud, healthCheckerClientForCoordinator, err + return cloudClients, coordinatorClient, err } - healthCheckerClientForCoordinator = c + coordinatorClient = c - return healthCheckerClientsForCloud, healthCheckerClientForCoordinator, nil + return cloudClients, coordinatorClient, nil } diff --git a/cmd/yurthub/yurthub.go b/cmd/yurthub/yurthub.go index fea2aa43148..439bb09339c 100644 --- a/cmd/yurthub/yurthub.go +++ b/cmd/yurthub/yurthub.go @@ -28,7 +28,7 @@ import ( func main() { rand.Seed(time.Now().UnixNano()) - cmd := app.NewCmdStartYurtHub(server.SetupSignalHandler()) + cmd := app.NewCmdStartYurtHub(server.SetupSignalContext()) cmd.Flags().AddGoFlagSet(flag.CommandLine) if err := cmd.Execute(); err != nil { panic(err) diff --git a/go.mod b/go.mod index b59d9a1d095..46ea2c50d24 100644 --- a/go.mod +++ b/go.mod @@ -26,7 +26,10 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/vishvananda/netlink v1.1.1-0.20200603190939-5a869a71f0cb - golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a + github.com/wI2L/jsondiff v0.3.0 + go.etcd.io/etcd/client/pkg/v3 v3.5.0 + go.etcd.io/etcd/client/v3 v3.5.0 + golang.org/x/sys v0.0.0-20220319134239-a9b59b0215f8 google.golang.org/grpc v1.40.0 gopkg.in/cheggaaa/pb.v1 v1.0.25 gopkg.in/square/go-jose.v2 v2.2.2 diff --git a/pkg/yurthub/cachemanager/cache_manager.go b/pkg/yurthub/cachemanager/cache_manager.go index 449b4da5603..021b76fa6bb 100644 --- a/pkg/yurthub/cachemanager/cache_manager.go +++ b/pkg/yurthub/cachemanager/cache_manager.go @@ -31,6 +31,8 @@ import ( v1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/meta" + metainternalversion "k8s.io/apimachinery/pkg/apis/meta/internalversion" + metainternalversionscheme "k8s.io/apimachinery/pkg/apis/meta/internalversion/scheme" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" "k8s.io/apimachinery/pkg/runtime" @@ -44,7 +46,6 @@ import ( hubmeta "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/meta" "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/serializer" - proxyutil "github.com/openyurtio/openyurt/pkg/yurthub/proxy/util" "github.com/openyurtio/openyurt/pkg/yurthub/storage" "github.com/openyurtio/openyurt/pkg/yurthub/util" ) @@ -68,7 +69,6 @@ type cacheManager struct { restMapperManager *hubmeta.RESTMapperManager cacheAgents *CacheAgent listSelectorCollector map[storage.Key]string - sharedFactory informers.SharedInformerFactory inMemoryCache map[string]runtime.Object } @@ -86,7 +86,6 @@ func NewCacheManager( cacheAgents: cacheAgents, restMapperManager: restMapperMgr, listSelectorCollector: make(map[storage.Key]string), - sharedFactory: sharedFactory, inMemoryCache: make(map[string]runtime.Object), } @@ -181,7 +180,7 @@ func (cm *cacheManager) queryListObject(req *http.Request) (runtime.Object, erro } objs, err := cm.storage.List(key) - if err == storage.ErrStorageNotFound && proxyutil.IsListRequestWithNameFieldSelector(req) { + if err == storage.ErrStorageNotFound && isListRequestWithNameFieldSelector(req) { // When the request is a list request with FieldSelector "metadata.name", we should not return error // when the specified resource is not found return an empty list object, to keep same as APIServer. return listObj, nil @@ -858,3 +857,23 @@ func inMemoryCacheKeyFunc(reqInfo *apirequest.RequestInfo) (string, error) { key := filepath.Join(res, ns, name) return key, nil } + +// isListRequestWithNameFieldSelector will check if the request has FieldSelector "metadata.name". +// If found, return true, otherwise false. +func isListRequestWithNameFieldSelector(req *http.Request) bool { + ctx := req.Context() + if info, ok := apirequest.RequestInfoFrom(ctx); ok { + if info.IsResourceRequest && info.Verb == "list" { + opts := metainternalversion.ListOptions{} + if err := metainternalversionscheme.ParameterCodec.DecodeParameters(req.URL.Query(), metav1.SchemeGroupVersion, &opts); err == nil { + if opts.FieldSelector == nil { + return false + } + if _, found := opts.FieldSelector.RequiresExactMatch("metadata.name"); found { + return true + } + } + } + } + return false +} diff --git a/pkg/yurthub/cachemanager/cache_manager_test.go b/pkg/yurthub/cachemanager/cache_manager_test.go index a0c3adfd8cb..e4f8a251aac 100644 --- a/pkg/yurthub/cachemanager/cache_manager_test.go +++ b/pkg/yurthub/cachemanager/cache_manager_test.go @@ -3275,4 +3275,60 @@ func newTestRequestInfoResolver() *request.RequestInfoFactory { } } +func TestIsListRequestWithNameFieldSelector(t *testing.T) { + testcases := map[string]struct { + Verb string + Path string + Expect bool + }{ + "request has metadata.name fieldSelector": { + Verb: "GET", + Path: "/api/v1/namespaces/kube-system/pods?resourceVersion=1494416105&fieldSelector=metadata.name=test", + Expect: true, + }, + "request has no metadata.name fieldSelector": { + Verb: "GET", + Path: "/api/v1/namespaces/kube-system/pods?resourceVersion=1494416105&fieldSelector=spec.nodeName=test", + Expect: false, + }, + "request only has labelSelector": { + Verb: "GET", + Path: "/api/v1/namespaces/kube-system/pods?resourceVersion=1494416105&labelSelector=foo=bar", + Expect: false, + }, + "request has both labelSelector and fieldSelector and fieldSelector has metadata.name": { + Verb: "GET", + Path: "/api/v1/namespaces/kube-system/pods?fieldSelector=metadata.name=test&labelSelector=foo=bar", + Expect: true, + }, + "request has both labelSelector and fieldSelector but fieldSelector has no metadata.name": { + Verb: "GET", + Path: "/api/v1/namespaces/kube-system/pods?fieldSelector=spec.nodeName=test&labelSelector=foo=bar", + Expect: false, + }, + } + + resolver := newTestRequestInfoResolver() + + for k, tc := range testcases { + t.Run(k, func(t *testing.T) { + req, _ := http.NewRequest(tc.Verb, tc.Path, nil) + req.RemoteAddr = "127.0.0.1" + + var isMetadataNameFieldSelector bool + var handler http.Handler = http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + isMetadataNameFieldSelector = isListRequestWithNameFieldSelector(req) + }) + + handler = proxyutil.WithListRequestSelector(handler) + handler = filters.WithRequestInfo(handler, resolver) + handler.ServeHTTP(httptest.NewRecorder(), req) + + if isMetadataNameFieldSelector != tc.Expect { + t.Errorf("failed at case %s, want: %v, got: %v", k, tc.Expect, isMetadataNameFieldSelector) + } + }) + } +} + // TODO: in-memory cache unit tests diff --git a/pkg/yurthub/cachemanager/storage_wrapper.go b/pkg/yurthub/cachemanager/storage_wrapper.go index d992ccde61d..f9c8db77e7e 100644 --- a/pkg/yurthub/cachemanager/storage_wrapper.go +++ b/pkg/yurthub/cachemanager/storage_wrapper.go @@ -46,6 +46,7 @@ type StorageWrapper interface { DeleteComponentResources(component string) error SaveClusterInfo(key storage.ClusterInfoKey, content []byte) error GetClusterInfo(key storage.ClusterInfoKey) ([]byte, error) + GetStorage() storage.Store } type storageWrapper struct { @@ -70,6 +71,10 @@ func (sw *storageWrapper) KeyFunc(info storage.KeyBuildInfo) (storage.Key, error return sw.store.KeyFunc(info) } +func (sw *storageWrapper) GetStorage() storage.Store { + return sw.store +} + // Create store runtime object into backend storage // if obj is nil, the storage used to represent the key // will be created. for example: for disk storage, diff --git a/pkg/yurthub/healthchecker/health_checker.go b/pkg/yurthub/healthchecker/health_checker.go index eb78ac7e8f9..855720b489c 100644 --- a/pkg/yurthub/healthchecker/health_checker.go +++ b/pkg/yurthub/healthchecker/health_checker.go @@ -34,7 +34,7 @@ import ( ) const ( - delegateHeartBeat = "openyurt.io/delegate-heartbeat" + DelegateHeartBeat = "openyurt.io/delegate-heartbeat" ) type setNodeLease func(*coordinationv1.Lease) error @@ -114,9 +114,9 @@ func (chc *coordinatorHealthChecker) getLastNodeLease() *coordinationv1.Lease { if chc.latestLease.Annotations == nil { chc.latestLease.Annotations = make(map[string]string) } - chc.latestLease.Annotations[delegateHeartBeat] = "true" + chc.latestLease.Annotations[DelegateHeartBeat] = "true" } else { - delete(chc.latestLease.Annotations, delegateHeartBeat) + delete(chc.latestLease.Annotations, DelegateHeartBeat) } } diff --git a/pkg/yurthub/healthchecker/health_checker_test.go b/pkg/yurthub/healthchecker/health_checker_test.go index 85659c32b89..ba82ffbaf06 100644 --- a/pkg/yurthub/healthchecker/health_checker_test.go +++ b/pkg/yurthub/healthchecker/health_checker_test.go @@ -210,7 +210,7 @@ func TestNewCoordinatorHealthChecker(t *testing.T) { if tt.cloudAPIServerUnhealthy { if delegateLease == nil || len(delegateLease.Annotations) == 0 { t.Errorf("expect delegate heartbeat annotaion, but got nil") - } else if v, ok := delegateLease.Annotations[delegateHeartBeat]; !ok || v != "true" { + } else if v, ok := delegateLease.Annotations[DelegateHeartBeat]; !ok || v != "true" { t.Errorf("expect delegate heartbeat annotaion and v is true, but got empty or %v", v) } } diff --git a/pkg/yurthub/poolcoordinator/constants/constants.go b/pkg/yurthub/poolcoordinator/constants/constants.go new file mode 100644 index 00000000000..bcd06c86e76 --- /dev/null +++ b/pkg/yurthub/poolcoordinator/constants/constants.go @@ -0,0 +1,39 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 constants + +import ( + "k8s.io/apimachinery/pkg/runtime/schema" + + "github.com/openyurtio/openyurt/pkg/yurthub/storage" +) + +var ( + PoolScopedResources = map[schema.GroupVersionResource]struct{}{ + {Group: "", Version: "v1", Resource: "endpoints"}: {}, + {Group: "discovery.k8s.io", Version: "v1", Resource: "endpointslices"}: {}, + } + + UploadResourcesKeyBuildInfo = map[storage.KeyBuildInfo]struct{}{ + {Component: "kubelet", Resources: "pods", Group: "", Version: "v1"}: {}, + {Component: "kubelet", Resources: "nodes", Group: "", Version: "v1"}: {}, + } +) + +const ( + DefaultPoolScopedUserAgent = "leader-yurthub" +) diff --git a/pkg/yurthub/poolcoordinator/coordinator.go b/pkg/yurthub/poolcoordinator/coordinator.go index 6afdc8ffd84..ab3e8763fcb 100644 --- a/pkg/yurthub/poolcoordinator/coordinator.go +++ b/pkg/yurthub/poolcoordinator/coordinator.go @@ -17,73 +17,610 @@ limitations under the License. package poolcoordinator import ( + "context" + "encoding/json" + "fmt" + "strconv" + "sync" + "time" + + coordinationv1 "k8s.io/api/coordination/v1" + corev1 "k8s.io/api/core/v1" + apierrors "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/client-go/informers" + "k8s.io/client-go/kubernetes" + coordclientset "k8s.io/client-go/kubernetes/typed/coordination/v1" + "k8s.io/client-go/rest" + "k8s.io/client-go/tools/cache" + "k8s.io/client-go/tools/clientcmd" "k8s.io/klog/v2" + "github.com/openyurtio/openyurt/cmd/yurthub/app/config" + "github.com/openyurtio/openyurt/pkg/yurthub/cachemanager" "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" + "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/meta" + yurtrest "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/rest" + "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/serializer" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" + "github.com/openyurtio/openyurt/pkg/yurthub/storage" + "github.com/openyurtio/openyurt/pkg/yurthub/storage/etcd" + "github.com/openyurtio/openyurt/pkg/yurthub/transport" +) + +const ( + leaseDelegateRetryTimes = 5 + defaultInformerLeaseRenewDuration = 10 * time.Second + defaultPoolCacheStaleDuration = 30 * time.Second + namespaceInformerLease = "kube-system" + nameInformerLease = "leader-informer-sync" ) type Coordinator struct { - coordinatorHealthChecker healthchecker.HealthChecker - hubElector *HubElector - informerStarted bool + sync.Mutex + ctx context.Context + cancelEtcdStorage func() + informerFactory informers.SharedInformerFactory + restMapperMgr *meta.RESTMapperManager + serializerMgr *serializer.SerializerManager + restConfigMgr *yurtrest.RestConfigManager + etcdStorageCfg *etcd.EtcdStorageConfig + poolCacheManager cachemanager.CacheManager + diskStorage storage.Store + etcdStorage storage.Store + hubElector *HubElector + electStatus int32 + isPoolCacheSynced bool + needUploadLocalCache bool + // poolScopeCacheSyncManager is used to sync pool-scoped resources from cloud to poolcoordinator. + poolScopeCacheSyncManager *poolScopedCacheSyncManager + // informerSyncLeaseManager is used to detect the leader-informer-sync lease + // to check its RenewTime. If its renewTime is not updated after defaultInformerLeaseRenewDuration + // we can think that the poolcoordinator cache is stale and the poolcoordinator is not ready. + // It will start if yurthub becomes leader or follower. + informerSyncLeaseManager *coordinatorLeaseInformerManager + // delegateNodeLeaseManager is used to list/watch kube-node-lease from poolcoordinator. If the + // node lease contains DelegateHeartBeat label, it will triger the eventhandler which will + // use cloud client to send it to cloud APIServer. + delegateNodeLeaseManager *coordinatorLeaseInformerManager } -func NewCoordinator(coordinatorHealthChecker healthchecker.HealthChecker, elector *HubElector, stopCh <-chan struct{}) *Coordinator { - return &Coordinator{ - coordinatorHealthChecker: coordinatorHealthChecker, - hubElector: elector, +func NewCoordinator( + ctx context.Context, + cfg *config.YurtHubConfiguration, + restMgr *yurtrest.RestConfigManager, + transportMgr transport.Interface, + elector *HubElector) (*Coordinator, error) { + etcdStorageCfg := &etcd.EtcdStorageConfig{ + Prefix: cfg.CoordinatorStoragePrefix, + EtcdEndpoints: []string{cfg.CoordinatorStorageAddr}, + CaFile: cfg.CoordinatorStorageCaFile, + CertFile: cfg.CoordinatorStorageCertFile, + KeyFile: cfg.CoordinatorStorageKeyFile, + LocalCacheDir: cfg.DiskCachePath, + } + + coordinatorRESTCfg := &rest.Config{ + Host: cfg.CoordinatorServer.String(), + Transport: transportMgr.CurrentTransport(), + Timeout: defaultInformerLeaseRenewDuration, } + coordinatorClient, err := kubernetes.NewForConfig(coordinatorRESTCfg) + if err != nil { + return nil, fmt.Errorf("failed to create client for pool coordinator, %v", err) + } + + coordinator := &Coordinator{ + ctx: ctx, + etcdStorageCfg: etcdStorageCfg, + restConfigMgr: restMgr, + informerFactory: cfg.SharedFactory, + diskStorage: cfg.StorageWrapper.GetStorage(), + serializerMgr: cfg.SerializerManager, + restMapperMgr: cfg.RESTMapperManager, + hubElector: elector, + } + + informerSyncLeaseManager := &coordinatorLeaseInformerManager{ + ctx: ctx, + coordinatorClient: coordinatorClient, + } + + delegateNodeLeaseManager := &coordinatorLeaseInformerManager{ + ctx: ctx, + coordinatorClient: coordinatorClient, + } + + proxiedClient, err := buildProxiedClientWithUserAgent(fmt.Sprintf("http://%s", cfg.YurtHubProxyServerAddr), constants.DefaultPoolScopedUserAgent) + if err != nil { + return nil, fmt.Errorf("failed to create proxied client, %v", err) + } + poolScopedCacheSyncManager := &poolScopedCacheSyncManager{ + ctx: ctx, + proxiedClient: proxiedClient, + coordinatorClient: cfg.CoordinatorClient, + nodeName: cfg.NodeName, + getEtcdStore: coordinator.getEtcdStore, + } + + coordinator.informerSyncLeaseManager = informerSyncLeaseManager + coordinator.delegateNodeLeaseManager = delegateNodeLeaseManager + coordinator.poolScopeCacheSyncManager = poolScopedCacheSyncManager + + return coordinator, nil } -func (coordinator *Coordinator) Run(stopCh <-chan struct{}) { +func (coordinator *Coordinator) Run() { for { + var poolCacheManager cachemanager.CacheManager + var cancelEtcdStorage func() + var needUploadLocalCache bool + var needCancelEtcdStorage bool + var isPoolCacheSynced bool + var etcdStorage storage.Store + var err error + select { - case <-stopCh: - klog.Infof("exit normally in coordinator loop.") - if coordinator.informerStarted { - // stop shared informer - // - coordinator.informerStarted = false - } + case <-coordinator.ctx.Done(): + coordinator.poolScopeCacheSyncManager.EnsureStop() + coordinator.delegateNodeLeaseManager.EnsureStop() + coordinator.informerSyncLeaseManager.EnsureStop() + klog.Info("exit normally in coordinator loop.") return case electorStatus, ok := <-coordinator.hubElector.StatusChan(): if !ok { return } - if electorStatus != PendingHub && !coordinator.cacheIsUploaded() { - // upload local cache, and make sure yurthub pod is the last resource uploaded - } + switch electorStatus { + case PendingHub: + coordinator.poolScopeCacheSyncManager.EnsureStop() + coordinator.delegateNodeLeaseManager.EnsureStop() + coordinator.informerSyncLeaseManager.EnsureStop() + needUploadLocalCache = true + needCancelEtcdStorage = true + isPoolCacheSynced = false + etcdStorage = nil + poolCacheManager = nil + case LeaderHub: + poolCacheManager, etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() + if err != nil { + klog.Errorf("failed to create pool scoped cache store and manager, %v", err) + continue + } - if electorStatus == LeaderHub { - if !coordinator.informerStarted { - coordinator.informerStarted = true - // start shared informer for pool-scope data - // make sure + cloudLeaseClient, err := coordinator.newCloudLeaseClient() + if err != nil { + klog.Errorf("cloud not get cloud lease client when becoming leader yurthub, %v", err) + continue + } + if err := coordinator.poolScopeCacheSyncManager.EnsureStart(); err != nil { + klog.Errorf("failed to sync pool-scoped resource, %v", err) + continue + } - // start shared informer for lease delegating - // + coordinator.delegateNodeLeaseManager.EnsureStartWithHandler(cache.FilteringResourceEventHandler{ + FilterFunc: ifDelegateHeartBeat, + Handler: cache.ResourceEventHandlerFuncs{ + AddFunc: func(obj interface{}) { + coordinator.delegateNodeLease(cloudLeaseClient, obj) + }, + UpdateFunc: func(_, newObj interface{}) { + coordinator.delegateNodeLease(cloudLeaseClient, newObj) + }, + }, + }) + coordinator.informerSyncLeaseManager.EnsureStartWithHandler(cache.FilteringResourceEventHandler{ + FilterFunc: ifInformerSyncLease, + Handler: cache.ResourceEventHandlerFuncs{ + AddFunc: coordinator.detectPoolCacheSynced, + UpdateFunc: func(_, newObj interface{}) { + coordinator.detectPoolCacheSynced(newObj) + }, + DeleteFunc: func(_ interface{}) { + coordinator.Lock() + defer coordinator.Unlock() + coordinator.isPoolCacheSynced = false + }, + }, + }) + + if coordinator.needUploadLocalCache { + if err := coordinator.uploadLocalCache(etcdStorage); err != nil { + klog.Errorf("failed to upload local cache when yurthub becomes leader, %v", err) + } else { + needUploadLocalCache = false + } } - break - } + case FollowerHub: + poolCacheManager, etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() + if err != nil { + klog.Errorf("failed to create pool scoped cache store and manager, %v", err) + continue + } + + coordinator.poolScopeCacheSyncManager.EnsureStop() + coordinator.delegateNodeLeaseManager.EnsureStop() + coordinator.informerSyncLeaseManager.EnsureStartWithHandler(cache.FilteringResourceEventHandler{ + FilterFunc: ifInformerSyncLease, + Handler: cache.ResourceEventHandlerFuncs{ + AddFunc: coordinator.detectPoolCacheSynced, + UpdateFunc: func(_, newObj interface{}) { + coordinator.detectPoolCacheSynced(newObj) + }, + DeleteFunc: func(_ interface{}) { + coordinator.Lock() + defer coordinator.Unlock() + coordinator.isPoolCacheSynced = false + }, + }, + }) - if electorStatus == FollowerHub { - if coordinator.informerStarted { - // stop shared informer - // - coordinator.informerStarted = false + if coordinator.needUploadLocalCache { + if err := coordinator.uploadLocalCache(etcdStorage); err != nil { + klog.Errorf("failed to upload local cache when yurthub becomes follower, %v", err) + } else { + needUploadLocalCache = false + } } } + + // We should make sure that all fields update should happen + // after acquire lock to avoid race condition. + // Because the caller of IsReady() may be concurrent. + coordinator.Lock() + if needCancelEtcdStorage { + coordinator.cancelEtcdStorage() + } + coordinator.electStatus = electorStatus + coordinator.poolCacheManager = poolCacheManager + coordinator.etcdStorage = etcdStorage + coordinator.cancelEtcdStorage = cancelEtcdStorage + coordinator.needUploadLocalCache = needUploadLocalCache + coordinator.isPoolCacheSynced = isPoolCacheSynced + coordinator.Unlock() + } + } +} + +// IsReady will return the poolCacheManager and true if the pool-coordinator is ready. +// Pool-Coordinator ready means it is ready to handle request. To be specific, it should +// satisfy the following 3 condition: +// 1. Pool-Coordinator is healthy +// 2. Pool-Scoped resources have been synced with cloud, through list/watch +// 3. local cache has been uploaded to pool-coordinator +func (coordinator *Coordinator) IsReady() (cachemanager.CacheManager, bool) { + // If electStatus is not PendingHub, it means pool-coordinator is healthy. + coordinator.Lock() + defer coordinator.Unlock() + if coordinator.electStatus != PendingHub && coordinator.isPoolCacheSynced && !coordinator.needUploadLocalCache { + return coordinator.poolCacheManager, true + } + return nil, false +} + +// IsCoordinatorHealthy will return the poolCacheManager and true if the pool-coordinator is healthy. +// We assume coordinator is healthy when the elect status is LeaderHub and FollowerHub. +func (coordinator *Coordinator) IsHealthy() (cachemanager.CacheManager, bool) { + coordinator.Lock() + defer coordinator.Unlock() + if coordinator.electStatus != PendingHub { + return coordinator.poolCacheManager, true + } + return nil, false +} + +func (coordinator *Coordinator) buildPoolCacheStore() (cachemanager.CacheManager, storage.Store, func(), error) { + ctx, cancel := context.WithCancel(coordinator.ctx) + etcdStore, err := etcd.NewStorage(ctx, coordinator.etcdStorageCfg) + if err != nil { + cancel() + return nil, nil, nil, fmt.Errorf("failed to create etcd storage, %v", err) + } + poolCacheManager := cachemanager.NewCacheManager( + cachemanager.NewStorageWrapper(etcdStore), + coordinator.serializerMgr, + coordinator.restMapperMgr, + coordinator.informerFactory, + ) + return poolCacheManager, etcdStore, cancel, nil +} + +func (coordinator *Coordinator) getEtcdStore() storage.Store { + return coordinator.etcdStorage +} + +func (coordinator *Coordinator) newCloudLeaseClient() (coordclientset.LeaseInterface, error) { + restCfg := coordinator.restConfigMgr.GetRestConfig(true) + if restCfg == nil { + return nil, fmt.Errorf("no cloud server is healthy") + } + cloudClient, err := kubernetes.NewForConfig(restCfg) + if err != nil { + return nil, fmt.Errorf("failed to create cloud client, %v", err) + } + + return cloudClient.CoordinationV1().Leases(corev1.NamespaceNodeLease), nil +} + +func (coordinator *Coordinator) uploadLocalCache(etcdStore storage.Store) error { + uploader := &localCacheUploader{ + diskStorage: coordinator.diskStorage, + etcdStorage: etcdStore, + } + klog.Info("uploading local cache") + uploader.Upload() + return nil +} + +func (coordinator *Coordinator) delegateNodeLease(cloudLeaseClient coordclientset.LeaseInterface, obj interface{}) { + newLease := obj.(*coordinationv1.Lease) + for i := 0; i < leaseDelegateRetryTimes; i++ { + // ResourceVersions of lease objects in pool-coordinator always have different rv + // from what of cloud lease. So we should get cloud lease first and then update + // it with lease from pool-coordinator. + cloudLease, err := cloudLeaseClient.Get(coordinator.ctx, newLease.Name, metav1.GetOptions{}) + if apierrors.IsNotFound(err) { + if _, err := cloudLeaseClient.Create(coordinator.ctx, cloudLease, metav1.CreateOptions{}); err != nil { + klog.Errorf("failed to create lease %s at cloud, %v", newLease.Name, err) + continue + } + } + + lease := newLease.DeepCopy() + lease.ResourceVersion = cloudLease.ResourceVersion + if _, err := cloudLeaseClient.Update(coordinator.ctx, lease, metav1.UpdateOptions{}); err != nil { + klog.Errorf("failed to update lease %s at cloud, %v", newLease.Name, err) + continue + } + } +} + +func (coordinator *Coordinator) detectPoolCacheSynced(obj interface{}) { + lease := obj.(*coordinationv1.Lease) + renewTime := lease.Spec.RenewTime + if time.Now().After(renewTime.Add(defaultPoolCacheStaleDuration)) { + coordinator.Lock() + defer coordinator.Unlock() + coordinator.isPoolCacheSynced = false + } +} + +// poolScopedCacheSyncManager will continuously sync pool-scoped resources from cloud to pool-coordinator. +// After resource sync is completed, it will periodically renew the informer synced lease, which is used by +// other yurthub to determine if pool-coordinator is ready to handle requests of pool-scoped resources. +// It uses proxied client to list/watch pool-scoped resources from cloud APIServer, which +// will be automatically cached into pool-coordinator through YurtProxyServer. +type poolScopedCacheSyncManager struct { + ctx context.Context + isRunning bool + // proxiedClient is a client of Cloud APIServer which is proxied by yurthub. + proxiedClient kubernetes.Interface + // coordinatorClient is a client of APIServer in pool-coordinator. + coordinatorClient kubernetes.Interface + // nodeName will be used to update the ownerReference of informer synced lease. + nodeName string + informerSyncedLease *coordinationv1.Lease + getEtcdStore func() storage.Store + cancel func() +} + +func (p *poolScopedCacheSyncManager) EnsureStart() error { + if !p.isRunning { + if err := p.coordinatorClient.CoordinationV1().Leases(namespaceInformerLease).Delete(p.ctx, nameInformerLease, metav1.DeleteOptions{}); err != nil { + return fmt.Errorf("failed to delete informer sync lease, %v", err) + } + + etcdStore := p.getEtcdStore() + if etcdStore == nil { + return fmt.Errorf("got empty etcd storage") + } + if err := etcdStore.DeleteComponentResources(constants.DefaultPoolScopedUserAgent); err != nil { + return fmt.Errorf("failed to clean old pool-scoped cache, %v", err) + } + + ctx, cancel := context.WithCancel(p.ctx) + hasInformersSynced := []cache.InformerSynced{} + informerFactory := informers.NewSharedInformerFactory(p.proxiedClient, 0) + for gvr := range constants.PoolScopedResources { + informer, err := informerFactory.ForResource(gvr) + if err != nil { + cancel() + return fmt.Errorf("failed to add informer for %s, %v", gvr.String(), err) + } + hasInformersSynced = append(hasInformersSynced, informer.Informer().HasSynced) } + + informerFactory.Start(ctx.Done()) + go p.holdInformerSync(ctx, hasInformersSynced) + p.cancel = cancel + p.isRunning = true } + return nil } -func (coordinator *Coordinator) cacheIsUploaded() bool { - // check yurthub pod is uploaded - return true +func (p *poolScopedCacheSyncManager) EnsureStop() { + if p.isRunning { + p.cancel() + p.cancel = nil + p.isRunning = false + } } -func (coordinator *Coordinator) IsReady() bool { +func (p *poolScopedCacheSyncManager) holdInformerSync(ctx context.Context, hasInformersSynced []cache.InformerSynced) { + if cache.WaitForCacheSync(ctx.Done(), hasInformersSynced...) { + informerLease := NewInformerLease( + p.coordinatorClient, + nameInformerLease, + namespaceInformerLease, + p.nodeName, + int32(defaultInformerLeaseRenewDuration.Seconds()), + 5) + p.renewInformerLease(ctx, informerLease) + return + } + klog.Error("failed to wait for cache synced, it was canceled") +} + +func (p *poolScopedCacheSyncManager) renewInformerLease(ctx context.Context, lease informerLease) { + for { + t := time.NewTicker(defaultInformerLeaseRenewDuration) + select { + case <-ctx.Done(): + klog.Info("cancel renew informer lease") + return + case <-t.C: + newLease, err := lease.Update(p.informerSyncedLease) + if err != nil { + klog.Errorf("failed to update informer lease, %v", err) + continue + } + p.informerSyncedLease = newLease + } + } +} - return true +// coordinatorLeaseInformerManager will use pool-coordinator client to list/watch +// lease in pool-coordinator. Through passing different event handler, it can either +// delegating node lease by leader yurthub or detecting the informer synced lease to +// check if pool-coordinator is ready for requests of pool-scoped resources. +type coordinatorLeaseInformerManager struct { + ctx context.Context + coordinatorClient kubernetes.Interface + name string + isRunning bool + cancel func() +} + +func (c *coordinatorLeaseInformerManager) Name() string { + return c.name +} + +func (c *coordinatorLeaseInformerManager) EnsureStartWithHandler(handler cache.FilteringResourceEventHandler) { + if !c.isRunning { + ctx, cancel := context.WithCancel(c.ctx) + informerFactory := informers.NewSharedInformerFactory(c.coordinatorClient, 0) + informerFactory.Coordination().V1().Leases().Informer().AddEventHandler(handler) + informerFactory.Start(ctx.Done()) + c.isRunning = true + c.cancel = cancel + } +} + +func (c *coordinatorLeaseInformerManager) EnsureStop() { + if c.isRunning { + c.cancel() + c.isRunning = false + } +} + +// localCacheUploader can upload resources in local cache to pool cache. +// Currently, we only upload pods and nodes to pool-coordinator. +type localCacheUploader struct { + diskStorage storage.Store + etcdStorage storage.Store +} + +func (l *localCacheUploader) Upload() { + objBytes := l.resourcesToUpload() + for k, b := range objBytes { + rv, err := getRv(b) + if err != nil { + klog.Errorf("failed to get name from bytes %s, %v", string(b), err) + continue + } + + if err := l.createOrUpdate(k, b, rv); err != nil { + klog.Errorf("failed to upload %s, %v", k.Key(), err) + } + } +} + +func (l *localCacheUploader) createOrUpdate(key storage.Key, objBytes []byte, rv uint64) error { + err := l.etcdStorage.Create(key, objBytes) + + if err == storage.ErrKeyExists { + // try to update + _, updateErr := l.etcdStorage.Update(key, objBytes, rv) + if updateErr == storage.ErrUpdateConflict { + return nil + } + return updateErr + } + + return err +} + +func (l *localCacheUploader) resourcesToUpload() map[storage.Key][]byte { + objBytes := map[storage.Key][]byte{} + for info := range constants.UploadResourcesKeyBuildInfo { + gvr := schema.GroupVersionResource{ + Group: info.Group, + Version: info.Version, + Resource: info.Resources, + } + keys, err := l.diskStorage.ListResourceKeysOfComponent(info.Component, gvr) + if err != nil { + klog.Errorf("failed to get object keys from disk for %s, %v", gvr.String(), err) + continue + } + + for _, k := range keys { + buf, err := l.diskStorage.Get(k) + if err != nil { + klog.Errorf("failed to read local cache of key %s, %v", k.Key(), err) + continue + } + objBytes[k] = buf + } + } + return objBytes +} + +func getRv(objBytes []byte) (uint64, error) { + obj := &unstructured.Unstructured{} + if err := json.Unmarshal(objBytes, obj); err != nil { + return 0, fmt.Errorf("failed to unmarshal json: %v", err) + } + + rv, err := strconv.ParseUint(obj.GetResourceVersion(), 10, 64) + if err != nil { + return 0, fmt.Errorf("failed to parse rv %s of pod %s, %v", obj.GetName(), obj.GetResourceVersion(), err) + } + + return rv, nil +} + +func ifDelegateHeartBeat(obj interface{}) bool { + lease, ok := obj.(*coordinationv1.Lease) + if !ok { + return false + } + v, ok := lease.Labels[healthchecker.DelegateHeartBeat] + return ok && v == "true" +} + +func ifInformerSyncLease(obj interface{}) bool { + lease, ok := obj.(*coordinationv1.Lease) + if !ok { + return false + } + + return lease.Name == nameInformerLease && lease.Namespace == namespaceInformerLease +} + +func buildProxiedClientWithUserAgent(proxyAddr string, userAgent string) (kubernetes.Interface, error) { + kubeConfig, err := clientcmd.BuildConfigFromFlags(proxyAddr, "") + if err != nil { + return nil, err + } + + kubeConfig.UserAgent = userAgent + client, err := kubernetes.NewForConfig(kubeConfig) + if err != nil { + return nil, err + } + return client, nil } diff --git a/pkg/yurthub/poolcoordinator/informer_lease.go b/pkg/yurthub/poolcoordinator/informer_lease.go new file mode 100644 index 00000000000..b48a9ff5f59 --- /dev/null +++ b/pkg/yurthub/poolcoordinator/informer_lease.go @@ -0,0 +1,178 @@ +/* +Copyright 2021 The OpenYurt Authors. + +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 poolcoordinator + +import ( + "context" + "fmt" + "time" + + coordinationv1 "k8s.io/api/coordination/v1" + corev1 "k8s.io/api/core/v1" + apierrors "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + clientset "k8s.io/client-go/kubernetes" + coordclientset "k8s.io/client-go/kubernetes/typed/coordination/v1" + "k8s.io/klog/v2" + "k8s.io/utils/clock" + "k8s.io/utils/pointer" +) + +// TODO: reuse code of healthchecker.NodeLease +// Add the file temporarily for coordinator use, because healthchecker.NodeLease cannot +// be directly used by coordinator and modifying it will encounter a lot of changes. +// We currently want to focus on the implementation of coordinator, so making a copy of it +// and modifying it as we want. We can reuse the code of healthchecker.NodeLease in further work. + +const ( + maxBackoff = 1 * time.Second +) + +type informerLease interface { + Update(base *coordinationv1.Lease) (*coordinationv1.Lease, error) +} + +type informerLeaseTmpl struct { + client clientset.Interface + leaseClient coordclientset.LeaseInterface + leaseName string + leaseNamespace string + leaseDurationSeconds int32 + holderIdentity string + failedRetry int + clock clock.Clock +} + +func NewInformerLease(coordinatorClient clientset.Interface, leaseName string, leaseNamespace string, holderIdentity string, leaseDurationSeconds int32, failedRetry int) informerLease { + return &informerLeaseTmpl{ + client: coordinatorClient, + leaseClient: coordinatorClient.CoordinationV1().Leases(leaseNamespace), + leaseName: leaseName, + holderIdentity: holderIdentity, + failedRetry: failedRetry, + leaseDurationSeconds: leaseDurationSeconds, + clock: clock.RealClock{}, + } +} + +func (nl *informerLeaseTmpl) Update(base *coordinationv1.Lease) (*coordinationv1.Lease, error) { + if base != nil { + lease, err := nl.retryUpdateLease(base) + if err == nil { + return lease, nil + } + } + lease, created, err := nl.backoffEnsureLease() + if err != nil { + return nil, err + } + if !created { + return nl.retryUpdateLease(lease) + } + return lease, nil +} + +func (nl *informerLeaseTmpl) retryUpdateLease(base *coordinationv1.Lease) (*coordinationv1.Lease, error) { + var err error + var lease *coordinationv1.Lease + for i := 0; i < nl.failedRetry; i++ { + lease, err = nl.leaseClient.Update(context.Background(), nl.newLease(base), metav1.UpdateOptions{}) + if err == nil { + return lease, nil + } + if apierrors.IsConflict(err) { + base, _, err = nl.backoffEnsureLease() + if err != nil { + return nil, err + } + continue + } + klog.V(3).Infof("update node lease fail: %v, will try it.", err) + } + return nil, err +} + +func (nl *informerLeaseTmpl) backoffEnsureLease() (*coordinationv1.Lease, bool, error) { + var ( + lease *coordinationv1.Lease + created bool + err error + ) + + sleep := 100 * time.Millisecond + for { + lease, created, err = nl.ensureLease() + if err == nil { + break + } + sleep = sleep * 2 + if sleep > maxBackoff { + return nil, false, fmt.Errorf("backoff ensure lease error: %w", err) + } + nl.clock.Sleep(sleep) + } + return lease, created, err +} + +func (nl *informerLeaseTmpl) ensureLease() (*coordinationv1.Lease, bool, error) { + lease, err := nl.leaseClient.Get(context.Background(), nl.leaseName, metav1.GetOptions{}) + if apierrors.IsNotFound(err) { + lease, err := nl.leaseClient.Create(context.Background(), nl.newLease(nil), metav1.CreateOptions{}) + if err != nil { + return nil, false, err + } + return lease, true, nil + } else if err != nil { + return nil, false, err + } + return lease, false, nil +} + +func (nl *informerLeaseTmpl) newLease(base *coordinationv1.Lease) *coordinationv1.Lease { + var lease *coordinationv1.Lease + if base == nil { + lease = &coordinationv1.Lease{ + ObjectMeta: metav1.ObjectMeta{ + Name: nl.leaseName, + Namespace: nl.leaseNamespace, + }, + Spec: coordinationv1.LeaseSpec{ + HolderIdentity: pointer.StringPtr(nl.holderIdentity), + LeaseDurationSeconds: pointer.Int32Ptr(nl.leaseDurationSeconds), + }, + } + } else { + lease = base.DeepCopy() + } + + lease.Spec.RenewTime = &metav1.MicroTime{Time: nl.clock.Now()} + if lease.OwnerReferences == nil || len(lease.OwnerReferences) == 0 { + if node, err := nl.client.CoreV1().Nodes().Get(context.Background(), nl.holderIdentity, metav1.GetOptions{}); err == nil { + lease.OwnerReferences = []metav1.OwnerReference{ + { + APIVersion: corev1.SchemeGroupVersion.WithKind("Node").Version, + Kind: corev1.SchemeGroupVersion.WithKind("Node").Kind, + Name: nl.holderIdentity, + UID: node.UID, + }, + } + } else { + klog.Errorf("failed to get node %q when trying to set owner ref to the node lease: %v", nl.leaseName, err) + } + } + return lease +} diff --git a/pkg/yurthub/poolcoordinator/leader_election.go b/pkg/yurthub/poolcoordinator/leader_election.go index ed9eae6e620..565caf4c9ac 100644 --- a/pkg/yurthub/poolcoordinator/leader_election.go +++ b/pkg/yurthub/poolcoordinator/leader_election.go @@ -39,21 +39,21 @@ const ( type HubElector struct { coordinatorClient kubernetes.Interface coordinatorHealthChecker healthchecker.HealthChecker - cloudAPIServerHealthChecker healthchecker.HealthChecker + cloudAPIServerHealthChecker healthchecker.MultipleBackendsHealthChecker electorStatus chan int32 le *leaderelection.LeaderElector inElecting bool } -func NewHubElector(cfg *config.YurtHubConfiguration, coordinatorClient kubernetes.Interface, cloudAPIServerHealthyChecker healthchecker.HealthChecker, stopCh <-chan struct{}) (*HubElector, error) { - coordinatorHealthyChecker, err := healthchecker.NewCoordinatorHealthChecker(cfg, coordinatorClient, cloudAPIServerHealthyChecker, stopCh) - if err != nil { - return nil, err - } - +func NewHubElector( + cfg *config.YurtHubConfiguration, + coordinatorClient kubernetes.Interface, + coordinatorHealthChecker healthchecker.HealthChecker, + cloudAPIServerHealthyChecker healthchecker.MultipleBackendsHealthChecker, + stopCh <-chan struct{}) (*HubElector, error) { he := &HubElector{ coordinatorClient: coordinatorClient, - coordinatorHealthChecker: coordinatorHealthyChecker, + coordinatorHealthChecker: coordinatorHealthChecker, cloudAPIServerHealthChecker: cloudAPIServerHealthyChecker, electorStatus: make(chan int32), } @@ -79,7 +79,8 @@ func NewHubElector(cfg *config.YurtHubConfiguration, coordinatorClient kubernete he.electorStatus <- LeaderHub }, OnStoppedLeading: func() { - + klog.Infof("yurthub of %s is no more a leader", cfg.NodeName) + he.electorStatus <- PendingHub }, }, }) diff --git a/pkg/yurthub/proxy/local/local.go b/pkg/yurthub/proxy/local/local.go index 4b58dfb5521..77eff158a55 100644 --- a/pkg/yurthub/proxy/local/local.go +++ b/pkg/yurthub/proxy/local/local.go @@ -50,17 +50,19 @@ type IsHealthy func() bool // LocalProxy is responsible for handling requests when remote servers are unhealthy type LocalProxy struct { - cacheMgr manager.CacheManager - isHealthy IsHealthy - minRequestTimeout time.Duration + cacheMgr manager.CacheManager + isCloudHealthy IsHealthy + isCoordinatorReady IsHealthy + minRequestTimeout time.Duration } // NewLocalProxy creates a *LocalProxy -func NewLocalProxy(cacheMgr manager.CacheManager, isHealthy IsHealthy, minRequestTimeout time.Duration) *LocalProxy { +func NewLocalProxy(cacheMgr manager.CacheManager, isCloudHealthy IsHealthy, isCoordinatorHealthy IsHealthy, minRequestTimeout time.Duration) *LocalProxy { return &LocalProxy{ - cacheMgr: cacheMgr, - isHealthy: isHealthy, - minRequestTimeout: minRequestTimeout, + cacheMgr: cacheMgr, + isCloudHealthy: isCloudHealthy, + isCoordinatorReady: isCoordinatorHealthy, + minRequestTimeout: minRequestTimeout, } } @@ -182,6 +184,7 @@ func (lp *LocalProxy) localWatch(w http.ResponseWriter, req *http.Request) error timeout = time.Duration(float64(lp.minRequestTimeout) * (rand.Float64() + 1.0)) } + isPoolScopedListWatch := util.IsPoolScopedResouceListWatchRequest(req) watchTimer := time.NewTimer(timeout) intervalTicker := time.NewTicker(interval) defer watchTimer.Stop() @@ -196,7 +199,12 @@ func (lp *LocalProxy) localWatch(w http.ResponseWriter, req *http.Request) error return nil case <-intervalTicker.C: // if cluster becomes healthy, exit the watch wait - if lp.isHealthy() { + if lp.isCloudHealthy() { + return nil + } + + // if poolcoordinator becomes healthy, exit the watch wait + if isPoolScopedListWatch && lp.isCoordinatorReady() { return nil } } diff --git a/pkg/yurthub/proxy/local/local_test.go b/pkg/yurthub/proxy/local/local_test.go index 51481dcab5f..6ce5ca88361 100644 --- a/pkg/yurthub/proxy/local/local_test.go +++ b/pkg/yurthub/proxy/local/local_test.go @@ -70,7 +70,7 @@ func TestServeHTTPForWatch(t *testing.T) { return false } - lp := NewLocalProxy(cacheM, fn, 0) + lp := NewLocalProxy(cacheM, fn, fn, 0) testcases := map[string]struct { userAgent string @@ -164,7 +164,7 @@ func TestServeHTTPForWatchWithHealthyChange(t *testing.T) { return cnt > 2 // after 6 seconds, become healthy } - lp := NewLocalProxy(cacheM, fn, 0) + lp := NewLocalProxy(cacheM, fn, fn, 0) testcases := map[string]struct { userAgent string @@ -247,7 +247,7 @@ func TestServeHTTPForWatchWithMinRequestTimeout(t *testing.T) { return false } - lp := NewLocalProxy(cacheM, fn, 10*time.Second) + lp := NewLocalProxy(cacheM, fn, fn, 10*time.Second) testcases := map[string]struct { userAgent string @@ -339,7 +339,7 @@ func TestServeHTTPForPost(t *testing.T) { return false } - lp := NewLocalProxy(cacheM, fn, 0) + lp := NewLocalProxy(cacheM, fn, fn, 0) testcases := map[string]struct { userAgent string @@ -419,7 +419,7 @@ func TestServeHTTPForDelete(t *testing.T) { return false } - lp := NewLocalProxy(cacheM, fn, 0) + lp := NewLocalProxy(cacheM, fn, fn, 0) testcases := map[string]struct { userAgent string @@ -486,7 +486,7 @@ func TestServeHTTPForGetReqCache(t *testing.T) { return false } - lp := NewLocalProxy(cacheM, fn, 0) + lp := NewLocalProxy(cacheM, fn, fn, 0) testcases := map[string]struct { userAgent string @@ -639,7 +639,7 @@ func TestServeHTTPForListReqCache(t *testing.T) { return false } - lp := NewLocalProxy(cacheM, fn, 0) + lp := NewLocalProxy(cacheM, fn, fn, 0) testcases := map[string]struct { userAgent string diff --git a/pkg/yurthub/proxy/pool/pool.go b/pkg/yurthub/proxy/pool/pool.go new file mode 100644 index 00000000000..7d98982177c --- /dev/null +++ b/pkg/yurthub/proxy/pool/pool.go @@ -0,0 +1,256 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 pool + +import ( + "context" + "fmt" + "io" + "net/http" + "net/url" + "time" + + "k8s.io/apimachinery/pkg/api/errors" + apirequest "k8s.io/apiserver/pkg/endpoints/request" + "k8s.io/klog/v2" + + "github.com/openyurtio/openyurt/pkg/yurthub/cachemanager" + "github.com/openyurtio/openyurt/pkg/yurthub/filter/manager" + "github.com/openyurtio/openyurt/pkg/yurthub/proxy/util" + "github.com/openyurtio/openyurt/pkg/yurthub/transport" + hubutil "github.com/openyurtio/openyurt/pkg/yurthub/util" +) + +const ( + watchCheckInterval = 5 * time.Second +) + +// PoolCoordinatorProxy is responsible for handling requests when remote servers are unhealthy +type PoolCoordinatorProxy struct { + poolCoordinatorProxy *util.RemoteProxy + localCacheMgr cachemanager.CacheManager + filterMgr *manager.Manager + isCoordinatorReady func() bool + stopCh <-chan struct{} +} + +func NewPoolCoordinatorProxy( + poolCoordinatorAddr *url.URL, + localCacheMgr cachemanager.CacheManager, + transportMgr transport.Interface, + filterMgr *manager.Manager, + isCoordinatorReady func() bool, + stopCh <-chan struct{}) (*PoolCoordinatorProxy, error) { + if poolCoordinatorAddr == nil { + return nil, fmt.Errorf("pool-coordinator addr cannot be nil") + } + + pp := &PoolCoordinatorProxy{ + localCacheMgr: localCacheMgr, + isCoordinatorReady: isCoordinatorReady, + filterMgr: filterMgr, + stopCh: stopCh, + } + + proxy, err := util.NewRemoteProxy( + poolCoordinatorAddr, + pp.modifyResponse, + pp.errorHandler, + transportMgr, + stopCh) + if err != nil { + return nil, fmt.Errorf("failed to create remote proxy for pool-coordinator, %v", err) + } + + pp.poolCoordinatorProxy = proxy + return pp, nil +} + +// ServeHTTP of PoolCoordinatorProxy is able to handle read-only request, including +// watch, list, get. Other verbs that will write data to the cache are not supported +// currently. +func (pp *PoolCoordinatorProxy) ServeHTTP(rw http.ResponseWriter, req *http.Request) { + var err error + ctx := req.Context() + reqInfo, ok := apirequest.RequestInfoFrom(ctx) + if !ok || reqInfo == nil { + klog.Errorf("pool-coordinator proxy cannot handle request(%s), cannot get requestInfo", hubutil.ReqString(req), reqInfo) + util.Err(errors.NewBadRequest(fmt.Sprintf("pool-coordinator proxy cannot handle request(%s), cannot get requestInfo", hubutil.ReqString(req))), rw, req) + return + } + if reqInfo.IsResourceRequest { + switch reqInfo.Verb { + case "create": + err = pp.poolPost(rw, req) + case "list", "get": + err = pp.poolQuery(rw, req) + case "watch": + err = pp.poolWatch(rw, req) + default: + err = fmt.Errorf("unsupported verb for pool coordinator proxy: %s", reqInfo.Verb) + } + if err != nil { + klog.Errorf("could not proxy to pool-coordinator for %s, %v", hubutil.ReqString(req), err) + util.Err(errors.NewBadRequest(err.Error()), rw, req) + } + } else { + klog.Errorf("pool-coordinator does not support request(%s) when cluster is unhealthy, requestInfo: %v", hubutil.ReqString(req), reqInfo) + util.Err(errors.NewBadRequest(fmt.Sprintf("pool-coordinator does not support request(%s) when cluster is unhealthy", hubutil.ReqString(req))), rw, req) + } +} + +func (pp *PoolCoordinatorProxy) poolPost(rw http.ResponseWriter, req *http.Request) error { + ctx := req.Context() + info, _ := apirequest.RequestInfoFrom(ctx) + klog.V(4).Infof("pool handle post, req=%s, reqInfo=%s", hubutil.ReqString(req), hubutil.ReqInfoString(info)) + if util.IsSubjectAccessReviewCreateGetRequest(req) || util.IsEventCreateRequest(req) { + // kubelet needs to create subjectaccessreviews for auth + pp.poolCoordinatorProxy.ServeHTTP(rw, req) + return nil + } + + return fmt.Errorf("unsupported post request") +} + +func (pp *PoolCoordinatorProxy) poolQuery(rw http.ResponseWriter, req *http.Request) error { + if util.IsPoolScopedResouceListWatchRequest(req) || util.IsSubjectAccessReviewCreateGetRequest(req) { + pp.poolCoordinatorProxy.ServeHTTP(rw, req) + return nil + } + return fmt.Errorf("unsupported query request") +} + +func (pp *PoolCoordinatorProxy) poolWatch(rw http.ResponseWriter, req *http.Request) error { + if util.IsPoolScopedResouceListWatchRequest(req) { + clientReqCtx := req.Context() + poolServeCtx, poolServeCancel := context.WithCancel(clientReqCtx) + + go func() { + t := time.NewTicker(watchCheckInterval) + defer t.Stop() + for { + select { + case <-t.C: + if !pp.isCoordinatorReady() { + klog.Infof("notified the pool coordinator is not ready for handling request, cancel watch %s", hubutil.ReqString(req)) + poolServeCancel() + return + } + case <-clientReqCtx.Done(): + klog.Infof("notified client canceled the watch request %s, stop proxy it to pool coordinator", hubutil.ReqString(req)) + return + } + } + }() + + newReq := req.Clone(poolServeCtx) + pp.poolCoordinatorProxy.ServeHTTP(rw, newReq) + klog.Infof("watch %s to pool coordinator exited", hubutil.ReqString(req)) + return nil + } + return fmt.Errorf("unsupported watch request") +} + +func (pp *PoolCoordinatorProxy) errorHandler(rw http.ResponseWriter, req *http.Request, err error) { + klog.Errorf("remote proxy error handler: %s, %v", hubutil.ReqString(req), err) + ctx := req.Context() + if info, ok := apirequest.RequestInfoFrom(ctx); ok { + if info.Verb == "get" || info.Verb == "list" { + if obj, err := pp.localCacheMgr.QueryCache(req); err == nil { + hubutil.WriteObject(http.StatusOK, obj, rw, req) + return + } + } + } + rw.WriteHeader(http.StatusBadGateway) +} + +func (pp *PoolCoordinatorProxy) modifyResponse(resp *http.Response) error { + if resp == nil || resp.Request == nil { + klog.Infof("no request info in response, skip cache response") + return nil + } + + req := resp.Request + ctx := req.Context() + + // re-added transfer-encoding=chunked response header for watch request + info, exists := apirequest.RequestInfoFrom(ctx) + if exists { + if info.Verb == "watch" { + klog.V(5).Infof("add transfer-encoding=chunked header into response for req %s", hubutil.ReqString(req)) + h := resp.Header + if hv := h.Get("Transfer-Encoding"); hv == "" { + h.Add("Transfer-Encoding", "chunked") + } + } + } + + if resp.StatusCode >= http.StatusOK && resp.StatusCode <= http.StatusPartialContent { + // prepare response content type + reqContentType, _ := hubutil.ReqContentTypeFrom(ctx) + respContentType := resp.Header.Get("Content-Type") + if len(respContentType) == 0 { + respContentType = reqContentType + } + ctx = hubutil.WithRespContentType(ctx, respContentType) + req = req.WithContext(ctx) + + // filter response data + if pp.filterMgr != nil { + if ok, runner := pp.filterMgr.FindRunner(req); ok { + wrapBody, needUncompressed := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "filter") + size, filterRc, err := runner.Filter(req, wrapBody, pp.stopCh) + if err != nil { + klog.Errorf("failed to filter response for %s, %v", hubutil.ReqString(req), err) + return err + } + resp.Body = filterRc + if size > 0 { + resp.ContentLength = int64(size) + resp.Header.Set("Content-Length", fmt.Sprint(size)) + } + + // after gunzip in filter, the header content encoding should be removed. + // because there's no need to gunzip response.body again. + if needUncompressed { + resp.Header.Del("Content-Encoding") + } + } + } + // cache resp with storage interface + pp.cacheResponse(req, resp) + } + + return nil +} + +func (pp *PoolCoordinatorProxy) cacheResponse(req *http.Request, resp *http.Response) { + if pp.localCacheMgr.CanCacheFor(req) { + ctx := req.Context() + req = req.WithContext(ctx) + wrapPrc, _ := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "cache-manager") + + rc, prc := hubutil.NewDualReadCloser(req, wrapPrc, false) + go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { + if err := pp.localCacheMgr.CacheResponse(req, prc, stopCh); err != nil { + klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) + } + }(req, prc, ctx.Done()) + req.Body = rc + } +} diff --git a/pkg/yurthub/proxy/proxy.go b/pkg/yurthub/proxy/proxy.go index 49dba96c1dc..88fe2e3188a 100644 --- a/pkg/yurthub/proxy/proxy.go +++ b/pkg/yurthub/proxy/proxy.go @@ -17,6 +17,7 @@ limitations under the License. package proxy import ( + "errors" "net/http" "k8s.io/apimachinery/pkg/util/sets" @@ -28,31 +29,38 @@ import ( "github.com/openyurtio/openyurt/cmd/yurthub/app/config" "github.com/openyurtio/openyurt/pkg/yurthub/cachemanager" "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator" "github.com/openyurtio/openyurt/pkg/yurthub/proxy/local" + "github.com/openyurtio/openyurt/pkg/yurthub/proxy/pool" "github.com/openyurtio/openyurt/pkg/yurthub/proxy/remote" "github.com/openyurtio/openyurt/pkg/yurthub/proxy/util" "github.com/openyurtio/openyurt/pkg/yurthub/tenant" "github.com/openyurtio/openyurt/pkg/yurthub/transport" + hubutil "github.com/openyurtio/openyurt/pkg/yurthub/util" ) type yurtReverseProxy struct { - resolver apirequest.RequestInfoResolver - loadBalancer remote.LoadBalancer - checker healthchecker.HealthChecker - localProxy http.Handler - cacheMgr cachemanager.CacheManager - maxRequestsInFlight int - tenantMgr tenant.Interface - stopCh <-chan struct{} + resolver apirequest.RequestInfoResolver + loadBalancer remote.LoadBalancer + cloudHealthChecker healthchecker.MultipleBackendsHealthChecker + coordinatorHealtChecker healthchecker.HealthChecker + localProxy http.Handler + poolProxy http.Handler + maxRequestsInFlight int + tenantMgr tenant.Interface + coordinator *poolcoordinator.Coordinator + workingMode hubutil.WorkingMode } // NewYurtReverseProxyHandler creates a http handler for proxying // all of incoming requests. func NewYurtReverseProxyHandler( yurtHubCfg *config.YurtHubConfiguration, - cacheMgr cachemanager.CacheManager, + localCacheMgr cachemanager.CacheManager, transportMgr transport.Interface, - healthChecker healthchecker.MultipleBackendsHealthChecker, + coordinator *poolcoordinator.Coordinator, + cloudHealthChecker healthchecker.MultipleBackendsHealthChecker, + coordinatorHealthChecker healthchecker.HealthChecker, tenantMgr tenant.Interface, stopCh <-chan struct{}) (http.Handler, error) { cfg := &server.Config{ @@ -63,32 +71,57 @@ func NewYurtReverseProxyHandler( lb, err := remote.NewLoadBalancer( yurtHubCfg.LBMode, yurtHubCfg.RemoteServers, - cacheMgr, + localCacheMgr, transportMgr, - healthChecker, + coordinator, + cloudHealthChecker, yurtHubCfg.FilterManager, + yurtHubCfg.WorkingMode, stopCh) if err != nil { return nil, err } - var localProxy http.Handler - // When yurthub is working in cloud mode, cacheMgr will be set to nil which means the local cache is disabled, - // so we don't need to create a LocalProxy. - if cacheMgr != nil { - localProxy = local.NewLocalProxy(cacheMgr, healthChecker.IsHealthy, yurtHubCfg.MinRequestTimeout) + var localProxy, poolProxy http.Handler + + if yurtHubCfg.WorkingMode == hubutil.WorkingModeEdge { + // When yurthub works in Edge mode, we may use local proxy or pool proxy to handle + // the request when offline. + localProxy = local.NewLocalProxy(localCacheMgr, + cloudHealthChecker.IsHealthy, + func() bool { + _, ready := coordinator.IsHealthy() + return ready + }, + yurtHubCfg.MinRequestTimeout, + ) localProxy = local.WithFakeTokenInject(localProxy, yurtHubCfg.SerializerManager) + poolProxy, err = pool.NewPoolCoordinatorProxy( + yurtHubCfg.CoordinatorServer, + localCacheMgr, + transportMgr, + yurtHubCfg.FilterManager, + func() bool { + _, isReady := coordinator.IsReady() + return isReady + }, + stopCh) + if err != nil { + return nil, err + } } yurtProxy := &yurtReverseProxy{ - resolver: resolver, - loadBalancer: lb, - checker: healthChecker, - localProxy: localProxy, - cacheMgr: cacheMgr, - maxRequestsInFlight: yurtHubCfg.MaxRequestInFlight, - tenantMgr: tenantMgr, - stopCh: stopCh, + resolver: resolver, + loadBalancer: lb, + cloudHealthChecker: cloudHealthChecker, + coordinatorHealtChecker: coordinatorHealthChecker, + localProxy: localProxy, + poolProxy: poolProxy, + maxRequestsInFlight: yurtHubCfg.MaxRequestInFlight, + coordinator: coordinator, + tenantMgr: tenantMgr, + workingMode: yurtHubCfg.WorkingMode, } return yurtProxy.buildHandlerChain(yurtProxy), nil @@ -97,16 +130,17 @@ func NewYurtReverseProxyHandler( func (p *yurtReverseProxy) buildHandlerChain(handler http.Handler) http.Handler { handler = util.WithRequestTrace(handler) handler = util.WithRequestContentType(handler) - if p.cacheMgr != nil { + if p.workingMode == hubutil.WorkingModeEdge { handler = util.WithCacheHeaderCheck(handler) } handler = util.WithRequestTimeout(handler) - if p.cacheMgr != nil { + if p.workingMode == hubutil.WorkingModeEdge { handler = util.WithListRequestSelector(handler) } handler = util.WithRequestTraceFull(handler) handler = util.WithMaxInFlightLimit(handler, p.maxRequestsInFlight) handler = util.WithRequestClientComponent(handler) + handler = util.WithIfPoolScopedResource(handler) if p.tenantMgr != nil && p.tenantMgr.GetTenantNs() != "" { handler = util.WithSaTokenSubstitute(handler, p.tenantMgr) @@ -120,13 +154,83 @@ func (p *yurtReverseProxy) buildHandlerChain(handler http.Handler) http.Handler } func (p *yurtReverseProxy) ServeHTTP(rw http.ResponseWriter, req *http.Request) { - isKubeletLeaseReq := util.IsKubeletLeaseReq(req) - if !isKubeletLeaseReq && p.checker.IsHealthy() || p.localProxy == nil { + if p.workingMode == hubutil.WorkingModeCloud { p.loadBalancer.ServeHTTP(rw, req) - } else { - if isKubeletLeaseReq { - p.checker.RenewKubeletLeaseTime() + return + } + + switch { + case util.IsKubeletLeaseReq(req): + p.handleKubeletLease(rw, req) + case util.IsEventCreateRequest(req): + p.eventHandler(rw, req) + case util.IsPoolScopedResouceListWatchRequest(req): + p.poolScopedResouceHandler(rw, req) + case util.IsSubjectAccessReviewCreateGetRequest(req): + p.subjectAccessReviewHandler(rw, req) + default: + // For resource request that do not need to be handled by pool-coordinator, + // handling the request with cloud apiserver or local cache. + if p.cloudHealthChecker.IsHealthy() { + p.loadBalancer.ServeHTTP(rw, req) + } else { + p.localProxy.ServeHTTP(rw, req) } + } +} + +func (p *yurtReverseProxy) handleKubeletLease(rw http.ResponseWriter, req *http.Request) { + p.cloudHealthChecker.RenewKubeletLeaseTime() + p.coordinatorHealtChecker.RenewKubeletLeaseTime() + if p.localProxy != nil { p.localProxy.ServeHTTP(rw, req) } } + +func (p *yurtReverseProxy) eventHandler(rw http.ResponseWriter, req *http.Request) { + if p.cloudHealthChecker.IsHealthy() { + p.loadBalancer.ServeHTTP(rw, req) + // TODO: We should also consider create the event in pool-coordinator when the cloud is healthy. + } else if _, isReady := p.coordinator.IsReady(); isReady { + p.poolProxy.ServeHTTP(rw, req) + } else { + p.localProxy.ServeHTTP(rw, req) + } +} + +func (p *yurtReverseProxy) poolScopedResouceHandler(rw http.ResponseWriter, req *http.Request) { + if _, isReady := p.coordinator.IsReady(); isReady { + p.poolProxy.ServeHTTP(rw, req) + } else if p.cloudHealthChecker.IsHealthy() { + p.loadBalancer.ServeHTTP(rw, req) + } else { + p.localProxy.ServeHTTP(rw, req) + } +} + +func (p *yurtReverseProxy) subjectAccessReviewHandler(rw http.ResponseWriter, req *http.Request) { + if isRequestFromPoolCoordinator(req) { + if _, isReady := p.coordinator.IsReady(); isReady { + p.poolProxy.ServeHTTP(rw, req) + } else { + err := errors.New("request is from pool-coordinator but it's currently not healthy") + klog.Errorf("could not handle SubjectAccessReview req %s, %v", hubutil.ReqString(req), err) + util.Err(err, rw, req) + } + } else { + if p.cloudHealthChecker.IsHealthy() { + p.loadBalancer.ServeHTTP(rw, req) + } else { + err := errors.New("request is from cloud APIServer but it's currently not healthy") + klog.Errorf("could not handle SubjectAccessReview req %s, %v", hubutil.ReqString(req), err) + util.Err(err, rw, req) + } + } +} + +func isRequestFromPoolCoordinator(req *http.Request) bool { + // TODO: need a way to check if the logs/exec request is from APIServer or PoolCoordinator. + // We should avoid sending SubjectAccessReview to Pool-Coordinator if the logs/exec requests + // come from APIServer, which may fail for RBAC differences, vise versa. + return false +} diff --git a/pkg/yurthub/proxy/remote/loadbalancer.go b/pkg/yurthub/proxy/remote/loadbalancer.go index 8158ef3b427..135a3e401e8 100644 --- a/pkg/yurthub/proxy/remote/loadbalancer.go +++ b/pkg/yurthub/proxy/remote/loadbalancer.go @@ -17,29 +17,41 @@ limitations under the License. package remote import ( + "context" "fmt" + "io" "net/http" "net/url" "sync" + "time" + "k8s.io/apimachinery/pkg/runtime/schema" + apirequest "k8s.io/apiserver/pkg/endpoints/request" "k8s.io/klog/v2" "github.com/openyurtio/openyurt/pkg/yurthub/cachemanager" "github.com/openyurtio/openyurt/pkg/yurthub/filter/manager" "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator" + coordinatorconstants "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" + "github.com/openyurtio/openyurt/pkg/yurthub/proxy/util" "github.com/openyurtio/openyurt/pkg/yurthub/transport" - "github.com/openyurtio/openyurt/pkg/yurthub/util" + hubutil "github.com/openyurtio/openyurt/pkg/yurthub/util" +) + +const ( + watchCheckInterval = 5 * time.Second ) type loadBalancerAlgo interface { - PickOne() *RemoteProxy + PickOne() *util.RemoteProxy Name() string } type rrLoadBalancerAlgo struct { sync.Mutex checker healthchecker.MultipleBackendsHealthChecker - backends []*RemoteProxy + backends []*util.RemoteProxy next int } @@ -47,7 +59,7 @@ func (rr *rrLoadBalancerAlgo) Name() string { return "rr algorithm" } -func (rr *rrLoadBalancerAlgo) PickOne() *RemoteProxy { +func (rr *rrLoadBalancerAlgo) PickOne() *util.RemoteProxy { if len(rr.backends) == 0 { return nil } else if len(rr.backends) == 1 { @@ -81,14 +93,14 @@ func (rr *rrLoadBalancerAlgo) PickOne() *RemoteProxy { type priorityLoadBalancerAlgo struct { sync.Mutex checker healthchecker.MultipleBackendsHealthChecker - backends []*RemoteProxy + backends []*util.RemoteProxy } func (prio *priorityLoadBalancerAlgo) Name() string { return "priority algorithm" } -func (prio *priorityLoadBalancerAlgo) PickOne() *RemoteProxy { +func (prio *priorityLoadBalancerAlgo) PickOne() *util.RemoteProxy { if len(prio.backends) == 0 { return nil } else if len(prio.backends) == 1 { @@ -116,22 +128,36 @@ type LoadBalancer interface { } type loadBalancer struct { - backends []*RemoteProxy - algo loadBalancerAlgo + backends []*util.RemoteProxy + algo loadBalancerAlgo + localCacheMgr cachemanager.CacheManager + filterManager *manager.Manager + coordinator *poolcoordinator.Coordinator + workingMode hubutil.WorkingMode + stopCh <-chan struct{} } // NewLoadBalancer creates a loadbalancer for specified remote servers func NewLoadBalancer( lbMode string, remoteServers []*url.URL, - cacheMgr cachemanager.CacheManager, + localCacheMgr cachemanager.CacheManager, transportMgr transport.Interface, + coordinator *poolcoordinator.Coordinator, healthChecker healthchecker.MultipleBackendsHealthChecker, filterManager *manager.Manager, + workingMode hubutil.WorkingMode, stopCh <-chan struct{}) (LoadBalancer, error) { - backends := make([]*RemoteProxy, 0, len(remoteServers)) + lb := &loadBalancer{ + localCacheMgr: localCacheMgr, + filterManager: filterManager, + coordinator: coordinator, + workingMode: workingMode, + stopCh: stopCh, + } + backends := make([]*util.RemoteProxy, 0, len(remoteServers)) for i := range remoteServers { - b, err := NewRemoteProxy(remoteServers[i], cacheMgr, transportMgr, filterManager, stopCh) + b, err := util.NewRemoteProxy(remoteServers[i], lb.modifyResponse, lb.errorHandler, transportMgr, stopCh) if err != nil { klog.Errorf("could not new proxy backend(%s), %v", remoteServers[i].String(), err) continue @@ -163,10 +189,234 @@ func (lb *loadBalancer) ServeHTTP(rw http.ResponseWriter, req *http.Request) { rp := lb.algo.PickOne() if rp == nil { // exceptional case - klog.Errorf("could not pick one healthy backends by %s for request %s", lb.algo.Name(), util.ReqString(req)) + klog.Errorf("could not pick one healthy backends by %s for request %s", lb.algo.Name(), hubutil.ReqString(req)) http.Error(rw, "could not pick one healthy backends, try again to go through local proxy.", http.StatusInternalServerError) return } - klog.V(3).Infof("picked backend %s by %s for request %s", rp.Name(), lb.algo.Name(), util.ReqString(req)) + klog.V(3).Infof("picked backend %s by %s for request %s", rp.Name(), lb.algo.Name(), hubutil.ReqString(req)) + if util.IsPoolScopedResouceListWatchRequest(req) { + // We get here possibly because the pool-coordinator is not ready. + // We should cancel the watch request when pool-coordinator becomes ready. + klog.Infof("pool-coordinator is not ready, we use cloud APIServer to temporarily handle the req: %s", hubutil.ReqString(req)) + clientReqCtx := req.Context() + cloudServeCtx, cloudServeCancel := context.WithCancel(clientReqCtx) + + go func() { + t := time.NewTicker(watchCheckInterval) + defer t.Stop() + for { + select { + case <-t.C: + if _, isReady := lb.coordinator.IsReady(); isReady { + klog.Infof("notified the pool coordinator is ready, cancel the req %s making it handled by pool coordinator", hubutil.ReqString(req)) + cloudServeCancel() + return + } + case <-clientReqCtx.Done(): + return + } + } + }() + + newReq := req.Clone(cloudServeCtx) + req = newReq + } + rp.ServeHTTP(rw, req) } + +func (lb *loadBalancer) errorHandler(rw http.ResponseWriter, req *http.Request, err error) { + klog.Errorf("remote proxy error handler: %s, %v", hubutil.ReqString(req), err) + if lb.localCacheMgr == nil || !lb.localCacheMgr.CanCacheFor(req) { + rw.WriteHeader(http.StatusBadGateway) + return + } + + ctx := req.Context() + if info, ok := apirequest.RequestInfoFrom(ctx); ok { + if info.Verb == "get" || info.Verb == "list" { + if obj, err := lb.localCacheMgr.QueryCache(req); err == nil { + hubutil.WriteObject(http.StatusOK, obj, rw, req) + return + } + } + } + rw.WriteHeader(http.StatusBadGateway) +} + +func (lb *loadBalancer) modifyResponse(resp *http.Response) error { + if resp == nil || resp.Request == nil { + klog.Infof("no request info in response, skip cache response") + return nil + } + + req := resp.Request + ctx := req.Context() + + // re-added transfer-encoding=chunked response header for watch request + info, exists := apirequest.RequestInfoFrom(ctx) + if exists { + if info.Verb == "watch" { + klog.V(5).Infof("add transfer-encoding=chunked header into response for req %s", hubutil.ReqString(req)) + h := resp.Header + if hv := h.Get("Transfer-Encoding"); hv == "" { + h.Add("Transfer-Encoding", "chunked") + } + } + } + + if resp.StatusCode >= http.StatusOK && resp.StatusCode <= http.StatusPartialContent { + // prepare response content type + reqContentType, _ := hubutil.ReqContentTypeFrom(ctx) + respContentType := resp.Header.Get("Content-Type") + if len(respContentType) == 0 { + respContentType = reqContentType + } + ctx = hubutil.WithRespContentType(ctx, respContentType) + req = req.WithContext(ctx) + + // filter response data + if lb.filterManager != nil { + if ok, runner := lb.filterManager.FindRunner(req); ok { + wrapBody, needUncompressed := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "filter") + size, filterRc, err := runner.Filter(req, wrapBody, lb.stopCh) + if err != nil { + klog.Errorf("failed to filter response for %s, %v", hubutil.ReqString(req), err) + return err + } + resp.Body = filterRc + if size > 0 { + resp.ContentLength = int64(size) + resp.Header.Set("Content-Length", fmt.Sprint(size)) + } + + // after gunzip in filter, the header content encoding should be removed. + // because there's no need to gunzip response.body again. + if needUncompressed { + resp.Header.Del("Content-Encoding") + } + } + } + + if lb.workingMode == hubutil.WorkingModeEdge { + // cache resp with storage interface + lb.cacheResponse(req, resp) + } + } else if resp.StatusCode == http.StatusNotFound && info.Verb == "list" && lb.localCacheMgr != nil { + // 404 Not Found: The CRD may have been unregistered and should be updated locally as well. + // Other types of requests may return a 404 response for other reasons (for example, getting a pod that doesn't exist). + // And the main purpose is to return 404 when list an unregistered resource locally, so here only consider the list request. + gvr := schema.GroupVersionResource{ + Group: info.APIGroup, + Version: info.APIVersion, + Resource: info.Resource, + } + + err := lb.localCacheMgr.DeleteKindFor(gvr) + if err != nil { + klog.Errorf("failed: %v", err) + } + } + return nil +} + +func (lb *loadBalancer) cacheResponse(req *http.Request, resp *http.Response) { + if lb.localCacheMgr.CanCacheFor(req) { + ctx := req.Context() + wrapPrc, _ := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "cache-manager") + resp.Body = wrapPrc + + poolCacheManager, isHealthy := lb.coordinator.IsHealthy() + if isHealthy && poolCacheManager != nil { + if !isLeaderHubUserAgent(ctx) { + if isRequestOfNodeAndPod(ctx) { + // Currently, for request that does not come from "leader-yurthub", + // we only cache pod and node resources to pool-coordinator. + // Note: We do not allow the non-leader yurthub to cache pool-scoped resources + // into pool-coordinator to ensure that only one yurthub can update pool-scoped + // cache to avoid inconsistency of data. + lb.cacheToLocalAndPool(req, resp, poolCacheManager) + } else { + lb.cacheToLocal(req, resp) + } + } else { + if isPoolScopedCtx(ctx) { + // Leader Yurthub will always list/watch all resources, which contain may resource this + // node does not need. + lb.cacheToPool(req, resp, poolCacheManager) + } else { + klog.Errorf("failed to cache response for request %s, leader yurthub does not cache non-poolscoped resources.", hubutil.ReqString(req)) + } + } + return + } + + // When pool-coordinator is not healthy or not be enabled, we can + // only cache the response at local. + lb.cacheToLocal(req, resp) + } +} + +func (lb *loadBalancer) cacheToLocal(req *http.Request, resp *http.Response) { + ctx := req.Context() + req = req.WithContext(ctx) + rc, prc := hubutil.NewDualReadCloser(req, resp.Body, false) + go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { + if err := lb.localCacheMgr.CacheResponse(req, prc, stopCh); err != nil { + klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) + } + }(req, prc, ctx.Done()) + req.Body = rc +} + +func (lb *loadBalancer) cacheToPool(req *http.Request, resp *http.Response, poolCacheManager cachemanager.CacheManager) { + ctx := req.Context() + req = req.WithContext(ctx) + rc, prc := hubutil.NewDualReadCloser(req, resp.Body, false) + go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { + if err := poolCacheManager.CacheResponse(req, prc, stopCh); err != nil { + klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) + } + }(req, prc, ctx.Done()) + req.Body = rc +} + +func (lb *loadBalancer) cacheToLocalAndPool(req *http.Request, resp *http.Response, poolCacheMgr cachemanager.CacheManager) { + ctx := req.Context() + req = req.WithContext(ctx) + rc, prc1, prc2 := hubutil.NewTripleReadCloser(req, resp.Body, false) + go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { + if err := lb.localCacheMgr.CacheResponse(req, prc, stopCh); err != nil { + klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) + } + }(req, prc1, ctx.Done()) + + if poolCacheMgr != nil { + go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { + if err := poolCacheMgr.CacheResponse(req, prc, stopCh); err != nil { + klog.Errorf("failed to cache req %s in pool cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) + } + }(req, prc2, ctx.Done()) + } + req.Body = rc +} + +func isLeaderHubUserAgent(reqCtx context.Context) bool { + comp, hasComp := hubutil.ClientComponentFrom(reqCtx) + return hasComp && comp == coordinatorconstants.DefaultPoolScopedUserAgent +} + +func isPoolScopedCtx(reqCtx context.Context) bool { + poolScoped, hasPoolScoped := hubutil.IfPoolScopedResourceFrom(reqCtx) + return hasPoolScoped && poolScoped +} + +func isRequestOfNodeAndPod(reqCtx context.Context) bool { + reqInfo, ok := apirequest.RequestInfoFrom(reqCtx) + if !ok { + return false + } + + return (reqInfo.Resource == "nodes" && reqInfo.APIGroup == "" && reqInfo.APIVersion == "v1") || + (reqInfo.Resource == "pods" && reqInfo.APIGroup == "" && reqInfo.APIVersion == "v1") +} diff --git a/pkg/yurthub/proxy/remote/loadbalancer_test.go b/pkg/yurthub/proxy/remote/loadbalancer_test.go index ceff84da8d4..f0477362978 100644 --- a/pkg/yurthub/proxy/remote/loadbalancer_test.go +++ b/pkg/yurthub/proxy/remote/loadbalancer_test.go @@ -17,12 +17,41 @@ limitations under the License. package remote import ( + "context" + "net/http" "net/url" "testing" "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" + "github.com/openyurtio/openyurt/pkg/yurthub/proxy/util" + "github.com/openyurtio/openyurt/pkg/yurthub/transport" ) +var neverStop <-chan struct{} = context.Background().Done() + +type nopRoundTrip struct{} + +func (n *nopRoundTrip) RoundTrip(r *http.Request) (*http.Response, error) { + return &http.Response{ + Status: http.StatusText(http.StatusOK), + StatusCode: http.StatusOK, + }, nil +} + +type fakeTransportManager struct{} + +func (f *fakeTransportManager) CurrentTransport() http.RoundTripper { + return &nopRoundTrip{} +} + +func (f *fakeTransportManager) BearerTransport() http.RoundTripper { + return &nopRoundTrip{} +} + +func (f *fakeTransportManager) Close(_ string) {} + +var transportMgr transport.Interface = &fakeTransportManager{} + type PickBackend struct { DeltaRequestsCnt int ReturnServer string @@ -66,11 +95,13 @@ func TestRrLoadBalancerAlgo(t *testing.T) { checker := healthchecker.NewFakeChecker(true, map[string]int{}) for k, tc := range testcases { - backends := make([]*RemoteProxy, len(tc.Servers)) + backends := make([]*util.RemoteProxy, len(tc.Servers)) for i := range tc.Servers { + var err error u, _ := url.Parse(tc.Servers[i]) - backends[i] = &RemoteProxy{ - remoteServer: u, + backends[i], err = util.NewRemoteProxy(u, nil, nil, transportMgr, neverStop) + if err != nil { + t.Errorf("failed to create remote server for %s, %v", u.String(), err) } } @@ -80,20 +111,20 @@ func TestRrLoadBalancerAlgo(t *testing.T) { } for i := range tc.PickBackends { - var b *RemoteProxy + var b *util.RemoteProxy for j := 0; j < tc.PickBackends[i].DeltaRequestsCnt; j++ { b = rr.PickOne() } if len(tc.PickBackends[i].ReturnServer) == 0 { if b != nil { - t.Errorf("%s rr lb pick: expect no backend server, but got %s", k, b.remoteServer.String()) + t.Errorf("%s rr lb pick: expect no backend server, but got %s", k, b.RemoteServer().String()) } } else { if b == nil { t.Errorf("%s rr lb pick: expect backend server: %s, but got no backend server", k, tc.PickBackends[i].ReturnServer) - } else if b.remoteServer.String() != tc.PickBackends[i].ReturnServer { - t.Errorf("%s rr lb pick(round %d): expect backend server: %s, but got %s", k, i+1, tc.PickBackends[i].ReturnServer, b.remoteServer.String()) + } else if b.RemoteServer().String() != tc.PickBackends[i].ReturnServer { + t.Errorf("%s rr lb pick(round %d): expect backend server: %s, but got %s", k, i+1, tc.PickBackends[i].ReturnServer, b.RemoteServer().String()) } } } @@ -127,11 +158,13 @@ func TestRrLoadBalancerAlgoWithReverseHealthy(t *testing.T) { "http://127.0.0.1:8081": 2, }) for k, tc := range testcases { - backends := make([]*RemoteProxy, len(tc.Servers)) + backends := make([]*util.RemoteProxy, len(tc.Servers)) for i := range tc.Servers { + var err error u, _ := url.Parse(tc.Servers[i]) - backends[i] = &RemoteProxy{ - remoteServer: u, + backends[i], err = util.NewRemoteProxy(u, nil, nil, transportMgr, neverStop) + if err != nil { + t.Errorf("failed to create remote server for %s, %v", u.String(), err) } } @@ -141,20 +174,20 @@ func TestRrLoadBalancerAlgoWithReverseHealthy(t *testing.T) { } for i := range tc.PickBackends { - var b *RemoteProxy + var b *util.RemoteProxy for j := 0; j < tc.PickBackends[i].DeltaRequestsCnt; j++ { b = rr.PickOne() } if len(tc.PickBackends[i].ReturnServer) == 0 { if b != nil { - t.Errorf("%s rr lb pick: expect no backend server, but got %s", k, b.remoteServer.String()) + t.Errorf("%s rr lb pick: expect no backend server, but got %s", k, b.RemoteServer().String()) } } else { if b == nil { t.Errorf("%s rr lb pick(round %d): expect backend server: %s, but got no backend server", k, i+1, tc.PickBackends[i].ReturnServer) - } else if b.remoteServer.String() != tc.PickBackends[i].ReturnServer { - t.Errorf("%s rr lb pick(round %d): expect backend server: %s, but got %s", k, i+1, tc.PickBackends[i].ReturnServer, b.remoteServer.String()) + } else if b.RemoteServer().String() != tc.PickBackends[i].ReturnServer { + t.Errorf("%s rr lb pick(round %d): expect backend server: %s, but got %s", k, i+1, tc.PickBackends[i].ReturnServer, b.RemoteServer().String()) } } } @@ -199,11 +232,13 @@ func TestPriorityLoadBalancerAlgo(t *testing.T) { checker := healthchecker.NewFakeChecker(true, map[string]int{}) for k, tc := range testcases { - backends := make([]*RemoteProxy, len(tc.Servers)) + backends := make([]*util.RemoteProxy, len(tc.Servers)) for i := range tc.Servers { + var err error u, _ := url.Parse(tc.Servers[i]) - backends[i] = &RemoteProxy{ - remoteServer: u, + backends[i], err = util.NewRemoteProxy(u, nil, nil, transportMgr, neverStop) + if err != nil { + t.Errorf("failed to create remote server for %s, %v", u.String(), err) } } @@ -213,20 +248,20 @@ func TestPriorityLoadBalancerAlgo(t *testing.T) { } for i := range tc.PickBackends { - var b *RemoteProxy + var b *util.RemoteProxy for j := 0; j < tc.PickBackends[i].DeltaRequestsCnt; j++ { b = rr.PickOne() } if len(tc.PickBackends[i].ReturnServer) == 0 { if b != nil { - t.Errorf("%s priority lb pick: expect no backend server, but got %s", k, b.remoteServer.String()) + t.Errorf("%s priority lb pick: expect no backend server, but got %s", k, b.RemoteServer().String()) } } else { if b == nil { t.Errorf("%s priority lb pick: expect backend server: %s, but got no backend server", k, tc.PickBackends[i].ReturnServer) - } else if b.remoteServer.String() != tc.PickBackends[i].ReturnServer { - t.Errorf("%s priority lb pick(round %d): expect backend server: %s, but got %s", k, i+1, tc.PickBackends[i].ReturnServer, b.remoteServer.String()) + } else if b.RemoteServer().String() != tc.PickBackends[i].ReturnServer { + t.Errorf("%s priority lb pick(round %d): expect backend server: %s, but got %s", k, i+1, tc.PickBackends[i].ReturnServer, b.RemoteServer().String()) } } } @@ -258,11 +293,13 @@ func TestPriorityLoadBalancerAlgoWithReverseHealthy(t *testing.T) { "http://127.0.0.1:8080": 2, "http://127.0.0.1:8081": 3}) for k, tc := range testcases { - backends := make([]*RemoteProxy, len(tc.Servers)) + backends := make([]*util.RemoteProxy, len(tc.Servers)) for i := range tc.Servers { + var err error u, _ := url.Parse(tc.Servers[i]) - backends[i] = &RemoteProxy{ - remoteServer: u, + backends[i], err = util.NewRemoteProxy(u, nil, nil, transportMgr, neverStop) + if err != nil { + t.Errorf("failed to create remote server for %s, %v", u.String(), err) } } @@ -272,20 +309,20 @@ func TestPriorityLoadBalancerAlgoWithReverseHealthy(t *testing.T) { } for i := range tc.PickBackends { - var b *RemoteProxy + var b *util.RemoteProxy for j := 0; j < tc.PickBackends[i].DeltaRequestsCnt; j++ { b = rr.PickOne() } if len(tc.PickBackends[i].ReturnServer) == 0 { if b != nil { - t.Errorf("%s priority lb pick: expect no backend server, but got %s", k, b.remoteServer.String()) + t.Errorf("%s priority lb pick: expect no backend server, but got %s", k, b.RemoteServer().String()) } } else { if b == nil { t.Errorf("%s priority lb pick: expect backend server: %s, but got no backend server", k, tc.PickBackends[i].ReturnServer) - } else if b.remoteServer.String() != tc.PickBackends[i].ReturnServer { - t.Errorf("%s priority lb pick(round %d): expect backend server: %s, but got %s", k, i+1, tc.PickBackends[i].ReturnServer, b.remoteServer.String()) + } else if b.RemoteServer().String() != tc.PickBackends[i].ReturnServer { + t.Errorf("%s priority lb pick(round %d): expect backend server: %s, but got %s", k, i+1, tc.PickBackends[i].ReturnServer, b.RemoteServer().String()) } } } diff --git a/pkg/yurthub/proxy/remote/remote.go b/pkg/yurthub/proxy/util/remote.go similarity index 50% rename from pkg/yurthub/proxy/remote/remote.go rename to pkg/yurthub/proxy/util/remote.go index 5a4e0f5ceff..ff30b688cd0 100644 --- a/pkg/yurthub/proxy/remote/remote.go +++ b/pkg/yurthub/proxy/util/remote.go @@ -14,27 +14,19 @@ See the License for the specific language governing permissions and limitations under the License. */ -package remote +package util import ( - "context" - "errors" "fmt" - "io" "net/http" "net/http/httputil" "net/url" "strings" - "k8s.io/apimachinery/pkg/runtime/schema" "k8s.io/apimachinery/pkg/util/httpstream" "k8s.io/apimachinery/pkg/util/proxy" - apirequest "k8s.io/apiserver/pkg/endpoints/request" "k8s.io/klog/v2" - "github.com/openyurtio/openyurt/pkg/yurthub/cachemanager" - "github.com/openyurtio/openyurt/pkg/yurthub/filter/manager" - "github.com/openyurtio/openyurt/pkg/yurthub/proxy/util" "github.com/openyurtio/openyurt/pkg/yurthub/transport" hubutil "github.com/openyurtio/openyurt/pkg/yurthub/util" ) @@ -42,9 +34,7 @@ import ( // RemoteProxy is an reverse proxy for remote server type RemoteProxy struct { reverseProxy *httputil.ReverseProxy - cacheMgr cachemanager.CacheManager remoteServer *url.URL - filterManager *manager.Manager currentTransport http.RoundTripper bearerTransport http.RoundTripper upgradeHandler *proxy.UpgradeAwareHandler @@ -61,9 +51,9 @@ func (r *responder) Error(w http.ResponseWriter, req *http.Request, err error) { // NewRemoteProxy creates an *RemoteProxy object, and will be used by LoadBalancer func NewRemoteProxy(remoteServer *url.URL, - cacheMgr cachemanager.CacheManager, + modifyResponse func(*http.Response) error, + errhandler func(http.ResponseWriter, *http.Request, error), transportMgr transport.Interface, - filterManager *manager.Manager, stopCh <-chan struct{}) (*RemoteProxy, error) { currentTransport := transportMgr.CurrentTransport() if currentTransport == nil { @@ -81,9 +71,7 @@ func NewRemoteProxy(remoteServer *url.URL, proxyBackend := &RemoteProxy{ reverseProxy: httputil.NewSingleHostReverseProxy(remoteServer), - cacheMgr: cacheMgr, remoteServer: remoteServer, - filterManager: filterManager, currentTransport: currentTransport, bearerTransport: bearerTransport, upgradeHandler: upgradeAwareHandler, @@ -92,9 +80,9 @@ func NewRemoteProxy(remoteServer *url.URL, } proxyBackend.reverseProxy.Transport = proxyBackend - proxyBackend.reverseProxy.ModifyResponse = proxyBackend.modifyResponse + proxyBackend.reverseProxy.ModifyResponse = modifyResponse proxyBackend.reverseProxy.FlushInterval = -1 - proxyBackend.reverseProxy.ErrorHandler = proxyBackend.errorHandler + proxyBackend.reverseProxy.ErrorHandler = errhandler return proxyBackend, nil } @@ -122,110 +110,6 @@ func (rp *RemoteProxy) ServeHTTP(rw http.ResponseWriter, req *http.Request) { rp.reverseProxy.ServeHTTP(rw, req) } -func (rp *RemoteProxy) modifyResponse(resp *http.Response) error { - if resp == nil || resp.Request == nil { - klog.Infof("no request info in response, skip cache response") - return nil - } - - req := resp.Request - ctx := req.Context() - - // re-added transfer-encoding=chunked response header for watch request - info, exists := apirequest.RequestInfoFrom(ctx) - if exists { - if info.Verb == "watch" { - klog.V(5).Infof("add transfer-encoding=chunked header into response for req %s", hubutil.ReqString(req)) - h := resp.Header - if hv := h.Get("Transfer-Encoding"); hv == "" { - h.Add("Transfer-Encoding", "chunked") - } - } - } - - if resp.StatusCode >= http.StatusOK && resp.StatusCode <= http.StatusPartialContent { - // prepare response content type - reqContentType, _ := hubutil.ReqContentTypeFrom(ctx) - respContentType := resp.Header.Get("Content-Type") - if len(respContentType) == 0 { - respContentType = reqContentType - } - ctx = hubutil.WithRespContentType(ctx, respContentType) - req = req.WithContext(ctx) - - // filter response data - if rp.filterManager != nil { - if ok, runner := rp.filterManager.FindRunner(req); ok { - wrapBody, needUncompressed := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "filter") - size, filterRc, err := runner.Filter(req, wrapBody, rp.stopCh) - if err != nil { - klog.Errorf("failed to filter response for %s, %v", hubutil.ReqString(req), err) - return err - } - resp.Body = filterRc - if size > 0 { - resp.ContentLength = int64(size) - resp.Header.Set("Content-Length", fmt.Sprint(size)) - } - - // after gunzip in filter, the header content encoding should be removed. - // because there's no need to gunzip response.body again. - if needUncompressed { - resp.Header.Del("Content-Encoding") - } - } - } - - // cache resp with storage interface - if rp.cacheMgr != nil && rp.cacheMgr.CanCacheFor(req) { - rc, prc := hubutil.NewDualReadCloser(req, resp.Body, true) - wrapPrc, _ := hubutil.NewGZipReaderCloser(resp.Header, prc, req, "cache-manager") - go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { - err := rp.cacheMgr.CacheResponse(req, prc, stopCh) - if err != nil && err != io.EOF && !errors.Is(err, context.Canceled) { - klog.Errorf("%s response cache ended with error, %v", hubutil.ReqString(req), err) - } - }(req, wrapPrc, rp.stopCh) - - resp.Body = rc - } - } else if resp.StatusCode == http.StatusNotFound && info.Verb == "list" && rp.cacheMgr != nil { - // 404 Not Found: The CRD may have been unregistered and should be updated locally as well. - // Other types of requests may return a 404 response for other reasons (for example, getting a pod that doesn't exist). - // And the main purpose is to return 404 when list an unregistered resource locally, so here only consider the list request. - gvr := schema.GroupVersionResource{ - Group: info.APIGroup, - Version: info.APIVersion, - Resource: info.Resource, - } - - err := rp.cacheMgr.DeleteKindFor(gvr) - if err != nil { - klog.Errorf("failed: %v", err) - } - } - return nil -} - -func (rp *RemoteProxy) errorHandler(rw http.ResponseWriter, req *http.Request, err error) { - klog.Errorf("remote proxy error handler: %s, %v", hubutil.ReqString(req), err) - if rp.cacheMgr == nil || !rp.cacheMgr.CanCacheFor(req) { - rw.WriteHeader(http.StatusBadGateway) - return - } - - ctx := req.Context() - if info, ok := apirequest.RequestInfoFrom(ctx); ok { - if info.Verb == "get" || info.Verb == "list" { - if obj, err := rp.cacheMgr.QueryCache(req); err == nil { - util.WriteObject(http.StatusOK, obj, rw, req) - return - } - } - } - rw.WriteHeader(http.StatusBadGateway) -} - // RoundTrip is used to implement http.RoundTripper for RemoteProxy. func (rp *RemoteProxy) RoundTrip(req *http.Request) (*http.Response, error) { // when edge client(like kube-proxy, flannel, etc) use service account(default InClusterConfig) to access yurthub, diff --git a/pkg/yurthub/proxy/util/util.go b/pkg/yurthub/proxy/util/util.go index ee5efeec407..76632db1605 100644 --- a/pkg/yurthub/proxy/util/util.go +++ b/pkg/yurthub/proxy/util/util.go @@ -176,6 +176,33 @@ func WithRequestClientComponent(handler http.Handler) http.Handler { }) } +func WithIfPoolScopedResource(handler http.Handler) http.Handler { + return http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + ctx := req.Context() + if info, ok := apirequest.RequestInfoFrom(ctx); ok { + var ifPoolScopedResource bool + if info.IsResourceRequest && isPoolScopedResource(info) { + ifPoolScopedResource = true + } + ctx = util.WithIfPoolScopedResource(ctx, ifPoolScopedResource) + req = req.WithContext(ctx) + } + handler.ServeHTTP(w, req) + }) +} + +func isPoolScopedResource(info *apirequest.RequestInfo) bool { + if info != nil { + if info.APIGroup == "" && info.APIVersion == "v1" && info.Resource == "endpoints" { + return true + } + if info.APIGroup == "discovery.k8s.io" && info.APIVersion == "v1" && info.Resource == "endpointslices" { + return true + } + } + return false +} + type wrapperResponseWriter struct { http.ResponseWriter http.Flusher @@ -433,3 +460,44 @@ func Err(err error, w http.ResponseWriter, req *http.Request) { klog.Errorf("request info is not found when err write, %s", util.ReqString(req)) } + +func IsPoolScopedResouceListWatchRequest(req *http.Request) bool { + ctx := req.Context() + info, ok := apirequest.RequestInfoFrom(ctx) + if !ok { + return false + } + + isPoolScopedResource, ok := util.IfPoolScopedResourceFrom(ctx) + return ok && isPoolScopedResource && (info.Verb == "list" || info.Verb == "watch") +} + +func IsSubjectAccessReviewCreateGetRequest(req *http.Request) bool { + ctx := req.Context() + info, ok := apirequest.RequestInfoFrom(ctx) + if !ok { + return false + } + + comp, ok := util.ClientComponentFrom(ctx) + if !ok { + return false + } + + return info.IsResourceRequest && + comp == "kubelet" && + info.Resource == "subjectaccessreviews" && + (info.Verb == "create" || info.Verb == "get") +} + +func IsEventCreateRequest(req *http.Request) bool { + ctx := req.Context() + info, ok := apirequest.RequestInfoFrom(ctx) + if !ok { + return false + } + + return info.IsResourceRequest && + info.Resource == "events" && + info.Verb == "create" +} diff --git a/pkg/yurthub/storage/disk/storage.go b/pkg/yurthub/storage/disk/storage.go index e7368904cee..794bff3b65f 100644 --- a/pkg/yurthub/storage/disk/storage.go +++ b/pkg/yurthub/storage/disk/storage.go @@ -104,7 +104,7 @@ func (ds *diskStorage) Name() string { // Create will create a new file with content. key indicates the path of the file. func (ds *diskStorage) Create(key storage.Key, content []byte) error { - if err := utils.ValidateKey(key, storageKey{}); err != nil { + if err := utils.ValidateKV(key, content, storageKey{}); err != nil { return err } storageKey := key.(storageKey) diff --git a/pkg/yurthub/storage/etcd/etcd_suite_test.go b/pkg/yurthub/storage/etcd/etcd_suite_test.go new file mode 100644 index 00000000000..160132c3aa4 --- /dev/null +++ b/pkg/yurthub/storage/etcd/etcd_suite_test.go @@ -0,0 +1,58 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 etcd + +import ( + "os" + "os/exec" + "testing" + + . "github.com/onsi/ginkgo/v2" + . "github.com/onsi/gomega" +) + +var keyCacheDir = "/tmp/etcd-test" +var etcdDataDir = "/tmp/storagetest.etcd" +var devNull *os.File +var etcdCmd *exec.Cmd + +var _ = BeforeSuite(func() { + Expect(os.RemoveAll(keyCacheDir)).To(BeNil()) + Expect(os.RemoveAll(etcdDataDir)).To(BeNil()) + + // start etcd + var err error + devNull, err = os.OpenFile("/dev/null", os.O_RDWR, 0755) + Expect(err).To(BeNil()) + etcdCmd = exec.Command("/usr/local/etcd/etcd", "--data-dir="+etcdDataDir) + etcdCmd.Stdout = devNull + etcdCmd.Stderr = devNull + Expect(etcdCmd.Start()).To(BeNil()) +}) + +var _ = AfterSuite(func() { + Expect(os.RemoveAll(keyCacheDir)).To(BeNil()) + + // stop etcd + Expect(etcdCmd.Process.Kill()).To(BeNil()) + Expect(devNull.Close()).To(BeNil()) +}) + +func TestEtcd(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "ComponentKeyCache Test Suite") +} diff --git a/pkg/yurthub/storage/etcd/key.go b/pkg/yurthub/storage/etcd/key.go new file mode 100644 index 00000000000..958ea8c7905 --- /dev/null +++ b/pkg/yurthub/storage/etcd/key.go @@ -0,0 +1,78 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 etcd + +import ( + "errors" + "path/filepath" + + "k8s.io/apimachinery/pkg/api/validation/path" + "k8s.io/apimachinery/pkg/runtime/schema" + + "github.com/openyurtio/openyurt/pkg/yurthub/storage" +) + +// SpecialDefaultResourcePrefixes are prefixes compiled into Kubernetes. +// refer to SpecialDefaultResourcePrefixes in k8s.io/pkg/kubeapiserver/default_storage_factory_builder.go +var SpecialDefaultResourcePrefixes = map[schema.GroupResource]string{ + {Group: "", Resource: "replicationcontrollers"}: "controllers", + {Group: "", Resource: "endpoints"}: "services/endpoints", + {Group: "", Resource: "nodes"}: "minions", + {Group: "", Resource: "services"}: "services/specs", + {Group: "extensions", Resource: "ingresses"}: "ingress", + {Group: "networking.k8s.io", Resource: "ingresses"}: "ingress", + {Group: "extensions", Resource: "podsecuritypolicies"}: "podsecuritypolicy", + {Group: "policy", Resource: "podsecuritypolicies"}: "podsecuritypolicy", +} + +type storageKey struct { + comp string + path string +} + +func (k storageKey) Key() string { + return k.path +} + +func (k storageKey) component() string { + return k.comp +} + +func (s *etcdStorage) KeyFunc(info storage.KeyBuildInfo) (storage.Key, error) { + if info.Component == "" { + return nil, storage.ErrEmptyComponent + } + if info.Resources == "" { + return nil, storage.ErrEmptyResource + } + if errStrs := path.IsValidPathSegmentName(info.Name); len(errStrs) != 0 { + return nil, errors.New(errStrs[0]) + } + + resource := info.Resources + gr := schema.GroupResource{Group: info.Group, Resource: info.Resources} + if override, ok := SpecialDefaultResourcePrefixes[gr]; ok { + resource = override + } + + path := filepath.Join(s.prefix, resource, info.Namespace, info.Name) + + return storageKey{ + comp: info.Component, + path: path, + }, nil +} diff --git a/pkg/yurthub/storage/etcd/key_test.go b/pkg/yurthub/storage/etcd/key_test.go new file mode 100644 index 00000000000..bc59676282d --- /dev/null +++ b/pkg/yurthub/storage/etcd/key_test.go @@ -0,0 +1,103 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 etcd + +import ( + "testing" + + "github.com/openyurtio/openyurt/pkg/yurthub/storage" +) + +var s = etcdStorage{ + prefix: "/registry", +} + +var keyFunc = s.KeyFunc + +func TestKeyFunc(t *testing.T) { + cases := map[string]struct { + info storage.KeyBuildInfo + key string + err error + }{ + "core group normal case": { + info: storage.KeyBuildInfo{ + Group: "", + Resources: "pods", + Version: "v1", + Namespace: "test", + Name: "test-pod", + }, + key: "/registry/pods/test/test-pod", + }, + + "special prefix for node resource": { + info: storage.KeyBuildInfo{ + Group: "", + Resources: "nodes", + Version: "v1", + Namespace: "", + Name: "test-node", + }, + key: "/registry/minions/test-node", + }, + "not core group": { + info: storage.KeyBuildInfo{ + Group: "apps", + Resources: "deployments", + Version: "v1", + Namespace: "test", + Name: "test-deploy", + }, + key: "/registry/deployments/test/test-deploy", + }, + "special prefix for service resource": { + info: storage.KeyBuildInfo{ + Group: "networking.k8s.io", + Resources: "ingresses", + Version: "v1", + Namespace: "test", + Name: "test-ingress", + }, + key: "/registry/ingress/test/test-ingress", + }, + "empty resources": { + info: storage.KeyBuildInfo{ + Group: "", + Resources: "", + Version: "v1", + Namespace: "", + Name: "", + }, + err: storage.ErrEmptyResource, + }, + } + + for n, c := range cases { + key, err := keyFunc(c.info) + if err != c.err { + t.Errorf("unexpected error in case %s, want: %v, got: %v", n, c.err, err) + continue + } + if err != nil { + continue + } + if key.Key() != c.key { + t.Errorf("unexpected key in case %s, want: %s, got: %s", n, c.key, key) + } + } +} diff --git a/pkg/yurthub/storage/etcd/keycache.go b/pkg/yurthub/storage/etcd/keycache.go new file mode 100644 index 00000000000..5a9c21ca227 --- /dev/null +++ b/pkg/yurthub/storage/etcd/keycache.go @@ -0,0 +1,280 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 etcd + +import ( + "bytes" + "context" + "fmt" + "strings" + "sync" + + clientv3 "go.etcd.io/etcd/client/v3" + + coordinatorconstants "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" + "github.com/openyurtio/openyurt/pkg/yurthub/storage" + "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" +) + +// type state int + +// const ( +// done state = 0 +// processing state = 1 +// ) + +// type status struct + +type keySet struct { + m map[storageKey]struct{} +} + +// Difference will return keys in s but not in s2 +func (s keySet) Difference(s2 keySet) []storageKey { + keys := []storageKey{} + if s2.m == nil { + for k := range s.m { + keys = append(keys, k) + } + return keys + } + + for k := range s.m { + if _, ok := s2.m[k]; !ok { + keys = append(keys, k) + } + } + return keys +} + +// Do not directly modify value returned from functions of componentKeyCache, such as Load. +// Because it usually returns reference of internal objects for efficiency. +// The format in file is: +// component0:key0,key1... +// component1:key0,key1... +// ... +type componentKeyCache struct { + sync.Mutex + ctx context.Context + cache map[string]keySet + filePath string + keyFunc func(storage.KeyBuildInfo) (storage.Key, error) + fsOperator fs.FileSystemOperator + getEtcdClient func() *clientv3.Client +} + +func (c *componentKeyCache) Recover() error { + var buf []byte + var err error + if buf, err = c.fsOperator.Read(c.filePath); err == fs.ErrNotExists { + if err := c.fsOperator.CreateFile(c.filePath, []byte{}); err != nil { + return fmt.Errorf("failed to create cache file at %s, %v", c.filePath, err) + } + return nil + } else if err != nil { + return fmt.Errorf("failed to recover key cache from %s, %v", c.filePath, err) + } + + // successfully read from file + if len(buf) == 0 { + return nil + } + lines := strings.Split(string(buf), "\n") + for i, l := range lines { + s := strings.Split(l, ":") + if len(s) != 2 { + return fmt.Errorf("failed to parse line %d, invalid format", i) + } + comp, keys := s[0], strings.Split(s[1], ",") + ks := keySet{m: map[storageKey]struct{}{}} + for _, key := range keys { + ks.m[storageKey{path: key}] = struct{}{} + } + c.cache[comp] = ks + } + + poolScopedKeyset, err := c.getPoolScopedKeyset() + if err != nil { + return fmt.Errorf("failed to get pool-scoped keys, %v", err) + } + // Overwrite the data we recovered from local disk, if any. Because we + // only respect to the resources stored in pool-coordinator to recover the + // pool-scoped keys. + c.cache[coordinatorconstants.DefaultPoolScopedUserAgent] = *poolScopedKeyset + + return nil +} + +func (c *componentKeyCache) getPoolScopedKeyset() (*keySet, error) { + client := c.getEtcdClient() + if client == nil { + return nil, fmt.Errorf("got empty etcd client") + } + + keys := &keySet{m: map[storageKey]struct{}{}} + for gvr := range coordinatorconstants.PoolScopedResources { + getCtx, cancel := context.WithTimeout(c.ctx, defaultTimeout) + defer cancel() + rootKey, err := c.keyFunc(storage.KeyBuildInfo{ + Component: coordinatorconstants.DefaultPoolScopedUserAgent, + Group: gvr.Group, + Version: gvr.Version, + Resources: gvr.Resource, + }) + if err != nil { + return nil, fmt.Errorf("failed to generate keys for %s, %v", gvr.String(), err) + } + getResp, err := client.Get(getCtx, rootKey.Key(), clientv3.WithPrefix(), clientv3.WithKeysOnly()) + if err != nil { + return nil, fmt.Errorf("failed to get from etcd for %s, %v", gvr.String(), err) + } + + for _, kv := range getResp.Kvs { + ns, name, err := getNamespaceAndNameFromKeyPath(string(kv.Key)) + if err != nil { + return nil, fmt.Errorf("failed to parse namespace and name of %s", kv.Key) + } + key, err := c.keyFunc(storage.KeyBuildInfo{ + Component: coordinatorconstants.DefaultPoolScopedUserAgent, + Group: gvr.Group, + Version: gvr.Version, + Resources: gvr.Resource, + Namespace: ns, + Name: name, + }) + if err != nil { + return nil, fmt.Errorf("failed to create resource key for %v", kv.Key) + } + keys.m[key.(storageKey)] = struct{}{} + } + } + return keys, nil +} + +func (c *componentKeyCache) Load(component string) (keySet, bool) { + c.Lock() + defer c.Unlock() + cache, ok := c.cache[component] + return cache, ok +} + +func (c *componentKeyCache) AddKey(component string, key storageKey) { + c.Lock() + defer c.Unlock() + defer c.flush() + if _, ok := c.cache[component]; !ok { + c.cache[component] = keySet{m: map[storageKey]struct{}{ + key: {}, + }} + return + } + + keyset := c.cache[component] + if keyset.m == nil { + keyset.m = map[storageKey]struct{}{ + key: {}, + } + return + } + + c.cache[component].m[key] = struct{}{} +} + +func (c *componentKeyCache) DeleteKey(component string, key storageKey) { + c.Lock() + defer c.Unlock() + delete(c.cache[component].m, key) + c.flush() +} + +func (c *componentKeyCache) LoadOrStore(component string, keyset keySet) (keySet, bool) { + c.Lock() + defer c.Unlock() + if cache, ok := c.cache[component]; ok { + return cache, true + } else { + c.cache[component] = keyset + c.flush() + return keyset, false + } +} + +func (c *componentKeyCache) LoadAndDelete(component string) (keySet, bool) { + c.Lock() + defer c.Unlock() + if cache, ok := c.cache[component]; ok { + delete(c.cache, component) + c.flush() + return cache, true + } + return keySet{}, false +} + +func (c *componentKeyCache) DeleteAllKeysOfComponent(component string) { + c.Lock() + defer c.Unlock() + delete(c.cache, component) + c.flush() +} + +// func (c *componentKeyCache) MarkAsProcessing() { + +// } + +// func (c *componentKeyCache) MarkAsDone() { + +// } + +func (c *componentKeyCache) flush() error { + buf := bytes.NewBuffer(make([]byte, 0, 1024)) + for comp, ks := range c.cache { + line := bytes.NewBufferString(fmt.Sprintf("%s:", comp)) + keys := make([]string, 0, len(ks.m)) + for k := range ks.m { + keys = append(keys, k.Key()) + } + line.WriteString(strings.Join(keys, ",")) + line.WriteByte('\n') + buf.Write(line.Bytes()) + } + if buf.Len() != 0 { + // discard last '\n' + buf.Truncate(buf.Len() - 1) + } + if err := c.fsOperator.Write(c.filePath, buf.Bytes()); err != nil { + return fmt.Errorf("failed to flush cache to file %s, %v", c.filePath, err) + } + return nil +} + +func newComponentKeyCache(filePath string) *componentKeyCache { + return &componentKeyCache{ + filePath: filePath, + cache: map[string]keySet{}, + fsOperator: fs.FileSystemOperator{}, + } +} + +// We assume that path points to a namespaced resource. +func getNamespaceAndNameFromKeyPath(path string) (string, string, error) { + elems := strings.Split(path, "/") + if len(elems) < 2 { + return "", "", fmt.Errorf("unrecognized path: %v", path) + } + + return elems[len(elems)-2], elems[len(elems)-1], nil +} diff --git a/pkg/yurthub/storage/etcd/keycache_test.go b/pkg/yurthub/storage/etcd/keycache_test.go new file mode 100644 index 00000000000..c9512e221e3 --- /dev/null +++ b/pkg/yurthub/storage/etcd/keycache_test.go @@ -0,0 +1,311 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 etcd + +import ( + "os" + "path/filepath" + "reflect" + "strings" + "testing" + + "github.com/google/uuid" + . "github.com/onsi/ginkgo/v2" + . "github.com/onsi/gomega" + + "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" +) + +var _ = Describe("Test componentKeyCache setup", func() { + var cache *componentKeyCache + var fileName string + var f fs.FileSystemOperator + BeforeEach(func() { + fileName = uuid.New().String() + cache = newComponentKeyCache(filepath.Join(keyCacheDir, fileName)) + }) + AfterEach(func() { + Expect(os.RemoveAll(filepath.Join(keyCacheDir, fileName))) + }) + + It("should recover when cache file does not exist", func() { + Expect(cache.Recover()).To(BeNil()) + Expect(len(cache.cache)).To(BeZero()) + }) + + It("should recover when cache file is empty", func() { + Expect(f.CreateFile(filepath.Join(keyCacheDir, fileName), []byte{})).To(BeNil()) + Expect(cache.Recover()).To(BeNil()) + Expect(len(cache.cache)).To(BeZero()) + }) + + It("should recover when cache file exists and contains valid data", func() { + Expect(f.CreateFile(filepath.Join(keyCacheDir, fileName), []byte( + "kubelet:/registry/pods/default/pod1,/registry/pods/default/pod2\n"+ + "kube-proxy:/registry/configmaps/kube-system/kube-proxy", + ))).To(BeNil()) + Expect(cache.Recover()).To(BeNil()) + Expect(cache.cache).To(Equal(map[string]keySet{ + "kubelet": { + m: map[storageKey]struct{}{ + {path: "/registry/pods/default/pod1"}: {}, + {path: "/registry/pods/default/pod2"}: {}, + }, + }, + "kube-proxy": { + m: map[storageKey]struct{}{ + {path: "/registry/configmaps/kube-system/kube-proxy"}: {}, + }, + }, + })) + }) + + It("should return err when cache file contains invalid data", func() { + Expect(f.CreateFile(filepath.Join(keyCacheDir, fileName), []byte( + "kubelet,/registry/pods/default/pod1", + ))).To(BeNil()) + Expect(cache.Recover()).NotTo(BeNil()) + }) +}) + +var _ = Describe("Test componentKeyCache function", func() { + var cache *componentKeyCache + var fileName string + var key1, key2, key3 storageKey + BeforeEach(func() { + fileName = uuid.New().String() + cache = newComponentKeyCache(filepath.Join(keyCacheDir, fileName)) + key1 = storageKey{ + path: "/registry/pods/default/pod1", + } + key2 = storageKey{ + path: "/registry/pods/default/pod2", + } + key3 = storageKey{ + path: "/registry/pods/kube-system/kube-proxy", + } + }) + AfterEach(func() { + Expect(os.RemoveAll(filepath.Join(keyCacheDir, fileName))).To(BeNil()) + }) + + Context("Test Load", func() { + BeforeEach(func() { + cache.Recover() + cache.cache = map[string]keySet{ + "kubelet": { + m: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, + } + cache.flush() + }) + It("should return nil,false if component is not in cache", func() { + c, found := cache.Load("kube-proxy") + Expect(c.m).To(BeNil()) + Expect(found).To(BeFalse()) + }) + It("should return keyset,true if component is in cache", func() { + c, found := cache.Load("kubelet") + Expect(c.m).To(Equal(map[storageKey]struct{}{ + key1: {}, + key2: {}, + })) + Expect(found).To(BeTrue()) + }) + }) + + Context("Test LoadAndDelete", func() { + BeforeEach(func() { + cache.Recover() + cache.cache = map[string]keySet{ + "kubelet": { + m: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, + "kube-proxy": { + m: map[storageKey]struct{}{ + key3: {}, + }, + }, + } + cache.flush() + }) + It("should return nil,false if component is not in cache", func() { + c, found := cache.LoadAndDelete("foo") + Expect(c.m).To(BeNil()) + Expect(found).To(BeFalse()) + }) + It("should return keyset,true and delete cache for this component if exists", func() { + c, found := cache.LoadAndDelete("kubelet") + Expect(c.m).To(Equal(map[storageKey]struct{}{ + key1: {}, + key2: {}, + })) + Expect(found).To(BeTrue()) + Expect(cache.cache).To(Equal(map[string]keySet{ + "kube-proxy": { + m: map[storageKey]struct{}{ + key3: {}, + }, + }, + })) + data, err := os.ReadFile(cache.filePath) + Expect(err).To(BeNil()) + Expect(data).To(Equal([]byte( + "kube-proxy:" + key3.path, + ))) + }) + }) + Context("Test LoadOrStore", func() { + BeforeEach(func() { + cache.Recover() + cache.cache = map[string]keySet{ + "kubelet": { + m: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, + } + cache.flush() + }) + It("should return data,false and store data if component currently does not in cache", func() { + c, found := cache.LoadOrStore("kube-proxy", keySet{ + m: map[storageKey]struct{}{ + key3: {}, + }, + }) + Expect(found).To(BeFalse()) + Expect(c.m).To(Equal(map[storageKey]struct{}{ + key3: {}, + })) + buf, err := os.ReadFile(cache.filePath) + Expect(err).To(BeNil()) + Expect(strings.Split(string(buf), "\n")).To(HaveLen(2)) + }) + It("should return original data and true if component already exists in cache", func() { + c, found := cache.LoadOrStore("kubelet", keySet{ + m: map[storageKey]struct{}{ + key3: {}, + }, + }) + Expect(found).To(BeTrue()) + Expect(c.m).To(Equal(map[storageKey]struct{}{ + key1: {}, + key2: {}, + })) + buf, err := os.ReadFile(cache.filePath) + Expect(err).To(BeNil()) + Expect(strings.Split(string(buf), "\n")).To(HaveLen(1)) + }) + }) +}) + +func TestKeySetDifference(t *testing.T) { + key1 := storageKey{path: "/registry/pods/test/test-pod"} + key2 := storageKey{path: "/registry/pods/test/test-pod2"} + key3 := storageKey{path: "/registry/pods/test/test-pod3"} + cases := []struct { + description string + s1 keySet + s2 keySet + want map[storageKey]struct{} + }{ + { + description: "s2 is nil", + s1: keySet{ + m: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, + s2: keySet{ + m: nil, + }, + want: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, { + description: "s2 is empty", + s1: keySet{ + m: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, + s2: keySet{ + m: map[storageKey]struct{}{}, + }, + want: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, + { + description: "s1 is empty", + s1: keySet{ + m: map[storageKey]struct{}{}, + }, + s2: keySet{ + m: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, + want: map[storageKey]struct{}{}, + }, + { + description: "s1 has intersection with s2", + s1: keySet{ + m: map[storageKey]struct{}{ + key1: {}, + key2: {}, + }, + }, + s2: keySet{ + m: map[storageKey]struct{}{ + key2: {}, + key3: {}, + }, + }, + want: map[storageKey]struct{}{ + key1: {}, + }, + }, + } + + for _, c := range cases { + got := c.s1.Difference(c.s2) + if len(got) != len(c.want) { + t.Errorf("unexpected num of keys at case %s, got: %d, want: %d", c.description, len(got), len(c.want)) + } + gotm := map[storageKey]struct{}{} + for _, k := range got { + gotm[k] = struct{}{} + } + + if !reflect.DeepEqual(gotm, c.want) { + t.Errorf("failed at case %s, got: %v, want: %v", c.description, got, c.want) + } + } +} diff --git a/pkg/yurthub/storage/etcd/storage.go b/pkg/yurthub/storage/etcd/storage.go new file mode 100644 index 00000000000..fe0b9503b1f --- /dev/null +++ b/pkg/yurthub/storage/etcd/storage.go @@ -0,0 +1,500 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 etcd + +import ( + "context" + "fmt" + "path/filepath" + "strings" + "time" + + "go.etcd.io/etcd/client/pkg/v3/transport" + clientv3 "go.etcd.io/etcd/client/v3" + healthpb "google.golang.org/grpc/health/grpc_health_v1" + "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/client-go/kubernetes/scheme" + "k8s.io/klog/v2" + + "github.com/openyurtio/openyurt/pkg/yurthub/storage" + "github.com/openyurtio/openyurt/pkg/yurthub/storage/utils" +) + +const ( + StorageName = "pool-coordinator" + defaultTimeout = 5 * time.Second + defaultHealthCheckPeriod = 10 * time.Second + defaultDialTimeout = 10 * time.Second + defaultComponentCacheFileName = "component-key-cache" + defaultRvLen = 32 +) + +type pathType string + +var ( + rvType pathType = "rv" +) + +type EtcdStorageConfig struct { + Prefix string + EtcdEndpoints []string + CertFile string + KeyFile string + CaFile string + LocalCacheDir string +} + +// TODO: consider how to recover the work if it was interrupted because of restart, in +// which case we've added/deleted key in local cache but failed to add/delete it in etcd. +type etcdStorage struct { + ctx context.Context + prefix string + mirrorPrefixMap map[pathType]string + client *clientv3.Client + clientConfig clientv3.Config + // localComponentKeyCache persistently records keys owned by different components + // It's useful to recover previous state when yurthub restarts. + // We need this cache at local host instead of in etcd, because we need to ensure each + // operation on etcd is atomic. If we store it in etcd, we have to get it first and then + // do the action, such as ReplaceComponentList, which makes it non-atomic. + // We assume that for resources listed by components on this node consist of two kinds: + // 1. common resources: which are also used by other nodes + // 2. special resources: which are only used by this nodes + // In local cache, we do not need to bother to distinguish these two kinds. + // For special resources, this node absolutely can create/update/delete them. + // For common resources, thanks to list/watch we can ensure that resources in pool-coordinator + // are finally consistent with the cloud, though there maybe a little jitter. + localComponentKeyCache *componentKeyCache + // For etcd storage, we do not need to cache cluster info, because + // we can get it form apiserver in pool-coordinator. + doNothingAboutClusterInfo +} + +func NewStorage(ctx context.Context, cfg *EtcdStorageConfig) (storage.Store, error) { + cacheFilePath := filepath.Join(cfg.LocalCacheDir, defaultComponentCacheFileName) + cache := newComponentKeyCache(cacheFilePath) + if err := cache.Recover(); err != nil { + return nil, fmt.Errorf("failed to recover component key cache from %s, %v", cacheFilePath, err) + } + + tlsInfo := transport.TLSInfo{ + CertFile: cfg.CertFile, + KeyFile: cfg.KeyFile, + TrustedCAFile: cfg.CaFile, + } + + tlsConfig, err := tlsInfo.ClientConfig() + if err != nil { + return nil, fmt.Errorf("failed to create tls config for etcd client, %v", err) + } + + clientConfig := clientv3.Config{ + Endpoints: cfg.EtcdEndpoints, + TLS: tlsConfig, + DialTimeout: defaultDialTimeout, + } + + client, err := clientv3.New(clientConfig) + if err != nil { + return nil, fmt.Errorf("failed to create etcd client, %v", err) + } + + s := &etcdStorage{ + ctx: ctx, + prefix: cfg.Prefix, + client: client, + clientConfig: clientConfig, + localComponentKeyCache: cache, + mirrorPrefixMap: map[pathType]string{ + rvType: "/mirror/rv", + }, + } + + go s.clientLifeCycleManagement() + return s, nil +} + +func (s *etcdStorage) mirrorPath(path string, pathType pathType) string { + return filepath.Join(s.mirrorPrefixMap[pathType], path) +} + +func (s *etcdStorage) Name() string { + return StorageName +} + +func (s *etcdStorage) clientLifeCycleManagement() { + reconnect := func(ctx context.Context) { + t := time.NewTicker(5 * time.Second) + for { + select { + case <-ctx.Done(): + return + case <-t.C: + if client, err := clientv3.New(s.clientConfig); err == nil { + klog.Infof("client reconnected to etcd server, %s", client.ActiveConnection().GetState().String()) + if err := s.client.Close(); err != nil { + klog.Errorf("failed to close old client, %v", err) + } + s.client = client + return + } + continue + } + } + } + + for { + select { + case <-s.ctx.Done(): + klog.Info("etcdstorage lifecycle routine exited") + return + default: + timeoutCtx, cancel := context.WithTimeout(s.ctx, defaultDialTimeout) + healthCli := healthpb.NewHealthClient(s.client.ActiveConnection()) + resp, err := healthCli.Check(timeoutCtx, &healthpb.HealthCheckRequest{}) + // We should call cancel in case Check request does not timeout, to release resource. + cancel() + if err != nil { + klog.Errorf("check health of etcd failed, err: %v, try to reconnect", err) + reconnect(s.ctx) + } else if resp != nil && resp.Status != healthpb.HealthCheckResponse_SERVING { + klog.Errorf("unexpected health status from etcd, status: %s", resp.Status.String()) + } + time.Sleep(defaultHealthCheckPeriod) + } + } +} + +func (s *etcdStorage) Create(key storage.Key, content []byte) error { + if err := utils.ValidateKV(key, content, storageKey{}); err != nil { + return err + } + + keyStr := key.Key() + originRv, err := getRvOfObject(content) + if err != nil { + return fmt.Errorf("failed to get rv from content when creating %s, %v", keyStr, err) + } + + ctx, cancel := context.WithTimeout(s.ctx, defaultTimeout) + defer cancel() + txnResp, err := s.client.KV.Txn(ctx).If( + notFound(keyStr), + ).Then( + clientv3.OpPut(keyStr, string(content)), + clientv3.OpPut(s.mirrorPath(keyStr, rvType), fixLenRvString(originRv)), + ).Commit() + + if err != nil { + return err + } + + if !txnResp.Succeeded { + return storage.ErrKeyExists + } + + storageKey := key.(storageKey) + s.localComponentKeyCache.AddKey(storageKey.component(), storageKey) + return nil +} + +func (s *etcdStorage) Delete(key storage.Key) error { + if err := utils.ValidateKey(key, storageKey{}); err != nil { + return err + } + + keyStr := key.Key() + ctx, cancel := context.WithTimeout(s.ctx, defaultTimeout) + defer cancel() + _, err := s.client.Txn(ctx).If().Then( + clientv3.OpDelete(keyStr), + clientv3.OpDelete(s.mirrorPath(keyStr, rvType)), + ).Commit() + if err != nil { + return err + } + + storageKey := key.(storageKey) + s.localComponentKeyCache.DeleteKey(storageKey.component(), storageKey) + return nil +} + +func (s *etcdStorage) Get(key storage.Key) ([]byte, error) { + if err := utils.ValidateKey(key, storageKey{}); err != nil { + return nil, err + } + + keyStr := key.Key() + ctx, cancel := context.WithTimeout(s.ctx, defaultTimeout) + defer cancel() + getResp, err := s.client.Get(ctx, keyStr) + if err != nil { + return nil, err + } + if len(getResp.Kvs) == 0 { + return nil, storage.ErrStorageNotFound + } + + return getResp.Kvs[0].Value, nil +} + +// TODO: When using etcd, do we have the case: +// "If the rootKey exists in the store but no keys has the prefix of rootKey"? +func (s *etcdStorage) List(key storage.Key) ([][]byte, error) { + if err := utils.ValidateKey(key, storageKey{}); err != nil { + return [][]byte{}, err + } + + rootKeyStr := key.Key() + ctx, cancel := context.WithTimeout(s.ctx, defaultTimeout) + defer cancel() + getResp, err := s.client.Get(ctx, rootKeyStr, clientv3.WithPrefix()) + if err != nil { + return nil, err + } + if len(getResp.Kvs) == 0 { + return nil, storage.ErrStorageNotFound + } + + values := make([][]byte, 0, len(getResp.Kvs)) + for _, kv := range getResp.Kvs { + values = append(values, kv.Value) + } + return values, nil +} + +func (s *etcdStorage) Update(key storage.Key, content []byte, rv uint64) ([]byte, error) { + if err := utils.ValidateKV(key, content, storageKey{}); err != nil { + return nil, err + } + + keyStr := key.Key() + ctx, cancel := context.WithTimeout(s.ctx, defaultTimeout) + defer cancel() + txnResp, err := s.client.KV.Txn(ctx).If( + found(keyStr), + fresherThan(fixLenRvUint64(rv), s.mirrorPath(keyStr, rvType)), + ).Then( + clientv3.OpPut(keyStr, string(content)), + clientv3.OpPut(s.mirrorPath(keyStr, rvType), fixLenRvUint64(rv)), + ).Else( + // Possibly we have two cases here: + // 1. key does not exist + // 2. key exists with a higher rv + // We can distinguish them by OpGet. If it gets no value back, it's case 1. + // Otherwise is case 2. + clientv3.OpGet(keyStr), + ).Commit() + + if err != nil { + return nil, err + } + + if !txnResp.Succeeded { + getResp := (*clientv3.GetResponse)(txnResp.Responses[0].GetResponseRange()) + if len(getResp.Kvs) == 0 { + return nil, storage.ErrStorageNotFound + } + return getResp.Kvs[0].Value, storage.ErrUpdateConflict + } + + return content, nil +} + +func (s *etcdStorage) ListResourceKeysOfComponent(component string, gvr schema.GroupVersionResource) ([]storage.Key, error) { + if component == "" { + return nil, storage.ErrEmptyComponent + } + + rootKey, err := s.KeyFunc(storage.KeyBuildInfo{ + Component: component, + Resources: gvr.Resource, + Group: gvr.Group, + Version: gvr.Version, + }) + if err != nil { + return nil, err + } + + keys := []storage.Key{} + keyCache, ok := s.localComponentKeyCache.Load(component) + if !ok { + return nil, storage.ErrStorageNotFound + } + for k := range keyCache.m { + if strings.HasPrefix(k.Key(), rootKey.Key()) { + keys = append(keys, k) + } + } + return keys, nil +} + +func (s *etcdStorage) ReplaceComponentList(component string, gvr schema.GroupVersionResource, namespace string, contents map[storage.Key][]byte) error { + if component == "" { + return storage.ErrEmptyComponent + } + rootKey, err := s.KeyFunc(storage.KeyBuildInfo{ + Component: component, + Resources: gvr.Resource, + Group: gvr.Group, + Version: gvr.Version, + Namespace: namespace, + }) + if err != nil { + return err + } + for key := range contents { + if !strings.HasPrefix(key.Key(), rootKey.Key()) { + return storage.ErrInvalidContent + } + } + + newKeyCache := keySet{m: map[storageKey]struct{}{}} + for k := range contents { + storageKey, ok := k.(storageKey) + if !ok { + return storage.ErrUnrecognizedKey + } + newKeyCache.m[storageKey] = struct{}{} + } + var addedOrUpdated, deleted []storageKey + oldKeyCache, loaded := s.localComponentKeyCache.LoadOrStore(component, newKeyCache) + addedOrUpdated = newKeyCache.Difference(keySet{}) + if loaded { + deleted = oldKeyCache.Difference(newKeyCache) + } + + ops := []clientv3.Op{} + for _, k := range addedOrUpdated { + rv, err := getRvOfObject(contents[k]) + if err != nil { + klog.Errorf("failed to process %s in list object, %v", k.Key(), err) + continue + } + createOrUpdateOp := clientv3.OpTxn( + []clientv3.Cmp{ + // if + found(k.Key()), + }, + []clientv3.Op{ + // then + clientv3.OpTxn([]clientv3.Cmp{ + // if + fresherThan(fixLenRvString(rv), s.mirrorPath(k.Key(), rvType)), + }, []clientv3.Op{ + // then + clientv3.OpPut(k.Key(), string(contents[k])), + clientv3.OpPut(s.mirrorPath(k.Key(), rvType), fixLenRvString(rv)), + }, []clientv3.Op{ + // else + // do nothing + }), + }, + []clientv3.Op{ + // else + clientv3.OpPut(k.Key(), string(contents[k])), + clientv3.OpPut(s.mirrorPath(k.Key(), rvType), fixLenRvString(rv)), + }, + ) + ops = append(ops, createOrUpdateOp) + } + for _, k := range deleted { + ops = append(ops, + clientv3.OpDelete(k.Key()), + clientv3.OpDelete(s.mirrorPath(k.Key(), rvType)), + ) + } + + ctx, cancel := context.WithTimeout(s.ctx, defaultTimeout) + defer cancel() + _, err = s.client.Txn(ctx).If().Then(ops...).Commit() + if err != nil { + return err + } + + return nil +} + +func (s *etcdStorage) DeleteComponentResources(component string) error { + if component == "" { + return storage.ErrEmptyComponent + } + keyCache, loaded := s.localComponentKeyCache.LoadAndDelete(component) + if !loaded { + // no need to delete + return nil + } + + ops := []clientv3.Op{} + for k := range keyCache.m { + ops = append(ops, + clientv3.OpDelete(k.Key()), + clientv3.OpDelete(s.mirrorPath(k.Key(), rvType)), + ) + } + + ctx, cancel := context.WithTimeout(s.ctx, defaultTimeout) + defer cancel() + _, err := s.client.Txn(ctx).If().Then(ops...).Commit() + if err != nil { + return err + } + return nil +} + +func fixLenRvUint64(rv uint64) string { + return fmt.Sprintf("%0*d", defaultRvLen, rv) +} + +func fixLenRvString(rv string) string { + return fmt.Sprintf("%0*s", defaultRvLen, rv) +} + +// TODO: do not get rv through decoding, which means we have to +// unmarshal bytes. We should not do any serialization in storage. +func getRvOfObject(object []byte) (string, error) { + decoder := scheme.Codecs.UniversalDeserializer() + unstructuredObj := new(unstructured.Unstructured) + _, _, err := decoder.Decode(object, nil, unstructuredObj) + if err != nil { + return "", err + } + + return unstructuredObj.GetResourceVersion(), nil +} + +func notFound(key string) clientv3.Cmp { + return clientv3.Compare(clientv3.ModRevision(key), "=", 0) +} + +func found(key string) clientv3.Cmp { + return clientv3.Compare(clientv3.ModRevision(key), ">", 0) +} + +func fresherThan(rv string, key string) clientv3.Cmp { + return clientv3.Compare(clientv3.Value(key), "<", rv) +} + +type doNothingAboutClusterInfo struct{} + +func (d doNothingAboutClusterInfo) SaveClusterInfo(_ storage.ClusterInfoKey, _ []byte) error { + return nil +} +func (d doNothingAboutClusterInfo) GetClusterInfo(_ storage.ClusterInfoKey) ([]byte, error) { + return nil, nil +} diff --git a/pkg/yurthub/storage/etcd/storage_test.go b/pkg/yurthub/storage/etcd/storage_test.go new file mode 100644 index 00000000000..ea3d0e3952c --- /dev/null +++ b/pkg/yurthub/storage/etcd/storage_test.go @@ -0,0 +1,548 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 etcd + +import ( + "context" + "encoding/json" + "os" + "os/exec" + "path/filepath" + "time" + + "github.com/google/uuid" + . "github.com/onsi/ginkgo/v2" + . "github.com/onsi/gomega" + v1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime/schema" + + "github.com/openyurtio/openyurt/pkg/yurthub/storage" +) + +var _ = Describe("Test EtcdStorage", func() { + var etcdstore *etcdStorage + var key1 storage.Key + var podObj *v1.Pod + var podJson []byte + var ctx context.Context + BeforeEach(func() { + ctx = context.Background() + randomize := uuid.New().String() + cfg := &EtcdStorageConfig{ + Prefix: "/" + randomize, + EtcdEndpoints: []string{"127.0.0.1:2379"}, + LocalCacheDir: filepath.Join(keyCacheDir, randomize), + } + s, err := NewStorage(context.Background(), cfg) + Expect(err).To(BeNil()) + etcdstore = s.(*etcdStorage) + key1, err = etcdstore.KeyFunc(storage.KeyBuildInfo{ + Component: "kubelet", + Resources: "pods", + Group: "", + Version: "v1", + Namespace: "default", + Name: "pod1-" + randomize, + }) + Expect(err).To(BeNil()) + podObj = &v1.Pod{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "v1", + Kind: "Pod", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "pod1-" + randomize, + Namespace: "default", + ResourceVersion: "890", + }, + } + podJson, err = json.Marshal(podObj) + Expect(err).To(BeNil()) + }) + + Context("Test Lifecycle", Focus, func() { + It("should reconnect to etcd if connect once broken", Focus, func() { + Expect(etcdstore.Create(key1, podJson)).Should(BeNil()) + Expect(etcdCmd.Process.Kill()).To(BeNil()) + By("waiting for the etcd exited") + Eventually(func() bool { + _, err := etcdstore.Get(key1) + return err != nil + }, 10*time.Second, 1*time.Second).Should(BeTrue()) + + devNull, err := os.OpenFile("/dev/null", os.O_RDWR, 0755) + Expect(err).To(BeNil()) + etcdCmd = exec.Command("/usr/local/etcd/etcd", "--data-dir="+etcdDataDir) + etcdCmd.Stdout = devNull + etcdCmd.Stderr = devNull + Expect(etcdCmd.Start()).To(BeNil()) + By("waiting for storage function recovery") + Eventually(func() bool { + if _, err := etcdstore.Get(key1); err != nil { + return false + } + return true + }, 30*time.Second, 500*time.Microsecond).Should(BeTrue()) + }) + }) + + Context("Test Create", func() { + It("should return ErrKeyIsEmpty if key is nil", func() { + Expect(etcdstore.Create(nil, []byte("foo"))).To(Equal(storage.ErrKeyIsEmpty)) + Expect(etcdstore.Create(storageKey{}, []byte("foo"))).To(Equal(storage.ErrKeyIsEmpty)) + }) + It("should return ErrKeyHasNoContent if content is empty", func() { + Expect(etcdstore.Create(key1, []byte{})).To(Equal(storage.ErrKeyHasNoContent)) + Expect(etcdstore.Create(key1, nil)).To(Equal(storage.ErrKeyHasNoContent)) + }) + It("should return ErrKeyExists if key already exists in etcd", func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + Expect(etcdstore.Create(key1, podJson)).To(Equal(storage.ErrKeyExists)) + }) + It("should create key with content in etcd", func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + resp, err := etcdstore.client.Get(ctx, key1.Key()) + Expect(err).To(BeNil()) + Expect(resp.Kvs).To(HaveLen(1)) + Expect(resp.Kvs[0].Value).To(Equal([]byte(podJson))) + resp, err = etcdstore.client.Get(ctx, etcdstore.mirrorPath(key1.Key(), rvType)) + Expect(err).To(BeNil()) + Expect(resp.Kvs).To(HaveLen(1)) + Expect(resp.Kvs[0].Value).To(Equal([]byte(fixLenRvString(podObj.ResourceVersion)))) + }) + }) + + Context("Test Delete", func() { + It("should return ErrKeyIsEmpty if key is nil", func() { + Expect(etcdstore.Delete(nil)).To(Equal(storage.ErrKeyIsEmpty)) + Expect(etcdstore.Delete(storageKey{})).To(Equal(storage.ErrKeyIsEmpty)) + }) + It("should delete key from etcd if it exists", func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + Expect(etcdstore.Delete(key1)).To(BeNil()) + resp, err := etcdstore.client.Get(ctx, key1.Key()) + Expect(err).To(BeNil()) + Expect(resp.Kvs).To(HaveLen(0)) + resp, err = etcdstore.client.Get(ctx, etcdstore.mirrorPath(key1.Key(), rvType)) + Expect(err).To(BeNil()) + Expect(resp.Kvs).To(HaveLen(0)) + }) + It("should not return error if key does not exist in etcd", func() { + Expect(etcdstore.Delete(key1)).To(BeNil()) + }) + }) + + Context("Test Get", func() { + It("should return ErrKeyIsEmpty if key is nil", func() { + _, err := etcdstore.Get(nil) + Expect(err).To(Equal(storage.ErrKeyIsEmpty)) + _, err = etcdstore.Get(storageKey{}) + Expect(err).To(Equal(storage.ErrKeyIsEmpty)) + }) + It("should return ErrStorageNotFound if key does not exist in etcd", func() { + _, err := etcdstore.Get(key1) + Expect(err).To(Equal(storage.ErrStorageNotFound)) + }) + It("should return content of key if it exists in etcd", func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + buf, err := etcdstore.Get(key1) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(podJson)) + }) + }) + + Context("Test List", func() { + var err error + var podsRootKey storage.Key + BeforeEach(func() { + podsRootKey, err = etcdstore.KeyFunc(storage.KeyBuildInfo{ + Component: "kubelet", + Resources: "pods", + }) + }) + It("should return ErrKeyIsEmpty if key is nil", func() { + _, err = etcdstore.List(nil) + Expect(err).To(Equal(storage.ErrKeyIsEmpty)) + _, err = etcdstore.List(storageKey{}) + Expect(err).To(Equal(storage.ErrKeyIsEmpty)) + }) + It("should return ErrStorageNotFound if key does not exist in etcd", func() { + _, err = etcdstore.List(podsRootKey) + Expect(err).To(Equal(storage.ErrStorageNotFound)) + _, err = etcdstore.List(key1) + Expect(err).To(Equal(storage.ErrStorageNotFound)) + }) + It("should return a single resource if key points to a specific resource", func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + buf, err := etcdstore.List(key1) + Expect(err).To(BeNil()) + Expect(buf).To(Equal([][]byte{podJson})) + }) + It("should return a list of resources if its is a root key", func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + info := storage.KeyBuildInfo{ + Component: "kubelet", + Resources: "pods", + Group: "", + Version: "v1", + Namespace: "default", + } + + info.Name = "pod2" + key2, err := etcdstore.KeyFunc(info) + Expect(err).To(BeNil()) + pod2Obj := podObj.DeepCopy() + pod2Obj.Name = "pod2" + pod2Json, err := json.Marshal(pod2Obj) + Expect(err).To(BeNil()) + Expect(etcdstore.Create(key2, pod2Json)).To(BeNil()) + + info.Name = "pod3" + info.Namespace = "kube-system" + key3, err := etcdstore.KeyFunc(info) + Expect(err).To(BeNil()) + pod3Obj := podObj.DeepCopy() + pod3Obj.Name = "pod3" + pod3Obj.Namespace = "kube-system" + pod3Json, err := json.Marshal(pod3Obj) + Expect(err).To(BeNil()) + Expect(etcdstore.Create(key3, pod3Json)).To(BeNil()) + + buf, err := etcdstore.List(podsRootKey) + Expect(err).To(BeNil()) + Expect(buf).To(HaveLen(len(buf))) + Expect(buf).To(ContainElements([][]byte{podJson, pod2Json, pod3Json})) + + namespacesRootKey, _ := etcdstore.KeyFunc(storage.KeyBuildInfo{ + Component: "kubelet", + Resources: "pods", + Namespace: "default", + }) + buf, err = etcdstore.List(namespacesRootKey) + Expect(err).To(BeNil()) + Expect(buf).To(ContainElements([][]byte{podJson, pod2Json})) + }) + }) + + Context("Test Update", func() { + It("should return ErrKeyIsEmpty if key is nil", func() { + _, err := etcdstore.Update(nil, []byte("foo"), 100) + Expect(err).To(Equal(storage.ErrKeyIsEmpty)) + _, err = etcdstore.Update(storageKey{}, []byte("foo"), 100) + Expect(err).To(Equal(storage.ErrKeyIsEmpty)) + }) + It("should return ErrStorageNotFound if key does not exist in etcd", func() { + _, err := etcdstore.Update(key1, podJson, 890) + Expect(err).To(Equal(storage.ErrStorageNotFound)) + }) + It("should return resource in etcd and ErrUpdateConflict if the provided resource has staler rv than resource in etcd", func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + podObj.ResourceVersion = "100" + podObj.Labels = map[string]string{ + "new": "label", + } + newPodJson, err := json.Marshal(podObj) + Expect(err).To(BeNil()) + stored, err := etcdstore.Update(key1, newPodJson, 100) + Expect(err).To(Equal(storage.ErrUpdateConflict)) + Expect(stored).To(Equal(podJson)) + }) + It("should update resource in etcd and return the stored resource", func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + + podObj.ResourceVersion = "900" + podObj.Labels = map[string]string{ + "rv": "900", + } + newPodJson, err := json.Marshal(podObj) + Expect(err).To(BeNil()) + stored, err := etcdstore.Update(key1, newPodJson, 900) + Expect(err).To(BeNil()) + Expect(stored).To(Equal(newPodJson)) + + podObj.ResourceVersion = "1000" + podObj.Labels = map[string]string{ + "rv": "1000", + } + newPodJson, err = json.Marshal(podObj) + Expect(err).To(BeNil()) + stored, err = etcdstore.Update(key1, newPodJson, 1000) + Expect(err).To(BeNil()) + Expect(stored).To(Equal(newPodJson)) + }) + }) + + Context("Test ComponentRelatedInterface", func() { + var cmKey, key2, key3 storage.Key + var cmObj *v1.ConfigMap + var pod2Obj, pod3Obj *v1.Pod + var cmJson, pod2Json, pod3Json []byte + var gvr schema.GroupVersionResource + var err error + BeforeEach(func() { + info := storage.KeyBuildInfo{ + Component: "kubelet", + Resources: "pods", + Group: "", + Version: "v1", + } + gvr = schema.GroupVersionResource{ + Group: "", + Version: "v1", + Resource: "pods", + } + + info.Namespace = "default" + info.Name = "pod2" + key2, _ = etcdstore.KeyFunc(info) + info.Namespace = "kube-system" + info.Name = "pod3" + key3, _ = etcdstore.KeyFunc(info) + cmKey, _ = etcdstore.KeyFunc(storage.KeyBuildInfo{ + Group: "", + Resources: "configmaps", + Version: "v1", + Namespace: "default", + Name: "cm", + Component: "kubelet", + }) + + pod2Obj = podObj.DeepCopy() + pod2Obj.Namespace = "default" + pod2Obj.Name = "pod2" + pod2Obj.ResourceVersion = "920" + pod3Obj = podObj.DeepCopy() + pod3Obj.Namespace = "kube-system" + pod3Obj.Name = "pod3" + pod3Obj.ResourceVersion = "930" + cmObj = &v1.ConfigMap{ + ObjectMeta: metav1.ObjectMeta{ + Name: "cm", + Namespace: "default", + }, + TypeMeta: metav1.TypeMeta{ + Kind: "ConfigMap", + APIVersion: "v1", + }, + Data: map[string]string{ + "foo": "bar", + }, + } + + pod2Json, err = json.Marshal(pod2Obj) + Expect(err).To(BeNil()) + pod3Json, err = json.Marshal(pod3Obj) + Expect(err).To(BeNil()) + cmJson, err = json.Marshal(cmObj) + Expect(err).To(BeNil()) + }) + Context("Test ListResourceKeysOfComponent", func() { + It("should return ErrEmptyComponent if component is empty", func() { + _, err = etcdstore.ListResourceKeysOfComponent("", gvr) + Expect(err).To(Equal(storage.ErrEmptyComponent)) + }) + It("should return ErrEmptyResource if resource of gvr is empty", func() { + _, err = etcdstore.ListResourceKeysOfComponent("kubelet", schema.GroupVersionResource{ + Resource: "", + Version: "v1", + Group: "", + }) + Expect(err).To(Equal(storage.ErrEmptyResource)) + }) + It("should return ErrStorageNotFound if this component has no cache", func() { + _, err = etcdstore.ListResourceKeysOfComponent("flannel", gvr) + Expect(err).To(Equal(storage.ErrStorageNotFound)) + }) + It("should return all keys of gvr if cache of this component is found", func() { + By("creating objects in cache") + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + Expect(etcdstore.Create(key3, pod3Json)).To(BeNil()) + Expect(etcdstore.Create(cmKey, cmJson)).To(BeNil()) + + keys, err := etcdstore.ListResourceKeysOfComponent("kubelet", schema.GroupVersionResource{ + Group: "", + Version: "v1", + Resource: "pods", + }) + Expect(err).To(BeNil()) + Expect(keys).To(HaveLen(2)) + Expect(keys).To(ContainElements(key1, key3)) + }) + }) + + Context("Test ReplaceComponentList", func() { + BeforeEach(func() { + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + Expect(etcdstore.Create(key2, pod2Json)).To(BeNil()) + Expect(etcdstore.Create(key3, pod3Json)).To(BeNil()) + }) + It("should return ErrEmptyComponent if component is empty", func() { + Expect(etcdstore.ReplaceComponentList("", gvr, "", map[storage.Key][]byte{})).To(Equal(storage.ErrEmptyComponent)) + }) + It("should return ErrEmptyResource if resource of gvr is empty", func() { + gvr.Resource = "" + Expect(etcdstore.ReplaceComponentList("kubelet", gvr, "", map[storage.Key][]byte{})).To(Equal(storage.ErrEmptyResource)) + }) + It("should return ErrInvalidContent if it exists keys that are not passed-in gvr", func() { + invalidKey, err := etcdstore.KeyFunc(storage.KeyBuildInfo{ + Component: "kubelet", + Resources: "configmaps", + Group: "", + Version: "v1", + Namespace: "default", + Name: "cm", + }) + Expect(err).To(BeNil()) + Expect(etcdstore.ReplaceComponentList("kubelet", gvr, "default", map[storage.Key][]byte{ + key2: pod2Json, + invalidKey: {}, + })).To(Equal(storage.ErrInvalidContent)) + + invalidKey, err = etcdstore.KeyFunc(storage.KeyBuildInfo{ + Component: "kubelet", + Resources: "pods", + Group: "", + Version: "v1", + Namespace: "kube-system", + Name: "pod4", + }) + Expect(err).To(BeNil()) + Expect(etcdstore.ReplaceComponentList("kubelet", gvr, "default", map[storage.Key][]byte{ + key2: pod2Json, + key3: pod3Json, + invalidKey: {}, + })).To(Equal(storage.ErrInvalidContent)) + }) + It("should only use fresher resources in contents to update cache in etcd", func() { + pod2Obj.ResourceVersion = "921" + newPod2Json, err := json.Marshal(pod2Obj) + Expect(err).To(BeNil()) + pod3Obj.ResourceVersion = "1001" // case of different len(ResourceVersion) + newPod3Json, err := json.Marshal(pod3Obj) + Expect(err).To(BeNil()) + Expect(etcdstore.ReplaceComponentList("kubelet", gvr, "", map[storage.Key][]byte{ + key1: podJson, + key2: newPod2Json, + key3: newPod3Json, + })).To(BeNil()) + + buf, err := etcdstore.Get(key1) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(podJson)) + buf, err = etcdstore.Get(key2) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(newPod2Json)) + buf, err = etcdstore.Get(key3) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(newPod3Json)) + }) + It("should create resource if it does not in etcd", func() { + key4, _ := etcdstore.KeyFunc(storage.KeyBuildInfo{ + Component: "kubelet", + Resources: "pods", + Version: "v1", + Group: "", + Namespace: "default", + Name: "pod4", + }) + pod4Obj := podObj.DeepCopy() + pod4Obj.ResourceVersion = "940" + pod4Obj.Name = "pod4" + pod4Json, err := json.Marshal(pod4Obj) + Expect(err).To(BeNil()) + Expect(etcdstore.ReplaceComponentList("kubelet", gvr, "", map[storage.Key][]byte{ + key1: podJson, + key2: pod2Json, + key3: pod3Json, + key4: pod4Json, + })).To(BeNil()) + + buf, err := etcdstore.Get(key1) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(podJson)) + buf, err = etcdstore.Get(key2) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(pod2Json)) + buf, err = etcdstore.Get(key3) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(pod3Json)) + buf, err = etcdstore.Get(key4) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(pod4Json)) + }) + It("should delete resources in etcd if they were in local cache but are not in current contents", func() { + Expect(etcdstore.ReplaceComponentList("kubelet", gvr, "", map[storage.Key][]byte{ + key1: podJson, + })).To(BeNil()) + buf, err := etcdstore.Get(key1) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(podJson)) + _, err = etcdstore.Get(key2) + Expect(err).To(Equal(storage.ErrStorageNotFound)) + _, err = etcdstore.Get(key3) + Expect(err).To(Equal(storage.ErrStorageNotFound)) + }) + }) + + Context("Test DeleteComponentResources", func() { + It("should return ErrEmptyComponent if component is empty", func() { + Expect(etcdstore.DeleteComponentResources("")).To(Equal(storage.ErrEmptyComponent)) + }) + It("should not return err even there is no cache of component", func() { + Expect(etcdstore.DeleteComponentResources("flannel")).To(BeNil()) + }) + It("should delete cache of component from local cache and etcd", func() { + Expect(etcdstore.Create(cmKey, cmJson)).To(BeNil()) + Expect(etcdstore.Create(key1, podJson)).To(BeNil()) + Expect(etcdstore.Create(key3, pod3Json)).To(BeNil()) + keys := []storage.Key{cmKey, key1, key3} + cmKey, _ = etcdstore.KeyFunc(storage.KeyBuildInfo{ + Component: "kube-proxy", + Resources: "configmaps", + Group: "", + Version: "v1", + Namespace: "default", + Name: "cm-kube-proxy", + }) + cmObj.Name = "cm-kube-proxy" + cmJson, err = json.Marshal(cmObj) + Expect(err).To(BeNil()) + Expect(etcdstore.Create(cmKey, cmJson)).To(BeNil()) + + Expect(etcdstore.DeleteComponentResources("kubelet")).To(BeNil()) + for _, k := range keys { + _, err := etcdstore.Get(k) + Expect(err).To(Equal(storage.ErrStorageNotFound)) + } + buf, err := etcdstore.Get(cmKey) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(cmJson)) + + _, found := etcdstore.localComponentKeyCache.Load("kubelet") + Expect(found).To(BeFalse()) + keyset, found := etcdstore.localComponentKeyCache.Load("kube-proxy") + Expect(found).To(BeTrue()) + Expect(keyset).To(Equal(keySet{ + m: map[storageKey]struct{}{ + cmKey.(storageKey): {}, + }, + })) + }) + }) + }) +}) diff --git a/pkg/yurthub/storage/store.go b/pkg/yurthub/storage/store.go index f0126a8669b..6aad24574e8 100644 --- a/pkg/yurthub/storage/store.go +++ b/pkg/yurthub/storage/store.go @@ -61,6 +61,7 @@ type objectRelatedHandler interface { // Create will create content of key in the store. // The key must indicate a specific resource. // If key is empty, ErrKeyIsEmpty will be returned. + // If content is empty, either nil or []byte{}, ErrKeyHasNoContent will be returned. // If this key has already existed in this store, ErrKeyExists will be returned. Create(key Key, content []byte) error @@ -78,7 +79,7 @@ type objectRelatedHandler interface { // List will retrieve all contents whose keys have the prefix of rootKey. // If key is empty, ErrKeyIsEmpty will be returned. // If the key does not exist in the store, ErrStorageNotFound will be returned. - // If the key exists in the store but no other keys has it as prefix, an empty slice + // If the key exists in the store but no other keys having it as prefix, an empty slice // of content will be returned. List(key Key) ([][]byte, error) @@ -88,7 +89,7 @@ type objectRelatedHandler interface { // The key must indicate a specific resource. // If key is empty, ErrKeyIsEmpty will be returned. // If the key does not exist in the store, ErrStorageNotFound will be returned. - // If force is not set and the rv is staler than what is in the store, ErrUpdateConflict will be returned. + // If rv is staler than what is in the store, ErrUpdateConflict will be returned. Update(key Key, contents []byte, rv uint64) ([]byte, error) // KeyFunc will generate the key used by this store. diff --git a/pkg/yurthub/storage/utils/validate.go b/pkg/yurthub/storage/utils/validate.go index 32ba6301b62..3ba8ae3e24a 100644 --- a/pkg/yurthub/storage/utils/validate.go +++ b/pkg/yurthub/storage/utils/validate.go @@ -24,12 +24,12 @@ import ( // TODO: should also valid the key format func ValidateKey(key storage.Key, validKeyType interface{}) error { + if key == nil || key.Key() == "" { + return storage.ErrKeyIsEmpty + } if reflect.TypeOf(key) != reflect.TypeOf(validKeyType) { return storage.ErrUnrecognizedKey } - if key.Key() == "" { - return storage.ErrKeyIsEmpty - } return nil } diff --git a/pkg/yurthub/util/util.go b/pkg/yurthub/util/util.go index e99af4d9f75..e6ae72bedb4 100644 --- a/pkg/yurthub/util/util.go +++ b/pkg/yurthub/util/util.go @@ -59,6 +59,8 @@ const ( ProxyReqCanCache // ProxyListSelector represents label selector and filed selector string for list request ProxyListSelector + // ProxyPoolScopedResource represents if this request is asking for pool-scoped resources + ProxyPoolScopedResource YurtHubNamespace = "kube-system" CacheUserAgentsKey = "cache_agents" @@ -132,6 +134,19 @@ func ListSelectorFrom(ctx context.Context) (string, bool) { return info, ok } +// WithIfPoolScopedResource returns a copy of parent in which IfPoolScopedResource is set, +// indicating whether this request is asking for pool-scoped resources. +func WithIfPoolScopedResource(parent context.Context, ifPoolScoped bool) context.Context { + return WithValue(parent, ProxyPoolScopedResource, ifPoolScoped) +} + +// IfPoolScopedResourceFrom returns the value of IfPoolScopedResource indicating whether this request +// is asking for pool-scoped resource. +func IfPoolScopedResourceFrom(ctx context.Context) (bool, bool) { + info, ok := ctx.Value(ProxyPoolScopedResource).(bool) + return info, ok +} + // ReqString formats a string for request func ReqString(req *http.Request) string { ctx := req.Context() @@ -168,6 +183,84 @@ func WriteObject(statusCode int, obj runtime.Object, w http.ResponseWriter, req return fmt.Errorf("request info is not found when write object, %s", ReqString(req)) } +func NewTripleReadCloser(req *http.Request, rc io.ReadCloser, isRespBody bool) (io.ReadCloser, io.ReadCloser, io.ReadCloser) { + pr1, pw1 := io.Pipe() + pr2, pw2 := io.Pipe() + tr := &tripleReadCloser{ + req: req, + rc: rc, + pw1: pw1, + pw2: pw2, + } + return tr, pr1, pr2 +} + +type tripleReadCloser struct { + req *http.Request + rc io.ReadCloser + pw1 *io.PipeWriter + pw2 *io.PipeWriter + // isRespBody shows rc(is.ReadCloser) is a response.Body + // or not(maybe a request.Body). if it is true(it's a response.Body), + // we should close the response body in Close func, else not, + // it(request body) will be closed by http request caller + isRespBody bool +} + +// Read read data into p and write into pipe +func (dr *tripleReadCloser) Read(p []byte) (n int, err error) { + defer func() { + if dr.req != nil && dr.isRespBody { + ctx := dr.req.Context() + info, _ := apirequest.RequestInfoFrom(ctx) + if info.IsResourceRequest { + comp, _ := ClientComponentFrom(ctx) + metrics.Metrics.AddProxyTrafficCollector(comp, info.Verb, info.Resource, info.Subresource, n) + } + } + }() + + n, err = dr.rc.Read(p) + if n > 0 { + var n1, n2 int + var err error + if n1, err = dr.pw1.Write(p[:n]); err != nil { + klog.Errorf("tripleReader: failed to write to pw1 %v", err) + return n1, err + } + if n2, err = dr.pw2.Write(p[:n]); err != nil { + klog.Errorf("tripleReader: failed to write to pw2 %v", err) + return n2, err + } + } + + return +} + +// Close close two readers +func (dr *tripleReadCloser) Close() error { + errs := make([]error, 0) + if dr.isRespBody { + if err := dr.rc.Close(); err != nil { + errs = append(errs, err) + } + } + + if err := dr.pw1.Close(); err != nil { + errs = append(errs, err) + } + + if err := dr.pw2.Close(); err != nil { + errs = append(errs, err) + } + + if len(errs) != 0 { + return fmt.Errorf("failed to close dualReader, %v", errs) + } + + return nil +} + // NewDualReadCloser create an dualReadCloser object func NewDualReadCloser(req *http.Request, rc io.ReadCloser, isRespBody bool) (io.ReadCloser, io.ReadCloser) { pr, pw := io.Pipe() From 0e6b9d64ff6fc8e385464dd657c89a46d4074564 Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Wed, 4 Jan 2023 15:04:16 +0800 Subject: [PATCH 03/20] generate coordinator client and pass through SubjectAccessReview according to secret (#1103) * list/watch secret to get client certs Signed-off-by: Congrool * add --enable-coordinator option to yurthub Signed-off-by: Congrool * send SubjectAccessReview to pool-coordinator or cloud apiserver according to the group Signed-off-by: Congrool * BugFix: cannot cache response Signed-off-by: Congrool * do not disable nodelifecycle controller Signed-off-by: Congrool * start informer factory after coordinator certMgr registering its secret informer Signed-off-by: Congrool --- cmd/yurthub/app/config/config.go | 26 ++- cmd/yurthub/app/options/options.go | 88 +++++---- cmd/yurthub/app/start.go | 170 +++++++++++++----- pkg/node-servant/config/control-plane.go | 32 ---- pkg/yurthub/healthchecker/health_checker.go | 2 +- .../healthchecker/health_checker_test.go | 2 +- .../certmanager/certmanager.go | 163 +++++++++++++++++ .../poolcoordinator/constants/constants.go | 4 +- pkg/yurthub/poolcoordinator/coordinator.go | 57 ++++-- .../poolcoordinator/fake_coordinator.go | 33 ++++ pkg/yurthub/proxy/pool/pool.go | 4 +- pkg/yurthub/proxy/proxy.go | 108 ++++++++--- pkg/yurthub/proxy/remote/loadbalancer.go | 16 +- pkg/yurthub/transport/transport.go | 23 ++- 14 files changed, 534 insertions(+), 194 deletions(-) create mode 100644 pkg/yurthub/poolcoordinator/certmanager/certmanager.go create mode 100644 pkg/yurthub/poolcoordinator/fake_coordinator.go diff --git a/cmd/yurthub/app/config/config.go b/cmd/yurthub/app/config/config.go index f4b20c50120..924e3549486 100644 --- a/cmd/yurthub/app/config/config.go +++ b/cmd/yurthub/app/config/config.go @@ -20,6 +20,7 @@ import ( "fmt" "net" "net/url" + "path/filepath" "strings" "time" @@ -91,15 +92,13 @@ type YurtHubConfiguration struct { YurtHubDummyProxyServerServing *apiserver.DeprecatedInsecureServingInfo YurtHubSecureProxyServerServing *apiserver.SecureServingInfo YurtHubProxyServerAddr string + ProxiedClient kubernetes.Interface DiskCachePath string CoordinatorPKIDir string EnableCoordinator bool CoordinatorServerURL *url.URL CoordinatorStoragePrefix string CoordinatorStorageAddr string // ip:port - CoordinatorStorageCaFile string - CoordinatorStorageCertFile string - CoordinatorStorageKeyFile string CoordinatorClient kubernetes.Interface LeaderElection componentbaseconfig.LeaderElectionConfiguration } @@ -111,6 +110,14 @@ func Complete(options *options.YurtHubOptions) (*YurtHubConfiguration, error) { return nil, err } + var coordinatorServerURL *url.URL + if options.EnableCoordinator { + coordinatorServerURL, err = url.Parse(options.CoordinatorServerAddr) + if err != nil { + return nil, err + } + } + storageManager, err := disk.NewDiskStorage(options.DiskCachePath) if err != nil { klog.Errorf("could not create storage manager, %v", err) @@ -125,7 +132,7 @@ func Complete(options *options.YurtHubOptions) (*YurtHubConfiguration, error) { } workingMode := util.WorkingMode(options.WorkingMode) - sharedFactory, yurtSharedFactory, err := createSharedInformers(fmt.Sprintf("http://%s:%d", options.YurtHubProxyHost, options.YurtHubProxyPort), options.EnableNodePool) + proxiedClient, sharedFactory, yurtSharedFactory, err := createClientAndSharedInformers(fmt.Sprintf("http://%s:%d", options.YurtHubProxyHost, options.YurtHubProxyPort), options.EnableNodePool) if err != nil { return nil, err } @@ -159,6 +166,7 @@ func Complete(options *options.YurtHubOptions) (*YurtHubConfiguration, error) { MinRequestTimeout: options.MinRequestTimeout, TenantNs: tenantNs, YurtHubProxyServerAddr: fmt.Sprintf("http://%s:%d", options.YurtHubProxyHost, options.YurtHubProxyPort), + ProxiedClient: proxiedClient, DiskCachePath: options.DiskCachePath, CoordinatorPKIDir: filepath.Join(options.RootDir, "poolcoordinator"), EnableCoordinator: options.EnableCoordinator, @@ -221,18 +229,18 @@ func parseRemoteServers(serverAddr string) ([]*url.URL, error) { } // createSharedInformers create sharedInformers from the given proxyAddr. -func createSharedInformers(proxyAddr string, enableNodePool bool) (informers.SharedInformerFactory, yurtinformers.SharedInformerFactory, error) { +func createClientAndSharedInformers(proxyAddr string, enableNodePool bool) (kubernetes.Interface, informers.SharedInformerFactory, yurtinformers.SharedInformerFactory, error) { var kubeConfig *rest.Config var yurtClient yurtclientset.Interface var err error kubeConfig, err = clientcmd.BuildConfigFromFlags(proxyAddr, "") if err != nil { - return nil, nil, err + return nil, nil, nil, err } client, err := kubernetes.NewForConfig(kubeConfig) if err != nil { - return nil, nil, err + return nil, nil, nil, err } fakeYurtClient := &fake.Clientset{} @@ -243,11 +251,11 @@ func createSharedInformers(proxyAddr string, enableNodePool bool) (informers.Sha if enableNodePool { yurtClient, err = yurtclientset.NewForConfig(kubeConfig) if err != nil { - return nil, nil, err + return nil, nil, nil, err } } - return informers.NewSharedInformerFactory(client, 24*time.Hour), + return client, informers.NewSharedInformerFactory(client, 24*time.Hour), yurtinformers.NewSharedInformerFactory(yurtClient, 24*time.Hour), nil } diff --git a/cmd/yurthub/app/options/options.go b/cmd/yurthub/app/options/options.go index e7d4fa16ade..ffa241b16d4 100644 --- a/cmd/yurthub/app/options/options.go +++ b/cmd/yurthub/app/options/options.go @@ -44,47 +44,46 @@ const ( // YurtHubOptions is the main settings for the yurthub type YurtHubOptions struct { - ServerAddr string - YurtHubHost string // YurtHub server host (e.g.: expose metrics API) - YurtHubProxyHost string // YurtHub proxy server host - YurtHubPort int - YurtHubProxyPort int - YurtHubProxySecurePort int - GCFrequency int - YurtHubCertOrganizations []string - NodeName string - NodePoolName string - LBMode string - HeartbeatFailedRetry int - HeartbeatHealthyThreshold int - HeartbeatTimeoutSeconds int - HeartbeatIntervalSeconds int - MaxRequestInFlight int - JoinToken string - RootDir string - Version bool - EnableProfiling bool - EnableDummyIf bool - EnableIptables bool - HubAgentDummyIfIP string - HubAgentDummyIfName string - DiskCachePath string - AccessServerThroughHub bool - EnableResourceFilter bool - DisabledResourceFilters []string - WorkingMode string - KubeletHealthGracePeriod time.Duration - EnableNodePool bool - MinRequestTimeout time.Duration - CACertHashes []string - UnsafeSkipCAVerification bool - ClientForTest kubernetes.Interface - CoordinatorStoragePrefix string - CoordinatorStorageAddr string - CoordinatorStorageCaFile string - CoordinatorStorageCertFile string - CoordinatorStorageKeyFile string - LeaderElection componentbaseconfig.LeaderElectionConfiguration + ServerAddr string + YurtHubHost string // YurtHub server host (e.g.: expose metrics API) + YurtHubProxyHost string // YurtHub proxy server host + YurtHubPort int + YurtHubProxyPort int + YurtHubProxySecurePort int + GCFrequency int + YurtHubCertOrganizations []string + NodeName string + NodePoolName string + LBMode string + HeartbeatFailedRetry int + HeartbeatHealthyThreshold int + HeartbeatTimeoutSeconds int + HeartbeatIntervalSeconds int + MaxRequestInFlight int + JoinToken string + RootDir string + Version bool + EnableProfiling bool + EnableDummyIf bool + EnableIptables bool + HubAgentDummyIfIP string + HubAgentDummyIfName string + DiskCachePath string + AccessServerThroughHub bool + EnableResourceFilter bool + DisabledResourceFilters []string + WorkingMode string + KubeletHealthGracePeriod time.Duration + EnableNodePool bool + MinRequestTimeout time.Duration + CACertHashes []string + UnsafeSkipCAVerification bool + ClientForTest kubernetes.Interface + EnableCoordinator bool + CoordinatorServerAddr string + CoordinatorStoragePrefix string + CoordinatorStorageAddr string + LeaderElection componentbaseconfig.LeaderElectionConfiguration } // NewYurtHubOptions creates a new YurtHubOptions with a default config. @@ -201,11 +200,10 @@ func (o *YurtHubOptions) AddFlags(fs *pflag.FlagSet) { fs.DurationVar(&o.MinRequestTimeout, "min-request-timeout", o.MinRequestTimeout, "An optional field indicating at least how long a proxy handler must keep a request open before timing it out. Currently only honored by the local watch request handler(use request parameter timeoutSeconds firstly), which picks a randomized value above this number as the connection timeout, to spread out load.") fs.StringSliceVar(&o.CACertHashes, "discovery-token-ca-cert-hash", o.CACertHashes, "For token-based discovery, validate that the root CA public key matches this hash (format: \":\").") fs.BoolVar(&o.UnsafeSkipCAVerification, "discovery-token-unsafe-skip-ca-verification", o.UnsafeSkipCAVerification, "For token-based discovery, allow joining without --discovery-token-ca-cert-hash pinning.") + fs.BoolVar(&o.EnableCoordinator, "enable-coordinator", o.EnableCoordinator, "make yurthub aware of the pool coordinator") + fs.StringVar(&o.CoordinatorServerAddr, "coordinator-server-addr", o.CoordinatorServerAddr, "Coordinator APIServer address in format https://host:port") fs.StringVar(&o.CoordinatorStoragePrefix, "coordinator-storage-prefix", o.CoordinatorStoragePrefix, "Pool-Coordinator etcd storage prefix, same as etcd-prefix of Kube-APIServer") - fs.StringVar(&o.CoordinatorStorageAddr, "coordinator-storage-addr", o.CoordinatorStorageAddr, "Address of Pool-Coordinator etcd, in the format ip:port") - fs.StringVar(&o.CoordinatorStorageCaFile, "coordinator-storage-ca", o.CoordinatorStorageCaFile, "CA file path to communicate with Pool-Coordinator etcd") - fs.StringVar(&o.CoordinatorStorageCertFile, "coordinator-storage-cert", o.CoordinatorStorageCertFile, "Cert file path to communicate with Pool-Coordinator etcd") - fs.StringVar(&o.CoordinatorStorageKeyFile, "coordinator-storage-key", o.CoordinatorStorageKeyFile, "Key file path to communicate with Pool-Coordinator etcd") + fs.StringVar(&o.CoordinatorStorageAddr, "coordinator-storage-addr", o.CoordinatorStorageAddr, "Address of Pool-Coordinator etcd, in the format host:port") bindFlags(&o.LeaderElection, fs) } diff --git a/cmd/yurthub/app/start.go b/cmd/yurthub/app/start.go index 00b97b1baff..e40cb88fe3f 100644 --- a/cmd/yurthub/app/start.go +++ b/cmd/yurthub/app/start.go @@ -37,6 +37,7 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" hubrest "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/rest" "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator" + coordinatorcertmgr "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/certmanager" "github.com/openyurtio/openyurt/pkg/yurthub/proxy" "github.com/openyurtio/openyurt/pkg/yurthub/server" "github.com/openyurtio/openyurt/pkg/yurthub/tenant" @@ -93,32 +94,25 @@ func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { } trace++ - klog.Infof("%d. prepare for health checker clients", trace) - cloudClients, coordinatorClient, err := createClients(cfg.HeartbeatTimeoutSeconds, cfg.RemoteServers, cfg.CoordinatorServer, transportManager) + klog.Infof("%d. prepare cloud kube clients", trace) + cloudClients, err := createClients(cfg.HeartbeatTimeoutSeconds, cfg.RemoteServers, cfg.CoordinatorServerURL, transportManager) if err != nil { - return fmt.Errorf("failed to create health checker clients, %w", err) + return fmt.Errorf("failed to create cloud clients, %w", err) } trace++ var cloudHealthChecker healthchecker.MultipleBackendsHealthChecker - var coordinatorHealthChecker healthchecker.HealthChecker if cfg.WorkingMode == util.WorkingModeEdge { klog.Infof("%d. create health checkers for remote servers and pool coordinator", trace) cloudHealthChecker, err = healthchecker.NewCloudAPIServerHealthChecker(cfg, cloudClients, ctx.Done()) if err != nil { return fmt.Errorf("could not new cloud health checker, %w", err) } - coordinatorHealthChecker, err = healthchecker.NewCoordinatorHealthChecker(cfg, coordinatorClient, cloudHealthChecker, ctx.Done()) - if err != nil { - return fmt.Errorf("failed to create coordinator health checker, %v", err) - } - } else { klog.Infof("%d. disable health checker for node %s because it is a cloud node", trace, cfg.NodeName) - // In cloud mode, cloud health checker and pool coordinator health checker are not needed. + // In cloud mode, cloud health checker is not needed. // This fake checker will always report that the cloud is healthy and pool coordinator is unhealthy. cloudHealthChecker = healthchecker.NewFakeChecker(true, make(map[string]int)) - coordinatorHealthChecker = healthchecker.NewFakeChecker(false, make(map[string]int)) } trace++ @@ -154,25 +148,41 @@ func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { tenantMgr := tenant.New(cfg.TenantNs, cfg.SharedFactory, ctx.Done()) trace++ - klog.Infof("%d. create yurthub elector", trace) - elector, err := poolcoordinator.NewHubElector(cfg, coordinatorClient, coordinatorHealthChecker, cloudHealthChecker, ctx.Done()) - if err != nil { - klog.Errorf("failed to create hub elector, %v", err) + var coordinator poolcoordinator.Coordinator = &poolcoordinator.FakeCoordinator{} + var coordinatorHealthChecker healthchecker.HealthChecker = healthchecker.NewFakeChecker(false, make(map[string]int)) + var coordinatorTransportManager transport.Interface = nil + var waittingForCoordinator func() (healthchecker.HealthChecker, transport.Interface, poolcoordinator.Coordinator, error) = nil + if cfg.EnableCoordinator { + klog.Infof("%d. start to run coordinator", trace) + // coordinatorRun will register secret informer into sharedInformerFactory, and start a new goroutine to periodically check + // if certs has been got from cloud APIServer. + waittingForCoordinator = coordinatorRun(ctx, cfg, restConfigMgr, cloudHealthChecker) + trace++ } - elector.Run(ctx.Done()) - trace++ - // TODO: cloud client load balance - klog.Infof("%d. create coordinator", trace) - coordinator, err := poolcoordinator.NewCoordinator(ctx, cfg, restConfigMgr, transportManager, elector) - if err != nil { - klog.Errorf("failed to create coordinator, %v", err) + // Start the informer factory if all informers have been registered + cfg.SharedFactory.Start(ctx.Done()) + cfg.YurtSharedFactory.Start(ctx.Done()) + + if waittingForCoordinator != nil { + // Waitting for the coordinator to run, before using it to create other components. + coordinatorHealthChecker, coordinatorTransportManager, coordinator, err = waittingForCoordinator() + if err != nil { + return fmt.Errorf("failed to wait for coordinator to run, %v", err) + } } - coordinator.Run() - trace++ klog.Infof("%d. new reverse proxy handler for remote servers", trace) - yurtProxyHandler, err := proxy.NewYurtReverseProxyHandler(cfg, cacheMgr, transportManager, coordinator, cloudHealthChecker, coordinatorHealthChecker, tenantMgr, ctx.Done()) + yurtProxyHandler, err := proxy.NewYurtReverseProxyHandler( + cfg, + cacheMgr, + transportManager, + cloudHealthChecker, + tenantMgr, + coordinator, + coordinatorTransportManager, + coordinatorHealthChecker, + ctx.Done()) if err != nil { return fmt.Errorf("could not create reverse proxy handler, %w", err) } @@ -182,10 +192,6 @@ func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { cfg.NetworkMgr.Run(ctx.Done()) } - // start shared informers before start hub server - cfg.SharedFactory.Start(ctx.Done()) - cfg.YurtSharedFactory.Start(ctx.Done()) - klog.Infof("%d. new %s server and begin to serve", trace, projectinfo.GetHubName()) if err := server.RunYurtHubServers(cfg, yurtProxyHandler, restConfigMgr, ctx.Done()); err != nil { return fmt.Errorf("could not run hub servers, %w", err) @@ -197,8 +203,7 @@ func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { // createClients will create clients for all cloud APIServer and client for pool coordinator // It will return a map, mapping cloud APIServer URL to its client, and a pool coordinator client -func createClients(heartbeatTimeoutSeconds int, remoteServers []*url.URL, coordinatorServer *url.URL, tp transport.Interface) (map[string]kubernetes.Interface, kubernetes.Interface, error) { - var coordinatorClient kubernetes.Interface +func createClients(heartbeatTimeoutSeconds int, remoteServers []*url.URL, coordinatorServer *url.URL, tp transport.Interface) (map[string]kubernetes.Interface, error) { cloudClients := make(map[string]kubernetes.Interface) for i := range remoteServers { restConf := &rest.Config{ @@ -208,21 +213,106 @@ func createClients(heartbeatTimeoutSeconds int, remoteServers []*url.URL, coordi } c, err := kubernetes.NewForConfig(restConf) if err != nil { - return cloudClients, coordinatorClient, err + return cloudClients, err } cloudClients[remoteServers[i].String()] = c } + return cloudClients, nil +} + +// coordinatorRun will initialize and start all coordinator-related components in an async way. +// It returns a func waittingForReady, which will block until the initialization routine exited. +// If the initialization succeeds, waittingForReady will return these coordinator-related components, including: +// 1. coordinator HealthChecker +// 2. coordinator TransportManager +// 3. and the coordinator +// Otherwise, the error is not nil. +func coordinatorRun(ctx context.Context, + cfg *config.YurtHubConfiguration, + restConfigMgr *hubrest.RestConfigManager, + cloudHealthChecker healthchecker.MultipleBackendsHealthChecker) (waittingForReady func() (healthchecker.HealthChecker, transport.Interface, poolcoordinator.Coordinator, error)) { + var coordinatorHealthChecker healthchecker.HealthChecker + var coordinatorTransportMgr transport.Interface + var coordinator poolcoordinator.Coordinator + var coordinatorCertManager *coordinatorcertmgr.CertManager + var returnErr error + + readyCh := make(chan struct{}) + + go func() { + coorCertManager, err := coordinatorcertmgr.NewCertManager(cfg.CoordinatorPKIDir, cfg.ProxiedClient, cfg.SharedFactory) + close(coordinatorInformerRegistryChan) // notify the coordinator secret informer registry event + if err != nil { + returnErr = fmt.Errorf("failed to create coordinator cert manager, %v", err) + return + } - cfg := &rest.Config{ - Host: coordinatorServer.String(), - Transport: tp.CurrentTransport(), - Timeout: time.Duration(heartbeatTimeoutSeconds) * time.Second, + coorTransportMgr, err := poolCoordinatorTransportMgrGetter(cfg.HeartbeatTimeoutSeconds, cfg.CoordinatorServerURL, coordinatorCertManager, ctx.Done()) + if err != nil { + returnErr = fmt.Errorf("failed to create coordinator transport manager, %v", err) + return + } + + coordinatorClient, err := kubernetes.NewForConfig(&rest.Config{ + Host: cfg.CoordinatorServerURL.String(), + Transport: coorTransportMgr.CurrentTransport(), + Timeout: time.Duration(cfg.HeartbeatTimeoutSeconds), + }) + if err != nil { + returnErr = fmt.Errorf("failed to get coordinator client for pool coordinator, %v", err) + return + } + + coorHealthChecker, err := healthchecker.NewCoordinatorHealthChecker(cfg, coordinatorClient, cloudHealthChecker, ctx.Done()) + if err != nil { + returnErr = fmt.Errorf("failed to create coordinator health checker, %v", err) + return + } + + var elector *poolcoordinator.HubElector + elector, err = poolcoordinator.NewHubElector(cfg, coordinatorClient, coorHealthChecker, cloudHealthChecker, ctx.Done()) + if err != nil { + returnErr = fmt.Errorf("failed to create hub elector, %v", err) + return + } + elector.Run(ctx.Done()) + + coor, err := poolcoordinator.NewCoordinator(ctx, cfg, restConfigMgr, coorCertManager, coorTransportMgr, elector) + if err != nil { + returnErr = fmt.Errorf("failed to create coordinator, %v", err) + return + } + coor.Run() + + coordinatorTransportMgr = coorTransportMgr + coordinatorHealthChecker = coorHealthChecker + coordinator = coor + returnErr = nil + }() + + waittingForReady = func() (healthchecker.HealthChecker, transport.Interface, poolcoordinator.Coordinator, error) { + <-readyCh + return coordinatorHealthChecker, coordinatorTransportMgr, coordinator, returnErr } - c, err := kubernetes.NewForConfig(cfg) + + return waittingForReady +} + +func poolCoordinatorTransportMgrGetter(heartbeatTimeoutSeconds int, coordinatorServer *url.URL, coordinatorCertMgr *coordinatorcertmgr.CertManager, stopCh <-chan struct{}) (transport.Interface, error) { + err := wait.PollImmediate(5*time.Second, 4*time.Minute, func() (done bool, err error) { + if coordinatorCertMgr.GetAPIServerClientCert() != nil { + return true, nil + } + klog.Infof("waiting for preparing coordinator client certificate") + return false, nil + }) if err != nil { - return cloudClients, coordinatorClient, err + klog.Errorf("timeout when waiting for coordinator client certificate") } - coordinatorClient = c - return cloudClients, coordinatorClient, nil + coordinatorTransportMgr, err := transport.NewTransportManager(coordinatorCertMgr, stopCh) + if err != nil { + return nil, fmt.Errorf("failed to create transport manager for pool coordinator, %v", err) + } + return coordinatorTransportMgr, nil } diff --git a/pkg/node-servant/config/control-plane.go b/pkg/node-servant/config/control-plane.go index d0bfcab2f75..b37df84afb7 100644 --- a/pkg/node-servant/config/control-plane.go +++ b/pkg/node-servant/config/control-plane.go @@ -70,7 +70,6 @@ func newStaticPodRunner(podManifestsPath string) (Runner, error) { func (spr *staticPodRunner) Do() error { var kasPodUpdated bool - var kcmPodUpdated bool // read kube-apiserver static pod kasObj, err := fileutil.ReadObjectFromYamlFile(spr.kasStaticPodPath) if err != nil { @@ -101,31 +100,6 @@ func (spr *staticPodRunner) Do() error { kasPodUpdated = true } - // read kube-controller-manager static pod - kcmObj, err := fileutil.ReadObjectFromYamlFile(spr.kcmStaticPodPath) - if err != nil { - return err - } - kcmPod, ok := kcmObj.(*v1.Pod) - if !ok { - return fmt.Errorf("manifest file(%s) is not a static pod", spr.kcmStaticPodPath) - } - - // disable NodeLifeCycle controller - for i := range kcmPod.Spec.Containers { - for j := range kcmPod.Spec.Containers[i].Command { - if strings.Contains(kcmPod.Spec.Containers[i].Command[j], "--controllers=") { - if !strings.Contains(kcmPod.Spec.Containers[i].Command[j], "-nodelifecycle,") { - // insert -nodelifecycle, after = - insertPoint := strings.Index(kcmPod.Spec.Containers[i].Command[j], "=") + 1 - kcmPod.Spec.Containers[i].Command[j] = kcmPod.Spec.Containers[i].Command[j][:insertPoint] + "-nodelifecycle," + kcmPod.Spec.Containers[i].Command[j][insertPoint:] - kcmPodUpdated = true - break - } - } - } - } - // update static pod files if kasPodUpdated { if err := fileutil.WriteObjectToYamlFile(kasPod, spr.kasStaticPodPath); err != nil { @@ -133,11 +107,5 @@ func (spr *staticPodRunner) Do() error { } } - if kcmPodUpdated { - if err := fileutil.WriteObjectToYamlFile(kcmPod, spr.kcmStaticPodPath); err != nil { - return err - } - } - return nil } diff --git a/pkg/yurthub/healthchecker/health_checker.go b/pkg/yurthub/healthchecker/health_checker.go index 855720b489c..dff13801b3c 100644 --- a/pkg/yurthub/healthchecker/health_checker.go +++ b/pkg/yurthub/healthchecker/health_checker.go @@ -65,7 +65,7 @@ func NewCoordinatorHealthChecker(cfg *config.YurtHubConfiguration, checkerClient heartbeatInterval: cfg.HeartbeatIntervalSeconds, } chc.coordinatorProber = newProber(checkerClient, - cfg.CoordinatorServer.String(), + cfg.CoordinatorServerURL.String(), cfg.NodeName, cfg.HeartbeatFailedRetry, cfg.HeartbeatHealthyThreshold, diff --git a/pkg/yurthub/healthchecker/health_checker_test.go b/pkg/yurthub/healthchecker/health_checker_test.go index ba82ffbaf06..174b9b51863 100644 --- a/pkg/yurthub/healthchecker/health_checker_test.go +++ b/pkg/yurthub/healthchecker/health_checker_test.go @@ -176,7 +176,7 @@ func TestNewCoordinatorHealthChecker(t *testing.T) { for k, tt := range testcases { t.Run(k, func(t *testing.T) { cfg := &config.YurtHubConfiguration{ - CoordinatorServer: &url.URL{Host: "127.0.0.1:18080"}, + CoordinatorServerURL: &url.URL{Host: "127.0.0.1:18080"}, NodeName: node.Name, HeartbeatFailedRetry: 2, HeartbeatHealthyThreshold: 1, diff --git a/pkg/yurthub/poolcoordinator/certmanager/certmanager.go b/pkg/yurthub/poolcoordinator/certmanager/certmanager.go new file mode 100644 index 00000000000..2f4811f58bc --- /dev/null +++ b/pkg/yurthub/poolcoordinator/certmanager/certmanager.go @@ -0,0 +1,163 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 certmanager + +import ( + "crypto/tls" + "fmt" + "path/filepath" + "sync" + "time" + + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/fields" + "k8s.io/client-go/informers" + coreinformers "k8s.io/client-go/informers/core/v1" + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/tools/cache" + "k8s.io/klog/v2" + + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" + "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" +) + +type CertFileType string + +var ( + RootCA CertFileType = "CA" + YurthubClientCert CertFileType = "YurthubClientCert" + YurthubClientKey CertFileType = "YurthubClientKey" +) + +var certFileNames = map[CertFileType]string{ + RootCA: "pool-coordinator-ca.crt", + YurthubClientCert: "pool-coordinator-yurthub-client.crt", + YurthubClientKey: "pool-coordinator-yurthub-client.key", +} + +func NewCertManager(caFilePath string, yurtClient kubernetes.Interface, informerFactory informers.SharedInformerFactory) (*CertManager, error) { + store := fs.FileSystemOperator{} + dir, _ := filepath.Split(caFilePath) + if err := store.CreateDir(dir); err != nil && err != fs.ErrExists { + return nil, fmt.Errorf("failed to create dir %s, %v", dir, err) + } + + certMgr := &CertManager{ + pkiDir: dir, + store: store, + } + + // try to use last cert files when restart. + certPath, keyPath := certMgr.GetFilePath(YurthubClientCert), certMgr.GetFilePath(YurthubClientKey) + if cert, err := tls.LoadX509KeyPair(certPath, keyPath); err == nil { + certMgr.cert = &cert + } + + secretInformerFunc := func(client kubernetes.Interface, resyncPeriod time.Duration) cache.SharedIndexInformer { + tweakListOptions := func(options *metav1.ListOptions) { + options.FieldSelector = fields.Set{"metadata.name": constants.PoolCoordinatorClientSecretName}.String() + } + return coreinformers.NewFilteredSecretInformer(yurtClient, constants.PoolCoordinatorClientSecretNamespace, 0, nil, tweakListOptions) + } + secretInformer := informerFactory.InformerFor(&corev1.Secret{}, secretInformerFunc) + secretInformer.AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: func(obj interface{}) { + klog.V(4).Infof("notify secret add event for %s", constants.PoolCoordinatorClientSecretName) + secret := obj.(*corev1.Secret) + certMgr.updateCerts(secret) + }, + UpdateFunc: func(_, newObj interface{}) { + klog.V(4).Infof("notify secret update event for %s", constants.PoolCoordinatorClientSecretName) + secret := newObj.(*corev1.Secret) + certMgr.updateCerts(secret) + }, + DeleteFunc: func(_ interface{}) { + klog.V(4).Infof("notify secret delete event for %s", constants.PoolCoordinatorClientSecretName) + certMgr.deleteCerts() + }, + }) + + return certMgr, nil +} + +type CertManager struct { + sync.Mutex + pkiDir string + cert *tls.Certificate + store fs.FileSystemOperator +} + +func (c *CertManager) GetAPIServerClientCert() *tls.Certificate { + c.Lock() + defer c.Unlock() + return c.cert +} + +func (c *CertManager) GetCaFile() string { + return c.GetFilePath(RootCA) +} + +func (c *CertManager) GetFilePath(t CertFileType) string { + return filepath.Join(c.pkiDir, certFileNames[t]) +} + +func (c *CertManager) updateCerts(secret *corev1.Secret) { + ca := secret.Data["ca.crt"] + coordinatorClientCrt := secret.Data["pool-coordinator-yurthub-client.crt"] + coordinatorClientKey := secret.Data["pool-coordinator-yurthub-client.key"] + + cert, err := tls.X509KeyPair(coordinatorClientCrt, coordinatorClientKey) + if err != nil { + klog.Errorf("failed to create tls certificate for coordinator, %v", err) + return + } + + caPath := c.GetCaFile() + certPath := c.GetFilePath(YurthubClientCert) + keyPath := c.GetFilePath(YurthubClientKey) + + c.Lock() + defer c.Unlock() + // TODO: The following updates should rollback on failure, + // making the certs in-memory and certs on disk consistent. + if err := c.createOrUpdateFile(caPath, ca); err != nil { + klog.Errorf("failed to update ca, %v", err) + } + if err := c.createOrUpdateFile(keyPath, coordinatorClientKey); err != nil { + klog.Errorf("failed to update client key, %v", err) + } + if err := c.createOrUpdateFile(certPath, coordinatorClientCrt); err != nil { + klog.Errorf("failed to update client cert, %v", err) + } + c.cert = &cert +} + +func (c *CertManager) deleteCerts() { + c.cert = nil +} + +func (c *CertManager) createOrUpdateFile(path string, data []byte) error { + if err := c.store.Write(path, data); err == fs.ErrNotExists { + if err := c.store.CreateFile(path, data); err != nil { + return fmt.Errorf("failed to create file at %s, %v", path, err) + } + } else if err != nil { + return fmt.Errorf("failed to update file at %s, %v", path, err) + } + return nil +} diff --git a/pkg/yurthub/poolcoordinator/constants/constants.go b/pkg/yurthub/poolcoordinator/constants/constants.go index bcd06c86e76..70c044bb3c9 100644 --- a/pkg/yurthub/poolcoordinator/constants/constants.go +++ b/pkg/yurthub/poolcoordinator/constants/constants.go @@ -35,5 +35,7 @@ var ( ) const ( - DefaultPoolScopedUserAgent = "leader-yurthub" + DefaultPoolScopedUserAgent = "leader-yurthub" + PoolCoordinatorClientSecretName = "pool-coordinator-yurthub-certs" + PoolCoordinatorClientSecretNamespace = "kube-system" ) diff --git a/pkg/yurthub/poolcoordinator/coordinator.go b/pkg/yurthub/poolcoordinator/coordinator.go index ab3e8763fcb..4cebbe9d7a5 100644 --- a/pkg/yurthub/poolcoordinator/coordinator.go +++ b/pkg/yurthub/poolcoordinator/coordinator.go @@ -44,6 +44,7 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/meta" yurtrest "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/rest" "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/serializer" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/certmanager" "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" "github.com/openyurtio/openyurt/pkg/yurthub/storage" "github.com/openyurtio/openyurt/pkg/yurthub/storage/etcd" @@ -58,7 +59,24 @@ const ( nameInformerLease = "leader-informer-sync" ) -type Coordinator struct { +// Coordinator will track the status of pool coordinator, and change the +// cache and proxy behaviour of yurthub accordingly. +type Coordinator interface { + // Start the Coordinator. + Run() + // IsReady will return the poolCacheManager and true if the pool-coordinator is ready. + // Pool-Coordinator ready means it is ready to handle request. To be specific, it should + // satisfy the following 3 condition: + // 1. Pool-Coordinator is healthy + // 2. Pool-Scoped resources have been synced with cloud, through list/watch + // 3. local cache has been uploaded to pool-coordinator + IsReady() (cachemanager.CacheManager, bool) + // IsCoordinatorHealthy will return the poolCacheManager and true if the pool-coordinator is healthy. + // We assume coordinator is healthy when the elect status is LeaderHub and FollowerHub. + IsHealthy() (cachemanager.CacheManager, bool) +} + +type coordinator struct { sync.Mutex ctx context.Context cancelEtcdStorage func() @@ -73,6 +91,7 @@ type Coordinator struct { hubElector *HubElector electStatus int32 isPoolCacheSynced bool + certMgr *certmanager.CertManager needUploadLocalCache bool // poolScopeCacheSyncManager is used to sync pool-scoped resources from cloud to poolcoordinator. poolScopeCacheSyncManager *poolScopedCacheSyncManager @@ -91,20 +110,21 @@ func NewCoordinator( ctx context.Context, cfg *config.YurtHubConfiguration, restMgr *yurtrest.RestConfigManager, - transportMgr transport.Interface, - elector *HubElector) (*Coordinator, error) { + certMgr *certmanager.CertManager, + coordinatorTransMgr transport.Interface, + elector *HubElector) (*coordinator, error) { etcdStorageCfg := &etcd.EtcdStorageConfig{ Prefix: cfg.CoordinatorStoragePrefix, EtcdEndpoints: []string{cfg.CoordinatorStorageAddr}, - CaFile: cfg.CoordinatorStorageCaFile, - CertFile: cfg.CoordinatorStorageCertFile, - KeyFile: cfg.CoordinatorStorageKeyFile, + CaFile: certMgr.GetCaFile(), + CertFile: certMgr.GetFilePath(certmanager.YurthubClientCert), + KeyFile: certMgr.GetFilePath(certmanager.YurthubClientKey), LocalCacheDir: cfg.DiskCachePath, } coordinatorRESTCfg := &rest.Config{ - Host: cfg.CoordinatorServer.String(), - Transport: transportMgr.CurrentTransport(), + Host: cfg.CoordinatorServerURL.String(), + Transport: coordinatorTransMgr.CurrentTransport(), Timeout: defaultInformerLeaseRenewDuration, } coordinatorClient, err := kubernetes.NewForConfig(coordinatorRESTCfg) @@ -112,10 +132,11 @@ func NewCoordinator( return nil, fmt.Errorf("failed to create client for pool coordinator, %v", err) } - coordinator := &Coordinator{ + coordinator := &coordinator{ ctx: ctx, etcdStorageCfg: etcdStorageCfg, restConfigMgr: restMgr, + certMgr: certMgr, informerFactory: cfg.SharedFactory, diskStorage: cfg.StorageWrapper.GetStorage(), serializerMgr: cfg.SerializerManager, @@ -152,7 +173,7 @@ func NewCoordinator( return coordinator, nil } -func (coordinator *Coordinator) Run() { +func (coordinator *coordinator) Run() { for { var poolCacheManager cachemanager.CacheManager var cancelEtcdStorage func() @@ -291,7 +312,7 @@ func (coordinator *Coordinator) Run() { // 1. Pool-Coordinator is healthy // 2. Pool-Scoped resources have been synced with cloud, through list/watch // 3. local cache has been uploaded to pool-coordinator -func (coordinator *Coordinator) IsReady() (cachemanager.CacheManager, bool) { +func (coordinator *coordinator) IsReady() (cachemanager.CacheManager, bool) { // If electStatus is not PendingHub, it means pool-coordinator is healthy. coordinator.Lock() defer coordinator.Unlock() @@ -303,7 +324,7 @@ func (coordinator *Coordinator) IsReady() (cachemanager.CacheManager, bool) { // IsCoordinatorHealthy will return the poolCacheManager and true if the pool-coordinator is healthy. // We assume coordinator is healthy when the elect status is LeaderHub and FollowerHub. -func (coordinator *Coordinator) IsHealthy() (cachemanager.CacheManager, bool) { +func (coordinator *coordinator) IsHealthy() (cachemanager.CacheManager, bool) { coordinator.Lock() defer coordinator.Unlock() if coordinator.electStatus != PendingHub { @@ -312,7 +333,7 @@ func (coordinator *Coordinator) IsHealthy() (cachemanager.CacheManager, bool) { return nil, false } -func (coordinator *Coordinator) buildPoolCacheStore() (cachemanager.CacheManager, storage.Store, func(), error) { +func (coordinator *coordinator) buildPoolCacheStore() (cachemanager.CacheManager, storage.Store, func(), error) { ctx, cancel := context.WithCancel(coordinator.ctx) etcdStore, err := etcd.NewStorage(ctx, coordinator.etcdStorageCfg) if err != nil { @@ -328,11 +349,11 @@ func (coordinator *Coordinator) buildPoolCacheStore() (cachemanager.CacheManager return poolCacheManager, etcdStore, cancel, nil } -func (coordinator *Coordinator) getEtcdStore() storage.Store { +func (coordinator *coordinator) getEtcdStore() storage.Store { return coordinator.etcdStorage } -func (coordinator *Coordinator) newCloudLeaseClient() (coordclientset.LeaseInterface, error) { +func (coordinator *coordinator) newCloudLeaseClient() (coordclientset.LeaseInterface, error) { restCfg := coordinator.restConfigMgr.GetRestConfig(true) if restCfg == nil { return nil, fmt.Errorf("no cloud server is healthy") @@ -345,7 +366,7 @@ func (coordinator *Coordinator) newCloudLeaseClient() (coordclientset.LeaseInter return cloudClient.CoordinationV1().Leases(corev1.NamespaceNodeLease), nil } -func (coordinator *Coordinator) uploadLocalCache(etcdStore storage.Store) error { +func (coordinator *coordinator) uploadLocalCache(etcdStore storage.Store) error { uploader := &localCacheUploader{ diskStorage: coordinator.diskStorage, etcdStorage: etcdStore, @@ -355,7 +376,7 @@ func (coordinator *Coordinator) uploadLocalCache(etcdStore storage.Store) error return nil } -func (coordinator *Coordinator) delegateNodeLease(cloudLeaseClient coordclientset.LeaseInterface, obj interface{}) { +func (coordinator *coordinator) delegateNodeLease(cloudLeaseClient coordclientset.LeaseInterface, obj interface{}) { newLease := obj.(*coordinationv1.Lease) for i := 0; i < leaseDelegateRetryTimes; i++ { // ResourceVersions of lease objects in pool-coordinator always have different rv @@ -378,7 +399,7 @@ func (coordinator *Coordinator) delegateNodeLease(cloudLeaseClient coordclientse } } -func (coordinator *Coordinator) detectPoolCacheSynced(obj interface{}) { +func (coordinator *coordinator) detectPoolCacheSynced(obj interface{}) { lease := obj.(*coordinationv1.Lease) renewTime := lease.Spec.RenewTime if time.Now().After(renewTime.Add(defaultPoolCacheStaleDuration)) { diff --git a/pkg/yurthub/poolcoordinator/fake_coordinator.go b/pkg/yurthub/poolcoordinator/fake_coordinator.go new file mode 100644 index 00000000000..9b8ea520a7a --- /dev/null +++ b/pkg/yurthub/poolcoordinator/fake_coordinator.go @@ -0,0 +1,33 @@ +/* +Copyright 2022 The OpenYurt Authors. + +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 poolcoordinator + +import "github.com/openyurtio/openyurt/pkg/yurthub/cachemanager" + +type FakeCoordinator struct{} + +var _ Coordinator = &FakeCoordinator{} + +func (fc *FakeCoordinator) Run() {} + +func (fc *FakeCoordinator) IsReady() (cachemanager.CacheManager, bool) { + return nil, false +} + +func (fc *FakeCoordinator) IsHealthy() (cachemanager.CacheManager, bool) { + return nil, false +} diff --git a/pkg/yurthub/proxy/pool/pool.go b/pkg/yurthub/proxy/pool/pool.go index 7d98982177c..e81eec4129f 100644 --- a/pkg/yurthub/proxy/pool/pool.go +++ b/pkg/yurthub/proxy/pool/pool.go @@ -245,12 +245,12 @@ func (pp *PoolCoordinatorProxy) cacheResponse(req *http.Request, resp *http.Resp req = req.WithContext(ctx) wrapPrc, _ := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "cache-manager") - rc, prc := hubutil.NewDualReadCloser(req, wrapPrc, false) + rc, prc := hubutil.NewDualReadCloser(req, wrapPrc, true) go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { if err := pp.localCacheMgr.CacheResponse(req, prc, stopCh); err != nil { klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) } }(req, prc, ctx.Done()) - req.Body = rc + resp.Body = rc } } diff --git a/pkg/yurthub/proxy/proxy.go b/pkg/yurthub/proxy/proxy.go index 88fe2e3188a..75168388717 100644 --- a/pkg/yurthub/proxy/proxy.go +++ b/pkg/yurthub/proxy/proxy.go @@ -17,13 +17,20 @@ limitations under the License. package proxy import ( + "bytes" "errors" + "io/ioutil" "net/http" + "strings" + v1 "k8s.io/api/authorization/v1" + "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/apimachinery/pkg/runtime/serializer" "k8s.io/apimachinery/pkg/util/sets" "k8s.io/apiserver/pkg/endpoints/filters" apirequest "k8s.io/apiserver/pkg/endpoints/request" "k8s.io/apiserver/pkg/server" + "k8s.io/client-go/kubernetes/scheme" "k8s.io/klog/v2" "github.com/openyurtio/openyurt/cmd/yurthub/app/config" @@ -48,8 +55,9 @@ type yurtReverseProxy struct { poolProxy http.Handler maxRequestsInFlight int tenantMgr tenant.Interface - coordinator *poolcoordinator.Coordinator + isCoordinatorReady func() bool workingMode hubutil.WorkingMode + enablePoolCoordinator bool } // NewYurtReverseProxyHandler creates a http handler for proxying @@ -58,10 +66,11 @@ func NewYurtReverseProxyHandler( yurtHubCfg *config.YurtHubConfiguration, localCacheMgr cachemanager.CacheManager, transportMgr transport.Interface, - coordinator *poolcoordinator.Coordinator, cloudHealthChecker healthchecker.MultipleBackendsHealthChecker, - coordinatorHealthChecker healthchecker.HealthChecker, tenantMgr tenant.Interface, + coordinator poolcoordinator.Coordinator, + coordinatorTransportMgr transport.Interface, + coordinatorHealthChecker healthchecker.HealthChecker, stopCh <-chan struct{}) (http.Handler, error) { cfg := &server.Config{ LegacyAPIGroupPrefixes: sets.NewString(server.DefaultLegacyAPIPrefix), @@ -83,31 +92,36 @@ func NewYurtReverseProxyHandler( } var localProxy, poolProxy http.Handler + isCoordinatorHealthy := func() bool { + _, healthy := coordinator.IsHealthy() + return healthy + } + isCoordinatorReady := func() bool { + _, ready := coordinator.IsReady() + return ready + } if yurtHubCfg.WorkingMode == hubutil.WorkingModeEdge { // When yurthub works in Edge mode, we may use local proxy or pool proxy to handle // the request when offline. localProxy = local.NewLocalProxy(localCacheMgr, cloudHealthChecker.IsHealthy, - func() bool { - _, ready := coordinator.IsHealthy() - return ready - }, + isCoordinatorHealthy, yurtHubCfg.MinRequestTimeout, ) localProxy = local.WithFakeTokenInject(localProxy, yurtHubCfg.SerializerManager) - poolProxy, err = pool.NewPoolCoordinatorProxy( - yurtHubCfg.CoordinatorServer, - localCacheMgr, - transportMgr, - yurtHubCfg.FilterManager, - func() bool { - _, isReady := coordinator.IsReady() - return isReady - }, - stopCh) - if err != nil { - return nil, err + + if yurtHubCfg.EnableCoordinator { + poolProxy, err = pool.NewPoolCoordinatorProxy( + yurtHubCfg.CoordinatorServerURL, + localCacheMgr, + coordinatorTransportMgr, + yurtHubCfg.FilterManager, + isCoordinatorReady, + stopCh) + if err != nil { + return nil, err + } } } @@ -119,7 +133,8 @@ func NewYurtReverseProxyHandler( localProxy: localProxy, poolProxy: poolProxy, maxRequestsInFlight: yurtHubCfg.MaxRequestInFlight, - coordinator: coordinator, + isCoordinatorReady: isCoordinatorReady, + enablePoolCoordinator: yurtHubCfg.EnableCoordinator, tenantMgr: tenantMgr, workingMode: yurtHubCfg.WorkingMode, } @@ -140,7 +155,10 @@ func (p *yurtReverseProxy) buildHandlerChain(handler http.Handler) http.Handler handler = util.WithRequestTraceFull(handler) handler = util.WithMaxInFlightLimit(handler, p.maxRequestsInFlight) handler = util.WithRequestClientComponent(handler) - handler = util.WithIfPoolScopedResource(handler) + + if p.enablePoolCoordinator { + handler = util.WithIfPoolScopedResource(handler) + } if p.tenantMgr != nil && p.tenantMgr.GetTenantNs() != "" { handler = util.WithSaTokenSubstitute(handler, p.tenantMgr) @@ -191,7 +209,7 @@ func (p *yurtReverseProxy) eventHandler(rw http.ResponseWriter, req *http.Reques if p.cloudHealthChecker.IsHealthy() { p.loadBalancer.ServeHTTP(rw, req) // TODO: We should also consider create the event in pool-coordinator when the cloud is healthy. - } else if _, isReady := p.coordinator.IsReady(); isReady { + } else if p.isCoordinatorReady() && p.poolProxy != nil { p.poolProxy.ServeHTTP(rw, req) } else { p.localProxy.ServeHTTP(rw, req) @@ -199,7 +217,7 @@ func (p *yurtReverseProxy) eventHandler(rw http.ResponseWriter, req *http.Reques } func (p *yurtReverseProxy) poolScopedResouceHandler(rw http.ResponseWriter, req *http.Request) { - if _, isReady := p.coordinator.IsReady(); isReady { + if p.isCoordinatorReady() && p.poolProxy != nil { p.poolProxy.ServeHTTP(rw, req) } else if p.cloudHealthChecker.IsHealthy() { p.loadBalancer.ServeHTTP(rw, req) @@ -209,8 +227,11 @@ func (p *yurtReverseProxy) poolScopedResouceHandler(rw http.ResponseWriter, req } func (p *yurtReverseProxy) subjectAccessReviewHandler(rw http.ResponseWriter, req *http.Request) { - if isRequestFromPoolCoordinator(req) { - if _, isReady := p.coordinator.IsReady(); isReady { + if isSubjectAccessReviewFromPoolCoordinator(req) { + // check if the logs/exec request is from APIServer or PoolCoordinator. + // We should avoid sending SubjectAccessReview to Pool-Coordinator if the logs/exec requests + // come from APIServer, which may fail for RBAC differences, vise versa. + if p.isCoordinatorReady() { p.poolProxy.ServeHTTP(rw, req) } else { err := errors.New("request is from pool-coordinator but it's currently not healthy") @@ -228,9 +249,38 @@ func (p *yurtReverseProxy) subjectAccessReviewHandler(rw http.ResponseWriter, re } } -func isRequestFromPoolCoordinator(req *http.Request) bool { - // TODO: need a way to check if the logs/exec request is from APIServer or PoolCoordinator. - // We should avoid sending SubjectAccessReview to Pool-Coordinator if the logs/exec requests - // come from APIServer, which may fail for RBAC differences, vise versa. +func isSubjectAccessReviewFromPoolCoordinator(req *http.Request) bool { + var buf bytes.Buffer + if n, err := buf.ReadFrom(req.Body); err != nil || n == 0 { + klog.Errorf("failed to read SubjectAccessReview from request %s, read %d bytes, %v", hubutil.ReqString(req), n, err) + return false + } + req.Body = ioutil.NopCloser(&buf) + + subjectAccessReviewGVK := schema.GroupVersionKind{ + Group: v1.SchemeGroupVersion.Group, + Version: v1.SchemeGroupVersion.Version, + Kind: "SubjectAccessReview"} + decoder := serializer.NewCodecFactory(scheme.Scheme).UniversalDeserializer() + obj := &v1.SubjectAccessReview{} + got, gvk, err := decoder.Decode(buf.Bytes(), nil, obj) + if err != nil { + klog.Errorf("failed to decode SubjectAccessReview in request %s, %v", hubutil.ReqString(req), err) + return false + } + if (*gvk) != subjectAccessReviewGVK { + klog.Errorf("unexpected gvk: %s in request: %s, want: %s", gvk.String(), hubutil.ReqString(req), subjectAccessReviewGVK.String()) + return false + } + + sav := got.(*v1.SubjectAccessReview) + for _, g := range sav.Spec.Groups { + if g == "openyurt:pool-coordinator" { + return true + } + } + + klog.V(4).Infof("SubjectAccessReview in request %s is not for pool-coordinator, whose group: %s, user: %s", + hubutil.ReqString(req), strings.Join(sav.Spec.Groups, ";"), sav.Spec.User) return false } diff --git a/pkg/yurthub/proxy/remote/loadbalancer.go b/pkg/yurthub/proxy/remote/loadbalancer.go index 135a3e401e8..9029a1eaa9d 100644 --- a/pkg/yurthub/proxy/remote/loadbalancer.go +++ b/pkg/yurthub/proxy/remote/loadbalancer.go @@ -132,7 +132,7 @@ type loadBalancer struct { algo loadBalancerAlgo localCacheMgr cachemanager.CacheManager filterManager *manager.Manager - coordinator *poolcoordinator.Coordinator + coordinator poolcoordinator.Coordinator workingMode hubutil.WorkingMode stopCh <-chan struct{} } @@ -143,7 +143,7 @@ func NewLoadBalancer( remoteServers []*url.URL, localCacheMgr cachemanager.CacheManager, transportMgr transport.Interface, - coordinator *poolcoordinator.Coordinator, + coordinator poolcoordinator.Coordinator, healthChecker healthchecker.MultipleBackendsHealthChecker, filterManager *manager.Manager, workingMode hubutil.WorkingMode, @@ -360,31 +360,31 @@ func (lb *loadBalancer) cacheResponse(req *http.Request, resp *http.Response) { func (lb *loadBalancer) cacheToLocal(req *http.Request, resp *http.Response) { ctx := req.Context() req = req.WithContext(ctx) - rc, prc := hubutil.NewDualReadCloser(req, resp.Body, false) + rc, prc := hubutil.NewDualReadCloser(req, resp.Body, true) go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { if err := lb.localCacheMgr.CacheResponse(req, prc, stopCh); err != nil { klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) } }(req, prc, ctx.Done()) - req.Body = rc + resp.Body = rc } func (lb *loadBalancer) cacheToPool(req *http.Request, resp *http.Response, poolCacheManager cachemanager.CacheManager) { ctx := req.Context() req = req.WithContext(ctx) - rc, prc := hubutil.NewDualReadCloser(req, resp.Body, false) + rc, prc := hubutil.NewDualReadCloser(req, resp.Body, true) go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { if err := poolCacheManager.CacheResponse(req, prc, stopCh); err != nil { klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) } }(req, prc, ctx.Done()) - req.Body = rc + resp.Body = rc } func (lb *loadBalancer) cacheToLocalAndPool(req *http.Request, resp *http.Response, poolCacheMgr cachemanager.CacheManager) { ctx := req.Context() req = req.WithContext(ctx) - rc, prc1, prc2 := hubutil.NewTripleReadCloser(req, resp.Body, false) + rc, prc1, prc2 := hubutil.NewTripleReadCloser(req, resp.Body, true) go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { if err := lb.localCacheMgr.CacheResponse(req, prc, stopCh); err != nil { klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) @@ -398,7 +398,7 @@ func (lb *loadBalancer) cacheToLocalAndPool(req *http.Request, resp *http.Respon } }(req, prc2, ctx.Done()) } - req.Body = rc + resp.Body = rc } func isLeaderHubUserAgent(reqCtx context.Context) bool { diff --git a/pkg/yurthub/transport/transport.go b/pkg/yurthub/transport/transport.go index f7ce7505e0d..2e18a95a7b0 100644 --- a/pkg/yurthub/transport/transport.go +++ b/pkg/yurthub/transport/transport.go @@ -27,10 +27,17 @@ import ( "k8s.io/klog/v2" "github.com/openyurtio/openyurt/pkg/util/certmanager" - "github.com/openyurtio/openyurt/pkg/yurthub/certificate" "github.com/openyurtio/openyurt/pkg/yurthub/util" ) +type CertGetter interface { + // GetAPIServerClientCert returns the currently selected certificate, as well as + // the associated certificate and key data in PEM format. + GetAPIServerClientCert() *tls.Certificate + // Return CA file path. + GetCaFile() string +} + // Interface is an transport interface for managing clients that used to connecting kube-apiserver type Interface interface { // CurrentTransport get transport that used by load balancer @@ -45,21 +52,21 @@ type Interface interface { type transportManager struct { currentTransport *http.Transport bearerTransport *http.Transport - certManager certificate.YurtCertificateManager + certGetter CertGetter closeAll func() close func(string) stopCh <-chan struct{} } // NewTransportManager create a transport interface object. -func NewTransportManager(certMgr certificate.YurtCertificateManager, stopCh <-chan struct{}) (Interface, error) { - caFile := certMgr.GetCaFile() +func NewTransportManager(certGetter CertGetter, stopCh <-chan struct{}) (Interface, error) { + caFile := certGetter.GetCaFile() if len(caFile) == 0 { return nil, fmt.Errorf("ca cert file was not prepared when new transport") } klog.V(2).Infof("use %s ca cert file to access remote server", caFile) - cfg, err := tlsConfig(certMgr.GetAPIServerClientCert, caFile) + cfg, err := tlsConfig(certGetter.GetAPIServerClientCert, caFile) if err != nil { klog.Errorf("could not get tls config when new transport, %v", err) return nil, err @@ -91,7 +98,7 @@ func NewTransportManager(certMgr certificate.YurtCertificateManager, stopCh <-ch tm := &transportManager{ currentTransport: t, bearerTransport: bt, - certManager: certMgr, + certGetter: certGetter, closeAll: d.CloseAll, close: d.Close, stopCh: stopCh, @@ -114,10 +121,10 @@ func (tm *transportManager) Close(address string) { } func (tm *transportManager) start() { - lastCert := tm.certManager.GetAPIServerClientCert() + lastCert := tm.certGetter.GetAPIServerClientCert() go wait.Until(func() { - curr := tm.certManager.GetAPIServerClientCert() + curr := tm.certGetter.GetAPIServerClientCert() if lastCert == nil && curr == nil { // maybe at yurthub startup, just wait for cert generated, do nothing From 7ae8fbb9df078325c61af194e75d8fd0373c37cc Mon Sep 17 00:00:00 2001 From: Laurence <45508533+LaurenceLiZhixin@users.noreply.github.com> Date: Wed, 4 Jan 2023 22:48:28 +0800 Subject: [PATCH 04/20] Fix (#1125) --- cmd/yurthub/app/start.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cmd/yurthub/app/start.go b/cmd/yurthub/app/start.go index e40cb88fe3f..90a3213747d 100644 --- a/cmd/yurthub/app/start.go +++ b/cmd/yurthub/app/start.go @@ -234,7 +234,6 @@ func coordinatorRun(ctx context.Context, var coordinatorHealthChecker healthchecker.HealthChecker var coordinatorTransportMgr transport.Interface var coordinator poolcoordinator.Coordinator - var coordinatorCertManager *coordinatorcertmgr.CertManager var returnErr error readyCh := make(chan struct{}) @@ -247,7 +246,7 @@ func coordinatorRun(ctx context.Context, return } - coorTransportMgr, err := poolCoordinatorTransportMgrGetter(cfg.HeartbeatTimeoutSeconds, cfg.CoordinatorServerURL, coordinatorCertManager, ctx.Done()) + coorTransportMgr, err := poolCoordinatorTransportMgrGetter(cfg.HeartbeatTimeoutSeconds, cfg.CoordinatorServerURL, coorCertManager, ctx.Done()) if err != nil { returnErr = fmt.Errorf("failed to create coordinator transport manager, %v", err) return From 2bff621e13eb2873682971ec8c10fe73740e13f0 Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Tue, 10 Jan 2023 17:28:37 +0800 Subject: [PATCH 05/20] set isPoolCacheSynced as true if lease updated (#1131) Signed-off-by: Congrool Signed-off-by: Congrool --- pkg/yurthub/poolcoordinator/coordinator.go | 156 +++++++----- .../poolcoordinator/coordinator_test.go | 239 ++++++++++++++++++ 2 files changed, 338 insertions(+), 57 deletions(-) create mode 100644 pkg/yurthub/poolcoordinator/coordinator_test.go diff --git a/pkg/yurthub/poolcoordinator/coordinator.go b/pkg/yurthub/poolcoordinator/coordinator.go index 4cebbe9d7a5..d7b0df604f3 100644 --- a/pkg/yurthub/poolcoordinator/coordinator.go +++ b/pkg/yurthub/poolcoordinator/coordinator.go @@ -93,13 +93,13 @@ type coordinator struct { isPoolCacheSynced bool certMgr *certmanager.CertManager needUploadLocalCache bool - // poolScopeCacheSyncManager is used to sync pool-scoped resources from cloud to poolcoordinator. - poolScopeCacheSyncManager *poolScopedCacheSyncManager - // informerSyncLeaseManager is used to detect the leader-informer-sync lease - // to check its RenewTime. If its renewTime is not updated after defaultInformerLeaseRenewDuration - // we can think that the poolcoordinator cache is stale and the poolcoordinator is not ready. - // It will start if yurthub becomes leader or follower. - informerSyncLeaseManager *coordinatorLeaseInformerManager + // poolCacheSyncManager is used to sync pool-scoped resources from cloud to poolcoordinator. + poolCacheSyncManager *poolScopedCacheSyncManager + // poolCacheSyncedDector is used to detect if pool cache is synced and ready for use. + // It will list/watch the informer sync lease, and if it's renewed by leader yurthub, isPoolCacheSynced will + // be set as true which means the pool cache is ready for use. It also starts a routine which will set + // isPoolCacheSynced as false if the informer sync lease has not been updated for a duration. + poolCacheSyncedDetector *poolCacheSyncedDetector // delegateNodeLeaseManager is used to list/watch kube-node-lease from poolcoordinator. If the // node lease contains DelegateHeartBeat label, it will triger the eventhandler which will // use cloud client to send it to cloud APIServer. @@ -144,9 +144,19 @@ func NewCoordinator( hubElector: elector, } - informerSyncLeaseManager := &coordinatorLeaseInformerManager{ - ctx: ctx, - coordinatorClient: coordinatorClient, + poolCacheSyncedDetector := &poolCacheSyncedDetector{ + ctx: ctx, + updateNotifyCh: make(chan struct{}), + syncLeaseManager: &coordinatorLeaseInformerManager{ + ctx: ctx, + coordinatorClient: coordinatorClient, + }, + staleTimeout: defaultPoolCacheStaleDuration, + isPoolCacheSyncSetter: func(value bool) { + coordinator.Lock() + defer coordinator.Unlock() + coordinator.isPoolCacheSynced = value + }, } delegateNodeLeaseManager := &coordinatorLeaseInformerManager{ @@ -166,9 +176,9 @@ func NewCoordinator( getEtcdStore: coordinator.getEtcdStore, } - coordinator.informerSyncLeaseManager = informerSyncLeaseManager + coordinator.poolCacheSyncedDetector = poolCacheSyncedDetector coordinator.delegateNodeLeaseManager = delegateNodeLeaseManager - coordinator.poolScopeCacheSyncManager = poolScopedCacheSyncManager + coordinator.poolCacheSyncManager = poolScopedCacheSyncManager return coordinator, nil } @@ -185,9 +195,9 @@ func (coordinator *coordinator) Run() { select { case <-coordinator.ctx.Done(): - coordinator.poolScopeCacheSyncManager.EnsureStop() + coordinator.poolCacheSyncManager.EnsureStop() coordinator.delegateNodeLeaseManager.EnsureStop() - coordinator.informerSyncLeaseManager.EnsureStop() + coordinator.poolCacheSyncedDetector.EnsureStop() klog.Info("exit normally in coordinator loop.") return case electorStatus, ok := <-coordinator.hubElector.StatusChan(): @@ -197,9 +207,9 @@ func (coordinator *coordinator) Run() { switch electorStatus { case PendingHub: - coordinator.poolScopeCacheSyncManager.EnsureStop() + coordinator.poolCacheSyncManager.EnsureStop() coordinator.delegateNodeLeaseManager.EnsureStop() - coordinator.informerSyncLeaseManager.EnsureStop() + coordinator.poolCacheSyncedDetector.EnsureStop() needUploadLocalCache = true needCancelEtcdStorage = true isPoolCacheSynced = false @@ -217,11 +227,10 @@ func (coordinator *coordinator) Run() { klog.Errorf("cloud not get cloud lease client when becoming leader yurthub, %v", err) continue } - if err := coordinator.poolScopeCacheSyncManager.EnsureStart(); err != nil { + if err := coordinator.poolCacheSyncManager.EnsureStart(); err != nil { klog.Errorf("failed to sync pool-scoped resource, %v", err) continue } - coordinator.delegateNodeLeaseManager.EnsureStartWithHandler(cache.FilteringResourceEventHandler{ FilterFunc: ifDelegateHeartBeat, Handler: cache.ResourceEventHandlerFuncs{ @@ -233,20 +242,7 @@ func (coordinator *coordinator) Run() { }, }, }) - coordinator.informerSyncLeaseManager.EnsureStartWithHandler(cache.FilteringResourceEventHandler{ - FilterFunc: ifInformerSyncLease, - Handler: cache.ResourceEventHandlerFuncs{ - AddFunc: coordinator.detectPoolCacheSynced, - UpdateFunc: func(_, newObj interface{}) { - coordinator.detectPoolCacheSynced(newObj) - }, - DeleteFunc: func(_ interface{}) { - coordinator.Lock() - defer coordinator.Unlock() - coordinator.isPoolCacheSynced = false - }, - }, - }) + coordinator.poolCacheSyncedDetector.EnsureStart() if coordinator.needUploadLocalCache { if err := coordinator.uploadLocalCache(etcdStorage); err != nil { @@ -262,22 +258,9 @@ func (coordinator *coordinator) Run() { continue } - coordinator.poolScopeCacheSyncManager.EnsureStop() + coordinator.poolCacheSyncManager.EnsureStop() coordinator.delegateNodeLeaseManager.EnsureStop() - coordinator.informerSyncLeaseManager.EnsureStartWithHandler(cache.FilteringResourceEventHandler{ - FilterFunc: ifInformerSyncLease, - Handler: cache.ResourceEventHandlerFuncs{ - AddFunc: coordinator.detectPoolCacheSynced, - UpdateFunc: func(_, newObj interface{}) { - coordinator.detectPoolCacheSynced(newObj) - }, - DeleteFunc: func(_ interface{}) { - coordinator.Lock() - defer coordinator.Unlock() - coordinator.isPoolCacheSynced = false - }, - }, - }) + coordinator.poolCacheSyncedDetector.EnsureStart() if coordinator.needUploadLocalCache { if err := coordinator.uploadLocalCache(etcdStorage); err != nil { @@ -399,16 +382,6 @@ func (coordinator *coordinator) delegateNodeLease(cloudLeaseClient coordclientse } } -func (coordinator *coordinator) detectPoolCacheSynced(obj interface{}) { - lease := obj.(*coordinationv1.Lease) - renewTime := lease.Spec.RenewTime - if time.Now().After(renewTime.Add(defaultPoolCacheStaleDuration)) { - coordinator.Lock() - defer coordinator.Unlock() - coordinator.isPoolCacheSynced = false - } -} - // poolScopedCacheSyncManager will continuously sync pool-scoped resources from cloud to pool-coordinator. // After resource sync is completed, it will periodically renew the informer synced lease, which is used by // other yurthub to determine if pool-coordinator is ready to handle requests of pool-scoped resources. @@ -600,6 +573,75 @@ func (l *localCacheUploader) resourcesToUpload() map[storage.Key][]byte { return objBytes } +// poolCacheSyncedDector will list/watch informer-sync-lease to detect if pool cache can be used. +// The leader yurthub should periodically renew the lease. If the lease is not updated for staleTimeout +// duration, it will think the pool cache cannot be used. +type poolCacheSyncedDetector struct { + ctx context.Context + updateNotifyCh chan struct{} + isRunning bool + staleTimeout time.Duration + // syncLeaseManager is used to list/watch the informer-sync-lease, and set the + // isPoolCacheSync as ture when it is renewed. + syncLeaseManager *coordinatorLeaseInformerManager + isPoolCacheSyncSetter func(value bool) + cancelLoop func() +} + +func (p *poolCacheSyncedDetector) EnsureStart() { + if !p.isRunning { + p.syncLeaseManager.EnsureStartWithHandler(cache.FilteringResourceEventHandler{ + FilterFunc: ifInformerSyncLease, + Handler: cache.ResourceEventHandlerFuncs{ + AddFunc: p.detectPoolCacheSynced, + UpdateFunc: func(_, newObj interface{}) { + p.detectPoolCacheSynced(newObj) + }, + DeleteFunc: func(_ interface{}) { + p.isPoolCacheSyncSetter(false) + }, + }, + }) + + ctx, cancel := context.WithCancel(p.ctx) + p.cancelLoop = cancel + go p.loopForChange(ctx) + } +} + +func (p *poolCacheSyncedDetector) EnsureStop() { + if p.isRunning { + p.syncLeaseManager.EnsureStop() + p.cancelLoop() + } +} + +func (p *poolCacheSyncedDetector) loopForChange(ctx context.Context) { + t := time.NewTicker(p.staleTimeout) + defer t.Stop() + for { + select { + case <-ctx.Done(): + return + case <-p.updateNotifyCh: + t.Reset(p.staleTimeout) + p.isPoolCacheSyncSetter(true) + case <-t.C: + klog.V(4).Infof("timeout waitting for pool cache sync lease being updated, do not use pool cache") + p.isPoolCacheSyncSetter(false) + } + } +} + +func (p *poolCacheSyncedDetector) detectPoolCacheSynced(obj interface{}) { + lease := obj.(*coordinationv1.Lease) + renewTime := lease.Spec.RenewTime + if time.Now().Before(renewTime.Add(p.staleTimeout)) { + // The lease is updated before pool cache being considered as stale. + p.updateNotifyCh <- struct{}{} + } +} + func getRv(objBytes []byte) (uint64, error) { obj := &unstructured.Unstructured{} if err := json.Unmarshal(objBytes, obj); err != nil { diff --git a/pkg/yurthub/poolcoordinator/coordinator_test.go b/pkg/yurthub/poolcoordinator/coordinator_test.go new file mode 100644 index 00000000000..6ba49afd4eb --- /dev/null +++ b/pkg/yurthub/poolcoordinator/coordinator_test.go @@ -0,0 +1,239 @@ +/* +Copyright 2023 The OpenYurt Authors. + +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 poolcoordinator + +import ( + "context" + "sync" + "testing" + "time" + + coordinationv1 "k8s.io/api/coordination/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/client-go/kubernetes/fake" + "k8s.io/utils/pointer" +) + +var leaseGVR = schema.GroupVersionResource{ + Group: coordinationv1.SchemeGroupVersion.Group, + Version: coordinationv1.SchemeGroupVersion.Version, + Resource: "leases", +} + +func TestInformerSyncLeaseAddedAndUpdated(t *testing.T) { + var isPoolCacheSynced bool + var mtx sync.Mutex + var poolCacheSyncLease *coordinationv1.Lease = &coordinationv1.Lease{ + ObjectMeta: v1.ObjectMeta{ + Name: nameInformerLease, + Namespace: namespaceInformerLease, + }, + Spec: coordinationv1.LeaseSpec{}, + } + + cases := []struct { + Description string + LeaseUpdateInterval time.Duration + StaleTimeout time.Duration + LeaseUpdateTimes int + PollInterval time.Duration + Expect bool + }{ + { + Description: "should set isPoolCacheSynced as true if lease is updated before timeout", + LeaseUpdateInterval: 100 * time.Millisecond, + StaleTimeout: 2 * time.Second, + LeaseUpdateTimes: 10, + PollInterval: 50 * time.Millisecond, + Expect: true, + }, + { + Description: "should set isPoolCacheSynced as false is lease is not updated until timeout", + LeaseUpdateInterval: 100 * time.Millisecond, + StaleTimeout: 2 * time.Second, + LeaseUpdateTimes: 1, + PollInterval: 4 * time.Second, + Expect: false, + }, + } + + for _, c := range cases { + t.Run(c.Description, func(t *testing.T) { + ctx := context.Background() + fakeClient := fake.NewSimpleClientset() + exited := false + + poolCacheSyncedDetector := &poolCacheSyncedDetector{ + ctx: ctx, + updateNotifyCh: make(chan struct{}), + syncLeaseManager: &coordinatorLeaseInformerManager{ + ctx: ctx, + coordinatorClient: fakeClient, + }, + staleTimeout: c.StaleTimeout, + isPoolCacheSyncSetter: func(value bool) { + mtx.Lock() + defer mtx.Unlock() + isPoolCacheSynced = value + }, + } + + poolCacheSyncedDetector.EnsureStart() + defer poolCacheSyncedDetector.EnsureStop() + + go func() { + initLease := poolCacheSyncLease.DeepCopy() + initLease.Spec.RenewTime = &v1.MicroTime{ + Time: time.Now(), + } + if err := fakeClient.Tracker().Add(initLease); err != nil { + t.Errorf("failed to add lease at case %s, %v", c.Description, err) + } + for i := 0; i < c.LeaseUpdateTimes; i++ { + time.Sleep(c.LeaseUpdateInterval) + newLease := poolCacheSyncLease.DeepCopy() + newLease.Spec.RenewTime = &v1.MicroTime{ + Time: time.Now(), + } + if err := fakeClient.Tracker().Update(leaseGVR, newLease, namespaceInformerLease); err != nil { + t.Errorf("failed to update lease at case %s, %v", c.Description, err) + } + } + exited = true + }() + + ticker := time.NewTicker(c.PollInterval) + defer ticker.Stop() + for { + <-ticker.C + if isPoolCacheSynced != c.Expect { + t.Errorf("unexpected value at case: %s, want: %v, got: %v", c.Description, c.Expect, isPoolCacheSynced) + } + if exited { + return + } + } + }) + } +} + +func TestInformerSyncLeaseDelete(t *testing.T) { + t.Run("should set isPoolCacheSynced as false if the lease is deleted", func(t *testing.T) { + var isPoolCacheSynced bool + var mtx sync.Mutex + var poolCacheSyncLease *coordinationv1.Lease = &coordinationv1.Lease{ + ObjectMeta: v1.ObjectMeta{ + Name: nameInformerLease, + Namespace: namespaceInformerLease, + }, + Spec: coordinationv1.LeaseSpec{ + RenewTime: &v1.MicroTime{ + Time: time.Now(), + }, + }, + } + ctx := context.Background() + fakeClient := fake.NewSimpleClientset(poolCacheSyncLease) + poolCacheSyncedDetector := &poolCacheSyncedDetector{ + ctx: ctx, + updateNotifyCh: make(chan struct{}), + syncLeaseManager: &coordinatorLeaseInformerManager{ + ctx: ctx, + coordinatorClient: fakeClient, + }, + staleTimeout: 100 * time.Second, + isPoolCacheSyncSetter: func(value bool) { + mtx.Lock() + defer mtx.Unlock() + isPoolCacheSynced = value + }, + } + + poolCacheSyncedDetector.EnsureStart() + defer poolCacheSyncedDetector.EnsureStop() + + err := wait.PollUntil(50*time.Millisecond, func() (done bool, err error) { + if isPoolCacheSynced { + return true, nil + } + return false, nil + }, ctx.Done()) + if err != nil { + t.Errorf("failed to wait isPoolCacheSynced to be initialized as true") + } + + if err := fakeClient.Tracker().Delete(leaseGVR, namespaceInformerLease, nameInformerLease); err != nil { + t.Errorf("failed to delete lease, %v", err) + } + + err = wait.PollUntil(50*time.Millisecond, func() (done bool, err error) { + if isPoolCacheSynced { + return false, nil + } + return true, nil + }, ctx.Done()) + if err != nil { + t.Errorf("unexpect err when waitting isPoolCacheSynced to be false, %v", err) + } + }) +} + +func TestIfInformerSyncLease(t *testing.T) { + cases := []struct { + Description string + Lease *coordinationv1.Lease + Expect bool + }{ + { + Description: "return true if it is informer sync lease", + Lease: &coordinationv1.Lease{ + ObjectMeta: v1.ObjectMeta{ + Name: nameInformerLease, + Namespace: namespaceInformerLease, + }, + Spec: coordinationv1.LeaseSpec{ + HolderIdentity: pointer.StringPtr("leader-yurthub"), + }, + }, + Expect: true, + }, + { + Description: "return false if it is not informer sync lease", + Lease: &coordinationv1.Lease{ + ObjectMeta: v1.ObjectMeta{ + Name: "other-lease", + Namespace: "kube-system", + }, + Spec: coordinationv1.LeaseSpec{ + HolderIdentity: pointer.StringPtr("other-lease"), + }, + }, + Expect: false, + }, + } + + for _, c := range cases { + t.Run(c.Description, func(t *testing.T) { + got := ifInformerSyncLease(c.Lease) + if got != c.Expect { + t.Errorf("unexpected value for %s, want: %v, got: %v", c.Description, c.Expect, got) + } + }) + } +} From 39029b3753b0d852bd5912f04a4e0cdd30a8a390 Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Wed, 11 Jan 2023 10:06:55 +0800 Subject: [PATCH 06/20] bugfix: do not send pool-scoped resource list/watch request to pool-coordinator (#1134) Signed-off-by: Congrool Signed-off-by: Congrool --- pkg/yurthub/proxy/proxy.go | 10 ++++++++++ pkg/yurthub/proxy/remote/loadbalancer.go | 14 +++++++++++++- 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/pkg/yurthub/proxy/proxy.go b/pkg/yurthub/proxy/proxy.go index 75168388717..b112ce62dcc 100644 --- a/pkg/yurthub/proxy/proxy.go +++ b/pkg/yurthub/proxy/proxy.go @@ -37,6 +37,7 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/cachemanager" "github.com/openyurtio/openyurt/pkg/yurthub/healthchecker" "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator" + coordinatorconstants "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" "github.com/openyurtio/openyurt/pkg/yurthub/proxy/local" "github.com/openyurtio/openyurt/pkg/yurthub/proxy/pool" "github.com/openyurtio/openyurt/pkg/yurthub/proxy/remote" @@ -217,6 +218,15 @@ func (p *yurtReverseProxy) eventHandler(rw http.ResponseWriter, req *http.Reques } func (p *yurtReverseProxy) poolScopedResouceHandler(rw http.ResponseWriter, req *http.Request) { + agent, ok := hubutil.ClientComponentFrom(req.Context()) + if ok && agent == coordinatorconstants.DefaultPoolScopedUserAgent { + // list/watch request from leader-yurthub + // It should always be proxied to cloud APIServer to get latest resource, which will + // be cached into pool cache. + p.loadBalancer.ServeHTTP(rw, req) + return + } + if p.isCoordinatorReady() && p.poolProxy != nil { p.poolProxy.ServeHTTP(rw, req) } else if p.cloudHealthChecker.IsHealthy() { diff --git a/pkg/yurthub/proxy/remote/loadbalancer.go b/pkg/yurthub/proxy/remote/loadbalancer.go index 9029a1eaa9d..7a3f89efc46 100644 --- a/pkg/yurthub/proxy/remote/loadbalancer.go +++ b/pkg/yurthub/proxy/remote/loadbalancer.go @@ -194,7 +194,10 @@ func (lb *loadBalancer) ServeHTTP(rw http.ResponseWriter, req *http.Request) { return } klog.V(3).Infof("picked backend %s by %s for request %s", rp.Name(), lb.algo.Name(), hubutil.ReqString(req)) - if util.IsPoolScopedResouceListWatchRequest(req) { + // If pool-scoped resource request is from leader-yurthub, it should always be sent to the cloud APIServer. + // Thus we do not need to start a check routine for it. But for other requests, we need to periodically check + // the pool-coordinator status, and switch the traffic to pool-coordinator if it is ready. + if util.IsPoolScopedResouceListWatchRequest(req) && !isRequestFromLeaderYurthub(req) { // We get here possibly because the pool-coordinator is not ready. // We should cancel the watch request when pool-coordinator becomes ready. klog.Infof("pool-coordinator is not ready, we use cloud APIServer to temporarily handle the req: %s", hubutil.ReqString(req)) @@ -420,3 +423,12 @@ func isRequestOfNodeAndPod(reqCtx context.Context) bool { return (reqInfo.Resource == "nodes" && reqInfo.APIGroup == "" && reqInfo.APIVersion == "v1") || (reqInfo.Resource == "pods" && reqInfo.APIGroup == "" && reqInfo.APIVersion == "v1") } + +func isRequestFromLeaderYurthub(req *http.Request) bool { + ctx := req.Context() + agent, ok := hubutil.ClientComponentFrom(ctx) + if !ok { + return false + } + return agent == coordinatorconstants.DefaultPoolScopedUserAgent +} From 0a8659cbbbeed84cce832be9f07cebf10477383b Mon Sep 17 00:00:00 2001 From: Laurence <45508533+LaurenceLiZhixin@users.noreply.github.com> Date: Wed, 11 Jan 2023 15:51:48 +0800 Subject: [PATCH 07/20] Fix: pool-coordinator (#1126) * Fix: pool-coordinator --- cmd/yurthub/app/start.go | 56 ++++++------- pkg/yurthub/poolcoordinator/coordinator.go | 33 ++++---- .../poolcoordinator/leader_election.go | 4 +- pkg/yurthub/proxy/pool/pool.go | 52 ++++++++---- pkg/yurthub/proxy/proxy.go | 68 +++++++++------- pkg/yurthub/proxy/remote/loadbalancer.go | 54 +++++++++---- pkg/yurthub/storage/etcd/keycache.go | 81 ++++++++++--------- 7 files changed, 204 insertions(+), 144 deletions(-) diff --git a/cmd/yurthub/app/start.go b/cmd/yurthub/app/start.go index 90a3213747d..1f0ef5bdd55 100644 --- a/cmd/yurthub/app/start.go +++ b/cmd/yurthub/app/start.go @@ -148,30 +148,30 @@ func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { tenantMgr := tenant.New(cfg.TenantNs, cfg.SharedFactory, ctx.Done()) trace++ - var coordinator poolcoordinator.Coordinator = &poolcoordinator.FakeCoordinator{} - var coordinatorHealthChecker healthchecker.HealthChecker = healthchecker.NewFakeChecker(false, make(map[string]int)) - var coordinatorTransportManager transport.Interface = nil - var waittingForCoordinator func() (healthchecker.HealthChecker, transport.Interface, poolcoordinator.Coordinator, error) = nil + var coordinatorGetter func() poolcoordinator.Coordinator + var coordinatorHealthCheckerGetter func() healthchecker.HealthChecker + var coordinatorTransportManagerGetter func() transport.Interface = nil + coordinatorInformerRegistryChan := make(chan struct{}) + if cfg.EnableCoordinator { klog.Infof("%d. start to run coordinator", trace) // coordinatorRun will register secret informer into sharedInformerFactory, and start a new goroutine to periodically check // if certs has been got from cloud APIServer. - waittingForCoordinator = coordinatorRun(ctx, cfg, restConfigMgr, cloudHealthChecker) trace++ - } - - // Start the informer factory if all informers have been registered - cfg.SharedFactory.Start(ctx.Done()) - cfg.YurtSharedFactory.Start(ctx.Done()) - - if waittingForCoordinator != nil { // Waitting for the coordinator to run, before using it to create other components. - coordinatorHealthChecker, coordinatorTransportManager, coordinator, err = waittingForCoordinator() + coordinatorHealthCheckerGetter, coordinatorTransportManagerGetter, coordinatorGetter, err = coordinatorRun(ctx, cfg, restConfigMgr, cloudHealthChecker, coordinatorInformerRegistryChan) if err != nil { return fmt.Errorf("failed to wait for coordinator to run, %v", err) } } + // wait for coordinator informer registry + <-coordinatorInformerRegistryChan + + // Start the informer factory if all informers have been registered + cfg.SharedFactory.Start(ctx.Done()) + cfg.YurtSharedFactory.Start(ctx.Done()) + klog.Infof("%d. new reverse proxy handler for remote servers", trace) yurtProxyHandler, err := proxy.NewYurtReverseProxyHandler( cfg, @@ -179,9 +179,9 @@ func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { transportManager, cloudHealthChecker, tenantMgr, - coordinator, - coordinatorTransportManager, - coordinatorHealthChecker, + coordinatorGetter, + coordinatorTransportManagerGetter, + coordinatorHealthCheckerGetter, ctx.Done()) if err != nil { return fmt.Errorf("could not create reverse proxy handler, %w", err) @@ -230,14 +230,13 @@ func createClients(heartbeatTimeoutSeconds int, remoteServers []*url.URL, coordi func coordinatorRun(ctx context.Context, cfg *config.YurtHubConfiguration, restConfigMgr *hubrest.RestConfigManager, - cloudHealthChecker healthchecker.MultipleBackendsHealthChecker) (waittingForReady func() (healthchecker.HealthChecker, transport.Interface, poolcoordinator.Coordinator, error)) { + cloudHealthChecker healthchecker.MultipleBackendsHealthChecker, + coordinatorInformerRegistryChan chan struct{}) (func() healthchecker.HealthChecker, func() transport.Interface, func() poolcoordinator.Coordinator, error) { var coordinatorHealthChecker healthchecker.HealthChecker var coordinatorTransportMgr transport.Interface var coordinator poolcoordinator.Coordinator var returnErr error - readyCh := make(chan struct{}) - go func() { coorCertManager, err := coordinatorcertmgr.NewCertManager(cfg.CoordinatorPKIDir, cfg.ProxiedClient, cfg.SharedFactory) close(coordinatorInformerRegistryChan) // notify the coordinator secret informer registry event @@ -255,7 +254,7 @@ func coordinatorRun(ctx context.Context, coordinatorClient, err := kubernetes.NewForConfig(&rest.Config{ Host: cfg.CoordinatorServerURL.String(), Transport: coorTransportMgr.CurrentTransport(), - Timeout: time.Duration(cfg.HeartbeatTimeoutSeconds), + Timeout: time.Duration(cfg.HeartbeatTimeoutSeconds) * time.Second, }) if err != nil { returnErr = fmt.Errorf("failed to get coordinator client for pool coordinator, %v", err) @@ -274,14 +273,14 @@ func coordinatorRun(ctx context.Context, returnErr = fmt.Errorf("failed to create hub elector, %v", err) return } - elector.Run(ctx.Done()) + go elector.Run(ctx.Done()) coor, err := poolcoordinator.NewCoordinator(ctx, cfg, restConfigMgr, coorCertManager, coorTransportMgr, elector) if err != nil { returnErr = fmt.Errorf("failed to create coordinator, %v", err) return } - coor.Run() + go coor.Run() coordinatorTransportMgr = coorTransportMgr coordinatorHealthChecker = coorHealthChecker @@ -289,12 +288,13 @@ func coordinatorRun(ctx context.Context, returnErr = nil }() - waittingForReady = func() (healthchecker.HealthChecker, transport.Interface, poolcoordinator.Coordinator, error) { - <-readyCh - return coordinatorHealthChecker, coordinatorTransportMgr, coordinator, returnErr - } - - return waittingForReady + return func() healthchecker.HealthChecker { + return coordinatorHealthChecker + }, func() transport.Interface { + return coordinatorTransportMgr + }, func() poolcoordinator.Coordinator { + return coordinator + }, returnErr } func poolCoordinatorTransportMgrGetter(heartbeatTimeoutSeconds int, coordinatorServer *url.URL, coordinatorCertMgr *coordinatorcertmgr.CertManager, stopCh <-chan struct{}) (transport.Interface, error) { diff --git a/pkg/yurthub/poolcoordinator/coordinator.go b/pkg/yurthub/poolcoordinator/coordinator.go index d7b0df604f3..d70f9b357dc 100644 --- a/pkg/yurthub/poolcoordinator/coordinator.go +++ b/pkg/yurthub/poolcoordinator/coordinator.go @@ -171,7 +171,7 @@ func NewCoordinator( poolScopedCacheSyncManager := &poolScopedCacheSyncManager{ ctx: ctx, proxiedClient: proxiedClient, - coordinatorClient: cfg.CoordinatorClient, + coordinatorClient: coordinatorClient, nodeName: cfg.NodeName, getEtcdStore: coordinator.getEtcdStore, } @@ -185,12 +185,10 @@ func NewCoordinator( func (coordinator *coordinator) Run() { for { - var poolCacheManager cachemanager.CacheManager - var cancelEtcdStorage func() + var cancelEtcdStorage = func() {} var needUploadLocalCache bool var needCancelEtcdStorage bool var isPoolCacheSynced bool - var etcdStorage storage.Store var err error select { @@ -213,10 +211,13 @@ func (coordinator *coordinator) Run() { needUploadLocalCache = true needCancelEtcdStorage = true isPoolCacheSynced = false - etcdStorage = nil - poolCacheManager = nil + coordinator.Lock() + coordinator.poolCacheManager = nil + coordinator.Unlock() case LeaderHub: - poolCacheManager, etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() + coordinator.Lock() + coordinator.poolCacheManager, coordinator.etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() + coordinator.Unlock() if err != nil { klog.Errorf("failed to create pool scoped cache store and manager, %v", err) continue @@ -245,14 +246,16 @@ func (coordinator *coordinator) Run() { coordinator.poolCacheSyncedDetector.EnsureStart() if coordinator.needUploadLocalCache { - if err := coordinator.uploadLocalCache(etcdStorage); err != nil { + if err := coordinator.uploadLocalCache(coordinator.etcdStorage); err != nil { klog.Errorf("failed to upload local cache when yurthub becomes leader, %v", err) } else { needUploadLocalCache = false } } case FollowerHub: - poolCacheManager, etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() + coordinator.Lock() + coordinator.poolCacheManager, coordinator.etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() + coordinator.Unlock() if err != nil { klog.Errorf("failed to create pool scoped cache store and manager, %v", err) continue @@ -263,7 +266,7 @@ func (coordinator *coordinator) Run() { coordinator.poolCacheSyncedDetector.EnsureStart() if coordinator.needUploadLocalCache { - if err := coordinator.uploadLocalCache(etcdStorage); err != nil { + if err := coordinator.uploadLocalCache(coordinator.etcdStorage); err != nil { klog.Errorf("failed to upload local cache when yurthub becomes follower, %v", err) } else { needUploadLocalCache = false @@ -276,11 +279,9 @@ func (coordinator *coordinator) Run() { // Because the caller of IsReady() may be concurrent. coordinator.Lock() if needCancelEtcdStorage { - coordinator.cancelEtcdStorage() + cancelEtcdStorage() } coordinator.electStatus = electorStatus - coordinator.poolCacheManager = poolCacheManager - coordinator.etcdStorage = etcdStorage coordinator.cancelEtcdStorage = cancelEtcdStorage coordinator.needUploadLocalCache = needUploadLocalCache coordinator.isPoolCacheSynced = isPoolCacheSynced @@ -299,7 +300,8 @@ func (coordinator *coordinator) IsReady() (cachemanager.CacheManager, bool) { // If electStatus is not PendingHub, it means pool-coordinator is healthy. coordinator.Lock() defer coordinator.Unlock() - if coordinator.electStatus != PendingHub && coordinator.isPoolCacheSynced && !coordinator.needUploadLocalCache { + // fixme: coordinator.isPoolCacheSynced now is not considered + if coordinator.electStatus != PendingHub && !coordinator.needUploadLocalCache { return coordinator.poolCacheManager, true } return nil, false @@ -403,7 +405,8 @@ type poolScopedCacheSyncManager struct { func (p *poolScopedCacheSyncManager) EnsureStart() error { if !p.isRunning { - if err := p.coordinatorClient.CoordinationV1().Leases(namespaceInformerLease).Delete(p.ctx, nameInformerLease, metav1.DeleteOptions{}); err != nil { + err := p.coordinatorClient.CoordinationV1().Leases(namespaceInformerLease).Delete(p.ctx, nameInformerLease, metav1.DeleteOptions{}) + if err != nil && !apierrors.IsNotFound(err) { return fmt.Errorf("failed to delete informer sync lease, %v", err) } diff --git a/pkg/yurthub/poolcoordinator/leader_election.go b/pkg/yurthub/poolcoordinator/leader_election.go index 565caf4c9ac..ffafe9a39bf 100644 --- a/pkg/yurthub/poolcoordinator/leader_election.go +++ b/pkg/yurthub/poolcoordinator/leader_election.go @@ -55,7 +55,7 @@ func NewHubElector( coordinatorClient: coordinatorClient, coordinatorHealthChecker: coordinatorHealthChecker, cloudAPIServerHealthChecker: cloudAPIServerHealthyChecker, - electorStatus: make(chan int32), + electorStatus: make(chan int32, 1), } rl, err := resourcelock.New(cfg.LeaderElection.ResourceLock, @@ -75,7 +75,7 @@ func NewHubElector( RetryPeriod: cfg.LeaderElection.RetryPeriod.Duration, Callbacks: leaderelection.LeaderCallbacks{ OnStartedLeading: func(ctx context.Context) { - klog.Infof("yurthub of %s became lease", cfg.NodeName) + klog.Infof("yurthub of %s became leader", cfg.NodeName) he.electorStatus <- LeaderHub }, OnStoppedLeading: func() { diff --git a/pkg/yurthub/proxy/pool/pool.go b/pkg/yurthub/proxy/pool/pool.go index e81eec4129f..6080dc5da0e 100644 --- a/pkg/yurthub/proxy/pool/pool.go +++ b/pkg/yurthub/proxy/pool/pool.go @@ -51,7 +51,7 @@ type PoolCoordinatorProxy struct { func NewPoolCoordinatorProxy( poolCoordinatorAddr *url.URL, localCacheMgr cachemanager.CacheManager, - transportMgr transport.Interface, + transportMgrGetter func() transport.Interface, filterMgr *manager.Manager, isCoordinatorReady func() bool, stopCh <-chan struct{}) (*PoolCoordinatorProxy, error) { @@ -66,17 +66,33 @@ func NewPoolCoordinatorProxy( stopCh: stopCh, } - proxy, err := util.NewRemoteProxy( - poolCoordinatorAddr, - pp.modifyResponse, - pp.errorHandler, - transportMgr, - stopCh) - if err != nil { - return nil, fmt.Errorf("failed to create remote proxy for pool-coordinator, %v", err) - } + go func() { + ticker := time.NewTicker(time.Second * 5) + for { + select { + case <-ticker.C: + transportMgr := transportMgrGetter() + if transportMgr == nil { + break + } + proxy, err := util.NewRemoteProxy( + poolCoordinatorAddr, + pp.modifyResponse, + pp.errorHandler, + transportMgr, + stopCh) + if err != nil { + klog.Errorf("failed to create remote proxy for pool-coordinator, %v", err) + return + } + + pp.poolCoordinatorProxy = proxy + klog.Infof("create remote proxy for pool-coordinator success") + return + } + } + }() - pp.poolCoordinatorProxy = proxy return pp, nil } @@ -117,7 +133,7 @@ func (pp *PoolCoordinatorProxy) poolPost(rw http.ResponseWriter, req *http.Reque ctx := req.Context() info, _ := apirequest.RequestInfoFrom(ctx) klog.V(4).Infof("pool handle post, req=%s, reqInfo=%s", hubutil.ReqString(req), hubutil.ReqInfoString(info)) - if util.IsSubjectAccessReviewCreateGetRequest(req) || util.IsEventCreateRequest(req) { + if (util.IsSubjectAccessReviewCreateGetRequest(req) || util.IsEventCreateRequest(req)) && pp.poolCoordinatorProxy != nil { // kubelet needs to create subjectaccessreviews for auth pp.poolCoordinatorProxy.ServeHTTP(rw, req) return nil @@ -127,7 +143,7 @@ func (pp *PoolCoordinatorProxy) poolPost(rw http.ResponseWriter, req *http.Reque } func (pp *PoolCoordinatorProxy) poolQuery(rw http.ResponseWriter, req *http.Request) error { - if util.IsPoolScopedResouceListWatchRequest(req) || util.IsSubjectAccessReviewCreateGetRequest(req) { + if (util.IsPoolScopedResouceListWatchRequest(req) || util.IsSubjectAccessReviewCreateGetRequest(req)) && pp.poolCoordinatorProxy != nil { pp.poolCoordinatorProxy.ServeHTTP(rw, req) return nil } @@ -135,7 +151,7 @@ func (pp *PoolCoordinatorProxy) poolQuery(rw http.ResponseWriter, req *http.Requ } func (pp *PoolCoordinatorProxy) poolWatch(rw http.ResponseWriter, req *http.Request) error { - if util.IsPoolScopedResouceListWatchRequest(req) { + if util.IsPoolScopedResouceListWatchRequest(req) && pp.poolCoordinatorProxy != nil { clientReqCtx := req.Context() poolServeCtx, poolServeCancel := context.WithCancel(clientReqCtx) @@ -243,7 +259,7 @@ func (pp *PoolCoordinatorProxy) cacheResponse(req *http.Request, resp *http.Resp if pp.localCacheMgr.CanCacheFor(req) { ctx := req.Context() req = req.WithContext(ctx) - wrapPrc, _ := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "cache-manager") + wrapPrc, needUncompressed := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "cache-manager") rc, prc := hubutil.NewDualReadCloser(req, wrapPrc, true) go func(req *http.Request, prc io.ReadCloser, stopCh <-chan struct{}) { @@ -251,6 +267,12 @@ func (pp *PoolCoordinatorProxy) cacheResponse(req *http.Request, resp *http.Resp klog.Errorf("failed to cache req %s in local cache when cluster is unhealthy, %v", hubutil.ReqString(req), err) } }(req, prc, ctx.Done()) + + // after gunzip in filter, the header content encoding should be removed. + // because there's no need to gunzip response.body again. + if needUncompressed { + resp.Header.Del("Content-Encoding") + } resp.Body = rc } } diff --git a/pkg/yurthub/proxy/proxy.go b/pkg/yurthub/proxy/proxy.go index b112ce62dcc..a04601d2c01 100644 --- a/pkg/yurthub/proxy/proxy.go +++ b/pkg/yurthub/proxy/proxy.go @@ -48,17 +48,17 @@ import ( ) type yurtReverseProxy struct { - resolver apirequest.RequestInfoResolver - loadBalancer remote.LoadBalancer - cloudHealthChecker healthchecker.MultipleBackendsHealthChecker - coordinatorHealtChecker healthchecker.HealthChecker - localProxy http.Handler - poolProxy http.Handler - maxRequestsInFlight int - tenantMgr tenant.Interface - isCoordinatorReady func() bool - workingMode hubutil.WorkingMode - enablePoolCoordinator bool + resolver apirequest.RequestInfoResolver + loadBalancer remote.LoadBalancer + cloudHealthChecker healthchecker.MultipleBackendsHealthChecker + coordinatorHealtCheckerGetter func() healthchecker.HealthChecker + localProxy http.Handler + poolProxy http.Handler + maxRequestsInFlight int + tenantMgr tenant.Interface + isCoordinatorReady func() bool + workingMode hubutil.WorkingMode + enablePoolCoordinator bool } // NewYurtReverseProxyHandler creates a http handler for proxying @@ -69,9 +69,9 @@ func NewYurtReverseProxyHandler( transportMgr transport.Interface, cloudHealthChecker healthchecker.MultipleBackendsHealthChecker, tenantMgr tenant.Interface, - coordinator poolcoordinator.Coordinator, - coordinatorTransportMgr transport.Interface, - coordinatorHealthChecker healthchecker.HealthChecker, + coordinatorGetter func() poolcoordinator.Coordinator, + coordinatorTransportMgrGetter func() transport.Interface, + coordinatorHealthCheckerGetter func() healthchecker.HealthChecker, stopCh <-chan struct{}) (http.Handler, error) { cfg := &server.Config{ LegacyAPIGroupPrefixes: sets.NewString(server.DefaultLegacyAPIPrefix), @@ -83,7 +83,7 @@ func NewYurtReverseProxyHandler( yurtHubCfg.RemoteServers, localCacheMgr, transportMgr, - coordinator, + coordinatorGetter, cloudHealthChecker, yurtHubCfg.FilterManager, yurtHubCfg.WorkingMode, @@ -94,10 +94,18 @@ func NewYurtReverseProxyHandler( var localProxy, poolProxy http.Handler isCoordinatorHealthy := func() bool { + coordinator := coordinatorGetter() + if coordinator == nil { + return false + } _, healthy := coordinator.IsHealthy() return healthy } isCoordinatorReady := func() bool { + coordinator := coordinatorGetter() + if coordinator == nil { + return false + } _, ready := coordinator.IsReady() return ready } @@ -116,7 +124,7 @@ func NewYurtReverseProxyHandler( poolProxy, err = pool.NewPoolCoordinatorProxy( yurtHubCfg.CoordinatorServerURL, localCacheMgr, - coordinatorTransportMgr, + coordinatorTransportMgrGetter, yurtHubCfg.FilterManager, isCoordinatorReady, stopCh) @@ -127,17 +135,17 @@ func NewYurtReverseProxyHandler( } yurtProxy := &yurtReverseProxy{ - resolver: resolver, - loadBalancer: lb, - cloudHealthChecker: cloudHealthChecker, - coordinatorHealtChecker: coordinatorHealthChecker, - localProxy: localProxy, - poolProxy: poolProxy, - maxRequestsInFlight: yurtHubCfg.MaxRequestInFlight, - isCoordinatorReady: isCoordinatorReady, - enablePoolCoordinator: yurtHubCfg.EnableCoordinator, - tenantMgr: tenantMgr, - workingMode: yurtHubCfg.WorkingMode, + resolver: resolver, + loadBalancer: lb, + cloudHealthChecker: cloudHealthChecker, + coordinatorHealtCheckerGetter: coordinatorHealthCheckerGetter, + localProxy: localProxy, + poolProxy: poolProxy, + maxRequestsInFlight: yurtHubCfg.MaxRequestInFlight, + isCoordinatorReady: isCoordinatorReady, + enablePoolCoordinator: yurtHubCfg.EnableCoordinator, + tenantMgr: tenantMgr, + workingMode: yurtHubCfg.WorkingMode, } return yurtProxy.buildHandlerChain(yurtProxy), nil @@ -200,7 +208,11 @@ func (p *yurtReverseProxy) ServeHTTP(rw http.ResponseWriter, req *http.Request) func (p *yurtReverseProxy) handleKubeletLease(rw http.ResponseWriter, req *http.Request) { p.cloudHealthChecker.RenewKubeletLeaseTime() - p.coordinatorHealtChecker.RenewKubeletLeaseTime() + coordinatorHealtChecker := p.coordinatorHealtCheckerGetter() + if coordinatorHealtChecker != nil { + coordinatorHealtChecker.RenewKubeletLeaseTime() + } + if p.localProxy != nil { p.localProxy.ServeHTTP(rw, req) } diff --git a/pkg/yurthub/proxy/remote/loadbalancer.go b/pkg/yurthub/proxy/remote/loadbalancer.go index 7a3f89efc46..648f02f2270 100644 --- a/pkg/yurthub/proxy/remote/loadbalancer.go +++ b/pkg/yurthub/proxy/remote/loadbalancer.go @@ -128,13 +128,13 @@ type LoadBalancer interface { } type loadBalancer struct { - backends []*util.RemoteProxy - algo loadBalancerAlgo - localCacheMgr cachemanager.CacheManager - filterManager *manager.Manager - coordinator poolcoordinator.Coordinator - workingMode hubutil.WorkingMode - stopCh <-chan struct{} + backends []*util.RemoteProxy + algo loadBalancerAlgo + localCacheMgr cachemanager.CacheManager + filterManager *manager.Manager + coordinatorGetter func() poolcoordinator.Coordinator + workingMode hubutil.WorkingMode + stopCh <-chan struct{} } // NewLoadBalancer creates a loadbalancer for specified remote servers @@ -143,17 +143,17 @@ func NewLoadBalancer( remoteServers []*url.URL, localCacheMgr cachemanager.CacheManager, transportMgr transport.Interface, - coordinator poolcoordinator.Coordinator, + coordinatorGetter func() poolcoordinator.Coordinator, healthChecker healthchecker.MultipleBackendsHealthChecker, filterManager *manager.Manager, workingMode hubutil.WorkingMode, stopCh <-chan struct{}) (LoadBalancer, error) { lb := &loadBalancer{ - localCacheMgr: localCacheMgr, - filterManager: filterManager, - coordinator: coordinator, - workingMode: workingMode, - stopCh: stopCh, + localCacheMgr: localCacheMgr, + filterManager: filterManager, + coordinatorGetter: coordinatorGetter, + workingMode: workingMode, + stopCh: stopCh, } backends := make([]*util.RemoteProxy, 0, len(remoteServers)) for i := range remoteServers { @@ -194,6 +194,7 @@ func (lb *loadBalancer) ServeHTTP(rw http.ResponseWriter, req *http.Request) { return } klog.V(3).Infof("picked backend %s by %s for request %s", rp.Name(), lb.algo.Name(), hubutil.ReqString(req)) + // If pool-scoped resource request is from leader-yurthub, it should always be sent to the cloud APIServer. // Thus we do not need to start a check routine for it. But for other requests, we need to periodically check // the pool-coordinator status, and switch the traffic to pool-coordinator if it is ready. @@ -210,7 +211,14 @@ func (lb *loadBalancer) ServeHTTP(rw http.ResponseWriter, req *http.Request) { for { select { case <-t.C: - if _, isReady := lb.coordinator.IsReady(); isReady { + if lb.coordinatorGetter == nil { + continue + } + coordinator := lb.coordinatorGetter() + if coordinator == nil { + continue + } + if _, isReady := coordinator.IsReady(); isReady { klog.Infof("notified the pool coordinator is ready, cancel the req %s making it handled by pool coordinator", hubutil.ReqString(req)) cloudServeCancel() return @@ -326,10 +334,24 @@ func (lb *loadBalancer) modifyResponse(resp *http.Response) error { func (lb *loadBalancer) cacheResponse(req *http.Request, resp *http.Response) { if lb.localCacheMgr.CanCacheFor(req) { ctx := req.Context() - wrapPrc, _ := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "cache-manager") + wrapPrc, needUncompressed := hubutil.NewGZipReaderCloser(resp.Header, resp.Body, req, "cache-manager") + // after gunzip in filter, the header content encoding should be removed. + // because there's no need to gunzip response.body again. + if needUncompressed { + resp.Header.Del("Content-Encoding") + } resp.Body = wrapPrc - poolCacheManager, isHealthy := lb.coordinator.IsHealthy() + var poolCacheManager cachemanager.CacheManager + var isHealthy bool + + coordinator := lb.coordinatorGetter() + if coordinator == nil { + isHealthy = false + } else { + poolCacheManager, isHealthy = coordinator.IsHealthy() + } + if isHealthy && poolCacheManager != nil { if !isLeaderHubUserAgent(ctx) { if isRequestOfNodeAndPod(ctx) { diff --git a/pkg/yurthub/storage/etcd/keycache.go b/pkg/yurthub/storage/etcd/keycache.go index 5a9c21ca227..8122c7f7b66 100644 --- a/pkg/yurthub/storage/etcd/keycache.go +++ b/pkg/yurthub/storage/etcd/keycache.go @@ -120,48 +120,49 @@ func (c *componentKeyCache) Recover() error { } func (c *componentKeyCache) getPoolScopedKeyset() (*keySet, error) { - client := c.getEtcdClient() - if client == nil { - return nil, fmt.Errorf("got empty etcd client") - } + // FIXME: now getEtcdClient would cause nil pointer + //client := c.getEtcdClient() + //if client == nil { + // return nil, fmt.Errorf("got empty etcd client") + //} keys := &keySet{m: map[storageKey]struct{}{}} - for gvr := range coordinatorconstants.PoolScopedResources { - getCtx, cancel := context.WithTimeout(c.ctx, defaultTimeout) - defer cancel() - rootKey, err := c.keyFunc(storage.KeyBuildInfo{ - Component: coordinatorconstants.DefaultPoolScopedUserAgent, - Group: gvr.Group, - Version: gvr.Version, - Resources: gvr.Resource, - }) - if err != nil { - return nil, fmt.Errorf("failed to generate keys for %s, %v", gvr.String(), err) - } - getResp, err := client.Get(getCtx, rootKey.Key(), clientv3.WithPrefix(), clientv3.WithKeysOnly()) - if err != nil { - return nil, fmt.Errorf("failed to get from etcd for %s, %v", gvr.String(), err) - } - - for _, kv := range getResp.Kvs { - ns, name, err := getNamespaceAndNameFromKeyPath(string(kv.Key)) - if err != nil { - return nil, fmt.Errorf("failed to parse namespace and name of %s", kv.Key) - } - key, err := c.keyFunc(storage.KeyBuildInfo{ - Component: coordinatorconstants.DefaultPoolScopedUserAgent, - Group: gvr.Group, - Version: gvr.Version, - Resources: gvr.Resource, - Namespace: ns, - Name: name, - }) - if err != nil { - return nil, fmt.Errorf("failed to create resource key for %v", kv.Key) - } - keys.m[key.(storageKey)] = struct{}{} - } - } + //for gvr := range coordinatorconstants.PoolScopedResources { + //getCtx, cancel := context.WithTimeout(c.ctx, defaultTimeout) + //defer cancel() + //rootKey, err := c.keyFunc(storage.KeyBuildInfo{ + // Component: coordinatorconstants.DefaultPoolScopedUserAgent, + // Group: gvr.Group, + // Version: gvr.Version, + // Resources: gvr.Resource, + //}) + //if err != nil { + // return nil, fmt.Errorf("failed to generate keys for %s, %v", gvr.String(), err) + //} + //getResp, err := client.Get(getCtx, rootKey.Key(), clientv3.WithPrefix(), clientv3.WithKeysOnly()) + //if err != nil { + // return nil, fmt.Errorf("failed to get from etcd for %s, %v", gvr.String(), err) + //} + // + //for _, kv := range getResp.Kvs { + // ns, name, err := getNamespaceAndNameFromKeyPath(string(kv.Key)) + // if err != nil { + // return nil, fmt.Errorf("failed to parse namespace and name of %s", kv.Key) + // } + // key, err := c.keyFunc(storage.KeyBuildInfo{ + // Component: coordinatorconstants.DefaultPoolScopedUserAgent, + // Group: gvr.Group, + // Version: gvr.Version, + // Resources: gvr.Resource, + // Namespace: ns, + // Name: name, + // }) + // if err != nil { + // return nil, fmt.Errorf("failed to create resource key for %v", kv.Key) + // } + // keys.m[key.(storageKey)] = struct{}{} + //} + //} return keys, nil } From 2c1d57baca1caaa7f2359f9d74eb9c68b0b9a4ee Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Wed, 11 Jan 2023 19:38:42 +0800 Subject: [PATCH 08/20] Bugfix of pool-coordinator and enable unit test to pass (#1137) * initialize etcdClient of componentKeyCache Signed-off-by: Congrool --- go.mod | 1 + go.sum | 1 + pkg/profile/profile_test.go | 24 ++-- pkg/yurthub/storage/disk/storage.go | 6 +- pkg/yurthub/storage/disk/storage_test.go | 4 + pkg/yurthub/storage/etcd/etcd_suite_test.go | 46 ++++++- pkg/yurthub/storage/etcd/key_test.go | 15 +++ pkg/yurthub/storage/etcd/keycache.go | 135 +++++++++---------- pkg/yurthub/storage/etcd/keycache_test.go | 140 +++++++++++++++++++- pkg/yurthub/storage/etcd/mock/kv.go | 60 +++++++++ pkg/yurthub/storage/etcd/storage.go | 51 ++++--- pkg/yurthub/storage/etcd/storage_test.go | 10 +- 12 files changed, 378 insertions(+), 115 deletions(-) create mode 100644 pkg/yurthub/storage/etcd/mock/kv.go diff --git a/go.mod b/go.mod index 46ea2c50d24..c909697fa7b 100644 --- a/go.mod +++ b/go.mod @@ -27,6 +27,7 @@ require ( github.com/stretchr/testify v1.7.0 github.com/vishvananda/netlink v1.1.1-0.20200603190939-5a869a71f0cb github.com/wI2L/jsondiff v0.3.0 + go.etcd.io/etcd/api/v3 v3.5.0 go.etcd.io/etcd/client/pkg/v3 v3.5.0 go.etcd.io/etcd/client/v3 v3.5.0 golang.org/x/sys v0.0.0-20220319134239-a9b59b0215f8 diff --git a/go.sum b/go.sum index d4ec6f07a85..c0491239db0 100644 --- a/go.sum +++ b/go.sum @@ -572,6 +572,7 @@ github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.2.0 h1:Hbg2NidpLE8veEBkEZTL3CvlkUIVzuU9jDplZO54c48= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= diff --git a/pkg/profile/profile_test.go b/pkg/profile/profile_test.go index 002af5275b3..cfb7f2e7e78 100644 --- a/pkg/profile/profile_test.go +++ b/pkg/profile/profile_test.go @@ -29,15 +29,17 @@ func fakeServer(h http.Handler) error { } func TestInstall(t *testing.T) { - m := mux.NewRouter() - Install(m) - go fakeServer(m) - r, err := http.Get("http://localhost:9090/debug/pprof/") - if err != nil { - t.Error(" failed to send request to fake server") - } - - if r.StatusCode != http.StatusOK { - t.Error(err) - } + t.Run("TestInstall", func(t *testing.T) { + m := mux.NewRouter() + Install(m) + go fakeServer(m) + r, err := http.Get("http://localhost:9090/debug/pprof/") + if err != nil { + t.Errorf("failed to send request to fake server, %v", err) + } + + if r.StatusCode != http.StatusOK { + t.Error(err) + } + }) } diff --git a/pkg/yurthub/storage/disk/storage.go b/pkg/yurthub/storage/disk/storage.go index 794bff3b65f..b052f6d73d5 100644 --- a/pkg/yurthub/storage/disk/storage.go +++ b/pkg/yurthub/storage/disk/storage.go @@ -104,11 +104,15 @@ func (ds *diskStorage) Name() string { // Create will create a new file with content. key indicates the path of the file. func (ds *diskStorage) Create(key storage.Key, content []byte) error { - if err := utils.ValidateKV(key, content, storageKey{}); err != nil { + if err := utils.ValidateKey(key, storageKey{}); err != nil { return err } storageKey := key.(storageKey) + if !storageKey.isRootKey() && len(content) == 0 { + return storage.ErrKeyHasNoContent + } + if !ds.lockKey(storageKey) { return storage.ErrStorageAccessConflict } diff --git a/pkg/yurthub/storage/disk/storage_test.go b/pkg/yurthub/storage/disk/storage_test.go index 88ec1315f9c..4017fb2703d 100644 --- a/pkg/yurthub/storage/disk/storage_test.go +++ b/pkg/yurthub/storage/disk/storage_test.go @@ -332,6 +332,10 @@ var _ = Describe("Test DiskStorage Exposed Functions", func() { Expect(err).To(BeNil()) Expect(info.IsDir()).To(BeTrue()) }) + It("should return ErrKeyHasNoContent if it is not rootKey and has no content", func() { + err = store.Create(podKey, []byte{}) + Expect(err).To(Equal(storage.ErrKeyHasNoContent)) + }) It("should return ErrKeyIsEmpty if key is empty", func() { err = store.Create(storageKey{}, podBytes) Expect(err).To(Equal(storage.ErrKeyIsEmpty)) diff --git a/pkg/yurthub/storage/etcd/etcd_suite_test.go b/pkg/yurthub/storage/etcd/etcd_suite_test.go index 160132c3aa4..f508ee9c8ba 100644 --- a/pkg/yurthub/storage/etcd/etcd_suite_test.go +++ b/pkg/yurthub/storage/etcd/etcd_suite_test.go @@ -17,8 +17,10 @@ limitations under the License. package etcd import ( + "fmt" "os" "os/exec" + "path/filepath" "testing" . "github.com/onsi/ginkgo/v2" @@ -29,6 +31,9 @@ var keyCacheDir = "/tmp/etcd-test" var etcdDataDir = "/tmp/storagetest.etcd" var devNull *os.File var etcdCmd *exec.Cmd +var downloadURL = "https://github.com/etcd-io/etcd/releases/download" +var etcdVersion = "v3.5.0" +var etcdCmdPath = "/tmp/etcd/etcd" var _ = BeforeSuite(func() { Expect(os.RemoveAll(keyCacheDir)).To(BeNil()) @@ -38,7 +43,14 @@ var _ = BeforeSuite(func() { var err error devNull, err = os.OpenFile("/dev/null", os.O_RDWR, 0755) Expect(err).To(BeNil()) - etcdCmd = exec.Command("/usr/local/etcd/etcd", "--data-dir="+etcdDataDir) + + // It will check if etcd cmd can be found in PATH, otherwise + // it will be installed. + etcdCmdPath, err = ensureEtcdCmd() + Expect(err).To(BeNil()) + Expect(len(etcdCmdPath)).ShouldNot(BeZero()) + + etcdCmd = exec.Command(etcdCmdPath, "--data-dir="+etcdDataDir) etcdCmd.Stdout = devNull etcdCmd.Stderr = devNull Expect(etcdCmd.Start()).To(BeNil()) @@ -52,6 +64,38 @@ var _ = AfterSuite(func() { Expect(devNull.Close()).To(BeNil()) }) +func ensureEtcdCmd() (string, error) { + path, err := exec.LookPath("etcd") + if err == nil { + return path, nil + } + + return installEtcd() +} + +func installEtcd() (string, error) { + releaseURL := fmt.Sprintf("%s/%s/etcd-%s-linux-amd64.tar.gz", downloadURL, etcdVersion, etcdVersion) + downloadPath := fmt.Sprintf("/tmp/etcd/etcd-%s-linux-amd64.tar.gz", etcdVersion) + downloadDir := "/tmp/etcd" + if err := exec.Command("bash", "-c", "rm -rf "+downloadDir).Run(); err != nil { + return "", fmt.Errorf("failed to delete %s, %v", downloadDir, err) + } + + if err := exec.Command("bash", "-c", "mkdir "+downloadDir).Run(); err != nil { + return "", fmt.Errorf("failed to create dir %s, %v", downloadDir, err) + } + + if err := exec.Command("bash", "-c", "curl -L "+releaseURL+" -o "+downloadPath).Run(); err != nil { + return "", fmt.Errorf("failed to download etcd release %s at %s, %v", releaseURL, downloadPath, err) + } + + if err := exec.Command("tar", "-zxvf", downloadPath, "-C", downloadDir, "--strip-components=1").Run(); err != nil { + return "", fmt.Errorf("failed to extract tar at %s, %v", downloadPath, err) + } + + return filepath.Join(downloadDir, "etcd"), nil +} + func TestEtcd(t *testing.T) { RegisterFailHandler(Fail) RunSpecs(t, "ComponentKeyCache Test Suite") diff --git a/pkg/yurthub/storage/etcd/key_test.go b/pkg/yurthub/storage/etcd/key_test.go index bc59676282d..7eafaf9fe1a 100644 --- a/pkg/yurthub/storage/etcd/key_test.go +++ b/pkg/yurthub/storage/etcd/key_test.go @@ -36,6 +36,7 @@ func TestKeyFunc(t *testing.T) { }{ "core group normal case": { info: storage.KeyBuildInfo{ + Component: "kubelet", Group: "", Resources: "pods", Version: "v1", @@ -47,6 +48,7 @@ func TestKeyFunc(t *testing.T) { "special prefix for node resource": { info: storage.KeyBuildInfo{ + Component: "kubelet", Group: "", Resources: "nodes", Version: "v1", @@ -57,6 +59,7 @@ func TestKeyFunc(t *testing.T) { }, "not core group": { info: storage.KeyBuildInfo{ + Component: "kubelet", Group: "apps", Resources: "deployments", Version: "v1", @@ -67,6 +70,7 @@ func TestKeyFunc(t *testing.T) { }, "special prefix for service resource": { info: storage.KeyBuildInfo{ + Component: "kube-proxy", Group: "networking.k8s.io", Resources: "ingresses", Version: "v1", @@ -77,6 +81,7 @@ func TestKeyFunc(t *testing.T) { }, "empty resources": { info: storage.KeyBuildInfo{ + Component: "yurthub", Group: "", Resources: "", Version: "v1", @@ -85,6 +90,16 @@ func TestKeyFunc(t *testing.T) { }, err: storage.ErrEmptyResource, }, + "empty component": { + info: storage.KeyBuildInfo{ + Group: "", + Resources: "nodes", + Version: "v1", + Namespace: "", + Name: "test-node", + }, + err: storage.ErrEmptyComponent, + }, } for n, c := range cases { diff --git a/pkg/yurthub/storage/etcd/keycache.go b/pkg/yurthub/storage/etcd/keycache.go index 8122c7f7b66..df670ebca7e 100644 --- a/pkg/yurthub/storage/etcd/keycache.go +++ b/pkg/yurthub/storage/etcd/keycache.go @@ -69,12 +69,12 @@ func (s keySet) Difference(s2 keySet) []storageKey { // ... type componentKeyCache struct { sync.Mutex - ctx context.Context - cache map[string]keySet - filePath string - keyFunc func(storage.KeyBuildInfo) (storage.Key, error) - fsOperator fs.FileSystemOperator - getEtcdClient func() *clientv3.Client + ctx context.Context + cache map[string]keySet + filePath string + keyFunc func(storage.KeyBuildInfo) (storage.Key, error) + fsOperator fs.FileSystemOperator + etcdClient *clientv3.Client } func (c *componentKeyCache) Recover() error { @@ -84,27 +84,28 @@ func (c *componentKeyCache) Recover() error { if err := c.fsOperator.CreateFile(c.filePath, []byte{}); err != nil { return fmt.Errorf("failed to create cache file at %s, %v", c.filePath, err) } - return nil } else if err != nil { return fmt.Errorf("failed to recover key cache from %s, %v", c.filePath, err) } - // successfully read from file - if len(buf) == 0 { - return nil - } - lines := strings.Split(string(buf), "\n") - for i, l := range lines { - s := strings.Split(l, ":") - if len(s) != 2 { - return fmt.Errorf("failed to parse line %d, invalid format", i) - } - comp, keys := s[0], strings.Split(s[1], ",") - ks := keySet{m: map[storageKey]struct{}{}} - for _, key := range keys { - ks.m[storageKey{path: key}] = struct{}{} + if len(buf) != 0 { + // We've got content from file + lines := strings.Split(string(buf), "\n") + for i, l := range lines { + s := strings.Split(l, ":") + if len(s) != 2 { + return fmt.Errorf("failed to parse line %d, invalid format", i) + } + comp, keys := s[0], strings.Split(s[1], ",") + ks := keySet{m: map[storageKey]struct{}{}} + for _, key := range keys { + ks.m[storageKey{ + comp: comp, + path: key, + }] = struct{}{} + } + c.cache[comp] = ks } - c.cache[comp] = ks } poolScopedKeyset, err := c.getPoolScopedKeyset() @@ -120,49 +121,43 @@ func (c *componentKeyCache) Recover() error { } func (c *componentKeyCache) getPoolScopedKeyset() (*keySet, error) { - // FIXME: now getEtcdClient would cause nil pointer - //client := c.getEtcdClient() - //if client == nil { - // return nil, fmt.Errorf("got empty etcd client") - //} - keys := &keySet{m: map[storageKey]struct{}{}} - //for gvr := range coordinatorconstants.PoolScopedResources { - //getCtx, cancel := context.WithTimeout(c.ctx, defaultTimeout) - //defer cancel() - //rootKey, err := c.keyFunc(storage.KeyBuildInfo{ - // Component: coordinatorconstants.DefaultPoolScopedUserAgent, - // Group: gvr.Group, - // Version: gvr.Version, - // Resources: gvr.Resource, - //}) - //if err != nil { - // return nil, fmt.Errorf("failed to generate keys for %s, %v", gvr.String(), err) - //} - //getResp, err := client.Get(getCtx, rootKey.Key(), clientv3.WithPrefix(), clientv3.WithKeysOnly()) - //if err != nil { - // return nil, fmt.Errorf("failed to get from etcd for %s, %v", gvr.String(), err) - //} - // - //for _, kv := range getResp.Kvs { - // ns, name, err := getNamespaceAndNameFromKeyPath(string(kv.Key)) - // if err != nil { - // return nil, fmt.Errorf("failed to parse namespace and name of %s", kv.Key) - // } - // key, err := c.keyFunc(storage.KeyBuildInfo{ - // Component: coordinatorconstants.DefaultPoolScopedUserAgent, - // Group: gvr.Group, - // Version: gvr.Version, - // Resources: gvr.Resource, - // Namespace: ns, - // Name: name, - // }) - // if err != nil { - // return nil, fmt.Errorf("failed to create resource key for %v", kv.Key) - // } - // keys.m[key.(storageKey)] = struct{}{} - //} - //} + for gvr := range coordinatorconstants.PoolScopedResources { + getCtx, cancel := context.WithTimeout(c.ctx, defaultTimeout) + defer cancel() + rootKey, err := c.keyFunc(storage.KeyBuildInfo{ + Component: coordinatorconstants.DefaultPoolScopedUserAgent, + Group: gvr.Group, + Version: gvr.Version, + Resources: gvr.Resource, + }) + if err != nil { + return nil, fmt.Errorf("failed to generate keys for %s, %v", gvr.String(), err) + } + getResp, err := c.etcdClient.Get(getCtx, rootKey.Key(), clientv3.WithPrefix(), clientv3.WithKeysOnly()) + if err != nil { + return nil, fmt.Errorf("failed to get from etcd for %s, %v", gvr.String(), err) + } + + for _, kv := range getResp.Kvs { + ns, name, err := getNamespaceAndNameFromKeyPath(string(kv.Key)) + if err != nil { + return nil, fmt.Errorf("failed to parse namespace and name of %s", kv.Key) + } + key, err := c.keyFunc(storage.KeyBuildInfo{ + Component: coordinatorconstants.DefaultPoolScopedUserAgent, + Group: gvr.Group, + Version: gvr.Version, + Resources: gvr.Resource, + Namespace: ns, + Name: name, + }) + if err != nil { + return nil, fmt.Errorf("failed to create resource key for %v", kv.Key) + } + keys.m[key.(storageKey)] = struct{}{} + } + } return keys, nil } @@ -262,19 +257,11 @@ func (c *componentKeyCache) flush() error { return nil } -func newComponentKeyCache(filePath string) *componentKeyCache { - return &componentKeyCache{ - filePath: filePath, - cache: map[string]keySet{}, - fsOperator: fs.FileSystemOperator{}, - } -} - // We assume that path points to a namespaced resource. func getNamespaceAndNameFromKeyPath(path string) (string, string, error) { - elems := strings.Split(path, "/") + elems := strings.Split(strings.TrimPrefix(path, "/"), "/") if len(elems) < 2 { - return "", "", fmt.Errorf("unrecognized path: %v", path) + return "", "", fmt.Errorf("unrecognized path: %s", path) } return elems[len(elems)-2], elems[len(elems)-1], nil diff --git a/pkg/yurthub/storage/etcd/keycache_test.go b/pkg/yurthub/storage/etcd/keycache_test.go index c9512e221e3..8120ae1bf57 100644 --- a/pkg/yurthub/storage/etcd/keycache_test.go +++ b/pkg/yurthub/storage/etcd/keycache_test.go @@ -17,6 +17,7 @@ limitations under the License. package etcd import ( + "context" "os" "path/filepath" "reflect" @@ -26,7 +27,12 @@ import ( "github.com/google/uuid" . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" + "github.com/stretchr/testify/mock" + mvccpb "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + coordinatorconstants "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" + etcdmock "github.com/openyurtio/openyurt/pkg/yurthub/storage/etcd/mock" "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" ) @@ -34,9 +40,24 @@ var _ = Describe("Test componentKeyCache setup", func() { var cache *componentKeyCache var fileName string var f fs.FileSystemOperator + var mockedClient *clientv3.Client BeforeEach(func() { + kv := etcdmock.KV{} + kv.On("Get", "/registry/services/endpoints", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). + Return(&clientv3.GetResponse{}) + kv.On("Get", "/registry/endpointslices", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). + Return(&clientv3.GetResponse{}) + etcdStorage := &etcdStorage{prefix: "/registry"} + mockedClient = &clientv3.Client{KV: kv} fileName = uuid.New().String() - cache = newComponentKeyCache(filepath.Join(keyCacheDir, fileName)) + cache = &componentKeyCache{ + ctx: context.Background(), + filePath: filepath.Join(keyCacheDir, fileName), + cache: map[string]keySet{}, + fsOperator: fs.FileSystemOperator{}, + etcdClient: mockedClient, + keyFunc: etcdStorage.KeyFunc, + } }) AfterEach(func() { Expect(os.RemoveAll(filepath.Join(keyCacheDir, fileName))) @@ -44,13 +65,92 @@ var _ = Describe("Test componentKeyCache setup", func() { It("should recover when cache file does not exist", func() { Expect(cache.Recover()).To(BeNil()) - Expect(len(cache.cache)).To(BeZero()) + Expect(len(cache.cache)).To(Equal(1)) }) It("should recover when cache file is empty", func() { Expect(f.CreateFile(filepath.Join(keyCacheDir, fileName), []byte{})).To(BeNil()) Expect(cache.Recover()).To(BeNil()) - Expect(len(cache.cache)).To(BeZero()) + Expect(len(cache.cache)).To(Equal(1)) + }) + + Context("Test get pool-scoped resource keys from etcd", func() { + BeforeEach(func() { + kv := etcdmock.KV{} + kv.On("Get", "/registry/services/endpoints", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). + Return(&clientv3.GetResponse{ + Kvs: []*mvccpb.KeyValue{ + {Key: []byte("/registry/services/endpoints/default/nginx")}, + {Key: []byte("/registry/services/endpoints/kube-system/kube-dns")}, + }, + }) + kv.On("Get", "/registry/endpointslices", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). + Return(&clientv3.GetResponse{ + Kvs: []*mvccpb.KeyValue{ + {Key: []byte("/registry/endpointslices/default/nginx")}, + {Key: []byte("/registry/endpointslices/kube-system/kube-dns")}, + }, + }) + mockedClient.KV = kv + }) + + It("should recover leader-yurthub cache from etcd", func() { + Expect(cache.Recover()).To(BeNil()) + Expect(cache.cache[coordinatorconstants.DefaultPoolScopedUserAgent]).Should(Equal( + keySet{ + m: map[storageKey]struct{}{ + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + path: "/registry/services/endpoints/default/nginx", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + path: "/registry/services/endpoints/kube-system/kube-dns", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + path: "/registry/endpointslices/default/nginx", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + path: "/registry/endpointslices/kube-system/kube-dns", + }: {}, + }, + }, + )) + }) + + It("should replace leader-yurthub cache read from local file with keys from etcd", func() { + Expect(f.CreateFile(filepath.Join(keyCacheDir, fileName), []byte( + "leader-yurthub:/registry/services/endpoints/default/nginx-local,"+ + "/registry/services/endpoints/kube-system/kube-dns-local,"+ + "/registry/endpointslices/default/nginx-local,"+ + "/registry/endpointslices/kube-system/kube-dns-local", + ))).To(BeNil()) + Expect(cache.Recover()).To(BeNil()) + Expect(cache.cache[coordinatorconstants.DefaultPoolScopedUserAgent]).Should(Equal( + keySet{ + m: map[storageKey]struct{}{ + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + path: "/registry/services/endpoints/default/nginx", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + path: "/registry/services/endpoints/kube-system/kube-dns", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + path: "/registry/endpointslices/default/nginx", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + path: "/registry/endpointslices/kube-system/kube-dns", + }: {}, + }, + }, + )) + }) }) It("should recover when cache file exists and contains valid data", func() { @@ -62,15 +162,27 @@ var _ = Describe("Test componentKeyCache setup", func() { Expect(cache.cache).To(Equal(map[string]keySet{ "kubelet": { m: map[storageKey]struct{}{ - {path: "/registry/pods/default/pod1"}: {}, - {path: "/registry/pods/default/pod2"}: {}, + { + comp: "kubelet", + path: "/registry/pods/default/pod1", + }: {}, + { + comp: "kubelet", + path: "/registry/pods/default/pod2", + }: {}, }, }, "kube-proxy": { m: map[storageKey]struct{}{ - {path: "/registry/configmaps/kube-system/kube-proxy"}: {}, + { + comp: "kube-proxy", + path: "/registry/configmaps/kube-system/kube-proxy", + }: {}, }, }, + coordinatorconstants.DefaultPoolScopedUserAgent: { + m: map[storageKey]struct{}{}, + }, })) }) @@ -87,8 +199,22 @@ var _ = Describe("Test componentKeyCache function", func() { var fileName string var key1, key2, key3 storageKey BeforeEach(func() { + kv := etcdmock.KV{} + kv.On("Get", "/registry/services/endpoints", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). + Return(&clientv3.GetResponse{}) + kv.On("Get", "/registry/endpointslices", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). + Return(&clientv3.GetResponse{}) + mockedClient := &clientv3.Client{KV: kv} + etcdStorage := etcdStorage{prefix: "/registry"} fileName = uuid.New().String() - cache = newComponentKeyCache(filepath.Join(keyCacheDir, fileName)) + cache = &componentKeyCache{ + ctx: context.Background(), + filePath: filepath.Join(keyCacheDir, fileName), + cache: map[string]keySet{}, + fsOperator: fs.FileSystemOperator{}, + etcdClient: mockedClient, + keyFunc: etcdStorage.KeyFunc, + } key1 = storageKey{ path: "/registry/pods/default/pod1", } diff --git a/pkg/yurthub/storage/etcd/mock/kv.go b/pkg/yurthub/storage/etcd/mock/kv.go new file mode 100644 index 00000000000..3839aa01dfa --- /dev/null +++ b/pkg/yurthub/storage/etcd/mock/kv.go @@ -0,0 +1,60 @@ +/* +Copyright 2023 The OpenYurt Authors. + +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 mock + +import ( + "context" + + "github.com/stretchr/testify/mock" + clientv3 "go.etcd.io/etcd/client/v3" +) + +var _ clientv3.KV = KV{} + +type KV struct { + mock.Mock +} + +func (kv KV) Put(ctx context.Context, key, val string, opts ...clientv3.OpOption) (*clientv3.PutResponse, error) { + return nil, nil +} + +func (kv KV) Get(ctx context.Context, key string, opts ...clientv3.OpOption) (*clientv3.GetResponse, error) { + interfaceOpts := []interface{}{key} + for _, opt := range opts { + interfaceOpts = append(interfaceOpts, opt) + } + args := kv.Called(interfaceOpts...) + resp := args.Get(0).(*clientv3.GetResponse) + return resp, nil +} + +func (kv KV) Delete(ctx context.Context, key string, opts ...clientv3.OpOption) (*clientv3.DeleteResponse, error) { + return nil, nil +} + +func (kv KV) Compact(ctx context.Context, rev int64, opts ...clientv3.CompactOption) (*clientv3.CompactResponse, error) { + return nil, nil +} + +func (kv KV) Do(ctx context.Context, op clientv3.Op) (clientv3.OpResponse, error) { + return clientv3.OpResponse{}, nil +} + +func (kv KV) Txn(ctx context.Context) clientv3.Txn { + return nil +} diff --git a/pkg/yurthub/storage/etcd/storage.go b/pkg/yurthub/storage/etcd/storage.go index fe0b9503b1f..7e5df7b0b61 100644 --- a/pkg/yurthub/storage/etcd/storage.go +++ b/pkg/yurthub/storage/etcd/storage.go @@ -18,6 +18,7 @@ package etcd import ( "context" + "crypto/tls" "fmt" "path/filepath" "strings" @@ -33,6 +34,7 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/storage" "github.com/openyurtio/openyurt/pkg/yurthub/storage/utils" + "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" ) const ( @@ -57,6 +59,7 @@ type EtcdStorageConfig struct { KeyFile string CaFile string LocalCacheDir string + UnSecure bool } // TODO: consider how to recover the work if it was interrupted because of restart, in @@ -86,21 +89,20 @@ type etcdStorage struct { } func NewStorage(ctx context.Context, cfg *EtcdStorageConfig) (storage.Store, error) { + var tlsConfig *tls.Config + var err error cacheFilePath := filepath.Join(cfg.LocalCacheDir, defaultComponentCacheFileName) - cache := newComponentKeyCache(cacheFilePath) - if err := cache.Recover(); err != nil { - return nil, fmt.Errorf("failed to recover component key cache from %s, %v", cacheFilePath, err) - } - - tlsInfo := transport.TLSInfo{ - CertFile: cfg.CertFile, - KeyFile: cfg.KeyFile, - TrustedCAFile: cfg.CaFile, - } + if !cfg.UnSecure { + tlsInfo := transport.TLSInfo{ + CertFile: cfg.CertFile, + KeyFile: cfg.KeyFile, + TrustedCAFile: cfg.CaFile, + } - tlsConfig, err := tlsInfo.ClientConfig() - if err != nil { - return nil, fmt.Errorf("failed to create tls config for etcd client, %v", err) + tlsConfig, err = tlsInfo.ClientConfig() + if err != nil { + return nil, fmt.Errorf("failed to create tls config for etcd client, %v", err) + } } clientConfig := clientv3.Config{ @@ -115,17 +117,30 @@ func NewStorage(ctx context.Context, cfg *EtcdStorageConfig) (storage.Store, err } s := &etcdStorage{ - ctx: ctx, - prefix: cfg.Prefix, - client: client, - clientConfig: clientConfig, - localComponentKeyCache: cache, + ctx: ctx, + prefix: cfg.Prefix, + client: client, + clientConfig: clientConfig, mirrorPrefixMap: map[pathType]string{ rvType: "/mirror/rv", }, } + cache := &componentKeyCache{ + ctx: ctx, + filePath: cacheFilePath, + cache: map[string]keySet{}, + fsOperator: fs.FileSystemOperator{}, + keyFunc: s.KeyFunc, + etcdClient: client, + } + if err := cache.Recover(); err != nil { + return nil, fmt.Errorf("failed to recover component key cache from %s, %v", cacheFilePath, err) + } + s.localComponentKeyCache = cache + go s.clientLifeCycleManagement() + return s, nil } diff --git a/pkg/yurthub/storage/etcd/storage_test.go b/pkg/yurthub/storage/etcd/storage_test.go index ea3d0e3952c..20e1ccbd79b 100644 --- a/pkg/yurthub/storage/etcd/storage_test.go +++ b/pkg/yurthub/storage/etcd/storage_test.go @@ -34,6 +34,9 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/storage" ) +// TODO: These tests should be integration tests instead of unit tests. +// Currently, we will install the etcd cmd BeforeSuite to make these tests work around. +// But they are better moved to integration test dir. var _ = Describe("Test EtcdStorage", func() { var etcdstore *etcdStorage var key1 storage.Key @@ -47,6 +50,7 @@ var _ = Describe("Test EtcdStorage", func() { Prefix: "/" + randomize, EtcdEndpoints: []string{"127.0.0.1:2379"}, LocalCacheDir: filepath.Join(keyCacheDir, randomize), + UnSecure: true, } s, err := NewStorage(context.Background(), cfg) Expect(err).To(BeNil()) @@ -75,8 +79,8 @@ var _ = Describe("Test EtcdStorage", func() { Expect(err).To(BeNil()) }) - Context("Test Lifecycle", Focus, func() { - It("should reconnect to etcd if connect once broken", Focus, func() { + Context("Test Lifecycle", func() { + It("should reconnect to etcd if connect once broken", func() { Expect(etcdstore.Create(key1, podJson)).Should(BeNil()) Expect(etcdCmd.Process.Kill()).To(BeNil()) By("waiting for the etcd exited") @@ -87,7 +91,7 @@ var _ = Describe("Test EtcdStorage", func() { devNull, err := os.OpenFile("/dev/null", os.O_RDWR, 0755) Expect(err).To(BeNil()) - etcdCmd = exec.Command("/usr/local/etcd/etcd", "--data-dir="+etcdDataDir) + etcdCmd = exec.Command(etcdCmdPath, "--data-dir="+etcdDataDir) etcdCmd.Stdout = devNull etcdCmd.Stderr = devNull Expect(etcdCmd.Start()).To(BeNil()) From 948ce9fd28cc40376a3b17c518f9b4d00d5a1f21 Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Thu, 12 Jan 2023 10:15:18 +0800 Subject: [PATCH 09/20] bugfix: return fake getter if poolcoordinator is not enabled (#1138) * do not wait for coordinator informer registry when coordinator is disabled Signed-off-by: Congrool --- cmd/yurthub/app/start.go | 40 ++++++++++++++++++++++++---------------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/cmd/yurthub/app/start.go b/cmd/yurthub/app/start.go index 1f0ef5bdd55..f4ce630b2a6 100644 --- a/cmd/yurthub/app/start.go +++ b/cmd/yurthub/app/start.go @@ -148,26 +148,26 @@ func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { tenantMgr := tenant.New(cfg.TenantNs, cfg.SharedFactory, ctx.Done()) trace++ - var coordinatorGetter func() poolcoordinator.Coordinator - var coordinatorHealthCheckerGetter func() healthchecker.HealthChecker - var coordinatorTransportManagerGetter func() transport.Interface = nil - coordinatorInformerRegistryChan := make(chan struct{}) + var coordinatorHealthCheckerGetter func() healthchecker.HealthChecker = getFakeCoordinatorHealthChecker + var coordinatorTransportManagerGetter func() transport.Interface = getFakeCoordinatorTransportManager + var coordinatorGetter func() poolcoordinator.Coordinator = getFakeCoordinator if cfg.EnableCoordinator { klog.Infof("%d. start to run coordinator", trace) - // coordinatorRun will register secret informer into sharedInformerFactory, and start a new goroutine to periodically check - // if certs has been got from cloud APIServer. trace++ - // Waitting for the coordinator to run, before using it to create other components. + + coordinatorInformerRegistryChan := make(chan struct{}) + // coordinatorRun will register secret informer into sharedInformerFactory, and start a new goroutine to periodically check + // if certs has been got from cloud APIServer. It will close the coordinatorInformerRegistryChan if the secret channel has + // been registered into informer factory. coordinatorHealthCheckerGetter, coordinatorTransportManagerGetter, coordinatorGetter, err = coordinatorRun(ctx, cfg, restConfigMgr, cloudHealthChecker, coordinatorInformerRegistryChan) if err != nil { return fmt.Errorf("failed to wait for coordinator to run, %v", err) } + // wait for coordinator informer registry + <-coordinatorInformerRegistryChan } - // wait for coordinator informer registry - <-coordinatorInformerRegistryChan - // Start the informer factory if all informers have been registered cfg.SharedFactory.Start(ctx.Done()) cfg.YurtSharedFactory.Start(ctx.Done()) @@ -221,12 +221,8 @@ func createClients(heartbeatTimeoutSeconds int, remoteServers []*url.URL, coordi } // coordinatorRun will initialize and start all coordinator-related components in an async way. -// It returns a func waittingForReady, which will block until the initialization routine exited. -// If the initialization succeeds, waittingForReady will return these coordinator-related components, including: -// 1. coordinator HealthChecker -// 2. coordinator TransportManager -// 3. and the coordinator -// Otherwise, the error is not nil. +// It returns Getter function for coordinator, coordinator health checker and coordinator transport manager, +// which will return the relative component if it has been initialized, otherwise it will return nil. func coordinatorRun(ctx context.Context, cfg *config.YurtHubConfiguration, restConfigMgr *hubrest.RestConfigManager, @@ -315,3 +311,15 @@ func poolCoordinatorTransportMgrGetter(heartbeatTimeoutSeconds int, coordinatorS } return coordinatorTransportMgr, nil } + +func getFakeCoordinator() poolcoordinator.Coordinator { + return &poolcoordinator.FakeCoordinator{} +} + +func getFakeCoordinatorHealthChecker() healthchecker.HealthChecker { + return healthchecker.NewFakeChecker(false, make(map[string]int)) +} + +func getFakeCoordinatorTransportManager() transport.Interface { + return nil +} From ce76fdaa64eb096b05d3d19750a73f7d882cba8a Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Thu, 12 Jan 2023 10:24:11 +0800 Subject: [PATCH 10/20] mocked etcd client should not use copied locks (#1139) Signed-off-by: Congrool --- pkg/yurthub/storage/etcd/keycache_test.go | 6 +++--- pkg/yurthub/storage/etcd/mock/kv.go | 14 +++++++------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/pkg/yurthub/storage/etcd/keycache_test.go b/pkg/yurthub/storage/etcd/keycache_test.go index 8120ae1bf57..d24398699e3 100644 --- a/pkg/yurthub/storage/etcd/keycache_test.go +++ b/pkg/yurthub/storage/etcd/keycache_test.go @@ -42,7 +42,7 @@ var _ = Describe("Test componentKeyCache setup", func() { var f fs.FileSystemOperator var mockedClient *clientv3.Client BeforeEach(func() { - kv := etcdmock.KV{} + kv := &etcdmock.KV{} kv.On("Get", "/registry/services/endpoints", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). Return(&clientv3.GetResponse{}) kv.On("Get", "/registry/endpointslices", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). @@ -76,7 +76,7 @@ var _ = Describe("Test componentKeyCache setup", func() { Context("Test get pool-scoped resource keys from etcd", func() { BeforeEach(func() { - kv := etcdmock.KV{} + kv := &etcdmock.KV{} kv.On("Get", "/registry/services/endpoints", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). Return(&clientv3.GetResponse{ Kvs: []*mvccpb.KeyValue{ @@ -199,7 +199,7 @@ var _ = Describe("Test componentKeyCache function", func() { var fileName string var key1, key2, key3 storageKey BeforeEach(func() { - kv := etcdmock.KV{} + kv := &etcdmock.KV{} kv.On("Get", "/registry/services/endpoints", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). Return(&clientv3.GetResponse{}) kv.On("Get", "/registry/endpointslices", mock.AnythingOfType("clientv3.OpOption"), mock.AnythingOfType("clientv3.OpOption")). diff --git a/pkg/yurthub/storage/etcd/mock/kv.go b/pkg/yurthub/storage/etcd/mock/kv.go index 3839aa01dfa..5c30525f5ba 100644 --- a/pkg/yurthub/storage/etcd/mock/kv.go +++ b/pkg/yurthub/storage/etcd/mock/kv.go @@ -23,17 +23,17 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" ) -var _ clientv3.KV = KV{} +var _ clientv3.KV = &KV{} type KV struct { mock.Mock } -func (kv KV) Put(ctx context.Context, key, val string, opts ...clientv3.OpOption) (*clientv3.PutResponse, error) { +func (kv *KV) Put(ctx context.Context, key, val string, opts ...clientv3.OpOption) (*clientv3.PutResponse, error) { return nil, nil } -func (kv KV) Get(ctx context.Context, key string, opts ...clientv3.OpOption) (*clientv3.GetResponse, error) { +func (kv *KV) Get(ctx context.Context, key string, opts ...clientv3.OpOption) (*clientv3.GetResponse, error) { interfaceOpts := []interface{}{key} for _, opt := range opts { interfaceOpts = append(interfaceOpts, opt) @@ -43,18 +43,18 @@ func (kv KV) Get(ctx context.Context, key string, opts ...clientv3.OpOption) (*c return resp, nil } -func (kv KV) Delete(ctx context.Context, key string, opts ...clientv3.OpOption) (*clientv3.DeleteResponse, error) { +func (kv *KV) Delete(ctx context.Context, key string, opts ...clientv3.OpOption) (*clientv3.DeleteResponse, error) { return nil, nil } -func (kv KV) Compact(ctx context.Context, rev int64, opts ...clientv3.CompactOption) (*clientv3.CompactResponse, error) { +func (kv *KV) Compact(ctx context.Context, rev int64, opts ...clientv3.CompactOption) (*clientv3.CompactResponse, error) { return nil, nil } -func (kv KV) Do(ctx context.Context, op clientv3.Op) (clientv3.OpResponse, error) { +func (kv *KV) Do(ctx context.Context, op clientv3.Op) (clientv3.OpResponse, error) { return clientv3.OpResponse{}, nil } -func (kv KV) Txn(ctx context.Context) clientv3.Txn { +func (kv *KV) Txn(ctx context.Context) clientv3.Txn { return nil } From 512cec67d8bcc1b6d06540529ac3fa5aa18f2872 Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Thu, 12 Jan 2023 11:22:19 +0800 Subject: [PATCH 11/20] add unit test for poolcoordinator cert manager (#1140) Signed-off-by: Congrool --- .../certmanager/certmanager.go | 19 +- .../certmanager/certmanager_test.go | 382 ++++++++++++++++++ 2 files changed, 390 insertions(+), 11 deletions(-) create mode 100644 pkg/yurthub/poolcoordinator/certmanager/certmanager_test.go diff --git a/pkg/yurthub/poolcoordinator/certmanager/certmanager.go b/pkg/yurthub/poolcoordinator/certmanager/certmanager.go index 2f4811f58bc..429020ffeab 100644 --- a/pkg/yurthub/poolcoordinator/certmanager/certmanager.go +++ b/pkg/yurthub/poolcoordinator/certmanager/certmanager.go @@ -50,24 +50,17 @@ var certFileNames = map[CertFileType]string{ YurthubClientKey: "pool-coordinator-yurthub-client.key", } -func NewCertManager(caFilePath string, yurtClient kubernetes.Interface, informerFactory informers.SharedInformerFactory) (*CertManager, error) { +func NewCertManager(pkiDir string, yurtClient kubernetes.Interface, informerFactory informers.SharedInformerFactory) (*CertManager, error) { store := fs.FileSystemOperator{} - dir, _ := filepath.Split(caFilePath) - if err := store.CreateDir(dir); err != nil && err != fs.ErrExists { - return nil, fmt.Errorf("failed to create dir %s, %v", dir, err) + if err := store.CreateDir(pkiDir); err != nil && err != fs.ErrExists { + return nil, fmt.Errorf("failed to create dir %s, %v", pkiDir, err) } certMgr := &CertManager{ - pkiDir: dir, + pkiDir: pkiDir, store: store, } - // try to use last cert files when restart. - certPath, keyPath := certMgr.GetFilePath(YurthubClientCert), certMgr.GetFilePath(YurthubClientKey) - if cert, err := tls.LoadX509KeyPair(certPath, keyPath); err == nil { - certMgr.cert = &cert - } - secretInformerFunc := func(client kubernetes.Interface, resyncPeriod time.Duration) cache.SharedIndexInformer { tweakListOptions := func(options *metav1.ListOptions) { options.FieldSelector = fields.Set{"metadata.name": constants.PoolCoordinatorClientSecretName}.String() @@ -100,6 +93,9 @@ type CertManager struct { pkiDir string cert *tls.Certificate store fs.FileSystemOperator + + // Used for unit test. + secret *corev1.Secret } func (c *CertManager) GetAPIServerClientCert() *tls.Certificate { @@ -145,6 +141,7 @@ func (c *CertManager) updateCerts(secret *corev1.Secret) { klog.Errorf("failed to update client cert, %v", err) } c.cert = &cert + c.secret = secret.DeepCopy() } func (c *CertManager) deleteCerts() { diff --git a/pkg/yurthub/poolcoordinator/certmanager/certmanager_test.go b/pkg/yurthub/poolcoordinator/certmanager/certmanager_test.go new file mode 100644 index 00000000000..bb5afd23ac4 --- /dev/null +++ b/pkg/yurthub/poolcoordinator/certmanager/certmanager_test.go @@ -0,0 +1,382 @@ +/* +Copyright 2023 The OpenYurt Authors. + +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 certmanager + +import ( + "fmt" + "reflect" + "testing" + "time" + + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/client-go/informers" + "k8s.io/client-go/kubernetes/fake" + + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" + "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" +) + +const ( + testPKIDir = "/tmp/pool-coordinator-pki" + caByte = `-----BEGIN CERTIFICATE----- +MIIC/jCCAeagAwIBAgIBADANBgkqhkiG9w0BAQsFADAVMRMwEQYDVQQDEwprdWJl +cm5ldGVzMB4XDTIyMTIyODAzMzgyM1oXDTMyMTIyNTAzMzgyM1owFTETMBEGA1UE +AxMKa3ViZXJuZXRlczCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAKUI +4IgEu/xH2orH1uLx1ad+eBy8WcqwOaJKZMZqEgEorWRXUvsM/UAE447V/eGkvwT/ +rFlFuyhVzpsecE4n2zK13lf7/cHD6raS4XR2vvbgX/KRkNPHPK38326zCu+rvZVU +9zq5rxXGHKytL+2uVuCnjP8xOtgEy9iB8kML2wWBMuO8Seyh4/F/jJ5Zrhi/zgHp +swfgvmEYz0BGFBqnVYYx7CST2ek95LVXnc3xS8wlmo+X4foiJG9mVSTGtfQoBQ2H +hg3vZV3+fsXNNYT4xigZ5kU97npaZk/nfZGyaHuEeiNWQOimQYCvJWFHJ6G/Vuyt +gpujDjMpH9nYwZkKb8UCAwEAAaNZMFcwDgYDVR0PAQH/BAQDAgKkMA8GA1UdEwEB +/wQFMAMBAf8wHQYDVR0OBBYEFKKux0rxaMSl/ks3ndmrOeu8PN4mMBUGA1UdEQQO +MAyCCmt1YmVybmV0ZXMwDQYJKoZIhvcNAQELBQADggEBAGN4uO2xB10zcrbCYjeG +hM3v3rfYaV1vbZlVk/EHf/rtaP+GPIOhv0cdeARKS9VaUXnf4j5a3/nHGDLKVvEv ++ExJqLzgMLTcCKzkSRR+vIETzAmrfsp6xDILMn3yKxTcKRjFGJGVRfuyFH9rMKhQ +M+H4VUQcFGYRPhU+2bxRCxuEHe2tDXBGp7N36SPFJLNSvpf7RYdHPu00n8rKJ69D +XI0fjWnZMbOV7tUWVd/6rW4mhez3xgxW8H8h0IWHY6cdAjO3q0J9PHyaCFB1yZ0A +WOkCYynzE8EVrosIUIko+6IopX5wheTJ0IcU4yCQNo+avzYKMFztVh6eQLoe7afq +GFQ= +-----END CERTIFICATE----- +` + certByte = `-----BEGIN CERTIFICATE----- +MIIDLjCCAhagAwIBAgIIDOMcH2sIQDowDQYJKoZIhvcNAQELBQAwFTETMBEGA1UE +AxMKa3ViZXJuZXRlczAeFw0yMjEyMjgwMzM4MjNaFw0yMzEyMjgwMzM4MjNaMEEx +FzAVBgNVBAoTDnN5c3RlbTptYXN0ZXJzMSYwJAYDVQQDEx1rdWJlLWFwaXNlcnZl +ci1rdWJlbGV0LWNsaWVudDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEB +AMUtJEadOe43qPTAzphJ+efJXmkTgbsdSHGI7BigqCXOgQ8kEeTQSIVqTLpvpkJ1 +fCmv6CbNNQqrABSIvH9oPo1ATY04EreAW5krHdSFaOPO1T/TrySyG7NW5ikEZoji +IBFEQ1B2JbpJWCHsDspaB7BMI/yKgrs2RunTqgLd8VPoGz+QFrXe1DEZ93q7qHqs +U3dW2UD+h8igVLVefXx6NM4e3c1wE2u4IzeUbVVJ/72CpeFmmz3QGiofrvk0NXWY +D9xGmajI1vj5hs+IuN/2lSahZIDfv9Lf2TUDG0faRfnhPluS8X5klicwCOnZQAzD +w3X89RkaRhH3R05ky5wXjYECAwEAAaNWMFQwDgYDVR0PAQH/BAQDAgWgMBMGA1Ud +JQQMMAoGCCsGAQUFBwMCMAwGA1UdEwEB/wQCMAAwHwYDVR0jBBgwFoAUoq7HSvFo +xKX+Szed2as567w83iYwDQYJKoZIhvcNAQELBQADggEBAHV63tYhK7fGSOfL7PDZ +Ox3DySyMy2HYVFXmzFo/netFqARNZB58AqY1iCib1Fg4BJYUpPqfcg7fswft9dY/ +1SSekPEfZRz46yPN9JZlMEqtVEoqsNA0qUDWOotPjwb2+vgEroh4+rMk0gqgzx5m +dXqJMpWGIYWNH2Sa8yvHo2qGsShl5/uRNHycBVu2fGHCcLOCfPTslPzZYYJxQ33O +mNW/2WySzy7YL9wLyBRbYPoZK1ATt8ZtmUv/R03a4J8iSKBZwVrn5Yvr5gS+7JNC +ip2++hBi1NIyUYAhdktGas6FZPORtn+kvVs5A/V88EacqkWqVWRW0582gcyL8uJD +QXo= +-----END CERTIFICATE----- +` + keyByte = `-----BEGIN RSA PRIVATE KEY----- +MIIEowIBAAKCAQEAxS0kRp057jeo9MDOmEn558leaROBux1IcYjsGKCoJc6BDyQR +5NBIhWpMum+mQnV8Ka/oJs01CqsAFIi8f2g+jUBNjTgSt4BbmSsd1IVo487VP9Ov +JLIbs1bmKQRmiOIgEURDUHYluklYIewOyloHsEwj/IqCuzZG6dOqAt3xU+gbP5AW +td7UMRn3eruoeqxTd1bZQP6HyKBUtV59fHo0zh7dzXATa7gjN5RtVUn/vYKl4Wab +PdAaKh+u+TQ1dZgP3EaZqMjW+PmGz4i43/aVJqFkgN+/0t/ZNQMbR9pF+eE+W5Lx +fmSWJzAI6dlADMPDdfz1GRpGEfdHTmTLnBeNgQIDAQABAoIBAQCzW/fWoCjVMB5p +3YVQdGJ2XO+bh5oH+oAufs29LU8nbOxrOHVqfaiqa+K16OAFLleumAwGV757IMfm +5ecJwmq8FJU2853a/FDWSKlO67hZGYlUERwNtlKKVW7yOsWGmKNw8XaGF6MEDLm1 +ycQ+f5zk2q4ViG2ZHKtvAhJxnzBqEGtVssHZya4j3E0WJjv1TRlLYxzgIQHgk49p +ysxD23O5EJ/nCexCnZizAKLLNmDDhC4KVVUts3sQVVG5I4wRHfg61w7KiEpLinMA +mYhhomRJKSz46QI/i4Clrsi3et2DjiZdyNmGTSi2TpNL/1pci9qmhh8sUdV6Cqjz +hgAF9OCtAoGBAMzlzGlBJAOnbm8OpItiN+R10AgYEjHe1WBjJicfnAAIXEIY1tpH +KhSN0RplhNZcXZJn45fIP5YRMeRUKp1fRWtluoQp210hkyroScRz1ELFBYoBXnx3 +d++KfODcCiGjgFys1VYYWiUT9wgNFJzFMinUcddUtGZWKC37N0OTZlbTAoGBAPZa +W0heH2gz+zQtH2wpqD/ac6aaY9BN/OwHC2Xoya57PJ2jqbHO33hWSUL4qap0pA3G +Ji3Ibnsj81ObXaB3d28Pmtp3BHAZOiBNuI3n3mVqSiwsfTefdAWKAswsqf36yL3w +EVWc0J/OnfDUX9nUWX2w8qE5alqMhCFkmYdY2T3bAoGAdMAwNH1gpxBdVbyzN5TU +okIbMrF8lJwTW2PDlqFlQ4OABk2fBytrp+CTGIZmJbrluoml3pPE356WnjLzQU7L +AIIrwCkVjMCX2egYOG+DsDQRjuxuyV9NoNl5hKr8vuQqPSRiPzeLDfuNVDIX36hh +iAI8h+UFEhbfuCuf9spjku8CgYBzjC/ygosyoeb6Mwvg/Kz4viqugw27/0hZIHi9 +JPGr0Au/WKtYRdLVK4uTSPSziaAFAeKYaMFBKryPg3jnsgEn62bTfy1qsrprumiM +zqumX7NIgtl8hGKz0ma7g1t8T+tmAzruL+4+dnfoJISMtCgBZ0R2UGrM68lxrDDC +pe7HLwKBgF9lHHhy76nDW8VMlcEtYIZf329VoqeTMUmvDWFyHAWuY4ZQ4ugAoBUK +9izEbjs0oFHJtF1waWhD9MXJ0BGJK7Zcxy0413CK4dwJT8euSnY81Td7goKTM4Ud +otCqT57JeYWq2hEFromJoSiBgai7weO/E2lAR2Qs99uEPp45q9JQ +-----END RSA PRIVATE KEY----- +` + + newCertByte = `-----BEGIN CERTIFICATE----- +MIIDKDCCAhCgAwIBAgIIYxZk3ye/TxMwDQYJKoZIhvcNAQELBQAwEjEQMA4GA1UE +AxMHZXRjZC1jYTAeFw0yMjEyMjgwMzM4MjRaFw0yMzEyMjgwMzM4MjRaMD4xFzAV +BgNVBAoTDnN5c3RlbTptYXN0ZXJzMSMwIQYDVQQDExprdWJlLWFwaXNlcnZlci1l +dGNkLWNsaWVudDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAKwHHoOt +iwe3aPgqCcKjwdVpu02UuGQO+tjfQNayPeLWwz9QbHRyVOVOeTnMgc9lHmE6XFcn +99CYsqrasUS6k4MJGpbLLzVU/7uja7mj5cO6LcRu3gCtxYanEBFCC6KHx1tWZuUA +UWN+r9UWpBAf1tByhZKLmRHJh/Zca332OOhD79oAQwDmmNt+jSW2f+bGHji1+k8j +OugCV6lDo2K/ywCklL4nnRbdJ0tWDT3J30AotZVlgzt9QDPKLiw+4LxRaFgQQjgP +Da/TZ/A5g2YVXjvUP/tpX3kppJ43Fd2NlXmDlEmKeqq8KH+HAmoG4hnU3g9N2heE +c90oChRfHE2iquMCAwEAAaNWMFQwDgYDVR0PAQH/BAQDAgWgMBMGA1UdJQQMMAoG +CCsGAQUFBwMCMAwGA1UdEwEB/wQCMAAwHwYDVR0jBBgwFoAUn/K9YUtK7mBi+FRD +AiRmCuf3DFMwDQYJKoZIhvcNAQELBQADggEBADFJE6DUF6FRPLECCxfl4fvtoewp +Q1OPTKM3m50GDU8xM8ir6DBHq9UkO4CEXhMnRTezu+39dQ7McJsfp0Ttgq+ImLVF +uH5wsrgwMk24AGpGbVkh5WHaHPChyBFezdSvO8vi+1hxIA8un4caUXzEj/ptKstU +R9glF1lbzAsjxmL80ZOdWsltX5ZxduyDEIkSyqSwAIZaQp+deJdrBUx3UpVKznd7 +/kPv/J2zCjZt8Vp1A+6ikwnFyiIe46Mk/MHCkAvuv5tEh7DFSCtd7ndfT8jlSChz +hO5Jx+cUDzD4du+hY8IwWmTIqBm6hLw31B/qTfd0HMCMf1yDl3ctFwsBKDI= +-----END CERTIFICATE----- +` + newKeyByte = `-----BEGIN RSA PRIVATE KEY----- +MIIEpAIBAAKCAQEArAceg62LB7do+CoJwqPB1Wm7TZS4ZA762N9A1rI94tbDP1Bs +dHJU5U55OcyBz2UeYTpcVyf30JiyqtqxRLqTgwkalssvNVT/u6NruaPlw7otxG7e +AK3FhqcQEUILoofHW1Zm5QBRY36v1RakEB/W0HKFkouZEcmH9lxrffY46EPv2gBD +AOaY236NJbZ/5sYeOLX6TyM66AJXqUOjYr/LAKSUviedFt0nS1YNPcnfQCi1lWWD +O31AM8ouLD7gvFFoWBBCOA8Nr9Nn8DmDZhVeO9Q/+2lfeSmknjcV3Y2VeYOUSYp6 +qrwof4cCagbiGdTeD03aF4Rz3SgKFF8cTaKq4wIDAQABAoIBAHBVxctfDCbh0h4b +9Xuwy+a8wJ8Musw8K/pq70BD7L2wWJeDwQ7Zii6ja+4eabYw5gG/xoTziJQi4qlH +XfLvk1xCGabWz+EXvFefg70aFfQWI8TeUQJId3BSr99VLZvY5onyhgaMiplaJSAV +RNVytSgxYKAtoKtI2ww5lcgPfWHNyQJaJ1WnFclImzbEcFirJHBX+u7ATLPNJs1v +rylPiayVB6zQwKTolPchvgJsCdPGP9iopEAhY0ccduKvqNPcDakGJJYUli0l+b+X +cBp+K8pG8UeWF4NxVNWKlMtfIDg0RkJ3/fI+0M9fyCVU5eSPTP7YMfv3fSIfz4Vx +A/N6ikECgYEAyQqaPNv1Qk54II1SrXf4h8uIM+/eQtZDZpBr4gEuYLCLww3mHnae +V/KJbcoqohEpsQ56n0ndWg3Sw3nvLpomwdg8YJqgY2tlEl0pl3LvXicP7aXWyuj/ +FS8oJKQfFkiIH3Env81+TCpEH4HIQGCgjE8vV5eUy00Vqqo4fUvPz7kCgYEA2w4R +0CpDmqVw06F15H3PBEGzOQQof6RKYCEC2F6VunH8amgQaSEQQoLxfP6/feJzpHb6 +mvXft5Uccc7dkJDr7Wn1iaHgMwze2Qvpqdm/bvt1jhcHqa6SsOQjk+VBWSByBrby +DZFvUwxNiXWsdqUxoVIFkoe6SyoKFX7F7AC1RXsCgYBxaMO9VS+zqeRmKJLdPHI8 +2HoLImM1PP1knE/ffF8XOEB/VhXcVXnZjv4rqwIFzrzAHrTZqqdtp6KfludwWJFI +hJz6uf+EVg78HwXZY4LYkBySKR1T9b//yUxR7yuCPIRdiE2uC1QVzzoCtAmtF1U6 +EWlZdi7/yIpSbhfTxrKCMQKBgQCQNC/n0JrWiEjBGM5qT6PjUnjwdNtQQ9AufizI +UWPR7E3VopIDEyAIGPluZqma7mNghm6tamUPDps+FIdpLu4RSaq5IxZbpQJi8eOt +y8mo/uLBWknSGzk4N8dwCgC98oz9/JtV8ULO8g9tCUkyhccpQrymXLF338Hpqp4S +odizVwKBgQCImXprzRCsIJvjsz7pbqj6fvfev/9xxnmlZhHBQq8PRdBubA2wRRLn +lrVcO/z7xgv9knoKvSQ5lZRtACA4/u3ZOzBRr56ZtkvbWH0Ch1QafJ7suomsMHAx +KAGM4g6DY68asv37ATNrYjLZ0MGsArWhKXsbxiR9CrzrNFVVtVIc6g== +-----END RSA PRIVATE KEY-----` +) + +var ( + fileStore = fs.FileSystemOperator{} + secretGVR = schema.GroupVersionResource{Group: "", Version: "v1", Resource: "secrets"} + poolCoordinatorSecret = &corev1.Secret{ + ObjectMeta: v1.ObjectMeta{ + Name: constants.PoolCoordinatorClientSecretName, + Namespace: constants.PoolCoordinatorClientSecretNamespace, + }, + TypeMeta: v1.TypeMeta{ + Kind: "Secret", + APIVersion: "v1", + }, + Data: map[string][]byte{ + "ca.crt": []byte(caByte), + "pool-coordinator-yurthub-client.crt": []byte(certByte), + "pool-coordinator-yurthub-client.key": []byte(keyByte), + }, + } + + // Used to test FieldSelector + otherSecret = &corev1.Secret{ + ObjectMeta: v1.ObjectMeta{ + Name: "default-token", + Namespace: "default", + }, + TypeMeta: v1.TypeMeta{ + Kind: "Secret", + APIVersion: "v1", + }, + Data: map[string][]byte{ + "token": []byte("token"), + }, + } +) + +func TestSecretAdd(t *testing.T) { + t.Run("CertManager should not react for secret that is not pool-coordinator-yurthub-certs", func(t *testing.T) { + fakeClient, certMgr, cancel, err := initFakeClientAndCertManager() + if err != nil { + t.Errorf("failed to initialize, %v", err) + } + defer cancel() + + if err := fakeClient.Tracker().Add(otherSecret); err != nil { + t.Errorf("failed to add secret %s, %v", otherSecret.Name, err) + } + + // Expect to timeout which indicates the CertManager does not save the cert + // that is not pool-coordinator-yurthub-certs. + err = wait.PollImmediate(50*time.Millisecond, 10*time.Second, func() (done bool, err error) { + if certMgr.cert != nil { + return false, fmt.Errorf("unexpect cert initialization") + } + + if _, err := fileStore.Read(certMgr.GetCaFile()); err == nil { + return false, fs.ErrExists + } else if err != fs.ErrNotExists { + return false, err + } + + return false, nil + }) + + if err != wait.ErrWaitTimeout { + t.Errorf("CertManager should not react for add event of secret that is not pool-coordinator-yurthub-certs, %v", err) + } + + if err := fileStore.DeleteDir(testPKIDir); err != nil { + t.Errorf("failed to clean test dir %s, %v", testPKIDir, err) + } + }) + + t.Run("CertManager should react for pool-coordinator-yurthub-certs", func(t *testing.T) { + fakeClient, certMgr, cancel, err := initFakeClientAndCertManager() + if err != nil { + t.Errorf("failed to initialize, %v", err) + } + defer cancel() + + if err := fakeClient.Tracker().Add(poolCoordinatorSecret); err != nil { + t.Errorf("failed to add secret %s, %v", poolCoordinatorSecret.Name, err) + } + + err = wait.PollImmediate(50*time.Millisecond, 10*time.Second, func() (done bool, err error) { + return checkSecret(certMgr, poolCoordinatorSecret) + }) + + if err != nil { + t.Errorf("failed to check poolcoordinator cert, %v", err) + } + + if err := fileStore.DeleteDir(testPKIDir); err != nil { + t.Errorf("failed to clean test dir %s, %v", testPKIDir, err) + } + }) +} + +func TestSecretUpdate(t *testing.T) { + t.Run("CertManager should update cert files when secret is updated", func(t *testing.T) { + fakeClient, certMgr, cancel, err := initFakeClientAndCertManager() + if err != nil { + t.Errorf("failed to initialize, %v", err) + } + defer cancel() + + if err := fakeClient.Tracker().Add(poolCoordinatorSecret); err != nil { + t.Errorf("failed to add secret %s, %v", poolCoordinatorSecret.Name, err) + } + + err = wait.Poll(50*time.Millisecond, 10*time.Second, func() (done bool, err error) { + return checkSecret(certMgr, poolCoordinatorSecret) + }) + if err != nil { + t.Errorf("failed to wait cert manager to be initialized, %v", err) + } + + newSecret := poolCoordinatorSecret.DeepCopy() + newSecret.Data["pool-coordinator-yurthub-client.key"] = []byte(newKeyByte) + newSecret.Data["pool-coordinator-yurthub-client.crt"] = []byte(newCertByte) + if err := fakeClient.Tracker().Update(secretGVR, newSecret, newSecret.Namespace); err != nil { + t.Errorf("failed to update secret, %v", err) + } + + err = wait.PollImmediate(50*time.Millisecond, 10*time.Second, func() (done bool, err error) { + return checkSecret(certMgr, newSecret) + }) + if err != nil { + t.Errorf("failed to wait cert manager to be updated, %v", err) + } + + if err := fileStore.DeleteDir(testPKIDir); err != nil { + t.Errorf("failed to clean test dir %s, %v", testPKIDir, err) + } + }) +} + +func TestSecretDelete(t *testing.T) { + t.Run("Cert manager should clean cert when secret has been deleted", func(t *testing.T) { + fakeClient, certMgr, cancel, err := initFakeClientAndCertManager() + if err != nil { + t.Errorf("failed to initialize, %v", err) + } + defer cancel() + + if err := fakeClient.Tracker().Add(poolCoordinatorSecret); err != nil { + t.Errorf("failed to add secret %s, %v", poolCoordinatorSecret.Name, err) + } + + err = wait.PollImmediate(50*time.Millisecond, 10*time.Second, func() (done bool, err error) { + return checkSecret(certMgr, poolCoordinatorSecret) + }) + if err != nil { + t.Errorf("failed to wait cert manager to be initialized, %v", err) + } + + if err := fakeClient.Tracker().Delete(secretGVR, poolCoordinatorSecret.Namespace, poolCoordinatorSecret.Name); err != nil { + t.Errorf("failed to delete secret, %v", err) + } + + err = wait.PollImmediate(50*time.Millisecond, 10*time.Second, func() (done bool, err error) { + if certMgr.cert == nil { + return true, nil + } + return false, nil + }) + if err != nil { + t.Errorf("failed to clean cert, %v", err) + } + + if err := fileStore.DeleteDir(testPKIDir); err != nil { + t.Errorf("failed to clean test dir %s, %v", testPKIDir, err) + } + }) +} + +func initFakeClientAndCertManager() (*fake.Clientset, *CertManager, func(), error) { + fakeClientSet := fake.NewSimpleClientset() + fakeInformerFactory := informers.NewSharedInformerFactory(fakeClientSet, 0) + certMgr, err := NewCertManager(testPKIDir, fakeClientSet, fakeInformerFactory) + if err != nil { + return nil, nil, nil, fmt.Errorf("failed to create cert manager, %v", err) + } + stopCh := make(chan struct{}) + fakeInformerFactory.Start(stopCh) + + return fakeClientSet, certMgr, func() { close(stopCh) }, nil +} + +func checkSecret(certMgr *CertManager, secret *corev1.Secret) (bool, error) { + if certMgr.secret == nil { + return false, nil + } + if !reflect.DeepEqual(certMgr.secret, secret) { + return false, nil + } + + files := []struct { + path string + expect []byte + }{ + { + path: certMgr.GetFilePath(RootCA), + expect: secret.Data["ca.crt"], + }, + { + path: certMgr.GetFilePath(YurthubClientCert), + expect: secret.Data["pool-coordinator-yurthub-client.crt"], + }, + { + path: certMgr.GetFilePath(YurthubClientKey), + expect: secret.Data["pool-coordinator-yurthub-client.key"], + }, + } + + for _, f := range files { + buf, err := fileStore.Read(f.path) + if err != nil { + return false, fmt.Errorf("failed to read file at %s, %v", f.path, err) + } + if string(buf) != string(f.expect) { + return false, fmt.Errorf("unexpected value of file %s", f.path) + } + } + + return true, nil +} From edf7ccd23aedfeb2aab4a67cc69d0e434b6fb07f Mon Sep 17 00:00:00 2001 From: Laurence <45508533+LaurenceLiZhixin@users.noreply.github.com> Date: Thu, 12 Jan 2023 14:05:05 +0800 Subject: [PATCH 12/20] Fix (#1141) --- charts/openyurt/templates/yurt-controller-manager.yaml | 6 ++++-- cmd/yurthub/app/options/options.go | 2 ++ pkg/yurthub/storage/etcd/storage.go | 10 +++++++--- pkg/yurthub/util/util.go | 9 +++++++++ 4 files changed, 22 insertions(+), 5 deletions(-) diff --git a/charts/openyurt/templates/yurt-controller-manager.yaml b/charts/openyurt/templates/yurt-controller-manager.yaml index 9a81eb42e08..32f7e8e06a4 100644 --- a/charts/openyurt/templates/yurt-controller-manager.yaml +++ b/charts/openyurt/templates/yurt-controller-manager.yaml @@ -159,8 +159,10 @@ spec: spec: serviceAccountName: yurt-controller-manager hostNetwork: true + imagePullSecrets: + - name: regsecret tolerations: - - operator: "Exists" + - operator: "Exists" affinity: nodeAffinity: # we prefer allocating ecm on cloud node @@ -180,4 +182,4 @@ spec: - yurt-controller-manager {{- if .Values.imagePullSecrets }} imagePullSecrets: {{ toYaml .Values.imagePullSecrets | nindent 8 }} - {{- end }} \ No newline at end of file + {{- end }} diff --git a/cmd/yurthub/app/options/options.go b/cmd/yurthub/app/options/options.go index ffa241b16d4..db34ca60e01 100644 --- a/cmd/yurthub/app/options/options.go +++ b/cmd/yurthub/app/options/options.go @@ -117,6 +117,8 @@ func NewYurtHubOptions() *YurtHubOptions { MinRequestTimeout: time.Second * 1800, CACertHashes: make([]string, 0), UnsafeSkipCAVerification: true, + CoordinatorServerAddr: fmt.Sprintf("https://%s:%s", util.DefaultPoolCoordinatorAPIServerSvcName, util.DefaultPoolCoordinatorAPIServerSvcPort), + CoordinatorStorageAddr: fmt.Sprintf("https://%s:%s", util.DefaultPoolCoordinatorEtcdSvcName, util.DefaultPoolCoordinatorEtcdSvcPort), CoordinatorStoragePrefix: "/registry", LeaderElection: componentbaseconfig.LeaderElectionConfiguration{ LeaderElect: true, diff --git a/pkg/yurthub/storage/etcd/storage.go b/pkg/yurthub/storage/etcd/storage.go index 7e5df7b0b61..1e33a7de827 100644 --- a/pkg/yurthub/storage/etcd/storage.go +++ b/pkg/yurthub/storage/etcd/storage.go @@ -42,6 +42,8 @@ const ( defaultTimeout = 5 * time.Second defaultHealthCheckPeriod = 10 * time.Second defaultDialTimeout = 10 * time.Second + defaultMaxSendSize = 100 * 1024 * 1024 + defaultMaxReceiveSize = 100 * 1024 * 1024 defaultComponentCacheFileName = "component-key-cache" defaultRvLen = 32 ) @@ -106,9 +108,11 @@ func NewStorage(ctx context.Context, cfg *EtcdStorageConfig) (storage.Store, err } clientConfig := clientv3.Config{ - Endpoints: cfg.EtcdEndpoints, - TLS: tlsConfig, - DialTimeout: defaultDialTimeout, + Endpoints: cfg.EtcdEndpoints, + TLS: tlsConfig, + DialTimeout: defaultDialTimeout, + MaxCallRecvMsgSize: defaultMaxReceiveSize, + MaxCallSendMsgSize: defaultMaxSendSize, } client, err := clientv3.New(clientConfig) diff --git a/pkg/yurthub/util/util.go b/pkg/yurthub/util/util.go index e6ae72bedb4..56f2d8cc3e0 100644 --- a/pkg/yurthub/util/util.go +++ b/pkg/yurthub/util/util.go @@ -61,6 +61,15 @@ const ( ProxyListSelector // ProxyPoolScopedResource represents if this request is asking for pool-scoped resources ProxyPoolScopedResource + // DefaultPoolCoordinatorEtcdSvcName represents default pool coordinator etcd service + DefaultPoolCoordinatorEtcdSvcName = "pool-coordinator-etcd" + // DefaultPoolCoordinatorAPIServerSvcName represents default pool coordinator apiServer service + DefaultPoolCoordinatorAPIServerSvcName = "pool-coordinator-apiserver" + // DefaultPoolCoordinatorEtcdSvcPort represents default pool coordinator etcd port + DefaultPoolCoordinatorEtcdSvcPort = "2379" + // DefaultPoolCoordinatorAPIServerSvcPort represents default pool coordinator apiServer port + DefaultPoolCoordinatorAPIServerSvcPort = "443" + YurtHubNamespace = "kube-system" CacheUserAgentsKey = "cache_agents" From 68b3de92d6e31af33ff2e01889ba43d0128de440 Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Thu, 12 Jan 2023 20:24:02 +0800 Subject: [PATCH 13/20] fix coordinator sync problem and update ready condition (#1142) * make coordinator fields update atomic Signed-off-by: Congrool --- pkg/yurthub/poolcoordinator/coordinator.go | 24 ++++++++++------------ 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/pkg/yurthub/poolcoordinator/coordinator.go b/pkg/yurthub/poolcoordinator/coordinator.go index d70f9b357dc..6812cafd3bc 100644 --- a/pkg/yurthub/poolcoordinator/coordinator.go +++ b/pkg/yurthub/poolcoordinator/coordinator.go @@ -185,10 +185,12 @@ func NewCoordinator( func (coordinator *coordinator) Run() { for { + var poolCacheManager cachemanager.CacheManager var cancelEtcdStorage = func() {} var needUploadLocalCache bool var needCancelEtcdStorage bool var isPoolCacheSynced bool + var etcdStorage storage.Store var err error select { @@ -211,13 +213,10 @@ func (coordinator *coordinator) Run() { needUploadLocalCache = true needCancelEtcdStorage = true isPoolCacheSynced = false - coordinator.Lock() - coordinator.poolCacheManager = nil - coordinator.Unlock() + etcdStorage = nil + poolCacheManager = nil case LeaderHub: - coordinator.Lock() - coordinator.poolCacheManager, coordinator.etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() - coordinator.Unlock() + poolCacheManager, etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() if err != nil { klog.Errorf("failed to create pool scoped cache store and manager, %v", err) continue @@ -246,16 +245,14 @@ func (coordinator *coordinator) Run() { coordinator.poolCacheSyncedDetector.EnsureStart() if coordinator.needUploadLocalCache { - if err := coordinator.uploadLocalCache(coordinator.etcdStorage); err != nil { + if err := coordinator.uploadLocalCache(etcdStorage); err != nil { klog.Errorf("failed to upload local cache when yurthub becomes leader, %v", err) } else { needUploadLocalCache = false } } case FollowerHub: - coordinator.Lock() - coordinator.poolCacheManager, coordinator.etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() - coordinator.Unlock() + poolCacheManager, etcdStorage, cancelEtcdStorage, err = coordinator.buildPoolCacheStore() if err != nil { klog.Errorf("failed to create pool scoped cache store and manager, %v", err) continue @@ -266,7 +263,7 @@ func (coordinator *coordinator) Run() { coordinator.poolCacheSyncedDetector.EnsureStart() if coordinator.needUploadLocalCache { - if err := coordinator.uploadLocalCache(coordinator.etcdStorage); err != nil { + if err := coordinator.uploadLocalCache(etcdStorage); err != nil { klog.Errorf("failed to upload local cache when yurthub becomes follower, %v", err) } else { needUploadLocalCache = false @@ -282,6 +279,8 @@ func (coordinator *coordinator) Run() { cancelEtcdStorage() } coordinator.electStatus = electorStatus + coordinator.poolCacheManager = poolCacheManager + coordinator.etcdStorage = etcdStorage coordinator.cancelEtcdStorage = cancelEtcdStorage coordinator.needUploadLocalCache = needUploadLocalCache coordinator.isPoolCacheSynced = isPoolCacheSynced @@ -300,8 +299,7 @@ func (coordinator *coordinator) IsReady() (cachemanager.CacheManager, bool) { // If electStatus is not PendingHub, it means pool-coordinator is healthy. coordinator.Lock() defer coordinator.Unlock() - // fixme: coordinator.isPoolCacheSynced now is not considered - if coordinator.electStatus != PendingHub && !coordinator.needUploadLocalCache { + if coordinator.electStatus != PendingHub && coordinator.isPoolCacheSynced && !coordinator.needUploadLocalCache { return coordinator.poolCacheManager, true } return nil, false From 157ac4522f35fe8ba3e2499d8c6458784577ee5a Mon Sep 17 00:00:00 2001 From: Laurence <45508533+LaurenceLiZhixin@users.noreply.github.com> Date: Fri, 13 Jan 2023 14:18:56 +0800 Subject: [PATCH 14/20] Fix: dns bug for endpointslices v1beta1 version (#1144) --- pkg/yurthub/server/nonresource.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/pkg/yurthub/server/nonresource.go b/pkg/yurthub/server/nonresource.go index 327c8eaf0c4..3768f0170a8 100644 --- a/pkg/yurthub/server/nonresource.go +++ b/pkg/yurthub/server/nonresource.go @@ -42,9 +42,12 @@ type NonResourceHandler func(kubeClient *kubernetes.Clientset, sw cachemanager.S func wrapNonResourceHandler(proxyHandler http.Handler, config *config.YurtHubConfiguration, restMgr *rest.RestConfigManager) http.Handler { wrapMux := mux.NewRouter() - // register handler for non resource requests - for path := range nonResourceReqPaths { - wrapMux.Handle(path, localCacheHandler(nonResourceHandler, restMgr, config.StorageWrapper, path)).Methods("GET") + + if !config.EnableCoordinator { + // register handler for non resource requests + for path := range nonResourceReqPaths { + wrapMux.Handle(path, localCacheHandler(nonResourceHandler, restMgr, config.StorageWrapper, path)).Methods("GET") + } } // register handler for other requests From 7886dbc855179f8b4f52ce98e94c5f052e86e7ee Mon Sep 17 00:00:00 2001 From: Laurence <45508533+LaurenceLiZhixin@users.noreply.github.com> Date: Fri, 13 Jan 2023 21:44:32 +0800 Subject: [PATCH 15/20] Fix: set inElecting to false when handling follow hub, and add metrics (#1145) * fix: add metrics --- cmd/yurthub/app/start.go | 26 +++---- pkg/yurthub/metrics/metrics.go | 73 +++++++++++++++---- pkg/yurthub/poolcoordinator/coordinator.go | 9 +++ .../poolcoordinator/leader_election.go | 3 +- 4 files changed, 82 insertions(+), 29 deletions(-) diff --git a/cmd/yurthub/app/start.go b/cmd/yurthub/app/start.go index f4ce630b2a6..a491b65adc7 100644 --- a/cmd/yurthub/app/start.go +++ b/cmd/yurthub/app/start.go @@ -160,12 +160,11 @@ func Run(ctx context.Context, cfg *config.YurtHubConfiguration) error { // coordinatorRun will register secret informer into sharedInformerFactory, and start a new goroutine to periodically check // if certs has been got from cloud APIServer. It will close the coordinatorInformerRegistryChan if the secret channel has // been registered into informer factory. - coordinatorHealthCheckerGetter, coordinatorTransportManagerGetter, coordinatorGetter, err = coordinatorRun(ctx, cfg, restConfigMgr, cloudHealthChecker, coordinatorInformerRegistryChan) - if err != nil { - return fmt.Errorf("failed to wait for coordinator to run, %v", err) - } + coordinatorHealthCheckerGetter, coordinatorTransportManagerGetter, coordinatorGetter = coordinatorRun(ctx, cfg, restConfigMgr, cloudHealthChecker, coordinatorInformerRegistryChan) // wait for coordinator informer registry + klog.Infof("waiting for coordinator informer registry") <-coordinatorInformerRegistryChan + klog.Infof("coordinator informer registry finished") } // Start the informer factory if all informers have been registered @@ -227,23 +226,23 @@ func coordinatorRun(ctx context.Context, cfg *config.YurtHubConfiguration, restConfigMgr *hubrest.RestConfigManager, cloudHealthChecker healthchecker.MultipleBackendsHealthChecker, - coordinatorInformerRegistryChan chan struct{}) (func() healthchecker.HealthChecker, func() transport.Interface, func() poolcoordinator.Coordinator, error) { + coordinatorInformerRegistryChan chan struct{}) (func() healthchecker.HealthChecker, func() transport.Interface, func() poolcoordinator.Coordinator) { var coordinatorHealthChecker healthchecker.HealthChecker var coordinatorTransportMgr transport.Interface var coordinator poolcoordinator.Coordinator - var returnErr error go func() { coorCertManager, err := coordinatorcertmgr.NewCertManager(cfg.CoordinatorPKIDir, cfg.ProxiedClient, cfg.SharedFactory) close(coordinatorInformerRegistryChan) // notify the coordinator secret informer registry event if err != nil { - returnErr = fmt.Errorf("failed to create coordinator cert manager, %v", err) + klog.Errorf("coordinator failed to create coordinator cert manager, %v", err) return } + klog.Infof("coordinator new certManager success") coorTransportMgr, err := poolCoordinatorTransportMgrGetter(cfg.HeartbeatTimeoutSeconds, cfg.CoordinatorServerURL, coorCertManager, ctx.Done()) if err != nil { - returnErr = fmt.Errorf("failed to create coordinator transport manager, %v", err) + klog.Errorf("coordinator failed to create coordinator transport manager, %v", err) return } @@ -253,27 +252,27 @@ func coordinatorRun(ctx context.Context, Timeout: time.Duration(cfg.HeartbeatTimeoutSeconds) * time.Second, }) if err != nil { - returnErr = fmt.Errorf("failed to get coordinator client for pool coordinator, %v", err) + klog.Errorf("coordinator failed to get coordinator client for pool coordinator, %v", err) return } coorHealthChecker, err := healthchecker.NewCoordinatorHealthChecker(cfg, coordinatorClient, cloudHealthChecker, ctx.Done()) if err != nil { - returnErr = fmt.Errorf("failed to create coordinator health checker, %v", err) + klog.Errorf("coordinator failed to create coordinator health checker, %v", err) return } var elector *poolcoordinator.HubElector elector, err = poolcoordinator.NewHubElector(cfg, coordinatorClient, coorHealthChecker, cloudHealthChecker, ctx.Done()) if err != nil { - returnErr = fmt.Errorf("failed to create hub elector, %v", err) + klog.Errorf("coordinator failed to create hub elector, %v", err) return } go elector.Run(ctx.Done()) coor, err := poolcoordinator.NewCoordinator(ctx, cfg, restConfigMgr, coorCertManager, coorTransportMgr, elector) if err != nil { - returnErr = fmt.Errorf("failed to create coordinator, %v", err) + klog.Errorf("coordinator failed to create coordinator, %v", err) return } go coor.Run() @@ -281,7 +280,6 @@ func coordinatorRun(ctx context.Context, coordinatorTransportMgr = coorTransportMgr coordinatorHealthChecker = coorHealthChecker coordinator = coor - returnErr = nil }() return func() healthchecker.HealthChecker { @@ -290,7 +288,7 @@ func coordinatorRun(ctx context.Context, return coordinatorTransportMgr }, func() poolcoordinator.Coordinator { return coordinator - }, returnErr + } } func poolCoordinatorTransportMgrGetter(heartbeatTimeoutSeconds int, coordinatorServer *url.URL, coordinatorCertMgr *coordinatorcertmgr.CertManager, stopCh <-chan struct{}) (transport.Interface, error) { diff --git a/pkg/yurthub/metrics/metrics.go b/pkg/yurthub/metrics/metrics.go index be4cb276813..79e95c52912 100644 --- a/pkg/yurthub/metrics/metrics.go +++ b/pkg/yurthub/metrics/metrics.go @@ -44,13 +44,16 @@ var ( ) type HubMetrics struct { - serversHealthyCollector *prometheus.GaugeVec - inFlightRequestsCollector *prometheus.GaugeVec - inFlightRequestsGauge prometheus.Gauge - rejectedRequestsCounter prometheus.Counter - closableConnsCollector *prometheus.GaugeVec - proxyTrafficCollector *prometheus.CounterVec - proxyLatencyCollector *prometheus.GaugeVec + serversHealthyCollector *prometheus.GaugeVec + inFlightRequestsCollector *prometheus.GaugeVec + inFlightRequestsGauge prometheus.Gauge + rejectedRequestsCounter prometheus.Counter + closableConnsCollector *prometheus.GaugeVec + proxyTrafficCollector *prometheus.CounterVec + proxyLatencyCollector *prometheus.GaugeVec + poolCoordinatorYurthubRoleCollector *prometheus.GaugeVec + poolCoordinatorHealthyStatusCollector *prometheus.GaugeVec + poolCoordinatorReadyStatusCollector *prometheus.GaugeVec } func newHubMetrics() *HubMetrics { @@ -108,6 +111,30 @@ func newHubMetrics() *HubMetrics { Help: "collector of proxy latency of incoming requests(unit: ms)", }, []string{"client", "verb", "resource", "subresources", "type"}) + poolCoordinatorYurthubRoleCollector := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "pool_coordinator_yurthub_role", + Help: "pool coordinator status of yurthub. 1: LeaderHub, 2: FollowerHub 3: Pending", + }, + []string{}) + poolCoordinatorHealthyStatusCollector := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "pool_coordinator_healthy_status", + Help: "pool coordinator heahty status 1: healthy, 0: unhealthy", + }, + []string{}) + poolCoordinatorReadyStatusCollector := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "pool_coordinator_ready_status", + Help: "pool coordinator ready status 1: ready, 0: notReady", + }, + []string{}) prometheus.MustRegister(serversHealthyCollector) prometheus.MustRegister(inFlightRequestsCollector) prometheus.MustRegister(inFlightRequestsGauge) @@ -115,14 +142,20 @@ func newHubMetrics() *HubMetrics { prometheus.MustRegister(closableConnsCollector) prometheus.MustRegister(proxyTrafficCollector) prometheus.MustRegister(proxyLatencyCollector) + prometheus.MustRegister(poolCoordinatorYurthubRoleCollector) + prometheus.MustRegister(poolCoordinatorHealthyStatusCollector) + prometheus.MustRegister(poolCoordinatorReadyStatusCollector) return &HubMetrics{ - serversHealthyCollector: serversHealthyCollector, - inFlightRequestsCollector: inFlightRequestsCollector, - inFlightRequestsGauge: inFlightRequestsGauge, - rejectedRequestsCounter: rejectedRequestsCounter, - closableConnsCollector: closableConnsCollector, - proxyTrafficCollector: proxyTrafficCollector, - proxyLatencyCollector: proxyLatencyCollector, + serversHealthyCollector: serversHealthyCollector, + inFlightRequestsCollector: inFlightRequestsCollector, + inFlightRequestsGauge: inFlightRequestsGauge, + rejectedRequestsCounter: rejectedRequestsCounter, + closableConnsCollector: closableConnsCollector, + proxyTrafficCollector: proxyTrafficCollector, + proxyLatencyCollector: proxyLatencyCollector, + poolCoordinatorHealthyStatusCollector: poolCoordinatorHealthyStatusCollector, + poolCoordinatorReadyStatusCollector: poolCoordinatorReadyStatusCollector, + poolCoordinatorYurthubRoleCollector: poolCoordinatorYurthubRoleCollector, } } @@ -139,6 +172,18 @@ func (hm *HubMetrics) ObserveServerHealthy(server string, status int) { hm.serversHealthyCollector.WithLabelValues(server).Set(float64(status)) } +func (hm *HubMetrics) ObservePoolCoordinatorYurthubRole(status int32) { + hm.poolCoordinatorYurthubRoleCollector.WithLabelValues().Set(float64(status)) +} + +func (hm *HubMetrics) ObservePoolCoordinatorReadyStatus(status int32) { + hm.poolCoordinatorReadyStatusCollector.WithLabelValues().Set(float64(status)) +} + +func (hm *HubMetrics) ObservePoolCoordinatorHealthyStatus(status int32) { + hm.poolCoordinatorHealthyStatusCollector.WithLabelValues().Set(float64(status)) +} + func (hm *HubMetrics) IncInFlightRequests(verb, resource, subresource, client string) { hm.inFlightRequestsCollector.WithLabelValues(verb, resource, subresource, client).Inc() hm.inFlightRequestsGauge.Inc() diff --git a/pkg/yurthub/poolcoordinator/coordinator.go b/pkg/yurthub/poolcoordinator/coordinator.go index 6812cafd3bc..9b31e04fc2b 100644 --- a/pkg/yurthub/poolcoordinator/coordinator.go +++ b/pkg/yurthub/poolcoordinator/coordinator.go @@ -44,6 +44,7 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/meta" yurtrest "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/rest" "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/serializer" + "github.com/openyurtio/openyurt/pkg/yurthub/metrics" "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/certmanager" "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" "github.com/openyurtio/openyurt/pkg/yurthub/storage" @@ -204,6 +205,7 @@ func (coordinator *coordinator) Run() { if !ok { return } + metrics.Metrics.ObservePoolCoordinatorYurthubRole(electorStatus) switch electorStatus { case PendingHub: @@ -227,10 +229,12 @@ func (coordinator *coordinator) Run() { klog.Errorf("cloud not get cloud lease client when becoming leader yurthub, %v", err) continue } + klog.Infof("coordinator newCloudLeaseClient success.") if err := coordinator.poolCacheSyncManager.EnsureStart(); err != nil { klog.Errorf("failed to sync pool-scoped resource, %v", err) continue } + klog.Infof("coordinator poolCacheSyncManager has ensure started") coordinator.delegateNodeLeaseManager.EnsureStartWithHandler(cache.FilteringResourceEventHandler{ FilterFunc: ifDelegateHeartBeat, Handler: cache.ResourceEventHandlerFuncs{ @@ -300,8 +304,10 @@ func (coordinator *coordinator) IsReady() (cachemanager.CacheManager, bool) { coordinator.Lock() defer coordinator.Unlock() if coordinator.electStatus != PendingHub && coordinator.isPoolCacheSynced && !coordinator.needUploadLocalCache { + metrics.Metrics.ObservePoolCoordinatorReadyStatus(1) return coordinator.poolCacheManager, true } + metrics.Metrics.ObservePoolCoordinatorReadyStatus(0) return nil, false } @@ -311,8 +317,10 @@ func (coordinator *coordinator) IsHealthy() (cachemanager.CacheManager, bool) { coordinator.Lock() defer coordinator.Unlock() if coordinator.electStatus != PendingHub { + metrics.Metrics.ObservePoolCoordinatorHealthyStatus(1) return coordinator.poolCacheManager, true } + metrics.Metrics.ObservePoolCoordinatorHealthyStatus(0) return nil, false } @@ -420,6 +428,7 @@ func (p *poolScopedCacheSyncManager) EnsureStart() error { hasInformersSynced := []cache.InformerSynced{} informerFactory := informers.NewSharedInformerFactory(p.proxiedClient, 0) for gvr := range constants.PoolScopedResources { + klog.Infof("coordinator informer with resources gvr %+v registered", gvr) informer, err := informerFactory.ForResource(gvr) if err != nil { cancel() diff --git a/pkg/yurthub/poolcoordinator/leader_election.go b/pkg/yurthub/poolcoordinator/leader_election.go index ffafe9a39bf..c8e3003f325 100644 --- a/pkg/yurthub/poolcoordinator/leader_election.go +++ b/pkg/yurthub/poolcoordinator/leader_election.go @@ -80,7 +80,8 @@ func NewHubElector( }, OnStoppedLeading: func() { klog.Infof("yurthub of %s is no more a leader", cfg.NodeName) - he.electorStatus <- PendingHub + he.electorStatus <- FollowerHub + he.inElecting = false }, }, }) From 578120340094b24bd4ead5ec01f9350e27b9c4cb Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Mon, 16 Jan 2023 09:38:25 +0800 Subject: [PATCH 16/20] fix coredns cannot get /apis/discovery.k8s.io/v1 if cluster does not have such resource (#1147) Signed-off-by: Congrool Signed-off-by: Congrool --- pkg/yurthub/server/nonresource.go | 34 ++++++++++++++++-------- pkg/yurthub/server/nonresource_test.go | 36 ++++++++++++++++++++------ 2 files changed, 51 insertions(+), 19 deletions(-) diff --git a/pkg/yurthub/server/nonresource.go b/pkg/yurthub/server/nonresource.go index 3768f0170a8..8c53c328eee 100644 --- a/pkg/yurthub/server/nonresource.go +++ b/pkg/yurthub/server/nonresource.go @@ -25,6 +25,7 @@ import ( "k8s.io/apiserver/pkg/endpoints/handlers/responsewriters" "k8s.io/client-go/kubernetes" "k8s.io/klog/v2" + "k8s.io/utils/pointer" "github.com/openyurtio/openyurt/cmd/yurthub/app/config" "github.com/openyurtio/openyurt/pkg/yurthub/cachemanager" @@ -43,11 +44,9 @@ type NonResourceHandler func(kubeClient *kubernetes.Clientset, sw cachemanager.S func wrapNonResourceHandler(proxyHandler http.Handler, config *config.YurtHubConfiguration, restMgr *rest.RestConfigManager) http.Handler { wrapMux := mux.NewRouter() - if !config.EnableCoordinator { - // register handler for non resource requests - for path := range nonResourceReqPaths { - wrapMux.Handle(path, localCacheHandler(nonResourceHandler, restMgr, config.StorageWrapper, path)).Methods("GET") - } + // register handler for non resource requests + for path := range nonResourceReqPaths { + wrapMux.Handle(path, localCacheHandler(nonResourceHandler, restMgr, config.StorageWrapper, path)).Methods("GET") } // register handler for other requests @@ -64,16 +63,22 @@ func localCacheHandler(handler NonResourceHandler, restMgr *rest.RestConfigManag restCfg := restMgr.GetRestConfig(true) if restCfg == nil { klog.Infof("get %s non resource data from local cache when cloud-edge line off", path) - if nonResourceData, err := sw.GetClusterInfo(key); err != nil { + if nonResourceData, err := sw.GetClusterInfo(key); err == nil { + w.WriteHeader(http.StatusOK) + writeRawJSON(nonResourceData, w) + } else if err == storage.ErrStorageNotFound { + w.WriteHeader(http.StatusNotFound) writeErrResponse(path, err, w) } else { - writeRawJSON(nonResourceData, w) + w.WriteHeader(http.StatusInternalServerError) + writeErrResponse(path, err, w) } return } kubeClient, err := kubernetes.NewForConfig(restCfg) if err != nil { + w.WriteHeader(http.StatusInternalServerError) writeErrResponse(path, err, w) return } @@ -87,11 +92,19 @@ func nonResourceHandler(kubeClient *kubernetes.Clientset, sw cachemanager.Storag ClusterInfoType: nonResourceReqPaths[path], UrlPath: path, } - if nonResourceData, err := kubeClient.RESTClient().Get().AbsPath(path).Do(context.TODO()).Raw(); err != nil { + + result := kubeClient.RESTClient().Get().AbsPath(path).Do(context.TODO()) + code := pointer.IntPtr(0) + result.StatusCode(code) + if result.Error() != nil { + err := result.Error() + w.WriteHeader(*code) writeErrResponse(path, err, w) } else { - writeRawJSON(nonResourceData, w) - sw.SaveClusterInfo(key, nonResourceData) + body, _ := result.Raw() + w.WriteHeader(*code) + writeRawJSON(body, w) + sw.SaveClusterInfo(key, body) } }) } @@ -109,6 +122,5 @@ func writeErrResponse(path string, err error, w http.ResponseWriter) { func writeRawJSON(output []byte, w http.ResponseWriter) { w.Header().Set("Content-Type", "application/json") - w.WriteHeader(http.StatusOK) w.Write(output) } diff --git a/pkg/yurthub/server/nonresource_test.go b/pkg/yurthub/server/nonresource_test.go index da61c6ebbaf..d371faff3b3 100644 --- a/pkg/yurthub/server/nonresource_test.go +++ b/pkg/yurthub/server/nonresource_test.go @@ -44,6 +44,11 @@ import ( var rootDir = "/tmp/cache-local" +var ( + notFoundError = errors.New("not found") + internalError = errors.New("internal error") +) + func TestLocalCacheHandler(t *testing.T) { dStorage, err := disk.NewDiskStorage(rootDir) if err != nil { @@ -65,10 +70,10 @@ func TestLocalCacheHandler(t *testing.T) { statusCode int metav1StatusCode int }{ - "failed to get from local cache": { + "failed to get from local cache, because it does not exist": { path: "/version", initData: nil, - statusCode: http.StatusOK, + statusCode: http.StatusNotFound, metav1StatusCode: http.StatusInternalServerError, }, "get from local cache normally": { @@ -144,25 +149,40 @@ func TestNonResourceHandler(t *testing.T) { initData: []byte("fake resource"), statusCode: http.StatusOK, }, - "failed to get non resource": { + "failed to get non resource because of internal error": { path: "/apis/discovery.k8s.io/v1beta1", - err: errors.New("fake error"), - statusCode: http.StatusOK, + err: internalError, + statusCode: http.StatusInternalServerError, metav1StatusCode: http.StatusInternalServerError, }, + "failed to get non resource because it does not exist": { + path: "/apis/discovery.k8s.io/v1", + err: notFoundError, + statusCode: http.StatusNotFound, + metav1StatusCode: http.StatusNotFound, + }, } for k, tt := range testcases { t.Run(k, func(t *testing.T) { fakeClient := &fakerest.RESTClient{ Client: fakerest.CreateHTTPClient(func(request *http.Request) (*http.Response, error) { - if tt.err == nil { + switch tt.err { + case nil: return &http.Response{ StatusCode: http.StatusOK, Body: ioutil.NopCloser(strings.NewReader(string(tt.initData))), }, nil - } else { - return nil, tt.err + case notFoundError: + return &http.Response{ + StatusCode: http.StatusNotFound, + Body: ioutil.NopCloser(bytes.NewReader([]byte{})), + }, nil + default: + return &http.Response{ + StatusCode: http.StatusInternalServerError, + Body: ioutil.NopCloser(bytes.NewReader([]byte{})), + }, err } }), NegotiatedSerializer: scheme.Codecs.WithoutConversion(), From 735b40a33b7cef00fd78996343e784b07bc10f85 Mon Sep 17 00:00:00 2001 From: Laurence <45508533+LaurenceLiZhixin@users.noreply.github.com> Date: Tue, 17 Jan 2023 15:35:04 +0800 Subject: [PATCH 17/20] Fix: add PoolScopeResource validation and dynamic configuration support (#1148) * Fix: add PoolScopeResource validation and dynamic configuration support --- .../poolcoordinator/constants/constants.go | 7 - pkg/yurthub/poolcoordinator/coordinator.go | 47 +++-- .../poolcoordinator/resources/resources.go | 166 ++++++++++++++++++ .../verifiable_pool_scope_resource.go | 20 +++ pkg/yurthub/proxy/pool/pool.go | 1 + pkg/yurthub/proxy/util/util.go | 15 +- pkg/yurthub/storage/etcd/keycache.go | 3 +- pkg/yurthub/storage/etcd/storage.go | 1 + pkg/yurthub/util/util.go | 5 +- 9 files changed, 231 insertions(+), 34 deletions(-) create mode 100644 pkg/yurthub/poolcoordinator/resources/resources.go create mode 100644 pkg/yurthub/poolcoordinator/resources/verifiable_pool_scope_resource.go diff --git a/pkg/yurthub/poolcoordinator/constants/constants.go b/pkg/yurthub/poolcoordinator/constants/constants.go index 70c044bb3c9..43924dd50b5 100644 --- a/pkg/yurthub/poolcoordinator/constants/constants.go +++ b/pkg/yurthub/poolcoordinator/constants/constants.go @@ -17,17 +17,10 @@ limitations under the License. package constants import ( - "k8s.io/apimachinery/pkg/runtime/schema" - "github.com/openyurtio/openyurt/pkg/yurthub/storage" ) var ( - PoolScopedResources = map[schema.GroupVersionResource]struct{}{ - {Group: "", Version: "v1", Resource: "endpoints"}: {}, - {Group: "discovery.k8s.io", Version: "v1", Resource: "endpointslices"}: {}, - } - UploadResourcesKeyBuildInfo = map[storage.KeyBuildInfo]struct{}{ {Component: "kubelet", Resources: "pods", Group: "", Version: "v1"}: {}, {Component: "kubelet", Resources: "nodes", Group: "", Version: "v1"}: {}, diff --git a/pkg/yurthub/poolcoordinator/coordinator.go b/pkg/yurthub/poolcoordinator/coordinator.go index 9b31e04fc2b..15c269ebccf 100644 --- a/pkg/yurthub/poolcoordinator/coordinator.go +++ b/pkg/yurthub/poolcoordinator/coordinator.go @@ -30,6 +30,8 @@ import ( metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" "k8s.io/apimachinery/pkg/runtime/schema" + "k8s.io/client-go/dynamic" + "k8s.io/client-go/dynamic/dynamicinformer" "k8s.io/client-go/informers" "k8s.io/client-go/kubernetes" coordclientset "k8s.io/client-go/kubernetes/typed/coordination/v1" @@ -47,6 +49,7 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/metrics" "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/certmanager" "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/resources" "github.com/openyurtio/openyurt/pkg/yurthub/storage" "github.com/openyurtio/openyurt/pkg/yurthub/storage/etcd" "github.com/openyurtio/openyurt/pkg/yurthub/transport" @@ -169,9 +172,18 @@ func NewCoordinator( if err != nil { return nil, fmt.Errorf("failed to create proxied client, %v", err) } + + // init pool scope resources + resources.InitPoolScopeResourcesManger(proxiedClient, cfg.SharedFactory) + + dynamicClient, err := buildDynamicClientWithUserAgent(fmt.Sprintf("http://%s", cfg.YurtHubProxyServerAddr), constants.DefaultPoolScopedUserAgent) + if err != nil { + return nil, fmt.Errorf("failed to create dynamic client, %v", err) + } + poolScopedCacheSyncManager := &poolScopedCacheSyncManager{ ctx: ctx, - proxiedClient: proxiedClient, + dynamicClient: dynamicClient, coordinatorClient: coordinatorClient, nodeName: cfg.NodeName, getEtcdStore: coordinator.getEtcdStore, @@ -185,6 +197,9 @@ func NewCoordinator( } func (coordinator *coordinator) Run() { + // waiting for pool scope resource synced + resources.WaitUntilPoolScopeResourcesSync(coordinator.ctx) + for { var poolCacheManager cachemanager.CacheManager var cancelEtcdStorage = func() {} @@ -398,8 +413,8 @@ func (coordinator *coordinator) delegateNodeLease(cloudLeaseClient coordclientse type poolScopedCacheSyncManager struct { ctx context.Context isRunning bool - // proxiedClient is a client of Cloud APIServer which is proxied by yurthub. - proxiedClient kubernetes.Interface + // dynamicClient is a dynamic client of Cloud APIServer which is proxied by yurthub. + dynamicClient dynamic.Interface // coordinatorClient is a client of APIServer in pool-coordinator. coordinatorClient kubernetes.Interface // nodeName will be used to update the ownerReference of informer synced lease. @@ -426,18 +441,14 @@ func (p *poolScopedCacheSyncManager) EnsureStart() error { ctx, cancel := context.WithCancel(p.ctx) hasInformersSynced := []cache.InformerSynced{} - informerFactory := informers.NewSharedInformerFactory(p.proxiedClient, 0) - for gvr := range constants.PoolScopedResources { + dynamicInformerFactory := dynamicinformer.NewFilteredDynamicSharedInformerFactory(p.dynamicClient, 0, metav1.NamespaceAll, nil) + for _, gvr := range resources.GetPoolScopeResources() { klog.Infof("coordinator informer with resources gvr %+v registered", gvr) - informer, err := informerFactory.ForResource(gvr) - if err != nil { - cancel() - return fmt.Errorf("failed to add informer for %s, %v", gvr.String(), err) - } + informer := dynamicInformerFactory.ForResource(gvr) hasInformersSynced = append(hasInformersSynced, informer.Informer().HasSynced) } - informerFactory.Start(ctx.Done()) + dynamicInformerFactory.Start(ctx.Done()) go p.holdInformerSync(ctx, hasInformersSynced) p.cancel = cancel p.isRunning = true @@ -697,3 +708,17 @@ func buildProxiedClientWithUserAgent(proxyAddr string, userAgent string) (kubern } return client, nil } + +func buildDynamicClientWithUserAgent(proxyAddr string, userAgent string) (dynamic.Interface, error) { + kubeConfig, err := clientcmd.BuildConfigFromFlags(proxyAddr, "") + if err != nil { + return nil, err + } + + kubeConfig.UserAgent = userAgent + client, err := dynamic.NewForConfig(kubeConfig) + if err != nil { + return nil, err + } + return client, nil +} diff --git a/pkg/yurthub/poolcoordinator/resources/resources.go b/pkg/yurthub/poolcoordinator/resources/resources.go new file mode 100644 index 00000000000..6826a31ae0b --- /dev/null +++ b/pkg/yurthub/poolcoordinator/resources/resources.go @@ -0,0 +1,166 @@ +package resources + +import ( + "context" + "encoding/json" + "sync" + "time" + + corev1 "k8s.io/api/core/v1" + apierrors "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/apimachinery/pkg/runtime/schema" + apirequest "k8s.io/apiserver/pkg/endpoints/request" + "k8s.io/client-go/informers" + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/tools/cache" + "k8s.io/klog/v2" + + "github.com/openyurtio/openyurt/pkg/yurthub/util" +) + +type PoolScopeResourcesManger struct { + factory informers.SharedInformerFactory + validPoolScopedResources map[string]*verifiablePoolScopeResource + validPoolScopedResourcesLock sync.RWMutex + k8sClient kubernetes.Interface + hasSynced func() bool +} + +var poolScopeResourcesManger *PoolScopeResourcesManger + +func InitPoolScopeResourcesManger(client kubernetes.Interface, factory informers.SharedInformerFactory) *PoolScopeResourcesManger { + poolScopeResourcesManger = &PoolScopeResourcesManger{ + k8sClient: client, + validPoolScopedResources: make(map[string]*verifiablePoolScopeResource), + } + configmapInformer := factory.Core().V1().ConfigMaps().Informer() + configmapInformer.AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: poolScopeResourcesManger.addConfigmap, + // todo: now we do not support update of pool scope resources definition + }) + poolScopeResourcesManger.hasSynced = configmapInformer.HasSynced + + klog.Infof("init pool scope resources manager") + + poolScopeResourcesManger.setVerifiableGVRs(poolScopeResourcesManger.getInitPoolScopeResources()...) + return poolScopeResourcesManger +} + +func WaitUntilPoolScopeResourcesSync(ctx context.Context) { + cache.WaitForCacheSync(ctx.Done(), poolScopeResourcesManger.hasSynced) +} + +func IsPoolScopeResources(info *apirequest.RequestInfo) bool { + if info == nil || poolScopeResourcesManger == nil { + return false + } + _, ok := poolScopeResourcesManger.validPoolScopedResources[schema.GroupVersionResource{ + Group: info.APIGroup, + Version: info.APIVersion, + Resource: info.Resource, + }.String()] + return ok +} + +func GetPoolScopeResources() []schema.GroupVersionResource { + if poolScopeResourcesManger == nil { + return []schema.GroupVersionResource{} + } + return poolScopeResourcesManger.getPoolScopeResources() +} + +func (m *PoolScopeResourcesManger) getPoolScopeResources() []schema.GroupVersionResource { + poolScopeResources := make([]schema.GroupVersionResource, 0) + m.validPoolScopedResourcesLock.RLock() + defer m.validPoolScopedResourcesLock.RUnlock() + for _, v := range m.validPoolScopedResources { + poolScopeResources = append(poolScopeResources, v.GroupVersionResource) + } + return poolScopeResources +} + +// addVerifiableGVRs add given gvrs to validPoolScopedResources map +func (m *PoolScopeResourcesManger) addVerifiableGVRs(gvrs ...*verifiablePoolScopeResource) { + m.validPoolScopedResourcesLock.Lock() + defer m.validPoolScopedResourcesLock.Unlock() + for _, gvr := range gvrs { + if ok, errMsg := gvr.Verify(); ok { + m.validPoolScopedResources[gvr.String()] = gvr + klog.Infof("PoolScopeResourcesManger add gvr %s success", gvr.String()) + } else { + klog.Warningf("PoolScopeResourcesManger add gvr %s failed, because %s", gvr.String(), errMsg) + } + } +} + +// addVerifiableGVRs clear validPoolScopedResources and set given gvrs to validPoolScopedResources map +func (m *PoolScopeResourcesManger) setVerifiableGVRs(gvrs ...*verifiablePoolScopeResource) { + m.validPoolScopedResourcesLock.Lock() + defer m.validPoolScopedResourcesLock.Unlock() + m.validPoolScopedResources = make(map[string]*verifiablePoolScopeResource) + for _, gvr := range gvrs { + if ok, errMsg := gvr.Verify(); ok { + m.validPoolScopedResources[gvr.String()] = gvr + klog.Infof("PoolScopeResourcesManger update gvr %s success", gvr.String()) + } else { + klog.Warningf("PoolScopeResourcesManger update gvr %s failed, because %s", gvr.String(), errMsg) + } + } +} + +func (m *PoolScopeResourcesManger) addConfigmap(obj interface{}) { + cfg, ok := obj.(*corev1.ConfigMap) + if !ok { + return + } + + poolScopeResources := cfg.Data[util.PoolScopeResourcesKey] + poolScopeResourcesGVRs := make([]schema.GroupVersionResource, 0) + verifiablePoolScopeResourcesGVRs := make([]*verifiablePoolScopeResource, 0) + if err := json.Unmarshal([]byte(poolScopeResources), &poolScopeResourcesGVRs); err != nil { + klog.Errorf("PoolScopeResourcesManger unmarshal poolScopeResources %s failed with error = %s", + poolScopeResourcesGVRs, err.Error()) + return + } + klog.Infof("PoolScopeResourcesManger add configured pool scope resources %v", poolScopeResourcesGVRs) + for _, v := range poolScopeResourcesGVRs { + verifiablePoolScopeResourcesGVRs = append(verifiablePoolScopeResourcesGVRs, + newVerifiablePoolScopeResource(v, m.getGroupVersionVerifyFunction(m.k8sClient))) + } + m.addVerifiableGVRs(verifiablePoolScopeResourcesGVRs...) +} + +func (m *PoolScopeResourcesManger) getGroupVersionVerifyFunction(client kubernetes.Interface) func(gvr schema.GroupVersionResource) (bool, string) { + return func(gvr schema.GroupVersionResource) (bool, string) { + maxRetry := 3 + duration := time.Second * 5 + counter := 0 + var err error + for counter <= maxRetry { + if _, err = client.Discovery().ServerResourcesForGroupVersion(gvr.GroupVersion().String()); err == nil { + return true, "" // gvr found + } + if apierrors.IsNotFound(err) { + return false, err.Error() // gvr not found + } + // unexpected error, retry + counter++ + time.Sleep(duration) + } + return false, err.Error() + } +} + +func (m *PoolScopeResourcesManger) getInitPoolScopeResources() []*verifiablePoolScopeResource { + return []*verifiablePoolScopeResource{ + newVerifiablePoolScopeResource( + schema.GroupVersionResource{Group: "", Version: "v1", Resource: "endpoints"}, + m.getGroupVersionVerifyFunction(m.k8sClient)), + newVerifiablePoolScopeResource( + schema.GroupVersionResource{Group: "discovery.k8s.io", Version: "v1", Resource: "endpointslices"}, + m.getGroupVersionVerifyFunction(m.k8sClient)), + newVerifiablePoolScopeResource( + schema.GroupVersionResource{Group: "discovery.k8s.io", Version: "v1beta1", Resource: "endpointslices"}, + m.getGroupVersionVerifyFunction(m.k8sClient)), + } +} diff --git a/pkg/yurthub/poolcoordinator/resources/verifiable_pool_scope_resource.go b/pkg/yurthub/poolcoordinator/resources/verifiable_pool_scope_resource.go new file mode 100644 index 00000000000..18ca63b5099 --- /dev/null +++ b/pkg/yurthub/poolcoordinator/resources/verifiable_pool_scope_resource.go @@ -0,0 +1,20 @@ +package resources + +import "k8s.io/apimachinery/pkg/runtime/schema" + +type verifiablePoolScopeResource struct { + schema.GroupVersionResource + checkFunction func(gvr schema.GroupVersionResource) (bool, string) +} + +func newVerifiablePoolScopeResource(gvr schema.GroupVersionResource, + checkFunction func(gvr schema.GroupVersionResource) (bool, string)) *verifiablePoolScopeResource { + return &verifiablePoolScopeResource{ + GroupVersionResource: gvr, + checkFunction: checkFunction, + } +} + +func (v *verifiablePoolScopeResource) Verify() (bool, string) { + return v.checkFunction(v.GroupVersionResource) +} diff --git a/pkg/yurthub/proxy/pool/pool.go b/pkg/yurthub/proxy/pool/pool.go index 6080dc5da0e..869ed31a0b9 100644 --- a/pkg/yurthub/proxy/pool/pool.go +++ b/pkg/yurthub/proxy/pool/pool.go @@ -108,6 +108,7 @@ func (pp *PoolCoordinatorProxy) ServeHTTP(rw http.ResponseWriter, req *http.Requ util.Err(errors.NewBadRequest(fmt.Sprintf("pool-coordinator proxy cannot handle request(%s), cannot get requestInfo", hubutil.ReqString(req))), rw, req) return } + req.Header.Del("Authorization") // delete token with cloud apiServer RBAC and use yurthub authorization if reqInfo.IsResourceRequest { switch reqInfo.Verb { case "create": diff --git a/pkg/yurthub/proxy/util/util.go b/pkg/yurthub/proxy/util/util.go index 76632db1605..a86486620b0 100644 --- a/pkg/yurthub/proxy/util/util.go +++ b/pkg/yurthub/proxy/util/util.go @@ -40,6 +40,7 @@ import ( "github.com/openyurtio/openyurt/pkg/yurthub/kubernetes/serializer" "github.com/openyurtio/openyurt/pkg/yurthub/metrics" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/resources" "github.com/openyurtio/openyurt/pkg/yurthub/tenant" "github.com/openyurtio/openyurt/pkg/yurthub/util" ) @@ -181,7 +182,7 @@ func WithIfPoolScopedResource(handler http.Handler) http.Handler { ctx := req.Context() if info, ok := apirequest.RequestInfoFrom(ctx); ok { var ifPoolScopedResource bool - if info.IsResourceRequest && isPoolScopedResource(info) { + if info.IsResourceRequest && resources.IsPoolScopeResources(info) { ifPoolScopedResource = true } ctx = util.WithIfPoolScopedResource(ctx, ifPoolScopedResource) @@ -191,18 +192,6 @@ func WithIfPoolScopedResource(handler http.Handler) http.Handler { }) } -func isPoolScopedResource(info *apirequest.RequestInfo) bool { - if info != nil { - if info.APIGroup == "" && info.APIVersion == "v1" && info.Resource == "endpoints" { - return true - } - if info.APIGroup == "discovery.k8s.io" && info.APIVersion == "v1" && info.Resource == "endpointslices" { - return true - } - } - return false -} - type wrapperResponseWriter struct { http.ResponseWriter http.Flusher diff --git a/pkg/yurthub/storage/etcd/keycache.go b/pkg/yurthub/storage/etcd/keycache.go index df670ebca7e..57863948ead 100644 --- a/pkg/yurthub/storage/etcd/keycache.go +++ b/pkg/yurthub/storage/etcd/keycache.go @@ -26,6 +26,7 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" coordinatorconstants "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/resources" "github.com/openyurtio/openyurt/pkg/yurthub/storage" "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" ) @@ -122,7 +123,7 @@ func (c *componentKeyCache) Recover() error { func (c *componentKeyCache) getPoolScopedKeyset() (*keySet, error) { keys := &keySet{m: map[storageKey]struct{}{}} - for gvr := range coordinatorconstants.PoolScopedResources { + for _, gvr := range resources.GetPoolScopeResources() { getCtx, cancel := context.WithTimeout(c.ctx, defaultTimeout) defer cancel() rootKey, err := c.keyFunc(storage.KeyBuildInfo{ diff --git a/pkg/yurthub/storage/etcd/storage.go b/pkg/yurthub/storage/etcd/storage.go index 1e33a7de827..2aef0da789d 100644 --- a/pkg/yurthub/storage/etcd/storage.go +++ b/pkg/yurthub/storage/etcd/storage.go @@ -395,6 +395,7 @@ func (s *etcdStorage) ReplaceComponentList(component string, gvr schema.GroupVer oldKeyCache, loaded := s.localComponentKeyCache.LoadOrStore(component, newKeyCache) addedOrUpdated = newKeyCache.Difference(keySet{}) if loaded { + // FIXME: delete keys may cause unexpected problem deleted = oldKeyCache.Difference(newKeyCache) } diff --git a/pkg/yurthub/util/util.go b/pkg/yurthub/util/util.go index 56f2d8cc3e0..179e3916695 100644 --- a/pkg/yurthub/util/util.go +++ b/pkg/yurthub/util/util.go @@ -70,8 +70,9 @@ const ( // DefaultPoolCoordinatorAPIServerSvcPort represents default pool coordinator apiServer port DefaultPoolCoordinatorAPIServerSvcPort = "443" - YurtHubNamespace = "kube-system" - CacheUserAgentsKey = "cache_agents" + YurtHubNamespace = "kube-system" + CacheUserAgentsKey = "cache_agents" + PoolScopeResourcesKey = "pool_scope_resources" YurtHubProxyPort = 10261 YurtHubPort = 10267 From e84c4f68f7f4a3ce2630223988824038ea5dd552 Mon Sep 17 00:00:00 2001 From: Yifei Zhang Date: Wed, 18 Jan 2023 10:19:33 +0800 Subject: [PATCH 18/20] bugfix: ReplaceComponentList should not delete resources of other gvr (#1153) * bugfix: ReplaceComponentList should not delete resources of other gvr Signed-off-by: Congrool --- .../poolcoordinator/resources/resources.go | 16 + .../verifiable_pool_scope_resource.go | 16 + pkg/yurthub/storage/etcd/key.go | 6 + pkg/yurthub/storage/etcd/keycache.go | 250 +++++--- pkg/yurthub/storage/etcd/keycache_test.go | 588 +++++++++++++----- pkg/yurthub/storage/etcd/storage.go | 69 +- pkg/yurthub/storage/etcd/storage_test.go | 16 +- 7 files changed, 679 insertions(+), 282 deletions(-) diff --git a/pkg/yurthub/poolcoordinator/resources/resources.go b/pkg/yurthub/poolcoordinator/resources/resources.go index 6826a31ae0b..76f195fe149 100644 --- a/pkg/yurthub/poolcoordinator/resources/resources.go +++ b/pkg/yurthub/poolcoordinator/resources/resources.go @@ -1,3 +1,19 @@ +/* +Copyright 2023 The OpenYurt Authors. + +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 resources import ( diff --git a/pkg/yurthub/poolcoordinator/resources/verifiable_pool_scope_resource.go b/pkg/yurthub/poolcoordinator/resources/verifiable_pool_scope_resource.go index 18ca63b5099..e4c09259093 100644 --- a/pkg/yurthub/poolcoordinator/resources/verifiable_pool_scope_resource.go +++ b/pkg/yurthub/poolcoordinator/resources/verifiable_pool_scope_resource.go @@ -1,3 +1,19 @@ +/* +Copyright 2023 The OpenYurt Authors. + +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 resources import "k8s.io/apimachinery/pkg/runtime/schema" diff --git a/pkg/yurthub/storage/etcd/key.go b/pkg/yurthub/storage/etcd/key.go index 958ea8c7905..0192993eda2 100644 --- a/pkg/yurthub/storage/etcd/key.go +++ b/pkg/yurthub/storage/etcd/key.go @@ -42,6 +42,7 @@ var SpecialDefaultResourcePrefixes = map[schema.GroupResource]string{ type storageKey struct { comp string path string + gvr schema.GroupVersionResource } func (k storageKey) Key() string { @@ -74,5 +75,10 @@ func (s *etcdStorage) KeyFunc(info storage.KeyBuildInfo) (storage.Key, error) { return storageKey{ comp: info.Component, path: path, + gvr: schema.GroupVersionResource{ + Group: info.Group, + Version: info.Version, + Resource: info.Resources, + }, }, nil } diff --git a/pkg/yurthub/storage/etcd/keycache.go b/pkg/yurthub/storage/etcd/keycache.go index 57863948ead..43357ef7170 100644 --- a/pkg/yurthub/storage/etcd/keycache.go +++ b/pkg/yurthub/storage/etcd/keycache.go @@ -24,58 +24,54 @@ import ( "sync" clientv3 "go.etcd.io/etcd/client/v3" + "k8s.io/apimachinery/pkg/runtime/schema" coordinatorconstants "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" - "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/resources" "github.com/openyurtio/openyurt/pkg/yurthub/storage" "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" ) -// type state int - -// const ( -// done state = 0 -// processing state = 1 -// ) - -// type status struct - -type keySet struct { - m map[storageKey]struct{} -} +type storageKeySet map[storageKey]struct{} // Difference will return keys in s but not in s2 -func (s keySet) Difference(s2 keySet) []storageKey { - keys := []storageKey{} - if s2.m == nil { - for k := range s.m { - keys = append(keys, k) +func (s storageKeySet) Difference(s2 storageKeySet) storageKeySet { + keys := storageKeySet{} + if s2 == nil { + for k := range s { + keys[k] = struct{}{} } return keys } - for k := range s.m { - if _, ok := s2.m[k]; !ok { - keys = append(keys, k) + for k := range s { + if _, ok := s2[k]; !ok { + keys[k] = struct{}{} } } + return keys } +type keyCache struct { + m map[schema.GroupVersionResource]storageKeySet +} + // Do not directly modify value returned from functions of componentKeyCache, such as Load. // Because it usually returns reference of internal objects for efficiency. // The format in file is: -// component0:key0,key1... -// component1:key0,key1... +// component0#group.version.resource:key0,key1;group.version.resource:key2,key3... +// component1#group.version.resource:key4,key5... // ... type componentKeyCache struct { sync.Mutex - ctx context.Context - cache map[string]keySet - filePath string - keyFunc func(storage.KeyBuildInfo) (storage.Key, error) - fsOperator fs.FileSystemOperator - etcdClient *clientv3.Client + ctx context.Context + // map component to keyCache + cache map[string]keyCache + filePath string + keyFunc func(storage.KeyBuildInfo) (storage.Key, error) + fsOperator fs.FileSystemOperator + etcdClient *clientv3.Client + poolScopedResourcesGetter func() []schema.GroupVersionResource } func (c *componentKeyCache) Recover() error { @@ -91,22 +87,11 @@ func (c *componentKeyCache) Recover() error { if len(buf) != 0 { // We've got content from file - lines := strings.Split(string(buf), "\n") - for i, l := range lines { - s := strings.Split(l, ":") - if len(s) != 2 { - return fmt.Errorf("failed to parse line %d, invalid format", i) - } - comp, keys := s[0], strings.Split(s[1], ",") - ks := keySet{m: map[storageKey]struct{}{}} - for _, key := range keys { - ks.m[storageKey{ - comp: comp, - path: key, - }] = struct{}{} - } - c.cache[comp] = ks + cache, err := unmarshal(buf) + if err != nil { + return fmt.Errorf("failed to parse file content at %s, %v", c.filePath, err) } + c.cache = cache } poolScopedKeyset, err := c.getPoolScopedKeyset() @@ -121,9 +106,9 @@ func (c *componentKeyCache) Recover() error { return nil } -func (c *componentKeyCache) getPoolScopedKeyset() (*keySet, error) { - keys := &keySet{m: map[storageKey]struct{}{}} - for _, gvr := range resources.GetPoolScopeResources() { +func (c *componentKeyCache) getPoolScopedKeyset() (*keyCache, error) { + keys := &keyCache{m: make(map[schema.GroupVersionResource]storageKeySet)} + for _, gvr := range c.poolScopedResourcesGetter() { getCtx, cancel := context.WithTimeout(c.ctx, defaultTimeout) defer cancel() rootKey, err := c.keyFunc(storage.KeyBuildInfo{ @@ -156,61 +141,107 @@ func (c *componentKeyCache) getPoolScopedKeyset() (*keySet, error) { if err != nil { return nil, fmt.Errorf("failed to create resource key for %v", kv.Key) } - keys.m[key.(storageKey)] = struct{}{} + + if _, ok := keys.m[gvr]; !ok { + keys.m[gvr] = storageKeySet{key.(storageKey): {}} + } else { + keys.m[gvr][key.(storageKey)] = struct{}{} + } } } return keys, nil } -func (c *componentKeyCache) Load(component string) (keySet, bool) { +// Load returns keyCache of component which contains keys of all gvr. +func (c *componentKeyCache) Load(component string) (keyCache, bool) { c.Lock() defer c.Unlock() cache, ok := c.cache[component] return cache, ok } +// AddKey will add key to the key cache of such component. If the component +// does not have its cache, it will be created first. func (c *componentKeyCache) AddKey(component string, key storageKey) { c.Lock() defer c.Unlock() defer c.flush() if _, ok := c.cache[component]; !ok { - c.cache[component] = keySet{m: map[storageKey]struct{}{ - key: {}, + c.cache[component] = keyCache{m: map[schema.GroupVersionResource]storageKeySet{ + key.gvr: { + key: struct{}{}, + }, }} return } - keyset := c.cache[component] - if keyset.m == nil { - keyset.m = map[storageKey]struct{}{ - key: {}, + keyCache := c.cache[component] + if keyCache.m == nil { + keyCache.m = map[schema.GroupVersionResource]storageKeySet{ + key.gvr: { + key: struct{}{}, + }, } return } - c.cache[component].m[key] = struct{}{} + if _, ok := keyCache.m[key.gvr]; !ok { + keyCache.m[key.gvr] = storageKeySet{key: {}} + return + } + keyCache.m[key.gvr][key] = struct{}{} } +// DeleteKey deletes specified key from the key cache of the component. func (c *componentKeyCache) DeleteKey(component string, key storageKey) { c.Lock() defer c.Unlock() - delete(c.cache[component].m, key) + if _, ok := c.cache[component]; !ok { + return + } + if c.cache[component].m == nil { + return + } + if _, ok := c.cache[component].m[key.gvr]; !ok { + return + } + delete(c.cache[component].m[key.gvr], key) c.flush() } -func (c *componentKeyCache) LoadOrStore(component string, keyset keySet) (keySet, bool) { +// LoadOrStore will load the keyset of specified gvr from cache of the component if it exists, +// otherwise it will be created with passed-in keyset argument. It will return the key set +// finally in the component cache, and a bool value indicating whether the returned key set +// is loaded or stored. +func (c *componentKeyCache) LoadOrStore(component string, gvr schema.GroupVersionResource, keyset storageKeySet) (storageKeySet, bool) { c.Lock() defer c.Unlock() if cache, ok := c.cache[component]; ok { - return cache, true + if cache.m == nil { + cache.m = make(map[schema.GroupVersionResource]storageKeySet) + } + + if set, ok := cache.m[gvr]; ok { + return set, true + } else { + cache.m[gvr] = keyset + c.flush() + return keyset, false + } } else { - c.cache[component] = keyset + c.cache[component] = keyCache{ + m: map[schema.GroupVersionResource]storageKeySet{ + gvr: keyset, + }, + } c.flush() return keyset, false } } -func (c *componentKeyCache) LoadAndDelete(component string) (keySet, bool) { +// LoadAndDelete will load and delete the key cache of specified component. +// Return the original cache and true if it was deleted, otherwise empty cache and false. +func (c *componentKeyCache) LoadAndDelete(component string) (keyCache, bool) { c.Lock() defer c.Unlock() if cache, ok := c.cache[component]; ok { @@ -218,33 +249,33 @@ func (c *componentKeyCache) LoadAndDelete(component string) (keySet, bool) { c.flush() return cache, true } - return keySet{}, false + return keyCache{}, false } -func (c *componentKeyCache) DeleteAllKeysOfComponent(component string) { - c.Lock() - defer c.Unlock() - delete(c.cache, component) - c.flush() +func (c *componentKeyCache) flush() error { + buf := marshal(c.cache) + if err := c.fsOperator.Write(c.filePath, buf); err != nil { + return fmt.Errorf("failed to flush cache to file %s, %v", c.filePath, err) + } + return nil } -// func (c *componentKeyCache) MarkAsProcessing() { - -// } - -// func (c *componentKeyCache) MarkAsDone() { - -// } - -func (c *componentKeyCache) flush() error { +func marshal(cache map[string]keyCache) []byte { buf := bytes.NewBuffer(make([]byte, 0, 1024)) - for comp, ks := range c.cache { - line := bytes.NewBufferString(fmt.Sprintf("%s:", comp)) - keys := make([]string, 0, len(ks.m)) - for k := range ks.m { - keys = append(keys, k.Key()) + for comp, ks := range cache { + line := bytes.NewBufferString(fmt.Sprintf("%s#", comp)) + for gvr, s := range ks.m { + gvrStr := strings.Join([]string{gvr.Group, gvr.Version, gvr.Resource}, "_") + keys := make([]string, 0, len(s)) + for k := range s { + keys = append(keys, k.Key()) + } + line.WriteString(fmt.Sprintf("%s:%s;", gvrStr, strings.Join(keys, ","))) + } + if len(ks.m) != 0 { + // discard last ';' + line.Truncate(line.Len() - 1) } - line.WriteString(strings.Join(keys, ",")) line.WriteByte('\n') buf.Write(line.Bytes()) } @@ -252,10 +283,59 @@ func (c *componentKeyCache) flush() error { // discard last '\n' buf.Truncate(buf.Len() - 1) } - if err := c.fsOperator.Write(c.filePath, buf.Bytes()); err != nil { - return fmt.Errorf("failed to flush cache to file %s, %v", c.filePath, err) + return buf.Bytes() +} + +func unmarshal(buf []byte) (map[string]keyCache, error) { + cache := map[string]keyCache{} + if len(buf) == 0 { + return cache, nil } - return nil + + lines := strings.Split(string(buf), "\n") + for i, l := range lines { + s := strings.Split(l, "#") + if len(s) != 2 { + return nil, fmt.Errorf("failed to parse line %d, invalid format", i) + } + comp := s[0] + + keySet := keyCache{m: map[schema.GroupVersionResource]storageKeySet{}} + if len(s[1]) > 0 { + gvrKeys := strings.Split(s[1], ";") + for _, gvrKey := range gvrKeys { + ss := strings.Split(gvrKey, ":") + if len(ss) != 2 { + return nil, fmt.Errorf("failed to parse gvr keys %s at line %d, invalid format", gvrKey, i) + } + gvrStrs := strings.Split(ss[0], "_") + if len(gvrStrs) != 3 { + return nil, fmt.Errorf("failed to parse gvr %s at line %d, invalid format", ss[0], i) + } + gvr := schema.GroupVersionResource{ + Group: gvrStrs[0], + Version: gvrStrs[1], + Resource: gvrStrs[2], + } + + set := storageKeySet{} + if len(ss[1]) != 0 { + keys := strings.Split(ss[1], ",") + for _, k := range keys { + key := storageKey{ + comp: comp, + path: k, + gvr: gvr, + } + set[key] = struct{}{} + } + } + keySet.m[gvr] = set + } + } + cache[comp] = keySet + } + return cache, nil } // We assume that path points to a namespaced resource. diff --git a/pkg/yurthub/storage/etcd/keycache_test.go b/pkg/yurthub/storage/etcd/keycache_test.go index d24398699e3..6ee556eb372 100644 --- a/pkg/yurthub/storage/etcd/keycache_test.go +++ b/pkg/yurthub/storage/etcd/keycache_test.go @@ -30,12 +30,41 @@ import ( "github.com/stretchr/testify/mock" mvccpb "go.etcd.io/etcd/api/v3/mvccpb" clientv3 "go.etcd.io/etcd/client/v3" + "k8s.io/apimachinery/pkg/runtime/schema" coordinatorconstants "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/constants" etcdmock "github.com/openyurtio/openyurt/pkg/yurthub/storage/etcd/mock" "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" ) +var ( + podGVR = schema.GroupVersionResource{ + Group: "", + Version: "v1", + Resource: "pods", + } + endpointSliceGVR = schema.GroupVersionResource{ + Group: "discovery.k8s.io", + Version: "v1", + Resource: "endpointslices", + } + endpointGVR = schema.GroupVersionResource{ + Group: "", + Version: "v1", + Resource: "endpoints", + } + cmGVR = schema.GroupVersionResource{ + Group: "", + Version: "v1", + Resource: "configmaps", + } + svcGVR = schema.GroupVersionResource{ + Group: "", + Version: "v1", + Resource: "services", + } +) + var _ = Describe("Test componentKeyCache setup", func() { var cache *componentKeyCache var fileName string @@ -53,10 +82,15 @@ var _ = Describe("Test componentKeyCache setup", func() { cache = &componentKeyCache{ ctx: context.Background(), filePath: filepath.Join(keyCacheDir, fileName), - cache: map[string]keySet{}, + cache: map[string]keyCache{}, fsOperator: fs.FileSystemOperator{}, etcdClient: mockedClient, keyFunc: etcdStorage.KeyFunc, + poolScopedResourcesGetter: func() []schema.GroupVersionResource { + return []schema.GroupVersionResource{ + endpointGVR, endpointSliceGVR, + } + }, } }) AfterEach(func() { @@ -97,24 +131,32 @@ var _ = Describe("Test componentKeyCache setup", func() { It("should recover leader-yurthub cache from etcd", func() { Expect(cache.Recover()).To(BeNil()) Expect(cache.cache[coordinatorconstants.DefaultPoolScopedUserAgent]).Should(Equal( - keySet{ - m: map[storageKey]struct{}{ - { - comp: coordinatorconstants.DefaultPoolScopedUserAgent, - path: "/registry/services/endpoints/default/nginx", - }: {}, - { - comp: coordinatorconstants.DefaultPoolScopedUserAgent, - path: "/registry/services/endpoints/kube-system/kube-dns", - }: {}, - { - comp: coordinatorconstants.DefaultPoolScopedUserAgent, - path: "/registry/endpointslices/default/nginx", - }: {}, - { - comp: coordinatorconstants.DefaultPoolScopedUserAgent, - path: "/registry/endpointslices/kube-system/kube-dns", - }: {}, + keyCache{ + m: map[schema.GroupVersionResource]storageKeySet{ + endpointGVR: { + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + gvr: endpointGVR, + path: "/registry/services/endpoints/default/nginx", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + gvr: endpointGVR, + path: "/registry/services/endpoints/kube-system/kube-dns", + }: {}, + }, + endpointSliceGVR: { + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + gvr: endpointSliceGVR, + path: "/registry/endpointslices/default/nginx", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + gvr: endpointSliceGVR, + path: "/registry/endpointslices/kube-system/kube-dns", + }: {}, + }, }, }, )) @@ -122,31 +164,39 @@ var _ = Describe("Test componentKeyCache setup", func() { It("should replace leader-yurthub cache read from local file with keys from etcd", func() { Expect(f.CreateFile(filepath.Join(keyCacheDir, fileName), []byte( - "leader-yurthub:/registry/services/endpoints/default/nginx-local,"+ - "/registry/services/endpoints/kube-system/kube-dns-local,"+ - "/registry/endpointslices/default/nginx-local,"+ + "leader-yurthub#_v1_endpoints:/registry/services/endpoints/default/nginx-local,"+ + "/registry/services/endpoints/kube-system/kube-dns-local;"+ + "discovery.k8s.io_v1_endpointslices:/registry/endpointslices/default/nginx-local,"+ "/registry/endpointslices/kube-system/kube-dns-local", ))).To(BeNil()) Expect(cache.Recover()).To(BeNil()) Expect(cache.cache[coordinatorconstants.DefaultPoolScopedUserAgent]).Should(Equal( - keySet{ - m: map[storageKey]struct{}{ - { - comp: coordinatorconstants.DefaultPoolScopedUserAgent, - path: "/registry/services/endpoints/default/nginx", - }: {}, - { - comp: coordinatorconstants.DefaultPoolScopedUserAgent, - path: "/registry/services/endpoints/kube-system/kube-dns", - }: {}, - { - comp: coordinatorconstants.DefaultPoolScopedUserAgent, - path: "/registry/endpointslices/default/nginx", - }: {}, - { - comp: coordinatorconstants.DefaultPoolScopedUserAgent, - path: "/registry/endpointslices/kube-system/kube-dns", - }: {}, + keyCache{ + m: map[schema.GroupVersionResource]storageKeySet{ + endpointGVR: { + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + gvr: endpointGVR, + path: "/registry/services/endpoints/default/nginx", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + gvr: endpointGVR, + path: "/registry/services/endpoints/kube-system/kube-dns", + }: {}, + }, + endpointSliceGVR: { + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + gvr: endpointSliceGVR, + path: "/registry/endpointslices/default/nginx", + }: {}, + { + comp: coordinatorconstants.DefaultPoolScopedUserAgent, + gvr: endpointSliceGVR, + path: "/registry/endpointslices/kube-system/kube-dns", + }: {}, + }, }, }, )) @@ -155,33 +205,40 @@ var _ = Describe("Test componentKeyCache setup", func() { It("should recover when cache file exists and contains valid data", func() { Expect(f.CreateFile(filepath.Join(keyCacheDir, fileName), []byte( - "kubelet:/registry/pods/default/pod1,/registry/pods/default/pod2\n"+ - "kube-proxy:/registry/configmaps/kube-system/kube-proxy", + "kubelet#_v1_pods:/registry/pods/default/pod1,/registry/pods/default/pod2\n"+ + "kube-proxy#_v1_configmaps:/registry/configmaps/kube-system/kube-proxy", ))).To(BeNil()) Expect(cache.Recover()).To(BeNil()) - Expect(cache.cache).To(Equal(map[string]keySet{ + Expect(cache.cache).To(Equal(map[string]keyCache{ "kubelet": { - m: map[storageKey]struct{}{ - { - comp: "kubelet", - path: "/registry/pods/default/pod1", - }: {}, - { - comp: "kubelet", - path: "/registry/pods/default/pod2", - }: {}, + m: map[schema.GroupVersionResource]storageKeySet{ + podGVR: { + { + comp: "kubelet", + gvr: podGVR, + path: "/registry/pods/default/pod1", + }: {}, + { + comp: "kubelet", + gvr: podGVR, + path: "/registry/pods/default/pod2", + }: {}, + }, }, }, "kube-proxy": { - m: map[storageKey]struct{}{ - { - comp: "kube-proxy", - path: "/registry/configmaps/kube-system/kube-proxy", - }: {}, + m: map[schema.GroupVersionResource]storageKeySet{ + cmGVR: { + { + comp: "kube-proxy", + gvr: cmGVR, + path: "/registry/configmaps/kube-system/kube-proxy", + }: {}, + }, }, }, coordinatorconstants.DefaultPoolScopedUserAgent: { - m: map[storageKey]struct{}{}, + m: map[schema.GroupVersionResource]storageKeySet{}, }, })) }) @@ -210,10 +267,15 @@ var _ = Describe("Test componentKeyCache function", func() { cache = &componentKeyCache{ ctx: context.Background(), filePath: filepath.Join(keyCacheDir, fileName), - cache: map[string]keySet{}, + cache: map[string]keyCache{}, fsOperator: fs.FileSystemOperator{}, etcdClient: mockedClient, keyFunc: etcdStorage.KeyFunc, + poolScopedResourcesGetter: func() []schema.GroupVersionResource { + return []schema.GroupVersionResource{ + endpointGVR, endpointSliceGVR, + } + }, } key1 = storageKey{ path: "/registry/pods/default/pod1", @@ -232,11 +294,13 @@ var _ = Describe("Test componentKeyCache function", func() { Context("Test Load", func() { BeforeEach(func() { cache.Recover() - cache.cache = map[string]keySet{ + cache.cache = map[string]keyCache{ "kubelet": { - m: map[storageKey]struct{}{ - key1: {}, - key2: {}, + m: map[schema.GroupVersionResource]storageKeySet{ + podGVR: { + key1: {}, + key2: {}, + }, }, }, } @@ -249,9 +313,11 @@ var _ = Describe("Test componentKeyCache function", func() { }) It("should return keyset,true if component is in cache", func() { c, found := cache.Load("kubelet") - Expect(c.m).To(Equal(map[storageKey]struct{}{ - key1: {}, - key2: {}, + Expect(c.m).To(Equal(map[schema.GroupVersionResource]storageKeySet{ + podGVR: { + key1: {}, + key2: {}, + }, })) Expect(found).To(BeTrue()) }) @@ -260,16 +326,20 @@ var _ = Describe("Test componentKeyCache function", func() { Context("Test LoadAndDelete", func() { BeforeEach(func() { cache.Recover() - cache.cache = map[string]keySet{ + cache.cache = map[string]keyCache{ "kubelet": { - m: map[storageKey]struct{}{ - key1: {}, - key2: {}, + m: map[schema.GroupVersionResource]storageKeySet{ + podGVR: { + key1: {}, + key2: {}, + }, }, }, "kube-proxy": { - m: map[storageKey]struct{}{ - key3: {}, + m: map[schema.GroupVersionResource]storageKeySet{ + cmGVR: { + key3: {}, + }, }, }, } @@ -282,60 +352,56 @@ var _ = Describe("Test componentKeyCache function", func() { }) It("should return keyset,true and delete cache for this component if exists", func() { c, found := cache.LoadAndDelete("kubelet") - Expect(c.m).To(Equal(map[storageKey]struct{}{ - key1: {}, - key2: {}, + Expect(c.m).To(Equal(map[schema.GroupVersionResource]storageKeySet{ + podGVR: { + key1: {}, + key2: {}, + }, })) Expect(found).To(BeTrue()) - Expect(cache.cache).To(Equal(map[string]keySet{ + Expect(cache.cache).To(Equal(map[string]keyCache{ "kube-proxy": { - m: map[storageKey]struct{}{ - key3: {}, + m: map[schema.GroupVersionResource]storageKeySet{ + cmGVR: { + key3: {}, + }, }, }, })) data, err := os.ReadFile(cache.filePath) Expect(err).To(BeNil()) Expect(data).To(Equal([]byte( - "kube-proxy:" + key3.path, + "kube-proxy#_v1_configmaps:" + key3.path, ))) }) }) Context("Test LoadOrStore", func() { BeforeEach(func() { cache.Recover() - cache.cache = map[string]keySet{ + cache.cache = map[string]keyCache{ "kubelet": { - m: map[storageKey]struct{}{ - key1: {}, - key2: {}, + m: map[schema.GroupVersionResource]storageKeySet{ + podGVR: { + key1: {}, + key2: {}, + }, }, }, } cache.flush() }) It("should return data,false and store data if component currently does not in cache", func() { - c, found := cache.LoadOrStore("kube-proxy", keySet{ - m: map[storageKey]struct{}{ - key3: {}, - }, - }) + c, found := cache.LoadOrStore("kube-proxy", cmGVR, storageKeySet{key3: {}}) Expect(found).To(BeFalse()) - Expect(c.m).To(Equal(map[storageKey]struct{}{ - key3: {}, - })) + Expect(c).To(Equal(storageKeySet{key3: {}})) buf, err := os.ReadFile(cache.filePath) Expect(err).To(BeNil()) Expect(strings.Split(string(buf), "\n")).To(HaveLen(2)) }) It("should return original data and true if component already exists in cache", func() { - c, found := cache.LoadOrStore("kubelet", keySet{ - m: map[storageKey]struct{}{ - key3: {}, - }, - }) + c, found := cache.LoadOrStore("kubelet", podGVR, storageKeySet{key3: {}}) Expect(found).To(BeTrue()) - Expect(c.m).To(Equal(map[storageKey]struct{}{ + Expect(c).To(Equal(storageKeySet{ key1: {}, key2: {}, })) @@ -346,92 +412,302 @@ var _ = Describe("Test componentKeyCache function", func() { }) }) -func TestKeySetDifference(t *testing.T) { - key1 := storageKey{path: "/registry/pods/test/test-pod"} - key2 := storageKey{path: "/registry/pods/test/test-pod2"} - key3 := storageKey{path: "/registry/pods/test/test-pod3"} +func TestMarshal(t *testing.T) { cases := []struct { description string - s1 keySet - s2 keySet - want map[storageKey]struct{} + cache map[string]keyCache + want []byte }{ { - description: "s2 is nil", - s1: keySet{ - m: map[storageKey]struct{}{ - key1: {}, - key2: {}, + description: "cache is nil", + cache: map[string]keyCache{}, + want: []byte{}, + }, + { + description: "component has empty cache", + cache: map[string]keyCache{ + "kubelet": {m: map[schema.GroupVersionResource]storageKeySet{}}, + "kube-proxy": {m: map[schema.GroupVersionResource]storageKeySet{}}, + }, + }, + { + description: "empty gvr keySet", + cache: map[string]keyCache{ + "kubelet": { + m: map[schema.GroupVersionResource]storageKeySet{ + podGVR: {}, + }, }, }, - s2: keySet{ - m: nil, + }, + { + description: "marshal cache with keys", + cache: map[string]keyCache{ + "kubelet": { + m: map[schema.GroupVersionResource]storageKeySet{ + podGVR: { + { + comp: "kubelet", + gvr: podGVR, + path: "/registry/pods/default/nginx", + }: struct{}{}, + { + comp: "kubelet", + gvr: podGVR, + path: "/registry/pods/kube-system/kube-proxy", + }: struct{}{}, + }, + cmGVR: { + { + comp: "kubelet", + gvr: cmGVR, + path: "/registry/configmaps/kube-system/coredns", + }: struct{}{}, + }, + }, + }, + "kube-proxy": { + m: map[schema.GroupVersionResource]storageKeySet{ + endpointGVR: { + { + comp: "kube-proxy", + gvr: endpointGVR, + path: "/registry/services/endpoints/kube-system/kube-dns", + }: {}, + { + comp: "kube-proxy", + gvr: endpointGVR, + path: "/registry/services/endpoints/default/kubernetes", + }: {}, + }, + endpointSliceGVR: { + { + comp: "kube-proxy", + gvr: endpointSliceGVR, + path: "/registry/discovery.k8s.io/endpointslices/kube-system/kube-dns", + }: {}, + { + comp: "kube-proxy", + gvr: endpointSliceGVR, + path: "/registry/discovery.k8s.io/endpointslices/default/kubernetes", + }: {}, + }, + svcGVR: { + { + comp: "kube-proxy", + gvr: svcGVR, + path: "/registry/services/specs/kube-system/kube-dns", + }: {}, + }, + }, + }, }, - want: map[storageKey]struct{}{ - key1: {}, - key2: {}, + }, + } + + for _, c := range cases { + t.Run(c.description, func(t *testing.T) { + buf := marshal(c.cache) + if c.want != nil && !reflect.DeepEqual(buf, c.want) { + t.Errorf("unexpected result want: %s, got: %s", c.want, buf) + } + cache, err := unmarshal(buf) + if err != nil { + t.Errorf("unexpected error: %v", err) + } + + if !reflect.DeepEqual(cache, c.cache) { + t.Errorf("unexpected cache, want: %v, got: %v", c.cache, cache) + } + }) + } +} + +func TestUnmarshal(t *testing.T) { + cases := []struct { + description string + content string + want map[string]keyCache + wantErr bool + }{ + { + description: "empty content", + content: "", + want: map[string]keyCache{}, + }, + { + description: "components have empty keyCache", + content: "kubelet#\n" + + "kube-proxy#", + want: map[string]keyCache{ + "kubelet": {m: map[schema.GroupVersionResource]storageKeySet{}}, + "kube-proxy": {m: map[schema.GroupVersionResource]storageKeySet{}}, }, - }, { - description: "s2 is empty", - s1: keySet{ - m: map[storageKey]struct{}{ - key1: {}, - key2: {}, + }, + { + description: "invalid component format", + content: "kubelet\n" + + "kube-proxy", + wantErr: true, + }, + { + description: "gvr of component has empty keySet", + content: "kubelet#\n" + + "kube-proxy#_v1_endpoints:;discovery.k8s.io_v1_endpointslices:", + want: map[string]keyCache{ + "kubelet": {m: map[schema.GroupVersionResource]storageKeySet{}}, + "kube-proxy": { + m: map[schema.GroupVersionResource]storageKeySet{ + endpointGVR: {}, + endpointSliceGVR: {}, + }, }, }, - s2: keySet{ - m: map[storageKey]struct{}{}, + }, + { + description: "invalid gvr format that do not have suffix colon", + content: "kubelet#_v1_pods", + wantErr: true, + }, + { + description: "invalid gvr format that uses unrecognized separator", + content: "kubelet#.v1.pods", + wantErr: true, + }, + { + description: "unmarshal keys and generate cache", + content: "kubelet#_v1_pods:/registry/pods/default/nginx,/registry/pods/kube-system/kube-proxy\n" + + "kube-proxy#discovery.k8s.io_v1_endpointslices:/registry/endpointslices/kube-system/kube-dns;" + + "_v1_endpoints:/registry/services/endpoints/kube-system/kube-dns", + want: map[string]keyCache{ + "kubelet": { + m: map[schema.GroupVersionResource]storageKeySet{ + podGVR: { + { + comp: "kubelet", + gvr: podGVR, + path: "/registry/pods/default/nginx", + }: {}, + { + comp: "kubelet", + gvr: podGVR, + path: "/registry/pods/kube-system/kube-proxy", + }: {}, + }, + }, + }, + "kube-proxy": { + m: map[schema.GroupVersionResource]storageKeySet{ + endpointGVR: { + { + comp: "kube-proxy", + gvr: endpointGVR, + path: "/registry/services/endpoints/kube-system/kube-dns", + }: {}, + }, + endpointSliceGVR: { + { + comp: "kube-proxy", + gvr: endpointSliceGVR, + path: "/registry/endpointslices/kube-system/kube-dns", + }: {}, + }, + }, + }, }, - want: map[storageKey]struct{}{ - key1: {}, - key2: {}, + }, + } + + for _, c := range cases { + t.Run(c.description, func(t *testing.T) { + cache, err := unmarshal([]byte(c.content)) + if (c.wantErr && err == nil) || (!c.wantErr && err != nil) { + t.Errorf("unexpected err, if want error: %v, got: %v", c.wantErr, err) + } + + if err != nil { + return + } + + if !reflect.DeepEqual(cache, c.want) { + t.Errorf("unexpected cache, want: %v, got: %v", c.want, cache) + } + }) + } +} + +func TestStorageKeySetDifference(t *testing.T) { + podKey1 := storageKey{path: "/registry/pods/test/test-pod"} + podKey2 := storageKey{path: "/registry/pods/test/test-pod2"} + podKey3 := storageKey{path: "/registry/pods/test/test-pod3"} + cases := []struct { + description string + s1 storageKeySet + s2 storageKeySet + gvr schema.GroupVersionResource + want storageKeySet + }{ + { + description: "s2 is nil", + s1: storageKeySet{ + podKey1: {}, + podKey2: {}, + }, + s2: nil, + gvr: podGVR, + want: storageKeySet{ + podKey1: {}, + podKey2: {}, + }, + }, { + description: "s2 is empty", + s1: storageKeySet{ + podKey1: {}, + podKey2: {}, + }, + s2: storageKeySet{}, + gvr: podGVR, + want: storageKeySet{ + podKey1: {}, + podKey2: {}, }, }, { description: "s1 is empty", - s1: keySet{ - m: map[storageKey]struct{}{}, + s1: storageKeySet{}, + s2: storageKeySet{ + podKey1: {}, + podKey2: {}, }, - s2: keySet{ - m: map[storageKey]struct{}{ - key1: {}, - key2: {}, - }, - }, - want: map[storageKey]struct{}{}, + gvr: podGVR, + want: storageKeySet{}, }, { description: "s1 has intersection with s2", - s1: keySet{ - m: map[storageKey]struct{}{ - key1: {}, - key2: {}, - }, + s1: storageKeySet{ + podKey1: {}, + podKey2: {}, }, - s2: keySet{ - m: map[storageKey]struct{}{ - key2: {}, - key3: {}, - }, + s2: storageKeySet{ + podKey2: {}, + podKey3: {}, }, want: map[storageKey]struct{}{ - key1: {}, + podKey1: {}, }, }, } for _, c := range cases { - got := c.s1.Difference(c.s2) - if len(got) != len(c.want) { - t.Errorf("unexpected num of keys at case %s, got: %d, want: %d", c.description, len(got), len(c.want)) - } - gotm := map[storageKey]struct{}{} - for _, k := range got { - gotm[k] = struct{}{} - } + t.Run(c.description, func(t *testing.T) { + got := c.s1.Difference(c.s2) + if len(got) != len(c.want) { + t.Errorf("unexpected num of keys at case %s, got: %d, want: %d", c.description, len(got), len(c.want)) + } - if !reflect.DeepEqual(gotm, c.want) { - t.Errorf("failed at case %s, got: %v, want: %v", c.description, got, c.want) - } + if !reflect.DeepEqual(got, c.want) { + t.Errorf("failed at case %s, got: %v, want: %v", c.description, got, c.want) + } + }) } } diff --git a/pkg/yurthub/storage/etcd/storage.go b/pkg/yurthub/storage/etcd/storage.go index 2aef0da789d..b2728fe2221 100644 --- a/pkg/yurthub/storage/etcd/storage.go +++ b/pkg/yurthub/storage/etcd/storage.go @@ -32,6 +32,7 @@ import ( "k8s.io/client-go/kubernetes/scheme" "k8s.io/klog/v2" + "github.com/openyurtio/openyurt/pkg/yurthub/poolcoordinator/resources" "github.com/openyurtio/openyurt/pkg/yurthub/storage" "github.com/openyurtio/openyurt/pkg/yurthub/storage/utils" "github.com/openyurtio/openyurt/pkg/yurthub/util/fs" @@ -131,12 +132,13 @@ func NewStorage(ctx context.Context, cfg *EtcdStorageConfig) (storage.Store, err } cache := &componentKeyCache{ - ctx: ctx, - filePath: cacheFilePath, - cache: map[string]keySet{}, - fsOperator: fs.FileSystemOperator{}, - keyFunc: s.KeyFunc, - etcdClient: client, + ctx: ctx, + filePath: cacheFilePath, + cache: map[string]keyCache{}, + fsOperator: fs.FileSystemOperator{}, + keyFunc: s.KeyFunc, + etcdClient: client, + poolScopedResourcesGetter: resources.GetPoolScopeResources, } if err := cache.Recover(); err != nil { return nil, fmt.Errorf("failed to recover component key cache from %s, %v", cacheFilePath, err) @@ -339,15 +341,8 @@ func (s *etcdStorage) ListResourceKeysOfComponent(component string, gvr schema.G if component == "" { return nil, storage.ErrEmptyComponent } - - rootKey, err := s.KeyFunc(storage.KeyBuildInfo{ - Component: component, - Resources: gvr.Resource, - Group: gvr.Group, - Version: gvr.Version, - }) - if err != nil { - return nil, err + if gvr.Resource == "" { + return nil, storage.ErrEmptyResource } keys := []storage.Key{} @@ -355,8 +350,8 @@ func (s *etcdStorage) ListResourceKeysOfComponent(component string, gvr schema.G if !ok { return nil, storage.ErrStorageNotFound } - for k := range keyCache.m { - if strings.HasPrefix(k.Key(), rootKey.Key()) { + if keyCache.m != nil { + for k := range keyCache.m[gvr] { keys = append(keys, k) } } @@ -377,30 +372,28 @@ func (s *etcdStorage) ReplaceComponentList(component string, gvr schema.GroupVer if err != nil { return err } - for key := range contents { - if !strings.HasPrefix(key.Key(), rootKey.Key()) { - return storage.ErrInvalidContent - } - } - newKeyCache := keySet{m: map[storageKey]struct{}{}} + newKeySet := storageKeySet{} for k := range contents { storageKey, ok := k.(storageKey) if !ok { return storage.ErrUnrecognizedKey } - newKeyCache.m[storageKey] = struct{}{} + if !strings.HasPrefix(k.Key(), rootKey.Key()) { + return storage.ErrInvalidContent + } + newKeySet[storageKey] = struct{}{} } - var addedOrUpdated, deleted []storageKey - oldKeyCache, loaded := s.localComponentKeyCache.LoadOrStore(component, newKeyCache) - addedOrUpdated = newKeyCache.Difference(keySet{}) + + var addedOrUpdated, deleted storageKeySet + oldKeySet, loaded := s.localComponentKeyCache.LoadOrStore(component, gvr, newKeySet) + addedOrUpdated = newKeySet.Difference(storageKeySet{}) if loaded { - // FIXME: delete keys may cause unexpected problem - deleted = oldKeyCache.Difference(newKeyCache) + deleted = oldKeySet.Difference(newKeySet) } ops := []clientv3.Op{} - for _, k := range addedOrUpdated { + for k := range addedOrUpdated { rv, err := getRvOfObject(contents[k]) if err != nil { klog.Errorf("failed to process %s in list object, %v", k.Key(), err) @@ -433,7 +426,7 @@ func (s *etcdStorage) ReplaceComponentList(component string, gvr schema.GroupVer ) ops = append(ops, createOrUpdateOp) } - for _, k := range deleted { + for k := range deleted { ops = append(ops, clientv3.OpDelete(k.Key()), clientv3.OpDelete(s.mirrorPath(k.Key(), rvType)), @@ -455,17 +448,19 @@ func (s *etcdStorage) DeleteComponentResources(component string) error { return storage.ErrEmptyComponent } keyCache, loaded := s.localComponentKeyCache.LoadAndDelete(component) - if !loaded { + if !loaded || keyCache.m == nil { // no need to delete return nil } ops := []clientv3.Op{} - for k := range keyCache.m { - ops = append(ops, - clientv3.OpDelete(k.Key()), - clientv3.OpDelete(s.mirrorPath(k.Key(), rvType)), - ) + for _, keySet := range keyCache.m { + for k := range keySet { + ops = append(ops, + clientv3.OpDelete(k.Key()), + clientv3.OpDelete(s.mirrorPath(k.Key(), rvType)), + ) + } } ctx, cancel := context.WithTimeout(s.ctx, defaultTimeout) diff --git a/pkg/yurthub/storage/etcd/storage_test.go b/pkg/yurthub/storage/etcd/storage_test.go index 20e1ccbd79b..2ac1faf0a45 100644 --- a/pkg/yurthub/storage/etcd/storage_test.go +++ b/pkg/yurthub/storage/etcd/storage_test.go @@ -490,6 +490,7 @@ var _ = Describe("Test EtcdStorage", func() { Expect(buf).To(Equal(pod4Json)) }) It("should delete resources in etcd if they were in local cache but are not in current contents", func() { + Expect(etcdstore.Create(cmKey, cmJson)).Should(BeNil()) Expect(etcdstore.ReplaceComponentList("kubelet", gvr, "", map[storage.Key][]byte{ key1: podJson, })).To(BeNil()) @@ -500,6 +501,11 @@ var _ = Describe("Test EtcdStorage", func() { Expect(err).To(Equal(storage.ErrStorageNotFound)) _, err = etcdstore.Get(key3) Expect(err).To(Equal(storage.ErrStorageNotFound)) + + // Should not delete resources of other gvr + buf, err = etcdstore.Get(cmKey) + Expect(err).To(BeNil()) + Expect(buf).To(Equal(cmJson)) }) }) @@ -539,11 +545,13 @@ var _ = Describe("Test EtcdStorage", func() { _, found := etcdstore.localComponentKeyCache.Load("kubelet") Expect(found).To(BeFalse()) - keyset, found := etcdstore.localComponentKeyCache.Load("kube-proxy") + cache, found := etcdstore.localComponentKeyCache.Load("kube-proxy") Expect(found).To(BeTrue()) - Expect(keyset).To(Equal(keySet{ - m: map[storageKey]struct{}{ - cmKey.(storageKey): {}, + Expect(cache).To(Equal(keyCache{ + m: map[schema.GroupVersionResource]storageKeySet{ + cmGVR: { + cmKey.(storageKey): {}, + }, }, })) }) From 9bc4d1c75a0e3d6a64c75bda038a0a85be2387f6 Mon Sep 17 00:00:00 2001 From: Congrool Date: Tue, 17 Jan 2023 11:00:36 +0800 Subject: [PATCH 19/20] fix go.mod Signed-off-by: Congrool --- go.mod | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/go.mod b/go.mod index c909697fa7b..2f6541a7993 100644 --- a/go.mod +++ b/go.mod @@ -26,11 +26,10 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/vishvananda/netlink v1.1.1-0.20200603190939-5a869a71f0cb - github.com/wI2L/jsondiff v0.3.0 go.etcd.io/etcd/api/v3 v3.5.0 go.etcd.io/etcd/client/pkg/v3 v3.5.0 go.etcd.io/etcd/client/v3 v3.5.0 - golang.org/x/sys v0.0.0-20220319134239-a9b59b0215f8 + golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a google.golang.org/grpc v1.40.0 gopkg.in/cheggaaa/pb.v1 v1.0.25 gopkg.in/square/go-jose.v2 v2.2.2 From 634409dfb0f085dd9afa3846b28f41d1809982d8 Mon Sep 17 00:00:00 2001 From: Congrool Date: Tue, 17 Jan 2023 13:46:49 +0800 Subject: [PATCH 20/20] fix unit test of yurthub options Signed-off-by: Congrool --- cmd/yurthub/app/options/options_test.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/cmd/yurthub/app/options/options_test.go b/cmd/yurthub/app/options/options_test.go index 5a005a2467c..2e1dad3f888 100644 --- a/cmd/yurthub/app/options/options_test.go +++ b/cmd/yurthub/app/options/options_test.go @@ -24,6 +24,9 @@ import ( "time" "github.com/spf13/cobra" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/client-go/tools/leaderelection/resourcelock" + componentbaseconfig "k8s.io/component-base/config" "github.com/openyurtio/openyurt/pkg/projectinfo" "github.com/openyurtio/openyurt/pkg/yurthub/storage/disk" @@ -60,6 +63,18 @@ func TestNewYurtHubOptions(t *testing.T) { MinRequestTimeout: time.Second * 1800, CACertHashes: make([]string, 0), UnsafeSkipCAVerification: true, + CoordinatorServerAddr: fmt.Sprintf("https://%s:%s", util.DefaultPoolCoordinatorAPIServerSvcName, util.DefaultPoolCoordinatorAPIServerSvcPort), + CoordinatorStorageAddr: fmt.Sprintf("https://%s:%s", util.DefaultPoolCoordinatorEtcdSvcName, util.DefaultPoolCoordinatorEtcdSvcPort), + CoordinatorStoragePrefix: "/registry", + LeaderElection: componentbaseconfig.LeaderElectionConfiguration{ + LeaderElect: true, + LeaseDuration: metav1.Duration{Duration: 15 * time.Second}, + RenewDeadline: metav1.Duration{Duration: 10 * time.Second}, + RetryPeriod: metav1.Duration{Duration: 2 * time.Second}, + ResourceLock: resourcelock.LeasesResourceLock, + ResourceName: projectinfo.GetHubName(), + ResourceNamespace: "kube-system", + }, } options := NewYurtHubOptions()