Skip to content

Commit

Permalink
cliccl,engineccl: Generate and load JWKS-format keys for EAR
Browse files Browse the repository at this point in the history
Fixes: #119767
Release note (enterprise change): `cockroach gen encryption-key` now
accepts a `--version=2` parameter. Version 2 keys activate a new
encryption implementation with improved performance. This is
expected to become the default in CockroachDB 24.2.
  • Loading branch information
bdarnell committed Mar 6, 2024
1 parent 822745b commit 1a22e17
Show file tree
Hide file tree
Showing 7 changed files with 195 additions and 52 deletions.
2 changes: 2 additions & 0 deletions pkg/ccl/cliccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ go_library(
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_pebble//vfs",
"@com_github_cockroachdb_redact//:redact",
"@com_github_lestrrat_go_jwx//jwk",
"@com_github_olekukonko_tablewriter//:tablewriter",
"@com_github_spf13_cobra//:cobra",
"@com_github_spf13_pflag//:pflag",
Expand Down Expand Up @@ -82,6 +83,7 @@ go_test(
"//pkg/util/log",
"//pkg/util/randutil",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_pebble//vfs",
"@com_github_spf13_cobra//:cobra",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
Expand Down
74 changes: 62 additions & 12 deletions pkg/ccl/cliccl/gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,28 +10,79 @@ package cliccl

import (
"crypto/rand"
"encoding/hex"
"encoding/json"
"fmt"
"io"
"os"

"github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl/enginepbccl"
"github.com/cockroachdb/cockroach/pkg/cli"
"github.com/cockroachdb/errors"
"github.com/lestrrat-go/jwx/jwk"
"github.com/spf13/cobra"
)

var aesSizeFlag int
var overwriteKeyFlag bool
var keyVersionFlag int

func genEncryptionKey(encryptionKeyPath string, aesSize int, overwriteKey bool) error {
func genEncryptionKey(
encryptionKeyPath string, aesSize int, overwriteKey bool, keyVersion int,
) error {
// Check encryptionKeySize is suitable for the encryption algorithm.
if aesSize != 128 && aesSize != 192 && aesSize != 256 {
return fmt.Errorf("store key size should be 128, 192, or 256 bits, got %d", aesSize)
}

// 32 bytes are reserved for key ID.
kSize := aesSize/8 + 32
b := make([]byte, kSize)
if _, err := rand.Read(b); err != nil {
return fmt.Errorf("failed to create key with size %d bytes", kSize)
keyID := make([]byte, 32)
if _, err := rand.Read(keyID); err != nil {
return fmt.Errorf("failed to create random key ID")
}
key := make([]byte, aesSize/8)
if _, err := rand.Read(key); err != nil {
return fmt.Errorf("failed to create key with size %d bytes", aesSize/8)
}

var b []byte
switch keyVersion {
case 1:
b = append(b, keyID...)
b = append(b, key...)
case 2:
var et enginepbccl.EncryptionType
switch aesSize {
case 128:
et = enginepbccl.EncryptionType_AES_128_CTR_V2
case 192:
et = enginepbccl.EncryptionType_AES_192_CTR_V2
case 256:
et = enginepbccl.EncryptionType_AES_256_CTR_V2
}

symKey := jwk.NewSymmetricKey()
if err := symKey.FromRaw(key); err != nil {
return errors.Wrap(err, "error setting key bytes")
}
if err := symKey.Set(jwk.KeyIDKey, hex.EncodeToString(keyID)); err != nil {
return errors.Wrap(err, "error setting key id")
}
alg, err := et.JWKAlgorithm()
if err != nil {
return err
}
if err := symKey.Set(jwk.AlgorithmKey, alg); err != nil {
return errors.Wrap(err, "error setting algorithm")
}

keySet := jwk.NewSet()
keySet.Add(symKey)

b, err = json.Marshal(keySet)
if err != nil {
return errors.Wrap(err, "error writing key to json: %s")
}
default:
return fmt.Errorf("unsupported version %d", keyVersion)
}

// Write key to the file with owner read/write permission.
Expand All @@ -44,10 +95,7 @@ func genEncryptionKey(encryptionKeyPath string, aesSize int, overwriteKey bool)
if err != nil {
return err
}
n, err := f.Write(b)
if err == nil && n < len(b) {
err = io.ErrShortWrite
}
_, err = f.Write(b)
if err1 := f.Close(); err == nil {
err = err1
}
Expand All @@ -68,7 +116,7 @@ The resulting key file will be 32 bytes (random key ID) + key_size in bytes.
RunE: func(cmd *cobra.Command, args []string) error {
encryptionKeyPath := args[0]

err := genEncryptionKey(encryptionKeyPath, aesSizeFlag, overwriteKeyFlag)
err := genEncryptionKey(encryptionKeyPath, aesSizeFlag, overwriteKeyFlag, keyVersionFlag)

if err != nil {
return err
Expand All @@ -86,4 +134,6 @@ func init() {
"AES key size for encryption at rest (one of: 128, 192, 256)")
genEncryptionKeyCmd.PersistentFlags().BoolVar(&overwriteKeyFlag, "overwrite", false,
"Overwrite key if it exists")
genEncryptionKeyCmd.PersistentFlags().IntVar(&keyVersionFlag, "version", 1,
"Encryption format version (1 or 2)")
}
54 changes: 29 additions & 25 deletions pkg/ccl/cliccl/gen_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,30 +28,34 @@ func TestGenEncryptionKey(t *testing.T) {

dir := t.TempDir()

for _, keySize := range []int{128, 192, 256} {
t.Run(fmt.Sprintf("size=%d", keySize), func(t *testing.T) {
keyName := fmt.Sprintf("aes-%d.key", keySize)
keyPath := filepath.Join(dir, keyName)

err := genEncryptionKey(keyPath, keySize, false)
require.NoError(t, err)

info, err := os.Stat(keyPath)
require.NoError(t, err)
// 32-byte id plus the key.
assert.EqualValues(t, 32+(keySize/8), info.Size())

key, err := engineccl.LoadKeyFromFile(vfs.Default, keyPath)
require.NoError(t, err)
assert.EqualValues(t, keySize/8, len(key.Key))
// Key ID is hex encoded on load so it's 64 bytes here but 32 in the file size.
assert.EqualValues(t, 64, len(key.Info.KeyId))

err = genEncryptionKey(keyPath, keySize, false)
require.ErrorContains(t, err, fmt.Sprintf("%s: file exists", keyName))

err = genEncryptionKey(keyPath, keySize, true /* overwrite */)
require.NoError(t, err)
})
for _, keyVersion := range []int{1, 2} {
for _, keySize := range []int{128, 192, 256} {
t.Run(fmt.Sprintf("version=%d/size=%d", keyVersion, keySize), func(t *testing.T) {
keyName := fmt.Sprintf("aes-%d-v%d.key", keySize, keyVersion)
keyPath := filepath.Join(dir, keyName)

err := genEncryptionKey(keyPath, keySize, false, keyVersion)
require.NoError(t, err)

if keyVersion == 1 {
info, err := os.Stat(keyPath)
require.NoError(t, err)
// 32-byte id plus the key.
assert.EqualValues(t, 32+(keySize/8), info.Size())
}

key, err := engineccl.LoadKeyFromFile(vfs.Default, keyPath)
require.NoError(t, err)
assert.EqualValues(t, keySize/8, len(key.Key))
// Key ID is hex encoded on load so it's 64 bytes here but 32 in the file size.
assert.EqualValues(t, 64, len(key.Info.KeyId))

err = genEncryptionKey(keyPath, keySize, false, keyVersion)
require.ErrorContains(t, err, fmt.Sprintf("%s: file exists", keyName))

err = genEncryptionKey(keyPath, keySize, true /* overwrite */, keyVersion)
require.NoError(t, err)
})
}
}
}
2 changes: 2 additions & 0 deletions pkg/ccl/storageccl/engineccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ go_library(
"@com_github_cockroachdb_pebble//vfs",
"@com_github_cockroachdb_pebble//vfs/atomicfs",
"@com_github_gogo_protobuf//proto",
"@com_github_lestrrat_go_jwx//jwa",
"@com_github_lestrrat_go_jwx//jwk",
],
)

Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/storageccl/engineccl/enginepbccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_proto_library(

go_library(
name = "enginepbccl",
srcs = ["key_registry.go"],
embed = [":enginepbccl_go_proto"],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl/enginepbccl",
visibility = ["//visibility:public"],
Expand Down
50 changes: 50 additions & 0 deletions pkg/ccl/storageccl/engineccl/enginepbccl/key_registry.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// Copyright 2024 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package enginepbccl

import fmt "fmt"

// JWKAlgorithm returns the JWK algorithm name for this EncryptionType.
func (e EncryptionType) JWKAlgorithm() (string, error) {
switch e {
case EncryptionType_AES128_CTR:
return "cockroach-aes-128-ctr-v1", nil
case EncryptionType_AES192_CTR:
return "cockroach-aes-192-ctr-v1", nil
case EncryptionType_AES256_CTR:
return "cockroach-aes-256-ctr-v1", nil

case EncryptionType_AES_128_CTR_V2:
return "cockroach-aes-128-ctr-v2", nil
case EncryptionType_AES_192_CTR_V2:
return "cockroach-aes-192-ctr-v2", nil
case EncryptionType_AES_256_CTR_V2:
return "cockroach-aes-192-ctr-v2", nil
}
return "", fmt.Errorf("unknown EncryptionType %d", e)
}

// EncryptionTypeFromJWKAlgorithm returns the EncryptionType for the given algorithm.
func EncryptionTypeFromJWKAlgorithm(s string) (EncryptionType, error) {
switch s {
case "cockroach-aes-128-ctr-v1":
return EncryptionType_AES128_CTR, nil
case "cockroach-aes-192-ctr-v1":
return EncryptionType_AES192_CTR, nil
case "cockroach-aes-256-ctr-v1":
return EncryptionType_AES256_CTR, nil
case "cockroach-aes-128-ctr-v2":
return EncryptionType_AES_128_CTR_V2, nil
case "cockroach-aes-192-ctr-v2":
return EncryptionType_AES_192_CTR_V2, nil
case "cockroach-aes-256-ctr-v2":
return EncryptionType_AES_256_CTR_V2, nil
}
return 0, fmt.Errorf("unknown JWK algorithm name %s", s)
}
64 changes: 49 additions & 15 deletions pkg/ccl/storageccl/engineccl/pebble_key_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ import (
"github.com/cockroachdb/pebble/vfs"
"github.com/cockroachdb/pebble/vfs/atomicfs"
"github.com/gogo/protobuf/proto"
"github.com/lestrrat-go/jwx/jwa"
"github.com/lestrrat-go/jwx/jwk"
)

const (
Expand Down Expand Up @@ -137,21 +139,53 @@ func LoadKeyFromFile(fs vfs.FS, filename string) (*enginepbccl.SecretKey, error)
if err != nil {
return nil, err
}
// keyIDLength bytes for the ID, followed by the key.
keyLength := len(b) - keyIDLength
switch keyLength {
case 16:
key.Info.EncryptionType = enginepbccl.EncryptionType_AES128_CTR
case 24:
key.Info.EncryptionType = enginepbccl.EncryptionType_AES192_CTR
case 32:
key.Info.EncryptionType = enginepbccl.EncryptionType_AES256_CTR
default:
return nil, fmt.Errorf("store key of unsupported length: %d", keyLength)
}
key.Key = b[keyIDLength:]
// Hex encoding to make it human readable.
key.Info.KeyId = hex.EncodeToString(b[:keyIDLength])

// We support two file formats:
// - Old-style keys are just raw random data with no delimiters; the only
// format validity requirement is that the file has the right length.
// - New-style keys are in JWK format (we support both JWK (single-key)
// and JWKS (set of keys)) formats, although we currently require
// that JWKS sets contain exactly one key).
// Since random generation of 48+ bytes will not produce a valid json object,
// if the file parses as JWK, assume that was the intended format.
if keySet, jwkErr := jwk.Parse(b); jwkErr == nil {
jwKey, ok := keySet.Get(0)
if !ok {
return nil, fmt.Errorf("JWKS file contains no keys")
}
if keySet.Len() != 1 {
return nil, fmt.Errorf("expected exactly 1 key in JWKS file, found %d", keySet.Len())
}
if jwKey.KeyType() != jwa.OctetSeq {
return nil, fmt.Errorf("expected kty=oct, found %s", jwKey.KeyType())
}
key.Info.EncryptionType, err = enginepbccl.EncryptionTypeFromJWKAlgorithm(jwKey.Algorithm())
if err != nil {
return nil, err
}
key.Info.KeyId = jwKey.KeyID()
symKey, ok := jwKey.(jwk.SymmetricKey)
if !ok {
return nil, fmt.Errorf("error converting jwk.Key to SymmetricKey")
}
key.Key = symKey.Octets()
} else {
// keyIDLength bytes for the ID, followed by the key.
keyLength := len(b) - keyIDLength
switch keyLength {
case 16:
key.Info.EncryptionType = enginepbccl.EncryptionType_AES128_CTR
case 24:
key.Info.EncryptionType = enginepbccl.EncryptionType_AES192_CTR
case 32:
key.Info.EncryptionType = enginepbccl.EncryptionType_AES256_CTR
default:
return nil, errors.Wrapf(jwkErr, "could not parse store key. Key length %d is not valid for old-style key", keyLength)
}
key.Key = b[keyIDLength:]
// Hex encoding to make it human readable.
key.Info.KeyId = hex.EncodeToString(b[:keyIDLength])
}
key.Info.CreationTime = now
key.Info.Source = filename

Expand Down

0 comments on commit 1a22e17

Please sign in to comment.