From 05180971a590f86fd438fd329062a25ef5ddde4d Mon Sep 17 00:00:00 2001 From: Tim Usner Date: Thu, 27 May 2021 17:39:57 +0200 Subject: [PATCH 1/7] Add etcd status checks --- .ci/test | 2 +- controllers/config/custodian.go | 22 ++ ...uite_test.go => controllers_suite_test.go} | 11 +- controllers/etcd_custodian_controller.go | 41 ++- main.go | 9 +- pkg/health/condition/builder.go | 117 ++++++++ pkg/health/condition/builder_test.go | 172 ++++++++++++ pkg/health/condition/check_all_members.go | 54 ++++ .../condition/check_all_members_test.go | 86 ++++++ pkg/health/condition/check_ready.go | 62 +++++ pkg/health/condition/check_ready_test.go | 105 +++++++ pkg/health/condition/condition_suite_test.go | 52 ++++ pkg/health/condition/types.go | 52 ++++ pkg/health/etcdmember/builder.go | 112 ++++++++ pkg/health/etcdmember/builder_test.go | 137 +++++++++ pkg/health/etcdmember/check_ready.go | 56 ++++ pkg/health/etcdmember/check_ready_test.go | 118 ++++++++ .../etcdmember/etcdmember_suite_test.go | 47 ++++ pkg/health/etcdmember/types.go | 46 +++ pkg/health/status/check.go | 145 ++++++++++ pkg/health/status/check_test.go | 263 ++++++++++++++++++ pkg/health/status/status_suite_test.go | 27 ++ 22 files changed, 1705 insertions(+), 31 deletions(-) create mode 100644 controllers/config/custodian.go rename controllers/{suite_test.go => controllers_suite_test.go} (96%) create mode 100644 pkg/health/condition/builder.go create mode 100644 pkg/health/condition/builder_test.go create mode 100644 pkg/health/condition/check_all_members.go create mode 100644 pkg/health/condition/check_all_members_test.go create mode 100644 pkg/health/condition/check_ready.go create mode 100644 pkg/health/condition/check_ready_test.go create mode 100644 pkg/health/condition/condition_suite_test.go create mode 100644 pkg/health/condition/types.go create mode 100644 pkg/health/etcdmember/builder.go create mode 100644 pkg/health/etcdmember/builder_test.go create mode 100644 pkg/health/etcdmember/check_ready.go create mode 100644 pkg/health/etcdmember/check_ready_test.go create mode 100644 pkg/health/etcdmember/etcdmember_suite_test.go create mode 100644 pkg/health/etcdmember/types.go create mode 100644 pkg/health/status/check.go create mode 100644 pkg/health/status/check_test.go create mode 100644 pkg/health/status/status_suite_test.go diff --git a/.ci/test b/.ci/test index ee3fdbd69..8bcda5ca7 100755 --- a/.ci/test +++ b/.ci/test @@ -159,7 +159,7 @@ function test_with_coverage() { fetch_envtest_tools "$kb_root_dir" setup_envtest_env "$kb_root_dir" -TEST_PACKAGES="api api_tests controllers" +TEST_PACKAGES="api api_tests controllers pkg" GINKGO_COMMON_FLAGS="-r -timeout=1h0m0s --randomizeAllSpecs --randomizeSuites --failOnPending --progress" if [ -z $COVER ] || [ "$COVER" = false ] ; then echo "[INFO] Test coverage is disabled." diff --git a/controllers/config/custodian.go b/controllers/config/custodian.go new file mode 100644 index 000000000..278c55249 --- /dev/null +++ b/controllers/config/custodian.go @@ -0,0 +1,22 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 config + +import "time" + +// EtcdCustodianController contains configuration for the etcd custodian controller. +type EtcdCustodianController struct { + EtcdStaleMemberThreshold time.Duration +} diff --git a/controllers/suite_test.go b/controllers/controllers_suite_test.go similarity index 96% rename from controllers/suite_test.go rename to controllers/controllers_suite_test.go index 8610cb0a1..f94f86f81 100644 --- a/controllers/suite_test.go +++ b/controllers/controllers_suite_test.go @@ -21,13 +21,12 @@ import ( "testing" "time" - "github.com/gardener/gardener/pkg/utils/test" + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + "github.com/gardener/etcd-druid/controllers/config" + "github.com/gardener/gardener/pkg/utils/test" . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" - - druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" - appsv1 "k8s.io/api/apps/v1" corev1 "k8s.io/api/core/v1" "k8s.io/client-go/kubernetes/scheme" @@ -111,7 +110,9 @@ var _ = BeforeSuite(func(done Done) { err = er.SetupWithManager(mgr, 1, true) Expect(err).NotTo(HaveOccurred()) - custodian := NewEtcdCustodian(mgr) + custodian := NewEtcdCustodian(mgr, config.EtcdCustodianController{ + EtcdStaleMemberThreshold: 1 * time.Minute, + }) err = custodian.SetupWithManager(mgrCtx, mgr, 1) Expect(err).NotTo(HaveOccurred()) diff --git a/controllers/etcd_custodian_controller.go b/controllers/etcd_custodian_controller.go index 5eef7f051..ef53b2b02 100644 --- a/controllers/etcd_custodian_controller.go +++ b/controllers/etcd_custodian_controller.go @@ -26,7 +26,6 @@ import ( "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/runtime" - "k8s.io/apimachinery/pkg/types" "k8s.io/client-go/util/retry" "k8s.io/utils/pointer" ctrl "sigs.k8s.io/controller-runtime" @@ -38,6 +37,8 @@ import ( "sigs.k8s.io/controller-runtime/pkg/source" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + "github.com/gardener/etcd-druid/controllers/config" + "github.com/gardener/etcd-druid/pkg/health/status" druidmapper "github.com/gardener/etcd-druid/pkg/mapper" druidpredicates "github.com/gardener/etcd-druid/pkg/predicate" ) @@ -47,14 +48,16 @@ type EtcdCustodian struct { client.Client Scheme *runtime.Scheme logger logr.Logger + config config.EtcdCustodianController } // NewEtcdCustodian creates a new EtcdCustodian object -func NewEtcdCustodian(mgr manager.Manager) *EtcdCustodian { +func NewEtcdCustodian(mgr manager.Manager, config config.EtcdCustodianController) *EtcdCustodian { return &EtcdCustodian{ Client: mgr.GetClient(), Scheme: mgr.GetScheme(), logger: log.Log.WithName("custodian-controller"), + config: config, } } @@ -85,31 +88,19 @@ func (ec *EtcdCustodian) Reconcile(ctx context.Context, req ctrl.Request) (ctrl. }, nil } - // If any adoptions are attempted, we should first recheck for deletion with - // an uncached quorum read some time after listing Machines (see #42639). - canAdoptFunc := RecheckDeletionTimestamp(func() (metav1.Object, error) { - foundEtcd := &druidv1alpha1.Etcd{} - err := ec.Get(ctx, types.NamespacedName{Name: etcd.Name, Namespace: etcd.Namespace}, foundEtcd) - if err != nil { - return nil, err - } - - if foundEtcd.GetDeletionTimestamp() != nil { - return nil, fmt.Errorf("%v/%v etcd is marked for deletion", etcd.Namespace, etcd.Name) - } - if foundEtcd.UID != etcd.UID { - return nil, fmt.Errorf("original %v/%v etcd gone: got uid %v, wanted %v", etcd.Namespace, etcd.Name, foundEtcd.UID, etcd.UID) - } - return foundEtcd, nil - }) - selector, err := metav1.LabelSelectorAsSelector(etcd.Spec.Selector) if err != nil { logger.Error(err, "Error converting etcd selector to selector") return ctrl.Result{}, err } - refMgr := NewEtcdDruidRefManager(ec.Client, ec.Scheme, etcd, selector, etcdGVK, canAdoptFunc) + statusCheck := status.NewChecker(ec.config) + if err := statusCheck.Check(ctx, &etcd.Status); err != nil { + logger.Error(err, "Error executing status checks") + return ctrl.Result{}, err + } + + refMgr := NewEtcdDruidRefManager(ec.Client, ec.Scheme, etcd, selector, etcdGVK, nil) stsList, err := refMgr.FetchStatefulSet(ctx, etcd) if err != nil { @@ -134,6 +125,7 @@ func (ec *EtcdCustodian) Reconcile(ctx context.Context, req ctrl.Request) (ctrl. func (ec *EtcdCustodian) updateEtcdStatus(ctx context.Context, logger logr.Logger, etcd *druidv1alpha1.Etcd, sts *appsv1.StatefulSet) error { logger.Info("Updating etcd status with statefulset information") + conditions := etcd.Status.Conditions return kutil.TryUpdateStatus(ctx, retry.DefaultBackoff, ec.Client, etcd, func() error { etcd.Status.Etcd = &druidv1alpha1.CrossVersionObjectReference{ @@ -141,6 +133,9 @@ func (ec *EtcdCustodian) updateEtcdStatus(ctx context.Context, logger logr.Logge Kind: sts.Kind, Name: sts.Name, } + + etcd.Status.Conditions = conditions + ready := CheckStatefulSet(etcd, sts) == nil // To be changed once we have multiple replicas. @@ -155,12 +150,10 @@ func (ec *EtcdCustodian) updateEtcdStatus(ctx context.Context, logger logr.Logge func (ec *EtcdCustodian) updateEtcdStatusWithNoSts(ctx context.Context, logger logr.Logger, etcd *druidv1alpha1.Etcd) { logger.Info("Updating etcd status when no statefulset found") + conditions := etcd.Status.Conditions if err := kutil.TryUpdateStatus(ctx, retry.DefaultBackoff, ec.Client, etcd, func() error { - // TODO: (timuthy) Don't reset all conditions as some of them will be maintained by other actors (e.g. etcd-backup-restore) - conditions := []druidv1alpha1.Condition{} etcd.Status.Conditions = conditions - // To be changed once we have multiple replicas. etcd.Status.CurrentReplicas = 0 etcd.Status.ReadyReplicas = 0 diff --git a/main.go b/main.go index 9abc68229..6da4dff51 100644 --- a/main.go +++ b/main.go @@ -18,9 +18,11 @@ package main import ( "flag" "os" + "time" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" "github.com/gardener/etcd-druid/controllers" + "github.com/gardener/etcd-druid/controllers/config" "k8s.io/apimachinery/pkg/runtime" utilruntime "k8s.io/apimachinery/pkg/util/runtime" @@ -54,6 +56,8 @@ func main() { custodianWorkers int ignoreOperationAnnotation bool + etcdStaleMemberThreshold time.Duration + // TODO: migrate default to `leases` in one of the next releases defaultLeaderElectionResourceLock = resourcelock.ConfigMapsLeasesResourceLock defaultLeaderElectionID = "druid-leader-election" @@ -69,6 +73,7 @@ func main() { flag.StringVar(&leaderElectionResourceLock, "leader-election-resource-lock", defaultLeaderElectionResourceLock, "Which resource type to use for leader election. "+ "Supported options are 'endpoints', 'configmaps', 'leases', 'endpointsleases' and 'configmapsleases'.") flag.BoolVar(&ignoreOperationAnnotation, "ignore-operation-annotation", true, "Ignore the operation annotation or not.") + flag.DurationVar(&etcdStaleMemberThreshold, "etcd-member-threshold", 1*time.Minute, "Threshold after which an etcd member status is considered unknown if no heartbeat happened.") flag.Parse() @@ -100,7 +105,9 @@ func main() { os.Exit(1) } - custodian := controllers.NewEtcdCustodian(mgr) + custodian := controllers.NewEtcdCustodian(mgr, config.EtcdCustodianController{ + EtcdStaleMemberThreshold: etcdStaleMemberThreshold, + }) if err := custodian.SetupWithManager(ctx, mgr, custodianWorkers); err != nil { setupLog.Error(err, "Unable to create controller", "Controller", "Etcd Custodian") diff --git a/pkg/health/condition/builder.go b/pkg/health/condition/builder.go new file mode 100644 index 000000000..8532f8590 --- /dev/null +++ b/pkg/health/condition/builder.go @@ -0,0 +1,117 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 condition + +import ( + "time" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +// Builder is an interface for building conditions. +type Builder interface { + WithOldConditions(conditions []druidv1alpha1.Condition) Builder + WithResults(result []Result) Builder + WithNowFunc(now func() metav1.Time) Builder + Build() []druidv1alpha1.Condition +} + +type defaultBuilder struct { + old map[druidv1alpha1.ConditionType]druidv1alpha1.Condition + results map[druidv1alpha1.ConditionType]Result + nowFunc func() metav1.Time +} + +// NewBuilder returns a Builder for a specific condition. +func NewBuilder() Builder { + return &defaultBuilder{ + old: make(map[druidv1alpha1.ConditionType]druidv1alpha1.Condition), + results: make(map[druidv1alpha1.ConditionType]Result), + nowFunc: func() metav1.Time { + return metav1.NewTime(time.Now().UTC()) + }, + } +} + +// WithOldConditions sets the old conditions. It can be used to provide default values. +func (b *defaultBuilder) WithOldConditions(conditions []druidv1alpha1.Condition) Builder { + for _, cond := range conditions { + b.old[cond.Type] = cond + } + + return b +} + +// WithResults adds the results. +func (b *defaultBuilder) WithResults(results []Result) Builder { + for _, result := range results { + if result == nil { + continue + } + b.results[result.ConditionType()] = result + } + + return b +} + +// WithNowFunc sets the function used for getting the current time. +// Should only be used for tests. +func (b *defaultBuilder) WithNowFunc(now func() metav1.Time) Builder { + b.nowFunc = now + return b +} + +// Build creates the conditions. +// It merges the existing conditions with the results added to the builder. +// If OldCondition is provided: +// - Any changes to status set the `LastTransitionTime` +// - `LastUpdateTime` is always set. +func (b *defaultBuilder) Build() []druidv1alpha1.Condition { + var ( + now = b.nowFunc() + + conditions []druidv1alpha1.Condition + ) + + for condType, res := range b.results { + condition, ok := b.old[condType] + if !ok { + condition = druidv1alpha1.Condition{ + Type: condType, + LastTransitionTime: now, + } + } + + if condition.Status != res.Status() { + condition.LastTransitionTime = now + } + condition.LastUpdateTime = now + condition.Status = res.Status() + condition.Message = res.Message() + condition.Reason = res.Reason() + + conditions = append(conditions, condition) + delete(b.old, condType) + } + + for _, condition := range b.old { + // Add existing conditions as they were. This needs to be changed when SSA is used. + conditions = append(conditions, condition) + } + + return conditions +} diff --git a/pkg/health/condition/builder_test.go b/pkg/health/condition/builder_test.go new file mode 100644 index 000000000..5918b2720 --- /dev/null +++ b/pkg/health/condition/builder_test.go @@ -0,0 +1,172 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 condition_test + +import ( + "time" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + . "github.com/onsi/gomega/gstruct" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + . "github.com/gardener/etcd-druid/pkg/health/condition" +) + +var _ = Describe("Builder", func() { + var ( + builder Builder + now time.Time + ) + + BeforeEach(func() { + now, _ = time.Parse(time.RFC3339, "2021-06-01") + builder = NewBuilder() + }) + + JustBeforeEach(func() { + builder.WithNowFunc(func() metav1.Time { + return metav1.NewTime(now) + }) + }) + + Describe("#Build", func() { + Context("when Builder has old conditions", func() { + var ( + oldConditionTime time.Time + oldConditions []druidv1alpha1.Condition + ) + BeforeEach(func() { + oldConditionTime = now.Add(-12 * time.Hour) + + oldConditions = []druidv1alpha1.Condition{ + { + Type: druidv1alpha1.ConditionTypeAllMembersReady, + LastUpdateTime: metav1.NewTime(oldConditionTime), + LastTransitionTime: metav1.NewTime(oldConditionTime), + Status: druidv1alpha1.ConditionTrue, + Reason: "foo reason", + Message: "foo message", + }, + { + Type: druidv1alpha1.ConditionTypeReady, + LastUpdateTime: metav1.NewTime(oldConditionTime), + LastTransitionTime: metav1.NewTime(oldConditionTime), + Status: druidv1alpha1.ConditionFalse, + Reason: "bar reason", + Message: "bar message", + }, + { + Type: druidv1alpha1.ConditionTypeBackupReady, + LastUpdateTime: metav1.NewTime(oldConditionTime), + LastTransitionTime: metav1.NewTime(oldConditionTime), + Status: druidv1alpha1.ConditionTrue, + Reason: "foobar reason", + Message: "foobar message", + }, + } + + builder.WithOldConditions(oldConditions) + }) + + It("should correctly merge them", func() { + builder.WithResults([]Result{ + &result{ + ConType: druidv1alpha1.ConditionTypeAllMembersReady, + ConStatus: druidv1alpha1.ConditionTrue, + ConReason: "new reason", + ConMessage: "new message", + }, + &result{ + ConType: druidv1alpha1.ConditionTypeReady, + ConStatus: druidv1alpha1.ConditionTrue, + ConReason: "new reason", + ConMessage: "new message", + }, + }) + + conditions := builder.Build() + + Expect(conditions).To(ConsistOf( + MatchFields(IgnoreExtras, Fields{ + "Type": Equal(druidv1alpha1.ConditionTypeAllMembersReady), + "LastUpdateTime": Equal(metav1.NewTime(now)), + "LastTransitionTime": Equal(metav1.NewTime(oldConditionTime)), + "Status": Equal(druidv1alpha1.ConditionTrue), + "Reason": Equal("new reason"), + "Message": Equal("new message"), + }), + MatchFields(IgnoreExtras, Fields{ + "Type": Equal(druidv1alpha1.ConditionTypeReady), + "LastUpdateTime": Equal(metav1.NewTime(now)), + "LastTransitionTime": Equal(metav1.NewTime(now)), + "Status": Equal(druidv1alpha1.ConditionTrue), + "Reason": Equal("new reason"), + "Message": Equal("new message"), + }), + MatchFields(IgnoreExtras, Fields{ + "Type": Equal(druidv1alpha1.ConditionTypeBackupReady), + "LastUpdateTime": Equal(metav1.NewTime(oldConditionTime)), + "LastTransitionTime": Equal(metav1.NewTime(oldConditionTime)), + "Status": Equal(druidv1alpha1.ConditionTrue), + "Reason": Equal("foobar reason"), + "Message": Equal("foobar message"), + }), + )) + }) + }) + + Context("when Builder has no old conditions", func() { + It("should correctly set the new conditions", func() { + builder.WithResults([]Result{ + &result{ + ConType: druidv1alpha1.ConditionTypeAllMembersReady, + ConStatus: druidv1alpha1.ConditionTrue, + ConReason: "new reason", + ConMessage: "new message", + }, + &result{ + ConType: druidv1alpha1.ConditionTypeReady, + ConStatus: druidv1alpha1.ConditionTrue, + ConReason: "new reason", + ConMessage: "new message", + }, + }) + + conditions := builder.Build() + + Expect(conditions).To(ConsistOf( + MatchFields(IgnoreExtras, Fields{ + "Type": Equal(druidv1alpha1.ConditionTypeAllMembersReady), + "LastUpdateTime": Equal(metav1.NewTime(now)), + "LastTransitionTime": Equal(metav1.NewTime(now)), + "Status": Equal(druidv1alpha1.ConditionTrue), + "Reason": Equal("new reason"), + "Message": Equal("new message"), + }), + MatchFields(IgnoreExtras, Fields{ + "Type": Equal(druidv1alpha1.ConditionTypeReady), + "LastUpdateTime": Equal(metav1.NewTime(now)), + "LastTransitionTime": Equal(metav1.NewTime(now)), + "Status": Equal(druidv1alpha1.ConditionTrue), + "Reason": Equal("new reason"), + "Message": Equal("new message"), + }), + )) + }) + }) + }) +}) diff --git a/pkg/health/condition/check_all_members.go b/pkg/health/condition/check_all_members.go new file mode 100644 index 000000000..7c01d5c82 --- /dev/null +++ b/pkg/health/condition/check_all_members.go @@ -0,0 +1,54 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 condition + +import druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + +type allMembersReady struct{} + +func (a *allMembersReady) Check(status druidv1alpha1.EtcdStatus) Result { + if len(status.Members) == 0 { + return &result{ + conType: druidv1alpha1.ConditionTypeAllMembersReady, + status: druidv1alpha1.ConditionUnknown, + reason: "NoMembersInStatus", + message: "Cannot determine readiness since status has no members", + } + } + + result := &result{ + conType: druidv1alpha1.ConditionTypeAllMembersReady, + status: druidv1alpha1.ConditionTrue, + reason: "AllMembersReady", + message: "All members are ready", + } + + for _, member := range status.Members { + if member.Status != druidv1alpha1.EtcdMemeberStatusReady { + result.status = druidv1alpha1.ConditionFalse + result.reason = "NotAllMembersReady" + result.message = "At least one member is not ready" + + return result + } + } + + return result +} + +// AllMembersCheck returns a check for the "AllMembersReady" condition. +func AllMembersCheck() Checker { + return &allMembersReady{} +} diff --git a/pkg/health/condition/check_all_members_test.go b/pkg/health/condition/check_all_members_test.go new file mode 100644 index 000000000..8b87e6469 --- /dev/null +++ b/pkg/health/condition/check_all_members_test.go @@ -0,0 +1,86 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 condition_test + +import ( + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + . "github.com/gardener/etcd-druid/pkg/health/condition" +) + +var _ = Describe("AllMembersReadyCheck", func() { + Describe("#Check", func() { + var readyMember, notReadyMember druidv1alpha1.EtcdMemberStatus + + BeforeEach(func() { + readyMember = druidv1alpha1.EtcdMemberStatus{ + Status: druidv1alpha1.EtcdMemeberStatusReady, + } + notReadyMember = druidv1alpha1.EtcdMemberStatus{ + Status: druidv1alpha1.EtcdMemeberStatusNotReady, + } + }) + + Context("when members in status", func() { + It("should return that all members are ready", func() { + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + readyMember, + readyMember, + readyMember, + }, + } + check := AllMembersCheck() + + result := check.Check(status) + + Expect(result.ConditionType()).To(Equal(druidv1alpha1.ConditionTypeAllMembersReady)) + Expect(result.Status()).To(Equal(druidv1alpha1.ConditionTrue)) + }) + + It("should return that members are not ready", func() { + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + readyMember, + notReadyMember, + readyMember, + }, + } + check := AllMembersCheck() + + result := check.Check(status) + + Expect(result.ConditionType()).To(Equal(druidv1alpha1.ConditionTypeAllMembersReady)) + Expect(result.Status()).To(Equal(druidv1alpha1.ConditionFalse)) + }) + }) + + Context("when no members in status", func() { + It("should return that readiness is unknown", func() { + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{}, + } + check := AllMembersCheck() + + result := check.Check(status) + + Expect(result.ConditionType()).To(Equal(druidv1alpha1.ConditionTypeAllMembersReady)) + Expect(result.Status()).To(Equal(druidv1alpha1.ConditionUnknown)) + }) + }) + }) +}) diff --git a/pkg/health/condition/check_ready.go b/pkg/health/condition/check_ready.go new file mode 100644 index 000000000..7453f610e --- /dev/null +++ b/pkg/health/condition/check_ready.go @@ -0,0 +1,62 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 condition + +import druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + +type readyCheck struct{} + +func (r *readyCheck) Check(status druidv1alpha1.EtcdStatus) Result { + if len(status.Members) == 0 { + return &result{ + conType: druidv1alpha1.ConditionTypeReady, + status: druidv1alpha1.ConditionUnknown, + reason: "NoMembersInStatus", + message: "Cannot determine readiness of cluster since status has no members", + } + } + + var ( + quorum = len(status.Members)/2 + 1 + readyMembers = 0 + ) + + for _, member := range status.Members { + if member.Status == druidv1alpha1.EtcdMemeberStatusReady { + readyMembers++ + } + } + + if readyMembers < quorum { + return &result{ + conType: druidv1alpha1.ConditionTypeReady, + status: druidv1alpha1.ConditionFalse, + reason: "QuorumLost", + message: "The majority of ETCD nodes are not ready", + } + } + + return &result{ + conType: druidv1alpha1.ConditionTypeReady, + status: druidv1alpha1.ConditionTrue, + reason: "Quorate", + message: "The majority of ETCD nodes is ready", + } +} + +// ReadyCheck returns a check for the "Ready" condition. +func ReadyCheck() Checker { + return &readyCheck{} +} diff --git a/pkg/health/condition/check_ready_test.go b/pkg/health/condition/check_ready_test.go new file mode 100644 index 000000000..fd40e7c4d --- /dev/null +++ b/pkg/health/condition/check_ready_test.go @@ -0,0 +1,105 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 condition_test + +import ( + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + . "github.com/gardener/etcd-druid/pkg/health/condition" +) + +var _ = Describe("ReadyCheck", func() { + Describe("#Check", func() { + var readyMember, notReadyMember druidv1alpha1.EtcdMemberStatus + + BeforeEach(func() { + readyMember = druidv1alpha1.EtcdMemberStatus{ + Status: druidv1alpha1.EtcdMemeberStatusReady, + } + notReadyMember = druidv1alpha1.EtcdMemberStatus{ + Status: druidv1alpha1.EtcdMemeberStatusNotReady, + } + }) + + Context("when members in status", func() { + It("should return that the cluster has a quorum (all members ready)", func() { + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + readyMember, + readyMember, + readyMember, + }, + } + check := ReadyCheck() + + result := check.Check(status) + + Expect(result.ConditionType()).To(Equal(druidv1alpha1.ConditionTypeReady)) + Expect(result.Status()).To(Equal(druidv1alpha1.ConditionTrue)) + }) + + It("should return that the cluster has a quorum (one member not ready)", func() { + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + readyMember, + notReadyMember, + readyMember, + }, + } + check := ReadyCheck() + + result := check.Check(status) + + Expect(result.ConditionType()).To(Equal(druidv1alpha1.ConditionTypeReady)) + Expect(result.Status()).To(Equal(druidv1alpha1.ConditionTrue)) + }) + + It("should return that the cluster has lost its quorum", func() { + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + readyMember, + notReadyMember, + notReadyMember, + }, + } + check := ReadyCheck() + + result := check.Check(status) + + Expect(result.ConditionType()).To(Equal(druidv1alpha1.ConditionTypeReady)) + Expect(result.Status()).To(Equal(druidv1alpha1.ConditionFalse)) + Expect(result.Reason()).To(Equal("QuorumLost")) + }) + }) + + Context("when no members in status", func() { + It("should return that quorum is unknown", func() { + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{}, + } + check := ReadyCheck() + + result := check.Check(status) + + Expect(result.ConditionType()).To(Equal(druidv1alpha1.ConditionTypeReady)) + Expect(result.Status()).To(Equal(druidv1alpha1.ConditionUnknown)) + Expect(result.Reason()).To(Equal("NoMembersInStatus")) + }) + }) + }) + +}) diff --git a/pkg/health/condition/condition_suite_test.go b/pkg/health/condition/condition_suite_test.go new file mode 100644 index 000000000..b38480bb8 --- /dev/null +++ b/pkg/health/condition/condition_suite_test.go @@ -0,0 +1,52 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 condition_test + +import ( + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" +) + +func TestCondition(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "Condition Suite") +} + +type result struct { + ConType druidv1alpha1.ConditionType + ConStatus druidv1alpha1.ConditionStatus + ConReason string + ConMessage string +} + +func (r *result) ConditionType() druidv1alpha1.ConditionType { + return r.ConType +} + +func (r *result) Status() druidv1alpha1.ConditionStatus { + return r.ConStatus +} + +func (r *result) Reason() string { + return r.ConReason +} + +func (r *result) Message() string { + return r.ConMessage +} diff --git a/pkg/health/condition/types.go b/pkg/health/condition/types.go new file mode 100644 index 000000000..25c02a57e --- /dev/null +++ b/pkg/health/condition/types.go @@ -0,0 +1,52 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 condition + +import druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + +// Checker is an interface to check the etcd resource and to return condition results. +type Checker interface { + Check(status druidv1alpha1.EtcdStatus) Result +} + +type Result interface { + ConditionType() druidv1alpha1.ConditionType + Status() druidv1alpha1.ConditionStatus + Reason() string + Message() string +} + +type result struct { + conType druidv1alpha1.ConditionType + status druidv1alpha1.ConditionStatus + reason string + message string +} + +func (r *result) ConditionType() druidv1alpha1.ConditionType { + return r.conType +} + +func (r *result) Status() druidv1alpha1.ConditionStatus { + return r.status +} + +func (r *result) Reason() string { + return r.reason +} + +func (r *result) Message() string { + return r.message +} diff --git a/pkg/health/etcdmember/builder.go b/pkg/health/etcdmember/builder.go new file mode 100644 index 000000000..8b70e34a7 --- /dev/null +++ b/pkg/health/etcdmember/builder.go @@ -0,0 +1,112 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 etcdmember + +import ( + "time" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +// Builder is an interface for building status objects for etcd members. +type Builder interface { + WithOldMembers(members []druidv1alpha1.EtcdMemberStatus) Builder + WithResults(results []Result) Builder + WithNowFunc(now func() metav1.Time) Builder + Build() []druidv1alpha1.EtcdMemberStatus +} + +type defaultBuilder struct { + old map[string]druidv1alpha1.EtcdMemberStatus + results map[string]Result + nowFunc func() metav1.Time +} + +// NewBuilder returns a Builder for a specific etcd member status. +func NewBuilder() Builder { + return &defaultBuilder{ + old: make(map[string]druidv1alpha1.EtcdMemberStatus), + results: make(map[string]Result), + nowFunc: func() metav1.Time { + return metav1.NewTime(time.Now().UTC()) + }, + } +} + +// WithOldMember sets the old etcd member statuses. It can be used to provide default values. +func (b *defaultBuilder) WithOldMembers(members []druidv1alpha1.EtcdMemberStatus) Builder { + for _, member := range members { + b.old[member.ID] = member + } + + return b +} + +// WithResults adds the results. +func (b *defaultBuilder) WithResults(results []Result) Builder { + for _, res := range results { + if res == nil { + continue + } + b.results[res.ID()] = res + } + + return b +} + +// WithNowFunc sets the function used for getting the current time. +// Should only be used for tests. +func (b *defaultBuilder) WithNowFunc(now func() metav1.Time) Builder { + b.nowFunc = now + return b +} + +// Build creates the etcd member statuses. +// It merges the existing members with the results added to the builder. +// If OldCondition is provided: +// - Any changes to status set the `LastTransitionTime` +// - `LastUpdateTime` is always set. +func (b *defaultBuilder) Build() []druidv1alpha1.EtcdMemberStatus { + var ( + now = b.nowFunc() + + members []druidv1alpha1.EtcdMemberStatus + ) + + for id, res := range b.results { + member, ok := b.old[id] + if !ok { + // Continue if we can't find an existing member because druid is not supposed to add one. + continue + } + + member.Status = res.Status() + member.LastTransitionTime = now + member.LastUpdateTime = now + member.Reason = res.Reason() + + members = append(members, member) + delete(b.old, id) + } + + for _, member := range b.old { + // Add existing members as they were. This needs to be changed when SSA is used. + members = append(members, member) + } + + return members +} diff --git a/pkg/health/etcdmember/builder_test.go b/pkg/health/etcdmember/builder_test.go new file mode 100644 index 000000000..f1656431c --- /dev/null +++ b/pkg/health/etcdmember/builder_test.go @@ -0,0 +1,137 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 etcdmember_test + +import ( + "time" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + . "github.com/onsi/gomega/gstruct" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + . "github.com/gardener/etcd-druid/pkg/health/etcdmember" +) + +var _ = Describe("Builder", func() { + var ( + builder Builder + now time.Time + ) + + BeforeEach(func() { + now, _ = time.Parse(time.RFC3339, "2021-06-01") + builder = NewBuilder() + }) + + JustBeforeEach(func() { + builder.WithNowFunc(func() metav1.Time { + return metav1.NewTime(now) + }) + }) + + Describe("#Build", func() { + Context("when Builder has old members", func() { + var ( + oldMembers map[string]druidv1alpha1.EtcdMemberStatus + ) + BeforeEach(func() { + oldMembers = map[string]druidv1alpha1.EtcdMemberStatus{ + "1": { + Name: "member1", + ID: "1", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "foo reason", + LastUpdateTime: metav1.NewTime(now.Add(-12 * time.Hour)), + LastTransitionTime: metav1.NewTime(now.Add(-12 * time.Hour)), + }, + "2": { + Name: "member2", + ID: "2", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "bar reason", + LastUpdateTime: metav1.NewTime(now.Add(-6 * time.Hour)), + LastTransitionTime: metav1.NewTime(now.Add(-6 * time.Hour)), + }, + "3": { + Name: "member3", + ID: "3", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "foobar reason", + LastUpdateTime: metav1.NewTime(now.Add(-18 * time.Hour)), + LastTransitionTime: metav1.NewTime(now.Add(-18 * time.Hour)), + }, + } + + builder.WithOldMembers([]druidv1alpha1.EtcdMemberStatus{ + oldMembers["1"], + oldMembers["2"], + oldMembers["3"], + }) + }) + + It("should correctly merge them", func() { + builder.WithResults([]Result{ + &result{ + MemberID: "3", + MemberStatus: druidv1alpha1.EtcdMemeberStatusUnknown, + MemberReason: "unknown reason", + }, + }) + + conditions := builder.Build() + + Expect(conditions).To(ConsistOf( + oldMembers["1"], + oldMembers["2"], + MatchFields(IgnoreExtras, Fields{ + "Name": Equal("member3"), + "ID": Equal("3"), + "Role": Equal(druidv1alpha1.EtcdRoleMember), + "Status": Equal(druidv1alpha1.EtcdMemeberStatusUnknown), + "Reason": Equal("unknown reason"), + "LastUpdateTime": Equal(metav1.NewTime(now)), + "LastTransitionTime": Equal(metav1.NewTime(now)), + }), + )) + }) + }) + + Context("when Builder has no old members", func() { + It("should not add any members", func() { + builder.WithResults([]Result{ + &result{ + MemberID: "1", + MemberStatus: druidv1alpha1.EtcdMemeberStatusUnknown, + MemberReason: "unknown reason", + }, + &result{ + MemberID: "2", + MemberStatus: druidv1alpha1.EtcdMemeberStatusReady, + MemberReason: "foo reason", + }, + }) + + conditions := builder.Build() + + Expect(conditions).To(BeEmpty()) + }) + }) + }) +}) diff --git a/pkg/health/etcdmember/check_ready.go b/pkg/health/etcdmember/check_ready.go new file mode 100644 index 000000000..615a6daa7 --- /dev/null +++ b/pkg/health/etcdmember/check_ready.go @@ -0,0 +1,56 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 etcdmember + +import ( + "time" + + "github.com/gardener/etcd-druid/controllers/config" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" +) + +type readyCheck struct { + etcdStaleMemberThreshold time.Duration +} + +// TimeNow is the function used by this check to get the current time. +var TimeNow = time.Now + +func (r *readyCheck) Check(status druidv1alpha1.EtcdStatus) []Result { + var ( + results []Result + threshold = TimeNow().UTC().Add(-1 * r.etcdStaleMemberThreshold) + ) + + for _, etcd := range status.Members { + if etcd.LastUpdateTime.Time.Before(threshold) { + results = append(results, &result{ + id: etcd.ID, + status: druidv1alpha1.EtcdMemeberStatusUnknown, + reason: "UnknownMemberStatus", + }) + } + } + + return results +} + +// ReadyCheck returns a check for the "Ready" condition. +func ReadyCheck(config config.EtcdCustodianController) Checker { + return &readyCheck{ + etcdStaleMemberThreshold: config.EtcdStaleMemberThreshold, + } +} diff --git a/pkg/health/etcdmember/check_ready_test.go b/pkg/health/etcdmember/check_ready_test.go new file mode 100644 index 000000000..d05cfae35 --- /dev/null +++ b/pkg/health/etcdmember/check_ready_test.go @@ -0,0 +1,118 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 etcdmember_test + +import ( + "time" + + "github.com/gardener/gardener/pkg/utils/test" + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + "github.com/gardener/etcd-druid/controllers/config" + . "github.com/gardener/etcd-druid/pkg/health/etcdmember" +) + +var _ = Describe("ReadyCheck", func() { + Describe("#Check", func() { + var ( + threshold time.Duration + now time.Time + check Checker + ) + BeforeEach(func() { + threshold = 300 * time.Second + now, _ = time.Parse(time.RFC3339, "2021-06-01T00:00:00Z") + check = ReadyCheck(config.EtcdCustodianController{ + EtcdStaleMemberThreshold: threshold, + }) + }) + + Context("when condition is outdated", func() { + It("should set the affected condition to UNKNOWN", func() { + defer test.WithVar(&TimeNow, func() time.Time { + return now + })() + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + { + Name: "member1", + ID: "1", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "foo reason", + LastTransitionTime: metav1.Now(), + LastUpdateTime: metav1.NewTime(now.Add(-301 * time.Second)), + }, + { + Name: "member2", + ID: "2", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "bar reason", + LastTransitionTime: metav1.Now(), + LastUpdateTime: metav1.NewTime(now.Add(-1 * threshold)), + }, + }, + } + + check := ReadyCheck(config.EtcdCustodianController{ + EtcdStaleMemberThreshold: threshold, + }) + + results := check.Check(status) + + Expect(results).To(HaveLen(1)) + Expect(results[0].Status()).To(Equal(druidv1alpha1.EtcdMemeberStatusUnknown)) + Expect(results[0].ID()).To(Equal("1")) + }) + }) + Context("when condition is not outdated", func() { + It("should not return any results", func() { + defer test.WithVar(&TimeNow, func() time.Time { + return now + })() + status := druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + { + Name: "member1", + ID: "1", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "foo reason", + LastTransitionTime: metav1.Now(), + LastUpdateTime: metav1.Now(), + }, + { + Name: "member2", + ID: "2", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "bar reason", + LastTransitionTime: metav1.Now(), + LastUpdateTime: metav1.NewTime(now.Add(-1 * threshold)), + }, + }, + } + + results := check.Check(status) + + Expect(results).To(BeEmpty()) + }) + }) + }) +}) diff --git a/pkg/health/etcdmember/etcdmember_suite_test.go b/pkg/health/etcdmember/etcdmember_suite_test.go new file mode 100644 index 000000000..779118980 --- /dev/null +++ b/pkg/health/etcdmember/etcdmember_suite_test.go @@ -0,0 +1,47 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 etcdmember_test + +import ( + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" +) + +func TestEtcdMember(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "Etcd Member Suite") +} + +type result struct { + MemberID string + MemberStatus druidv1alpha1.EtcdMemberConditionStatus + MemberReason string +} + +func (r *result) ID() string { + return r.MemberID +} + +func (r *result) Reason() string { + return r.MemberReason +} + +func (r *result) Status() druidv1alpha1.EtcdMemberConditionStatus { + return r.MemberStatus +} diff --git a/pkg/health/etcdmember/types.go b/pkg/health/etcdmember/types.go new file mode 100644 index 000000000..336afae10 --- /dev/null +++ b/pkg/health/etcdmember/types.go @@ -0,0 +1,46 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 etcdmember + +import druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + +// Checker is an interface to check the members of a etcd cluster. +type Checker interface { + Check(status druidv1alpha1.EtcdStatus) []Result +} + +type Result interface { + ID() string + Status() druidv1alpha1.EtcdMemberConditionStatus + Reason() string +} + +type result struct { + id string + status druidv1alpha1.EtcdMemberConditionStatus + reason string +} + +func (r *result) ID() string { + return r.id +} + +func (r *result) Status() druidv1alpha1.EtcdMemberConditionStatus { + return r.status +} + +func (r *result) Reason() string { + return r.reason +} diff --git a/pkg/health/status/check.go b/pkg/health/status/check.go new file mode 100644 index 000000000..3ace2b4b1 --- /dev/null +++ b/pkg/health/status/check.go @@ -0,0 +1,145 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 status + +import ( + "context" + "sync" + "time" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + "github.com/gardener/etcd-druid/controllers/config" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + "github.com/gardener/etcd-druid/pkg/health/condition" + "github.com/gardener/etcd-druid/pkg/health/etcdmember" +) + +// ConditionCheckFn is a type alias for a function which returns an implementation of `Check`. +type ConditionCheckFn func() condition.Checker + +// EtcdMemberCheckFn is a type alias for a function which returns an implementation of `Check`. +type EtcdMemberCheckFn func(config.EtcdCustodianController) etcdmember.Checker + +// TimeNow is the function used to get the current time. +var TimeNow = time.Now + +var ( + // NewDefaultConditionBuilder is the default condition builder. + NewDefaultConditionBuilder = condition.NewBuilder + // NewDefaultEtcdMemberBuilder is the default etcd member builder. + NewDefaultEtcdMemberBuilder = etcdmember.NewBuilder + // Checks are the registered condition checks. + ConditionChecks = []ConditionCheckFn{ + condition.AllMembersCheck, + condition.ReadyCheck, + } + // EtcdMemberChecks are the etcd member checks. + EtcdMemberChecks = []EtcdMemberCheckFn{ + etcdmember.ReadyCheck, + } +) + +type checker struct { + config config.EtcdCustodianController + conditionCheckFns []ConditionCheckFn + conditionBuilderFn func() condition.Builder + etcdMemberCheckFns []EtcdMemberCheckFn + etcdMemberBuilderFn func() etcdmember.Builder +} + +// Check executes the status checks and mutates the passed status object with the corresponding results. +func (c *checker) Check(ctx context.Context, status *druidv1alpha1.EtcdStatus) error { + // First execute the etcd member checks for the status. + if err := c.executeEtcdMemberChecks(status); err != nil { + return err + } + + // Execute condition checks after the etcd member checks because we need their result here. + if err := c.executeConditionChecks(status); err != nil { + return err + } + return nil +} + +// executeConditionChecks runs all registered condition checks **in parallel**. +func (c *checker) executeConditionChecks(status *druidv1alpha1.EtcdStatus) error { + var ( + resultCh = make(chan condition.Result) + + wg sync.WaitGroup + ) + + // Run condition checks in parallel since they work independently from each other. + for _, newCheck := range c.conditionCheckFns { + c := newCheck() + wg.Add(1) + go (func() { + defer wg.Done() + resultCh <- c.Check(*status) + })() + } + + go (func() { + defer close(resultCh) + wg.Wait() + })() + + results := make([]condition.Result, 0, len(ConditionChecks)) + for r := range resultCh { + results = append(results, r) + } + + conditions := c.conditionBuilderFn(). + WithNowFunc(func() metav1.Time { return metav1.NewTime(TimeNow()) }). + WithOldConditions(status.Conditions). + WithResults(results). + Build() + + status.Conditions = conditions + return nil +} + +// executeEtcdMemberChecks runs all registered etcd member checks **sequentially**. +// The result of a check is passed via the `status` sub-resources to the next check. +func (c *checker) executeEtcdMemberChecks(status *druidv1alpha1.EtcdStatus) error { + // Run etcd member checks sequentially as most of them act on multiple elements. + for _, newCheck := range c.etcdMemberCheckFns { + results := newCheck(c.config).Check(*status) + + // Build and assign the results after each check, so that the next check + // can act on the latest results. + memberStatuses := c.etcdMemberBuilderFn(). + WithNowFunc(func() metav1.Time { return metav1.NewTime(TimeNow()) }). + WithOldMembers(status.Members). + WithResults(results). + Build() + + status.Members = memberStatuses + } + return nil +} + +// NewChecker creates a new instance for checking the etcd status. +func NewChecker(config config.EtcdCustodianController) *checker { + return &checker{ + config: config, + conditionCheckFns: ConditionChecks, + conditionBuilderFn: NewDefaultConditionBuilder, + etcdMemberCheckFns: EtcdMemberChecks, + etcdMemberBuilderFn: NewDefaultEtcdMemberBuilder, + } +} diff --git a/pkg/health/status/check_test.go b/pkg/health/status/check_test.go new file mode 100644 index 000000000..81871b43c --- /dev/null +++ b/pkg/health/status/check_test.go @@ -0,0 +1,263 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 status_test + +import ( + "context" + "time" + + "github.com/gardener/gardener/pkg/utils/test" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + controllerconfig "github.com/gardener/etcd-druid/controllers/config" + "github.com/gardener/etcd-druid/pkg/health/condition" + "github.com/gardener/etcd-druid/pkg/health/etcdmember" + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + . "github.com/onsi/gomega/gstruct" + + . "github.com/gardener/etcd-druid/pkg/health/status" +) + +var _ = Describe("Check", func() { + Describe("#Check", func() { + It("should correctly execute checks and fill status", func() { + config := controllerconfig.EtcdCustodianController{ + EtcdStaleMemberThreshold: 1 * time.Minute, + } + timeBefore, _ := time.Parse(time.RFC3339, "2021-06-01T00:00:00Z") + timeNow := timeBefore.Add(1 * time.Hour) + + status := druidv1alpha1.EtcdStatus{ + Conditions: []druidv1alpha1.Condition{ + { + Type: druidv1alpha1.ConditionTypeReady, + Status: druidv1alpha1.ConditionTrue, + LastTransitionTime: metav1.NewTime(timeBefore), + LastUpdateTime: metav1.NewTime(timeBefore), + Reason: "foo reason", + Message: "foo message", + }, + { + Type: druidv1alpha1.ConditionTypeAllMembersReady, + Status: druidv1alpha1.ConditionTrue, + LastTransitionTime: metav1.NewTime(timeBefore), + LastUpdateTime: metav1.NewTime(timeBefore), + Reason: "bar reason", + Message: "bar message", + }, + { + Type: druidv1alpha1.ConditionTypeBackupReady, + Status: druidv1alpha1.ConditionUnknown, + LastTransitionTime: metav1.NewTime(timeBefore), + LastUpdateTime: metav1.NewTime(timeBefore), + Reason: "foobar reason", + Message: "foobar message", + }, + }, + Members: []druidv1alpha1.EtcdMemberStatus{ + { + ID: "1", + Name: "Member1", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + LastTransitionTime: metav1.NewTime(timeBefore), + LastUpdateTime: metav1.NewTime(timeBefore), + Reason: "foo reason", + }, + { + ID: "2", + Name: "Member2", + Role: druidv1alpha1.EtcdRoleLearner, + Status: druidv1alpha1.EtcdMemeberStatusNotReady, + LastTransitionTime: metav1.NewTime(timeBefore), + LastUpdateTime: metav1.NewTime(timeBefore), + Reason: "bar reason", + }, + { + ID: "3", + Name: "Member3", + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + LastTransitionTime: metav1.NewTime(timeBefore), + LastUpdateTime: metav1.NewTime(timeBefore), + Reason: "foobar reason", + }, + }, + } + + defer test.WithVar(&ConditionChecks, []ConditionCheckFn{ + func() condition.Checker { + return createConditionCheck(druidv1alpha1.ConditionTypeReady, druidv1alpha1.ConditionFalse, "FailedConditionCheck", "check failed") + }, + func() condition.Checker { + return createConditionCheck(druidv1alpha1.ConditionTypeAllMembersReady, druidv1alpha1.ConditionTrue, "bar reason", "bar message") + }, + })() + + defer test.WithVar(&EtcdMemberChecks, []EtcdMemberCheckFn{ + func(_ controllerconfig.EtcdCustodianController) etcdmember.Checker { + return createEtcdMemberCheck("1", druidv1alpha1.EtcdMemeberStatusUnknown, "Unknown") + }, + })() + + defer test.WithVar(&TimeNow, func() time.Time { return timeNow })() + + checker := NewChecker(config) + + Expect(checker.Check(context.Background(), &status)).To(Succeed()) + + Expect(status.Conditions).To(ConsistOf( + MatchFields(IgnoreExtras, Fields{ + "Type": Equal(druidv1alpha1.ConditionTypeReady), + "Status": Equal(druidv1alpha1.ConditionFalse), + "LastTransitionTime": Equal(metav1.NewTime(timeNow)), + "LastUpdateTime": Equal(metav1.NewTime(timeNow)), + "Reason": Equal("FailedConditionCheck"), + "Message": Equal("check failed"), + }), + MatchFields(IgnoreExtras, Fields{ + "Type": Equal(druidv1alpha1.ConditionTypeAllMembersReady), + "Status": Equal(druidv1alpha1.ConditionTrue), + "LastTransitionTime": Equal(metav1.NewTime(timeBefore)), + "LastUpdateTime": Equal(metav1.NewTime(timeNow)), + "Reason": Equal("bar reason"), + "Message": Equal("bar message"), + }), + MatchFields(IgnoreExtras, Fields{ + "Type": Equal(druidv1alpha1.ConditionTypeBackupReady), + "Status": Equal(druidv1alpha1.ConditionUnknown), + "LastTransitionTime": Equal(metav1.NewTime(timeBefore)), + "LastUpdateTime": Equal(metav1.NewTime(timeBefore)), + "Reason": Equal("foobar reason"), + "Message": Equal("foobar message"), + }), + )) + + Expect(status.Members).To(ConsistOf( + MatchFields(IgnoreExtras, Fields{ + "ID": Equal("1"), + "Name": Equal("Member1"), + "Role": Equal(druidv1alpha1.EtcdRoleMember), + "Status": Equal(druidv1alpha1.EtcdMemeberStatusUnknown), + "LastTransitionTime": Equal(metav1.NewTime(timeNow)), + "LastUpdateTime": Equal(metav1.NewTime(timeNow)), + "Reason": Equal("Unknown"), + }), + MatchFields(IgnoreExtras, Fields{ + "ID": Equal("2"), + "Name": Equal("Member2"), + "Role": Equal(druidv1alpha1.EtcdRoleLearner), + "Status": Equal(druidv1alpha1.EtcdMemeberStatusNotReady), + "LastTransitionTime": Equal(metav1.NewTime(timeBefore)), + "LastUpdateTime": Equal(metav1.NewTime(timeBefore)), + "Reason": Equal("bar reason"), + }), + MatchFields(IgnoreExtras, Fields{ + "ID": Equal("3"), + "Name": Equal("Member3"), + "Role": Equal(druidv1alpha1.EtcdRoleMember), + "Status": Equal(druidv1alpha1.EtcdMemeberStatusReady), + "LastTransitionTime": Equal(metav1.NewTime(timeBefore)), + "LastUpdateTime": Equal(metav1.NewTime(timeBefore)), + "Reason": Equal("foobar reason"), + }), + )) + + }) + }) +}) + +type conditionResult struct { + ConType druidv1alpha1.ConditionType + ConStatus druidv1alpha1.ConditionStatus + ConReason string + ConMessage string +} + +func (r *conditionResult) ConditionType() druidv1alpha1.ConditionType { + return r.ConType +} + +func (r *conditionResult) Status() druidv1alpha1.ConditionStatus { + return r.ConStatus +} + +func (r *conditionResult) Reason() string { + return r.ConReason +} + +func (r *conditionResult) Message() string { + return r.ConMessage +} + +type testChecker struct { + result *conditionResult +} + +func (t *testChecker) Check(_ druidv1alpha1.EtcdStatus) condition.Result { + return t.result +} + +func createConditionCheck(conType druidv1alpha1.ConditionType, status druidv1alpha1.ConditionStatus, reason, message string) condition.Checker { + return &testChecker{ + result: &conditionResult{ + ConType: conType, + ConStatus: status, + ConReason: reason, + ConMessage: message, + }, + } +} + +type etcdMemberResult struct { + id string + status druidv1alpha1.EtcdMemberConditionStatus + reason string +} + +func (r *etcdMemberResult) ID() string { + return r.id +} + +func (r *etcdMemberResult) Status() druidv1alpha1.EtcdMemberConditionStatus { + return r.status +} + +func (r *etcdMemberResult) Reason() string { + return r.reason +} + +type etcdMemberTestChecker struct { + result *etcdMemberResult +} + +func (t *etcdMemberTestChecker) Check(_ druidv1alpha1.EtcdStatus) []etcdmember.Result { + return []etcdmember.Result{ + t.result, + } +} + +func createEtcdMemberCheck(id string, status druidv1alpha1.EtcdMemberConditionStatus, reason string) etcdmember.Checker { + return &etcdMemberTestChecker{ + result: &etcdMemberResult{ + id: id, + status: status, + reason: reason, + }, + } +} diff --git a/pkg/health/status/status_suite_test.go b/pkg/health/status/status_suite_test.go new file mode 100644 index 000000000..4b27066ab --- /dev/null +++ b/pkg/health/status/status_suite_test.go @@ -0,0 +1,27 @@ +// Copyright (c) 2021 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file +// +// 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 status_test + +import ( + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +func TestStatus(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "Status Suite") +} From dde320bc8c9335da3443eb255922c362438325fb Mon Sep 17 00:00:00 2001 From: Tim Usner Date: Wed, 2 Jun 2021 11:24:59 +0200 Subject: [PATCH 2/7] Add sync period to custodian controller --- controllers/config/custodian.go | 1 + controllers/etcd_custodian_controller.go | 8 ++++++-- main.go | 3 +++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/controllers/config/custodian.go b/controllers/config/custodian.go index 278c55249..9e1993212 100644 --- a/controllers/config/custodian.go +++ b/controllers/config/custodian.go @@ -19,4 +19,5 @@ import "time" // EtcdCustodianController contains configuration for the etcd custodian controller. type EtcdCustodianController struct { EtcdStaleMemberThreshold time.Duration + SyncPeriod time.Duration } diff --git a/controllers/etcd_custodian_controller.go b/controllers/etcd_custodian_controller.go index ef53b2b02..35b5099c1 100644 --- a/controllers/etcd_custodian_controller.go +++ b/controllers/etcd_custodian_controller.go @@ -120,12 +120,15 @@ func (ec *EtcdCustodian) Reconcile(ctx context.Context, req ctrl.Request) (ctrl. return ctrl.Result{}, err } - return ctrl.Result{}, nil + return ctrl.Result{RequeueAfter: ec.config.SyncPeriod}, nil } func (ec *EtcdCustodian) updateEtcdStatus(ctx context.Context, logger logr.Logger, etcd *druidv1alpha1.Etcd, sts *appsv1.StatefulSet) error { logger.Info("Updating etcd status with statefulset information") - conditions := etcd.Status.Conditions + var ( + conditions = etcd.Status.Conditions + members = etcd.Status.Members + ) return kutil.TryUpdateStatus(ctx, retry.DefaultBackoff, ec.Client, etcd, func() error { etcd.Status.Etcd = &druidv1alpha1.CrossVersionObjectReference{ @@ -135,6 +138,7 @@ func (ec *EtcdCustodian) updateEtcdStatus(ctx context.Context, logger logr.Logge } etcd.Status.Conditions = conditions + etcd.Status.Members = members ready := CheckStatefulSet(etcd, sts) == nil diff --git a/main.go b/main.go index 6da4dff51..352eff0cb 100644 --- a/main.go +++ b/main.go @@ -54,6 +54,7 @@ func main() { leaderElectionResourceLock string etcdWorkers int custodianWorkers int + custodianSyncPeriod time.Duration ignoreOperationAnnotation bool etcdStaleMemberThreshold time.Duration @@ -65,6 +66,7 @@ func main() { flag.IntVar(&etcdWorkers, "workers", 3, "Number of worker threads of the etcd controller.") flag.IntVar(&custodianWorkers, "custodian-workers", 3, "Number of worker threads of the custodian controller.") + flag.DurationVar(&custodianSyncPeriod, "custodian-sync-period", 30*time.Second, "Sync period of the custodian controller.") flag.StringVar(&metricsAddr, "metrics-addr", ":8080", "The address the metric endpoint binds to.") flag.BoolVar(&enableLeaderElection, "enable-leader-election", false, "Enable leader election for controller manager. Enabling this will ensure there is only one active controller manager.") @@ -107,6 +109,7 @@ func main() { custodian := controllers.NewEtcdCustodian(mgr, config.EtcdCustodianController{ EtcdStaleMemberThreshold: etcdStaleMemberThreshold, + SyncPeriod: custodianSyncPeriod, }) if err := custodian.SetupWithManager(ctx, mgr, custodianWorkers); err != nil { From 82fce483d06f67c7b606f2ea9c6d5bd9a3e7c6a8 Mon Sep 17 00:00:00 2001 From: Tim Usner Date: Fri, 4 Jun 2021 10:37:40 +0200 Subject: [PATCH 3/7] Address minor comments from review --- pkg/health/condition/builder_test.go | 23 +++++++++++++++++ pkg/health/condition/check_ready.go | 4 +-- pkg/health/condition/condition_suite_test.go | 25 ------------------- pkg/health/etcdmember/builder_test.go | 18 +++++++++++++ .../etcdmember/etcdmember_suite_test.go | 20 --------------- 5 files changed, 43 insertions(+), 47 deletions(-) diff --git a/pkg/health/condition/builder_test.go b/pkg/health/condition/builder_test.go index 5918b2720..301bb9778 100644 --- a/pkg/health/condition/builder_test.go +++ b/pkg/health/condition/builder_test.go @@ -170,3 +170,26 @@ var _ = Describe("Builder", func() { }) }) }) + +type result struct { + ConType druidv1alpha1.ConditionType + ConStatus druidv1alpha1.ConditionStatus + ConReason string + ConMessage string +} + +func (r *result) ConditionType() druidv1alpha1.ConditionType { + return r.ConType +} + +func (r *result) Status() druidv1alpha1.ConditionStatus { + return r.ConStatus +} + +func (r *result) Reason() string { + return r.ConReason +} + +func (r *result) Message() string { + return r.ConMessage +} diff --git a/pkg/health/condition/check_ready.go b/pkg/health/condition/check_ready.go index 7453f610e..70a89661f 100644 --- a/pkg/health/condition/check_ready.go +++ b/pkg/health/condition/check_ready.go @@ -44,7 +44,7 @@ func (r *readyCheck) Check(status druidv1alpha1.EtcdStatus) Result { conType: druidv1alpha1.ConditionTypeReady, status: druidv1alpha1.ConditionFalse, reason: "QuorumLost", - message: "The majority of ETCD nodes are not ready", + message: "The majority of ETCD members is not ready", } } @@ -52,7 +52,7 @@ func (r *readyCheck) Check(status druidv1alpha1.EtcdStatus) Result { conType: druidv1alpha1.ConditionTypeReady, status: druidv1alpha1.ConditionTrue, reason: "Quorate", - message: "The majority of ETCD nodes is ready", + message: "The majority of ETCD members is ready", } } diff --git a/pkg/health/condition/condition_suite_test.go b/pkg/health/condition/condition_suite_test.go index b38480bb8..1624b2472 100644 --- a/pkg/health/condition/condition_suite_test.go +++ b/pkg/health/condition/condition_suite_test.go @@ -19,34 +19,9 @@ import ( . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" - - druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" ) func TestCondition(t *testing.T) { RegisterFailHandler(Fail) RunSpecs(t, "Condition Suite") } - -type result struct { - ConType druidv1alpha1.ConditionType - ConStatus druidv1alpha1.ConditionStatus - ConReason string - ConMessage string -} - -func (r *result) ConditionType() druidv1alpha1.ConditionType { - return r.ConType -} - -func (r *result) Status() druidv1alpha1.ConditionStatus { - return r.ConStatus -} - -func (r *result) Reason() string { - return r.ConReason -} - -func (r *result) Message() string { - return r.ConMessage -} diff --git a/pkg/health/etcdmember/builder_test.go b/pkg/health/etcdmember/builder_test.go index f1656431c..9a5bb3846 100644 --- a/pkg/health/etcdmember/builder_test.go +++ b/pkg/health/etcdmember/builder_test.go @@ -135,3 +135,21 @@ var _ = Describe("Builder", func() { }) }) }) + +type result struct { + MemberID string + MemberStatus druidv1alpha1.EtcdMemberConditionStatus + MemberReason string +} + +func (r *result) ID() string { + return r.MemberID +} + +func (r *result) Reason() string { + return r.MemberReason +} + +func (r *result) Status() druidv1alpha1.EtcdMemberConditionStatus { + return r.MemberStatus +} diff --git a/pkg/health/etcdmember/etcdmember_suite_test.go b/pkg/health/etcdmember/etcdmember_suite_test.go index 779118980..02df6ef0c 100644 --- a/pkg/health/etcdmember/etcdmember_suite_test.go +++ b/pkg/health/etcdmember/etcdmember_suite_test.go @@ -19,29 +19,9 @@ import ( . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" - - druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" ) func TestEtcdMember(t *testing.T) { RegisterFailHandler(Fail) RunSpecs(t, "Etcd Member Suite") } - -type result struct { - MemberID string - MemberStatus druidv1alpha1.EtcdMemberConditionStatus - MemberReason string -} - -func (r *result) ID() string { - return r.MemberID -} - -func (r *result) Reason() string { - return r.MemberReason -} - -func (r *result) Status() druidv1alpha1.EtcdMemberConditionStatus { - return r.MemberStatus -} From 134958db8e8cbbb9f6cec30fdbeaa05aa78587e7 Mon Sep 17 00:00:00 2001 From: Tim Usner Date: Fri, 4 Jun 2021 16:51:01 +0200 Subject: [PATCH 4/7] Handle Unknown -> Ready members --- api/v1alpha1/etcd_types.go | 2 + api/v1alpha1/zz_generated.deepcopy.go | 1 + .../crd/bases/druid.gardener.cloud_etcds.yaml | 10 + controllers/config/custodian.go | 13 +- controllers/controllers_suite_test.go | 9 +- controllers/etcd_custodian_controller.go | 10 +- main.go | 17 +- pkg/health/etcdmember/check_ready.go | 63 +++- pkg/health/etcdmember/check_ready_test.go | 273 ++++++++++++++---- pkg/health/etcdmember/types.go | 8 +- pkg/health/status/check.go | 26 +- pkg/health/status/check_test.go | 33 ++- .../etcd-druid/api/v1alpha1/etcd_types.go | 2 + .../api/v1alpha1/zz_generated.deepcopy.go | 1 + 14 files changed, 363 insertions(+), 105 deletions(-) diff --git a/api/v1alpha1/etcd_types.go b/api/v1alpha1/etcd_types.go index 8d8a76982..b109a1423 100644 --- a/api/v1alpha1/etcd_types.go +++ b/api/v1alpha1/etcd_types.go @@ -297,6 +297,8 @@ type EtcdMemberStatus struct { Name string `json:"name"` // ID is the ID of the etcd member. ID string `json:"id"` + // PodRef is the reference to the Pod which hosts the etcd member. + PodRef corev1.LocalObjectReference `json:"podRef"` // Role is the role in the etcd cluster, either `Member` or `Learner`. Role EtcdRole `json:"role"` // Status of the condition, one of True, False, Unknown. diff --git a/api/v1alpha1/zz_generated.deepcopy.go b/api/v1alpha1/zz_generated.deepcopy.go index c7604b824..24f8a33b5 100644 --- a/api/v1alpha1/zz_generated.deepcopy.go +++ b/api/v1alpha1/zz_generated.deepcopy.go @@ -268,6 +268,7 @@ func (in *EtcdList) DeepCopyObject() runtime.Object { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *EtcdMemberStatus) DeepCopyInto(out *EtcdMemberStatus) { *out = *in + out.PodRef = in.PodRef in.LastUpdateTime.DeepCopyInto(&out.LastUpdateTime) in.LastTransitionTime.DeepCopyInto(&out.LastTransitionTime) } diff --git a/config/crd/bases/druid.gardener.cloud_etcds.yaml b/config/crd/bases/druid.gardener.cloud_etcds.yaml index a1e375e2b..f6602fdfc 100644 --- a/config/crd/bases/druid.gardener.cloud_etcds.yaml +++ b/config/crd/bases/druid.gardener.cloud_etcds.yaml @@ -550,6 +550,15 @@ spec: name: description: Name is the name of the etcd member. type: string + podRef: + description: PodRef is the reference to the Pod which hosts + the etcd member. + properties: + name: + description: 'Name of the referent. More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + TODO: Add other useful fields. apiVersion, kind, uid?' + type: string + type: object reason: description: The reason for the condition's last transition. type: string @@ -565,6 +574,7 @@ spec: - lastTransitionTime - lastUpdateTime - name + - podRef - reason - role - status diff --git a/controllers/config/custodian.go b/controllers/config/custodian.go index 9e1993212..2c7602ab2 100644 --- a/controllers/config/custodian.go +++ b/controllers/config/custodian.go @@ -18,6 +18,15 @@ import "time" // EtcdCustodianController contains configuration for the etcd custodian controller. type EtcdCustodianController struct { - EtcdStaleMemberThreshold time.Duration - SyncPeriod time.Duration + // EtcdMember holds configuration related to etcd members. + EtcdMember EtcdMemberConfig + // SyncPeriod is the duration after which re-enqueuing happens. + SyncPeriod time.Duration +} + +type EtcdMemberConfig struct { + // EtcdMemberUnknownThreshold is the duration after which a etcd member's state is considered `Unknown`. + EtcdMemberUnknownThreshold time.Duration + // EtcdMemberUnknownThreshold is the duration after which a etcd member's state is considered `NotReady`. + EtcdMemberNotReadyThreshold time.Duration } diff --git a/controllers/controllers_suite_test.go b/controllers/controllers_suite_test.go index f94f86f81..65362b8d0 100644 --- a/controllers/controllers_suite_test.go +++ b/controllers/controllers_suite_test.go @@ -22,7 +22,7 @@ import ( "time" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" - "github.com/gardener/etcd-druid/controllers/config" + controllersconfig "github.com/gardener/etcd-druid/controllers/config" "github.com/gardener/gardener/pkg/utils/test" . "github.com/onsi/ginkgo" @@ -110,8 +110,11 @@ var _ = BeforeSuite(func(done Done) { err = er.SetupWithManager(mgr, 1, true) Expect(err).NotTo(HaveOccurred()) - custodian := NewEtcdCustodian(mgr, config.EtcdCustodianController{ - EtcdStaleMemberThreshold: 1 * time.Minute, + custodian := NewEtcdCustodian(mgr, controllersconfig.EtcdCustodianController{ + EtcdMember: controllersconfig.EtcdMemberConfig{ + EtcdMemberUnknownThreshold: 1 * time.Minute, + EtcdMemberNotReadyThreshold: 1 * time.Minute, + }, }) err = custodian.SetupWithManager(mgrCtx, mgr, 1) diff --git a/controllers/etcd_custodian_controller.go b/controllers/etcd_custodian_controller.go index 35b5099c1..cff0ee78f 100644 --- a/controllers/etcd_custodian_controller.go +++ b/controllers/etcd_custodian_controller.go @@ -37,7 +37,7 @@ import ( "sigs.k8s.io/controller-runtime/pkg/source" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" - "github.com/gardener/etcd-druid/controllers/config" + controllersconfig "github.com/gardener/etcd-druid/controllers/config" "github.com/gardener/etcd-druid/pkg/health/status" druidmapper "github.com/gardener/etcd-druid/pkg/mapper" druidpredicates "github.com/gardener/etcd-druid/pkg/predicate" @@ -48,11 +48,11 @@ type EtcdCustodian struct { client.Client Scheme *runtime.Scheme logger logr.Logger - config config.EtcdCustodianController + config controllersconfig.EtcdCustodianController } // NewEtcdCustodian creates a new EtcdCustodian object -func NewEtcdCustodian(mgr manager.Manager, config config.EtcdCustodianController) *EtcdCustodian { +func NewEtcdCustodian(mgr manager.Manager, config controllersconfig.EtcdCustodianController) *EtcdCustodian { return &EtcdCustodian{ Client: mgr.GetClient(), Scheme: mgr.GetScheme(), @@ -94,8 +94,8 @@ func (ec *EtcdCustodian) Reconcile(ctx context.Context, req ctrl.Request) (ctrl. return ctrl.Result{}, err } - statusCheck := status.NewChecker(ec.config) - if err := statusCheck.Check(ctx, &etcd.Status); err != nil { + statusCheck := status.NewChecker(ec.Client, ec.config) + if err := statusCheck.Check(ctx, etcd); err != nil { logger.Error(err, "Error executing status checks") return ctrl.Result{}, err } diff --git a/main.go b/main.go index 352eff0cb..910f0c010 100644 --- a/main.go +++ b/main.go @@ -22,7 +22,7 @@ import ( druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" "github.com/gardener/etcd-druid/controllers" - "github.com/gardener/etcd-druid/controllers/config" + controllersconfig "github.com/gardener/etcd-druid/controllers/config" "k8s.io/apimachinery/pkg/runtime" utilruntime "k8s.io/apimachinery/pkg/util/runtime" @@ -57,7 +57,8 @@ func main() { custodianSyncPeriod time.Duration ignoreOperationAnnotation bool - etcdStaleMemberThreshold time.Duration + etcdMemberUnknownThreshold time.Duration + etcdMemberNotReadyThreshold time.Duration // TODO: migrate default to `leases` in one of the next releases defaultLeaderElectionResourceLock = resourcelock.ConfigMapsLeasesResourceLock @@ -75,7 +76,8 @@ func main() { flag.StringVar(&leaderElectionResourceLock, "leader-election-resource-lock", defaultLeaderElectionResourceLock, "Which resource type to use for leader election. "+ "Supported options are 'endpoints', 'configmaps', 'leases', 'endpointsleases' and 'configmapsleases'.") flag.BoolVar(&ignoreOperationAnnotation, "ignore-operation-annotation", true, "Ignore the operation annotation or not.") - flag.DurationVar(&etcdStaleMemberThreshold, "etcd-member-threshold", 1*time.Minute, "Threshold after which an etcd member status is considered unknown if no heartbeat happened.") + flag.DurationVar(&etcdMemberUnknownThreshold, "etcd-member-unknown-threshold", 60*time.Second, "Threshold after which an etcd member status is considered unknown if no heartbeat happened.") + flag.DurationVar(&etcdMemberNotReadyThreshold, "etcd-member-notready-threshold", 5*time.Minute, "Threshold after which an etcd member is considered not ready if the status was unknown before.") flag.Parse() @@ -107,9 +109,12 @@ func main() { os.Exit(1) } - custodian := controllers.NewEtcdCustodian(mgr, config.EtcdCustodianController{ - EtcdStaleMemberThreshold: etcdStaleMemberThreshold, - SyncPeriod: custodianSyncPeriod, + custodian := controllers.NewEtcdCustodian(mgr, controllersconfig.EtcdCustodianController{ + EtcdMember: controllersconfig.EtcdMemberConfig{ + EtcdMemberUnknownThreshold: etcdMemberUnknownThreshold, + EtcdMemberNotReadyThreshold: etcdMemberNotReadyThreshold, + }, + SyncPeriod: custodianSyncPeriod, }) if err := custodian.SetupWithManager(ctx, mgr, custodianWorkers); err != nil { diff --git a/pkg/health/etcdmember/check_ready.go b/pkg/health/etcdmember/check_ready.go index 615a6daa7..42d19701e 100644 --- a/pkg/health/etcdmember/check_ready.go +++ b/pkg/health/etcdmember/check_ready.go @@ -15,42 +15,83 @@ package etcdmember import ( + "context" "time" - "github.com/gardener/etcd-druid/controllers/config" + kutil "github.com/gardener/gardener/pkg/utils/kubernetes" + corev1 "k8s.io/api/core/v1" + apierrors "k8s.io/apimachinery/pkg/api/errors" + "sigs.k8s.io/controller-runtime/pkg/client" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + controllersconfig "github.com/gardener/etcd-druid/controllers/config" ) type readyCheck struct { - etcdStaleMemberThreshold time.Duration + memberConfig controllersconfig.EtcdMemberConfig + cl client.Client } // TimeNow is the function used by this check to get the current time. var TimeNow = time.Now -func (r *readyCheck) Check(status druidv1alpha1.EtcdStatus) []Result { +func (r *readyCheck) Check(ctx context.Context, etcd druidv1alpha1.Etcd) []Result { var ( results []Result - threshold = TimeNow().UTC().Add(-1 * r.etcdStaleMemberThreshold) + checkTime = TimeNow().UTC() ) - for _, etcd := range status.Members { - if etcd.LastUpdateTime.Time.Before(threshold) { + for _, member := range etcd.Status.Members { + // Check if status must be changed from Unknown to NotReady. + if member.Status == druidv1alpha1.EtcdMemeberStatusUnknown && + member.LastTransitionTime.Time.Add(r.memberConfig.EtcdMemberNotReadyThreshold).Before(checkTime) { results = append(results, &result{ - id: etcd.ID, - status: druidv1alpha1.EtcdMemeberStatusUnknown, - reason: "UnknownMemberStatus", + id: member.ID, + status: druidv1alpha1.EtcdMemeberStatusNotReady, + reason: "UnkownStateTimeout", }) + continue } + + // Skip if status is not already Unknown and LastUpdateTime is within grace period. + if !member.LastUpdateTime.Time.Add(r.memberConfig.EtcdMemberUnknownThreshold).Before(checkTime) { + continue + } + + // If pod is not running or cannot be found then we deduce that the status is NotReady. + ready, err := r.checkPodIsRunning(ctx, etcd.Namespace, member) + if (err == nil && !ready) || apierrors.IsNotFound(err) { + results = append(results, &result{ + id: member.ID, + status: druidv1alpha1.EtcdMemeberStatusNotReady, + reason: "PodNotRunning", + }) + continue + } + + // For every other reason the status is Unknown. + results = append(results, &result{ + id: member.ID, + status: druidv1alpha1.EtcdMemeberStatusUnknown, + reason: "UnknownMemberStatus", + }) } return results } +func (r *readyCheck) checkPodIsRunning(ctx context.Context, namespace string, member druidv1alpha1.EtcdMemberStatus) (bool, error) { + pod := &corev1.Pod{} + if err := r.cl.Get(ctx, kutil.Key(namespace, member.PodRef.Name), pod); err != nil { + return false, err + } + return pod.Status.Phase == corev1.PodRunning, nil +} + // ReadyCheck returns a check for the "Ready" condition. -func ReadyCheck(config config.EtcdCustodianController) Checker { +func ReadyCheck(cl client.Client, config controllersconfig.EtcdCustodianController) Checker { return &readyCheck{ - etcdStaleMemberThreshold: config.EtcdStaleMemberThreshold, + cl: cl, + memberConfig: config.EtcdMember, } } diff --git a/pkg/health/etcdmember/check_ready_test.go b/pkg/health/etcdmember/check_ready_test.go index d05cfae35..d50995cf9 100644 --- a/pkg/health/etcdmember/check_ready_test.go +++ b/pkg/health/etcdmember/check_ready_test.go @@ -15,101 +15,274 @@ package etcdmember_test import ( + "context" + "errors" "time" + kutil "github.com/gardener/gardener/pkg/utils/kubernetes" "github.com/gardener/gardener/pkg/utils/test" + "github.com/golang/mock/gomock" . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" + corev1 "k8s.io/api/core/v1" + apierrors "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "sigs.k8s.io/controller-runtime/pkg/client" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" "github.com/gardener/etcd-druid/controllers/config" + controllersconfig "github.com/gardener/etcd-druid/controllers/config" . "github.com/gardener/etcd-druid/pkg/health/etcdmember" + mockclient "github.com/gardener/etcd-druid/pkg/mock/controller-runtime/client" ) var _ = Describe("ReadyCheck", func() { Describe("#Check", func() { var ( - threshold time.Duration - now time.Time - check Checker + ctx context.Context + mockCtrl *gomock.Controller + cl *mockclient.MockClient + unknownThreshold, notReadyThreshold time.Duration + now time.Time + check Checker ) BeforeEach(func() { - threshold = 300 * time.Second + ctx = context.Background() + mockCtrl = gomock.NewController(GinkgoT()) + cl = mockclient.NewMockClient(mockCtrl) + unknownThreshold = 300 * time.Second + notReadyThreshold = 60 * time.Second now, _ = time.Parse(time.RFC3339, "2021-06-01T00:00:00Z") - check = ReadyCheck(config.EtcdCustodianController{ - EtcdStaleMemberThreshold: threshold, + check = ReadyCheck(cl, controllersconfig.EtcdCustodianController{ + EtcdMember: controllersconfig.EtcdMemberConfig{ + EtcdMemberUnknownThreshold: unknownThreshold, + EtcdMemberNotReadyThreshold: notReadyThreshold, + }, }) }) + AfterEach(func() { + mockCtrl.Finish() + }) + Context("when condition is outdated", func() { + var ( + podName string + etcd druidv1alpha1.Etcd + ) + + BeforeEach(func() { + podName = "etcd-main-0" + etcd = druidv1alpha1.Etcd{ + ObjectMeta: metav1.ObjectMeta{ + Name: "etcd", + Namespace: "etcd-test", + }, + Status: druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + { + Name: "member1", + ID: "1", + PodRef: corev1.LocalObjectReference{ + Name: podName, + }, + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "foo reason", + LastTransitionTime: metav1.Now(), + LastUpdateTime: metav1.NewTime(now.Add(-301 * time.Second)), + }, + { + Name: "member2", + ID: "2", + PodRef: corev1.LocalObjectReference{ + Name: "etcd-main-1", + }, + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "bar reason", + LastTransitionTime: metav1.Now(), + LastUpdateTime: metav1.NewTime(now.Add(-1 * unknownThreshold)), + }, + }, + }, + } + }) + It("should set the affected condition to UNKNOWN", func() { defer test.WithVar(&TimeNow, func() time.Time { return now })() - status := druidv1alpha1.EtcdStatus{ - Members: []druidv1alpha1.EtcdMemberStatus{ - { - Name: "member1", - ID: "1", - Role: druidv1alpha1.EtcdRoleMember, - Status: druidv1alpha1.EtcdMemeberStatusReady, - Reason: "foo reason", - LastTransitionTime: metav1.Now(), - LastUpdateTime: metav1.NewTime(now.Add(-301 * time.Second)), - }, - { - Name: "member2", - ID: "2", - Role: druidv1alpha1.EtcdRoleMember, - Status: druidv1alpha1.EtcdMemeberStatusReady, - Reason: "bar reason", - LastTransitionTime: metav1.Now(), - LastUpdateTime: metav1.NewTime(now.Add(-1 * threshold)), - }, + + check := ReadyCheck(cl, controllersconfig.EtcdCustodianController{ + EtcdMember: controllersconfig.EtcdMemberConfig{ + EtcdMemberUnknownThreshold: unknownThreshold, }, - } + }) - check := ReadyCheck(config.EtcdCustodianController{ - EtcdStaleMemberThreshold: threshold, + cl.EXPECT().Get(ctx, kutil.Key(etcd.Namespace, podName), gomock.AssignableToTypeOf(&corev1.Pod{})).DoAndReturn( + func(_ context.Context, _ client.ObjectKey, pod *corev1.Pod) error { + *pod = corev1.Pod{ + Status: corev1.PodStatus{ + Phase: corev1.PodRunning, + }, + } + return nil + }, + ) + + results := check.Check(ctx, etcd) + + Expect(results).To(HaveLen(1)) + Expect(results[0].Status()).To(Equal(druidv1alpha1.EtcdMemeberStatusUnknown)) + Expect(results[0].ID()).To(Equal("1")) + }) + + It("should set the affected condition to UNKNOWN because Pod cannot be received", func() { + defer test.WithVar(&TimeNow, func() time.Time { + return now + })() + + check := ReadyCheck(cl, controllersconfig.EtcdCustodianController{ + EtcdMember: controllersconfig.EtcdMemberConfig{ + EtcdMemberUnknownThreshold: unknownThreshold, + }, }) - results := check.Check(status) + cl.EXPECT().Get(ctx, kutil.Key(etcd.Namespace, podName), gomock.AssignableToTypeOf(&corev1.Pod{})).DoAndReturn( + func(_ context.Context, _ client.ObjectKey, pod *corev1.Pod) error { + return errors.New("foo") + }, + ) + + results := check.Check(ctx, etcd) Expect(results).To(HaveLen(1)) Expect(results[0].Status()).To(Equal(druidv1alpha1.EtcdMemeberStatusUnknown)) Expect(results[0].ID()).To(Equal("1")) }) + + It("should set the affected condition to FAILED because Pod is not running", func() { + defer test.WithVar(&TimeNow, func() time.Time { + return now + })() + + check := ReadyCheck(cl, controllersconfig.EtcdCustodianController{ + EtcdMember: controllersconfig.EtcdMemberConfig{ + EtcdMemberUnknownThreshold: unknownThreshold, + }, + }) + + cl.EXPECT().Get(ctx, kutil.Key(etcd.Namespace, podName), gomock.AssignableToTypeOf(&corev1.Pod{})).DoAndReturn( + func(_ context.Context, _ client.ObjectKey, pod *corev1.Pod) error { + *pod = corev1.Pod{ + Status: corev1.PodStatus{ + Phase: corev1.PodFailed, + }, + } + return nil + }, + ) + + results := check.Check(ctx, etcd) + + Expect(results).To(HaveLen(1)) + Expect(results[0].Status()).To(Equal(druidv1alpha1.EtcdMemeberStatusNotReady)) + Expect(results[0].ID()).To(Equal("1")) + }) + + It("should set the affected condition to FAILED because Pod is not found", func() { + defer test.WithVar(&TimeNow, func() time.Time { + return now + })() + + check := ReadyCheck(cl, config.EtcdCustodianController{ + EtcdMember: controllersconfig.EtcdMemberConfig{ + EtcdMemberUnknownThreshold: unknownThreshold, + }, + }) + + cl.EXPECT().Get(ctx, kutil.Key(etcd.Namespace, podName), gomock.AssignableToTypeOf(&corev1.Pod{})).DoAndReturn( + func(_ context.Context, _ client.ObjectKey, pod *corev1.Pod) error { + return apierrors.NewNotFound(corev1.Resource("pods"), podName) + }, + ) + + results := check.Check(ctx, etcd) + + Expect(results).To(HaveLen(1)) + Expect(results[0].Status()).To(Equal(druidv1alpha1.EtcdMemeberStatusNotReady)) + Expect(results[0].ID()).To(Equal("1")) + }) + + It("should set the affected condition to FAILED because status was Unknown for a while", func() { + defer test.WithVar(&TimeNow, func() time.Time { + return now + })() + + latestWithinGrace := now.Add(-1 * notReadyThreshold) + + // member with unknown state within grace period + etcd.Status.Members[0].Status = druidv1alpha1.EtcdMemeberStatusUnknown + etcd.Status.Members[0].LastTransitionTime = metav1.NewTime(latestWithinGrace) + + // member with unknown state outside grace period + etcd.Status.Members[1].Status = druidv1alpha1.EtcdMemeberStatusUnknown + etcd.Status.Members[1].LastTransitionTime = metav1.NewTime(latestWithinGrace.Add(-1 * time.Second)) + + cl.EXPECT().Get(ctx, kutil.Key(etcd.Namespace, podName), gomock.AssignableToTypeOf(&corev1.Pod{})).DoAndReturn( + func(_ context.Context, _ client.ObjectKey, pod *corev1.Pod) error { + return errors.New("foo") + }, + ) + + results := check.Check(ctx, etcd) + + Expect(results).To(HaveLen(2)) + Expect(results[0].Status()).To(Equal(druidv1alpha1.EtcdMemeberStatusUnknown)) + Expect(results[0].ID()).To(Equal("1")) + Expect(results[1].Status()).To(Equal(druidv1alpha1.EtcdMemeberStatusNotReady)) + Expect(results[1].ID()).To(Equal("2")) + }) }) + Context("when condition is not outdated", func() { It("should not return any results", func() { defer test.WithVar(&TimeNow, func() time.Time { return now })() - status := druidv1alpha1.EtcdStatus{ - Members: []druidv1alpha1.EtcdMemberStatus{ - { - Name: "member1", - ID: "1", - Role: druidv1alpha1.EtcdRoleMember, - Status: druidv1alpha1.EtcdMemeberStatusReady, - Reason: "foo reason", - LastTransitionTime: metav1.Now(), - LastUpdateTime: metav1.Now(), - }, - { - Name: "member2", - ID: "2", - Role: druidv1alpha1.EtcdRoleMember, - Status: druidv1alpha1.EtcdMemeberStatusReady, - Reason: "bar reason", - LastTransitionTime: metav1.Now(), - LastUpdateTime: metav1.NewTime(now.Add(-1 * threshold)), + etcd := druidv1alpha1.Etcd{ + Status: druidv1alpha1.EtcdStatus{ + Members: []druidv1alpha1.EtcdMemberStatus{ + { + Name: "member1", + ID: "1", + PodRef: corev1.LocalObjectReference{ + Name: "etcd-main-0", + }, + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "foo reason", + LastTransitionTime: metav1.Now(), + LastUpdateTime: metav1.Now(), + }, + { + Name: "member2", + ID: "2", + PodRef: corev1.LocalObjectReference{ + Name: "etcd-main-1", + }, + Role: druidv1alpha1.EtcdRoleMember, + Status: druidv1alpha1.EtcdMemeberStatusReady, + Reason: "bar reason", + LastTransitionTime: metav1.Now(), + LastUpdateTime: metav1.NewTime(now.Add(-1 * unknownThreshold)), + }, }, }, } - results := check.Check(status) + results := check.Check(ctx, etcd) Expect(results).To(BeEmpty()) }) diff --git a/pkg/health/etcdmember/types.go b/pkg/health/etcdmember/types.go index 336afae10..c4d33abfd 100644 --- a/pkg/health/etcdmember/types.go +++ b/pkg/health/etcdmember/types.go @@ -14,11 +14,15 @@ package etcdmember -import druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" +import ( + "context" + + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" +) // Checker is an interface to check the members of a etcd cluster. type Checker interface { - Check(status druidv1alpha1.EtcdStatus) []Result + Check(ctx context.Context, etcd druidv1alpha1.Etcd) []Result } type Result interface { diff --git a/pkg/health/status/check.go b/pkg/health/status/check.go index 3ace2b4b1..914c5aede 100644 --- a/pkg/health/status/check.go +++ b/pkg/health/status/check.go @@ -20,10 +20,10 @@ import ( "time" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - - "github.com/gardener/etcd-druid/controllers/config" + "sigs.k8s.io/controller-runtime/pkg/client" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + controllersconfig "github.com/gardener/etcd-druid/controllers/config" "github.com/gardener/etcd-druid/pkg/health/condition" "github.com/gardener/etcd-druid/pkg/health/etcdmember" ) @@ -32,7 +32,7 @@ import ( type ConditionCheckFn func() condition.Checker // EtcdMemberCheckFn is a type alias for a function which returns an implementation of `Check`. -type EtcdMemberCheckFn func(config.EtcdCustodianController) etcdmember.Checker +type EtcdMemberCheckFn func(client.Client, controllersconfig.EtcdCustodianController) etcdmember.Checker // TimeNow is the function used to get the current time. var TimeNow = time.Now @@ -54,7 +54,8 @@ var ( ) type checker struct { - config config.EtcdCustodianController + cl client.Client + config controllersconfig.EtcdCustodianController conditionCheckFns []ConditionCheckFn conditionBuilderFn func() condition.Builder etcdMemberCheckFns []EtcdMemberCheckFn @@ -62,14 +63,14 @@ type checker struct { } // Check executes the status checks and mutates the passed status object with the corresponding results. -func (c *checker) Check(ctx context.Context, status *druidv1alpha1.EtcdStatus) error { +func (c *checker) Check(ctx context.Context, etcd *druidv1alpha1.Etcd) error { // First execute the etcd member checks for the status. - if err := c.executeEtcdMemberChecks(status); err != nil { + if err := c.executeEtcdMemberChecks(ctx, etcd); err != nil { return err } // Execute condition checks after the etcd member checks because we need their result here. - if err := c.executeConditionChecks(status); err != nil { + if err := c.executeConditionChecks(&etcd.Status); err != nil { return err } return nil @@ -115,27 +116,28 @@ func (c *checker) executeConditionChecks(status *druidv1alpha1.EtcdStatus) error // executeEtcdMemberChecks runs all registered etcd member checks **sequentially**. // The result of a check is passed via the `status` sub-resources to the next check. -func (c *checker) executeEtcdMemberChecks(status *druidv1alpha1.EtcdStatus) error { +func (c *checker) executeEtcdMemberChecks(ctx context.Context, etcd *druidv1alpha1.Etcd) error { // Run etcd member checks sequentially as most of them act on multiple elements. for _, newCheck := range c.etcdMemberCheckFns { - results := newCheck(c.config).Check(*status) + results := newCheck(c.cl, c.config).Check(ctx, *etcd) // Build and assign the results after each check, so that the next check // can act on the latest results. memberStatuses := c.etcdMemberBuilderFn(). WithNowFunc(func() metav1.Time { return metav1.NewTime(TimeNow()) }). - WithOldMembers(status.Members). + WithOldMembers(etcd.Status.Members). WithResults(results). Build() - status.Members = memberStatuses + etcd.Status.Members = memberStatuses } return nil } // NewChecker creates a new instance for checking the etcd status. -func NewChecker(config config.EtcdCustodianController) *checker { +func NewChecker(cl client.Client, config controllersconfig.EtcdCustodianController) *checker { return &checker{ + cl: cl, config: config, conditionCheckFns: ConditionChecks, conditionBuilderFn: NewDefaultConditionBuilder, diff --git a/pkg/health/status/check_test.go b/pkg/health/status/check_test.go index 81871b43c..48e49a3ed 100644 --- a/pkg/health/status/check_test.go +++ b/pkg/health/status/check_test.go @@ -18,26 +18,27 @@ import ( "context" "time" - "github.com/gardener/gardener/pkg/utils/test" - - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - - druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" - controllerconfig "github.com/gardener/etcd-druid/controllers/config" "github.com/gardener/etcd-druid/pkg/health/condition" "github.com/gardener/etcd-druid/pkg/health/etcdmember" + "github.com/gardener/gardener/pkg/utils/test" . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" . "github.com/onsi/gomega/gstruct" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "sigs.k8s.io/controller-runtime/pkg/client" + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + controllersconfig "github.com/gardener/etcd-druid/controllers/config" . "github.com/gardener/etcd-druid/pkg/health/status" ) var _ = Describe("Check", func() { Describe("#Check", func() { It("should correctly execute checks and fill status", func() { - config := controllerconfig.EtcdCustodianController{ - EtcdStaleMemberThreshold: 1 * time.Minute, + config := controllersconfig.EtcdCustodianController{ + EtcdMember: controllersconfig.EtcdMemberConfig{ + EtcdMemberUnknownThreshold: 1 * time.Minute, + }, } timeBefore, _ := time.Parse(time.RFC3339, "2021-06-01T00:00:00Z") timeNow := timeBefore.Add(1 * time.Hour) @@ -100,6 +101,10 @@ var _ = Describe("Check", func() { }, } + etcd := &druidv1alpha1.Etcd{ + Status: status, + } + defer test.WithVar(&ConditionChecks, []ConditionCheckFn{ func() condition.Checker { return createConditionCheck(druidv1alpha1.ConditionTypeReady, druidv1alpha1.ConditionFalse, "FailedConditionCheck", "check failed") @@ -110,18 +115,18 @@ var _ = Describe("Check", func() { })() defer test.WithVar(&EtcdMemberChecks, []EtcdMemberCheckFn{ - func(_ controllerconfig.EtcdCustodianController) etcdmember.Checker { + func(_ client.Client, _ controllersconfig.EtcdCustodianController) etcdmember.Checker { return createEtcdMemberCheck("1", druidv1alpha1.EtcdMemeberStatusUnknown, "Unknown") }, })() defer test.WithVar(&TimeNow, func() time.Time { return timeNow })() - checker := NewChecker(config) + checker := NewChecker(nil, config) - Expect(checker.Check(context.Background(), &status)).To(Succeed()) + Expect(checker.Check(context.Background(), etcd)).To(Succeed()) - Expect(status.Conditions).To(ConsistOf( + Expect(etcd.Status.Conditions).To(ConsistOf( MatchFields(IgnoreExtras, Fields{ "Type": Equal(druidv1alpha1.ConditionTypeReady), "Status": Equal(druidv1alpha1.ConditionFalse), @@ -148,7 +153,7 @@ var _ = Describe("Check", func() { }), )) - Expect(status.Members).To(ConsistOf( + Expect(etcd.Status.Members).To(ConsistOf( MatchFields(IgnoreExtras, Fields{ "ID": Equal("1"), "Name": Equal("Member1"), @@ -246,7 +251,7 @@ type etcdMemberTestChecker struct { result *etcdMemberResult } -func (t *etcdMemberTestChecker) Check(_ druidv1alpha1.EtcdStatus) []etcdmember.Result { +func (t *etcdMemberTestChecker) Check(_ context.Context, _ druidv1alpha1.Etcd) []etcdmember.Result { return []etcdmember.Result{ t.result, } diff --git a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go index 8d8a76982..b109a1423 100644 --- a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go +++ b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go @@ -297,6 +297,8 @@ type EtcdMemberStatus struct { Name string `json:"name"` // ID is the ID of the etcd member. ID string `json:"id"` + // PodRef is the reference to the Pod which hosts the etcd member. + PodRef corev1.LocalObjectReference `json:"podRef"` // Role is the role in the etcd cluster, either `Member` or `Learner`. Role EtcdRole `json:"role"` // Status of the condition, one of True, False, Unknown. diff --git a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go index c7604b824..24f8a33b5 100644 --- a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go +++ b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go @@ -268,6 +268,7 @@ func (in *EtcdList) DeepCopyObject() runtime.Object { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *EtcdMemberStatus) DeepCopyInto(out *EtcdMemberStatus) { *out = *in + out.PodRef = in.PodRef in.LastUpdateTime.DeepCopyInto(&out.LastUpdateTime) in.LastTransitionTime.DeepCopyInto(&out.LastTransitionTime) } From d55b7043dece3383a5ec4bbdef9d71cae7db14a0 Mon Sep 17 00:00:00 2001 From: Tim Usner Date: Mon, 14 Jun 2021 10:38:07 +0200 Subject: [PATCH 5/7] Remove status.podRef Remove `status.podRef` which was introduced by an earlier, unreleased commit and use `status.name` instead. --- api/v1alpha1/etcd_types.go | 4 +-- api/v1alpha1/zz_generated.deepcopy.go | 1 - .../crd/bases/druid.gardener.cloud_etcds.yaml | 13 ++------ pkg/health/etcdmember/check_ready.go | 2 +- pkg/health/etcdmember/check_ready_test.go | 30 ++++++------------- .../etcd-druid/api/v1alpha1/etcd_types.go | 4 +-- .../api/v1alpha1/zz_generated.deepcopy.go | 1 - 7 files changed, 14 insertions(+), 41 deletions(-) diff --git a/api/v1alpha1/etcd_types.go b/api/v1alpha1/etcd_types.go index b109a1423..86adeb24b 100644 --- a/api/v1alpha1/etcd_types.go +++ b/api/v1alpha1/etcd_types.go @@ -293,12 +293,10 @@ const ( // EtcdMemberStatus holds information about a etcd cluster membership. type EtcdMemberStatus struct { - // Name is the name of the etcd member. + // Name is the name of the etcd member. It is the name of the backing `Pod`. Name string `json:"name"` // ID is the ID of the etcd member. ID string `json:"id"` - // PodRef is the reference to the Pod which hosts the etcd member. - PodRef corev1.LocalObjectReference `json:"podRef"` // Role is the role in the etcd cluster, either `Member` or `Learner`. Role EtcdRole `json:"role"` // Status of the condition, one of True, False, Unknown. diff --git a/api/v1alpha1/zz_generated.deepcopy.go b/api/v1alpha1/zz_generated.deepcopy.go index 24f8a33b5..c7604b824 100644 --- a/api/v1alpha1/zz_generated.deepcopy.go +++ b/api/v1alpha1/zz_generated.deepcopy.go @@ -268,7 +268,6 @@ func (in *EtcdList) DeepCopyObject() runtime.Object { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *EtcdMemberStatus) DeepCopyInto(out *EtcdMemberStatus) { *out = *in - out.PodRef = in.PodRef in.LastUpdateTime.DeepCopyInto(&out.LastUpdateTime) in.LastTransitionTime.DeepCopyInto(&out.LastTransitionTime) } diff --git a/config/crd/bases/druid.gardener.cloud_etcds.yaml b/config/crd/bases/druid.gardener.cloud_etcds.yaml index f6602fdfc..f47801ef2 100644 --- a/config/crd/bases/druid.gardener.cloud_etcds.yaml +++ b/config/crd/bases/druid.gardener.cloud_etcds.yaml @@ -548,17 +548,9 @@ spec: format: date-time type: string name: - description: Name is the name of the etcd member. + description: Name is the name of the etcd member. It is the + name of the backing `Pod`. type: string - podRef: - description: PodRef is the reference to the Pod which hosts - the etcd member. - properties: - name: - description: 'Name of the referent. More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - TODO: Add other useful fields. apiVersion, kind, uid?' - type: string - type: object reason: description: The reason for the condition's last transition. type: string @@ -574,7 +566,6 @@ spec: - lastTransitionTime - lastUpdateTime - name - - podRef - reason - role - status diff --git a/pkg/health/etcdmember/check_ready.go b/pkg/health/etcdmember/check_ready.go index 42d19701e..c23a163d0 100644 --- a/pkg/health/etcdmember/check_ready.go +++ b/pkg/health/etcdmember/check_ready.go @@ -82,7 +82,7 @@ func (r *readyCheck) Check(ctx context.Context, etcd druidv1alpha1.Etcd) []Resul func (r *readyCheck) checkPodIsRunning(ctx context.Context, namespace string, member druidv1alpha1.EtcdMemberStatus) (bool, error) { pod := &corev1.Pod{} - if err := r.cl.Get(ctx, kutil.Key(namespace, member.PodRef.Name), pod); err != nil { + if err := r.cl.Get(ctx, kutil.Key(namespace, member.Name), pod); err != nil { return false, err } return pod.Status.Phase == corev1.PodRunning, nil diff --git a/pkg/health/etcdmember/check_ready_test.go b/pkg/health/etcdmember/check_ready_test.go index d50995cf9..a7129c020 100644 --- a/pkg/health/etcdmember/check_ready_test.go +++ b/pkg/health/etcdmember/check_ready_test.go @@ -72,7 +72,7 @@ var _ = Describe("ReadyCheck", func() { ) BeforeEach(func() { - podName = "etcd-main-0" + podName = "member1" etcd = druidv1alpha1.Etcd{ ObjectMeta: metav1.ObjectMeta{ Name: "etcd", @@ -81,11 +81,8 @@ var _ = Describe("ReadyCheck", func() { Status: druidv1alpha1.EtcdStatus{ Members: []druidv1alpha1.EtcdMemberStatus{ { - Name: "member1", - ID: "1", - PodRef: corev1.LocalObjectReference{ - Name: podName, - }, + Name: podName, + ID: "1", Role: druidv1alpha1.EtcdRoleMember, Status: druidv1alpha1.EtcdMemeberStatusReady, Reason: "foo reason", @@ -93,11 +90,8 @@ var _ = Describe("ReadyCheck", func() { LastUpdateTime: metav1.NewTime(now.Add(-301 * time.Second)), }, { - Name: "member2", - ID: "2", - PodRef: corev1.LocalObjectReference{ - Name: "etcd-main-1", - }, + Name: "member2", + ID: "2", Role: druidv1alpha1.EtcdRoleMember, Status: druidv1alpha1.EtcdMemeberStatusReady, Reason: "bar reason", @@ -255,11 +249,8 @@ var _ = Describe("ReadyCheck", func() { Status: druidv1alpha1.EtcdStatus{ Members: []druidv1alpha1.EtcdMemberStatus{ { - Name: "member1", - ID: "1", - PodRef: corev1.LocalObjectReference{ - Name: "etcd-main-0", - }, + Name: "member1", + ID: "1", Role: druidv1alpha1.EtcdRoleMember, Status: druidv1alpha1.EtcdMemeberStatusReady, Reason: "foo reason", @@ -267,11 +258,8 @@ var _ = Describe("ReadyCheck", func() { LastUpdateTime: metav1.Now(), }, { - Name: "member2", - ID: "2", - PodRef: corev1.LocalObjectReference{ - Name: "etcd-main-1", - }, + Name: "member2", + ID: "2", Role: druidv1alpha1.EtcdRoleMember, Status: druidv1alpha1.EtcdMemeberStatusReady, Reason: "bar reason", diff --git a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go index b109a1423..86adeb24b 100644 --- a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go +++ b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go @@ -293,12 +293,10 @@ const ( // EtcdMemberStatus holds information about a etcd cluster membership. type EtcdMemberStatus struct { - // Name is the name of the etcd member. + // Name is the name of the etcd member. It is the name of the backing `Pod`. Name string `json:"name"` // ID is the ID of the etcd member. ID string `json:"id"` - // PodRef is the reference to the Pod which hosts the etcd member. - PodRef corev1.LocalObjectReference `json:"podRef"` // Role is the role in the etcd cluster, either `Member` or `Learner`. Role EtcdRole `json:"role"` // Status of the condition, one of True, False, Unknown. diff --git a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go index 24f8a33b5..c7604b824 100644 --- a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go +++ b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go @@ -268,7 +268,6 @@ func (in *EtcdList) DeepCopyObject() runtime.Object { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *EtcdMemberStatus) DeepCopyInto(out *EtcdMemberStatus) { *out = *in - out.PodRef = in.PodRef in.LastUpdateTime.DeepCopyInto(&out.LastUpdateTime) in.LastTransitionTime.DeepCopyInto(&out.LastTransitionTime) } From 63273d9a1ecc1e6601157921ac4ebfcdfe8117a0 Mon Sep 17 00:00:00 2001 From: Tim Usner Date: Mon, 14 Jun 2021 19:46:21 +0200 Subject: [PATCH 6/7] Add ClusterSize --- api/v1alpha1/etcd_types.go | 3 + api/v1alpha1/zz_generated.deepcopy.go | 5 ++ .../crd/bases/druid.gardener.cloud_etcds.yaml | 4 + controllers/etcd_controller.go | 89 ++++++++++++------- controllers/etcd_controller_test.go | 6 ++ controllers/etcd_custodian_controller.go | 58 ++++++------ pkg/health/condition/check_ready.go | 14 +-- pkg/health/condition/check_ready_test.go | 6 +- pkg/utils/miscellaneous.go | 8 ++ .../etcd-druid/api/v1alpha1/etcd_types.go | 3 + .../api/v1alpha1/zz_generated.deepcopy.go | 5 ++ 11 files changed, 137 insertions(+), 64 deletions(-) diff --git a/api/v1alpha1/etcd_types.go b/api/v1alpha1/etcd_types.go index 86adeb24b..89825fe82 100644 --- a/api/v1alpha1/etcd_types.go +++ b/api/v1alpha1/etcd_types.go @@ -325,6 +325,9 @@ type EtcdStatus struct { // LastError represents the last occurred error. // +optional LastError *string `json:"lastError,omitempty"` + // Cluster size is the size of the etcd cluster. + // +optional + ClusterSize *int32 `json:"clusterSize,omitempty"` // CurrentReplicas is the current replica count for the etcd cluster. // +optional CurrentReplicas int32 `json:"currentReplicas,omitempty"` diff --git a/api/v1alpha1/zz_generated.deepcopy.go b/api/v1alpha1/zz_generated.deepcopy.go index c7604b824..3f064c656 100644 --- a/api/v1alpha1/zz_generated.deepcopy.go +++ b/api/v1alpha1/zz_generated.deepcopy.go @@ -369,6 +369,11 @@ func (in *EtcdStatus) DeepCopyInto(out *EtcdStatus) { *out = new(string) **out = **in } + if in.ClusterSize != nil { + in, out := &in.ClusterSize, &out.ClusterSize + *out = new(int32) + **out = **in + } if in.Ready != nil { in, out := &in.Ready, &out.Ready *out = new(bool) diff --git a/config/crd/bases/druid.gardener.cloud_etcds.yaml b/config/crd/bases/druid.gardener.cloud_etcds.yaml index f47801ef2..ae40f5621 100644 --- a/config/crd/bases/druid.gardener.cloud_etcds.yaml +++ b/config/crd/bases/druid.gardener.cloud_etcds.yaml @@ -423,6 +423,10 @@ spec: status: description: EtcdStatus defines the observed state of Etcd. properties: + clusterSize: + description: Cluster size is the size of the etcd cluster. + format: int32 + type: integer conditions: description: Conditions represents the latest available observations of an etcd's current state. diff --git a/controllers/etcd_controller.go b/controllers/etcd_controller.go index 2f2df8dce..3136e3a7e 100644 --- a/controllers/etcd_controller.go +++ b/controllers/etcd_controller.go @@ -51,6 +51,7 @@ import ( "k8s.io/apimachinery/pkg/util/yaml" "k8s.io/client-go/rest" "k8s.io/client-go/util/retry" + "k8s.io/utils/pointer" ctrl "sigs.k8s.io/controller-runtime" "sigs.k8s.io/controller-runtime/pkg/client" "sigs.k8s.io/controller-runtime/pkg/controller" @@ -209,7 +210,7 @@ func (r *EtcdReconciler) reconcile(ctx context.Context, etcd *druidv1alpha1.Etcd finalizers.Insert(FinalizerName) etcd.Finalizers = finalizers.UnsortedList() if err := r.Update(ctx, etcd); err != nil { - if err := r.updateEtcdErrorStatus(ctx, etcd, nil, err); err != nil { + if err := r.updateEtcdErrorStatus(ctx, noOp, etcd, nil, err); err != nil { return ctrl.Result{ Requeue: true, }, err @@ -220,7 +221,7 @@ func (r *EtcdReconciler) reconcile(ctx context.Context, etcd *druidv1alpha1.Etcd } } if err := r.addFinalizersToDependantSecrets(ctx, logger, etcd); err != nil { - if err := r.updateEtcdErrorStatus(ctx, etcd, nil, err); err != nil { + if err := r.updateEtcdErrorStatus(ctx, noOp, etcd, nil, err); err != nil { return ctrl.Result{ Requeue: true, }, err @@ -235,9 +236,9 @@ func (r *EtcdReconciler) reconcile(ctx context.Context, etcd *druidv1alpha1.Etcd Requeue: true, }, err } - svc, ss, err := r.reconcileEtcd(ctx, logger, etcd) + op, svc, sts, err := r.reconcileEtcd(ctx, logger, etcd) if err != nil { - if err := r.updateEtcdErrorStatus(ctx, etcd, ss, err); err != nil { + if err := r.updateEtcdErrorStatus(ctx, op, etcd, sts, err); err != nil { logger.Error(err, "Error during reconciling ETCD") return ctrl.Result{ Requeue: true, @@ -247,8 +248,7 @@ func (r *EtcdReconciler) reconcile(ctx context.Context, etcd *druidv1alpha1.Etcd Requeue: true, }, err } - - if err := r.updateEtcdStatus(ctx, etcd, svc, ss); err != nil { + if err := r.updateEtcdStatus(ctx, op, etcd, svc, sts); err != nil { return ctrl.Result{ Requeue: true, }, err @@ -264,7 +264,7 @@ func (r *EtcdReconciler) delete(ctx context.Context, etcd *druidv1alpha1.Etcd) ( logger.Info("Starting operation") if err := r.removeDependantStatefulset(ctx, logger, etcd); err != nil { - if err := r.updateEtcdErrorStatus(ctx, etcd, nil, err); err != nil { + if err := r.updateEtcdErrorStatus(ctx, deleteOp, etcd, nil, err); err != nil { return ctrl.Result{ Requeue: true, }, err @@ -275,7 +275,7 @@ func (r *EtcdReconciler) delete(ctx context.Context, etcd *druidv1alpha1.Etcd) ( } if err := r.removeFinalizersToDependantSecrets(ctx, logger, etcd); err != nil { - if err := r.updateEtcdErrorStatus(ctx, etcd, nil, err); err != nil { + if err := r.updateEtcdErrorStatus(ctx, deleteOp, etcd, nil, err); err != nil { return ctrl.Result{ Requeue: true, }, err @@ -557,7 +557,16 @@ func (r *EtcdReconciler) getConfigMapFromEtcd(etcd *druidv1alpha1.Etcd, rendered return decoded, nil } -func (r *EtcdReconciler) reconcileStatefulSet(ctx context.Context, logger logr.Logger, etcd *druidv1alpha1.Etcd, values map[string]interface{}) (*appsv1.StatefulSet, error) { +type operationResult string + +const ( + bootstrapOp operationResult = "bootstrap" + reconcileOp operationResult = "reconcile" + deleteOp operationResult = "delete" + noOp operationResult = "none" +) + +func (r *EtcdReconciler) reconcileStatefulSet(ctx context.Context, logger logr.Logger, etcd *druidv1alpha1.Etcd, values map[string]interface{}) (operationResult, *appsv1.StatefulSet, error) { logger.Info("Reconciling etcd statefulset") // If any adoptions are attempted, we should first recheck for deletion with @@ -582,19 +591,19 @@ func (r *EtcdReconciler) reconcileStatefulSet(ctx context.Context, logger logr.L selector, err := metav1.LabelSelectorAsSelector(etcd.Spec.Selector) if err != nil { logger.Error(err, "Error converting etcd selector to selector") - return nil, err + return noOp, nil, err } dm := NewEtcdDruidRefManager(r.Client, r.Scheme, etcd, selector, etcdGVK, canAdoptFunc) statefulSets, err := dm.FetchStatefulSet(ctx, etcd) if err != nil { logger.Error(err, "Error while fetching StatefulSet") - return nil, err + return noOp, nil, err } logger.Info("Claiming existing etcd StatefulSet") claimedStatefulSets, err := dm.ClaimStatefulsets(ctx, statefulSets) if err != nil { - return nil, err + return noOp, nil, err } if len(claimedStatefulSets) > 0 { @@ -612,41 +621,42 @@ func (r *EtcdReconciler) reconcileStatefulSet(ctx context.Context, logger logr.L // TODO: (timuthy) Check if this is really needed. sts := &appsv1.StatefulSet{} if err := r.Get(ctx, types.NamespacedName{Name: claimedStatefulSets[0].Name, Namespace: claimedStatefulSets[0].Namespace}, sts); err != nil { - return nil, err + return noOp, nil, err } // Statefulset is claimed by for this etcd. Just sync the specs if sts, err = r.syncStatefulSetSpec(ctx, logger, sts, etcd, values); err != nil { - return nil, err + return noOp, nil, err } // restart etcd pods in crashloop backoff selector, err := metav1.LabelSelectorAsSelector(sts.Spec.Selector) if err != nil { logger.Error(err, "error converting StatefulSet selector to selector") - return nil, err + return noOp, nil, err } podList := &v1.PodList{} if err := r.List(ctx, podList, client.InNamespace(etcd.Namespace), client.MatchingLabelsSelector{Selector: selector}); err != nil { - return nil, err + return noOp, nil, err } for _, pod := range podList.Items { if utils.IsPodInCrashloopBackoff(pod.Status) { if err := r.Delete(ctx, &pod); err != nil { logger.Error(err, fmt.Sprintf("error deleting etcd pod in crashloop: %s/%s", pod.Namespace, pod.Name)) - return nil, err + return noOp, nil, err } } } - return r.waitUntilStatefulSetReady(ctx, logger, etcd, sts) + sts, err = r.waitUntilStatefulSetReady(ctx, logger, etcd, sts) + return reconcileOp, sts, err } // Required statefulset doesn't exist. Create new sts, err := r.getStatefulSetFromEtcd(etcd, values) if err != nil { - return nil, err + return noOp, nil, err } err = r.Create(ctx, sts) @@ -658,10 +668,11 @@ func (r *EtcdReconciler) reconcileStatefulSet(ctx context.Context, logger logr.L err = nil } if err != nil { - return nil, err + return noOp, nil, err } - return r.waitUntilStatefulSetReady(ctx, logger, etcd, sts) + sts, err = r.waitUntilStatefulSetReady(ctx, logger, etcd, sts) + return bootstrapOp, sts, err } func getContainerMapFromPodTemplateSpec(spec v1.PodSpec) map[string]v1.Container { @@ -758,21 +769,20 @@ func (r *EtcdReconciler) getStatefulSetFromEtcd(etcd *druidv1alpha1.Etcd, values return decoded, nil } -func (r *EtcdReconciler) reconcileEtcd(ctx context.Context, logger logr.Logger, etcd *druidv1alpha1.Etcd) (*corev1.Service, *appsv1.StatefulSet, error) { - +func (r *EtcdReconciler) reconcileEtcd(ctx context.Context, logger logr.Logger, etcd *druidv1alpha1.Etcd) (operationResult, *corev1.Service, *appsv1.StatefulSet, error) { values, err := r.getMapFromEtcd(etcd) if err != nil { - return nil, nil, err + return noOp, nil, nil, err } chartPath := getChartPath() renderedChart, err := r.chartApplier.Render(chartPath, etcd.Name, etcd.Namespace, values) if err != nil { - return nil, nil, err + return noOp, nil, nil, err } svc, err := r.reconcileServices(ctx, logger, etcd, renderedChart) if err != nil { - return nil, nil, err + return noOp, nil, nil, err } if svc != nil { values["serviceName"] = svc.Name @@ -780,18 +790,18 @@ func (r *EtcdReconciler) reconcileEtcd(ctx context.Context, logger logr.Logger, cm, err := r.reconcileConfigMaps(ctx, logger, etcd, renderedChart) if err != nil { - return nil, nil, err + return noOp, nil, nil, err } if cm != nil { values["configMapName"] = cm.Name } - ss, err := r.reconcileStatefulSet(ctx, logger, etcd, values) + op, sts, err := r.reconcileStatefulSet(ctx, logger, etcd, values) if err != nil { - return nil, nil, err + return noOp, nil, nil, err } - return svc, ss, nil + return op, svc, sts, nil } func checkEtcdOwnerReference(refs []metav1.OwnerReference, etcd *druidv1alpha1.Etcd) bool { @@ -1118,7 +1128,12 @@ func canDeleteStatefulset(sts *appsv1.StatefulSet, etcd *druidv1alpha1.Etcd) boo } -func (r *EtcdReconciler) updateEtcdErrorStatus(ctx context.Context, etcd *druidv1alpha1.Etcd, sts *appsv1.StatefulSet, lastError error) error { +func bootstrapReset(etcd *druidv1alpha1.Etcd) { + etcd.Status.Members = nil + etcd.Status.ClusterSize = pointer.Int32Ptr(int32(etcd.Spec.Replicas)) +} + +func (r *EtcdReconciler) updateEtcdErrorStatus(ctx context.Context, op operationResult, etcd *druidv1alpha1.Etcd, sts *appsv1.StatefulSet, lastError error) error { err := kutil.TryUpdateStatus(ctx, retry.DefaultBackoff, r.Client, etcd, func() error { lastErrStr := fmt.Sprintf("%v", lastError) etcd.Status.LastError = &lastErrStr @@ -1126,6 +1141,11 @@ func (r *EtcdReconciler) updateEtcdErrorStatus(ctx context.Context, etcd *druidv if sts != nil { ready := CheckStatefulSet(etcd, sts) == nil etcd.Status.Ready = &ready + + if op == bootstrapOp { + // Reset members in bootstrap phase to ensure depending conditions can be calculated correctly. + bootstrapReset(etcd) + } } return nil }) @@ -1136,7 +1156,7 @@ func (r *EtcdReconciler) updateEtcdErrorStatus(ctx context.Context, etcd *druidv return r.removeOperationAnnotation(ctx, etcd) } -func (r *EtcdReconciler) updateEtcdStatus(ctx context.Context, etcd *druidv1alpha1.Etcd, svc *corev1.Service, sts *appsv1.StatefulSet) error { +func (r *EtcdReconciler) updateEtcdStatus(ctx context.Context, op operationResult, etcd *druidv1alpha1.Etcd, svc *corev1.Service, sts *appsv1.StatefulSet) error { err := kutil.TryUpdateStatus(ctx, retry.DefaultBackoff, r.Client, etcd, func() error { ready := CheckStatefulSet(etcd, sts) == nil etcd.Status.Ready = &ready @@ -1144,6 +1164,11 @@ func (r *EtcdReconciler) updateEtcdStatus(ctx context.Context, etcd *druidv1alph etcd.Status.ServiceName = &svcName etcd.Status.LastError = nil etcd.Status.ObservedGeneration = &etcd.Generation + + if op == bootstrapOp { + // Reset members in bootstrap phase to ensure depending conditions can be calculated correctly. + bootstrapReset(etcd) + } return nil }) diff --git a/controllers/etcd_controller_test.go b/controllers/etcd_controller_test.go index e9dd8e360..5aefc4cb9 100644 --- a/controllers/etcd_controller_test.go +++ b/controllers/etcd_controller_test.go @@ -191,6 +191,12 @@ var _ = Describe("Druid", func() { err = c.Status().Update(context.TODO(), sts) Eventually(func() error { return statefulsetIsCorrectlyReconciled(c, instance, sts) }, timeout, pollingInterval).Should(BeNil()) Expect(err).NotTo(HaveOccurred()) + Eventually(func() (*int32, error) { + if err := c.Get(context.TODO(), client.ObjectKeyFromObject(instance), instance); err != nil { + return nil, err + } + return instance.Status.ClusterSize, nil + }, timeout, pollingInterval).Should(Equal(pointer.Int32Ptr(int32(instance.Spec.Replicas)))) }) It("should create and adopt statefulset and printing events", func() { // Check StatefulSet requirements diff --git a/controllers/etcd_custodian_controller.go b/controllers/etcd_custodian_controller.go index cff0ee78f..5be550871 100644 --- a/controllers/etcd_custodian_controller.go +++ b/controllers/etcd_custodian_controller.go @@ -110,7 +110,9 @@ func (ec *EtcdCustodian) Reconcile(ctx context.Context, req ctrl.Request) (ctrl. // Requeue if we found more than one or no StatefulSet. // The Etcd controller needs to decide what to do in such situations. if len(stsList.Items) != 1 { - ec.updateEtcdStatusWithNoSts(ctx, logger, etcd) + if err := ec.updateEtcdStatus(ctx, logger, etcd, nil); err != nil { + logger.Error(err, "Error while updating ETCD status when no statefulset found") + } return ctrl.Result{ RequeueAfter: 5 * time.Second, }, nil @@ -131,43 +133,47 @@ func (ec *EtcdCustodian) updateEtcdStatus(ctx context.Context, logger logr.Logge ) return kutil.TryUpdateStatus(ctx, retry.DefaultBackoff, ec.Client, etcd, func() error { - etcd.Status.Etcd = &druidv1alpha1.CrossVersionObjectReference{ - APIVersion: sts.APIVersion, - Kind: sts.Kind, - Name: sts.Name, - } - etcd.Status.Conditions = conditions etcd.Status.Members = members - ready := CheckStatefulSet(etcd, sts) == nil - - // To be changed once we have multiple replicas. - etcd.Status.CurrentReplicas = sts.Status.CurrentReplicas - etcd.Status.ReadyReplicas = sts.Status.ReadyReplicas - etcd.Status.UpdatedReplicas = sts.Status.UpdatedReplicas - etcd.Status.Ready = &ready - logger.Info(fmt.Sprintf("ETCD status updated for statefulset current replicas: %v, ready replicas: %v, updated replicas: %v", sts.Status.CurrentReplicas, sts.Status.ReadyReplicas, sts.Status.UpdatedReplicas)) - return nil - }) -} + // Bootstrap is a special case which is handled by the etcd controller. + if !inBootstrap(etcd) && len(members) != 0 { + etcd.Status.ClusterSize = pointer.Int32Ptr(int32(len(members))) + } -func (ec *EtcdCustodian) updateEtcdStatusWithNoSts(ctx context.Context, logger logr.Logger, etcd *druidv1alpha1.Etcd) { - logger.Info("Updating etcd status when no statefulset found") - conditions := etcd.Status.Conditions + if sts != nil { + etcd.Status.Etcd = &druidv1alpha1.CrossVersionObjectReference{ + APIVersion: sts.APIVersion, + Kind: sts.Kind, + Name: sts.Name, + } + + ready := CheckStatefulSet(etcd, sts) == nil + + // To be changed once we have multiple replicas. + etcd.Status.CurrentReplicas = sts.Status.CurrentReplicas + etcd.Status.ReadyReplicas = sts.Status.ReadyReplicas + etcd.Status.UpdatedReplicas = sts.Status.UpdatedReplicas + etcd.Status.Ready = &ready + logger.Info(fmt.Sprintf("ETCD status updated for statefulset current replicas: %v, ready replicas: %v, updated replicas: %v", sts.Status.CurrentReplicas, sts.Status.ReadyReplicas, sts.Status.UpdatedReplicas)) + return nil + } - if err := kutil.TryUpdateStatus(ctx, retry.DefaultBackoff, ec.Client, etcd, func() error { - etcd.Status.Conditions = conditions - // To be changed once we have multiple replicas. etcd.Status.CurrentReplicas = 0 etcd.Status.ReadyReplicas = 0 etcd.Status.UpdatedReplicas = 0 etcd.Status.Ready = pointer.BoolPtr(false) return nil - }); err != nil { - logger.Error(err, "Error while updating ETCD status when no statefulset found") + }) +} + +func inBootstrap(etcd *druidv1alpha1.Etcd) bool { + if etcd.Status.ClusterSize == nil { + return true } + return len(etcd.Status.Members) == 0 || + (len(etcd.Status.Members) < etcd.Spec.Replicas && int32(etcd.Spec.Replicas) == *etcd.Status.ClusterSize) } // SetupWithManager sets up manager with a new controller and ec as the reconcile.Reconciler diff --git a/pkg/health/condition/check_ready.go b/pkg/health/condition/check_ready.go index 70a89661f..b4332fea3 100644 --- a/pkg/health/condition/check_ready.go +++ b/pkg/health/condition/check_ready.go @@ -14,22 +14,26 @@ package condition -import druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" +import ( + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + "github.com/gardener/etcd-druid/pkg/utils" +) type readyCheck struct{} func (r *readyCheck) Check(status druidv1alpha1.EtcdStatus) Result { - if len(status.Members) == 0 { + if status.ClusterSize == nil { return &result{ conType: druidv1alpha1.ConditionTypeReady, status: druidv1alpha1.ConditionUnknown, - reason: "NoMembersInStatus", - message: "Cannot determine readiness of cluster since status has no members", + reason: "ClusterSizeUnknown", + message: "Cannot determine readiness of cluster since no cluster size has been calculated", } } var ( - quorum = len(status.Members)/2 + 1 + size = utils.Max(int(*status.ClusterSize), len(status.Members)) + quorum = size/2 + 1 readyMembers = 0 ) diff --git a/pkg/health/condition/check_ready_test.go b/pkg/health/condition/check_ready_test.go index fd40e7c4d..1ef37053b 100644 --- a/pkg/health/condition/check_ready_test.go +++ b/pkg/health/condition/check_ready_test.go @@ -17,6 +17,7 @@ package condition_test import ( . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" + "k8s.io/utils/pointer" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" . "github.com/gardener/etcd-druid/pkg/health/condition" @@ -38,6 +39,7 @@ var _ = Describe("ReadyCheck", func() { Context("when members in status", func() { It("should return that the cluster has a quorum (all members ready)", func() { status := druidv1alpha1.EtcdStatus{ + ClusterSize: pointer.Int32Ptr(3), Members: []druidv1alpha1.EtcdMemberStatus{ readyMember, readyMember, @@ -54,6 +56,7 @@ var _ = Describe("ReadyCheck", func() { It("should return that the cluster has a quorum (one member not ready)", func() { status := druidv1alpha1.EtcdStatus{ + ClusterSize: pointer.Int32Ptr(3), Members: []druidv1alpha1.EtcdMemberStatus{ readyMember, notReadyMember, @@ -70,6 +73,7 @@ var _ = Describe("ReadyCheck", func() { It("should return that the cluster has lost its quorum", func() { status := druidv1alpha1.EtcdStatus{ + ClusterSize: pointer.Int32Ptr(3), Members: []druidv1alpha1.EtcdMemberStatus{ readyMember, notReadyMember, @@ -97,7 +101,7 @@ var _ = Describe("ReadyCheck", func() { Expect(result.ConditionType()).To(Equal(druidv1alpha1.ConditionTypeReady)) Expect(result.Status()).To(Equal(druidv1alpha1.ConditionUnknown)) - Expect(result.Reason()).To(Equal("NoMembersInStatus")) + Expect(result.Reason()).To(Equal("ClusterSizeUnknown")) }) }) }) diff --git a/pkg/utils/miscellaneous.go b/pkg/utils/miscellaneous.go index ad5180976..ed8424e15 100644 --- a/pkg/utils/miscellaneous.go +++ b/pkg/utils/miscellaneous.go @@ -197,3 +197,11 @@ func isContainerInCrashLoopBackOff(containerState v1.ContainerState) bool { } return false } + +// Max returns the larger of x or y. +func Max(x, y int) int { + if y > x { + return x + } + return x +} diff --git a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go index 86adeb24b..89825fe82 100644 --- a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go +++ b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/etcd_types.go @@ -325,6 +325,9 @@ type EtcdStatus struct { // LastError represents the last occurred error. // +optional LastError *string `json:"lastError,omitempty"` + // Cluster size is the size of the etcd cluster. + // +optional + ClusterSize *int32 `json:"clusterSize,omitempty"` // CurrentReplicas is the current replica count for the etcd cluster. // +optional CurrentReplicas int32 `json:"currentReplicas,omitempty"` diff --git a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go index c7604b824..3f064c656 100644 --- a/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go +++ b/vendor/github.com/gardener/etcd-druid/api/v1alpha1/zz_generated.deepcopy.go @@ -369,6 +369,11 @@ func (in *EtcdStatus) DeepCopyInto(out *EtcdStatus) { *out = new(string) **out = **in } + if in.ClusterSize != nil { + in, out := &in.ClusterSize, &out.ClusterSize + *out = new(int32) + **out = **in + } if in.Ready != nil { in, out := &in.Ready, &out.Ready *out = new(bool) From 1e76ea41361d5a22e5e578d2ab2c93c19edcd64c Mon Sep 17 00:00:00 2001 From: Tim Usner Date: Mon, 14 Jun 2021 19:56:27 +0200 Subject: [PATCH 7/7] Check ContainersReady condition --- pkg/health/etcdmember/check_ready.go | 15 +++++++++++---- pkg/health/etcdmember/check_ready_test.go | 16 +++++++++++++--- 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/pkg/health/etcdmember/check_ready.go b/pkg/health/etcdmember/check_ready.go index c23a163d0..df075aad8 100644 --- a/pkg/health/etcdmember/check_ready.go +++ b/pkg/health/etcdmember/check_ready.go @@ -59,12 +59,12 @@ func (r *readyCheck) Check(ctx context.Context, etcd druidv1alpha1.Etcd) []Resul } // If pod is not running or cannot be found then we deduce that the status is NotReady. - ready, err := r.checkPodIsRunning(ctx, etcd.Namespace, member) + ready, err := r.checkContainersAreReady(ctx, etcd.Namespace, member) if (err == nil && !ready) || apierrors.IsNotFound(err) { results = append(results, &result{ id: member.ID, status: druidv1alpha1.EtcdMemeberStatusNotReady, - reason: "PodNotRunning", + reason: "ContainersNotReady", }) continue } @@ -80,12 +80,19 @@ func (r *readyCheck) Check(ctx context.Context, etcd druidv1alpha1.Etcd) []Resul return results } -func (r *readyCheck) checkPodIsRunning(ctx context.Context, namespace string, member druidv1alpha1.EtcdMemberStatus) (bool, error) { +func (r *readyCheck) checkContainersAreReady(ctx context.Context, namespace string, member druidv1alpha1.EtcdMemberStatus) (bool, error) { pod := &corev1.Pod{} if err := r.cl.Get(ctx, kutil.Key(namespace, member.Name), pod); err != nil { return false, err } - return pod.Status.Phase == corev1.PodRunning, nil + + for _, cond := range pod.Status.Conditions { + if cond.Type == corev1.ContainersReady { + return cond.Status == corev1.ConditionTrue, nil + } + } + + return false, nil } // ReadyCheck returns a check for the "Ready" condition. diff --git a/pkg/health/etcdmember/check_ready_test.go b/pkg/health/etcdmember/check_ready_test.go index a7129c020..75588d21e 100644 --- a/pkg/health/etcdmember/check_ready_test.go +++ b/pkg/health/etcdmember/check_ready_test.go @@ -118,7 +118,12 @@ var _ = Describe("ReadyCheck", func() { func(_ context.Context, _ client.ObjectKey, pod *corev1.Pod) error { *pod = corev1.Pod{ Status: corev1.PodStatus{ - Phase: corev1.PodRunning, + Conditions: []corev1.PodCondition{ + { + Type: corev1.ContainersReady, + Status: corev1.ConditionTrue, + }, + }, }, } return nil @@ -156,7 +161,7 @@ var _ = Describe("ReadyCheck", func() { Expect(results[0].ID()).To(Equal("1")) }) - It("should set the affected condition to FAILED because Pod is not running", func() { + It("should set the affected condition to FAILED because containers are not ready", func() { defer test.WithVar(&TimeNow, func() time.Time { return now })() @@ -171,7 +176,12 @@ var _ = Describe("ReadyCheck", func() { func(_ context.Context, _ client.ObjectKey, pod *corev1.Pod) error { *pod = corev1.Pod{ Status: corev1.PodStatus{ - Phase: corev1.PodFailed, + Conditions: []corev1.PodCondition{ + { + Type: corev1.ContainersReady, + Status: corev1.ConditionFalse, + }, + }, }, } return nil