kmsv2: re-use DEK while key ID is unchanged

This change updates KMS v2 to not create a new DEK for every
encryption.  Instead, we re-use the DEK while the key ID is stable.

Specifically:

We no longer use a random 12 byte nonce per encryption.  Instead, we
use both a random 4 byte nonce and an 8 byte nonce set via an atomic
counter.  Since each DEK is randomly generated and never re-used,
the combination of DEK and counter are always unique.  Thus there
can never be a nonce collision.  AES GCM strongly encourages the use
of a 12 byte nonce, hence the additional 4 byte random nonce.  We
could leave those 4 bytes set to all zeros, but there is no harm in
setting them to random data (it may help in some edge cases such as
live VM migration).

If the plugin is not healthy, the last DEK will be used for
encryption for up to three minutes (there is no difference on the
behavior of reads which have always used the DEK cache).  This will
reduce the impact of a short plugin outage while making it easy to
perform storage migration after a key ID change (i.e. simply wait
ten minutes after the key ID change before starting the migration).

The DEK rotation cycle is performed in sync with the KMS v2 status
poll thus we always have the correct information to determine if a
read is stale in regards to storage migration.

Signed-off-by: Monis Khan <mok@microsoft.com>
This commit is contained in:
Monis Khan
2023-02-24 16:51:08 -05:00
parent 346f39e18b
commit 832d6f0e19
15 changed files with 1484 additions and 338 deletions

View File

