Skip to content

Commit

Permalink
Merge pull request #4207 from pvsune/feat/version-field
Browse files Browse the repository at this point in the history
Add a version field in the operator's status
  • Loading branch information
pvsune committed Apr 12, 2022
2 parents 546996a + 3b3642b commit 568553c
Show file tree
Hide file tree
Showing 8 changed files with 101 additions and 7 deletions.
3 changes: 3 additions & 0 deletions src/go/k8s/apis/redpanda/v1alpha1/cluster_types.go
Original file line number Diff line number Diff line change
Expand Up @@ -286,6 +286,9 @@ type ClusterStatus struct {
// Indicates cluster is upgrading
// +optional
Upgrading bool `json:"upgrading"`
// Current version of the cluster.
// +optional
Version string `json:"version"`
// Current state of the cluster.
// +optional
Conditions []ClusterCondition `json:"conditions,omitempty"`
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -905,6 +905,9 @@ spec:
upgrading:
description: Indicates cluster is upgrading
type: boolean
version:
description: Current version of the cluster.
type: string
type: object
type: object
served: true
Expand Down
16 changes: 9 additions & 7 deletions src/go/k8s/controllers/redpanda/cluster_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,7 @@ func (r *ClusterReconciler) Reconcile(
err = r.reportStatus(
ctx,
&redpandaCluster,
sts.LastObservedState,
sts,
headlessSvc.HeadlessServiceFQDN(r.clusterDomain),
clusterSvc.ServiceFQDN(r.clusterDomain),
schemaRegistryPort,
Expand Down Expand Up @@ -279,7 +279,7 @@ func validateImagePullPolicy(imagePullPolicy corev1.PullPolicy) error {
func (r *ClusterReconciler) reportStatus(
ctx context.Context,
redpandaCluster *redpandav1alpha1.Cluster,
lastObservedSts *appsv1.StatefulSet,
sts *resources.StatefulSetResource,
internalFQDN string,
clusterFQDN string,
schemaRegistryPort int,
Expand Down Expand Up @@ -307,7 +307,7 @@ func (r *ClusterReconciler) reportStatus(
return fmt.Errorf("failed to construct external node list: %w", err)
}

if lastObservedSts == nil {
if sts.LastObservedState == nil {
return errNonexistentLastObservesState
}

Expand All @@ -319,7 +319,7 @@ func (r *ClusterReconciler) reportStatus(
nodeList.Internal = observedNodesInternal
nodeList.SchemaRegistry.Internal = fmt.Sprintf("%s:%d", clusterFQDN, schemaRegistryPort)

if statusShouldBeUpdated(&redpandaCluster.Status, nodeList, lastObservedSts.Status.ReadyReplicas) {
if statusShouldBeUpdated(&redpandaCluster.Status, nodeList, sts) {
err := retry.RetryOnConflict(retry.DefaultRetry, func() error {
var cluster redpandav1alpha1.Cluster
err := r.Get(ctx, types.NamespacedName{
Expand All @@ -331,7 +331,8 @@ func (r *ClusterReconciler) reportStatus(
}

cluster.Status.Nodes = *nodeList
cluster.Status.Replicas = lastObservedSts.Status.ReadyReplicas
cluster.Status.Replicas = sts.LastObservedState.Status.ReadyReplicas
cluster.Status.Version = sts.Version()

err = r.Status().Update(ctx, &cluster)
if err == nil {
Expand All @@ -350,7 +351,7 @@ func (r *ClusterReconciler) reportStatus(
func statusShouldBeUpdated(
status *redpandav1alpha1.ClusterStatus,
nodeList *redpandav1alpha1.NodesList,
readyReplicas int32,
sts *resources.StatefulSetResource,
) bool {
return nodeList != nil &&
(!reflect.DeepEqual(nodeList.Internal, status.Nodes.Internal) ||
Expand All @@ -359,7 +360,8 @@ func statusShouldBeUpdated(
!reflect.DeepEqual(nodeList.ExternalPandaproxy, status.Nodes.ExternalPandaproxy) ||
!reflect.DeepEqual(nodeList.SchemaRegistry, status.Nodes.SchemaRegistry) ||
!reflect.DeepEqual(nodeList.ExternalBootstrap, status.Nodes.ExternalBootstrap)) ||
status.Replicas != readyReplicas
status.Replicas != sts.LastObservedState.Status.ReadyReplicas ||
status.Version != sts.Version()
}

// WithConfiguratorSettings set the configurator image settings
Expand Down
25 changes: 25 additions & 0 deletions src/go/k8s/pkg/resources/statefulset.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"path"
"path/filepath"
"strconv"
"strings"

"github.com/go-logr/logr"
cmetav1 "github.com/jetstack/cert-manager/pkg/apis/meta/v1"
Expand Down Expand Up @@ -872,3 +873,27 @@ func statefulSetKind() string {
func (r *StatefulSetResource) fullConfiguratorImage() string {
return fmt.Sprintf("%s:%s", r.configuratorSettings.ConfiguratorBaseImage, r.configuratorSettings.ConfiguratorTag)
}

// Version returns the cluster version specified in the image tag.
func (r *StatefulSetResource) Version() string {
lastObservedSts := r.LastObservedState
if lastObservedSts != nil {
cc := lastObservedSts.Spec.Template.Spec.Containers
for i := range cc {
c := cc[i]
if c.Name != redpandaContainerName {
continue
}
// Will always have tag even for latest because of pandaCluster.FullImageName().
if s := strings.Split(c.Image, ":"); len(s) > 1 {
version := s[len(s)-1]
// Image uses registry with port and no tag (e.g. localhost:5000/redpanda)
if strings.Contains(version, "/") {
version = ""
}
return version
}
}
}
return ""
}
34 changes: 34 additions & 0 deletions src/go/k8s/pkg/resources/statefulset_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -301,3 +301,37 @@ func pandaCluster() *redpandav1alpha1.Cluster {
},
}
}

func TestVersion(t *testing.T) {
var (
redpandaContainerName = "redpanda"
)

tests := []struct {
Containers []corev1.Container
ExpectedVersion string
}{
{Containers: []corev1.Container{{Name: redpandaContainerName, Image: "vectorized/redpanda:v21.11.11"}}, ExpectedVersion: "v21.11.11"},
{Containers: []corev1.Container{{Name: redpandaContainerName, Image: "vectorized/redpanda:"}}, ExpectedVersion: ""},
// Image with no tag does not return "latest" as version.
{Containers: []corev1.Container{{Name: redpandaContainerName, Image: "vectorized/redpanda"}}, ExpectedVersion: ""},
{Containers: []corev1.Container{{Name: redpandaContainerName, Image: "localhost:5000/redpanda:v21.11.11"}}, ExpectedVersion: "v21.11.11"},
{Containers: []corev1.Container{{Name: redpandaContainerName, Image: "localhost:5000/redpanda"}}, ExpectedVersion: ""},
{Containers: []corev1.Container{{Name: "", Image: "vectorized/redpanda"}}, ExpectedVersion: ""},
}

for _, tt := range tests {
sts := &res.StatefulSetResource{
LastObservedState: &v1.StatefulSet{
Spec: v1.StatefulSetSpec{
Template: corev1.PodTemplateSpec{
Spec: corev1.PodSpec{
Containers: tt.Containers,
},
},
},
},
}
assert.Equal(t, tt.ExpectedVersion, sts.Version())
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,3 +50,12 @@ collectors:
- type: pod
selector: app.kubernetes.io/name=redpanda
tail: -1

---

apiVersion: redpanda.vectorized.io/v1alpha1
kind: Cluster
metadata:
name: update-image-cluster-and-node-port
status:
version: "v21.11.1"
Original file line number Diff line number Diff line change
Expand Up @@ -78,3 +78,12 @@ collectors:
- type: pod
selector: app.kubernetes.io/name=redpanda
tail: -1

---

apiVersion: redpanda.vectorized.io/v1alpha1
kind: Cluster
metadata:
name: update-image-cluster-and-node-port
status:
version: "v21.11.2"
Original file line number Diff line number Diff line change
Expand Up @@ -39,3 +39,12 @@ collectors:
- type: pod
selector: app.kubernetes.io/name=redpanda
tail: -1

---

apiVersion: redpanda.vectorized.io/v1alpha1
kind: Cluster
metadata:
name: update-image-cluster-and-node-port
status:
version: "dev"

0 comments on commit 568553c

Please sign in to comment.