Skip to content

Commit

Permalink
Migration from cm to CRs
Browse files Browse the repository at this point in the history
In case the IP configuration ConfigMap exists:
- Create IPPools CR according to config spec
- Read Nodes ranges annotation
- Populate the IPpools Status allocations
- Clear Nodes ranges annotation
- Delete ConfigMap

In case an issue is preventing the migration flow,
it can be skipped by setting the env var 'MIGRATOR_DISABLE_MIGRATION'.

Signed-off-by: Fred Rolland <frolland@nvidia.com>
  • Loading branch information
rollandf committed Sep 12, 2023
1 parent a95a67b commit d6513cd
Show file tree
Hide file tree
Showing 6 changed files with 590 additions and 12 deletions.
16 changes: 16 additions & 0 deletions cmd/ipam-controller/app/app.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ package app
import (
"context"
"fmt"
"os"

"github.com/go-logr/logr"
"github.com/spf13/cobra"
Expand All @@ -28,6 +29,7 @@ import (
"k8s.io/component-base/term"
"k8s.io/klog/v2"
ctrl "sigs.k8s.io/controller-runtime"
"sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/event"
"sigs.k8s.io/controller-runtime/pkg/healthz"

Expand All @@ -42,6 +44,7 @@ import (
"github.com/Mellanox/nvidia-k8s-ipam/pkg/common"
poolctrl "github.com/Mellanox/nvidia-k8s-ipam/pkg/ipam-controller/controllers/ippool"
nodectrl "github.com/Mellanox/nvidia-k8s-ipam/pkg/ipam-controller/controllers/node"
"github.com/Mellanox/nvidia-k8s-ipam/pkg/ipam-controller/migrator"
"github.com/Mellanox/nvidia-k8s-ipam/pkg/version"
)

Expand Down Expand Up @@ -125,6 +128,19 @@ func RunController(ctx context.Context, config *rest.Config, opts *options.Optio
return err
}

k8sClient, err := client.New(config,
client.Options{Scheme: mgr.GetScheme(), Mapper: mgr.GetRESTMapper()})
if err != nil {
logger.Error(err, "failed to create k8sClient client")
os.Exit(1)
}

if err := migrator.Migrate(ctx, k8sClient, opts.IPPoolsNamespace); err != nil {
logger.Error(err, fmt.Sprintf("failed to migrate NV-IPAM config from ConfigMap, "+
"set %s env variable to disable migration", migrator.EnvDisableMigration))
return err
}

nodeEventCH := make(chan event.GenericEvent, 1)

if err = (&nodectrl.NodeReconciler{
Expand Down
2 changes: 2 additions & 0 deletions deploy/nv-ipam.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@ rules:
- get
- list
- watch
- delete
- apiGroups:
- nv-ipam.nvidia.com
resources:
Expand All @@ -155,6 +156,7 @@ rules:
- get
- list
- watch
- create
- apiGroups:
- nv-ipam.nvidia.com
resources:
Expand Down
252 changes: 252 additions & 0 deletions pkg/ipam-controller/migrator/migrator.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,252 @@
/*
Copyright 2023, NVIDIA CORPORATION & AFFILIATES
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 migrator

import (
"context"
"encoding/json"
"fmt"
"net"
"os"
"reflect"
"strings"

"github.com/go-logr/logr"
corev1 "k8s.io/api/core/v1"
apiErrors "k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/sets"
"sigs.k8s.io/controller-runtime/pkg/client"

ipamv1alpha1 "github.com/Mellanox/nvidia-k8s-ipam/api/v1alpha1"
"github.com/Mellanox/nvidia-k8s-ipam/pkg/ipam-controller/config"
"github.com/Mellanox/nvidia-k8s-ipam/pkg/pool"
)

const (
// EnvDisableMigration contains the name of the environment variable which can be used
// to disable migration
EnvDisableMigration = "MIGRATOR_DISABLE_MIGRATION"
// EnvConfigMapName contains the name of the environment variable which can be used
// to specify the ConfigMap name containing the configuration to migrate from
EnvConfigMapName = "CONFIGMAP_NAME"
// EnvConfigMapNamespace contains the name of the environment variable which can be used
// to specify the namespace of the ConfigMap containing the configuration to migrate from
EnvConfigMapNamespace = "CONFIGMAP_NAMESPACE"
// DefaultConfigMapName is the default ConfigMap name used to read the configuration to
// migrate from
DefaultConfigMapName = "nvidia-k8s-ipam-config"
)

// Migrate reads the ConfigMap with the IPAM configuration, reads the allocations
// from the Nodes annotation, create IPPool CRs and delete the ConfigMap and annotations
func Migrate(ctx context.Context, c client.Client, poolNamespace string) error {
logger := logr.FromContextOrDiscard(ctx).WithName("migrator")
if os.Getenv(EnvDisableMigration) != "" {
logger.Info(fmt.Sprintf("%s set, skip controller migration", EnvDisableMigration))
return nil
}

cmName := DefaultConfigMapName
if os.Getenv(EnvConfigMapName) != "" {
cmName = os.Getenv(EnvConfigMapName)
}
cmNamespace := poolNamespace
if os.Getenv(EnvConfigMapNamespace) != "" {
cmNamespace = os.Getenv(EnvConfigMapNamespace)
}

cfg := &corev1.ConfigMap{}
key := types.NamespacedName{
Name: cmName,
Namespace: cmNamespace,
}
err := c.Get(ctx, key, cfg)
if err != nil {
if apiErrors.IsNotFound(err) {
logger.Info("ConfigMap not found, skipping migration")
return nil
}
logger.Error(err, "failed to read ConfigMap object")
return err
}

confData, exist := cfg.Data[config.ConfigMapKey]
if !exist {
err = fmt.Errorf("invalid configuration: ConfigMap %s doesn't contain %s key",
key, config.ConfigMapKey)
logger.Error(err, "Invalid config, no data")
return err
}
controllerConfig := &config.Config{}
if err := json.Unmarshal([]byte(confData), controllerConfig); err != nil {
logger.Error(err, fmt.Sprintf("invalid configuration: ConfigMap %s contains invalid JSON",
config.ConfigMapKey))
return err
}
if err := controllerConfig.Validate(); err != nil {
logger.Error(err, fmt.Sprintf("invalid configuration: ConfigMap %s contains invalid config",
config.ConfigMapKey))
return err
}

pools := buildIPPools(controllerConfig, poolNamespace)

for name, p := range pools {
err = c.Create(ctx, p)
logger.Info(fmt.Sprintf("Creating IPPool: %v", p))
if apiErrors.IsAlreadyExists(err) {
existingPool := &ipamv1alpha1.IPPool{}
err = c.Get(ctx, client.ObjectKeyFromObject(p), existingPool)
if err != nil {
logger.Info("fail to get existing pool", "pool name", name)
return err
}
if !reflect.DeepEqual(existingPool.Spec, p.Spec) {
logger.Info("existing pool has different spec than config map setting", "pool name", name)
return fmt.Errorf("existing pool has different spec than config map setting")
}
} else if err != nil {
logger.Info("fail to create pool", "pool name", name)
return err
}
}

err = updateAllocations(ctx, c, logger, pools, poolNamespace)
if err != nil {
return err
}

err = c.Delete(ctx, &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Namespace: cmNamespace, Name: cmName}})
if err != nil {
logger.Info("fail to delete nv-ipam config map")
return err
}

return nil
}

func updateAllocations(ctx context.Context, c client.Client,
logger logr.Logger, pools map[string]*ipamv1alpha1.IPPool, poolNamespace string) error {
nodeList := &corev1.NodeList{}
if err := c.List(ctx, nodeList); err != nil {
logger.Error(err, "failed to list nodes")
return err
}
nodesToClearAnnotation := sets.New[string]()
for poolName := range pools {
p := &ipamv1alpha1.IPPool{}
key := types.NamespacedName{
Name: poolName,
Namespace: poolNamespace,
}
err := c.Get(ctx, key, p)
if err != nil {
logger.Info("fail getting IPPool", "reason", err.Error())
return err
}
if len(p.Status.Allocations) > 0 {
logger.Info("skipping migration for IPPool, already has allocation", "ippool", poolName)
continue
}
allocs := make([]ipamv1alpha1.Allocation, 0)
for i := range nodeList.Items {
node := nodeList.Items[i]
nodeLog := logger.WithValues("node", node.Name)
poolCfg, err := pool.NewConfigReader(&node)
if err != nil {
nodeLog.Info("skip loading data from the node", "reason", err.Error())
continue
}
nodesToClearAnnotation.Insert(node.Name)
nodeIPPoolConfig := poolCfg.GetPoolByName(poolName)
if nodeIPPoolConfig == nil {
nodeLog.Info("skip loading data for pool from the node, pool not configured", "node", node.Name, "pool", poolName)
continue
}
alloc := ipamv1alpha1.Allocation{
NodeName: node.Name,
StartIP: nodeIPPoolConfig.StartIP,
EndIP: nodeIPPoolConfig.EndIP,
}
allocs = append(allocs, alloc)
}
if len(allocs) != 0 {
p.Status.Allocations = allocs
logger.Info(fmt.Sprintf("Updating IPPool status: %v", p))
err = c.Status().Update(ctx, p)
if err != nil {
logger.Info("fail to update pool allocation from node", "reason", err.Error())
return err
}
}
}

for _, nodeName := range sets.List(nodesToClearAnnotation) {
logger.Info("clear IPBlocksAnnotation from node", "name", nodeName)
fmtKey := strings.ReplaceAll(pool.IPBlocksAnnotation, "/", "~1")
patch := []byte(fmt.Sprintf("[{\"op\": \"remove\", \"path\": \"/metadata/annotations/%s\"}]", fmtKey))
err := c.Patch(ctx, &corev1.Node{
ObjectMeta: metav1.ObjectMeta{
Name: nodeName,
},
}, client.RawPatch(types.JSONPatchType, patch))
if err != nil {
logger.Info("fail to remove IPBlocksAnnotation from node", "name", nodeName, "reason", err.Error())
return err
}
}
return nil
}

func buildIPPools(controllerConfig *config.Config, poolNamespace string) map[string]*ipamv1alpha1.IPPool {
var nodeSelector *corev1.NodeSelector
if len(controllerConfig.NodeSelector) > 0 {
nodeSelector = &corev1.NodeSelector{}
selectorsItems := make([]corev1.NodeSelectorTerm, 0, len(controllerConfig.NodeSelector))
for k, v := range controllerConfig.NodeSelector {
selector := corev1.NodeSelectorTerm{
MatchExpressions: []corev1.NodeSelectorRequirement{
{
Key: k,
Operator: corev1.NodeSelectorOpIn,
Values: []string{v},
},
},
MatchFields: nil,
}
selectorsItems = append(selectorsItems, selector)
}
nodeSelector.NodeSelectorTerms = selectorsItems
}
pools := make(map[string]*ipamv1alpha1.IPPool)
for pName, p := range controllerConfig.Pools {
// already validated by Validate function
_, subnet, _ := net.ParseCIDR(p.Subnet)
pools[pName] = &ipamv1alpha1.IPPool{
ObjectMeta: metav1.ObjectMeta{
Name: pName,
Namespace: poolNamespace,
},
Spec: ipamv1alpha1.IPPoolSpec{
Subnet: subnet.String(),
Gateway: p.Gateway,
PerNodeBlockSize: p.PerNodeBlockSize,
NodeSelector: nodeSelector,
},
}
}
return pools
}
82 changes: 82 additions & 0 deletions pkg/ipam-controller/migrator/migrator_suite_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
Copyright 2023, NVIDIA CORPORATION & AFFILIATES
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 migrator_test

import (
"context"
"testing"

. "github.com/onsi/ginkgo/v2"
. "github.com/onsi/gomega"
corev1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/client-go/kubernetes/scheme"
"k8s.io/client-go/rest"
"sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/envtest"

ipamv1alpha1 "github.com/Mellanox/nvidia-k8s-ipam/api/v1alpha1"
)

const (
TestNamespace = "test-ns"
TestConfigMapName = "nvidia-k8s-ipam-config"
)

var (
cfg *rest.Config
k8sClient client.Client
testEnv *envtest.Environment
cFunc context.CancelFunc
ctx context.Context
)

func TestMigrator(t *testing.T) {
RegisterFailHandler(Fail)
RunSpecs(t, "IPAM Controller Migrator Suite")
}

var _ = BeforeSuite(func() {
By("bootstrapping test environment")
testEnv = &envtest.Environment{
CRDDirectoryPaths: []string{"../../../deploy/crds"},
CRDInstallOptions: envtest.CRDInstallOptions{
ErrorIfPathMissing: true,
},
}

ctx, cFunc = context.WithCancel(context.Background())

var err error
err = ipamv1alpha1.AddToScheme(scheme.Scheme)
Expect(err).NotTo(HaveOccurred())

// cfg is defined in this file globally.
cfg, err = testEnv.Start()
Expect(err).NotTo(HaveOccurred())
Expect(cfg).NotTo(BeNil())

k8sClient, err = client.New(cfg, client.Options{Scheme: scheme.Scheme})
Expect(err).NotTo(HaveOccurred())
Expect(k8sClient).NotTo(BeNil())

Expect(k8sClient.Create(ctx, &corev1.Namespace{ObjectMeta: metav1.ObjectMeta{Name: TestNamespace}})).To(BeNil())
})

var _ = AfterSuite(func() {
cFunc()
By("tearing down the test environment")
err := testEnv.Stop()
Expect(err).NotTo(HaveOccurred())
})
Loading

0 comments on commit d6513cd

Please sign in to comment.