keystore serialization (#13106)

This changeset implements the keystore serialization/deserialization:

* Adds a JSON serialization extension for the `RootKey` struct, along with a metadata stub. When we serialize RootKey to the on-disk keystore, we want to base64 encode the key material but also exclude any frequently-changing fields which are stored in raft.
* Implements methods for loading/saving keys to the keystore.
* Implements methods for restoring the whole keystore from disk.
* Wires it all up with the `Keyring` RPC handlers and fixes up any fallout on tests.
This commit is contained in:
Tim Gross
2022-05-25 15:05:30 -04:00
parent 0b0aa3efe8
commit ce8e7f1788
12 changed files with 542 additions and 82 deletions

View File

@@ -32,9 +32,9 @@ func TestKeyring_CRUD(t *testing.T) {
// Write a new active key, forcing a rotation
id := "fd77c376-9785-4c80-8e62-4ec3ab5f8b9a"
buf := make([]byte, 128)
buf := make([]byte, 32)
rand.Read(buf)
encodedKey := make([]byte, base64.StdEncoding.EncodedLen(128))
encodedKey := make([]byte, base64.StdEncoding.EncodedLen(32))
base64.StdEncoding.Encode(encodedKey, buf)
wm, err = kr.Update(&RootKey{

View File

@@ -94,8 +94,10 @@ func (s *HTTPServer) keyringUpsertRequest(resp http.ResponseWriter, req *http.Re
return nil, CodedError(400, "decoded key did not include metadata")
}
decodedKey := make([]byte, base64.StdEncoding.DecodedLen(len(key.Key)))
_, err := base64.StdEncoding.Decode(decodedKey, []byte(key.Key))
const keyLen = 32
decodedKey := make([]byte, keyLen)
_, err := base64.StdEncoding.Decode(decodedKey, []byte(key.Key)[:keyLen])
if err != nil {
return nil, CodedError(400, fmt.Sprintf("could not decode key: %v", err))
}

View File

@@ -46,9 +46,9 @@ func TestHTTP_Keyring_CRUD(t *testing.T) {
// Update
keyMeta := rotateResp.Key
keyBuf := make([]byte, 128)
keyBuf := make([]byte, 32)
rand.Read(keyBuf)
encodedKey := make([]byte, base64.StdEncoding.EncodedLen(128))
encodedKey := make([]byte, base64.StdEncoding.EncodedLen(32))
base64.StdEncoding.Encode(encodedKey, keyBuf)
newID := uuid.Generate()

View File

@@ -1,19 +1,95 @@
package nomad
import (
"bytes"
"crypto/aes"
"crypto/cipher"
"encoding/base64"
"encoding/json"
"fmt"
"io/fs"
"os"
"path/filepath"
"strings"
"sync"
"github.com/hashicorp/go-msgpack/codec"
"golang.org/x/crypto/chacha20poly1305"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/nomad/structs"
)
const nomadKeystoreExtension = ".nks.json"
// Encrypter is the keyring for secure variables.
type Encrypter struct {
ciphers map[string]cipher.AEAD // map of key IDs to ciphers
lock sync.RWMutex
keys map[string]*structs.RootKey // map of key IDs to key material
ciphers map[string]cipher.AEAD // map of key IDs to ciphers
keystorePath string
}
func NewEncrypter() *Encrypter {
return &Encrypter{
ciphers: make(map[string]cipher.AEAD),
// NewEncrypter loads or creates a new local keystore and returns an
// encryption keyring with the keys it finds.
func NewEncrypter(keystorePath string) (*Encrypter, error) {
err := os.MkdirAll(keystorePath, 0700)
if err != nil {
return nil, err
}
encrypter, err := encrypterFromKeystore(keystorePath)
if err != nil {
return nil, err
}
return encrypter, nil
}
func encrypterFromKeystore(keystoreDirectory string) (*Encrypter, error) {
encrypter := &Encrypter{
ciphers: make(map[string]cipher.AEAD),
keys: make(map[string]*structs.RootKey),
keystorePath: keystoreDirectory,
}
err := filepath.Walk(keystoreDirectory, func(path string, info fs.FileInfo, err error) error {
if err != nil {
return fmt.Errorf("could not read path %s from keystore: %v", path, err)
}
// skip over subdirectories and non-key files; they shouldn't
// be here but there's no reason to fail startup for it if the
// administrator has left something there
if path != keystoreDirectory && info.IsDir() {
return filepath.SkipDir
}
if !strings.HasSuffix(path, nomadKeystoreExtension) {
return nil
}
id := strings.TrimSuffix(filepath.Base(path), nomadKeystoreExtension)
if !helper.IsUUID(id) {
return nil
}
key, err := encrypter.loadKeyFromStore(path)
if err != nil {
return fmt.Errorf("could not load key file %s from keystore: %v", path, err)
}
if key.Meta.KeyID != id {
return fmt.Errorf("root key ID %s must match key file %s", key.Meta.KeyID, path)
}
err = encrypter.AddKey(key)
if err != nil {
return fmt.Errorf("could not add key file %s to keystore: %v", path, err)
}
return nil
})
if err != nil {
return nil, err
}
return encrypter, nil
}
// Encrypt takes the serialized map[string][]byte from
@@ -21,6 +97,9 @@ func NewEncrypter() *Encrypter {
// for the algorithm, and encrypts the data with the ciper for the
// CurrentRootKeyID. The buffer returned includes the nonce.
func (e *Encrypter) Encrypt(unencryptedData []byte, keyID string) []byte {
e.lock.RLock()
defer e.lock.RUnlock()
// TODO: actually encrypt!
return unencryptedData
}
@@ -28,16 +107,145 @@ func (e *Encrypter) Encrypt(unencryptedData []byte, keyID string) []byte {
// Decrypt takes an encrypted buffer and then root key ID. It extracts
// the nonce, decrypts the content, and returns the cleartext data.
func (e *Encrypter) Decrypt(encryptedData []byte, keyID string) ([]byte, error) {
e.lock.RLock()
defer e.lock.RUnlock()
// TODO: actually decrypt!
return encryptedData, nil
}
// GenerateNewRootKey returns a new root key and its metadata.
func (e *Encrypter) GenerateNewRootKey(algorithm structs.EncryptionAlgorithm) *structs.RootKey {
meta := structs.NewRootKeyMeta()
meta.Algorithm = algorithm
// AddKey stores the key in the keystore and creates a new cipher for it.
func (e *Encrypter) AddKey(rootKey *structs.RootKey) error {
if err := e.addCipher(rootKey); err != nil {
return err
}
if err := e.saveKeyToStore(rootKey); err != nil {
return err
}
return nil
}
// addCipher stores the key in the keyring and creates a new cipher for it.
func (e *Encrypter) addCipher(rootKey *structs.RootKey) error {
if rootKey.Meta == nil {
return fmt.Errorf("missing metadata")
}
var aead cipher.AEAD
var err error
switch rootKey.Meta.Algorithm {
case structs.EncryptionAlgorithmAES256GCM:
block, err := aes.NewCipher(rootKey.Key)
if err != nil {
return fmt.Errorf("could not create cipher: %v", err)
}
aead, err = cipher.NewGCM(block)
if err != nil {
return fmt.Errorf("could not create cipher: %v", err)
}
case structs.EncryptionAlgorithmXChaCha20:
aead, err = chacha20poly1305.NewX(rootKey.Key)
if err != nil {
return fmt.Errorf("could not create cipher: %v", err)
}
default:
return fmt.Errorf("invalid algorithm %s", rootKey.Meta.Algorithm)
}
e.lock.Lock()
defer e.lock.Unlock()
e.ciphers[rootKey.Meta.KeyID] = aead
e.keys[rootKey.Meta.KeyID] = rootKey
return nil
}
// GetKey retrieves the key material by ID from the keyring
func (e *Encrypter) GetKey(keyID string) ([]byte, error) {
e.lock.RLock()
defer e.lock.RUnlock()
key, ok := e.keys[keyID]
if !ok {
return []byte{}, fmt.Errorf("no such key %s in keyring", keyID)
}
return key.Key, nil
}
// RemoveKey removes a key by ID from the keyring
func (e *Encrypter) RemoveKey(keyID string) error {
// TODO: should the server remove the serialized file here?
// TODO: given that it's irreversible, should the server *ever*
// remove the serialized file?
e.lock.Lock()
defer e.lock.Unlock()
delete(e.ciphers, keyID)
delete(e.keys, keyID)
return nil
}
// saveKeyToStore serializes a root key to the on-disk keystore.
func (e *Encrypter) saveKeyToStore(rootKey *structs.RootKey) error {
var buf bytes.Buffer
enc := codec.NewEncoder(&buf, structs.JsonHandleWithExtensions)
err := enc.Encode(rootKey)
if err != nil {
return err
}
path := filepath.Join(e.keystorePath, rootKey.Meta.KeyID+nomadKeystoreExtension)
err = os.WriteFile(path, buf.Bytes(), 0600)
if err != nil {
return err
}
return nil
}
// loadKeyFromStore deserializes a root key from disk.
func (e *Encrypter) loadKeyFromStore(path string) (*structs.RootKey, error) {
raw, err := os.ReadFile(path)
if err != nil {
return nil, err
}
storedKey := &struct {
Meta *structs.RootKeyMetaStub
Key string
}{}
if err := json.Unmarshal(raw, storedKey); err != nil {
return nil, err
}
meta := &structs.RootKeyMeta{
Active: storedKey.Meta.Active,
KeyID: storedKey.Meta.KeyID,
Algorithm: storedKey.Meta.Algorithm,
CreateTime: storedKey.Meta.CreateTime,
}
if err = meta.Validate(); err != nil {
return nil, err
}
// Note: we expect to have null bytes for padding, but we don't
// want to use RawStdEncoding which breaks a lot of command line
// tools. So we'll truncate the key to the expected length.
var keyLen int
switch storedKey.Meta.Algorithm {
case structs.EncryptionAlgorithmXChaCha20, structs.EncryptionAlgorithmAES256GCM:
keyLen = 32
default:
return nil, fmt.Errorf("invalid algorithm")
}
key := make([]byte, keyLen)
_, err = base64.StdEncoding.Decode(key, []byte(storedKey.Key)[:keyLen])
if err != nil {
return nil, fmt.Errorf("could not decode key: %v", err)
}
return &structs.RootKey{
Meta: meta,
Key: []byte{}, // TODO: generate based on algorithm
}
Key: key,
}, nil
}

117
nomad/encrypter_test.go Normal file
View File

@@ -0,0 +1,117 @@
package nomad
import (
"path/filepath"
"testing"
msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc"
"github.com/stretchr/testify/require"
"github.com/hashicorp/nomad/ci"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/testutil"
)
// TestEncrypter_LoadSave exercises round-tripping keys to disk
func TestEncrypter_LoadSave(t *testing.T) {
ci.Parallel(t)
tmpDir := t.TempDir()
encrypter, err := NewEncrypter(tmpDir)
require.NoError(t, err)
algos := []structs.EncryptionAlgorithm{
structs.EncryptionAlgorithmAES256GCM,
structs.EncryptionAlgorithmXChaCha20,
}
for _, algo := range algos {
t.Run(string(algo), func(t *testing.T) {
key, err := structs.NewRootKey(algo)
require.NoError(t, err)
require.NoError(t, encrypter.saveKeyToStore(key))
gotKey, err := encrypter.loadKeyFromStore(
filepath.Join(tmpDir, key.Meta.KeyID+".nks.json"))
require.NoError(t, err)
require.NoError(t, encrypter.addCipher(gotKey))
})
}
}
// TestEncrypter_Restore exercises the entire reload of a keystore,
// including pairing metadata with key material
func TestEncrypter_Restore(t *testing.T) {
ci.Parallel(t)
// use a known tempdir so that we can restore from it
tmpDir := t.TempDir()
srv, rootToken, shutdown := TestACLServer(t, func(c *Config) {
c.NodeName = "node1"
c.NumSchedulers = 0
c.DevMode = false
c.DataDir = tmpDir
})
defer shutdown()
testutil.WaitForLeader(t, srv.RPC)
codec := rpcClient(t, srv)
nodeID := srv.GetConfig().NodeID
// Send a few key rotations to add keys
rotateReq := &structs.KeyringRotateRootKeyRequest{
WriteRequest: structs.WriteRequest{
Region: "global",
AuthToken: rootToken.SecretID,
},
}
var rotateResp structs.KeyringRotateRootKeyResponse
for i := 0; i < 4; i++ {
err := msgpackrpc.CallWithCodec(codec, "Keyring.Rotate", rotateReq, &rotateResp)
require.NoError(t, err)
}
shutdown()
srv2, rootToken, shutdown2 := TestACLServer(t, func(c *Config) {
c.NodeID = nodeID
c.NodeName = "node1"
c.NumSchedulers = 0
c.DevMode = false
c.DataDir = tmpDir
})
defer shutdown2()
testutil.WaitForLeader(t, srv2.RPC)
codec = rpcClient(t, srv2)
// Verify we've restored all the keys from the old keystore
listReq := &structs.KeyringListRootKeyMetaRequest{
QueryOptions: structs.QueryOptions{
Region: "global",
},
}
var listResp structs.KeyringListRootKeyMetaResponse
err := msgpackrpc.CallWithCodec(codec, "Keyring.List", listReq, &listResp)
require.NoError(t, err)
require.Len(t, listResp.Keys, 4)
for _, keyMeta := range listResp.Keys {
getReq := &structs.KeyringGetRootKeyRequest{
KeyID: keyMeta.KeyID,
QueryOptions: structs.QueryOptions{
Region: "global",
},
}
var getResp structs.KeyringGetRootKeyResponse
err = msgpackrpc.CallWithCodec(codec, "Keyring.Get", getReq, &getResp)
require.NoError(t, err)
gotKey := getResp.Key
require.Len(t, gotKey.Key, 32)
}
}

View File

@@ -8,7 +8,6 @@ import (
"github.com/hashicorp/go-hclog"
memdb "github.com/hashicorp/go-memdb"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/nomad/state"
"github.com/hashicorp/nomad/nomad/structs"
)
@@ -42,19 +41,24 @@ func (k *Keyring) Rotate(args *structs.KeyringRotateRootKeyRequest, reply *struc
args.Algorithm = structs.EncryptionAlgorithmXChaCha20
}
meta := structs.NewRootKeyMeta()
meta.Algorithm = args.Algorithm
meta.Active = true
rootKey, err := structs.NewRootKey(args.Algorithm)
if err != nil {
return err
}
// TODO: have the Encrypter generate and persist the actual key
// material. this is just here to silence the structcheck lint
for keyID := range k.encrypter.ciphers {
k.logger.Trace("TODO", "key", keyID)
rootKey.Meta.Active = true
// make sure it's been added to the local keystore before we write
// it to raft, so that followers don't try to Get a key that
// hasn't yet been written to disk
err = k.encrypter.AddKey(rootKey)
if err != nil {
return err
}
// Update metadata via Raft so followers can retrieve this key
req := structs.KeyringUpdateRootKeyMetaRequest{
RootKeyMeta: meta,
RootKeyMeta: rootKey.Meta,
WriteRequest: args.WriteRequest,
}
out, index, err := k.srv.raftApply(structs.RootKeyMetaUpsertRequestType, req)
@@ -64,7 +68,7 @@ func (k *Keyring) Rotate(args *structs.KeyringRotateRootKeyRequest, reply *struc
if err, ok := out.(error); ok && err != nil {
return err
}
reply.Key = meta
reply.Key = rootKey.Meta
reply.Index = index
return nil
}
@@ -138,13 +142,21 @@ func (k *Keyring) Update(args *structs.KeyringUpdateRootKeyRequest, reply *struc
return err
}
// make sure it's been added to the local keystore before we write
// it to raft, so that followers don't try to Get a key that
// hasn't yet been written to disk
err = k.encrypter.AddKey(args.RootKey)
if err != nil {
return err
}
// unwrap the request to turn it into a meta update only
metaReq := &structs.KeyringUpdateRootKeyMetaRequest{
RootKeyMeta: args.RootKey.Meta,
WriteRequest: args.WriteRequest,
}
// update via Raft
// update the metadata via Raft
out, index, err := k.srv.raftApply(structs.RootKeyMetaUpsertRequestType, metaReq)
if err != nil {
return err
@@ -152,6 +164,7 @@ func (k *Keyring) Update(args *structs.KeyringUpdateRootKeyRequest, reply *struc
if err, ok := out.(error); ok && err != nil {
return err
}
reply.Index = index
return nil
}
@@ -160,20 +173,13 @@ func (k *Keyring) Update(args *structs.KeyringUpdateRootKeyRequest, reply *struc
// existing key is valid
func (k *Keyring) validateUpdate(args *structs.KeyringUpdateRootKeyRequest) error {
if args.RootKey.Meta == nil {
return fmt.Errorf("root key metadata is required")
err := args.RootKey.Meta.Validate()
if err != nil {
return err
}
if args.RootKey.Meta.KeyID == "" || !helper.IsUUID(args.RootKey.Meta.KeyID) {
return fmt.Errorf("root key UUID is required")
if len(args.RootKey.Key) == 0 {
return fmt.Errorf("root key material is required")
}
if args.RootKey.Meta.Algorithm == "" {
return fmt.Errorf("algorithm is required")
}
// TODO: once the encrypter is implemented
// if len(args.RootKey.Key) == 0 {
// return fmt.Errorf("root key material is required")
// }
// lookup any existing key and validate the update
snap, err := k.srv.fsm.State().Snapshot()
@@ -230,12 +236,16 @@ func (k *Keyring) Get(args *structs.KeyringGetRootKeyRequest, reply *structs.Key
return k.srv.replySetIndex(state.TableRootKeyMeta, &reply.QueryMeta)
}
// TODO: retrieve the key material from the keyring
key := &structs.RootKey{
Meta: keyMeta,
Key: []byte{},
// retrieve the key material from the keyring
key, err := k.encrypter.GetKey(keyMeta.KeyID)
if err != nil {
return err
}
reply.Key = key
rootKey := &structs.RootKey{
Meta: keyMeta,
Key: key,
}
reply.Key = rootKey
reply.Index = keyMeta.ModifyIndex
return nil
},
@@ -285,6 +295,10 @@ func (k *Keyring) Delete(args *structs.KeyringDeleteRootKeyRequest, reply *struc
if err, ok := out.(error); ok && err != nil {
return err
}
// remove the key from the keyring too
k.encrypter.RemoveKey(args.KeyID)
reply.Index = index
return nil
}

View File

@@ -8,7 +8,6 @@ import (
"github.com/stretchr/testify/require"
"github.com/hashicorp/nomad/ci"
"github.com/hashicorp/nomad/helper/uuid"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/testutil"
)
@@ -23,23 +22,19 @@ func TestKeyringEndpoint_CRUD(t *testing.T) {
defer shutdown()
testutil.WaitForLeader(t, srv.RPC)
codec := rpcClient(t, srv)
id := uuid.Generate()
// Upsert a new key
key, err := structs.NewRootKey(structs.EncryptionAlgorithmXChaCha20)
require.NoError(t, err)
id := key.Meta.KeyID
key.Meta.Active = true
updateReq := &structs.KeyringUpdateRootKeyRequest{
RootKey: &structs.RootKey{
Meta: &structs.RootKeyMeta{
KeyID: id,
Algorithm: structs.EncryptionAlgorithmXChaCha20,
Active: true,
},
Key: []byte{},
},
RootKey: key,
WriteRequest: structs.WriteRequest{Region: "global"},
}
var updateResp structs.KeyringUpdateRootKeyResponse
var err error
err = msgpackrpc.CallWithCodec(codec, "Keyring.Update", updateReq, &updateResp)
require.EqualError(t, err, structs.ErrPermissionDenied.Error())
@@ -136,26 +131,22 @@ func TestKeyringEndpoint_InvalidUpdates(t *testing.T) {
defer shutdown()
testutil.WaitForLeader(t, srv.RPC)
codec := rpcClient(t, srv)
id := uuid.Generate()
// Setup an existing key
key, err := structs.NewRootKey(structs.EncryptionAlgorithmXChaCha20)
require.NoError(t, err)
id := key.Meta.KeyID
key.Meta.Active = true
updateReq := &structs.KeyringUpdateRootKeyRequest{
RootKey: &structs.RootKey{
Meta: &structs.RootKeyMeta{
KeyID: id,
Algorithm: structs.EncryptionAlgorithmXChaCha20,
Active: true,
},
Key: []byte{},
},
RootKey: key,
WriteRequest: structs.WriteRequest{
Region: "global",
AuthToken: rootToken.SecretID,
},
}
var updateResp structs.KeyringUpdateRootKeyResponse
err := msgpackrpc.CallWithCodec(codec, "Keyring.Update", updateReq, &updateResp)
err = msgpackrpc.CallWithCodec(codec, "Keyring.Update", updateReq, &updateResp)
require.NoError(t, err)
testCases := []struct {
@@ -177,8 +168,17 @@ func TestKeyringEndpoint_InvalidUpdates(t *testing.T) {
{
key: &structs.RootKey{Meta: &structs.RootKeyMeta{
KeyID: id,
Algorithm: structs.EncryptionAlgorithmAES256GCM,
Algorithm: structs.EncryptionAlgorithmXChaCha20,
}},
expectedErrMsg: "root key material is required",
},
{
key: &structs.RootKey{
Key: []byte{0x01},
Meta: &structs.RootKeyMeta{
KeyID: id,
Algorithm: structs.EncryptionAlgorithmAES256GCM,
}},
expectedErrMsg: "root key algorithm cannot be changed after a key is created",
},
}
@@ -211,26 +211,22 @@ func TestKeyringEndpoint_Rotate(t *testing.T) {
defer shutdown()
testutil.WaitForLeader(t, srv.RPC)
codec := rpcClient(t, srv)
id := uuid.Generate()
// Setup an existing key
key, err := structs.NewRootKey(structs.EncryptionAlgorithmXChaCha20)
require.NoError(t, err)
id := key.Meta.KeyID
key.Meta.Active = true
updateReq := &structs.KeyringUpdateRootKeyRequest{
RootKey: &structs.RootKey{
Meta: &structs.RootKeyMeta{
KeyID: id,
Algorithm: structs.EncryptionAlgorithmXChaCha20,
Active: true,
},
Key: []byte{},
},
RootKey: key,
WriteRequest: structs.WriteRequest{
Region: "global",
AuthToken: rootToken.SecretID,
},
}
var updateResp structs.KeyringUpdateRootKeyResponse
err := msgpackrpc.CallWithCodec(codec, "Keyring.Update", updateReq, &updateResp)
err = msgpackrpc.CallWithCodec(codec, "Keyring.Update", updateReq, &updateResp)
require.NoError(t, err)
// Rotate the key
@@ -262,14 +258,27 @@ func TestKeyringEndpoint_Rotate(t *testing.T) {
require.Greater(t, listResp.Index, updateResp.Index)
require.Len(t, listResp.Keys, 2)
var newID string
for _, keyMeta := range listResp.Keys {
if keyMeta.KeyID == id {
require.False(t, keyMeta.Active, "expected old key to be inactive")
} else {
require.True(t, keyMeta.Active, "expected new key to be inactive")
newID = keyMeta.KeyID
}
}
// TODO: verify that Encrypter has been updated
getReq := &structs.KeyringGetRootKeyRequest{
KeyID: newID,
QueryOptions: structs.QueryOptions{
Region: "global",
},
}
var getResp structs.KeyringGetRootKeyResponse
err = msgpackrpc.CallWithCodec(codec, "Keyring.Get", getReq, &getResp)
require.NoError(t, err)
gotKey := getResp.Key
require.Len(t, gotKey.Key, 32)
}

View File

@@ -1100,7 +1100,10 @@ func (s *Server) setupVaultClient() error {
// setupRPC is used to setup the RPC listener
func (s *Server) setupRPC(tlsWrap tlsutil.RegionWrapper) error {
// Populate the static RPC server
s.setupRpcServer(s.rpcServer, nil)
err := s.setupRpcServer(s.rpcServer, nil)
if err != nil {
return err
}
listener, err := s.createRPCListener()
if err != nil {
@@ -1159,11 +1162,15 @@ func (s *Server) setupRPC(tlsWrap tlsutil.RegionWrapper) error {
}
// setupRpcServer is used to populate an RPC server with endpoints
func (s *Server) setupRpcServer(server *rpc.Server, ctx *RPCContext) {
func (s *Server) setupRpcServer(server *rpc.Server, ctx *RPCContext) error {
// Set up the keyring
encrypter, err := NewEncrypter(filepath.Join(s.config.DataDir, "keystore"))
if err != nil {
return err
}
// Add the static endpoints to the RPC server.
encrypter := NewEncrypter()
if s.staticEndpoints.Status == nil {
// Initialize the list just once
s.staticEndpoints.ACL = &ACL{srv: s, logger: s.logger.Named("acl")}
@@ -1248,6 +1255,7 @@ func (s *Server) setupRpcServer(server *rpc.Server, ctx *RPCContext) {
server.Register(plan)
_ = server.Register(serviceReg)
_ = server.Register(keyringReg)
return nil
}
// setupRaft is used to setup and initialize Raft

View File

@@ -6681,6 +6681,11 @@ func (s *StateStore) UpsertRootKeyMeta(index uint64, rootKeyMeta *structs.RootKe
existing := raw.(*structs.RootKeyMeta)
rootKeyMeta.CreateIndex = existing.CreateIndex
rootKeyMeta.CreateTime = existing.CreateTime
// prevent resetting the encryptions count
if existing.EncryptionsCount > rootKeyMeta.EncryptionsCount {
rootKeyMeta.EncryptionsCount = existing.EncryptionsCount
}
isRotation = !existing.Active && rootKeyMeta.Active
} else {
rootKeyMeta.CreateIndex = index

View File

@@ -1,6 +1,7 @@
package structs
import (
"encoding/base64"
"reflect"
)
@@ -12,6 +13,8 @@ var (
reflect.TypeOf(&Node{}): nodeExt,
reflect.TypeOf(CSIVolume{}): csiVolumeExt,
reflect.TypeOf(&CSIVolume{}): csiVolumeExt,
reflect.TypeOf(&RootKey{}): rootKeyExt,
reflect.TypeOf(RootKey{}): rootKeyExt,
}
)
@@ -76,3 +79,23 @@ func csiVolumeExt(v interface{}) interface{} {
return apiVol
}
// rootKeyExt safely serializes a RootKey by base64 encoding the key
// material and extracting the metadata stub. We only store the root
// key in the keystore and never in raft or return it via the API, so
// by having this extension as the default we make it slightly harder
// to misuse.
func rootKeyExt(v interface{}) interface{} {
key := v.(*RootKey)
encodedKey := make([]byte, base64.StdEncoding.EncodedLen(len(key.Key)))
base64.StdEncoding.Encode(encodedKey, key.Key)
return &struct {
Meta *RootKeyMetaStub
Key string
}{
Meta: key.Meta.Stub(),
Key: string(encodedKey),
}
}

View File

@@ -1,8 +1,16 @@
package structs
import (
"fmt"
"time"
// note: this is aliased so that it's more noticeable if someone
// accidentally swaps it out for math/rand via running goimports
cryptorand "crypto/rand"
"golang.org/x/crypto/chacha20poly1305"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/helper/uuid"
)
@@ -143,6 +151,34 @@ type RootKey struct {
Key []byte // serialized to keystore as base64 blob
}
// NewRootKey returns a new root key and its metadata.
func NewRootKey(algorithm EncryptionAlgorithm) (*RootKey, error) {
meta := NewRootKeyMeta()
meta.Algorithm = algorithm
rootKey := &RootKey{
Meta: meta,
}
switch algorithm {
case EncryptionAlgorithmAES256GCM:
key := make([]byte, 32)
if _, err := cryptorand.Read(key); err != nil {
return nil, err
}
rootKey.Key = key
case EncryptionAlgorithmXChaCha20:
key := make([]byte, chacha20poly1305.KeySize)
if _, err := cryptorand.Read(key); err != nil {
return nil, err
}
rootKey.Key = key
}
return rootKey, nil
}
// RootKeyMeta is the metadata used to refer to a RootKey. It is
// stored in raft.
type RootKeyMeta struct {
@@ -164,6 +200,30 @@ func NewRootKeyMeta() *RootKeyMeta {
}
}
// RootKeyMetaStub is for serializing root key metadata to the
// keystore, not for the List API. It excludes frequently-changing
// fields such as EncryptionsCount or ModifyIndex so we don't have to
// sync them to the on-disk keystore when the fields are already in
// raft.
type RootKeyMetaStub struct {
KeyID string
Algorithm EncryptionAlgorithm
CreateTime time.Time
Active bool
}
func (rkm *RootKeyMeta) Stub() *RootKeyMetaStub {
if rkm == nil {
return nil
}
return &RootKeyMetaStub{
KeyID: rkm.KeyID,
Algorithm: rkm.Algorithm,
CreateTime: rkm.CreateTime,
Active: rkm.Active,
}
}
func (rkm *RootKeyMeta) Copy() *RootKeyMeta {
if rkm == nil {
return nil
@@ -172,6 +232,19 @@ func (rkm *RootKeyMeta) Copy() *RootKeyMeta {
return &out
}
func (rkm *RootKeyMeta) Validate() error {
if rkm == nil {
return fmt.Errorf("root key metadata is required")
}
if rkm.KeyID == "" || !helper.IsUUID(rkm.KeyID) {
return fmt.Errorf("root key UUID is required")
}
if rkm.Algorithm == "" {
return fmt.Errorf("root key algorithm is required")
}
return nil
}
// EncryptionAlgorithm chooses which algorithm is used for
// encrypting / decrypting entries with this key
type EncryptionAlgorithm string

View File

@@ -53,6 +53,7 @@ func TestServerErr(t *testing.T, cb func(*Config)) (*Server, func(), error) {
config.Build = version.Version + "+unittest"
config.DevMode = true
config.DataDir = t.TempDir()
config.EnableEventBroker = true
config.BootstrapExpect = 1
nodeNum := atomic.AddInt32(&nodeNumber, 1)