Skip to content

Commit

Permalink
[node-agent] Integrate gardener-node-agent into gardenlet's `Shoo…
Browse files Browse the repository at this point in the history
…t` controller (gardener#8847)

* Add kubelet bootstrap kubeconfig handling to `gardener-node-agent`

* Add `gardener-node-agent` image to Skaffold

* Populate OSC sync jitter period to `operatingsystemconfig` component

This value is needed later when generating the `gardener-node-agent`'s component configuration, see: https://github.com/gardener/gardener/blob/764df0ee5ebc13b2634eba98169b409244f19bfe/pkg/component/extensions/operatingsystemconfig/original/components/nodeagent/component.go#L127

* Deploy shoot access secret for `gardener-node-agent`

* Add `gardener-node-{init,agent}` to `OperatingSystemConfig`

when feature gate is enabled

* Inject correct `hyperkube` image into original components context

Previously (with `cloud-config-downloader`), this was done as part of the `executor.Script` function (see https://github.com/gardener/gardener/blob/67a049a3f66ce489002a67dd59ac7b95e8d2573b/pkg/operation/botanist/operatingsystemconfig.go#L181-L189).
Now (with `gardener-node-agent`), the `hyperkube` image is added as a file with type `imageRef` to the OSC's `.spec.files`, see https://github.com/gardener/gardener/blob/67a049a3f66ce489002a67dd59ac7b95e8d2573b/pkg/component/extensions/operatingsystemconfig/original/components/kubelet/component.go#L192-L218. Hence, the original components context must have the correct `hyperkube` image for the Kubernetes version of the worker pool.

* Define RBAC resources for `gardener-node-agent`

They will be deployed as part of a `ManagedResource` in a subsequent commit.

* OSC secrets must be annotated with checksum of OSC data

- For backwards-compatibility, we use the same annotation keys like before (`checksum/cloud-config-data` and `checksum/data-script`), even if these names are not fully accurate. Maybe we can change them later
- `gardenlet` will compute the checksum of the OSC secret and adds the result as annotation
- `gardener-node-agent` reads this checksum from the OSC secret and adds it as annotation to the `Node` after successful reconciliation
- This allows `gardenlet` to check whether GNA applied the most recent OSC on the nodes (health checks, adapted in a subsequent commit)

* Define OSC `Secret` generation for `gardener-node-agent`

These `Secret`s get reconciled later by `gardener-node-agent`'s `OperatingSystemConfig` controller.
They will be deployed as part of a `ManagedResource` in a subsequent commit.

* Make logic for deploying cloud-config `ManagedResource` reusable

Earlier, this function deployed a `ManagedResource` containing the RBAC rules of `cloud-config-downloader` as well as the secrets containing the bash scripts that get executed by `cloud-config-downloader`.

In the next commit, we introduce a functon that deploys a `ManagedResource` containing the RBAC rules for `gardener-node-agent` as well as the secrets containing the OSC that get reconciled by `gardener-node-agent`

* Deploy `ManagedResource` with OSC secrets + RBAC for `gardener-node-agent`

* Adapt shoot care health checks

- For backwards-compatibility, we use the same annotation keys like before (`checksum/cloud-config-data` and `checksum/data-script`), even if these names are not fully accurate. Maybe we can change them later
- Only when a secret with label
  `gardener.cloud/role=operating-system-config` is found in the shoot,
those secrets are considered for the health checks. This is to make the
checks compatible with both (a) shoots that weren't reconciled yet, i.e.
don't have gardener-node-agent yet, and (b) shoots that were reconciled
and have gardener-node-agent

* Add `cloud-config-downloader` cleanup code

- `gardener-node-agent` uses `cloud-config-downloader` token to download its own access token when deployed on an existing node
- It deletes the directory and systemd files on the node after start-up
- After the OSC on all nodes was updated, `gardenlet` deletes the `cloud-config-downloader` access secret from both seed and shoot, and the no longer needed `ManagedResource` and `Secret`s for the cloud config execution bash scripts

* Enable `UseGardenerNodeAgent` feature gate locally

* Address PR review feedback
  • Loading branch information
rfranzke authored Nov 27, 2023
1 parent 0fc36d4 commit 3d3887c
Show file tree
Hide file tree
Showing 40 changed files with 2,710 additions and 663 deletions.
31 changes: 28 additions & 3 deletions cmd/gardener-node-agent/app/app.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,12 @@ import (
"sigs.k8s.io/controller-runtime/pkg/manager"
metricsserver "sigs.k8s.io/controller-runtime/pkg/metrics/server"

"github.com/gardener/gardener/cmd/gardener-node-agent/app/bootstrappers"
"github.com/gardener/gardener/cmd/utils"
resourcesv1alpha1 "github.com/gardener/gardener/pkg/apis/resources/v1alpha1"
"github.com/gardener/gardener/pkg/client/kubernetes"
"github.com/gardener/gardener/pkg/component/extensions/operatingsystemconfig/downloader"
"github.com/gardener/gardener/pkg/controllerutils"
"github.com/gardener/gardener/pkg/controllerutils/routes"
"github.com/gardener/gardener/pkg/features"
gardenerhealthz "github.com/gardener/gardener/pkg/healthz"
Expand Down Expand Up @@ -185,9 +188,23 @@ func run(ctx context.Context, cancel context.CancelFunc, log logr.Logger, cfg *c
return err
}

log.Info("Adding controllers to manager")
if err := controller.AddToManager(cancel, mgr, cfg, hostName); err != nil {
return fmt.Errorf("failed adding controllers to manager: %w", err)
var (
fs = afero.Afero{Fs: afero.NewOsFs()}
dbus = dbus.New()
)

log.Info("Adding runnables to manager")
if err := mgr.Add(&controllerutils.ControlledRunner{
Manager: mgr,
BootstrapRunnables: []manager.Runnable{
&bootstrappers.KubeletBootstrapKubeconfig{Log: log.WithName("kubelet-bootstrap-kubeconfig-creator"), FS: fs, APIServerConfig: cfg.APIServer},
},
ActualRunnables: []manager.Runnable{
manager.RunnableFunc(func(_ context.Context) error { return controller.AddToManager(cancel, mgr, cfg, hostName) }),
&bootstrappers.CloudConfigDownloaderCleaner{Log: log.WithName("legacy-cloud-config-downloader-cleaner"), FS: fs, DBus: dbus},
},
}); err != nil {
return fmt.Errorf("failed adding runnables to manager: %w", err)
}

log.Info("Starting manager")
Expand All @@ -214,6 +231,14 @@ func getRESTConfig(log logr.Logger, cfg *config.NodeAgentConfiguration) (*rest.C
return restConfig, false, nil
}

if _, err := os.Stat(downloader.PathCredentialsToken); err != nil && !os.IsNotExist(err) {
return nil, false, fmt.Errorf("failed checking whether cloud-config-downloader token file %q exists: %w", downloader.PathCredentialsToken, err)
} else if err == nil {
log.Info("Token file does not exist, but legacy cloud-config-downloader token file does - using it", "path", downloader.PathCredentialsToken)
restConfig.BearerTokenFile = downloader.PathCredentialsToken
return restConfig, true, nil
}

if _, err := os.Stat(nodeagentv1alpha1.BootstrapTokenFilePath); err != nil && !os.IsNotExist(err) {
return nil, false, fmt.Errorf("failed checking whether bootstrap token file %q exists: %w", nodeagentv1alpha1.BootstrapTokenFilePath, err)
} else if err == nil {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
// Copyright 2023 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package bootstrappers_test

import (
"testing"

. "github.com/onsi/ginkgo/v2"
. "github.com/onsi/gomega"
)

func TestBootstrappers(t *testing.T) {
RegisterFailHandler(Fail)
RunSpecs(t, "Command NodeAgent App Bootstrappers Suite")
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
// Copyright 2023 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package bootstrappers

import (
"context"
"errors"
"fmt"
"path"

"github.com/go-logr/logr"
"github.com/spf13/afero"

"github.com/gardener/gardener/pkg/component/extensions/operatingsystemconfig/downloader"
"github.com/gardener/gardener/pkg/nodeagent/dbus"
)

// CloudConfigDownloaderCleaner is a runnable for cleaning up the legacy cloud-config-downloader resources.
// TODO(rfranzke): Remove this bootstrapper when the UseGardenerNodeAgent feature gate gets removed.
type CloudConfigDownloaderCleaner struct {
Log logr.Logger
FS afero.Afero
DBus dbus.DBus
}

// Start performs the cleanup logic. Note that this function does only delete the following directories/files:
// - /var/lib/cloud-config-downloader
// - /etc/systemd/system/multi-user.target.wants/cloud-config-downloader.service (typically symlinks to
// /etc/systemd/system/cloud-config-downloader.service
//
// The /etc/systemd/system/cloud-config-downloader.service file already gets removed by cloud-config-downloader itself
// when migrating to gardener-node-agent because it is no longer part of the original OperatingSystemConfig. Hence,
// cloud-config-downloader considers it as stale and cleans it up.
// All this still leaves some artefacts on the nodes (e.g., `systemctl status cloud-config-downloader` and
// `journalctl -u cloud-config-downloader` still works), however, maybe that's even a benefit in case of operations/
// debugging activities. All nodes get rolled/replaced eventually (latest with the next OS/Kubernetes version update),
// so we leave the final cleanup for then (new nodes will have no traces of cloud-config-downloader whatsoever).
func (c *CloudConfigDownloaderCleaner) Start(ctx context.Context) error {
c.Log.Info("Removing legacy directory if it exists", "path", downloader.PathCCDDirectory)
if err := c.FS.RemoveAll(downloader.PathCCDDirectory); err != nil {
return fmt.Errorf("failed to remove legacy directory %q: %w", downloader.PathCCDDirectory, err)
}

unitFilePath := path.Join("/", "etc", "systemd", "system", "multi-user.target.wants", downloader.UnitName)
c.Log.Info("Removing legacy unit file if it exists", "path", unitFilePath)
if err := c.FS.Remove(unitFilePath); err != nil {
if !errors.Is(err, afero.ErrFileNotFound) {
return fmt.Errorf("failed removing legacy unit file file %q: %w", unitFilePath, err)
}
return nil
}

return c.DBus.DaemonReload(ctx)
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
// Copyright 2023 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package bootstrappers

import (
"context"
"io/fs"
"path/filepath"

"github.com/go-logr/logr"
. "github.com/onsi/ginkgo/v2"
. "github.com/onsi/gomega"
"github.com/spf13/afero"
"sigs.k8s.io/controller-runtime/pkg/manager"

fakedbus "github.com/gardener/gardener/pkg/nodeagent/dbus/fake"
"github.com/gardener/gardener/pkg/utils/test"
)

var _ = Describe("CloudConfigDownloaderCleaner", func() {
var (
ctx = context.TODO()
log = logr.Discard()

fakeFS afero.Afero
fakeDBus *fakedbus.DBus
runnable manager.Runnable
)

BeforeEach(func() {
fakeFS = afero.Afero{Fs: afero.NewMemMapFs()}
fakeDBus = fakedbus.New()
runnable = &CloudConfigDownloaderCleaner{
Log: log,
FS: fakeFS,
DBus: fakeDBus,
}
})

Describe("#Start", func() {
var (
pathDirectory = filepath.Join("/", "var", "lib", "cloud-config-downloader")
pathSystemdUnitFileSymlink = filepath.Join("/", "etc", "systemd", "system", "multi-user.target.wants", "cloud-config-downloader.service")
)

It("should remove the directories and files, and reload systemd daemon", func() {
Expect(fakeFS.MkdirAll(pathDirectory, fs.ModeDir)).To(Succeed())
_, err := fakeFS.Create(pathSystemdUnitFileSymlink)
Expect(err).NotTo(HaveOccurred())

Expect(runnable.Start(ctx)).To(Succeed())

test.AssertNoDirectoryOnDisk(fakeFS, pathDirectory)
test.AssertNoFileOnDisk(fakeFS, pathSystemdUnitFileSymlink)
Expect(fakeDBus.Actions).To(ConsistOf(fakedbus.SystemdAction{Action: fakedbus.ActionDaemonReload}))
})

It("should not restart when the systemd unit file does not exist anymore", func() {
Expect(fakeFS.MkdirAll(pathDirectory, fs.ModeDir)).To(Succeed())

Expect(runnable.Start(ctx)).To(Succeed())

Expect(fakeDBus.Actions).To(BeEmpty())
})

It("should not fail when there is nothing to cleanup", func() {
Expect(runnable.Start(ctx)).To(Succeed())

Expect(fakeDBus.Actions).To(BeEmpty())
})
})
})
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// Copyright 2023 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package bootstrappers

import (
"context"
"errors"
"fmt"
"os"
"path/filepath"
"strings"

"github.com/go-logr/logr"
"github.com/spf13/afero"
"k8s.io/apimachinery/pkg/runtime"
clientcmdlatest "k8s.io/client-go/tools/clientcmd/api/latest"
clientcmdv1 "k8s.io/client-go/tools/clientcmd/api/v1"

"github.com/gardener/gardener/pkg/component/extensions/operatingsystemconfig/original/components/kubelet"
"github.com/gardener/gardener/pkg/nodeagent/apis/config"
nodeagentv1alpha1 "github.com/gardener/gardener/pkg/nodeagent/apis/config/v1alpha1"
kubernetesutils "github.com/gardener/gardener/pkg/utils/kubernetes"
)

// KubeletBootstrapKubeconfig is a runnable for creating a bootstrap kubeconfig for kubelet.
type KubeletBootstrapKubeconfig struct {
Log logr.Logger
FS afero.Afero
APIServerConfig config.APIServer
}

// Start performs creation of the bootstrap kubeconfig.
func (k *KubeletBootstrapKubeconfig) Start(_ context.Context) error {
k.Log.Info("Checking whether kubelet bootstrap kubeconfig needs to be created")

bootstrapToken, err := k.FS.ReadFile(nodeagentv1alpha1.BootstrapTokenFilePath)
if err != nil {
if !errors.Is(err, afero.ErrFileNotFound) {
return fmt.Errorf("failed checking whether bootstrap token file %q already exists: %w", nodeagentv1alpha1.BootstrapTokenFilePath, err)
}
k.Log.Info("Bootstrap token file does not exist, nothing to be done", "path", nodeagentv1alpha1.BootstrapTokenFilePath)
return nil
}

if _, err := k.FS.Stat(kubelet.PathKubeconfigReal); err != nil && !errors.Is(err, afero.ErrFileNotFound) {
return fmt.Errorf("failed checking whether kubelet kubeconfig file %q already exists: %w", kubelet.PathKubeconfigReal, err)
} else if err == nil {
k.Log.Info("Kubelet kubeconfig with client certificates already exists, nothing to be done", "path", kubelet.PathKubeconfigReal)
return nil
}

kubeletClientCertificatePath := filepath.Join(kubelet.PathKubeletDirectory, "pki", "kubelet-client-current.pem")
if _, err := k.FS.Stat(kubeletClientCertificatePath); err != nil && !errors.Is(err, afero.ErrFileNotFound) {
return fmt.Errorf("failed checking whether kubelet client certificate file %q already exists: %w", kubeletClientCertificatePath, err)
} else if err == nil {
k.Log.Info("Kubelet client certificates file already exists, nothing to be done", "path", kubeletClientCertificatePath)
return nil
}

k.Log.Info("Creating kubelet directory", "path", kubelet.PathKubeletDirectory)
if err := k.FS.MkdirAll(kubelet.PathKubeletDirectory, os.ModeDir); err != nil {
return fmt.Errorf("unable to create kubelet directory %q: %w", kubelet.PathKubeletDirectory, err)
}

kubeconfig, err := runtime.Encode(clientcmdlatest.Codec, kubernetesutils.NewKubeconfig(
"kubelet-bootstrap",
clientcmdv1.Cluster{Server: k.APIServerConfig.Server, CertificateAuthorityData: k.APIServerConfig.CABundle},
clientcmdv1.AuthInfo{Token: strings.TrimSpace(string(bootstrapToken))},
))
if err != nil {
return fmt.Errorf("unable to encode kubeconfig: %w", err)
}

k.Log.Info("Writing kubelet bootstrap kubeconfig file", "path", kubelet.PathKubeconfigBootstrap)
return k.FS.WriteFile(kubelet.PathKubeconfigBootstrap, kubeconfig, 0600)
}
Loading

0 comments on commit 3d3887c

Please sign in to comment.