diff --git a/apis/druid/v1alpha1/druid_types.go b/apis/druid/v1alpha1/druid_types.go index 34c68a1..e1039d7 100644 --- a/apis/druid/v1alpha1/druid_types.go +++ b/apis/druid/v1alpha1/druid_types.go @@ -160,6 +160,18 @@ type DruidSpec struct { // +kubebuilder:default:="IfNotPresent" ImagePullPolicy v1.PullPolicy `json:"imagePullPolicy,omitempty"` + // ForceRedeployToken forces a rollout when changed, even if the image tag is unchanged. + // This is primarily intended for mutable-tag redeploys driven by external automation. + // +optional + ForceRedeployToken string `json:"forceRedeployToken,omitempty"` + + // ExpectedBuildRevision is the Druid runtime build identifier that must be observed + // across live servers before a deployment lifecycle is considered complete. + // The operator verifies this against sys.servers.build_revision when available and + // falls back to sys.servers.version for Druid versions that do not expose build_revision. + // +optional + ExpectedBuildRevision string `json:"expectedBuildRevision,omitempty"` + // Env Environment variables for druid containers. // +optional Env []v1.EnvVar `json:"env,omitempty"` @@ -570,20 +582,56 @@ type DruidNodeTypeStatus struct { Reason string `json:"reason,omitempty"` } +type DeploymentLifecyclePhase string + +const ( + DeploymentLifecyclePending DeploymentLifecyclePhase = "Pending" + DeploymentLifecycleInProgress DeploymentLifecyclePhase = "InProgress" + DeploymentLifecycleSucceeded DeploymentLifecyclePhase = "Succeeded" + DeploymentLifecycleFailed DeploymentLifecyclePhase = "Failed" +) + +type DeploymentLifecycleTrigger string + +const ( + DeploymentTriggerSpecChange DeploymentLifecycleTrigger = "SpecChange" + DeploymentTriggerImageChange DeploymentLifecycleTrigger = "ImageChange" + DeploymentTriggerManualRollout DeploymentLifecycleTrigger = "ManualRollout" +) + +type DeploymentLifecycleStatus struct { + Revision string `json:"revision,omitempty"` + // +kubebuilder:validation:Enum=Pending;InProgress;Succeeded;Failed + Phase DeploymentLifecyclePhase `json:"phase,omitempty"` + Reason string `json:"reason,omitempty"` + ObservedGeneration int64 `json:"observedGeneration,omitempty"` + StartedAt *metav1.Time `json:"startedAt,omitempty"` + CompletedAt *metav1.Time `json:"completedAt,omitempty"` + + LastSuccessfulImage string `json:"lastSuccessfulImage,omitempty"` + LastSuccessfulImageInputsHash string `json:"lastSuccessfulImageInputsHash,omitempty"` + LastSuccessfulForceRedeployToken string `json:"lastSuccessfulForceRedeployToken,omitempty"` + + // +kubebuilder:validation:Enum=SpecChange;ImageChange;ManualRollout + Trigger DeploymentLifecycleTrigger `json:"trigger,omitempty"` + + ExpectedBuildRevision string `json:"expectedBuildRevision,omitempty"` + ObservedBuildRevisions []string `json:"observedBuildRevisions,omitempty"` +} + // DruidClusterStatus Defines the observed state of Druid. type DruidClusterStatus struct { - // INSERT ADDITIONAL STATUS FIELD - define observed state of cluster - // Important: Run "make" to regenerate code after modifying this file - DruidNodeStatus DruidNodeTypeStatus `json:"druidNodeStatus,omitempty"` - StatefulSets []string `json:"statefulSets,omitempty"` - Deployments []string `json:"deployments,omitempty"` - Services []string `json:"services,omitempty"` - ConfigMaps []string `json:"configMaps,omitempty"` - PodDisruptionBudgets []string `json:"podDisruptionBudgets,omitempty"` - Ingress []string `json:"ingress,omitempty"` - HPAutoScalers []string `json:"hpAutoscalers,omitempty"` - Pods []string `json:"pods,omitempty"` - PersistentVolumeClaims []string `json:"persistentVolumeClaims,omitempty"` + DeploymentLifecycle DeploymentLifecycleStatus `json:"deploymentLifecycle,omitempty"` + DruidNodeStatus DruidNodeTypeStatus `json:"druidNodeStatus,omitempty"` + StatefulSets []string `json:"statefulSets,omitempty"` + Deployments []string `json:"deployments,omitempty"` + Services []string `json:"services,omitempty"` + ConfigMaps []string `json:"configMaps,omitempty"` + PodDisruptionBudgets []string `json:"podDisruptionBudgets,omitempty"` + Ingress []string `json:"ingress,omitempty"` + HPAutoScalers []string `json:"hpAutoscalers,omitempty"` + Pods []string `json:"pods,omitempty"` + PersistentVolumeClaims []string `json:"persistentVolumeClaims,omitempty"` } // Druid is the Schema for the druids API. diff --git a/apis/druid/v1alpha1/zz_generated.deepcopy.go b/apis/druid/v1alpha1/zz_generated.deepcopy.go index 93eff59..71eef28 100644 --- a/apis/druid/v1alpha1/zz_generated.deepcopy.go +++ b/apis/druid/v1alpha1/zz_generated.deepcopy.go @@ -105,6 +105,34 @@ func (in *DeepStorageSpec) DeepCopy() *DeepStorageSpec { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *DeploymentLifecycleStatus) DeepCopyInto(out *DeploymentLifecycleStatus) { + *out = *in + if in.StartedAt != nil { + in, out := &in.StartedAt, &out.StartedAt + *out = (*in).DeepCopy() + } + if in.CompletedAt != nil { + in, out := &in.CompletedAt, &out.CompletedAt + *out = (*in).DeepCopy() + } + if in.ObservedBuildRevisions != nil { + in, out := &in.ObservedBuildRevisions, &out.ObservedBuildRevisions + *out = make([]string, len(*in)) + copy(*out, *in) + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new DeploymentLifecycleStatus. +func (in *DeploymentLifecycleStatus) DeepCopy() *DeploymentLifecycleStatus { + if in == nil { + return nil + } + out := new(DeploymentLifecycleStatus) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *Druid) DeepCopyInto(out *Druid) { *out = *in @@ -135,6 +163,7 @@ func (in *Druid) DeepCopyObject() runtime.Object { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *DruidClusterStatus) DeepCopyInto(out *DruidClusterStatus) { *out = *in + in.DeploymentLifecycle.DeepCopyInto(&out.DeploymentLifecycle) out.DruidNodeStatus = in.DruidNodeStatus if in.StatefulSets != nil { in, out := &in.StatefulSets, &out.StatefulSets diff --git a/chart/crds/druid.apache.org_druids.yaml b/chart/crds/druid.apache.org_druids.yaml index e2971b3..7d1f022 100644 --- a/chart/crds/druid.apache.org_druids.yaml +++ b/chart/crds/druid.apache.org_druids.yaml @@ -1712,6 +1712,13 @@ spec: x-kubernetes-map-type: atomic type: object type: array + expectedBuildRevision: + description: |- + ExpectedBuildRevision is the Druid runtime build identifier that must be observed + across live servers before a deployment lifecycle is considered complete. + The operator verifies this against sys.servers.build_revision when available and + falls back to sys.servers.version for Druid versions that do not expose build_revision. + type: string extraCommonConfig: description: |- ExtraCommonConfig References to ConfigMaps holding more configuration files to mount to the @@ -1768,6 +1775,11 @@ spec: issue: https://github.com/kubernetes/kubernetes/issues/67250 doc: https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/#forced-rollback type: boolean + forceRedeployToken: + description: |- + ForceRedeployToken forces a rollout when changed, even if the image tag is unchanged. + This is primarily intended for mutable-tag redeploys driven by external automation. + type: string hdfs-site.xml: description: HdfsSite Contents of `hdfs-site.xml`. type: string @@ -11851,14 +11863,52 @@ spec: items: type: string type: array + deploymentLifecycle: + properties: + completedAt: + format: date-time + type: string + expectedBuildRevision: + type: string + lastSuccessfulForceRedeployToken: + type: string + lastSuccessfulImage: + type: string + lastSuccessfulImageInputsHash: + type: string + observedBuildRevisions: + items: + type: string + type: array + observedGeneration: + format: int64 + type: integer + phase: + enum: + - Pending + - InProgress + - Succeeded + - Failed + type: string + reason: + type: string + revision: + type: string + startedAt: + format: date-time + type: string + trigger: + enum: + - SpecChange + - ImageChange + - ManualRollout + type: string + type: object deployments: items: type: string type: array druidNodeStatus: - description: |- - INSERT ADDITIONAL STATUS FIELD - define observed state of cluster - Important: Run "make" to regenerate code after modifying this file properties: druidNode: type: string diff --git a/config/crd/bases/druid.apache.org_druids.yaml b/config/crd/bases/druid.apache.org_druids.yaml index e2971b3..7d1f022 100644 --- a/config/crd/bases/druid.apache.org_druids.yaml +++ b/config/crd/bases/druid.apache.org_druids.yaml @@ -1712,6 +1712,13 @@ spec: x-kubernetes-map-type: atomic type: object type: array + expectedBuildRevision: + description: |- + ExpectedBuildRevision is the Druid runtime build identifier that must be observed + across live servers before a deployment lifecycle is considered complete. + The operator verifies this against sys.servers.build_revision when available and + falls back to sys.servers.version for Druid versions that do not expose build_revision. + type: string extraCommonConfig: description: |- ExtraCommonConfig References to ConfigMaps holding more configuration files to mount to the @@ -1768,6 +1775,11 @@ spec: issue: https://github.com/kubernetes/kubernetes/issues/67250 doc: https://kubernetes.io/docs/concepts/workloads/controllers/statefulset/#forced-rollback type: boolean + forceRedeployToken: + description: |- + ForceRedeployToken forces a rollout when changed, even if the image tag is unchanged. + This is primarily intended for mutable-tag redeploys driven by external automation. + type: string hdfs-site.xml: description: HdfsSite Contents of `hdfs-site.xml`. type: string @@ -11851,14 +11863,52 @@ spec: items: type: string type: array + deploymentLifecycle: + properties: + completedAt: + format: date-time + type: string + expectedBuildRevision: + type: string + lastSuccessfulForceRedeployToken: + type: string + lastSuccessfulImage: + type: string + lastSuccessfulImageInputsHash: + type: string + observedBuildRevisions: + items: + type: string + type: array + observedGeneration: + format: int64 + type: integer + phase: + enum: + - Pending + - InProgress + - Succeeded + - Failed + type: string + reason: + type: string + revision: + type: string + startedAt: + format: date-time + type: string + trigger: + enum: + - SpecChange + - ImageChange + - ManualRollout + type: string + type: object deployments: items: type: string type: array druidNodeStatus: - description: |- - INSERT ADDITIONAL STATUS FIELD - define observed state of cluster - Important: Run "make" to regenerate code after modifying this file properties: druidNode: type: string diff --git a/controllers/druid/druid_controller.go b/controllers/druid/druid_controller.go index b91dc14..5a79e9d 100644 --- a/controllers/druid/druid_controller.go +++ b/controllers/druid/druid_controller.go @@ -20,10 +20,11 @@ package druid import ( "context" + "errors" "os" "time" - "k8s.io/apimachinery/pkg/api/errors" + k8serrors "k8s.io/apimachinery/pkg/api/errors" "k8s.io/client-go/tools/record" "github.com/go-logr/logr" @@ -70,14 +71,16 @@ func NewDruidReconciler(mgr ctrl.Manager) *DruidReconciler { // +kubebuilder:rbac:groups=networking.k8s.io,resources=ingresses,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=storage.k8s.io,resources=storageclasses,verbs=get;list;watch -func (r *DruidReconciler) Reconcile(ctx context.Context, request reconcile.Request) (ctrl.Result, error) { +func (r *DruidReconciler) Reconcile(ctx context.Context, request reconcile.Request) (result ctrl.Result, err error) { _ = r.Log.WithValues("druid", request.NamespacedName) // Fetch the Druid instance instance := &druidv1alpha1.Druid{} - err := r.Get(ctx, request.NamespacedName, instance) + err = r.Get(ctx, request.NamespacedName, instance) if err != nil { - if errors.IsNotFound(err) { + if k8serrors.IsNotFound(err) { + defaultDeploymentLifecycleMetrics.deleteCluster(request.Namespace, request.Name) + defaultDruidRolloutMetrics.deleteCluster(request.Namespace, request.Name) // Request object not found, could have been deleted after reconcile request. // Owned objects are automatically garbage collected. For additional cleanup logic use finalizers. // Return and don't requeue @@ -87,16 +90,48 @@ func (r *DruidReconciler) Reconcile(ctx context.Context, request reconcile.Reque return ctrl.Result{}, err } + defer func() { + if metricsErr := defaultDruidRolloutMetrics.sync(ctx, r.Client, instance); metricsErr != nil { + r.Log.Error(metricsErr, "failed to sync rollout metrics", "name", instance.Name, "namespace", instance.Namespace) + } + }() + // Initialize Emit Events var emitEvent EventEmitter = EmitEventFuncs{r.Recorder} + if err := ensureDeploymentLifecycleStarted(ctx, r.Client, instance, emitEvent); err != nil { + return ctrl.Result{}, err + } + + runLifecycleStep := func(step func() error) error { + if err := step(); err != nil { + if shouldPersistDeploymentLifecycleFailure(err) { + if patchErr := markDeploymentLifecycleFailed(ctx, r.Client, instance, emitEvent, err); patchErr != nil { + return errors.Join(err, patchErr) + } + } + return err + } + return nil + } + // Deploy Druid Cluster - if err := deployDruidCluster(ctx, r.Client, instance, emitEvent); err != nil { + if err := runLifecycleStep(func() error { + return deployDruidCluster(ctx, r.Client, instance, emitEvent) + }); err != nil { return ctrl.Result{}, err } // Update Druid Dynamic Configs - if err := updateDruidDynamicConfigs(ctx, r.Client, instance, emitEvent); err != nil { + if err := runLifecycleStep(func() error { + return updateDruidDynamicConfigs(ctx, r.Client, instance, emitEvent) + }); err != nil { + return ctrl.Result{}, err + } + + if err := runLifecycleStep(func() error { + return reconcileDeploymentLifecycle(ctx, r.Client, instance, emitEvent) + }); err != nil { return ctrl.Result{}, err } diff --git a/controllers/druid/druid_controller_test.go b/controllers/druid/druid_controller_test.go index 457defa..0c1450c 100644 --- a/controllers/druid/druid_controller_test.go +++ b/controllers/druid/druid_controller_test.go @@ -25,12 +25,14 @@ import ( druidv1alpha1 "github.com/apache/druid-operator/apis/druid/v1alpha1" . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" + dto "github.com/prometheus/client_model/go" appsv1 "k8s.io/api/apps/v1" autoscalev2 "k8s.io/api/autoscaling/v2" v1 "k8s.io/api/core/v1" netv1 "k8s.io/api/networking/v1" policyv1 "k8s.io/api/policy/v1" "k8s.io/apimachinery/pkg/types" + ctrlmetrics "sigs.k8s.io/controller-runtime/pkg/metrics" ) // +kubebuilder:docs-gen:collapse=Imports @@ -128,6 +130,42 @@ var _ = Describe("Druid Operator", func() { }) + It("records lifecycle metrics during real reconcile", func() { + lifecycleDruidCR, err := readDruidClusterSpecFromFile(filePath) + Expect(err).Should(BeNil()) + + lifecycleDruidCR.Name = fmt.Sprintf("lifecycle-metrics-%d", GinkgoRandomSeed()) + baseline, err := currentGlobalMetricValue("druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": lifecycleDruidCR.Namespace, + "druid_instance": lifecycleDruidCR.Name, + "phase": "pending", + "trigger": "spec_change", + }) + Expect(err).NotTo(HaveOccurred()) + + Expect(k8sClient.Create(ctx, lifecycleDruidCR)).To(Succeed()) + + lifecycleStatus := &druidv1alpha1.Druid{} + Eventually(func() string { + err := k8sClient.Get(ctx, types.NamespacedName{Name: lifecycleDruidCR.Name, Namespace: lifecycleDruidCR.Namespace}, lifecycleStatus) + if err != nil { + return "" + } + return string(lifecycleStatus.Status.DeploymentLifecycle.Phase) + }, timeout, interval).Should(Equal(string(druidv1alpha1.DeploymentLifecycleInProgress))) + + Eventually(func() float64 { + value, metricErr := currentGlobalMetricValue("druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": lifecycleDruidCR.Namespace, + "druid_instance": lifecycleDruidCR.Name, + "phase": "pending", + "trigger": "spec_change", + }) + Expect(metricErr).NotTo(HaveOccurred()) + return value + }, timeout, interval).Should(BeNumerically(">", baseline)) + }) + It("Test broker deployment", func() { componentName := "brokers" createdDeploy := &appsv1.Deployment{} @@ -151,6 +189,7 @@ var _ = Describe("Druid Operator", func() { By("By updating broker deployment replicas") replicaCount := 2 + Expect(k8sClient.Get(ctx, types.NamespacedName{Name: druidCR.Name, Namespace: druidCR.Namespace}, druid)).To(Succeed()) if druidRep, ok := druid.Spec.Nodes[componentName]; ok { druidRep.Replicas = int32(replicaCount) druid.Spec.Nodes[componentName] = druidRep @@ -190,6 +229,7 @@ var _ = Describe("Druid Operator", func() { By(fmt.Sprintf("By updating statefulset replicas %s ", stsName)) replicaCount := 2 + Expect(k8sClient.Get(ctx, types.NamespacedName{Name: druidCR.Name, Namespace: druidCR.Namespace}, druid)).To(Succeed()) if druidRep, ok := druid.Spec.Nodes[componentName]; ok { druidRep.Replicas = int32(replicaCount) druid.Spec.Nodes[componentName] = druidRep @@ -336,6 +376,52 @@ var _ = Describe("Druid Operator", func() { }) }) +func currentGlobalMetricValue(name string, labels map[string]string) (float64, error) { + families, err := ctrlmetrics.Registry.Gather() + if err != nil { + return 0, err + } + return gatherMetricValue(families, name, labels), nil +} + +func gatherMetricValue(families []*dto.MetricFamily, name string, labels map[string]string) float64 { + for _, family := range families { + if family.GetName() != name { + continue + } + for _, metric := range family.Metric { + if !metricHasLabels(metric, labels) { + continue + } + if metric.Counter != nil { + return metric.Counter.GetValue() + } + if metric.Gauge != nil { + return metric.Gauge.GetValue() + } + } + } + return 0 +} + +func metricHasLabels(metric *dto.Metric, labels map[string]string) bool { + if len(labels) == 0 { + return true + } + matched := 0 + for _, label := range metric.Label { + expected, ok := labels[label.GetName()] + if !ok { + continue + } + if label.GetValue() != expected { + return false + } + matched++ + } + return matched == len(labels) +} + func areStringArraysEqual(a1, a2 []string) bool { if len(a1) == len(a2) { for i, v := range a1 { diff --git a/controllers/druid/druid_controller_unit_test.go b/controllers/druid/druid_controller_unit_test.go new file mode 100644 index 0000000..0060e69 --- /dev/null +++ b/controllers/druid/druid_controller_unit_test.go @@ -0,0 +1,145 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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 druid + +import ( + "context" + "errors" + "testing" + "time" + + "github.com/apache/druid-operator/apis/druid/v1alpha1" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/client-go/tools/record" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + "sigs.k8s.io/controller-runtime/pkg/reconcile" +) + +func TestReconcileMarksLifecycleFailedAndReturnsOriginalError(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "bad_key": {}, + }, + }, + } + + k8sClient := newLifecycleTestClient(t, drd) + reconciler := &DruidReconciler{ + Client: k8sClient, + Log: ctrl.Log.WithName("test").WithName("Druid"), + ReconcileWait: time.Second, + Recorder: record.NewFakeRecorder(10), + } + + result, err := reconciler.Reconcile(context.Background(), reconcile.Request{ + NamespacedName: client.ObjectKeyFromObject(drd), + }) + require.Error(t, err) + assert.Equal(t, ctrl.Result{}, result) + assert.Contains(t, err.Error(), "invalid DruidSpec") + + stored := &v1alpha1.Druid{} + require.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleFailed, stored.Status.DeploymentLifecycle.Phase) + assert.Contains(t, stored.Status.DeploymentLifecycle.Reason, "invalid DruidSpec") + assert.Equal(t, drd.Generation, stored.Status.DeploymentLifecycle.ObservedGeneration) + assert.NotNil(t, stored.Status.DeploymentLifecycle.StartedAt) + assert.NotNil(t, stored.Status.DeploymentLifecycle.CompletedAt) +} + +func TestReconcileReturnsJoinedErrorWhenFailureStatusPatchFails(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "bad_key": {}, + }, + }, + } + + k8sClient := newLifecycleTestClient(t, drd) + reconciler := &DruidReconciler{ + Client: k8sClient, + Log: ctrl.Log.WithName("test").WithName("Druid"), + ReconcileWait: time.Second, + Recorder: record.NewFakeRecorder(10), + } + + previousWriter := writers + writers = &failOnNthPatchWriter{Writer: previousWriter, failOnPatchCall: 2, err: errors.New("status patch failed")} + t.Cleanup(func() { + writers = previousWriter + }) + + result, err := reconciler.Reconcile(context.Background(), reconcile.Request{ + NamespacedName: client.ObjectKeyFromObject(drd), + }) + require.Error(t, err) + assert.Equal(t, ctrl.Result{}, result) + assert.Contains(t, err.Error(), "invalid DruidSpec") + assert.Contains(t, err.Error(), "status patch failed") + + stored := &v1alpha1.Druid{} + require.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.NotEqual(t, v1alpha1.DeploymentLifecycleFailed, stored.Status.DeploymentLifecycle.Phase) +} + +func TestShouldPersistDeploymentLifecycleFailure(t *testing.T) { + assert.True(t, shouldPersistDeploymentLifecycleFailure(markTerminalDeploymentLifecycleError(errors.New("invalid spec")))) + assert.False(t, shouldPersistDeploymentLifecycleFailure(errors.New("temporary api timeout"))) + assert.False(t, shouldPersistDeploymentLifecycleFailure(context.Canceled)) +} + +type failOnNthPatchWriter struct { + Writer + patchCalls int + failOnPatchCall int + err error +} + +func (w *failOnNthPatchWriter) Patch(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid, obj object, status bool, patch client.Patch, emitEvent EventEmitter) error { + w.patchCalls++ + if w.patchCalls == w.failOnPatchCall { + return w.err + } + return w.Writer.Patch(ctx, sdk, drd, obj, status, patch, emitEvent) +} diff --git a/controllers/druid/dynamic_config.go b/controllers/druid/dynamic_config.go index 6cfd170..880d76f 100644 --- a/controllers/druid/dynamic_config.go +++ b/controllers/druid/dynamic_config.go @@ -49,7 +49,7 @@ func updateDruidDynamicConfigs( dynamicConfig := nodeConfig.DynamicConfig.Raw - svcName, err := druidapi.GetRouterSvcUrl(druid.Namespace, druid.Name, client) + svcName, err := druidapi.GetRouterSvcUrl(ctx, druid.Namespace, druid.Name, client) if err != nil { emitEvent.EmitEventGeneric( druid, @@ -85,15 +85,19 @@ func updateDruidDynamicConfigs( var dynamicConfigPath string switch nodeType { case "middlemanagers": - dynamicConfigPath = druidapi.MakePath(svcName, "indexer", "worker") + dynamicConfigPath, err = druidapi.MakePath(svcName, "indexer", "worker") case "coordinators": - dynamicConfigPath = druidapi.MakePath(svcName, "coordinator", "config") + dynamicConfigPath, err = druidapi.MakePath(svcName, "coordinator", "config") default: return fmt.Errorf("unsupported node type: %s", nodeType) } + if err != nil { + return err + } // Fetch current dynamic configurations currentResp, err := httpClient.Do( + ctx, http.MethodGet, dynamicConfigPath, nil, @@ -147,6 +151,7 @@ func updateDruidDynamicConfigs( // Update the Druid cluster's dynamic configurations if needed respDynamicConfigs, err := httpClient.Do( + ctx, http.MethodPost, dynamicConfigPath, dynamicConfig, diff --git a/controllers/druid/handler.go b/controllers/druid/handler.go index b236ef4..6c3aca1 100644 --- a/controllers/druid/handler.go +++ b/controllers/druid/handler.go @@ -59,9 +59,9 @@ var logger = logf.Log.WithName("druid_operator_handler") func deployDruidCluster(ctx context.Context, sdk client.Client, m *v1alpha1.Druid, emitEvents EventEmitter) error { if err := verifyDruidSpec(m); err != nil { - e := fmt.Errorf("invalid DruidSpec[%s:%s] due to [%s]", m.Kind, m.Name, err.Error()) + e := markTerminalDeploymentLifecycleError(fmt.Errorf("invalid DruidSpec[%s:%s] due to [%s]", m.Kind, m.Name, err.Error())) emitEvents.EmitEventGeneric(m, "DruidOperatorInvalidSpec", "", e) - return nil + return e } allNodeSpecs := getNodeSpecsByOrder(m) @@ -575,49 +575,48 @@ func execCheckCrashStatus(ctx context.Context, sdk client.Client, nodeSpec *v1al } func checkCrashStatus(ctx context.Context, sdk client.Client, nodeSpec *v1alpha1.DruidNodeSpec, drd *v1alpha1.Druid, nodeSpecUniqueStr string, emitEvents EventEmitter) error { + return checkCrashStatusWithMetrics(ctx, sdk, nodeSpec, drd, nodeSpecUniqueStr, emitEvents, defaultDruidRolloutMetrics) +} - podList, err := readers.List(ctx, sdk, drd, makeLabelsForNodeSpec(nodeSpec, drd, drd.Name, nodeSpecUniqueStr), emitEvents, func() objectList { return &v1.PodList{} }, func(listObj runtime.Object) []object { - items := listObj.(*v1.PodList).Items - result := make([]object, len(items)) - for i := 0; i < len(items); i++ { - result[i] = &items[i] - } - return result - }) +func checkCrashStatusWithMetrics(ctx context.Context, sdk client.Client, nodeSpec *v1alpha1.DruidNodeSpec, drd *v1alpha1.Druid, nodeSpecUniqueStr string, emitEvents EventEmitter, metrics *druidRolloutMetrics) error { + podList, err := listNodePods(ctx, sdk, nodeSpec, drd, nodeSpecUniqueStr) if err != nil { return err } - // the below condition evalutes if a pod is in - // 1. failed state 2. unknown state - // OR condtion.status is false which evalutes if neither of these conditions are met - // 1. ContainersReady 2. PodInitialized 3. PodReady 4. PodScheduled - for _, p := range podList { - if p.(*v1.Pod).Status.Phase == v1.PodFailed || p.(*v1.Pod).Status.Phase == v1.PodUnknown { - err := writers.Delete(ctx, sdk, drd, p, emitEvents, &client.DeleteOptions{}) - if err != nil { - return err - } - msg := fmt.Sprintf("Deleted pod [%s] in namespace [%s], since it was in [%s] state.", p.GetName(), p.GetNamespace(), p.(*v1.Pod).Status.Phase) - logger.Info(msg, "Object", stringifyForLogging(p, drd), "name", drd.Name, "namespace", drd.Namespace) - } else { - for _, condition := range p.(*v1.Pod).Status.Conditions { - if condition.Type == v1.ContainersReady { - if condition.Status == v1.ConditionFalse { - for _, containerStatus := range p.(*v1.Pod).Status.ContainerStatuses { - if containerStatus.RestartCount > 1 { - err := writers.Delete(ctx, sdk, drd, p, emitEvents, &client.DeleteOptions{}) - if err != nil { - return err - } - msg := fmt.Sprintf("Deleted pod [%s] in namespace [%s], since the container [%s] was crashlooping.", p.GetName(), p.GetNamespace(), containerStatus.Name) - logger.Info(msg, "Object", stringifyForLogging(p, drd), "name", drd.Name, "namespace", drd.Namespace) - } - } - } - } - } + candidates := findForceDeleteCandidates(podList) + if len(candidates) == 0 { + return nil + } + + podsByName := map[string]object{} + for _, pod := range podList { + podsByName[pod.GetName()] = pod + } + + for _, candidate := range candidates { + pod := podsByName[candidate.podName] + if pod == nil { + continue } + + err := writers.Delete(ctx, sdk, drd, pod, emitEvents, &client.DeleteOptions{}) + if err != nil { + return err + } + + if metrics != nil { + metrics.recordForceDeleteAction(drd.Namespace, drd.Name, nodeSpecUniqueStr, nodeSpec.NodeType, candidate.reason) + } + + if candidate.reason == "pod_phase" { + msg := fmt.Sprintf("Deleted pod [%s] in namespace [%s], since it was in [%s] state.", pod.GetName(), pod.GetNamespace(), pod.(*v1.Pod).Status.Phase) + logger.Info(msg, "Object", stringifyForLogging(pod, drd), "name", drd.Name, "namespace", drd.Namespace) + continue + } + + msg := fmt.Sprintf("Deleted pod [%s] in namespace [%s], since the container [%s] was crashlooping.", pod.GetName(), pod.GetNamespace(), candidate.containerName) + logger.Info(msg, "Object", stringifyForLogging(pod, drd), "name", drd.Name, "namespace", drd.Namespace) } return nil @@ -738,7 +737,7 @@ func isObjFullyDeployed(ctx context.Context, sdk client.Client, nodeSpec v1alpha for _, condition := range obj.(*appsv1.Deployment).Status.Conditions { // This detects a failure condition, operator should send a rolling deployment failed event if condition.Type == appsv1.DeploymentReplicaFailure { - return false, errors.New(condition.Reason) + return false, markTerminalDeploymentLifecycleError(errors.New(condition.Reason)) } else if condition.Type == appsv1.DeploymentProgressing && condition.Status != v1.ConditionTrue || obj.(*appsv1.Deployment).Status.ReadyReplicas != obj.(*appsv1.Deployment).Status.Replicas { return false, nil } else { @@ -941,8 +940,11 @@ func getVolume(nodeSpec *v1alpha1.DruidNodeSpec, m *v1alpha1.Druid, nodeSpecUniq func getEnv(nodeSpec *v1alpha1.DruidNodeSpec, m *v1alpha1.Druid, configMapSHA string) []v1.EnvVar { envHolder := firstNonNilValue(nodeSpec.Env, m.Spec.Env).([]v1.EnvVar) - // enables to do the trick to force redeployment in case of configmap changes. + // These values intentionally change the pod template when config or rollout inputs change. envHolder = append(envHolder, v1.EnvVar{Name: "configMapSHA", Value: configMapSHA}) + if m.Spec.ForceRedeployToken != "" { + envHolder = append(envHolder, v1.EnvVar{Name: "forceRedeployToken", Value: m.Spec.ForceRedeployToken}) + } return envHolder } diff --git a/controllers/druid/handler_unit_test.go b/controllers/druid/handler_unit_test.go new file mode 100644 index 0000000..247fa8a --- /dev/null +++ b/controllers/druid/handler_unit_test.go @@ -0,0 +1,51 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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 druid + +import ( + "context" + "testing" + + "github.com/apache/druid-operator/apis/druid/v1alpha1" + "github.com/stretchr/testify/assert" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +func TestDeployDruidClusterReturnsInvalidSpecError(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "bad_key": {}, + }, + }, + } + + err := deployDruidCluster(context.Background(), newLifecycleTestClient(t, drd), drd, noopEventEmitter{}) + assert.Error(t, err) + assert.Contains(t, err.Error(), "invalid DruidSpec") +} diff --git a/controllers/druid/interface.go b/controllers/druid/interface.go index 1096632..6502cca 100644 --- a/controllers/druid/interface.go +++ b/controllers/druid/interface.go @@ -61,12 +61,15 @@ const ( druidFinalizerFailed druidEventReason = "DruidFinalizerFailed" druidFinalizerSuccess druidEventReason = "DruidFinalizerSuccess" - druidGetRouterSvcUrlFailed druidEventReason = "DruidAPIGetRouterSvcUrlFailed" - druidGetAuthCredsFailed druidEventReason = "DruidAPIGetAuthCredsFailed" - druidFetchCurrentConfigsFailed druidEventReason = "DruidAPIFetchCurrentConfigsFailed" - druidConfigComparisonFailed druidEventReason = "DruidAPIConfigComparisonFailed" - druidUpdateConfigsFailed druidEventReason = "DruidAPIUpdateConfigsFailed" - druidUpdateConfigsSuccess druidEventReason = "DruidAPIUpdateConfigsSuccess" + druidGetRouterSvcUrlFailed druidEventReason = "DruidAPIGetRouterSvcUrlFailed" + druidGetAuthCredsFailed druidEventReason = "DruidAPIGetAuthCredsFailed" + druidFetchCurrentConfigsFailed druidEventReason = "DruidAPIFetchCurrentConfigsFailed" + druidConfigComparisonFailed druidEventReason = "DruidAPIConfigComparisonFailed" + druidUpdateConfigsFailed druidEventReason = "DruidAPIUpdateConfigsFailed" + druidUpdateConfigsSuccess druidEventReason = "DruidAPIUpdateConfigsSuccess" + druidDeploymentLifecycleStarted druidEventReason = "DruidDeploymentLifecycleStarted" + druidDeploymentLifecycleSucceeded druidEventReason = "DruidDeploymentLifecycleSucceeded" + druidDeploymentLifecycleFailed druidEventReason = "DruidDeploymentLifecycleFailed" ) // Reader Interface diff --git a/controllers/druid/lifecycle.go b/controllers/druid/lifecycle.go new file mode 100644 index 0000000..88c2dda --- /dev/null +++ b/controllers/druid/lifecycle.go @@ -0,0 +1,648 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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 druid + +import ( + "context" + "crypto/sha1" + "encoding/base64" + "encoding/json" + "errors" + "fmt" + "reflect" + "sort" + "time" + + "github.com/apache/druid-operator/apis/druid/v1alpha1" + druidapi "github.com/apache/druid-operator/pkg/druidapi" + appsv1 "k8s.io/api/apps/v1" + v1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" + "sigs.k8s.io/controller-runtime/pkg/client" +) + +var errDeploymentLifecycleTerminal = errors.New("terminal deployment lifecycle failure") + +type terminalDeploymentLifecycleError struct { + err error +} + +func (e terminalDeploymentLifecycleError) Error() string { + return e.err.Error() +} + +func (e terminalDeploymentLifecycleError) Unwrap() error { + return e.err +} + +func (e terminalDeploymentLifecycleError) Is(target error) bool { + return target == errDeploymentLifecycleTerminal +} + +type lifecycleRevisionInput struct { + Generation int64 `json:"generation"` + Spec v1alpha1.DruidSpec `json:"spec"` +} + +type lifecycleDependencies struct { + now func() time.Time + getBuildRevisions func(context.Context, client.Client, string, string, druidapi.Auth) ([]string, error) +} + +func defaultLifecycleDependencies() lifecycleDependencies { + return lifecycleDependencies{ + now: time.Now, + getBuildRevisions: druidapi.GetClusterBuildRevisions, + } +} + +type deploymentLifecycleInputs struct { + Image string + ForceRedeployToken string + ImageInputsHash string +} + +func computeDeploymentLifecycleRevision(drd *v1alpha1.Druid) (string, error) { + specForRevision := drd.Spec.DeepCopy() + specForRevision.ExpectedBuildRevision = "" + + payload, err := json.Marshal(lifecycleRevisionInput{ + Generation: drd.Generation, + Spec: *specForRevision, + }) + if err != nil { + return "", err + } + + sum := sha1.Sum(payload) + return base64.StdEncoding.EncodeToString(sum[:]), nil +} + +func currentDeploymentLifecycleInputs(drd *v1alpha1.Druid) (deploymentLifecycleInputs, error) { + imageInputsHash, err := computeEffectiveImageInputsHash(drd) + if err != nil { + return deploymentLifecycleInputs{}, err + } + + return deploymentLifecycleInputs{ + Image: drd.Spec.Image, + ForceRedeployToken: drd.Spec.ForceRedeployToken, + ImageInputsHash: imageInputsHash, + }, nil +} + +type effectiveImageInput struct { + Name string `json:"name"` + Image string `json:"image"` +} + +func computeEffectiveImageInputsHash(drd *v1alpha1.Druid) (string, error) { + inputs := make([]effectiveImageInput, 0, len(drd.Spec.Nodes)+1) + inputs = append(inputs, effectiveImageInput{ + Name: "__default__/druid", + Image: drd.Spec.Image, + }) + inputs = appendEffectiveAdditionalContainerImages(inputs, "cluster", drd.Spec.AdditionalContainer) + + nodeNames := make([]string, 0, len(drd.Spec.Nodes)) + for nodeName := range drd.Spec.Nodes { + nodeNames = append(nodeNames, nodeName) + } + sort.Strings(nodeNames) + + for _, nodeName := range nodeNames { + nodeSpec := drd.Spec.Nodes[nodeName] + inputs = append(inputs, effectiveImageInput{ + Name: fmt.Sprintf("node/%s/druid", nodeName), + Image: firstNonEmptyStr(nodeSpec.Image, drd.Spec.Image), + }) + inputs = appendEffectiveAdditionalContainerImages(inputs, "node/"+nodeName, nodeSpec.AdditionalContainer) + } + + payload, err := json.Marshal(inputs) + if err != nil { + return "", err + } + + sum := sha1.Sum(payload) + return base64.StdEncoding.EncodeToString(sum[:]), nil +} + +func determineDeploymentTrigger( + current deploymentLifecycleInputs, + lastSuccessful v1alpha1.DeploymentLifecycleStatus, +) v1alpha1.DeploymentLifecycleTrigger { + if !hasDeploymentLifecycleBaseline(lastSuccessful) { + return v1alpha1.DeploymentTriggerSpecChange + } + + if current.ForceRedeployToken != lastSuccessful.LastSuccessfulForceRedeployToken { + return v1alpha1.DeploymentTriggerManualRollout + } + + lastSuccessfulImageHash := lastSuccessful.LastSuccessfulImageInputsHash + if lastSuccessfulImageHash == "" { + lastSuccessfulImageHash = computeLegacyImageInputsHash(lastSuccessful) + } + + if lastSuccessfulImageHash != "" && + current.ImageInputsHash != "" && + current.ImageInputsHash != lastSuccessfulImageHash { + return v1alpha1.DeploymentTriggerImageChange + } + + if current.Image != lastSuccessful.LastSuccessfulImage { + return v1alpha1.DeploymentTriggerImageChange + } + + return v1alpha1.DeploymentTriggerSpecChange +} + +func appendEffectiveAdditionalContainerImages( + inputs []effectiveImageInput, + scope string, + containers []v1alpha1.AdditionalContainer, +) []effectiveImageInput { + for _, container := range containers { + containerType := "sidecar" + if container.RunAsInit { + containerType = "init" + } + + inputs = append(inputs, effectiveImageInput{ + Name: fmt.Sprintf("%s/%s/%s", scope, containerType, container.ContainerName), + Image: container.Image, + }) + } + return inputs +} + +func computeLegacyImageInputsHash(lastSuccessful v1alpha1.DeploymentLifecycleStatus) string { + payload, err := json.Marshal([]effectiveImageInput{{ + Name: "__default__/druid", + Image: lastSuccessful.LastSuccessfulImage, + }}) + if err != nil { + return "" + } + + sum := sha1.Sum(payload) + return base64.StdEncoding.EncodeToString(sum[:]) +} + +func hasDeploymentLifecycleBaseline(status v1alpha1.DeploymentLifecycleStatus) bool { + return status.LastSuccessfulImage != "" || + status.LastSuccessfulImageInputsHash != "" || + status.LastSuccessfulForceRedeployToken != "" || + status.CompletedAt != nil +} + +func markTerminalDeploymentLifecycleError(err error) error { + if err == nil { + return nil + } + return terminalDeploymentLifecycleError{err: err} +} + +func shouldPersistDeploymentLifecycleFailure(err error) bool { + return errors.Is(err, errDeploymentLifecycleTerminal) +} + +func requiresBuildRevisionValidation(trigger v1alpha1.DeploymentLifecycleTrigger) bool { + return trigger == v1alpha1.DeploymentTriggerImageChange || trigger == v1alpha1.DeploymentTriggerManualRollout +} + +func ensureDeploymentLifecycleStarted( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, + emitEvent EventEmitter, +) error { + return ensureDeploymentLifecycleStartedWithDeps( + ctx, + sdk, + drd, + emitEvent, + defaultLifecycleDependencies(), + ) +} + +func ensureDeploymentLifecycleStartedWithDeps( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, + emitEvent EventEmitter, + deps lifecycleDependencies, +) error { + desiredRevision, err := computeDeploymentLifecycleRevision(drd) + if err != nil { + return err + } + + current := drd.Status.DeploymentLifecycle + if current.Revision == desiredRevision && current.Phase != "" { + return nil + } + + updated, err := buildLifecycleStatusForRevision(drd, current, desiredRevision) + if err != nil { + return err + } + updated.Phase = v1alpha1.DeploymentLifecyclePending + updated.Reason = "Waiting for Druid workloads to roll out" + now := metav1.NewTime(deps.now()) + updated.StartedAt = &now + + return patchDeploymentLifecycleStatus(ctx, sdk, drd, updated, emitEvent) +} + +func markDeploymentLifecycleFailed( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, + emitEvent EventEmitter, + err error, +) error { + return markDeploymentLifecycleFailedWithDeps( + ctx, + sdk, + drd, + emitEvent, + err, + defaultLifecycleDependencies(), + ) +} + +func markDeploymentLifecycleFailedWithDeps( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, + emitEvent EventEmitter, + err error, + deps lifecycleDependencies, +) error { + current := drd.Status.DeploymentLifecycle + desiredRevision, revisionErr := computeDeploymentLifecycleRevision(drd) + if revisionErr != nil { + return revisionErr + } + updated, buildErr := buildLifecycleStatusForRevision(drd, current, desiredRevision) + if buildErr != nil { + return buildErr + } + if updated.StartedAt == nil { + now := metav1.NewTime(deps.now()) + updated.StartedAt = &now + } + updated.Phase = v1alpha1.DeploymentLifecycleFailed + updated.Reason = err.Error() + completedAt := metav1.NewTime(deps.now()) + updated.CompletedAt = &completedAt + + return patchDeploymentLifecycleStatus(ctx, sdk, drd, updated, emitEvent) +} + +func reconcileDeploymentLifecycle( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, + emitEvent EventEmitter, +) error { + return reconcileDeploymentLifecycleWithDeps( + ctx, + sdk, + drd, + emitEvent, + defaultLifecycleDependencies(), + ) +} + +func reconcileDeploymentLifecycleWithDeps( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, + emitEvent EventEmitter, + deps lifecycleDependencies, +) error { + desiredRevision, err := computeDeploymentLifecycleRevision(drd) + if err != nil { + return err + } + + updated, err := buildLifecycleStatusForRevision(drd, drd.Status.DeploymentLifecycle, desiredRevision) + if err != nil { + return err + } + if updated.StartedAt == nil { + now := metav1.NewTime(deps.now()) + updated.StartedAt = &now + } + inputs, err := currentDeploymentLifecycleInputs(drd) + if err != nil { + return err + } + + workloadsReady, reason, err := areManagedWorkloadsReady(ctx, sdk, drd) + if err != nil { + return err + } + if !workloadsReady { + return patchDeploymentLifecycleStatus(ctx, sdk, drd, withDeploymentLifecycleInProgress(updated, reason), emitEvent) + } + + podsReady, reason, err := areManagedPodsReady(ctx, sdk, drd) + if err != nil { + return err + } + if !podsReady { + return patchDeploymentLifecycleStatus(ctx, sdk, drd, withDeploymentLifecycleInProgress(updated, reason), emitEvent) + } + + buildRevisionValidationRequired := requiresBuildRevisionValidation(updated.Trigger) + if buildRevisionValidationRequired { + if drd.Spec.ExpectedBuildRevision == "" { + updated = withDeploymentLifecycleInProgress(updated, "Waiting for spec.expectedBuildRevision before completing image rollout") + updated.ObservedBuildRevisions = nil + return patchDeploymentLifecycleStatus(ctx, sdk, drd, updated, emitEvent) + } + + revisions, err := deps.getBuildRevisions(ctx, sdk, drd.Namespace, drd.Name, drd.Spec.Auth) + if err != nil { + var pendingErr *druidapi.BuildRevisionLookupPendingError + if errors.As(err, &pendingErr) { + updated.ObservedBuildRevisions = nil + return patchDeploymentLifecycleStatus(ctx, sdk, drd, withDeploymentLifecycleInProgress(updated, buildRevisionPendingMessage(pendingErr)), emitEvent) + } + if errors.Is(err, context.Canceled) { + return err + } + return markTerminalDeploymentLifecycleError(err) + } + + updated.ObservedBuildRevisions = append([]string(nil), revisions...) + if len(revisions) == 0 { + return patchDeploymentLifecycleStatus(ctx, sdk, drd, withDeploymentLifecycleInProgress(updated, "Waiting for Druid servers to report a build revision"), emitEvent) + } + + if len(revisions) != 1 || revisions[0] != drd.Spec.ExpectedBuildRevision { + return patchDeploymentLifecycleStatus( + ctx, + sdk, + drd, + withDeploymentLifecycleInProgress(updated, fmt.Sprintf("Waiting for all Druid servers to report build revision [%s]", drd.Spec.ExpectedBuildRevision)), + emitEvent, + ) + } + } else { + updated.ObservedBuildRevisions = nil + } + + if updated.Phase == v1alpha1.DeploymentLifecycleSucceeded { + return nil + } + + updated = withDeploymentLifecycleSucceeded(updated, drd.Spec.ExpectedBuildRevision, buildRevisionValidationRequired, deps.now()) + updated.LastSuccessfulImage = inputs.Image + updated.LastSuccessfulImageInputsHash = inputs.ImageInputsHash + updated.LastSuccessfulForceRedeployToken = inputs.ForceRedeployToken + return patchDeploymentLifecycleStatus(ctx, sdk, drd, updated, emitEvent) +} + +func withDeploymentLifecycleInProgress( + status v1alpha1.DeploymentLifecycleStatus, + reason string, +) v1alpha1.DeploymentLifecycleStatus { + status.Phase = v1alpha1.DeploymentLifecycleInProgress + status.Reason = reason + status.CompletedAt = nil + return status +} + +func withDeploymentLifecycleSucceeded( + status v1alpha1.DeploymentLifecycleStatus, + expectedBuildRevision string, + buildRevisionValidated bool, + now time.Time, +) v1alpha1.DeploymentLifecycleStatus { + completedAt := metav1.NewTime(now) + status.Phase = v1alpha1.DeploymentLifecycleSucceeded + if buildRevisionValidated { + status.Reason = fmt.Sprintf("Deployment lifecycle completed for build revision [%s]", expectedBuildRevision) + } else { + status.Reason = "Deployment lifecycle completed after managed workloads became ready" + } + status.CompletedAt = &completedAt + return status +} + +func buildLifecycleStatusForRevision( + drd *v1alpha1.Druid, + current v1alpha1.DeploymentLifecycleStatus, + desiredRevision string, +) (v1alpha1.DeploymentLifecycleStatus, error) { + inputs, err := currentDeploymentLifecycleInputs(drd) + if err != nil { + return v1alpha1.DeploymentLifecycleStatus{}, err + } + + updated := current + updated.Revision = desiredRevision + updated.ObservedGeneration = drd.Generation + updated.Trigger = determineDeploymentTrigger(inputs, current) + updated.ExpectedBuildRevision = drd.Spec.ExpectedBuildRevision + updated.CompletedAt = nil + if updated.Revision != current.Revision { + updated.StartedAt = nil + updated.ObservedBuildRevisions = nil + } + return updated, nil +} + +func buildRevisionPendingMessage(err *druidapi.BuildRevisionLookupPendingError) string { + switch err.Reason { + case druidapi.BuildRevisionPendingQueryServiceDiscovery: + return "Waiting for Druid query service discovery" + case druidapi.BuildRevisionPendingQueryEndpoint: + return "Waiting for the Druid build revision query endpoint" + case druidapi.BuildRevisionPendingQueryResponse: + return "Waiting for the Druid build revision query to succeed" + default: + return "Waiting for Druid build revision verification" + } +} + +func areManagedWorkloadsReady(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid) (bool, string, error) { + listOpts := []client.ListOption{ + client.InNamespace(drd.Namespace), + client.MatchingLabels(makeLabelsForDruid(drd)), + } + + deployments := &appsv1.DeploymentList{} + if err := sdk.List(ctx, deployments, listOpts...); err != nil { + return false, "", err + } + for _, deployment := range deployments.Items { + for _, condition := range deployment.Status.Conditions { + if condition.Type == appsv1.DeploymentReplicaFailure { + return false, "", markTerminalDeploymentLifecycleError(fmt.Errorf("deployment [%s] replica failure: %s", deployment.Name, condition.Reason)) + } + } + specReplicas := int32(1) + if deployment.Spec.Replicas != nil { + specReplicas = *deployment.Spec.Replicas + } + if deployment.Status.ObservedGeneration < deployment.Generation { + return false, fmt.Sprintf("Waiting for Deployment [%s] controller to observe generation", deployment.Name), nil + } + if deployment.Status.UpdatedReplicas != specReplicas || + deployment.Status.ReadyReplicas != specReplicas || + deployment.Status.Replicas != specReplicas || + deployment.Status.UnavailableReplicas != 0 { + return false, fmt.Sprintf("Waiting for Deployment [%s] rollout", deployment.Name), nil + } + } + + statefulSets := &appsv1.StatefulSetList{} + if err := sdk.List(ctx, statefulSets, listOpts...); err != nil { + return false, "", err + } + for _, statefulSet := range statefulSets.Items { + specReplicas := int32(1) + if statefulSet.Spec.Replicas != nil { + specReplicas = *statefulSet.Spec.Replicas + } + if statefulSet.Status.ObservedGeneration < statefulSet.Generation { + return false, fmt.Sprintf("Waiting for StatefulSet [%s] controller to observe generation", statefulSet.Name), nil + } + if statefulSet.Status.CurrentRevision != statefulSet.Status.UpdateRevision || + statefulSet.Status.UpdatedReplicas != specReplicas { + return false, fmt.Sprintf("Waiting for StatefulSet [%s] revision rollout", statefulSet.Name), nil + } + if statefulSet.Status.ReadyReplicas != specReplicas { + return false, fmt.Sprintf("Waiting for StatefulSet [%s] ready replicas", statefulSet.Name), nil + } + } + + return true, "", nil +} + +func areManagedPodsReady(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid) (bool, string, error) { + pods := &v1.PodList{} + if err := sdk.List(ctx, pods, + client.InNamespace(drd.Namespace), + client.MatchingLabels(makeLabelsForDruid(drd)), + ); err != nil { + return false, "", err + } + + if len(pods.Items) == 0 { + return false, "Waiting for Druid pods to be created", nil + } + + for _, pod := range pods.Items { + if pod.Status.Phase != v1.PodRunning { + return false, fmt.Sprintf("Waiting for Pod [%s] to be running", pod.Name), nil + } + + ready := false + for _, condition := range pod.Status.Conditions { + if condition.Type == v1.PodReady { + if condition.Status == v1.ConditionTrue { + ready = true + } else { + return false, fmt.Sprintf("Waiting for Pod [%s] readiness", pod.Name), nil + } + } + } + if !ready { + return false, fmt.Sprintf("Waiting for Pod [%s] readiness", pod.Name), nil + } + } + + return true, "", nil +} + +func patchDeploymentLifecycleStatus( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, + updated v1alpha1.DeploymentLifecycleStatus, + emitEvent EventEmitter, +) error { + current := drd.Status.DeploymentLifecycle + if reflect.DeepEqual(current, updated) { + return nil + } + + patchBytes, err := json.Marshal(map[string]interface{}{ + "status": map[string]interface{}{ + "deploymentLifecycle": deploymentLifecycleStatusPatch(updated), + }, + }) + if err != nil { + return fmt.Errorf("failed to serialize deployment lifecycle status patch: %v", err) + } + + if err := writers.Patch(ctx, sdk, drd, drd, true, client.RawPatch(types.MergePatchType, patchBytes), emitEvent); err != nil { + return err + } + + defaultDeploymentLifecycleMetrics.recordTransition(drd.Namespace, drd.Name, current, updated) + drd.Status.DeploymentLifecycle = updated + emitDeploymentLifecycleEvent(drd, emitEvent, current, updated) + return nil +} + +func deploymentLifecycleStatusPatch(status v1alpha1.DeploymentLifecycleStatus) map[string]interface{} { + return map[string]interface{}{ + "revision": status.Revision, + "phase": status.Phase, + "reason": status.Reason, + "observedGeneration": status.ObservedGeneration, + "startedAt": status.StartedAt, + "completedAt": status.CompletedAt, + "lastSuccessfulImage": status.LastSuccessfulImage, + "lastSuccessfulImageInputsHash": status.LastSuccessfulImageInputsHash, + "lastSuccessfulForceRedeployToken": status.LastSuccessfulForceRedeployToken, + "trigger": status.Trigger, + "expectedBuildRevision": status.ExpectedBuildRevision, + "observedBuildRevisions": status.ObservedBuildRevisions, + } +} + +func emitDeploymentLifecycleEvent( + drd *v1alpha1.Druid, + emitEvent EventEmitter, + previous, current v1alpha1.DeploymentLifecycleStatus, +) { + if previous.Phase == current.Phase && previous.Revision == current.Revision && previous.Reason == current.Reason { + return + } + + msg := fmt.Sprintf("revision=%s phase=%s reason=%s", current.Revision, current.Phase, current.Reason) + switch current.Phase { + case v1alpha1.DeploymentLifecyclePending, v1alpha1.DeploymentLifecycleInProgress: + emitEvent.EmitEventGeneric(drd, string(druidDeploymentLifecycleStarted), msg, nil) + case v1alpha1.DeploymentLifecycleSucceeded: + emitEvent.EmitEventGeneric(drd, string(druidDeploymentLifecycleSucceeded), msg, nil) + case v1alpha1.DeploymentLifecycleFailed: + emitEvent.EmitEventGeneric(drd, string(druidDeploymentLifecycleFailed), msg, errors.New(current.Reason)) + } +} diff --git a/controllers/druid/lifecycle_metrics_test.go b/controllers/druid/lifecycle_metrics_test.go new file mode 100644 index 0000000..b586bd4 --- /dev/null +++ b/controllers/druid/lifecycle_metrics_test.go @@ -0,0 +1,524 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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 druid + +import ( + "context" + "errors" + "net/http" + "net/http/httptest" + "testing" + "time" + + "github.com/apache/druid-operator/apis/druid/v1alpha1" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promhttp" + dto "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "sigs.k8s.io/controller-runtime/pkg/client" + ctrlmetrics "sigs.k8s.io/controller-runtime/pkg/metrics" +) + +const ( + testLifecycleNamespace = "default" + testLifecycleCluster = "example" +) + +func TestDeploymentLifecycleMetricsRecordsPhaseTransitionsAndTerminalOutcome(t *testing.T) { + registry := prometheus.NewRegistry() + metrics := newDeploymentLifecycleMetrics(registry) + + startedAt := metav1TimePtr(time.Unix(100, 0)) + completedAt := metav1TimePtr(time.Unix(160, 0)) + + metrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{}, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerImageChange, + StartedAt: startedAt, + }, + ) + metrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerImageChange, + StartedAt: startedAt, + }, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleSucceeded, + Trigger: v1alpha1.DeploymentTriggerImageChange, + StartedAt: startedAt, + CompletedAt: completedAt, + }, + ) + + families, err := registry.Gather() + require.NoError(t, err) + + assert.Equal(t, 1.0, metricValue(t, families, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "phase": "pending", + "trigger": "image_change", + })) + assert.Equal(t, 1.0, metricValue(t, families, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "phase": "succeeded", + "trigger": "image_change", + })) + assert.Equal(t, 1.0, metricValue(t, families, "druid_operator_deployment_lifecycle_completions_total", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "result": "succeeded", + "trigger": "image_change", + })) + assert.Equal(t, 0.0, metricValue(t, families, "druid_operator_deployment_lifecycle_in_progress", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "trigger": "image_change", + })) + assert.Equal(t, uint64(1), histogramSampleCount(t, families, "druid_operator_deployment_lifecycle_duration_seconds", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "result": "succeeded", + "trigger": "image_change", + })) + assert.InDelta(t, 60.0, histogramSampleSum(t, families, "druid_operator_deployment_lifecycle_duration_seconds", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "result": "succeeded", + "trigger": "image_change", + }), 0.0001) +} + +func TestDeploymentLifecycleMetricsDoesNotDoubleCountNoOpUpdates(t *testing.T) { + registry := prometheus.NewRegistry() + metrics := newDeploymentLifecycleMetrics(registry) + + startedAt := metav1TimePtr(time.Unix(100, 0)) + current := v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleInProgress, + Trigger: v1alpha1.DeploymentTriggerSpecChange, + StartedAt: startedAt, + Reason: "Waiting for Pod [example] readiness", + } + + metrics.recordTransition(testLifecycleNamespace, testLifecycleCluster, current, current) + + families, err := registry.Gather() + require.NoError(t, err) + + assert.Equal(t, 0.0, metricValue(t, families, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "phase": "in_progress", + "trigger": "spec_change", + })) + assert.Equal(t, 0.0, metricValue(t, families, "druid_operator_deployment_lifecycle_in_progress", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "trigger": "spec_change", + })) +} + +func TestDeploymentLifecycleMetricsKeepsInProgressGaugeAcrossActiveTransitions(t *testing.T) { + registry := prometheus.NewRegistry() + metrics := newDeploymentLifecycleMetrics(registry) + + startedAt := metav1TimePtr(time.Unix(100, 0)) + metrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{}, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerManualRollout, + StartedAt: startedAt, + }, + ) + metrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerManualRollout, + StartedAt: startedAt, + }, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleInProgress, + Trigger: v1alpha1.DeploymentTriggerManualRollout, + StartedAt: startedAt, + }, + ) + + families, err := registry.Gather() + require.NoError(t, err) + + assert.Equal(t, 1.0, metricValue(t, families, "druid_operator_deployment_lifecycle_in_progress", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "trigger": "manual_rollout", + })) + assert.Equal(t, 1.0, metricValue(t, families, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "phase": "pending", + "trigger": "manual_rollout", + })) + assert.Equal(t, 1.0, metricValue(t, families, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "phase": "in_progress", + "trigger": "manual_rollout", + })) +} + +func TestDefaultDeploymentLifecycleMetricsAreRegistered(t *testing.T) { + startedAt := metav1TimePtr(time.Unix(100, 0)) + defaultDeploymentLifecycleMetrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{}, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerImageChange, + StartedAt: startedAt, + }, + ) + defaultDeploymentLifecycleMetrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerImageChange, + StartedAt: startedAt, + }, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleSucceeded, + Trigger: v1alpha1.DeploymentTriggerImageChange, + StartedAt: startedAt, + CompletedAt: metav1TimePtr(time.Unix(160, 0)), + }, + ) + + families, err := ctrlmetrics.Registry.Gather() + require.NoError(t, err) + + assertMetricFamilyExists(t, families, "druid_operator_deployment_lifecycle_transitions_total") + assertMetricFamilyExists(t, families, "druid_operator_deployment_lifecycle_completions_total") + assertMetricFamilyExists(t, families, "druid_operator_deployment_lifecycle_duration_seconds") + assertMetricFamilyExists(t, families, "druid_operator_deployment_lifecycle_in_progress") +} + +func TestDefaultDeploymentLifecycleMetricsAreExposedByMetricsHandler(t *testing.T) { + defaultDeploymentLifecycleMetrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{}, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerManualRollout, + StartedAt: metav1TimePtr(time.Unix(100, 0)), + }, + ) + + req := httptest.NewRequest(http.MethodGet, "/metrics", nil) + recorder := httptest.NewRecorder() + handler := promhttp.HandlerFor(ctrlmetrics.Registry, promhttp.HandlerOpts{}) + handler.ServeHTTP(recorder, req) + + assert.Equal(t, http.StatusOK, recorder.Code) + assert.Contains(t, recorder.Body.String(), "druid_operator_deployment_lifecycle_transitions_total") + assert.Contains(t, recorder.Body.String(), `trigger="manual_rollout"`) + assert.Contains(t, recorder.Body.String(), `namespace="default"`) + assert.Contains(t, recorder.Body.String(), `druid_instance="example"`) +} + +func TestPatchDeploymentLifecycleStatusRecordsMetricsAfterSuccessfulPatch(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + } + k8sClient := newLifecycleTestClient(t, drd) + + updated := v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerSpecChange, + StartedAt: metav1TimePtr(time.Unix(100, 0)), + Reason: "Waiting for Druid workloads to roll out", + } + + before := globalMetricValue(t, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": drd.Namespace, + "druid_instance": drd.Name, + "phase": "pending", + "trigger": "spec_change", + }) + beforeGauge := globalMetricValue(t, "druid_operator_deployment_lifecycle_in_progress", map[string]string{ + "namespace": drd.Namespace, + "druid_instance": drd.Name, + "trigger": "spec_change", + }) + + require.NoError(t, patchDeploymentLifecycleStatus(context.Background(), k8sClient, drd, updated, noopEventEmitter{})) + + after := globalMetricValue(t, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": drd.Namespace, + "druid_instance": drd.Name, + "phase": "pending", + "trigger": "spec_change", + }) + afterGauge := globalMetricValue(t, "druid_operator_deployment_lifecycle_in_progress", map[string]string{ + "namespace": drd.Namespace, + "druid_instance": drd.Name, + "trigger": "spec_change", + }) + + assert.Equal(t, before+1, after) + assert.Equal(t, beforeGauge+1, afterGauge) +} + +func TestPatchDeploymentLifecycleStatusDoesNotRecordMetricsWhenPatchFails(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + } + k8sClient := newLifecycleTestClient(t, drd) + + updated := v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerSpecChange, + StartedAt: metav1TimePtr(time.Unix(100, 0)), + Reason: "Waiting for Druid workloads to roll out", + } + + previousWriter := writers + writers = failingPatchWriter{err: errors.New("patch failed")} + t.Cleanup(func() { + writers = previousWriter + }) + + before := globalMetricValue(t, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": drd.Namespace, + "druid_instance": drd.Name, + "phase": "pending", + "trigger": "spec_change", + }) + + err := patchDeploymentLifecycleStatus(context.Background(), k8sClient, drd, updated, noopEventEmitter{}) + require.EqualError(t, err, "patch failed") + + after := globalMetricValue(t, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": drd.Namespace, + "druid_instance": drd.Name, + "phase": "pending", + "trigger": "spec_change", + }) + + assert.Equal(t, before, after) +} + +func TestDeploymentLifecycleMetricsDeleteClusterRemovesPerClusterSeries(t *testing.T) { + registry := prometheus.NewRegistry() + metrics := newDeploymentLifecycleMetrics(registry) + + startedAt := metav1TimePtr(time.Unix(100, 0)) + completedAt := metav1TimePtr(time.Unix(160, 0)) + metrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{}, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerSpecChange, + StartedAt: startedAt, + }, + ) + metrics.recordTransition( + testLifecycleNamespace, + testLifecycleCluster, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecyclePending, + Trigger: v1alpha1.DeploymentTriggerSpecChange, + StartedAt: startedAt, + }, + v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleSucceeded, + Trigger: v1alpha1.DeploymentTriggerSpecChange, + StartedAt: startedAt, + CompletedAt: completedAt, + }, + ) + + metrics.deleteCluster(testLifecycleNamespace, testLifecycleCluster) + + families, err := registry.Gather() + require.NoError(t, err) + assert.Equal(t, 0.0, metricValue(t, families, "druid_operator_deployment_lifecycle_transitions_total", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "phase": "pending", + "trigger": "spec_change", + })) + assert.Equal(t, 0.0, metricValue(t, families, "druid_operator_deployment_lifecycle_completions_total", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "result": "succeeded", + "trigger": "spec_change", + })) + assert.Equal(t, 0.0, metricValue(t, families, "druid_operator_deployment_lifecycle_in_progress", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "trigger": "spec_change", + })) + assert.False(t, metricExists(families, "druid_operator_deployment_lifecycle_duration_seconds", map[string]string{ + "namespace": testLifecycleNamespace, + "druid_instance": testLifecycleCluster, + "result": "succeeded", + "trigger": "spec_change", + })) +} + +func metricValue(t *testing.T, families []*dto.MetricFamily, name string, labels map[string]string) float64 { + t.Helper() + metric := findMetric(t, families, name, labels) + switch { + case metric.Counter != nil: + return metric.Counter.GetValue() + case metric.Gauge != nil: + return metric.Gauge.GetValue() + default: + return 0 + } +} + +func globalMetricValue(t *testing.T, name string, labels map[string]string) float64 { + t.Helper() + families, err := ctrlmetrics.Registry.Gather() + require.NoError(t, err) + return metricValue(t, families, name, labels) +} + +func histogramSampleCount(t *testing.T, families []*dto.MetricFamily, name string, labels map[string]string) uint64 { + t.Helper() + metric := findMetric(t, families, name, labels) + require.NotNil(t, metric.Histogram) + return metric.Histogram.GetSampleCount() +} + +func histogramSampleSum(t *testing.T, families []*dto.MetricFamily, name string, labels map[string]string) float64 { + t.Helper() + metric := findMetric(t, families, name, labels) + require.NotNil(t, metric.Histogram) + return metric.Histogram.GetSampleSum() +} + +func findMetric(t *testing.T, families []*dto.MetricFamily, name string, labels map[string]string) *dto.Metric { + t.Helper() + for _, family := range families { + if family.GetName() != name { + continue + } + for _, metric := range family.Metric { + if hasLabels(metric, labels) { + return metric + } + } + } + return &dto.Metric{} +} + +func metricExists(families []*dto.MetricFamily, name string, labels map[string]string) bool { + for _, family := range families { + if family.GetName() != name { + continue + } + for _, metric := range family.Metric { + if hasLabels(metric, labels) { + return true + } + } + } + return false +} + +func assertMetricFamilyExists(t *testing.T, families []*dto.MetricFamily, name string) { + t.Helper() + for _, family := range families { + if family.GetName() == name { + return + } + } + t.Fatalf("metric family %s not found", name) +} + +func hasLabels(metric *dto.Metric, labels map[string]string) bool { + if len(labels) == 0 { + return true + } + matched := 0 + for _, label := range metric.Label { + expected, ok := labels[label.GetName()] + if !ok { + continue + } + if expected != label.GetValue() { + return false + } + matched++ + } + return matched == len(labels) +} + +func metav1TimePtr(t time.Time) *metav1.Time { + value := metav1.NewTime(t) + return &value +} + +type failingPatchWriter struct { + err error +} + +func (w failingPatchWriter) Delete(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid, obj object, emitEvent EventEmitter, deleteOptions ...client.DeleteOption) error { + return nil +} + +func (w failingPatchWriter) Create(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid, obj object, emitEvent EventEmitter) (DruidNodeStatus, error) { + return "", nil +} + +func (w failingPatchWriter) Update(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid, obj object, emitEvent EventEmitter) (DruidNodeStatus, error) { + return "", nil +} + +func (w failingPatchWriter) Patch(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid, obj object, status bool, patch client.Patch, emitEvent EventEmitter) error { + return w.err +} diff --git a/controllers/druid/lifecycle_test.go b/controllers/druid/lifecycle_test.go new file mode 100644 index 0000000..61a6d03 --- /dev/null +++ b/controllers/druid/lifecycle_test.go @@ -0,0 +1,1139 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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 druid + +import ( + "context" + "encoding/json" + "errors" + "testing" + "time" + + "github.com/apache/druid-operator/apis/druid/v1alpha1" + druidapi "github.com/apache/druid-operator/pkg/druidapi" + "github.com/stretchr/testify/assert" + appsv1 "k8s.io/api/apps/v1" + v1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + "sigs.k8s.io/controller-runtime/pkg/client/fake" +) + +type noopEventEmitter struct{} + +func (noopEventEmitter) EmitEventGeneric(obj object, eventReason, msg string, err error) {} +func (noopEventEmitter) EmitEventRollingDeployWait(obj, k8sObj object, nodeSpecUniqueStr string) {} +func (noopEventEmitter) EmitEventOnGetError(obj, getObj object, err error) {} +func (noopEventEmitter) EmitEventOnUpdate(obj, updateObj object, err error) {} +func (noopEventEmitter) EmitEventOnDelete(obj, deleteObj object, err error) {} +func (noopEventEmitter) EmitEventOnCreate(obj, createObj object, err error) {} +func (noopEventEmitter) EmitEventOnPatch(obj, patchObj object, err error) {} +func (noopEventEmitter) EmitEventOnList(obj object, listObj objectList, err error) {} + +func TestComputeDeploymentLifecycleRevisionChangesWithForceRedeployToken(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 3, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + ForceRedeployToken: "token-a", + }, + } + + first, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + + drd.Spec.ForceRedeployToken = "token-b" + second, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + + assert.NotEqual(t, first, second) +} + +func TestComputeDeploymentLifecycleRevisionIgnoresExpectedBuildRevision(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 3, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ExpectedBuildRevision: "build-a", + }, + } + + first, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + + drd.Spec.ExpectedBuildRevision = "build-b" + second, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + + assert.Equal(t, first, second) +} + +func TestDeploymentLifecycleStatusPatchIncludesNullsForClearedFields(t *testing.T) { + startedAt := metav1.NewTime(time.Unix(100, 0)) + payload, err := json.Marshal(map[string]interface{}{ + "status": map[string]interface{}{ + "deploymentLifecycle": deploymentLifecycleStatusPatch(v1alpha1.DeploymentLifecycleStatus{ + Revision: "rev-1", + Phase: v1alpha1.DeploymentLifecycleInProgress, + Reason: "Waiting for rollout", + ObservedGeneration: 3, + StartedAt: &startedAt, + CompletedAt: nil, + ExpectedBuildRevision: "36.0.0", + ObservedBuildRevisions: nil, + }), + }, + }) + assert.NoError(t, err) + + var decoded map[string]interface{} + assert.NoError(t, json.Unmarshal(payload, &decoded)) + + status := decoded["status"].(map[string]interface{}) + lifecycle := status["deploymentLifecycle"].(map[string]interface{}) + assert.Equal(t, "rev-1", lifecycle["revision"]) + assert.Equal(t, nil, lifecycle["completedAt"]) + assert.Equal(t, nil, lifecycle["observedBuildRevisions"]) + assert.NotNil(t, lifecycle["startedAt"]) +} + +func newLifecycleTestClient(t *testing.T, drd *v1alpha1.Druid, objects ...client.Object) client.Client { + scheme := runtime.NewScheme() + assert.NoError(t, v1alpha1.AddToScheme(scheme)) + assert.NoError(t, appsv1.AddToScheme(scheme)) + assert.NoError(t, v1.AddToScheme(scheme)) + + allObjects := append([]client.Object{drd}, objects...) + return fake.NewClientBuilder(). + WithScheme(scheme). + WithStatusSubresource(drd). + WithObjects(allObjects...). + Build() +} + +func readyManagedObjects(drd *v1alpha1.Druid) (*appsv1.Deployment, *v1.Pod) { + replicas := int32(1) + deployment := &appsv1.Deployment{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example-broker", + Namespace: drd.Namespace, + Labels: makeLabelsForDruid(drd), + Generation: 2, + }, + Spec: appsv1.DeploymentSpec{ + Replicas: &replicas, + }, + Status: appsv1.DeploymentStatus{ + ObservedGeneration: 2, + Replicas: 1, + ReadyReplicas: 1, + UpdatedReplicas: 1, + }, + } + + pod := &v1.Pod{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example-broker-0", + Namespace: drd.Namespace, + Labels: makeLabelsForDruid(drd), + }, + Status: v1.PodStatus{ + Phase: v1.PodRunning, + Conditions: []v1.PodCondition{ + { + Type: v1.PodReady, + Status: v1.ConditionTrue, + }, + }, + }, + } + + return deployment, pod +} + +func readyManagedStatefulSetObjects(drd *v1alpha1.Druid) (*appsv1.StatefulSet, *v1.Pod) { + replicas := int32(1) + statefulSet := &appsv1.StatefulSet{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example-historical", + Namespace: drd.Namespace, + Labels: makeLabelsForDruid(drd), + Generation: 2, + }, + Spec: appsv1.StatefulSetSpec{ + Replicas: &replicas, + }, + Status: appsv1.StatefulSetStatus{ + ObservedGeneration: 2, + CurrentRevision: "rev-2", + UpdateRevision: "rev-2", + UpdatedReplicas: 1, + ReadyReplicas: 1, + }, + } + + pod := &v1.Pod{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example-historical-0", + Namespace: drd.Namespace, + Labels: makeLabelsForDruid(drd), + }, + Status: v1.PodStatus{ + Phase: v1.PodRunning, + Conditions: []v1.PodCondition{ + { + Type: v1.PodReady, + Status: v1.ConditionTrue, + }, + }, + }, + } + + return statefulSet, pod +} + +func TestReconcileDeploymentLifecycleSucceedsWithoutExpectedBuildRevision(t *testing.T) { + + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + assert.NoError(t, reconcileDeploymentLifecycle(context.Background(), k8sClient, drd, noopEventEmitter{})) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleSucceeded, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, "Deployment lifecycle completed after managed workloads became ready", stored.Status.DeploymentLifecycle.Reason) + assert.Equal(t, drd.Generation, stored.Status.DeploymentLifecycle.ObservedGeneration) + assert.NotEmpty(t, stored.Status.DeploymentLifecycle.Revision) + assert.NotNil(t, stored.Status.DeploymentLifecycle.CompletedAt) +} + +func TestReconcileDeploymentLifecycleWaitsForExpectedBuildRevision(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ExpectedBuildRevision: "new-build", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:30.0.0", + }, + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + assert.NoError(t, reconcileDeploymentLifecycleWithDeps(context.Background(), k8sClient, drd, noopEventEmitter{}, lifecycleDependencies{ + now: time.Now, + getBuildRevisions: func(ctx context.Context, sdk client.Client, namespace, druidClusterName string, auth druidapi.Auth) ([]string, error) { + return []string{"old-build", "new-build"}, nil + }, + })) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleInProgress, stored.Status.DeploymentLifecycle.Phase) + assert.Contains(t, stored.Status.DeploymentLifecycle.Reason, "build revision [new-build]") + assert.Equal(t, "new-build", stored.Status.DeploymentLifecycle.ExpectedBuildRevision) + assert.Equal(t, []string{"old-build", "new-build"}, stored.Status.DeploymentLifecycle.ObservedBuildRevisions) + assert.Nil(t, stored.Status.DeploymentLifecycle.CompletedAt) +} + +func TestReconcileDeploymentLifecycleWaitsForExpectedBuildRevisionForImageRollout(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:30.0.0", + }, + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + assert.NoError(t, reconcileDeploymentLifecycle(context.Background(), k8sClient, drd, noopEventEmitter{})) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleInProgress, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, stored.Status.DeploymentLifecycle.Trigger) + assert.Equal(t, "Waiting for spec.expectedBuildRevision before completing image rollout", stored.Status.DeploymentLifecycle.Reason) + assert.Nil(t, stored.Status.DeploymentLifecycle.ObservedBuildRevisions) + assert.Nil(t, stored.Status.DeploymentLifecycle.CompletedAt) +} + +func TestReconcileDeploymentLifecycleSucceedsForImageRolloutWhenBuildRevisionMatches(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ExpectedBuildRevision: "new-build", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:30.0.0", + }, + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + assert.NoError(t, reconcileDeploymentLifecycleWithDeps(context.Background(), k8sClient, drd, noopEventEmitter{}, lifecycleDependencies{ + now: time.Now, + getBuildRevisions: func(ctx context.Context, sdk client.Client, namespace, druidClusterName string, auth druidapi.Auth) ([]string, error) { + return []string{"new-build"}, nil + }, + })) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleSucceeded, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, stored.Status.DeploymentLifecycle.Trigger) + assert.Contains(t, stored.Status.DeploymentLifecycle.Reason, "completed for build revision [new-build]") + assert.Equal(t, []string{"new-build"}, stored.Status.DeploymentLifecycle.ObservedBuildRevisions) + assert.Equal(t, "apache/druid:31.0.0", stored.Status.DeploymentLifecycle.LastSuccessfulImage) + assert.Equal(t, "", stored.Status.DeploymentLifecycle.LastSuccessfulForceRedeployToken) + assert.NotNil(t, stored.Status.DeploymentLifecycle.CompletedAt) +} + +func TestDetermineDeploymentTriggerUsesLastSuccessfulInputs(t *testing.T) { + lastSuccessful := v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:30.0.0", + LastSuccessfulImageInputsHash: "hash-a", + LastSuccessfulForceRedeployToken: "token-a", + } + + assert.Equal(t, v1alpha1.DeploymentTriggerSpecChange, determineDeploymentTrigger(deploymentLifecycleInputs{ + Image: "apache/druid:30.0.0", + ForceRedeployToken: "token-a", + ImageInputsHash: "hash-a", + }, lastSuccessful)) + + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, determineDeploymentTrigger(deploymentLifecycleInputs{ + Image: "apache/druid:31.0.0", + ForceRedeployToken: "token-a", + ImageInputsHash: "hash-b", + }, lastSuccessful)) + + assert.Equal(t, v1alpha1.DeploymentTriggerManualRollout, determineDeploymentTrigger(deploymentLifecycleInputs{ + Image: "apache/druid:31.0.0", + ForceRedeployToken: "token-b", + ImageInputsHash: "hash-b", + }, lastSuccessful)) +} + +func TestDetermineDeploymentTriggerTreatsMissingLifecycleHistoryAsBootstrap(t *testing.T) { + assert.Equal(t, v1alpha1.DeploymentTriggerSpecChange, determineDeploymentTrigger(deploymentLifecycleInputs{ + Image: "apache/druid:31.0.0", + ForceRedeployToken: "token-a", + ImageInputsHash: "hash-a", + }, v1alpha1.DeploymentLifecycleStatus{})) +} + +func TestReconcileDeploymentLifecyclePersistsManualRolloutInputsOnSuccess(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ForceRedeployToken: "token-b", + ExpectedBuildRevision: "new-build", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:31.0.0", + LastSuccessfulForceRedeployToken: "token-a", + }, + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + assert.NoError(t, reconcileDeploymentLifecycleWithDeps(context.Background(), k8sClient, drd, noopEventEmitter{}, lifecycleDependencies{ + now: time.Now, + getBuildRevisions: func(ctx context.Context, sdk client.Client, namespace, druidClusterName string, auth druidapi.Auth) ([]string, error) { + return []string{"new-build"}, nil + }, + })) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleSucceeded, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, v1alpha1.DeploymentTriggerManualRollout, stored.Status.DeploymentLifecycle.Trigger) + assert.Contains(t, stored.Status.DeploymentLifecycle.Reason, "completed for build revision [new-build]") + assert.Equal(t, "token-b", stored.Status.DeploymentLifecycle.LastSuccessfulForceRedeployToken) + assert.Equal(t, "apache/druid:31.0.0", stored.Status.DeploymentLifecycle.LastSuccessfulImage) + assert.NotEmpty(t, stored.Status.DeploymentLifecycle.LastSuccessfulImageInputsHash) +} + +func TestReconcileDeploymentLifecycleIgnoresExpectedBuildRevisionForSpecChange(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ExpectedBuildRevision: "new-build", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:31.0.0", + LastSuccessfulImageInputsHash: func() string { + inputs, err := currentDeploymentLifecycleInputs(&v1alpha1.Druid{Spec: v1alpha1.DruidSpec{Image: "apache/druid:31.0.0"}}) + assert.NoError(t, err) + return inputs.ImageInputsHash + }(), + }, + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + assert.NoError(t, reconcileDeploymentLifecycle(context.Background(), k8sClient, drd, noopEventEmitter{})) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleSucceeded, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, v1alpha1.DeploymentTriggerSpecChange, stored.Status.DeploymentLifecycle.Trigger) + assert.Equal(t, "Deployment lifecycle completed after managed workloads became ready", stored.Status.DeploymentLifecycle.Reason) + assert.Nil(t, stored.Status.DeploymentLifecycle.ObservedBuildRevisions) +} + +func TestEnsureDeploymentLifecycleStartedPublishesObservedGenerationAndRevisionForCurrentRollout(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 5, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleSucceeded, + ObservedGeneration: 4, + Revision: "old-revision", + }, + }, + } + + expectedRevision, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + + k8sClient := newLifecycleTestClient(t, drd) + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecyclePending, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, drd.Generation, stored.Status.DeploymentLifecycle.ObservedGeneration) + assert.Equal(t, expectedRevision, stored.Status.DeploymentLifecycle.Revision) + assert.NotNil(t, stored.Status.DeploymentLifecycle.StartedAt) +} + +func TestEnsureDeploymentLifecycleStartedReplacesStaleSucceededStatusForNewGeneration(t *testing.T) { + startedAt := metav1.NewTime(time.Unix(100, 0)) + completedAt := metav1.NewTime(time.Unix(160, 0)) + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 6, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleSucceeded, + Reason: "Deployment lifecycle completed after managed workloads became ready", + ObservedGeneration: 5, + Revision: "revision-for-generation-5", + StartedAt: &startedAt, + CompletedAt: &completedAt, + }, + }, + } + + k8sClient := newLifecycleTestClient(t, drd) + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecyclePending, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, "Waiting for Druid workloads to roll out", stored.Status.DeploymentLifecycle.Reason) + assert.Equal(t, drd.Generation, stored.Status.DeploymentLifecycle.ObservedGeneration) + assert.NotEqual(t, "revision-for-generation-5", stored.Status.DeploymentLifecycle.Revision) + assert.NotNil(t, stored.Status.DeploymentLifecycle.StartedAt) + assert.False(t, stored.Status.DeploymentLifecycle.StartedAt.Time.Equal(startedAt.Time)) +} + +func TestExpectedBuildRevisionUpdateKeepsRevisionStableForCurrentRollout(t *testing.T) { + startedAt := metav1.NewTime(time.Unix(100, 0)) + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 7, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ExpectedBuildRevision: "new-build", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleInProgress, + Trigger: v1alpha1.DeploymentTriggerImageChange, + Reason: "Waiting for spec.expectedBuildRevision before completing image rollout", + ObservedGeneration: 7, + StartedAt: &startedAt, + LastSuccessfulImage: "apache/druid:30.0.0", + }, + }, + } + + revisionBefore, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + drd.Status.DeploymentLifecycle.Revision = revisionBefore + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + assert.NoError(t, reconcileDeploymentLifecycleWithDeps(context.Background(), k8sClient, drd, noopEventEmitter{}, lifecycleDependencies{ + now: time.Now, + getBuildRevisions: func(ctx context.Context, sdk client.Client, namespace, druidClusterName string, auth druidapi.Auth) ([]string, error) { + return []string{"new-build"}, nil + }, + })) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleSucceeded, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, drd.Generation, stored.Status.DeploymentLifecycle.ObservedGeneration) + assert.Equal(t, revisionBefore, stored.Status.DeploymentLifecycle.Revision) +} + +func TestDetermineDeploymentTriggerDetectsNodeLevelImageChanges(t *testing.T) { + lastSuccessful := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "brokers": { + Image: "apache/druid:31.0.0", + }, + }, + }, + } + lastSuccessfulInputs, err := currentDeploymentLifecycleInputs(lastSuccessful) + assert.NoError(t, err) + + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "brokers": { + Image: "apache/druid:32.0.0", + }, + }, + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:31.0.0", + LastSuccessfulImageInputsHash: lastSuccessfulInputs.ImageInputsHash, + }, + }, + } + + current, err := currentDeploymentLifecycleInputs(drd) + assert.NoError(t, err) + trigger := determineDeploymentTrigger(current, drd.Status.DeploymentLifecycle) + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, trigger) +} + +func TestDetermineDeploymentTriggerTreatsLegacyNodeLevelOverridesAsImageChange(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "brokers": { + Image: "apache/druid:32.0.0", + }, + }, + }, + } + + current, err := currentDeploymentLifecycleInputs(drd) + assert.NoError(t, err) + + trigger := determineDeploymentTrigger(current, v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:31.0.0", + }) + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, trigger) +} + +func TestDetermineDeploymentTriggerDetectsClusterSidecarImageChanges(t *testing.T) { + lastSuccessful := &v1alpha1.Druid{ + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + AdditionalContainer: []v1alpha1.AdditionalContainer{ + { + ContainerName: "log-forwarder", + Image: "sidecar:v1", + }, + }, + }, + } + lastSuccessfulInputs, err := currentDeploymentLifecycleInputs(lastSuccessful) + assert.NoError(t, err) + + current := &v1alpha1.Druid{ + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + AdditionalContainer: []v1alpha1.AdditionalContainer{ + { + ContainerName: "log-forwarder", + Image: "sidecar:v2", + }, + }, + }, + } + currentInputs, err := currentDeploymentLifecycleInputs(current) + assert.NoError(t, err) + + trigger := determineDeploymentTrigger(currentInputs, v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:31.0.0", + LastSuccessfulImageInputsHash: lastSuccessfulInputs.ImageInputsHash, + }) + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, trigger) +} + +func TestDetermineDeploymentTriggerTreatsLegacyClusterSidecarImageChangesAsImageChange(t *testing.T) { + current := &v1alpha1.Druid{ + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + AdditionalContainer: []v1alpha1.AdditionalContainer{ + { + ContainerName: "log-forwarder", + Image: "sidecar:v2", + }, + }, + }, + } + currentInputs, err := currentDeploymentLifecycleInputs(current) + assert.NoError(t, err) + + trigger := determineDeploymentTrigger(currentInputs, v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:31.0.0", + }) + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, trigger) +} + +func TestDetermineDeploymentTriggerDetectsNodeInitContainerImageChanges(t *testing.T) { + lastSuccessful := &v1alpha1.Druid{ + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "brokers": { + AdditionalContainer: []v1alpha1.AdditionalContainer{ + { + RunAsInit: true, + ContainerName: "warm-cache", + Image: "init:v1", + }, + }, + }, + }, + }, + } + lastSuccessfulInputs, err := currentDeploymentLifecycleInputs(lastSuccessful) + assert.NoError(t, err) + + current := &v1alpha1.Druid{ + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "brokers": { + AdditionalContainer: []v1alpha1.AdditionalContainer{ + { + RunAsInit: true, + ContainerName: "warm-cache", + Image: "init:v2", + }, + }, + }, + }, + }, + } + currentInputs, err := currentDeploymentLifecycleInputs(current) + assert.NoError(t, err) + + trigger := determineDeploymentTrigger(currentInputs, v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:31.0.0", + LastSuccessfulImageInputsHash: lastSuccessfulInputs.ImageInputsHash, + }) + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, trigger) +} + +func TestDetermineDeploymentTriggerTreatsLegacyNodeInitContainerImageChangesAsImageChange(t *testing.T) { + current := &v1alpha1.Druid{ + Spec: v1alpha1.DruidSpec{ + Image: "apache/druid:31.0.0", + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "brokers": { + AdditionalContainer: []v1alpha1.AdditionalContainer{ + { + RunAsInit: true, + ContainerName: "warm-cache", + Image: "init:v2", + }, + }, + }, + }, + }, + } + currentInputs, err := currentDeploymentLifecycleInputs(current) + assert.NoError(t, err) + + trigger := determineDeploymentTrigger(currentInputs, v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:31.0.0", + }) + assert.Equal(t, v1alpha1.DeploymentTriggerImageChange, trigger) +} + +func TestAreManagedWorkloadsReadyWaitsForDeploymentObservedGeneration(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + } + + replicas := int32(1) + deployment := &appsv1.Deployment{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example-broker", + Namespace: drd.Namespace, + Labels: makeLabelsForDruid(drd), + Generation: 2, + }, + Spec: appsv1.DeploymentSpec{ + Replicas: &replicas, + }, + Status: appsv1.DeploymentStatus{ + ObservedGeneration: 1, + Replicas: 1, + ReadyReplicas: 1, + UpdatedReplicas: 1, + }, + } + + k8sClient := newLifecycleTestClient(t, drd, deployment) + ready, reason, err := areManagedWorkloadsReady(context.Background(), k8sClient, drd) + assert.NoError(t, err) + assert.False(t, ready) + assert.Equal(t, "Waiting for Deployment [example-broker] controller to observe generation", reason) +} + +func TestAreManagedWorkloadsReadyWaitsForStatefulSetObservedGeneration(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + } + + statefulSet, _ := readyManagedStatefulSetObjects(drd) + statefulSet.Status.ObservedGeneration = 1 + + k8sClient := newLifecycleTestClient(t, drd, statefulSet) + ready, reason, err := areManagedWorkloadsReady(context.Background(), k8sClient, drd) + assert.NoError(t, err) + assert.False(t, ready) + assert.Equal(t, "Waiting for StatefulSet [example-historical] controller to observe generation", reason) +} + +func TestAreManagedWorkloadsReadyAcceptsObservedStatefulSetGeneration(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + }, + } + + statefulSet, _ := readyManagedStatefulSetObjects(drd) + + k8sClient := newLifecycleTestClient(t, drd, statefulSet) + ready, reason, err := areManagedWorkloadsReady(context.Background(), k8sClient, drd) + assert.NoError(t, err) + assert.True(t, ready) + assert.Equal(t, "", reason) +} + +func TestReconcileDeploymentLifecycleWaitsWhenBuildRevisionLookupIsUnavailable(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ExpectedBuildRevision: "new-build", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:30.0.0", + }, + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + assert.NoError(t, reconcileDeploymentLifecycleWithDeps(context.Background(), k8sClient, drd, noopEventEmitter{}, lifecycleDependencies{ + now: time.Now, + getBuildRevisions: func(ctx context.Context, sdk client.Client, namespace, druidClusterName string, auth druidapi.Auth) ([]string, error) { + return nil, &druidapi.BuildRevisionLookupPendingError{ + Reason: druidapi.BuildRevisionPendingQueryServiceDiscovery, + } + }, + })) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleInProgress, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, "Waiting for Druid query service discovery", stored.Status.DeploymentLifecycle.Reason) + assert.Nil(t, stored.Status.DeploymentLifecycle.ObservedBuildRevisions) + assert.Nil(t, stored.Status.DeploymentLifecycle.CompletedAt) +} + +func TestReconcileDeploymentLifecycleReturnsCanceledErrorWithoutOverwritingPendingStatus(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ExpectedBuildRevision: "new-build", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:30.0.0", + }, + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + err := reconcileDeploymentLifecycleWithDeps(context.Background(), k8sClient, drd, noopEventEmitter{}, lifecycleDependencies{ + now: time.Now, + getBuildRevisions: func(ctx context.Context, sdk client.Client, namespace, druidClusterName string, auth druidapi.Auth) ([]string, error) { + return nil, context.Canceled + }, + }) + assert.ErrorIs(t, err, context.Canceled) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecyclePending, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, "Waiting for Druid workloads to roll out", stored.Status.DeploymentLifecycle.Reason) + assert.Nil(t, stored.Status.DeploymentLifecycle.ObservedBuildRevisions) + assert.Nil(t, stored.Status.DeploymentLifecycle.CompletedAt) +} + +func TestReconcileDeploymentLifecycleReturnsTerminalBuildRevisionLookupError(t *testing.T) { + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + ExpectedBuildRevision: "new-build", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + LastSuccessfulImage: "apache/druid:30.0.0", + }, + }, + } + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + err := reconcileDeploymentLifecycleWithDeps(context.Background(), k8sClient, drd, noopEventEmitter{}, lifecycleDependencies{ + now: time.Now, + getBuildRevisions: func(ctx context.Context, sdk client.Client, namespace, druidClusterName string, auth druidapi.Auth) ([]string, error) { + return nil, errors.New("unauthorized build revision query") + }, + }) + assert.Error(t, err) + assert.True(t, shouldPersistDeploymentLifecycleFailure(err)) + assert.Contains(t, err.Error(), "unauthorized build revision query") +} + +func TestMarkDeploymentLifecycleFailedPreservesStartedAtAndRevisionMetadata(t *testing.T) { + startedAt := metav1.NewTime(time.Unix(100, 0)) + completedAt := time.Unix(180, 0) + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 3, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleInProgress, + StartedAt: &startedAt, + Trigger: v1alpha1.DeploymentTriggerSpecChange, + }, + }, + } + + expectedRevision, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + drd.Status.DeploymentLifecycle.Revision = expectedRevision + k8sClient := newLifecycleTestClient(t, drd) + + assert.NoError(t, markDeploymentLifecycleFailedWithDeps(context.Background(), k8sClient, drd, noopEventEmitter{}, errors.New("deploy failed"), lifecycleDependencies{ + now: func() time.Time { return completedAt }, + })) + + stored := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), stored)) + assert.Equal(t, v1alpha1.DeploymentLifecycleFailed, stored.Status.DeploymentLifecycle.Phase) + assert.Equal(t, "deploy failed", stored.Status.DeploymentLifecycle.Reason) + assert.Equal(t, expectedRevision, stored.Status.DeploymentLifecycle.Revision) + assert.Equal(t, drd.Generation, stored.Status.DeploymentLifecycle.ObservedGeneration) + assert.NotNil(t, stored.Status.DeploymentLifecycle.StartedAt) + assert.True(t, stored.Status.DeploymentLifecycle.StartedAt.Time.Equal(startedAt.Time)) + assert.NotNil(t, stored.Status.DeploymentLifecycle.CompletedAt) + assert.True(t, stored.Status.DeploymentLifecycle.CompletedAt.Time.Equal(completedAt)) +} + +func TestBuildLifecycleStatusForRevisionClearsObservedBuildRevisionsOnNewRevision(t *testing.T) { + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 4, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + Image: "apache/druid:31.0.0", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + Revision: "old-revision", + ObservedBuildRevisions: []string{"old-build"}, + }, + }, + } + + desiredRevision, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + + updated, err := buildLifecycleStatusForRevision(drd, drd.Status.DeploymentLifecycle, desiredRevision) + assert.NoError(t, err) + assert.Nil(t, updated.ObservedBuildRevisions) +} + +func TestReconcileDeploymentLifecycleIsIdempotentAfterSuccess(t *testing.T) { + startedAt := metav1.NewTime(time.Unix(100, 0)) + completedAt := metav1.NewTime(time.Unix(160, 0)) + drd := &v1alpha1.Druid{ + TypeMeta: metav1.TypeMeta{ + APIVersion: "druid.apache.org/v1alpha1", + Kind: "Druid", + }, + ObjectMeta: metav1.ObjectMeta{ + Name: "example", + Namespace: "default", + Generation: 2, + }, + Spec: v1alpha1.DruidSpec{ + CommonRuntimeProperties: "druid.service=druid/router", + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleSucceeded, + Trigger: v1alpha1.DeploymentTriggerSpecChange, + Reason: "Deployment lifecycle completed after managed workloads became ready", + ObservedGeneration: 2, + StartedAt: &startedAt, + CompletedAt: &completedAt, + }, + }, + } + revision, err := computeDeploymentLifecycleRevision(drd) + assert.NoError(t, err) + drd.Status.DeploymentLifecycle.Revision = revision + + deployment, pod := readyManagedObjects(drd) + k8sClient := newLifecycleTestClient(t, drd, deployment, pod) + assert.NoError(t, ensureDeploymentLifecycleStarted(context.Background(), k8sClient, drd, noopEventEmitter{})) + + before := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), before)) + + assert.NoError(t, reconcileDeploymentLifecycle(context.Background(), k8sClient, drd, noopEventEmitter{})) + + after := &v1alpha1.Druid{} + assert.NoError(t, k8sClient.Get(context.Background(), client.ObjectKeyFromObject(drd), after)) + assert.Equal(t, before.Status.DeploymentLifecycle, after.Status.DeploymentLifecycle) +} diff --git a/controllers/druid/metrics.go b/controllers/druid/metrics.go new file mode 100644 index 0000000..3590176 --- /dev/null +++ b/controllers/druid/metrics.go @@ -0,0 +1,764 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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 druid + +import ( + "context" + "fmt" + "sync" + "time" + + "github.com/apache/druid-operator/apis/druid/v1alpha1" + "github.com/prometheus/client_golang/prometheus" + appsv1 "k8s.io/api/apps/v1" + v1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "sigs.k8s.io/controller-runtime/pkg/client" + ctrlmetrics "sigs.k8s.io/controller-runtime/pkg/metrics" +) + +type deploymentLifecycleMetrics struct { + transitions *prometheus.CounterVec + completions *prometheus.CounterVec + duration *prometheus.HistogramVec + inProgress *prometheus.GaugeVec +} + +var defaultDeploymentLifecycleMetrics = newDeploymentLifecycleMetrics(ctrlmetrics.Registry) + +var deploymentLifecycleTriggers = []string{"spec_change", "image_change", "manual_rollout", "unknown"} +var deploymentLifecycleResults = []string{"succeeded", "failed", "unknown"} + +func newDeploymentLifecycleMetrics(registerer prometheus.Registerer) *deploymentLifecycleMetrics { + metrics := &deploymentLifecycleMetrics{ + transitions: prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: "druid_operator_deployment_lifecycle_transitions_total", + Help: "Total number of Druid deployment lifecycle phase transitions.", + }, + []string{"namespace", "druid_instance", "phase", "trigger"}, + ), + completions: prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: "druid_operator_deployment_lifecycle_completions_total", + Help: "Total number of terminal Druid deployment lifecycle results.", + }, + []string{"namespace", "druid_instance", "result", "trigger"}, + ), + duration: prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Name: "druid_operator_deployment_lifecycle_duration_seconds", + Help: "Duration of terminal Druid deployment lifecycle runs.", + Buckets: []float64{1, 5, 10, 30, 60, 120, 300, 600, 1800, 3600}, + }, + []string{"namespace", "druid_instance", "result", "trigger"}, + ), + inProgress: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_deployment_lifecycle_in_progress", + Help: "Current number of active Druid deployment lifecycle runs.", + }, + []string{"namespace", "druid_instance", "trigger"}, + ), + } + + if registerer != nil { + registerer.MustRegister( + metrics.transitions, + metrics.completions, + metrics.duration, + metrics.inProgress, + ) + } + + return metrics +} + +func (m *deploymentLifecycleMetrics) recordTransition(namespace, druidInstance string, previous, current v1alpha1.DeploymentLifecycleStatus) { + if m == nil { + return + } + + previousActive := isActiveLifecyclePhase(previous.Phase) + currentActive := isActiveLifecyclePhase(current.Phase) + previousTrigger := lifecycleTriggerLabel(previous.Trigger) + currentTrigger := lifecycleTriggerLabel(current.Trigger) + + switch { + case previousActive && currentActive && previousTrigger != currentTrigger: + m.inProgress.WithLabelValues(namespace, druidInstance, previousTrigger).Dec() + m.inProgress.WithLabelValues(namespace, druidInstance, currentTrigger).Inc() + case !previousActive && currentActive: + m.inProgress.WithLabelValues(namespace, druidInstance, currentTrigger).Inc() + case previousActive && !currentActive: + m.inProgress.WithLabelValues(namespace, druidInstance, previousTrigger).Dec() + } + + if previous.Phase == current.Phase { + return + } + + phase := lifecyclePhaseLabel(current.Phase) + m.transitions.WithLabelValues(namespace, druidInstance, phase, currentTrigger).Inc() + + if !isTerminalLifecyclePhase(current.Phase) { + return + } + + result := lifecycleResultLabel(current.Phase) + m.completions.WithLabelValues(namespace, druidInstance, result, currentTrigger).Inc() + startedAt := current.StartedAt + if startedAt == nil { + startedAt = previous.StartedAt + } + if startedAt == nil || current.CompletedAt == nil { + return + } + + durationSeconds := current.CompletedAt.Sub(startedAt.Time).Seconds() + if durationSeconds < 0 { + durationSeconds = 0 + } + m.duration.WithLabelValues(namespace, druidInstance, result, currentTrigger).Observe(durationSeconds) +} + +func (m *deploymentLifecycleMetrics) deleteCluster(namespace, druidInstance string) { + if m == nil { + return + } + + for _, trigger := range deploymentLifecycleTriggers { + m.inProgress.DeleteLabelValues(namespace, druidInstance, trigger) + for _, phase := range deploymentLifecyclePhases { + m.transitions.DeleteLabelValues(namespace, druidInstance, phase, trigger) + } + for _, result := range deploymentLifecycleResults { + m.completions.DeleteLabelValues(namespace, druidInstance, result, trigger) + m.duration.DeleteLabelValues(namespace, druidInstance, result, trigger) + } + } +} + +func isActiveLifecyclePhase(phase v1alpha1.DeploymentLifecyclePhase) bool { + return phase == v1alpha1.DeploymentLifecyclePending || phase == v1alpha1.DeploymentLifecycleInProgress +} + +func isTerminalLifecyclePhase(phase v1alpha1.DeploymentLifecyclePhase) bool { + return phase == v1alpha1.DeploymentLifecycleSucceeded || phase == v1alpha1.DeploymentLifecycleFailed +} + +func lifecyclePhaseLabel(phase v1alpha1.DeploymentLifecyclePhase) string { + switch phase { + case v1alpha1.DeploymentLifecyclePending: + return "pending" + case v1alpha1.DeploymentLifecycleInProgress: + return "in_progress" + case v1alpha1.DeploymentLifecycleSucceeded: + return "succeeded" + case v1alpha1.DeploymentLifecycleFailed: + return "failed" + default: + return "unknown" + } +} + +func lifecycleResultLabel(phase v1alpha1.DeploymentLifecyclePhase) string { + switch phase { + case v1alpha1.DeploymentLifecycleSucceeded: + return "succeeded" + case v1alpha1.DeploymentLifecycleFailed: + return "failed" + default: + return "unknown" + } +} + +func lifecycleTriggerLabel(trigger v1alpha1.DeploymentLifecycleTrigger) string { + switch trigger { + case v1alpha1.DeploymentTriggerSpecChange: + return "spec_change" + case v1alpha1.DeploymentTriggerImageChange: + return "image_change" + case v1alpha1.DeploymentTriggerManualRollout: + return "manual_rollout" + default: + return "unknown" + } +} + +type druidRolloutMetrics struct { + clusterFullyDeployed *prometheus.GaugeVec + clusterPhase *prometheus.GaugeVec + + workloadRolloutInProgress *prometheus.GaugeVec + workloadRolloutDuration *prometheus.GaugeVec + workloadDesiredReplicas *prometheus.GaugeVec + workloadUpdatedReplicas *prometheus.GaugeVec + workloadReadyReplicas *prometheus.GaugeVec + workloadObservedGeneration *prometheus.GaugeVec + workloadGeneration *prometheus.GaugeVec + workloadForceDeleteHealthy *prometheus.GaugeVec + workloadForceDeleteActions *prometheus.CounterVec + + mu sync.Mutex + activeRolloutStarts map[string]time.Time + knownWorkloads map[string]map[string]workloadMetricLabels + knownForceDeleteChecks map[string]map[string]workloadMetricLabels + knownForceDeleteActions map[string]map[string]forceDeleteActionMetricLabels +} + +type workloadMetricLabels struct { + namespace string + druidInstance string + workload string + nodeType string + kind string +} + +type workloadRolloutSnapshot struct { + labels workloadMetricLabels + desiredReplicas int32 + updatedReplicas int32 + readyReplicas int32 + observedGeneration int64 + generation int64 + inProgress bool +} + +type forceDeleteCandidate struct { + podName string + containerName string + reason string +} + +type forceDeleteActionMetricLabels struct { + namespace string + druidInstance string + workload string + nodeType string + reason string +} + +var deploymentLifecyclePhases = []string{"pending", "in_progress", "succeeded", "failed", "unknown"} + +var defaultDruidRolloutMetrics = newDruidRolloutMetrics(ctrlmetrics.Registry) + +func newDruidRolloutMetrics(registerer prometheus.Registerer) *druidRolloutMetrics { + metrics := &druidRolloutMetrics{ + clusterFullyDeployed: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_cluster_fully_deployed", + Help: "Whether the Druid cluster deployment lifecycle is fully deployed.", + }, + []string{"namespace", "druid_instance"}, + ), + clusterPhase: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_cluster_deployment_phase", + Help: "One-hot gauge for the current Druid cluster deployment lifecycle phase.", + }, + []string{"namespace", "druid_instance", "phase"}, + ), + workloadRolloutInProgress: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_workload_rollout_in_progress", + Help: "Whether a managed Druid workload is currently rolling out.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "kind"}, + ), + workloadRolloutDuration: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_workload_rollout_duration_seconds", + Help: "Last observed duration of the current managed Druid workload rollout at reconcile time.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "kind"}, + ), + workloadDesiredReplicas: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_workload_desired_replicas", + Help: "Desired replicas for a managed Druid workload.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "kind"}, + ), + workloadUpdatedReplicas: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_workload_updated_replicas", + Help: "Updated replicas reported by a managed Druid workload.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "kind"}, + ), + workloadReadyReplicas: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_workload_ready_replicas", + Help: "Ready replicas reported by a managed Druid workload.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "kind"}, + ), + workloadObservedGeneration: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_workload_observed_generation", + Help: "Observed generation reported by a managed Druid workload controller.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "kind"}, + ), + workloadGeneration: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_workload_generation", + Help: "Metadata generation for a managed Druid workload.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "kind"}, + ), + workloadForceDeleteHealthy: prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "druid_operator_workload_force_delete_healthcheck_healthy", + Help: "Whether the OrderedReady StatefulSet force-delete health check is currently passing.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "kind"}, + ), + workloadForceDeleteActions: prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: "druid_operator_workload_force_delete_actions_total", + Help: "Total number of pod force-delete remediation actions taken by the operator.", + }, + []string{"namespace", "druid_instance", "workload", "node_type", "reason"}, + ), + activeRolloutStarts: map[string]time.Time{}, + knownWorkloads: map[string]map[string]workloadMetricLabels{}, + knownForceDeleteChecks: map[string]map[string]workloadMetricLabels{}, + knownForceDeleteActions: map[string]map[string]forceDeleteActionMetricLabels{}, + } + + if registerer != nil { + registerer.MustRegister( + metrics.clusterFullyDeployed, + metrics.clusterPhase, + metrics.workloadRolloutInProgress, + metrics.workloadRolloutDuration, + metrics.workloadDesiredReplicas, + metrics.workloadUpdatedReplicas, + metrics.workloadReadyReplicas, + metrics.workloadObservedGeneration, + metrics.workloadGeneration, + metrics.workloadForceDeleteHealthy, + metrics.workloadForceDeleteActions, + ) + } + + return metrics +} + +func (m *druidRolloutMetrics) sync(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid) error { + return m.syncWithNow(ctx, sdk, drd, time.Now()) +} + +func (m *druidRolloutMetrics) syncWithNow(ctx context.Context, sdk client.Client, drd *v1alpha1.Druid, now time.Time) error { + if m == nil || drd == nil { + return nil + } + + workloadSnapshots, err := m.collectWorkloadSnapshots(ctx, sdk, drd) + if err != nil { + return err + } + forceDeleteChecks, err := m.collectForceDeleteChecks(ctx, sdk, drd) + if err != nil { + return err + } + + m.mu.Lock() + defer m.mu.Unlock() + + clusterKey := metricClusterKey(drd.Namespace, drd.Name) + m.updateClusterState(drd) + + currentWorkloads := map[string]workloadMetricLabels{} + for workloadKey, snapshot := range workloadSnapshots { + currentWorkloads[workloadKey] = snapshot.labels + m.updateWorkloadSnapshot(workloadKey, snapshot, now) + } + + for workloadKey, labels := range m.knownWorkloads[clusterKey] { + if _, exists := currentWorkloads[workloadKey]; exists { + continue + } + m.deleteWorkloadMetrics(workloadKey, labels) + } + m.knownWorkloads[clusterKey] = currentWorkloads + + currentChecks := map[string]workloadMetricLabels{} + for workloadKey, check := range forceDeleteChecks { + currentChecks[workloadKey] = check.labels + m.workloadForceDeleteHealthy.WithLabelValues(check.labels.values()...).Set(boolToFloat(check.healthy)) + } + for workloadKey, labels := range m.knownForceDeleteChecks[clusterKey] { + if _, exists := currentChecks[workloadKey]; exists { + continue + } + m.workloadForceDeleteHealthy.DeleteLabelValues(labels.values()...) + } + m.knownForceDeleteChecks[clusterKey] = currentChecks + + return nil +} + +func (m *druidRolloutMetrics) deleteCluster(namespace, druidName string) { + if m == nil { + return + } + + m.mu.Lock() + defer m.mu.Unlock() + + clusterKey := metricClusterKey(namespace, druidName) + m.clusterFullyDeployed.DeleteLabelValues(namespace, druidName) + for _, phase := range deploymentLifecyclePhases { + m.clusterPhase.DeleteLabelValues(namespace, druidName, phase) + } + + for workloadKey, labels := range m.knownWorkloads[clusterKey] { + m.deleteWorkloadMetrics(workloadKey, labels) + } + delete(m.knownWorkloads, clusterKey) + + for _, labels := range m.knownForceDeleteChecks[clusterKey] { + m.workloadForceDeleteHealthy.DeleteLabelValues(labels.values()...) + } + delete(m.knownForceDeleteChecks, clusterKey) + + for _, labels := range m.knownForceDeleteActions[clusterKey] { + m.workloadForceDeleteActions.DeleteLabelValues(labels.values()...) + } + delete(m.knownForceDeleteActions, clusterKey) +} + +func (m *druidRolloutMetrics) recordForceDeleteAction( + namespace, druidInstance, workload, nodeType, reason string, +) { + if m == nil { + return + } + + labels := forceDeleteActionMetricLabels{ + namespace: namespace, + druidInstance: druidInstance, + workload: workload, + nodeType: nodeType, + reason: reason, + } + + m.mu.Lock() + clusterKey := metricClusterKey(namespace, druidInstance) + if _, exists := m.knownForceDeleteActions[clusterKey]; !exists { + m.knownForceDeleteActions[clusterKey] = map[string]forceDeleteActionMetricLabels{} + } + m.knownForceDeleteActions[clusterKey][metricForceDeleteActionKey(labels)] = labels + m.mu.Unlock() + + m.workloadForceDeleteActions.WithLabelValues(labels.values()...).Inc() +} + +func (m *druidRolloutMetrics) updateClusterState(drd *v1alpha1.Druid) { + phase := lifecyclePhaseLabel(drd.Status.DeploymentLifecycle.Phase) + for _, candidate := range deploymentLifecyclePhases { + value := 0.0 + if candidate == phase { + value = 1 + } + m.clusterPhase.WithLabelValues(drd.Namespace, drd.Name, candidate).Set(value) + } + + m.clusterFullyDeployed.WithLabelValues(drd.Namespace, drd.Name).Set(boolToFloat( + drd.Status.DeploymentLifecycle.Phase == v1alpha1.DeploymentLifecycleSucceeded, + )) +} + +func (m *druidRolloutMetrics) updateWorkloadSnapshot( + workloadKey string, + snapshot workloadRolloutSnapshot, + now time.Time, +) { + values := snapshot.labels.values() + m.workloadDesiredReplicas.WithLabelValues(values...).Set(float64(snapshot.desiredReplicas)) + m.workloadUpdatedReplicas.WithLabelValues(values...).Set(float64(snapshot.updatedReplicas)) + m.workloadReadyReplicas.WithLabelValues(values...).Set(float64(snapshot.readyReplicas)) + m.workloadObservedGeneration.WithLabelValues(values...).Set(float64(snapshot.observedGeneration)) + m.workloadGeneration.WithLabelValues(values...).Set(float64(snapshot.generation)) + + if !snapshot.inProgress { + delete(m.activeRolloutStarts, workloadKey) + m.workloadRolloutInProgress.WithLabelValues(values...).Set(0) + m.workloadRolloutDuration.WithLabelValues(values...).Set(0) + return + } + + startedAt := m.rolloutStartTime(workloadKey, now) + durationSeconds := now.Sub(startedAt).Seconds() + if durationSeconds < 0 { + durationSeconds = 0 + } + m.workloadRolloutInProgress.WithLabelValues(values...).Set(1) + m.workloadRolloutDuration.WithLabelValues(values...).Set(durationSeconds) +} + +func (m *druidRolloutMetrics) rolloutStartTime(workloadKey string, now time.Time) time.Time { + startedAt, exists := m.activeRolloutStarts[workloadKey] + if exists { + return startedAt + } + + m.activeRolloutStarts[workloadKey] = now + return now +} + +func (m *druidRolloutMetrics) deleteWorkloadMetrics(workloadKey string, labels workloadMetricLabels) { + delete(m.activeRolloutStarts, workloadKey) + values := labels.values() + m.workloadRolloutInProgress.DeleteLabelValues(values...) + m.workloadRolloutDuration.DeleteLabelValues(values...) + m.workloadDesiredReplicas.DeleteLabelValues(values...) + m.workloadUpdatedReplicas.DeleteLabelValues(values...) + m.workloadReadyReplicas.DeleteLabelValues(values...) + m.workloadObservedGeneration.DeleteLabelValues(values...) + m.workloadGeneration.DeleteLabelValues(values...) +} + +func (m *druidRolloutMetrics) collectWorkloadSnapshots( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, +) (map[string]workloadRolloutSnapshot, error) { + listOpts := []client.ListOption{ + client.InNamespace(drd.Namespace), + client.MatchingLabels(makeLabelsForDruid(drd)), + } + + snapshots := map[string]workloadRolloutSnapshot{} + + deployments := &appsv1.DeploymentList{} + if err := sdk.List(ctx, deployments, listOpts...); err != nil { + return nil, err + } + for _, deployment := range deployments.Items { + snapshot := workloadRolloutSnapshot{ + labels: labelsForManagedObject(drd, &deployment, "Deployment"), + desiredReplicas: desiredReplicaCount(deployment.Spec.Replicas), + updatedReplicas: deployment.Status.UpdatedReplicas, + readyReplicas: deployment.Status.ReadyReplicas, + observedGeneration: deployment.Status.ObservedGeneration, + generation: deployment.Generation, + inProgress: deploymentRolloutInProgress(&deployment), + } + snapshots[metricWorkloadKey(snapshot.labels)] = snapshot + } + + statefulSets := &appsv1.StatefulSetList{} + if err := sdk.List(ctx, statefulSets, listOpts...); err != nil { + return nil, err + } + for _, statefulSet := range statefulSets.Items { + snapshot := workloadRolloutSnapshot{ + labels: labelsForManagedObject(drd, &statefulSet, "StatefulSet"), + desiredReplicas: desiredReplicaCount(statefulSet.Spec.Replicas), + updatedReplicas: statefulSet.Status.UpdatedReplicas, + readyReplicas: statefulSet.Status.ReadyReplicas, + observedGeneration: statefulSet.Status.ObservedGeneration, + generation: statefulSet.Generation, + inProgress: statefulSetRolloutInProgress(&statefulSet), + } + snapshots[metricWorkloadKey(snapshot.labels)] = snapshot + } + + return snapshots, nil +} + +type forceDeleteHealthCheck struct { + labels workloadMetricLabels + healthy bool +} + +func (m *druidRolloutMetrics) collectForceDeleteChecks( + ctx context.Context, + sdk client.Client, + drd *v1alpha1.Druid, +) (map[string]forceDeleteHealthCheck, error) { + checks := map[string]forceDeleteHealthCheck{} + if !drd.Spec.ForceDeleteStsPodOnError { + return checks, nil + } + + for key, nodeSpec := range drd.Spec.Nodes { + if nodeSpec.PodManagementPolicy != appsv1.OrderedReadyPodManagement { + continue + } + kind := firstNonEmptyStr(nodeSpec.Kind, "StatefulSet") + if kind != "StatefulSet" { + continue + } + + nodeSpecUniqueStr := makeNodeSpecificUniqueString(drd, key) + labels := workloadMetricLabels{ + namespace: drd.Namespace, + druidInstance: drd.Name, + workload: nodeSpecUniqueStr, + nodeType: nodeSpec.NodeType, + kind: "StatefulSet", + } + pods, err := listNodePods(ctx, sdk, &nodeSpec, drd, nodeSpecUniqueStr) + if err != nil { + return nil, err + } + checks[metricWorkloadKey(labels)] = forceDeleteHealthCheck{ + labels: labels, + healthy: len(findForceDeleteCandidates(pods)) == 0, + } + } + + return checks, nil +} + +func listNodePods( + ctx context.Context, + sdk client.Client, + nodeSpec *v1alpha1.DruidNodeSpec, + drd *v1alpha1.Druid, + nodeSpecUniqueStr string, +) ([]object, error) { + podList := &v1.PodList{} + if err := sdk.List(ctx, podList, + client.InNamespace(drd.Namespace), + client.MatchingLabels(makeLabelsForNodeSpec(nodeSpec, drd, drd.Name, nodeSpecUniqueStr)), + ); err != nil { + return nil, err + } + + result := make([]object, len(podList.Items)) + for i := range podList.Items { + result[i] = &podList.Items[i] + } + return result, nil +} + +func findForceDeleteCandidates(podList []object) []forceDeleteCandidate { + candidates := []forceDeleteCandidate{} + + for _, podObj := range podList { + pod := podObj.(*v1.Pod) + if pod.Status.Phase == v1.PodFailed || pod.Status.Phase == v1.PodUnknown { + candidates = append(candidates, forceDeleteCandidate{ + podName: pod.Name, + reason: "pod_phase", + }) + continue + } + + containersReadyFalse := false + for _, condition := range pod.Status.Conditions { + if condition.Type == v1.ContainersReady && condition.Status == v1.ConditionFalse { + containersReadyFalse = true + break + } + } + if !containersReadyFalse { + continue + } + + for _, containerStatus := range pod.Status.ContainerStatuses { + if containerStatus.RestartCount > 1 { + candidates = append(candidates, forceDeleteCandidate{ + podName: pod.Name, + containerName: containerStatus.Name, + reason: "crashloop", + }) + break + } + } + } + + return candidates +} + +func labelsForManagedObject( + drd *v1alpha1.Druid, + obj metav1.Object, + kind string, +) workloadMetricLabels { + nodeType := obj.GetLabels()["component"] + if nodeType == "" { + nodeType = "unknown" + } + + return workloadMetricLabels{ + namespace: drd.Namespace, + druidInstance: drd.Name, + workload: obj.GetName(), + nodeType: nodeType, + kind: kind, + } +} + +func (l workloadMetricLabels) values() []string { + return []string{l.namespace, l.druidInstance, l.workload, l.nodeType, l.kind} +} + +func (l forceDeleteActionMetricLabels) values() []string { + return []string{l.namespace, l.druidInstance, l.workload, l.nodeType, l.reason} +} + +func metricClusterKey(namespace, druidInstance string) string { + return fmt.Sprintf("%s/%s", namespace, druidInstance) +} + +func metricWorkloadKey(labels workloadMetricLabels) string { + return fmt.Sprintf("%s/%s/%s/%s", labels.namespace, labels.druidInstance, labels.kind, labels.workload) +} + +func metricForceDeleteActionKey(labels forceDeleteActionMetricLabels) string { + return fmt.Sprintf("%s/%s/%s/%s/%s", labels.namespace, labels.druidInstance, labels.workload, labels.nodeType, labels.reason) +} + +func boolToFloat(value bool) float64 { + if value { + return 1 + } + return 0 +} + +func desiredReplicaCount(replicas *int32) int32 { + if replicas == nil { + return 1 + } + return *replicas +} + +func deploymentRolloutInProgress(deployment *appsv1.Deployment) bool { + desiredReplicas := desiredReplicaCount(deployment.Spec.Replicas) + return deployment.Status.ObservedGeneration < deployment.Generation || + deployment.Status.UpdatedReplicas != desiredReplicas || + deployment.Status.ReadyReplicas != desiredReplicas || + deployment.Status.Replicas != desiredReplicas || + deployment.Status.UnavailableReplicas != 0 +} + +func statefulSetRolloutInProgress(statefulSet *appsv1.StatefulSet) bool { + desiredReplicas := desiredReplicaCount(statefulSet.Spec.Replicas) + return statefulSet.Status.ObservedGeneration < statefulSet.Generation || + statefulSet.Status.CurrentRevision != statefulSet.Status.UpdateRevision || + statefulSet.Status.UpdatedReplicas != desiredReplicas || + statefulSet.Status.ReadyReplicas != desiredReplicas +} diff --git a/controllers/druid/rollout_metrics_test.go b/controllers/druid/rollout_metrics_test.go new file mode 100644 index 0000000..c213a6c --- /dev/null +++ b/controllers/druid/rollout_metrics_test.go @@ -0,0 +1,322 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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 druid + +import ( + "context" + "testing" + "time" + + "github.com/apache/druid-operator/apis/druid/v1alpha1" + "github.com/prometheus/client_golang/prometheus" + dto "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + appsv1 "k8s.io/api/apps/v1" + v1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +const ( + testNamespace = "default" + testCluster = "example" +) + +func TestDruidRolloutMetricsSyncExportsCurrentRolloutState(t *testing.T) { + registry := prometheus.NewRegistry() + metrics := newDruidRolloutMetrics(registry) + + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: testCluster, + Namespace: testNamespace, + }, + Spec: v1alpha1.DruidSpec{ + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "historicals": { + NodeType: "historical", + Kind: "StatefulSet", + Replicas: 3, + }, + "brokers": { + NodeType: "broker", + Kind: "Deployment", + Replicas: 2, + }, + }, + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleInProgress, + }, + }, + } + + historicalLabels := map[string]string{ + "app": "druid", + "druid_cr": drd.Name, + "nodeSpecUniqueStr": "druid-example-historicals", + "component": "historical", + } + brokerLabels := map[string]string{ + "app": "druid", + "druid_cr": drd.Name, + "nodeSpecUniqueStr": "druid-example-brokers", + "component": "broker", + } + + statefulSet := &appsv1.StatefulSet{ + ObjectMeta: metav1.ObjectMeta{ + Name: "druid-example-historicals", + Namespace: drd.Namespace, + Generation: 7, + Labels: historicalLabels, + }, + Spec: appsv1.StatefulSetSpec{ + Replicas: int32Ptr(3), + }, + Status: appsv1.StatefulSetStatus{ + ObservedGeneration: 7, + CurrentRevision: "old", + UpdateRevision: "new", + UpdatedReplicas: 1, + ReadyReplicas: 1, + }, + } + deployment := &appsv1.Deployment{ + ObjectMeta: metav1.ObjectMeta{ + Name: "druid-example-brokers", + Namespace: drd.Namespace, + Generation: 4, + Labels: brokerLabels, + }, + Spec: appsv1.DeploymentSpec{ + Replicas: int32Ptr(2), + }, + Status: appsv1.DeploymentStatus{ + ObservedGeneration: 4, + UpdatedReplicas: 2, + ReadyReplicas: 2, + Replicas: 2, + UnavailableReplicas: 0, + }, + } + + k8sClient := newLifecycleTestClient(t, drd, statefulSet, deployment) + + require.NoError(t, metrics.syncWithNow(context.Background(), k8sClient, drd, time.Unix(100, 0))) + require.NoError(t, metrics.syncWithNow(context.Background(), k8sClient, drd, time.Unix(145, 0))) + + families, err := registry.Gather() + require.NoError(t, err) + + assertMetricValue(t, families, "druid_operator_cluster_fully_deployed", clusterMetricLabels(), 0.0) + assertMetricValue(t, families, "druid_operator_cluster_deployment_phase", clusterMetricLabels("phase", "in_progress"), 1.0) + assertMetricValue(t, families, "druid_operator_workload_rollout_in_progress", workloadMetricLabelSet("druid-example-historicals", "historical", "StatefulSet"), 1.0) + assertMetricValue(t, families, "druid_operator_workload_rollout_duration_seconds", workloadMetricLabelSet("druid-example-historicals", "historical", "StatefulSet"), 45.0) + assertMetricValue(t, families, "druid_operator_workload_desired_replicas", workloadMetricLabelSet("druid-example-historicals", "historical", "StatefulSet"), 3.0) + assertMetricValue(t, families, "druid_operator_workload_updated_replicas", workloadMetricLabelSet("druid-example-historicals", "historical", "StatefulSet"), 1.0) + assertMetricValue(t, families, "druid_operator_workload_ready_replicas", workloadMetricLabelSet("druid-example-historicals", "historical", "StatefulSet"), 1.0) + assertMetricValue(t, families, "druid_operator_workload_rollout_in_progress", workloadMetricLabelSet("druid-example-brokers", "broker", "Deployment"), 0.0) +} + +func TestDruidRolloutMetricsSyncRestartsDurationForNewWorkloadIdentity(t *testing.T) { + registry := prometheus.NewRegistry() + metrics := newDruidRolloutMetrics(registry) + + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: testCluster, + Namespace: testNamespace, + }, + Status: v1alpha1.DruidClusterStatus{ + DeploymentLifecycle: v1alpha1.DeploymentLifecycleStatus{ + Phase: v1alpha1.DeploymentLifecycleInProgress, + }, + }, + } + + first := &appsv1.StatefulSet{ + ObjectMeta: metav1.ObjectMeta{ + Name: "druid-example-historicals-v1", + Namespace: drd.Namespace, + Generation: 1, + Labels: map[string]string{ + "app": "druid", + "druid_cr": drd.Name, + "nodeSpecUniqueStr": "druid-example-historicals-v1", + "component": "historical", + }, + }, + Spec: appsv1.StatefulSetSpec{ + Replicas: int32Ptr(2), + }, + Status: appsv1.StatefulSetStatus{ + ObservedGeneration: 1, + CurrentRevision: "old", + UpdateRevision: "new", + UpdatedReplicas: 1, + ReadyReplicas: 1, + }, + } + second := first.DeepCopy() + second.Name = "druid-example-historicals-v2" + second.Labels["nodeSpecUniqueStr"] = "druid-example-historicals-v2" + + k8sClient := newLifecycleTestClient(t, drd, first) + require.NoError(t, metrics.syncWithNow(context.Background(), k8sClient, drd, time.Unix(100, 0))) + require.NoError(t, metrics.syncWithNow(context.Background(), k8sClient, drd, time.Unix(145, 0))) + + k8sClient = newLifecycleTestClient(t, drd, second) + require.NoError(t, metrics.syncWithNow(context.Background(), k8sClient, drd, time.Unix(200, 0))) + + families, err := registry.Gather() + require.NoError(t, err) + + assertMetricValue(t, families, "druid_operator_workload_rollout_duration_seconds", workloadMetricLabelSet("druid-example-historicals-v1", "historical", "StatefulSet"), 0.0) + assertMetricValue(t, families, "druid_operator_workload_rollout_duration_seconds", workloadMetricLabelSet("druid-example-historicals-v2", "historical", "StatefulSet"), 0.0) + assertMetricValue(t, families, "druid_operator_workload_rollout_in_progress", workloadMetricLabelSet("druid-example-historicals-v2", "historical", "StatefulSet"), 1.0) +} + +func TestDruidRolloutMetricsSyncExportsForceDeleteHealthAndActions(t *testing.T) { + registry := prometheus.NewRegistry() + metrics := newDruidRolloutMetrics(registry) + + drd := &v1alpha1.Druid{ + ObjectMeta: metav1.ObjectMeta{ + Name: testCluster, + Namespace: testNamespace, + }, + Spec: v1alpha1.DruidSpec{ + ForceDeleteStsPodOnError: true, + Nodes: map[string]v1alpha1.DruidNodeSpec{ + "historicals": { + NodeType: "historical", + Kind: "StatefulSet", + Replicas: 1, + PodManagementPolicy: appsv1.OrderedReadyPodManagement, + }, + }, + }, + } + + nodeSpec := drd.Spec.Nodes["historicals"] + nodeSpecUniqueStr := makeNodeSpecificUniqueString(drd, "historicals") + labels := makeLabelsForNodeSpec(&nodeSpec, drd, drd.Name, nodeSpecUniqueStr) + + statefulSet := &appsv1.StatefulSet{ + ObjectMeta: metav1.ObjectMeta{ + Name: nodeSpecUniqueStr, + Namespace: drd.Namespace, + Labels: labels, + }, + Spec: appsv1.StatefulSetSpec{ + Replicas: int32Ptr(1), + }, + Status: appsv1.StatefulSetStatus{ + ObservedGeneration: 1, + CurrentRevision: "rev", + UpdateRevision: "rev", + UpdatedReplicas: 1, + ReadyReplicas: 1, + }, + } + pod := &v1.Pod{ + ObjectMeta: metav1.ObjectMeta{ + Name: nodeSpecUniqueStr + "-0", + Namespace: drd.Namespace, + Labels: labels, + }, + Status: v1.PodStatus{ + Phase: v1.PodRunning, + Conditions: []v1.PodCondition{{ + Type: v1.ContainersReady, + Status: v1.ConditionFalse, + }}, + ContainerStatuses: []v1.ContainerStatus{{ + Name: "druid", + RestartCount: 3, + }}, + }, + } + + k8sClient := newLifecycleTestClient(t, drd, statefulSet, pod) + + require.NoError(t, metrics.syncWithNow(context.Background(), k8sClient, drd, time.Unix(200, 0))) + require.NoError(t, checkCrashStatusWithMetrics(context.Background(), k8sClient, &nodeSpec, drd, nodeSpecUniqueStr, noopEventEmitter{}, metrics)) + + families, err := registry.Gather() + require.NoError(t, err) + + assertMetricValue(t, families, "druid_operator_workload_force_delete_healthcheck_healthy", workloadMetricLabelSet(nodeSpecUniqueStr, "historical", "StatefulSet"), 0.0) + assertMetricValue(t, families, "druid_operator_workload_force_delete_actions_total", forceDeleteActionLabels(nodeSpecUniqueStr, "historical", "crashloop"), 1.0) +} + +func TestDruidRolloutMetricsDeleteClusterRemovesForceDeleteActionSeries(t *testing.T) { + registry := prometheus.NewRegistry() + metrics := newDruidRolloutMetrics(registry) + + metrics.recordForceDeleteAction(testNamespace, testCluster, "druid-example-historicals", "historical", "crashloop") + + beforeFamilies, err := registry.Gather() + require.NoError(t, err) + assertMetricValue(t, beforeFamilies, "druid_operator_workload_force_delete_actions_total", forceDeleteActionLabels("druid-example-historicals", "historical", "crashloop"), 1.0) + + metrics.deleteCluster(testNamespace, testCluster) + + afterFamilies, err := registry.Gather() + require.NoError(t, err) + assertMetricValue(t, afterFamilies, "druid_operator_workload_force_delete_actions_total", forceDeleteActionLabels("druid-example-historicals", "historical", "crashloop"), 0.0) +} + +func int32Ptr(value int32) *int32 { + return &value +} + +func assertMetricValue(t *testing.T, families []*dto.MetricFamily, name string, labels map[string]string, want float64) { + t.Helper() + assert.Equal(t, want, metricValue(t, families, name, labels)) +} + +func clusterMetricLabels(extra ...string) map[string]string { + labels := map[string]string{ + "namespace": testNamespace, + "druid_instance": testCluster, + } + for i := 0; i+1 < len(extra); i += 2 { + labels[extra[i]] = extra[i+1] + } + return labels +} + +func workloadMetricLabelSet(workload, nodeType, kind string) map[string]string { + labels := clusterMetricLabels() + labels["workload"] = workload + labels["node_type"] = nodeType + labels["kind"] = kind + return labels +} + +func forceDeleteActionLabels(workload, nodeType, reason string) map[string]string { + labels := clusterMetricLabels() + labels["workload"] = workload + labels["node_type"] = nodeType + labels["reason"] = reason + return labels +} diff --git a/controllers/druid/suite_test.go b/controllers/druid/suite_test.go index 81d311b..0b6f8a9 100644 --- a/controllers/druid/suite_test.go +++ b/controllers/druid/suite_test.go @@ -99,7 +99,8 @@ var _ = BeforeSuite(func() { Expect(k8sClient).NotTo(BeNil()) k8sManager, err := ctrl.NewManager(cfg, ctrl.Options{ - Scheme: scheme.Scheme, + Scheme: scheme.Scheme, + MetricsBindAddress: ":9090", }) Expect(err).ToNot(HaveOccurred()) diff --git a/controllers/ingestion/reconciler.go b/controllers/ingestion/reconciler.go index 09f5eb9..7b948ea 100644 --- a/controllers/ingestion/reconciler.go +++ b/controllers/ingestion/reconciler.go @@ -63,12 +63,12 @@ func (r *DruidIngestionReconciler) do(ctx context.Context, di *v1alpha1.DruidIng return err } - svcName, err := druidapi.GetRouterSvcUrl(di.Namespace, di.Spec.DruidClusterName, r.Client) + svcName, err := druidapi.GetRouterSvcUrl(ctx, di.Namespace, di.Spec.DruidClusterName, r.Client) if err != nil { return err } - _, err = r.CreateOrUpdate(di, svcName, internalhttp.Auth{BasicAuth: basicAuth}) + _, err = r.CreateOrUpdate(ctx, di, svcName, internalhttp.Auth{BasicAuth: basicAuth}) if err != nil { return err } @@ -80,13 +80,13 @@ func (r *DruidIngestionReconciler) do(ctx context.Context, di *v1alpha1.DruidIng if !controllerutil.ContainsFinalizer(di, DruidIngestionControllerFinalizer) { controllerutil.AddFinalizer(di, DruidIngestionControllerFinalizer) if err := r.Update(ctx, di.DeepCopyObject().(*v1alpha1.DruidIngestion)); err != nil { - return nil + return err } } } else { if controllerutil.ContainsFinalizer(di, DruidIngestionControllerFinalizer) { // our finalizer is present, so lets handle any external dependency - svcName, err := druidapi.GetRouterSvcUrl(di.Namespace, di.Spec.DruidClusterName, r.Client) + svcName, err := druidapi.GetRouterSvcUrl(ctx, di.Namespace, di.Spec.DruidClusterName, r.Client) if err != nil { return err } @@ -96,9 +96,15 @@ func (r *DruidIngestionReconciler) do(ctx context.Context, di *v1alpha1.DruidIng &internalhttp.Auth{BasicAuth: basicAuth}, ) + shutdownPath, err := getPath(di.Spec.Ingestion.Type, svcName, http.MethodPost, di.Status.TaskId, true) + if err != nil { + return err + } + respShutDownTask, err := posthttp.Do( + ctx, http.MethodPost, - getPath(di.Spec.Ingestion.Type, svcName, http.MethodPost, di.Status.TaskId, true), + shutdownPath, []byte{}, ) if err != nil { @@ -123,7 +129,7 @@ func (r *DruidIngestionReconciler) do(ctx context.Context, di *v1alpha1.DruidIng // remove our finalizer from the list and update it. controllerutil.RemoveFinalizer(di, DruidIngestionControllerFinalizer) if err := r.Update(ctx, di.DeepCopyObject().(*v1alpha1.DruidIngestion)); err != nil { - return nil + return err } } } @@ -242,6 +248,7 @@ func getCompactionJson(di *v1alpha1.DruidIngestion) (string, error) { // UpdateCompaction updates the compaction settings for a Druid data source. func (r *DruidIngestionReconciler) UpdateCompaction( + ctx context.Context, di *v1alpha1.DruidIngestion, svcName string, auth internalhttp.Auth, @@ -261,10 +268,16 @@ func (r *DruidIngestionReconciler) UpdateCompaction( return false, err } + currentCompactionPath, err := druidapi.MakePath(svcName, "coordinator", "config", "compaction", dataSource) + if err != nil { + return false, err + } + // Get current compaction settings currentResp, err := httpClient.Do( + ctx, http.MethodGet, - druidapi.MakePath(svcName, "coordinator", "config", "compaction", dataSource), + currentCompactionPath, nil, ) if err != nil { @@ -294,10 +307,16 @@ func (r *DruidIngestionReconciler) UpdateCompaction( return false, nil } + updateCompactionPath, err := druidapi.MakePath(svcName, "coordinator", "config", "compaction") + if err != nil { + return false, err + } + // Update compaction settings respUpdateCompaction, err := httpClient.Do( + ctx, http.MethodPost, - druidapi.MakePath(svcName, "coordinator", "config", "compaction"), + updateCompactionPath, []byte(desiredCompactionJson), ) if err != nil { @@ -315,6 +334,7 @@ func (r *DruidIngestionReconciler) UpdateCompaction( // UpdateRules updates the rules for a Druid data source. func (r *DruidIngestionReconciler) UpdateRules( + ctx context.Context, di *v1alpha1.DruidIngestion, svcName string, auth internalhttp.Auth, @@ -338,10 +358,16 @@ func (r *DruidIngestionReconciler) UpdateRules( return false, err } + rulesPath, err := druidapi.MakePath(svcName, "coordinator", "rules", dataSource) + if err != nil { + return false, err + } + // Update rules respUpdateRules, err := postHttp.Do( + ctx, http.MethodPost, - druidapi.MakePath(svcName, "coordinator", "rules", dataSource), + rulesPath, []byte(rulesData), ) @@ -357,6 +383,7 @@ func (r *DruidIngestionReconciler) UpdateRules( } func (r *DruidIngestionReconciler) CreateOrUpdate( + ctx context.Context, di *v1alpha1.DruidIngestion, svcName string, auth internalhttp.Auth, @@ -376,10 +403,16 @@ func (r *DruidIngestionReconciler) CreateOrUpdate( &auth, ) + createPath, err := getPath(di.Spec.Ingestion.Type, svcName, http.MethodPost, "", false) + if err != nil { + return controllerutil.OperationResultNone, err + } + // Create ingestion task respCreateTask, err := postHttp.Do( + ctx, http.MethodPost, - getPath(di.Spec.Ingestion.Type, svcName, http.MethodPost, "", false), + createPath, []byte(specJson), ) @@ -387,12 +420,12 @@ func (r *DruidIngestionReconciler) CreateOrUpdate( return controllerutil.OperationResultNone, err } - compactionOk, err := r.UpdateCompaction(di, svcName, auth) + compactionOk, err := r.UpdateCompaction(ctx, di, svcName, auth) if err != nil { return controllerutil.OperationResultNone, err } - rulesOk, err := r.UpdateRules(di, svcName, auth) + rulesOk, err := r.UpdateRules(ctx, di, svcName, auth) if err != nil { return controllerutil.OperationResultNone, err } @@ -469,9 +502,15 @@ func (r *DruidIngestionReconciler) CreateOrUpdate( &auth, ) + updatePath, err := getPath(di.Spec.Ingestion.Type, svcName, http.MethodPost, "", false) + if err != nil { + return controllerutil.OperationResultNone, err + } + respUpdateSpec, err := postHttp.Do( + ctx, http.MethodPost, - getPath(di.Spec.Ingestion.Type, svcName, http.MethodPost, "", false), + updatePath, []byte(specJson), ) if err != nil { @@ -510,7 +549,7 @@ func (r *DruidIngestionReconciler) CreateOrUpdate( } - compactionOk, err := r.UpdateCompaction(di, svcName, auth) + compactionOk, err := r.UpdateCompaction(ctx, di, svcName, auth) if err != nil { return controllerutil.OperationResultNone, err } @@ -540,7 +579,7 @@ func (r *DruidIngestionReconciler) CreateOrUpdate( rulesEqual := reflect.DeepEqual(di.Status.CurrentRules, di.Spec.Ingestion.Rules) if !rulesEqual { - rulesOk, err := r.UpdateRules(di, svcName, auth) + rulesOk, err := r.UpdateRules(ctx, di, svcName, auth) if err != nil { return controllerutil.OperationResultNone, err } @@ -610,7 +649,7 @@ func (r *DruidIngestionReconciler) makePatchDruidIngestionStatus( func getPath( ingestionType v1alpha1.DruidIngestionMethod, svcName, httpMethod, taskId string, - shutDownTask bool) string { + shutDownTask bool) (string, error) { switch ingestionType { case v1alpha1.NativeBatchIndexParallel: @@ -639,10 +678,10 @@ func getPath( case v1alpha1.Kinesis: case v1alpha1.QueryControllerSQL: default: - return "" + return "", fmt.Errorf("unsupported ingestion type %q", ingestionType) } - return "" + return "", fmt.Errorf("unsupported request for ingestion type %q with method %q", ingestionType, httpMethod) } type taskHolder struct { diff --git a/controllers/ingestion/reconciler_test.go b/controllers/ingestion/reconciler_test.go index b55beb3..d093638 100644 --- a/controllers/ingestion/reconciler_test.go +++ b/controllers/ingestion/reconciler_test.go @@ -19,6 +19,7 @@ under the License. package ingestion import ( + "context" "net/http" "net/http/httptest" "testing" @@ -179,7 +180,7 @@ func TestUpdateCompaction_Success(t *testing.T) { r := &DruidIngestionReconciler{} // Call UpdateCompaction - success, err := r.UpdateCompaction(di, server.URL, auth) + success, err := r.UpdateCompaction(context.Background(), di, server.URL, auth) if err != nil { t.Fatalf("expected no error, got %v", err) @@ -226,7 +227,7 @@ func TestUpdateCompaction_Failure(t *testing.T) { r := &DruidIngestionReconciler{} // Call UpdateCompaction - success, err := r.UpdateCompaction(di, server.URL, auth) + success, err := r.UpdateCompaction(context.Background(), di, server.URL, auth) if err == nil { t.Fatalf("expected error, got nil") @@ -469,10 +470,13 @@ func TestGetPath(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - actual := getPath(tt.ingestionType, tt.svcName, tt.httpMethod, tt.taskId, tt.shutDownTask) - if actual != tt.expected { - t.Errorf("getPath() = %v, expected %v", actual, tt.expected) + actual, err := getPath(tt.ingestionType, tt.svcName, tt.httpMethod, tt.taskId, tt.shutDownTask) + if tt.expected == "" { + assert.Error(t, err) + return } + assert.NoError(t, err) + assert.Equal(t, tt.expected, actual) }) } } diff --git a/docs/api_specifications/druid.md b/docs/api_specifications/druid.md index 687e50f..f3ad01e 100644 --- a/docs/api_specifications/druid.md +++ b/docs/api_specifications/druid.md @@ -443,6 +443,30 @@ Kubernetes core/v1.PullPolicy
forceRedeployTokenForceRedeployToken forces a rollout when changed, even if the image tag is unchanged. This is primarily intended for mutable-tag redeploys driven by external automation and is classified as a ManualRollout in status.deploymentLifecycle.trigger.
expectedBuildRevisionExpectedBuildRevision is the Druid runtime build identifier that must be observed across live servers before an image or manual deployment lifecycle is considered complete. The operator verifies this against sys.servers.build_revision when available and falls back to sys.servers.version for Druid versions that do not expose build_revision. Changing this field alone does not create a new lifecycle revision; it updates the completion criteria for the current rollout.
envdeploymentLifecyclePipeline-facing deployment lifecycle status for the current Druid rollout.
Automation should treat observedGeneration, revision, and phase as the supported polling contract. Wait for observedGeneration to match the applied object generation, require a non-empty revision, then interpret phase as the rollout outcome.
Treat reason, expectedBuildRevision, and observedBuildRevisions as diagnostics. Failed is reserved for terminal or configuration failures; transient reconcile errors may leave the lifecycle in a non-terminal phase. Kubernetes Events emitted by the operator are observability only and are not the supported automation API.
druidNodeStatus+(Appears on: +DruidClusterStatus) +
+DeploymentLifecycleStatus describes the rollout currently tracked by the operator. Pipelines should use observedGeneration first and revision second when polling.
| Field | +Description | +
|---|---|
+revision+ +string + + |
+
+(Optional)
+ Identity of the currently tracked rollout. Pipelines should require a non-empty value before accepting terminal status. + |
+
+phase+ +string + + |
+
+(Optional)
+ Current lifecycle phase. |
+
+reason+ +string + + |
+
+(Optional)
+ Human-readable lifecycle detail for operators and pipeline diagnostics. + |
+
+observedGeneration+ +integer + + |
+
+(Optional)
+ Freshness key for automation. Pipelines should wait for this value to match the applied object generation before trusting the lifecycle phase. + |
+
+startedAt+ +string + + |
+
+(Optional)
+ Timestamp for when the currently tracked rollout began. + |
+
+completedAt+ +string + + |
+
+(Optional)
+ Timestamp for when the currently tracked rollout reached a terminal phase. + |
+
+trigger+ +string + + |
+
+(Optional)
+ Cause classification for the tracked rollout: |
+
+expectedBuildRevision+ +string + + |
+
+(Optional)
+ Requested runtime build identifier for image and manual rollouts. If it is missing or mismatched, the lifecycle remains non-terminal until corrected or timed out by the pipeline. Verification uses |
+
+observedBuildRevisions+ +[]string + + |
+
+(Optional)
+ Observed server build revisions during rollout verification. This is diagnostic data, not a primary polling key. + |
+
+lastSuccessfulImage+ +string + + |
+
+(Optional)
+ Operator-maintained bookkeeping from the most recent successful rollout. Not a primary pipeline gating field. + |
+
+lastSuccessfulImageInputsHash+ +string + + |
+
+(Optional)
+ Operator-maintained hash of the image inputs from the most recent successful rollout. Not a primary pipeline gating field. + |
+
+lastSuccessfulForceRedeployToken+ +string + + |
+
+(Optional)
+ Operator-maintained manual rollout token from the most recent successful rollout. Not a primary pipeline gating field. + |
+
Ingestion is the Schema for the Ingestion API
diff --git a/pkg/druidapi/druidapi.go b/pkg/druidapi/druidapi.go index 3cf4749..05c917f 100644 --- a/pkg/druidapi/druidapi.go +++ b/pkg/druidapi/druidapi.go @@ -20,15 +20,22 @@ package druidapi import ( "context" + "encoding/json" "errors" "fmt" + "net" + "net/http" "net/url" "path" + "sort" + "strings" + "time" internalhttp "github.com/apache/druid-operator/pkg/http" v1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/types" "sigs.k8s.io/controller-runtime/pkg/client" + logf "sigs.k8s.io/controller-runtime/pkg/log" ) const ( @@ -37,6 +44,60 @@ const ( OperatorPassword = "OperatorPassword" ) +type BuildRevisionLookupPendingReason string + +const ( + BuildRevisionPendingQueryServiceDiscovery BuildRevisionLookupPendingReason = "QueryServiceDiscovery" + BuildRevisionPendingQueryEndpoint BuildRevisionLookupPendingReason = "QueryEndpoint" + BuildRevisionPendingQueryResponse BuildRevisionLookupPendingReason = "QueryResponse" +) + +type buildRevisionLookupDependencies struct { + resolveQueryServiceURL func(context.Context, string, string, client.Client) (string, error) + newHTTPClient func(*internalhttp.Auth) internalhttp.DruidHTTP +} + +var logger = logf.Log.WithName("druidapi") + +func defaultBuildRevisionLookupDependencies() buildRevisionLookupDependencies { + return buildRevisionLookupDependencies{ + resolveQueryServiceURL: GetBrokerSvcUrl, + newHTTPClient: func(auth *internalhttp.Auth) internalhttp.DruidHTTP { + return internalhttp.NewHTTPClient(&http.Client{Timeout: 30 * time.Second}, auth) + }, + } +} + +type BuildRevisionLookupPendingError struct { + Reason BuildRevisionLookupPendingReason + Err error +} + +func (e *BuildRevisionLookupPendingError) Error() string { + message := "waiting for Druid build revision verification" + switch e.Reason { + case BuildRevisionPendingQueryServiceDiscovery: + message = "waiting for Druid query service discovery" + case BuildRevisionPendingQueryEndpoint: + message = "waiting for Druid build revision query endpoint" + case BuildRevisionPendingQueryResponse: + message = "waiting for Druid build revision query response" + } + if e.Err == nil { + return message + } + return fmt.Sprintf("%s: %v", message, e.Err) +} + +func (e *BuildRevisionLookupPendingError) Unwrap() error { + return e.Err +} + +func IsBuildRevisionLookupPending(err error) bool { + var pendingErr *BuildRevisionLookupPendingError + return errors.As(err, &pendingErr) +} + type AuthType string const ( @@ -57,16 +118,6 @@ type Auth struct { } // GetAuthCreds retrieves basic authentication credentials from a Kubernetes secret. -// If the Auth object is empty, it returns an empty BasicAuth object. -// Parameters: -// -// ctx: The context object. -// c: The Kubernetes client. -// auth: The Auth object containing the secret reference. -// -// Returns: -// -// BasicAuth: The basic authentication credentials, or an error if authentication retrieval fails. func GetAuthCreds( ctx context.Context, c client.Client, @@ -112,22 +163,11 @@ func GetAuthCreds( return internalhttp.BasicAuth{}, nil } -// MakePath constructs the appropriate path for the specified Druid API. -// Parameters: -// -// baseURL: The base URL of the Druid cluster. For example, http://router-svc.namespace.svc.cluster.local:8088. -// componentType: The type of Druid component. For example, "indexer". -// apiType: The type of Druid API. For example, "worker". -// additionalPaths: Additional path components to be appended to the URL. -// -// Returns: -// -// string: The constructed path. -func MakePath(baseURL, componentType, apiType string, additionalPaths ...string) string { +// MakePath builds a Druid API path from the supplied base URL and path components. +func MakePath(baseURL, componentType, apiType string, additionalPaths ...string) (string, error) { u, err := url.Parse(baseURL) if err != nil { - fmt.Println("Error parsing URL:", err) - return "" + return "", fmt.Errorf("parse Druid base URL %q: %w", baseURL, err) } // Construct the initial path @@ -138,42 +178,209 @@ func MakePath(baseURL, componentType, apiType string, additionalPaths ...string) u.Path = path.Join(u.Path, p) } - return u.String() + return u.String(), nil +} + +// GetRouterSvcUrl returns the URL of the Druid router service. +func GetRouterSvcUrl(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return getComponentSvcURL(ctx, namespace, druidClusterName, "router", c) +} + +// GetBrokerSvcUrl returns the URL of the Druid broker service. +func GetBrokerSvcUrl(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return getComponentSvcURL(ctx, namespace, druidClusterName, "broker", c) } -// GetRouterSvcUrl retrieves the URL of the Druid router service. -// Parameters: -// -// namespace: The namespace of the Druid cluster. -// druidClusterName: The name of the Druid cluster. -// c: The Kubernetes client. -// -// Returns: -// -// string: The URL of the Druid router service. -func GetRouterSvcUrl(namespace, druidClusterName string, c client.Client) (string, error) { +func getComponentSvcURL(ctx context.Context, namespace, druidClusterName, component string, c client.Client) (string, error) { listOpts := []client.ListOption{ client.InNamespace(namespace), client.MatchingLabels(map[string]string{ "druid_cr": druidClusterName, - "component": "router", + "component": component, }), } svcList := &v1.ServiceList{} - if err := c.List(context.Background(), svcList, listOpts...); err != nil { + if err := c.List(ctx, svcList, listOpts...); err != nil { return "", err } - var svcName string + if len(svcList.Items) == 0 { + return "", fmt.Errorf("%s svc discovery fail", component) + } + + svcName := svcList.Items[0].Name + newName := "http://" + svcName + "." + namespace + ":" + DruidRouterPort + + return newName, nil +} - for range svcList.Items { - svcName = svcList.Items[0].Name +type sqlQueryRequest struct { + Query string `json:"query"` +} + +type sqlServerBuildRevision struct { + BuildRevision string `json:"build_revision"` +} + +const ( + buildRevisionSQLQuery = "SELECT DISTINCT(build_revision) AS build_revision FROM sys.servers" + versionSQLQuery = "SELECT DISTINCT(version) AS build_revision FROM sys.servers" +) + +func GetClusterBuildRevisions( + ctx context.Context, + c client.Client, + namespace, druidClusterName string, + auth Auth, +) ([]string, error) { + return getClusterBuildRevisionsWithDeps( + ctx, + c, + namespace, + druidClusterName, + auth, + defaultBuildRevisionLookupDependencies(), + ) +} + +// GetClusterBuildRevisions returns the unique build revisions reported by live Druid servers. +func getClusterBuildRevisionsWithDeps( + ctx context.Context, + c client.Client, + namespace, druidClusterName string, + auth Auth, + deps buildRevisionLookupDependencies, +) ([]string, error) { + ctx, cancel := context.WithTimeout(ctx, 30*time.Second) + defer cancel() + + basicAuth, err := GetAuthCreds(ctx, c, auth) + if err != nil { + return nil, err } - if svcName == "" { - return "", errors.New("router svc discovery fail") + httpClient := deps.newHTTPClient(&internalhttp.Auth{BasicAuth: basicAuth}) + + svcURL, resolveErr := deps.resolveQueryServiceURL(ctx, namespace, druidClusterName, c) + if resolveErr != nil { + return nil, &BuildRevisionLookupPendingError{ + Reason: BuildRevisionPendingQueryServiceDiscovery, + Err: resolveErr, + } } - newName := "http://" + svcName + "." + namespace + ":" + DruidRouterPort + sqlPath, pathErr := MakeSQLPath(svcURL) + if pathErr != nil { + return nil, pathErr + } - return newName, nil + rows, err := executeBuildRevisionQuery(ctx, httpClient, sqlPath, buildRevisionSQLQuery) + if err == nil { + return normalizeBuildRevisions(rows), nil + } + if !isMissingBuildRevisionColumnError(err) { + return nil, err + } + + logger.Info( + "Falling back to sys.servers.version for deployment lifecycle build verification because sys.servers.build_revision is unavailable", + "namespace", namespace, + "druidInstance", druidClusterName, + ) + + rows, err = executeBuildRevisionQuery(ctx, httpClient, sqlPath, versionSQLQuery) + if err != nil { + return nil, err + } + return normalizeBuildRevisions(rows), nil +} + +func MakeSQLPath(baseURL string) (string, error) { + u, err := url.Parse(baseURL) + if err != nil { + return "", fmt.Errorf("parse Druid SQL base URL %q: %w", baseURL, err) + } + + u.Path = path.Join("druid", "v2", "sql") + return u.String(), nil +} + +func normalizeBuildRevisions(rows []sqlServerBuildRevision) []string { + set := make(map[string]struct{}) + for _, row := range rows { + trimmed := strings.TrimSpace(row.BuildRevision) + if trimmed == "" { + continue + } + set[trimmed] = struct{}{} + } + + revisions := make([]string, 0, len(set)) + for revision := range set { + revisions = append(revisions, revision) + } + sort.Strings(revisions) + return revisions +} + +func executeBuildRevisionQuery( + ctx context.Context, + httpClient internalhttp.DruidHTTP, + sqlPath, query string, +) ([]sqlServerBuildRevision, error) { + requestBody, err := json.Marshal(sqlQueryRequest{Query: query}) + if err != nil { + return nil, err + } + + response, requestErr := httpClient.Do(ctx, "POST", sqlPath, requestBody) + if requestErr != nil { + if isRetryableBuildRevisionRequestError(requestErr) { + return nil, &BuildRevisionLookupPendingError{ + Reason: BuildRevisionPendingQueryEndpoint, + Err: requestErr, + } + } + return nil, requestErr + } + + if response.StatusCode != http.StatusOK { + err := fmt.Errorf("status code: %d, response body: %s", response.StatusCode, response.ResponseBody) + if isRetryableBuildRevisionStatus(response.StatusCode) { + return nil, &BuildRevisionLookupPendingError{ + Reason: BuildRevisionPendingQueryResponse, + Err: err, + } + } + return nil, err + } + + var rows []sqlServerBuildRevision + if err := json.Unmarshal([]byte(response.ResponseBody), &rows); err != nil { + return nil, err + } + return rows, nil +} + +func isMissingBuildRevisionColumnError(err error) bool { + if err == nil { + return false + } + + message := strings.ToLower(err.Error()) + return strings.Contains(message, "build_revision") && strings.Contains(message, "not found") +} + +func isRetryableBuildRevisionRequestError(err error) bool { + if errors.Is(err, context.DeadlineExceeded) { + return true + } + + var netErr net.Error + return errors.As(err, &netErr) +} + +func isRetryableBuildRevisionStatus(statusCode int) bool { + return statusCode == http.StatusBadGateway || + statusCode == http.StatusServiceUnavailable || + statusCode == http.StatusGatewayTimeout } diff --git a/pkg/druidapi/druidapi_test.go b/pkg/druidapi/druidapi_test.go index ebcb6db..c373154 100644 --- a/pkg/druidapi/druidapi_test.go +++ b/pkg/druidapi/druidapi_test.go @@ -20,14 +20,30 @@ package druidapi import ( "context" + "errors" internalhttp "github.com/apache/druid-operator/pkg/http" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" v1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "sigs.k8s.io/controller-runtime/pkg/client" "sigs.k8s.io/controller-runtime/pkg/client/fake" "testing" ) +type fakeDruidHTTP struct { + response *internalhttp.Response + err error + doFunc func(ctx context.Context, method, url string, body []byte) (*internalhttp.Response, error) +} + +func (f fakeDruidHTTP) Do(ctx context.Context, method, url string, body []byte) (*internalhttp.Response, error) { + if f.doFunc != nil { + return f.doFunc(ctx, method, url, body) + } + return f.response, f.err +} + func TestGetAuthCreds(t *testing.T) { tests := []struct { name string @@ -175,10 +191,221 @@ func TestMakePath(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - actual := MakePath(tt.baseURL, tt.componentType, tt.apiType, tt.additionalPaths...) - if actual != tt.expected { - t.Errorf("makePath() = %v, expected %v", actual, tt.expected) - } + actual, err := MakePath(tt.baseURL, tt.componentType, tt.apiType, tt.additionalPaths...) + assert.NoError(t, err) + assert.Equal(t, tt.expected, actual) }) } } + +func TestMakeSQLPath(t *testing.T) { + actual, err := MakeSQLPath("http://router.druid.svc.cluster.local:8088") + assert.NoError(t, err) + assert.Equal(t, "http://router.druid.svc.cluster.local:8088/druid/v2/sql", actual) +} + +func TestGetClusterBuildRevisions(t *testing.T) { + k8sClient := fake.NewClientBuilder(). + WithObjects(&v1.Secret{ + ObjectMeta: metav1.ObjectMeta{ + Name: "test-default", + Namespace: "test", + }, + Data: map[string][]byte{ + OperatorUserName: []byte("test-user"), + OperatorPassword: []byte("test-password"), + }, + }). + Build() + + revisions, err := getClusterBuildRevisionsWithDeps(context.TODO(), k8sClient, "test", "example", Auth{ + Type: BasicAuth, + SecretRef: v1.SecretReference{Name: "test-default", Namespace: "test"}, + }, buildRevisionLookupDependencies{ + resolveQueryServiceURL: func(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return "http://router.druid.svc.cluster.local:8088", nil + }, + newHTTPClient: func(auth *internalhttp.Auth) internalhttp.DruidHTTP { + return fakeDruidHTTP{ + response: &internalhttp.Response{ + StatusCode: 200, + ResponseBody: `[{"build_revision":"abc123"},{"build_revision":"abc123"},{"build_revision":"def456"}]`, + }, + } + }, + }) + assert.NoError(t, err) + assert.Equal(t, []string{"abc123", "def456"}, revisions) +} + +func TestGetClusterBuildRevisionsFallsBackToVersionWhenBuildRevisionColumnIsUnavailable(t *testing.T) { + k8sClient := fake.NewClientBuilder().Build() + queries := make([]string, 0, 2) + + revisions, err := getClusterBuildRevisionsWithDeps(context.TODO(), k8sClient, "test", "example", Auth{}, buildRevisionLookupDependencies{ + resolveQueryServiceURL: func(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return "http://router.druid.svc.cluster.local:8088", nil + }, + newHTTPClient: func(auth *internalhttp.Auth) internalhttp.DruidHTTP { + return fakeDruidHTTP{ + doFunc: func(ctx context.Context, method, url string, body []byte) (*internalhttp.Response, error) { + queries = append(queries, string(body)) + if len(queries) == 1 { + return &internalhttp.Response{ + StatusCode: 400, + ResponseBody: `{"error":"druidException","errorCode":"invalidInput","errorMessage":"Column 'build_revision' not found in any table"}`, + }, nil + } + return &internalhttp.Response{ + StatusCode: 200, + ResponseBody: `[{"build_revision":"36.0.0"},{"build_revision":"36.0.0"}]`, + }, nil + }, + } + }, + }) + assert.NoError(t, err) + assert.Equal(t, []string{"36.0.0"}, revisions) + require.Len(t, queries, 2) + assert.Contains(t, queries[0], buildRevisionSQLQuery) + assert.Contains(t, queries[1], versionSQLQuery) +} + +func TestDefaultBuildRevisionLookupDependenciesUsesBrokerService(t *testing.T) { + deps := defaultBuildRevisionLookupDependencies() + require.NotNil(t, deps.resolveQueryServiceURL) + + k8sClient := fake.NewClientBuilder(). + WithObjects(&v1.Service{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example-broker", + Namespace: "test", + Labels: map[string]string{ + "druid_cr": "example", + "component": "broker", + }, + }, + }). + WithObjects(&v1.Service{ + ObjectMeta: metav1.ObjectMeta{ + Name: "example-router", + Namespace: "test", + Labels: map[string]string{ + "druid_cr": "example", + "component": "router", + }, + }, + }). + Build() + + url, err := deps.resolveQueryServiceURL(context.Background(), "test", "example", k8sClient) + require.NoError(t, err) + assert.Equal(t, "http://example-broker.test:8088", url) +} + +func TestGetClusterBuildRevisionsReturnsPendingErrorWhenQueryServiceIsUnavailable(t *testing.T) { + k8sClient := fake.NewClientBuilder().Build() + revisions, err := getClusterBuildRevisionsWithDeps(context.TODO(), k8sClient, "test", "example", Auth{}, buildRevisionLookupDependencies{ + resolveQueryServiceURL: func(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return "", errors.New("service not found") + }, + newHTTPClient: func(auth *internalhttp.Auth) internalhttp.DruidHTTP { + return fakeDruidHTTP{} + }, + }) + assert.Nil(t, revisions) + assert.Error(t, err) + assert.True(t, IsBuildRevisionLookupPending(err)) + + var pendingErr *BuildRevisionLookupPendingError + assert.ErrorAs(t, err, &pendingErr) + assert.Equal(t, BuildRevisionPendingQueryServiceDiscovery, pendingErr.Reason) +} + +func TestGetClusterBuildRevisionsReturnsPendingErrorWhenQueryRequestTimesOut(t *testing.T) { + k8sClient := fake.NewClientBuilder().Build() + revisions, err := getClusterBuildRevisionsWithDeps(context.TODO(), k8sClient, "test", "example", Auth{}, buildRevisionLookupDependencies{ + resolveQueryServiceURL: func(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return "http://router.druid.svc.cluster.local:8088", nil + }, + newHTTPClient: func(auth *internalhttp.Auth) internalhttp.DruidHTTP { + return fakeDruidHTTP{ + doFunc: func(ctx context.Context, method, url string, body []byte) (*internalhttp.Response, error) { + return nil, context.DeadlineExceeded + }, + } + }, + }) + assert.Nil(t, revisions) + assert.Error(t, err) + assert.True(t, IsBuildRevisionLookupPending(err)) + + var pendingErr *BuildRevisionLookupPendingError + assert.ErrorAs(t, err, &pendingErr) + assert.Equal(t, BuildRevisionPendingQueryEndpoint, pendingErr.Reason) +} + +func TestGetClusterBuildRevisionsReturnsCanceledErrorWithoutPendingClassification(t *testing.T) { + k8sClient := fake.NewClientBuilder().Build() + revisions, err := getClusterBuildRevisionsWithDeps(context.TODO(), k8sClient, "test", "example", Auth{}, buildRevisionLookupDependencies{ + resolveQueryServiceURL: func(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return "http://router.druid.svc.cluster.local:8088", nil + }, + newHTTPClient: func(auth *internalhttp.Auth) internalhttp.DruidHTTP { + return fakeDruidHTTP{ + doFunc: func(ctx context.Context, method, url string, body []byte) (*internalhttp.Response, error) { + return nil, context.Canceled + }, + } + }, + }) + assert.Nil(t, revisions) + assert.ErrorIs(t, err, context.Canceled) + assert.False(t, IsBuildRevisionLookupPending(err)) +} + +func TestGetClusterBuildRevisionsReturnsTerminalErrorForUnauthorizedQuery(t *testing.T) { + k8sClient := fake.NewClientBuilder().Build() + revisions, err := getClusterBuildRevisionsWithDeps(context.TODO(), k8sClient, "test", "example", Auth{}, buildRevisionLookupDependencies{ + resolveQueryServiceURL: func(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return "http://router.druid.svc.cluster.local:8088", nil + }, + newHTTPClient: func(auth *internalhttp.Auth) internalhttp.DruidHTTP { + return fakeDruidHTTP{ + response: &internalhttp.Response{ + StatusCode: 401, + ResponseBody: "unauthorized", + }, + } + }, + }) + assert.Nil(t, revisions) + assert.EqualError(t, err, "status code: 401, response body: unauthorized") + assert.False(t, IsBuildRevisionLookupPending(err)) +} + +func TestGetClusterBuildRevisionsDoesNotFallbackForOtherTerminalQueryErrors(t *testing.T) { + k8sClient := fake.NewClientBuilder().Build() + queries := make([]string, 0, 2) + + revisions, err := getClusterBuildRevisionsWithDeps(context.TODO(), k8sClient, "test", "example", Auth{}, buildRevisionLookupDependencies{ + resolveQueryServiceURL: func(ctx context.Context, namespace, druidClusterName string, c client.Client) (string, error) { + return "http://router.druid.svc.cluster.local:8088", nil + }, + newHTTPClient: func(auth *internalhttp.Auth) internalhttp.DruidHTTP { + return fakeDruidHTTP{ + doFunc: func(ctx context.Context, method, url string, body []byte) (*internalhttp.Response, error) { + queries = append(queries, string(body)) + return &internalhttp.Response{ + StatusCode: 400, + ResponseBody: `{"error":"druidException","errorCode":"invalidInput","errorMessage":"Table 'sys.servers' not found"}`, + }, nil + }, + } + }, + }) + assert.Nil(t, revisions) + assert.EqualError(t, err, `status code: 400, response body: {"error":"druidException","errorCode":"invalidInput","errorMessage":"Table 'sys.servers' not found"}`) + require.Len(t, queries, 1) + assert.Contains(t, queries[0], buildRevisionSQLQuery) +} diff --git a/pkg/http/http.go b/pkg/http/http.go index 792bc93..fc640d5 100644 --- a/pkg/http/http.go +++ b/pkg/http/http.go @@ -20,16 +20,15 @@ package http import ( "bytes" + "context" "io" "net/http" ) -// DruidHTTP interface type DruidHTTP interface { - Do(method, url string, body []byte) (*Response, error) + Do(ctx context.Context, method, url string, body []byte) (*Response, error) } -// HTTP client type DruidClient struct { HTTPClient *http.Client Auth *Auth @@ -50,22 +49,20 @@ type Auth struct { BasicAuth BasicAuth } -// BasicAuth type BasicAuth struct { UserName string Password string } -// Response passed to controller type Response struct { ResponseBody string StatusCode int } -// Do method to be used schema and tenant controller. -func (c *DruidClient) Do(Method, url string, body []byte) (*Response, error) { +// Do issues a context-aware JSON request to a Druid control-plane endpoint. +func (c *DruidClient) Do(ctx context.Context, method, url string, body []byte) (*Response, error) { - req, err := http.NewRequest(Method, url, bytes.NewBuffer(body)) + req, err := http.NewRequestWithContext(ctx, method, url, bytes.NewBuffer(body)) if err != nil { return nil, err } diff --git a/pkg/http/http_test.go b/pkg/http/http_test.go new file mode 100644 index 0000000..d032deb --- /dev/null +++ b/pkg/http/http_test.go @@ -0,0 +1,74 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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 http + +import ( + "context" + "errors" + "net/http" + "strings" + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +type blockingRoundTripper struct{} + +func (blockingRoundTripper) RoundTrip(req *http.Request) (*http.Response, error) { + <-req.Context().Done() + return nil, req.Context().Err() +} + +func TestDoHonorsContextCancellation(t *testing.T) { + client := NewHTTPClient(&http.Client{Transport: blockingRoundTripper{}}, &Auth{}).(*DruidClient) + ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) + defer cancel() + + start := time.Now() + _, err := client.Do(ctx, http.MethodGet, "http://example.invalid", nil) + + assert.Error(t, err) + assert.True(t, errors.Is(err, context.DeadlineExceeded)) + assert.Less(t, time.Since(start), 500*time.Millisecond) +} + +func TestDoSetsBasicAuth(t *testing.T) { + var authHeader string + client := NewHTTPClient(&http.Client{ + Transport: roundTripperFunc(func(req *http.Request) (*http.Response, error) { + authHeader = req.Header.Get("Authorization") + return &http.Response{ + StatusCode: 200, + Body: http.NoBody, + Header: make(http.Header), + }, nil + }), + }, &Auth{BasicAuth: BasicAuth{UserName: "user", Password: "pass"}}).(*DruidClient) + + _, err := client.Do(context.Background(), http.MethodGet, "http://example.invalid", nil) + assert.NoError(t, err) + assert.True(t, strings.HasPrefix(authHeader, "Basic ")) +} + +type roundTripperFunc func(req *http.Request) (*http.Response, error) + +func (f roundTripperFunc) RoundTrip(req *http.Request) (*http.Response, error) { + return f(req) +}