@@ -95,7 +95,10 @@ func (r envelope) plainTextPayload(secretETCDPath string) ([]byte, error) {
// etcd path of the key is used as the authenticated context - need to pass it to decrypt
ctx := context.Background()
dataCtx := value.DefaultContext([]byte(secretETCDPath))
aesgcmTransformer := aestransformer.NewGCMTransformer(block)
aesgcmTransformer, err := aestransformer.NewGCMTransformer(block)
if err != nil {
return nil, fmt.Errorf("failed to create transformer from block: %v", err)
}
plainSecret, _, err := aesgcmTransformer.TransformFromStorage(ctx, r.cipherTextPayload(), dataCtx)
if err != nil {
return nil, fmt.Errorf("failed to transform from storage via AESGCM, err: %w", err)

View File

@@ -23,31 +23,38 @@ import (
"bytes"
"context"
"crypto/aes"
"encoding/binary"
"fmt"
"strings"
"testing"
"time"
"github.com/gogo/protobuf/proto"
clientv3 "go.etcd.io/etcd/client/v3"
corev1 "k8s.io/api/core/v1"
apiextensionsclientset "k8s.io/apiextensions-apiserver/pkg/client/clientset/clientset"
"k8s.io/apimachinery/pkg/api/meta"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime/schema"
"k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/apimachinery/pkg/util/uuid"
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/apiserver/pkg/features"
"k8s.io/apiserver/pkg/server/options/encryptionconfig"
"k8s.io/apiserver/pkg/storage/storagebackend"
"k8s.io/apiserver/pkg/storage/value"
aestransformer "k8s.io/apiserver/pkg/storage/value/encrypt/aes"
"k8s.io/apiserver/pkg/storage/value/encrypt/envelope/kmsv2"
kmstypes "k8s.io/apiserver/pkg/storage/value/encrypt/envelope/kmsv2/v2alpha1"
kmsv2mock "k8s.io/apiserver/pkg/storage/value/encrypt/envelope/testing/v2alpha1"
utilfeature "k8s.io/apiserver/pkg/util/feature"
"k8s.io/client-go/dynamic"
"k8s.io/client-go/kubernetes"
featuregatetesting "k8s.io/component-base/featuregate/testing"
kmsv2api "k8s.io/kms/apis/v2alpha1"
kmsv2svc "k8s.io/kms/pkg/service"
"k8s.io/kubernetes/test/integration"
"k8s.io/kubernetes/test/integration/etcd"
)
@@ -91,8 +98,11 @@ func (r envelopekmsv2) plainTextPayload(secretETCDPath string) ([]byte, error) {
return nil, fmt.Errorf("failed to initialize AES Cipher: %v", err)
}
ctx := context.Background()
dataCtx := value.DefaultContext([]byte(secretETCDPath))
aesgcmTransformer := aestransformer.NewGCMTransformer(block)
dataCtx := value.DefaultContext(secretETCDPath)
aesgcmTransformer, err := aestransformer.NewGCMTransformer(block)
if err != nil {
return nil, fmt.Errorf("failed to create transformer from block: %v", err)
}
data, err := r.cipherTextPayload()
if err != nil {
return nil, fmt.Errorf("failed to get cipher text payload: %v", err)
@@ -166,7 +176,7 @@ resources:
plainTextDEK: plainTextDEK,
}
wantPrefix := string(envelopeData.prefix())
wantPrefix := envelopeData.prefix()
if !bytes.HasPrefix(rawEnvelope, []byte(wantPrefix)) {
t.Fatalf("expected secret to be prefixed with %s, but got %s", wantPrefix, rawEnvelope)
}
@@ -177,7 +187,7 @@ resources:
if err != nil {
t.Fatalf("failed to get ciphertext DEK from KMSv2 Plugin: %v", err)
}
decryptResponse, err := pluginMock.Decrypt(ctx, &kmsv2api.DecryptRequest{Uid: string(types.UID(uuid.NewUUID())), Ciphertext: ciphertext})
decryptResponse, err := pluginMock.Decrypt(ctx, &kmsv2api.DecryptRequest{Uid: string(uuid.NewUUID()), Ciphertext: ciphertext})
if err != nil {
t.Fatalf("failed to decrypt DEK, %v", err)
}
@@ -213,8 +223,10 @@ resources:
// 1. When the key ID is unchanged, the resource version must not change
// 2. When the key ID changes, the resource version changes (but only once)
// 3. For all subsequent updates, the resource version must not change
// 4. When kms-plugin is down, expect creation of new pod and encryption to fail
// 5. when kms-plugin is down, no-op update for a pod should succeed and not result in RV change
// 4. When kms-plugin is down, expect creation of new pod and encryption to succeed while the DEK is valid
// 5. when kms-plugin is down, no-op update for a pod should succeed and not result in RV change while the DEK is valid
// 6. When kms-plugin is down, expect creation of new pod and encryption to fail once the DEK is invalid
// 7. when kms-plugin is down, no-op update for a pod should succeed and not result in RV change even once the DEK is valid
func TestKMSv2ProviderKeyIDStaleness(t *testing.T) {
defer featuregatetesting.SetFeatureGateDuringTest(t, utilfeature.DefaultFeatureGate, features.KMSv2, true)()
@@ -247,14 +259,16 @@ resources:
}
defer test.cleanUp()
testPod, err := test.createPod(testNamespace, dynamic.NewForConfigOrDie(test.kubeAPIServer.ClientConfig))
dynamicClient := dynamic.NewForConfigOrDie(test.kubeAPIServer.ClientConfig)
testPod, err := test.createPod(testNamespace, dynamicClient)
if err != nil {
t.Fatalf("Failed to create test pod, error: %v, ns: %s", err, testNamespace)
}
version1 := testPod.GetResourceVersion()
// 1. no-op update for the test pod should not result in any RV change
updatedPod, err := test.inplaceUpdatePod(testNamespace, testPod, dynamic.NewForConfigOrDie(test.kubeAPIServer.ClientConfig))
updatedPod, err := test.inplaceUpdatePod(testNamespace, testPod, dynamicClient)
if err != nil {
t.Fatalf("Failed to update test pod, error: %v, ns: %s", err, testNamespace)
}
@@ -262,6 +276,30 @@ resources:
if version1 != version2 {
t.Fatalf("Resource version should not have changed. old pod: %v, new pod: %v", testPod, updatedPod)
}
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute)
t.Cleanup(cancel)
var firstEncryptedDEK []byte
assertPodDEKs(ctx, t, test.kubeAPIServer.ServerOpts.Etcd.StorageConfig,
1, 1,
"k8s:enc:kms:v2:kms-provider:",
func(_ int, counter uint64, etcdKey string, obj kmstypes.EncryptedObject) {
firstEncryptedDEK = obj.EncryptedDEK
if obj.KeyID != "1" {
t.Errorf("key %s: want key ID %s, got %s", etcdKey, "1", obj.KeyID)
}
// with the first key we perform encryption during the following steps:
// - create
const want = 1_000_000_000 + 1 // zero value of counter is one billion
if want != counter {
t.Errorf("key %s: counter nonce is invalid: want %d, got %d", etcdKey, want, counter)
}
},
)
// 2. no-op update for the test pod with keyID update should result in RV change
pluginMock.UpdateKeyID()
if err := kmsv2mock.WaitForBase64PluginToBeUpdated(pluginMock); err != nil {
@@ -272,7 +310,8 @@ resources:
version3 := ""
err = wait.Poll(time.Second, time.Minute,
func() (bool, error) {
updatedPod, err = test.inplaceUpdatePod(testNamespace, updatedPod, dynamic.NewForConfigOrDie(test.kubeAPIServer.ClientConfig))
t.Log("polling for in-place update rv change")
updatedPod, err = test.inplaceUpdatePod(testNamespace, updatedPod, dynamicClient)
if err != nil {
return false, err
}
@@ -290,8 +329,29 @@ resources:
t.Fatalf("Resource version should have changed after keyID update. old pod: %v, new pod: %v", testPod, updatedPod)
}
var wantCount uint64 = 1_000_000_000 // zero value of counter is one billion
wantCount++ // in place update with RV change
// with the second key we perform encryption during the following steps:
// - in place update with RV change
// - delete (which does an update to set deletion timestamp)
// - create
checkDEK := func(_ int, counter uint64, etcdKey string, obj kmstypes.EncryptedObject) {
if bytes.Equal(obj.EncryptedDEK, firstEncryptedDEK) {
t.Errorf("key %s: incorrectly has the same EDEK", etcdKey)
}
if obj.KeyID != "2" {
t.Errorf("key %s: want key ID %s, got %s", etcdKey, "2", obj.KeyID)
}
if wantCount != counter {
t.Errorf("key %s: counter nonce is invalid: want %d, got %d", etcdKey, wantCount, counter)
}
}
// 3. no-op update for the updated pod should not result in RV change
updatedPod, err = test.inplaceUpdatePod(testNamespace, updatedPod, dynamic.NewForConfigOrDie(test.kubeAPIServer.ClientConfig))
updatedPod, err = test.inplaceUpdatePod(testNamespace, updatedPod, dynamicClient)
if err != nil {
t.Fatalf("Failed to update test pod, error: %v, ns: %s", err, testNamespace)
}
@@ -300,25 +360,180 @@ resources:
t.Fatalf("Resource version should not have changed again after the initial version updated as a result of the keyID update. old pod: %v, new pod: %v", testPod, updatedPod)
}
// 4. when kms-plugin is down, expect creation of new pod and encryption to fail
// delete the pod so that it can be recreated
if err := test.deletePod(testNamespace, dynamicClient); err != nil {
t.Fatalf("failed to delete test pod: %v", err)
}
wantCount++ // we cannot assert against the counter being 2 since the pod gets deleted
// 4. when kms-plugin is down, expect creation of new pod and encryption to succeed because the DEK is still valid
pluginMock.EnterFailedState()
mustBeUnHealthy(t, "/kms-providers",
"internal server error: kms-provider-0: rpc error: code = FailedPrecondition desc = failed precondition - key disabled",
test.kubeAPIServer.ClientConfig)
_, err = test.createPod(testNamespace, dynamic.NewForConfigOrDie(test.kubeAPIServer.ClientConfig))
if err == nil || !strings.Contains(err.Error(), "failed to encrypt") {
t.Fatalf("Create test pod should have failed due to encryption, ns: %s", testNamespace)
newPod, err := test.createPod(testNamespace, dynamicClient)
if err != nil {
t.Fatalf("Create test pod should have succeeded due to valid DEK, ns: %s, got: %v", testNamespace, err)
}
wantCount++
version5 := newPod.GetResourceVersion()
// 5. when kms-plugin is down, no-op update for a pod should succeed and not result in RV change
updatedPod, err = test.inplaceUpdatePod(testNamespace, updatedPod, dynamic.NewForConfigOrDie(test.kubeAPIServer.ClientConfig))
// 5. when kms-plugin is down and DEK is valid, no-op update for a pod should succeed and not result in RV change
updatedPod, err = test.inplaceUpdatePod(testNamespace, newPod, dynamicClient)
if err != nil {
t.Fatalf("Failed to perform no-op update on pod when kms-plugin is down, error: %v, ns: %s", err, testNamespace)
}
version5 := updatedPod.GetResourceVersion()
if version3 != version5 {
t.Fatalf("Resource version should not have changed again after the initial version updated as a result of the keyID update. old pod: %v, new pod: %v", testPod, updatedPod)
version6 := updatedPod.GetResourceVersion()
if version5 != version6 {
t.Fatalf("Resource version should not have changed again after the initial version updated as a result of the keyID update. old pod: %v, new pod: %v", newPod, updatedPod)
}
// Invalidate the DEK by moving the current time forward
origNowFunc := kmsv2.NowFunc
t.Cleanup(func() { kmsv2.NowFunc = origNowFunc })
kmsv2.NowFunc = func() time.Time { return origNowFunc().Add(5 * time.Minute) }
// 6. when kms-plugin is down, expect creation of new pod and encryption to fail because the DEK is invalid
_, err = test.createPod(testNamespace, dynamicClient)
if err == nil || !strings.Contains(err.Error(), `EDEK with keyID "2" expired at 2`) {
t.Fatalf("Create test pod should have failed due to encryption, ns: %s, got: %v", testNamespace, err)
}
// 7. when kms-plugin is down and DEK is invalid, no-op update for a pod should succeed and not result in RV change
updatedNewPod, err := test.inplaceUpdatePod(testNamespace, newPod, dynamicClient)
if err != nil {
t.Fatalf("Failed to perform no-op update on pod when kms-plugin is down, error: %v, ns: %s", err, testNamespace)
}
version7 := updatedNewPod.GetResourceVersion()
if version5 != version7 {
t.Fatalf("Resource version should not have changed again after the initial version updated as a result of the keyID update. old pod: %v, new pod: %v", newPod, updatedNewPod)
}
assertPodDEKs(ctx, t, test.kubeAPIServer.ServerOpts.Etcd.StorageConfig,
1, 1, "k8s:enc:kms:v2:kms-provider:", checkDEK,
)
}
func TestKMSv2ProviderDEKReuse(t *testing.T) {
defer featuregatetesting.SetFeatureGateDuringTest(t, utilfeature.DefaultFeatureGate, features.KMSv2, true)()
ctx, cancel := context.WithTimeout(context.Background(), time.Minute)
t.Cleanup(cancel)
encryptionConfig := `
kind: EncryptionConfiguration
apiVersion: apiserver.config.k8s.io/v1
resources:
- resources:
- pods
providers:
- kms:
apiVersion: v2
name: kms-provider
endpoint: unix:///@kms-provider.sock
`
pluginMock, err := kmsv2mock.NewBase64Plugin("@kms-provider.sock")
if err != nil {
t.Fatalf("failed to create mock of KMSv2 Plugin: %v", err)
}
go pluginMock.Start()
if err := kmsv2mock.WaitForBase64PluginToBeUp(pluginMock); err != nil {
t.Fatalf("Failed start plugin, err: %v", err)
}
defer pluginMock.CleanUp()
test, err := newTransformTest(t, encryptionConfig, false, "")
if err != nil {
t.Fatalf("failed to start KUBE API Server with encryptionConfig\n %s, error: %v", encryptionConfig, err)
}
t.Cleanup(test.cleanUp)
client := kubernetes.NewForConfigOrDie(test.kubeAPIServer.ClientConfig)
const podCount = 1_000
for i := 0; i < podCount; i++ {
if _, err := client.CoreV1().Pods(testNamespace).Create(ctx, &corev1.Pod{
ObjectMeta: metav1.ObjectMeta{
Name: fmt.Sprintf("dek-reuse-%04d", i+1), // making creation order match returned list order / nonce counter
},
Spec: corev1.PodSpec{
Containers: []corev1.Container{
{
Name: "busybox",
Image: "busybox",
},
},
},
}, metav1.CreateOptions{}); err != nil {
t.Fatal(err)
}
}
assertPodDEKs(ctx, t, test.kubeAPIServer.ServerOpts.Etcd.StorageConfig,
podCount, 1, // key ID does not change during the test so we should only have a single DEK
"k8s:enc:kms:v2:kms-provider:",
func(i int, counter uint64, etcdKey string, obj kmstypes.EncryptedObject) {
if obj.KeyID != "1" {
t.Errorf("key %s: want key ID %s, got %s", etcdKey, "1", obj.KeyID)
}
// zero value of counter is one billion so the first value will be one billion plus one
// hence we add that to our zero based index to calculate the expected nonce
if uint64(i+1_000_000_000+1) != counter {
t.Errorf("key %s: counter nonce is invalid: want %d, got %d", etcdKey, i+1, counter)
}
},
)
}
func assertPodDEKs(ctx context.Context, t *testing.T, config storagebackend.Config, podCount, dekCount int, kmsPrefix string,
f func(i int, counter uint64, etcdKey string, obj kmstypes.EncryptedObject)) {
t.Helper()
rawClient, etcdClient, err := integration.GetEtcdClients(config.Transport)
if err != nil {
t.Fatalf("failed to create etcd client: %v", err)
}
t.Cleanup(func() { _ = rawClient.Close() })
response, err := etcdClient.Get(ctx, "/"+config.Prefix+"/pods/"+testNamespace+"/", clientv3.WithPrefix())
if err != nil {
t.Fatal(err)
}
if len(response.Kvs) != podCount {
t.Fatalf("expected %d KVs, but got %d", podCount, len(response.Kvs))
}
out := make([]kmstypes.EncryptedObject, len(response.Kvs))
for i, kv := range response.Kvs {
v := bytes.TrimPrefix(kv.Value, []byte(kmsPrefix))
if err := proto.Unmarshal(v, &out[i]); err != nil {
t.Fatal(err)
}
nonce := out[i].EncryptedData[:12]
randN := nonce[:4]
count := nonce[4:]
if bytes.Equal(randN, make([]byte, len(randN))) {
t.Errorf("key %s: got all zeros for first four bytes", string(kv.Key))
}
counter := binary.LittleEndian.Uint64(count)
f(i, counter, string(kv.Key), out[i])
}
uniqueDEKs := sets.NewString()
for _, object := range out {
uniqueDEKs.Insert(string(object.EncryptedDEK))
}
if uniqueDEKs.Len() != dekCount {
t.Errorf("expected %d DEKs, got: %d", dekCount, uniqueDEKs.Len())
}
}

View File

@@ -140,7 +140,10 @@ func unSealWithGCMTransformer(ctx context.Context, cipherText []byte, dataCtx va
return nil, fmt.Errorf("failed to create block cipher: %v", err)
}
gcmTransformer := aestransformer.NewGCMTransformer(block)
gcmTransformer, err := aestransformer.NewGCMTransformer(block)
if err != nil {
return nil, fmt.Errorf("failed to create transformer from block: %v", err)
}
clearText, _, err := gcmTransformer.TransformFromStorage(ctx, cipherText, dataCtx)
if err != nil {

View File

@@ -454,6 +454,15 @@ func (e *transformTest) createPod(namespace string, dynamicInterface dynamic.Int
return pod, nil
}
func (e *transformTest) deletePod(namespace string, dynamicInterface dynamic.Interface) error {
podGVR := gvr("", "v1", "pods")
stubObj, err := getStubObj(podGVR)
if err != nil {
return err
}
return dynamicInterface.Resource(podGVR).Namespace(namespace).Delete(context.TODO(), stubObj.GetName(), metav1.DeleteOptions{})
}
func (e *transformTest) inplaceUpdatePod(namespace string, obj *unstructured.Unstructured, dynamicInterface dynamic.Interface) (*unstructured.Unstructured, error) {
podGVR := gvr("", "v1", "pods")
pod, err := inplaceUpdateResource(dynamicInterface, podGVR, namespace, obj)
@@ -519,7 +528,7 @@ func (e *transformTest) printMetrics() error {
func mustBeHealthy(t kubeapiservertesting.Logger, checkName, wantBodyContains string, clientConfig *rest.Config, excludes ...string) {
t.Helper()
var restErr error
pollErr := wait.PollImmediate(2*time.Second, wait.ForeverTestTimeout, func() (bool, error) {
pollErr := wait.PollImmediate(2*time.Second, time.Minute, func() (bool, error) {
body, ok, err := getHealthz(checkName, clientConfig, excludes...)
restErr = err
if err != nil {
@@ -540,7 +549,7 @@ func mustBeHealthy(t kubeapiservertesting.Logger, checkName, wantBodyContains st
func mustBeUnHealthy(t kubeapiservertesting.Logger, checkName, wantBodyContains string, clientConfig *rest.Config, excludes ...string) {
t.Helper()
var restErr error
pollErr := wait.PollImmediate(2*time.Second, wait.ForeverTestTimeout, func() (bool, error) {
pollErr := wait.PollImmediate(2*time.Second, time.Minute, func() (bool, error) {
body, ok, err := getHealthz(checkName, clientConfig, excludes...)
restErr = err
if err != nil {