introduce new package pkg/kms (#12019)

This commit introduces a new package `pkg/kms`.
It contains basic types and functions to interact
with various KMS implementations.

This commit also moves KMS-related code from `cmd/crypto`
to `pkg/kms`. Now, it is possible to implement a KMS-based
config data encryption in the `pkg/config` package.
This commit is contained in:
Andreas Auernhammer 2021-04-15 17:47:33 +02:00 committed by GitHub
parent 1456f9f090
commit 885c170a64
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 1176 additions and 274 deletions

View File

@ -46,6 +46,7 @@ import (
"github.com/minio/minio/pkg/dsync"
"github.com/minio/minio/pkg/handlers"
iampolicy "github.com/minio/minio/pkg/iam/policy"
"github.com/minio/minio/pkg/kms"
"github.com/minio/minio/pkg/madmin"
xnet "github.com/minio/minio/pkg/net"
trace "github.com/minio/minio/pkg/trace"
@ -1260,18 +1261,23 @@ func (a adminAPIHandlers) KMSKeyStatusHandler(w http.ResponseWriter, r *http.Req
writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErr(ErrKMSNotConfigured), r.URL)
return
}
stat, err := GlobalKMS.Stat()
if err != nil {
writeCustomErrorResponseJSON(ctx, w, errorCodes.ToAPIErr(ErrInternalError), err.Error(), r.URL)
return
}
keyID := r.URL.Query().Get("key-id")
if keyID == "" {
keyID = GlobalKMS.DefaultKeyID()
keyID = stat.DefaultKey
}
var response = madmin.KMSKeyStatus{
KeyID: keyID,
}
kmsContext := crypto.Context{"MinIO admin API": "KMSKeyStatusHandler"} // Context for a test key operation
kmsContext := kms.Context{"MinIO admin API": "KMSKeyStatusHandler"} // Context for a test key operation
// 1. Generate a new key using the KMS.
key, sealedKey, err := GlobalKMS.GenerateKey(keyID, kmsContext)
key, err := GlobalKMS.GenerateKey(keyID, kmsContext)
if err != nil {
response.EncryptionErr = err.Error()
resp, err := json.Marshal(response)
@ -1284,7 +1290,7 @@ func (a adminAPIHandlers) KMSKeyStatusHandler(w http.ResponseWriter, r *http.Req
}
// 2. Verify that we can indeed decrypt the (encrypted) key
decryptedKey, err := GlobalKMS.UnsealKey(keyID, sealedKey, kmsContext)
decryptedKey, err := GlobalKMS.DecryptKey(key.KeyID, key.Plaintext, kmsContext)
if err != nil {
response.DecryptionErr = err.Error()
resp, err := json.Marshal(response)
@ -1297,7 +1303,7 @@ func (a adminAPIHandlers) KMSKeyStatusHandler(w http.ResponseWriter, r *http.Req
}
// 3. Compare generated key with decrypted key
if subtle.ConstantTimeCompare(key[:], decryptedKey[:]) != 1 {
if subtle.ConstantTimeCompare(key.Plaintext, decryptedKey) != 1 {
response.DecryptionErr = "The generated and the decrypted data key do not match"
resp, err := json.Marshal(response)
if err != nil {
@ -1742,36 +1748,39 @@ func fetchKMSStatus() madmin.KMS {
kmsStat.Status = "disabled"
return kmsStat
}
keyID := GlobalKMS.DefaultKeyID()
kmsInfo := GlobalKMS.Info()
if len(kmsInfo.Endpoints) == 0 {
kmsStat.Status = "KMS configured using master key"
stat, err := GlobalKMS.Stat()
if err != nil {
kmsStat.Status = string(madmin.ItemOffline)
return kmsStat
}
if err := checkConnection(kmsInfo.Endpoints[0], 15*time.Second); err != nil {
kmsStat.Status = string(madmin.ItemOffline)
if len(stat.Endpoints) == 0 {
kmsStat.Status = stat.Name
} else {
kmsStat.Status = string(madmin.ItemOnline)
kmsContext := crypto.Context{"MinIO admin API": "ServerInfoHandler"} // Context for a test key operation
// 1. Generate a new key using the KMS.
key, sealedKey, err := GlobalKMS.GenerateKey(keyID, kmsContext)
if err != nil {
kmsStat.Encrypt = fmt.Sprintf("Encryption failed: %v", err)
if err := checkConnection(stat.Endpoints[0], 15*time.Second); err != nil {
kmsStat.Status = string(madmin.ItemOffline)
} else {
kmsStat.Encrypt = "success"
}
kmsStat.Status = string(madmin.ItemOnline)
// 2. Verify that we can indeed decrypt the (encrypted) key
decryptedKey, err := GlobalKMS.UnsealKey(keyID, sealedKey, kmsContext)
switch {
case err != nil:
kmsStat.Decrypt = fmt.Sprintf("Decryption failed: %v", err)
case subtle.ConstantTimeCompare(key[:], decryptedKey[:]) != 1:
kmsStat.Decrypt = "Decryption failed: decrypted key does not match generated key"
default:
kmsStat.Decrypt = "success"
kmsContext := kms.Context{"MinIO admin API": "ServerInfoHandler"} // Context for a test key operation
// 1. Generate a new key using the KMS.
key, err := GlobalKMS.GenerateKey("", kmsContext)
if err != nil {
kmsStat.Encrypt = fmt.Sprintf("Encryption failed: %v", err)
} else {
kmsStat.Encrypt = "success"
}
// 2. Verify that we can indeed decrypt the (encrypted) key
decryptedKey, err := GlobalKMS.DecryptKey(key.KeyID, key.Ciphertext, kmsContext)
switch {
case err != nil:
kmsStat.Decrypt = fmt.Sprintf("Decryption failed: %v", err)
case subtle.ConstantTimeCompare(key.Plaintext, decryptedKey) != 1:
kmsStat.Decrypt = "Decryption failed: decrypted key does not match generated key"
default:
kmsStat.Decrypt = "success"
}
}
}
return kmsStat

View File

@ -39,6 +39,7 @@ import (
"github.com/minio/minio/pkg/bucket/versioning"
"github.com/minio/minio/pkg/event"
"github.com/minio/minio/pkg/fips"
"github.com/minio/minio/pkg/kms"
"github.com/minio/minio/pkg/madmin"
"github.com/minio/sio"
)
@ -391,7 +392,7 @@ func (b *BucketMetadata) migrateTargetConfig(ctx context.Context, objectAPI Obje
return nil
}
encBytes, metaBytes, err := encryptBucketMetadata(b.Name, b.BucketTargetsConfigJSON, crypto.Context{b.Name: b.Name, bucketTargetsFile: bucketTargetsFile})
encBytes, metaBytes, err := encryptBucketMetadata(b.Name, b.BucketTargetsConfigJSON, kms.Context{b.Name: b.Name, bucketTargetsFile: bucketTargetsFile})
if err != nil {
return err
}
@ -402,26 +403,22 @@ func (b *BucketMetadata) migrateTargetConfig(ctx context.Context, objectAPI Obje
}
// encrypt bucket metadata if kms is configured.
func encryptBucketMetadata(bucket string, input []byte, kmsContext crypto.Context) (output, metabytes []byte, err error) {
var sealedKey crypto.SealedKey
func encryptBucketMetadata(bucket string, input []byte, kmsContext kms.Context) (output, metabytes []byte, err error) {
if GlobalKMS == nil {
output = input
return
}
var (
key [32]byte
encKey []byte
)
metadata := make(map[string]string)
key, encKey, err = GlobalKMS.GenerateKey(GlobalKMS.DefaultKeyID(), kmsContext)
key, err := GlobalKMS.GenerateKey("", kmsContext)
if err != nil {
return
}
outbuf := bytes.NewBuffer(nil)
objectKey := crypto.GenerateKey(key, rand.Reader)
sealedKey = objectKey.Seal(key, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, "")
crypto.S3.CreateMetadata(metadata, GlobalKMS.DefaultKeyID(), encKey, sealedKey)
objectKey := crypto.GenerateKey(key.Plaintext, rand.Reader)
sealedKey := objectKey.Seal(key.Plaintext, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, "")
crypto.S3.CreateMetadata(metadata, key.KeyID, key.Ciphertext, sealedKey)
_, err = sio.Encrypt(outbuf, bytes.NewBuffer(input), sio.Config{Key: objectKey[:], MinVersion: sio.Version20, CipherSuites: fips.CipherSuitesDARE()})
if err != nil {
return output, metabytes, err
@ -434,7 +431,7 @@ func encryptBucketMetadata(bucket string, input []byte, kmsContext crypto.Contex
}
// decrypt bucket metadata if kms is configured.
func decryptBucketMetadata(input []byte, bucket string, meta map[string]string, kmsContext crypto.Context) ([]byte, error) {
func decryptBucketMetadata(input []byte, bucket string, meta map[string]string, kmsContext kms.Context) ([]byte, error) {
if GlobalKMS == nil {
return nil, errKMSNotConfigured
}
@ -442,7 +439,7 @@ func decryptBucketMetadata(input []byte, bucket string, meta map[string]string,
if err != nil {
return nil, err
}
extKey, err := GlobalKMS.UnsealKey(keyID, kmsKey, kmsContext)
extKey, err := GlobalKMS.DecryptKey(keyID, kmsKey, kmsContext)
if err != nil {
return nil, err
}

138
cmd/config/crypto.go Normal file
View File

@ -0,0 +1,138 @@
// MinIO Cloud Storage, (C) 2021 MinIO, Inc.
//
// 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 config
import (
"bytes"
"crypto/rand"
"encoding/binary"
"encoding/json"
"errors"
"fmt"
"io"
"github.com/minio/minio/pkg/kms"
"github.com/secure-io/sio-go"
"github.com/secure-io/sio-go/sioutil"
)
// Encrypt encrypts the plaintext with a key managed by KMS.
// The context is bound to the returned ciphertext.
//
// The same context must be provided when decrypting the
// ciphertext.
func Encrypt(KMS kms.KMS, plaintext io.Reader, context kms.Context) (io.Reader, error) {
var algorithm = sio.AES_256_GCM
if !sioutil.NativeAES() {
algorithm = sio.ChaCha20Poly1305
}
key, err := KMS.GenerateKey("", context)
if err != nil {
return nil, err
}
stream, err := algorithm.Stream(key.Plaintext)
if err != nil {
return nil, err
}
nonce := make([]byte, stream.NonceSize())
if _, err := rand.Read(nonce); err != nil {
return nil, err
}
const (
MaxMetadataSize = 1 << 20 // max. size of the metadata
Version = 1
)
var (
header [5]byte
buffer bytes.Buffer
)
metadata, err := json.Marshal(encryptedObject{
KeyID: key.KeyID,
KMSKey: key.Ciphertext,
Algorithm: algorithm,
Nonce: nonce,
})
if err != nil {
return nil, err
}
if len(metadata) > MaxMetadataSize {
return nil, errors.New("config: encryption metadata is too large")
}
header[0] = Version
binary.LittleEndian.PutUint32(header[1:], uint32(len(metadata)))
buffer.Write(header[:])
buffer.Write(metadata)
return io.MultiReader(
&buffer,
stream.EncryptReader(plaintext, nonce, nil),
), nil
}
// Decrypt decrypts the ciphertext using a key managed by the KMS.
// The same context that have been used during encryption must be
// provided.
func Decrypt(KMS kms.KMS, ciphertext io.Reader, context kms.Context) (io.Reader, error) {
const (
MaxMetadataSize = 1 << 20 // max. size of the metadata
Version = 1
)
var header [5]byte
if _, err := io.ReadFull(ciphertext, header[:]); err != nil {
return nil, err
}
if header[0] != Version {
return nil, fmt.Errorf("config: unknown ciphertext version %d", header[0])
}
size := binary.LittleEndian.Uint32(header[1:])
if size > MaxMetadataSize {
return nil, errors.New("config: encryption metadata is too large")
}
var (
metadataBuffer = make([]byte, size)
metadata encryptedObject
)
if _, err := io.ReadFull(ciphertext, metadataBuffer); err != nil {
return nil, err
}
if err := json.Unmarshal(metadataBuffer, &metadata); err != nil {
return nil, err
}
key, err := KMS.DecryptKey(metadata.KeyID, metadata.KMSKey, context)
if err != nil {
return nil, err
}
stream, err := metadata.Algorithm.Stream(key)
if err != nil {
return nil, err
}
if stream.NonceSize() != len(metadata.Nonce) {
return nil, sio.NotAuthentic
}
return stream.DecryptReader(ciphertext, metadata.Nonce, nil), nil
}
type encryptedObject struct {
KeyID string `json:"keyid"`
KMSKey []byte `json:"kmskey"`
Algorithm sio.Algorithm `json:"algorithm"`
Nonce []byte `json:"nonce"`
}

116
cmd/config/crypto_test.go Normal file
View File

@ -0,0 +1,116 @@
// MinIO Cloud Storage, (C) 2021 MinIO, Inc.
//
// 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 config
import (
"bytes"
"encoding/hex"
"io"
"io/ioutil"
"testing"
"github.com/minio/minio/pkg/kms"
)
var encryptDecryptTests = []struct {
Data []byte
Context kms.Context
}{
{
Data: nil,
Context: nil,
},
{
Data: []byte{1},
Context: nil,
},
{
Data: []byte{1},
Context: kms.Context{"key": "value"},
},
{
Data: make([]byte, 1<<20),
Context: kms.Context{"key": "value", "a": "b"},
},
}
func TestEncryptDecrypt(t *testing.T) {
key, err := hex.DecodeString("ddedadb867afa3f73bd33c25499a723ed7f9f51172ee7b1b679e08dc795debcc")
if err != nil {
t.Fatalf("Failed to decode master key: %v", err)
}
KMS, err := kms.New("my-key", key)
if err != nil {
t.Fatalf("Failed to create KMS: %v", err)
}
for i, test := range encryptDecryptTests {
ciphertext, err := Encrypt(KMS, bytes.NewReader(test.Data), test.Context)
if err != nil {
t.Fatalf("Test %d: failed to encrypt stream: %v", i, err)
}
data, err := ioutil.ReadAll(ciphertext)
if err != nil {
t.Fatalf("Test %d: failed to encrypt stream: %v", i, err)
}
plaintext, err := Decrypt(KMS, bytes.NewReader(data), test.Context)
if err != nil {
t.Fatalf("Test %d: failed to decrypt stream: %v", i, err)
}
data, err = ioutil.ReadAll(plaintext)
if err != nil {
t.Fatalf("Test %d: failed to decrypt stream: %v", i, err)
}
if !bytes.Equal(data, test.Data) {
t.Fatalf("Test %d: decrypted data does not match original data", i)
}
}
}
func BenchmarkEncrypt(b *testing.B) {
key, err := hex.DecodeString("ddedadb867afa3f73bd33c25499a723ed7f9f51172ee7b1b679e08dc795debcc")
if err != nil {
b.Fatalf("Failed to decode master key: %v", err)
}
KMS, err := kms.New("my-key", key)
if err != nil {
b.Fatalf("Failed to create KMS: %v", err)
}
benchmarkEncrypt := func(size int, b *testing.B) {
var (
data = make([]byte, size)
plaintext = bytes.NewReader(data)
context = kms.Context{"key": "value"}
)
b.SetBytes(int64(size))
for i := 0; i < b.N; i++ {
ciphertext, err := Encrypt(KMS, plaintext, context)
if err != nil {
b.Fatal(err)
}
if _, err = io.Copy(ioutil.Discard, ciphertext); err != nil {
b.Fatal(err)
}
plaintext.Reset(data)
}
}
b.Run("1KB", func(b *testing.B) { benchmarkEncrypt(1*1024, b) })
b.Run("512KB", func(b *testing.B) { benchmarkEncrypt(512*1024, b) })
b.Run("1MB", func(b *testing.B) { benchmarkEncrypt(1024*1024, b) })
b.Run("10MB", func(b *testing.B) { benchmarkEncrypt(10*1024*1024, b) })
}

View File

@ -32,6 +32,7 @@ import (
jsoniter "github.com/json-iterator/go"
xhttp "github.com/minio/minio/cmd/http"
"github.com/minio/minio/pkg/kms"
xnet "github.com/minio/minio/pkg/net"
)
@ -155,22 +156,12 @@ func NewKes(cfg KesConfig) (KMS, error) {
}, nil
}
// DefaultKeyID returns the default key ID that should be
// used for SSE-S3 or SSE-KMS when the S3 client does not
// provide an explicit key ID.
func (kes *kesService) DefaultKeyID() string {
return kes.defaultKeyID
}
// Info returns some information about the KES,
// configuration - like the endpoint or authentication
// method.
func (kes *kesService) Info() KMSInfo {
return KMSInfo{
Endpoints: kes.endpoints,
Name: kes.DefaultKeyID(),
AuthType: "TLS",
}
func (kes *kesService) Stat() (kms.Status, error) {
return kms.Status{
Name: "KES",
Endpoints: kes.endpoints,
DefaultKey: kes.defaultKeyID,
}, nil
}
// CreateKey tries to create a new master key with the given keyID.
@ -180,22 +171,24 @@ func (kes *kesService) CreateKey(keyID string) error { return kes.client.CreateK
// and a sealed version of this plaintext key encrypted using the
// named key referenced by keyID. It also binds the generated key
// cryptographically to the provided context.
func (kes *kesService) GenerateKey(keyID string, ctx Context) (key [32]byte, sealedKey []byte, err error) {
func (kes *kesService) GenerateKey(keyID string, ctx Context) (kms.DEK, error) {
if keyID == "" {
keyID = kes.defaultKeyID
}
context, err := ctx.MarshalText()
if err != nil {
return key, nil, err
return kms.DEK{}, err
}
var plainKey []byte
plainKey, sealedKey, err = kes.client.GenerateDataKey(keyID, context)
plaintext, ciphertext, err := kes.client.GenerateDataKey(keyID, context)
if err != nil {
return key, nil, err
return kms.DEK{}, err
}
if len(plainKey) != len(key) {
return key, nil, Errorf("crypto: received invalid plaintext key size from KMS")
}
copy(key[:], plainKey)
return key, sealedKey, nil
return kms.DEK{
KeyID: keyID,
Plaintext: plaintext,
Ciphertext: ciphertext,
}, nil
}
// UnsealKey returns the decrypted sealedKey as plaintext key.
@ -205,22 +198,12 @@ func (kes *kesService) GenerateKey(keyID string, ctx Context) (key [32]byte, sea
//
// The context must be same context as the one provided while
// generating the plaintext key / sealedKey.
func (kes *kesService) UnsealKey(keyID string, sealedKey []byte, ctx Context) (key [32]byte, err error) {
func (kes *kesService) DecryptKey(keyID string, ciphertext []byte, ctx Context) ([]byte, error) {
context, err := ctx.MarshalText()
if err != nil {
return key, err
return nil, err
}
var plainKey []byte
plainKey, err = kes.client.DecryptDataKey(keyID, sealedKey, context)
if err != nil {
return key, err
}
if len(plainKey) != len(key) {
return key, Errorf("crypto: received invalid plaintext key size from KMS")
}
copy(key[:], plainKey)
return key, nil
return kes.client.DecryptDataKey(keyID, ciphertext, context)
}
// kesClient implements the bare minimum functionality needed for

View File

@ -37,7 +37,7 @@ type ObjectKey [32]byte
// GenerateKey generates a unique ObjectKey from a 256 bit external key
// and a source of randomness. If random is nil the default PRNG of the
// system (crypto/rand) is used.
func GenerateKey(extKey [32]byte, random io.Reader) (key ObjectKey) {
func GenerateKey(extKey []byte, random io.Reader) (key ObjectKey) {
if random == nil {
random = rand.Reader
}
@ -46,7 +46,7 @@ func GenerateKey(extKey [32]byte, random io.Reader) (key ObjectKey) {
logger.CriticalIf(context.Background(), errOutOfEntropy)
}
sha := sha256.New()
sha.Write(extKey[:])
sha.Write(extKey)
sha.Write(nonce[:])
sha.Sum(key[:0])
return key
@ -76,7 +76,7 @@ type SealedKey struct {
// Seal encrypts the ObjectKey using the 256 bit external key and IV. The sealed
// key is also cryptographically bound to the object's path (bucket/object) and the
// domain (SSE-C or SSE-S3).
func (key ObjectKey) Seal(extKey, iv [32]byte, domain, bucket, object string) SealedKey {
func (key ObjectKey) Seal(extKey []byte, iv [32]byte, domain, bucket, object string) SealedKey {
var (
sealingKey [32]byte
encryptedKey bytes.Buffer
@ -101,7 +101,7 @@ func (key ObjectKey) Seal(extKey, iv [32]byte, domain, bucket, object string) Se
// Unseal decrypts a sealed key using the 256 bit external key. Since the sealed key
// may be cryptographically bound to the object's path the same bucket/object as during sealing
// must be provided. On success the ObjectKey contains the decrypted sealed key.
func (key *ObjectKey) Unseal(extKey [32]byte, sealedKey SealedKey, domain, bucket, object string) error {
func (key *ObjectKey) Unseal(extKey []byte, sealedKey SealedKey, domain, bucket, object string) error {
var (
unsealConfig sio.Config
)

View File

@ -53,7 +53,7 @@ func TestGenerateKey(t *testing.T) {
i, test := i, test
func() {
defer recoverTest(i, test.ShouldPass, t)
key := GenerateKey(test.ExtKey, test.Random)
key := GenerateKey(test.ExtKey[:], test.Random)
if [32]byte(key) == [32]byte{} {
t.Errorf("Test %d: generated key is zero key", i) // check that we generate random and unique key
}
@ -125,9 +125,9 @@ var sealUnsealKeyTests = []struct {
func TestSealUnsealKey(t *testing.T) {
for i, test := range sealUnsealKeyTests {
key := GenerateKey(test.SealExtKey, rand.Reader)
sealedKey := key.Seal(test.SealExtKey, test.SealIV, test.SealDomain, test.SealBucket, test.SealObject)
if err := key.Unseal(test.UnsealExtKey, sealedKey, test.UnsealDomain, test.UnsealBucket, test.UnsealObject); err == nil && !test.ShouldPass {
key := GenerateKey(test.SealExtKey[:], rand.Reader)
sealedKey := key.Seal(test.SealExtKey[:], test.SealIV, test.SealDomain, test.SealBucket, test.SealObject)
if err := key.Unseal(test.UnsealExtKey[:], sealedKey, test.UnsealDomain, test.UnsealBucket, test.UnsealObject); err == nil && !test.ShouldPass {
t.Errorf("Test %d should fail but passed successfully", i)
} else if err != nil && test.ShouldPass {
t.Errorf("Test %d should pass put failed: %v", i, err)
@ -136,10 +136,10 @@ func TestSealUnsealKey(t *testing.T) {
// Test legacy InsecureSealAlgorithm
var extKey, iv [32]byte
key := GenerateKey(extKey, rand.Reader)
sealedKey := key.Seal(extKey, iv, "SSE-S3", "bucket", "object")
key := GenerateKey(extKey[:], rand.Reader)
sealedKey := key.Seal(extKey[:], iv, "SSE-S3", "bucket", "object")
sealedKey.Algorithm = InsecureSealAlgorithm
if err := key.Unseal(extKey, sealedKey, "SSE-S3", "bucket", "object"); err == nil {
if err := key.Unseal(extKey[:], sealedKey, "SSE-S3", "bucket", "object"); err == nil {
t.Errorf("'%s' test succeeded but it should fail because the legacy algorithm was used", sealedKey.Algorithm)
}
}

View File

@ -22,166 +22,86 @@ import (
"crypto/sha256"
"errors"
"io"
"sort"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/kms"
"github.com/minio/sio"
)
// Context is a list of key-value pairs cryptographically
// associated with a certain object.
type Context map[string]string
// MarshalText returns a canonical text representation of
// the Context.
// MarshalText sorts the context keys and writes the sorted
// key-value pairs as canonical JSON object. The sort order
// is based on the un-escaped keys.
func (c Context) MarshalText() ([]byte, error) {
if len(c) == 0 {
return []byte{'{', '}'}, nil
}
// Pre-allocate a buffer - 128 bytes is an arbitrary
// heuristic value that seems like a good starting size.
var b = bytes.NewBuffer(make([]byte, 0, 128))
if len(c) == 1 {
for k, v := range c {
b.WriteString(`{"`)
EscapeStringJSON(b, k)
b.WriteString(`":"`)
EscapeStringJSON(b, v)
b.WriteString(`"}`)
}
return b.Bytes(), nil
}
sortedKeys := make([]string, 0, len(c))
for k := range c {
sortedKeys = append(sortedKeys, k)
}
sort.Strings(sortedKeys)
b.WriteByte('{')
for i, k := range sortedKeys {
b.WriteByte('"')
EscapeStringJSON(b, k)
b.WriteString(`":"`)
EscapeStringJSON(b, c[k])
b.WriteByte('"')
if i < len(sortedKeys)-1 {
b.WriteByte(',')
}
}
b.WriteByte('}')
return b.Bytes(), nil
}
type Context = kms.Context
// KMS represents an active and authenticted connection
// to a Key-Management-Service. It supports generating
// data key generation and unsealing of KMS-generated
// data keys.
type KMS interface {
// DefaultKeyID returns the default master key ID. It should be
// used for SSE-S3 and whenever a S3 client requests SSE-KMS but
// does not specify an explicit SSE-KMS key ID.
DefaultKeyID() string
// CreateKey creates a new master key with the given key ID
// at the KMS.
CreateKey(keyID string) error
// GenerateKey generates a new random data key using
// the master key referenced by the keyID. It returns
// the plaintext key and the sealed plaintext key
// on success.
//
// The context is cryptographically bound to the
// generated key. The same context must be provided
// again to unseal the generated key.
GenerateKey(keyID string, context Context) (key [32]byte, sealedKey []byte, err error)
// UnsealKey unseals the sealedKey using the master key
// referenced by the keyID. The provided context must
// match the context used to generate the sealed key.
UnsealKey(keyID string, sealedKey []byte, context Context) (key [32]byte, err error)
// Info returns descriptive information about the KMS,
// like the default key ID and authentication method.
Info() KMSInfo
}
type KMS = kms.KMS
type masterKeyKMS struct {
keyID string
masterKey [32]byte
}
// KMSInfo contains some describing information about
// the KMS.
type KMSInfo struct {
Endpoints []string
Name string
AuthType string
}
// NewMasterKey returns a basic KMS implementation from a single 256 bit master key.
//
// The KMS accepts any keyID but binds the keyID and context cryptographically
// to the generated keys.
func NewMasterKey(keyID string, key [32]byte) KMS { return &masterKeyKMS{keyID: keyID, masterKey: key} }
func (kms *masterKeyKMS) DefaultKeyID() string {
return kms.keyID
func (m *masterKeyKMS) Stat() (kms.Status, error) {
return kms.Status{
Name: "MasterKey",
DefaultKey: m.keyID,
}, nil
}
func (kms *masterKeyKMS) CreateKey(keyID string) error {
func (m *masterKeyKMS) CreateKey(keyID string) error {
return errors.New("crypto: creating keys is not supported by a static master key")
}
func (kms *masterKeyKMS) GenerateKey(keyID string, ctx Context) (key [32]byte, sealedKey []byte, err error) {
if _, err = io.ReadFull(rand.Reader, key[:]); err != nil {
func (m *masterKeyKMS) GenerateKey(keyID string, ctx Context) (kms.DEK, error) {
if keyID == "" {
keyID = m.keyID
}
var key [32]byte
if _, err := io.ReadFull(rand.Reader, key[:]); err != nil {
logger.CriticalIf(context.Background(), errOutOfEntropy)
}
var (
buffer bytes.Buffer
derivedKey = kms.deriveKey(keyID, ctx)
derivedKey = m.deriveKey(keyID, ctx)
)
if n, err := sio.Encrypt(&buffer, bytes.NewReader(key[:]), sio.Config{Key: derivedKey[:]}); err != nil || n != 64 {
logger.CriticalIf(context.Background(), errors.New("KMS: unable to encrypt data key"))
}
sealedKey = buffer.Bytes()
return key, sealedKey, nil
return kms.DEK{
KeyID: m.keyID,
Plaintext: key[:],
Ciphertext: buffer.Bytes(),
}, nil
}
// KMS is configured directly using master key
func (kms *masterKeyKMS) Info() (info KMSInfo) {
return KMSInfo{
Endpoints: []string{},
Name: "",
AuthType: "master-key",
}
}
func (m *masterKeyKMS) DecryptKey(keyID string, sealedKey []byte, ctx Context) ([]byte, error) {
var derivedKey = m.deriveKey(keyID, ctx)
func (kms *masterKeyKMS) UnsealKey(keyID string, sealedKey []byte, ctx Context) (key [32]byte, err error) {
var (
derivedKey = kms.deriveKey(keyID, ctx)
)
var key [32]byte
out, err := sio.DecryptBuffer(key[:0], sealedKey, sio.Config{Key: derivedKey[:]})
if err != nil || len(out) != 32 {
return key, err // TODO(aead): upgrade sio to use sio.Error
return nil, err // TODO(aead): upgrade sio to use sio.Error
}
return key, nil
return key[:], nil
}
func (kms *masterKeyKMS) deriveKey(keyID string, context Context) (key [32]byte) {
func (m *masterKeyKMS) deriveKey(keyID string, context Context) (key [32]byte) {
if context == nil {
context = Context{}
}
ctxBytes, _ := context.MarshalText()
mac := hmac.New(sha256.New, kms.masterKey[:])
mac := hmac.New(sha256.New, m.masterKey[:])
mac.Write([]byte(keyID))
mac.Write(ctxBytes)
mac.Sum(key[:0])

View File

@ -43,18 +43,18 @@ func TestMasterKeyKMS(t *testing.T) {
for i, test := range masterKeyKMSTests {
kms := NewMasterKey(test.GenKeyID, [32]byte{})
key, sealedKey, err := kms.GenerateKey(test.GenKeyID, test.GenContext)
key, err := kms.GenerateKey(test.GenKeyID, test.GenContext)
if err != nil {
t.Errorf("Test %d: KMS failed to generate key: %v", i, err)
}
unsealedKey, err := kms.UnsealKey(test.UnsealKeyID, sealedKey, test.UnsealContext)
unsealedKey, err := kms.DecryptKey(test.UnsealKeyID, key.Ciphertext, test.UnsealContext)
if err != nil && !test.ShouldFail {
t.Errorf("Test %d: KMS failed to unseal the generated key: %v", i, err)
}
if err == nil && test.ShouldFail {
t.Errorf("Test %d: KMS unsealed the generated key successfully but should have failed", i)
}
if !test.ShouldFail && !bytes.Equal(key[:], unsealedKey[:]) {
if !test.ShouldFail && !bytes.Equal(key.Plaintext, unsealedKey[:]) {
t.Errorf("Test %d: The generated and unsealed key differ", i)
}
}

View File

@ -51,8 +51,11 @@ func TestParseMasterKey(t *testing.T) {
if !tt.success && err == nil {
t.Error("Unexpected failure")
}
if err == nil && kms.DefaultKeyID() != tt.expectedKeyID {
t.Errorf("Expected keyID %s, got %s", tt.expectedKeyID, kms.DefaultKeyID())
if kms != nil {
stat, _ := kms.Stat()
if err == nil && stat.DefaultKey != tt.expectedKeyID {
t.Errorf("Expected keyID %s, got %s", tt.expectedKeyID, stat.DefaultKey)
}
}
})
}

View File

@ -99,7 +99,7 @@ func (s3 ssec) UnsealObjectKey(h http.Header, metadata map[string]string, bucket
if err != nil {
return
}
return unsealObjectKey(clientKey, metadata, bucket, object)
return unsealObjectKey(clientKey[:], metadata, bucket, object)
}
// CreateMetadata encodes the sealed key into the metadata

View File

@ -96,11 +96,11 @@ func (s3 ssekms) UnsealObjectKey(kms KMS, metadata map[string]string, bucket, ob
if _, ok := ctx[bucket]; !ok {
ctx[bucket] = path.Join(bucket, object)
}
unsealKey, err := kms.UnsealKey(keyID, kmsKey, ctx)
unsealKey, err := kms.DecryptKey(keyID, kmsKey, ctx)
if err != nil {
return key, err
}
err = key.Unseal(unsealKey, sealedKey, s3.String(), bucket, object)
err = key.Unseal(unsealKey[:], sealedKey, s3.String(), bucket, object)
return key, err
}

View File

@ -73,11 +73,11 @@ func (s3 sses3) UnsealObjectKey(kms KMS, metadata map[string]string, bucket, obj
if err != nil {
return key, err
}
unsealKey, err := kms.UnsealKey(keyID, kmsKey, Context{bucket: path.Join(bucket, object)})
unsealKey, err := kms.DecryptKey(keyID, kmsKey, Context{bucket: path.Join(bucket, object)})
if err != nil {
return key, err
}
err = key.Unseal(unsealKey, sealedKey, s3.String(), bucket, object)
err = key.Unseal(unsealKey[:], sealedKey, s3.String(), bucket, object)
return key, err
}

View File

@ -77,12 +77,12 @@ func (sse ssecCopy) UnsealObjectKey(h http.Header, metadata map[string]string, b
if err != nil {
return
}
return unsealObjectKey(clientKey, metadata, bucket, object)
return unsealObjectKey(clientKey[:], metadata, bucket, object)
}
// unsealObjectKey decrypts and returns the sealed object key
// from the metadata using the SSE-C client key.
func unsealObjectKey(clientKey [32]byte, metadata map[string]string, bucket, object string) (key ObjectKey, err error) {
func unsealObjectKey(clientKey []byte, metadata map[string]string, bucket, object string) (key ObjectKey, err error) {
sealedKey, err := SSEC.ParseMetadata(metadata)
if err != nil {
return

View File

@ -22,6 +22,7 @@ import (
"time"
vault "github.com/hashicorp/vault/api"
"github.com/minio/minio/pkg/kms"
)
var (
@ -190,22 +191,15 @@ func (v *vaultService) authenticate() (err error) {
return
}
// DefaultKeyID returns the default key ID that should be
// used for SSE-S3 or SSE-KMS when the S3 client does not
// provide an explicit key ID.
func (v *vaultService) DefaultKeyID() string {
return v.config.Key.Name
}
// Info returns some information about the Vault,
// configuration - like the endpoints or authentication
// method.
func (v *vaultService) Info() KMSInfo {
return KMSInfo{
Endpoints: []string{v.config.Endpoint},
Name: v.DefaultKeyID(),
AuthType: v.config.Auth.Type,
}
func (v *vaultService) Stat() (kms.Status, error) {
return kms.Status{
Endpoints: []string{v.config.Endpoint},
Name: "Hashicorp Vault",
DefaultKey: v.config.Key.Name,
}, nil
}
// CreateKey is a stub that exists such that the Vault
@ -222,10 +216,13 @@ func (v *vaultService) CreateKey(keyID string) error {
// and a sealed version of this plaintext key encrypted using the
// named key referenced by keyID. It also binds the generated key
// cryptographically to the provided context.
func (v *vaultService) GenerateKey(keyID string, ctx Context) (key [32]byte, sealedKey []byte, err error) {
func (v *vaultService) GenerateKey(keyID string, ctx Context) (kms.DEK, error) {
if keyID == "" {
keyID = v.config.Key.Name
}
context, err := ctx.MarshalText()
if err != nil {
return key, sealedKey, err
return kms.DEK{}, err
}
payload := map[string]interface{}{
@ -233,24 +230,27 @@ func (v *vaultService) GenerateKey(keyID string, ctx Context) (key [32]byte, sea
}
s, err := v.client.Logical().Write(fmt.Sprintf("/transit/datakey/plaintext/%s", keyID), payload)
if err != nil {
return key, sealedKey, Errorf("crypto: client error %w", err)
return kms.DEK{}, Errorf("crypto: client error %w", err)
}
sealKey, ok := s.Data["ciphertext"].(string)
if !ok {
return key, sealedKey, Errorf("crypto: incorrect 'ciphertext' key type %v", s.Data["ciphertext"])
return kms.DEK{}, Errorf("crypto: incorrect 'ciphertext' key type %v", s.Data["ciphertext"])
}
plainKeyB64, ok := s.Data["plaintext"].(string)
if !ok {
return key, sealedKey, Errorf("crypto: incorrect 'plaintext' key type %v", s.Data["plaintext"])
return kms.DEK{}, Errorf("crypto: incorrect 'plaintext' key type %v", s.Data["plaintext"])
}
plainKey, err := base64.StdEncoding.DecodeString(plainKeyB64)
if err != nil {
return key, sealedKey, Errorf("crypto: invalid base64 key %w", err)
return kms.DEK{}, Errorf("crypto: invalid base64 key %w", err)
}
copy(key[:], plainKey)
return key, []byte(sealKey), nil
return kms.DEK{
KeyID: keyID,
Plaintext: plainKey,
Ciphertext: []byte(sealKey),
}, nil
}
// UnsealKey returns the decrypted sealedKey as plaintext key.
@ -260,10 +260,10 @@ func (v *vaultService) GenerateKey(keyID string, ctx Context) (key [32]byte, sea
//
// The context must be same context as the one provided while
// generating the plaintext key / sealedKey.
func (v *vaultService) UnsealKey(keyID string, sealedKey []byte, ctx Context) (key [32]byte, err error) {
func (v *vaultService) DecryptKey(keyID string, sealedKey []byte, ctx Context) ([]byte, error) {
context, err := ctx.MarshalText()
if err != nil {
return key, err
return nil, err
}
payload := map[string]interface{}{
@ -273,18 +273,17 @@ func (v *vaultService) UnsealKey(keyID string, sealedKey []byte, ctx Context) (k
s, err := v.client.Logical().Write(fmt.Sprintf("/transit/decrypt/%s", keyID), payload)
if err != nil {
return key, Errorf("crypto: client error %w", err)
return nil, Errorf("crypto: client error %w", err)
}
base64Key, ok := s.Data["plaintext"].(string)
if !ok {
return key, Errorf("crypto: incorrect 'plaintext' key type %v", s.Data["plaintext"])
return nil, Errorf("crypto: incorrect 'plaintext' key type %v", s.Data["plaintext"])
}
plainKey, err := base64.StdEncoding.DecodeString(base64Key)
if err != nil {
return key, Errorf("crypto: invalid base64 key %w", err)
return nil, Errorf("crypto: invalid base64 key %w", err)
}
copy(key[:], plainKey)
return key, nil
return plainKey, nil
}

View File

@ -40,6 +40,7 @@ import (
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/disk"
"github.com/minio/minio/pkg/fips"
"github.com/minio/minio/pkg/kms"
"github.com/minio/sio"
)
@ -673,14 +674,14 @@ func newCacheEncryptMetadata(bucket, object string, metadata map[string]string)
if globalCacheKMS == nil {
return nil, errKMSNotConfigured
}
key, encKey, err := globalCacheKMS.GenerateKey(globalCacheKMS.DefaultKeyID(), crypto.Context{bucket: pathJoin(bucket, object)})
key, err := globalCacheKMS.GenerateKey("", kms.Context{bucket: pathJoin(bucket, object)})
if err != nil {
return nil, err
}
objectKey := crypto.GenerateKey(key, rand.Reader)
sealedKey = objectKey.Seal(key, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, object)
crypto.S3.CreateMetadata(metadata, globalCacheKMS.DefaultKeyID(), encKey, sealedKey)
objectKey := crypto.GenerateKey(key.Plaintext, rand.Reader)
sealedKey = objectKey.Seal(key.Plaintext, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, object)
crypto.S3.CreateMetadata(metadata, key.KeyID, key.Ciphertext, sealedKey)
if etag, ok := metadata["etag"]; ok {
metadata["etag"] = hex.EncodeToString(objectKey.SealETag([]byte(etag)))

View File

@ -245,11 +245,10 @@ func decryptCacheObjectETag(info *ObjectInfo) error {
return errKMSNotConfigured
}
keyID, kmsKey, sealedKey, err := crypto.S3.ParseMetadata(info.UserDefined)
if err != nil {
return err
}
extKey, err := globalCacheKMS.UnsealKey(keyID, kmsKey, crypto.Context{info.Bucket: path.Join(info.Bucket, info.Name)})
extKey, err := globalCacheKMS.DecryptKey(keyID, kmsKey, crypto.Context{info.Bucket: path.Join(info.Bucket, info.Name)})
if err != nil {
return err
}

View File

@ -126,9 +126,7 @@ func rotateKey(oldKey []byte, newKey []byte, bucket, object string, metadata map
}
var objectKey crypto.ObjectKey
var extKey [32]byte
copy(extKey[:], oldKey)
if err = objectKey.Unseal(extKey, sealedKey, crypto.SSEC.String(), bucket, object); err != nil {
if err = objectKey.Unseal(oldKey, sealedKey, crypto.SSEC.String(), bucket, object); err != nil {
if subtle.ConstantTimeCompare(oldKey, newKey) == 1 {
return errInvalidSSEParameters // AWS returns special error for equal but invalid keys.
}
@ -138,8 +136,7 @@ func rotateKey(oldKey []byte, newKey []byte, bucket, object string, metadata map
if subtle.ConstantTimeCompare(oldKey, newKey) == 1 && sealedKey.Algorithm == crypto.SealAlgorithm {
return nil // don't rotate on equal keys if seal algorithm is latest
}
copy(extKey[:], newKey)
sealedKey = objectKey.Seal(extKey, sealedKey.IV, crypto.SSEC.String(), bucket, object)
sealedKey = objectKey.Seal(newKey, sealedKey.IV, crypto.SSEC.String(), bucket, object)
crypto.SSEC.CreateMetadata(metadata, sealedKey)
return nil
case crypto.S3.IsEncrypted(metadata):
@ -150,7 +147,7 @@ func rotateKey(oldKey []byte, newKey []byte, bucket, object string, metadata map
if err != nil {
return err
}
oldKey, err := GlobalKMS.UnsealKey(keyID, kmsKey, crypto.Context{bucket: path.Join(bucket, object)})
oldKey, err := GlobalKMS.DecryptKey(keyID, kmsKey, crypto.Context{bucket: path.Join(bucket, object)})
if err != nil {
return err
}
@ -159,12 +156,12 @@ func rotateKey(oldKey []byte, newKey []byte, bucket, object string, metadata map
return err
}
newKey, encKey, err := GlobalKMS.GenerateKey(GlobalKMS.DefaultKeyID(), crypto.Context{bucket: path.Join(bucket, object)})
newKey, err := GlobalKMS.GenerateKey("", crypto.Context{bucket: path.Join(bucket, object)})
if err != nil {
return err
}
sealedKey = objectKey.Seal(newKey, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, object)
crypto.S3.CreateMetadata(metadata, GlobalKMS.DefaultKeyID(), encKey, sealedKey)
sealedKey = objectKey.Seal(newKey.Plaintext, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, object)
crypto.S3.CreateMetadata(metadata, newKey.KeyID, newKey.Ciphertext, sealedKey)
return nil
}
}
@ -175,20 +172,18 @@ func newEncryptMetadata(key []byte, bucket, object string, metadata map[string]s
if GlobalKMS == nil {
return crypto.ObjectKey{}, errKMSNotConfigured
}
key, encKey, err := GlobalKMS.GenerateKey(GlobalKMS.DefaultKeyID(), crypto.Context{bucket: path.Join(bucket, object)})
key, err := GlobalKMS.GenerateKey("", crypto.Context{bucket: path.Join(bucket, object)})
if err != nil {
return crypto.ObjectKey{}, err
}
objectKey := crypto.GenerateKey(key, rand.Reader)
sealedKey = objectKey.Seal(key, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, object)
crypto.S3.CreateMetadata(metadata, GlobalKMS.DefaultKeyID(), encKey, sealedKey)
objectKey := crypto.GenerateKey(key.Plaintext, rand.Reader)
sealedKey = objectKey.Seal(key.Plaintext, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, object)
crypto.S3.CreateMetadata(metadata, key.KeyID, key.Ciphertext, sealedKey)
return objectKey, nil
}
var extKey [32]byte
copy(extKey[:], key)
objectKey := crypto.GenerateKey(extKey, rand.Reader)
sealedKey = objectKey.Seal(extKey, crypto.GenerateIV(rand.Reader), crypto.SSEC.String(), bucket, object)
objectKey := crypto.GenerateKey(key, rand.Reader)
sealedKey = objectKey.Seal(key, crypto.GenerateIV(rand.Reader), crypto.SSEC.String(), bucket, object)
crypto.SSEC.CreateMetadata(metadata, sealedKey)
return objectKey, nil
}
@ -276,12 +271,8 @@ func decryptObjectInfo(key []byte, bucket, object string, metadata map[string]st
if err != nil {
return nil, err
}
var (
objectKey crypto.ObjectKey
extKey [32]byte
)
copy(extKey[:], key)
if err = objectKey.Unseal(extKey, sealedKey, crypto.SSEC.String(), bucket, object); err != nil {
var objectKey crypto.ObjectKey
if err = objectKey.Unseal(key, sealedKey, crypto.SSEC.String(), bucket, object); err != nil {
return nil, err
}
return objectKey[:], nil

View File

@ -27,6 +27,7 @@ import (
"github.com/minio/minio-go/v7/pkg/set"
"github.com/minio/minio/pkg/bucket/bandwidth"
"github.com/minio/minio/pkg/handlers"
"github.com/minio/minio/pkg/kms"
humanize "github.com/dustin/go-humanize"
"github.com/minio/minio/cmd/config/cache"
@ -36,7 +37,6 @@ import (
"github.com/minio/minio/cmd/config/identity/openid"
"github.com/minio/minio/cmd/config/policy/opa"
"github.com/minio/minio/cmd/config/storageclass"
"github.com/minio/minio/cmd/crypto"
xhttp "github.com/minio/minio/cmd/http"
"github.com/minio/minio/pkg/auth"
etcd "go.etcd.io/etcd/clientv3"
@ -233,7 +233,7 @@ var (
globalCacheConfig cache.Config
// Initialized KMS configuration for disk cache
globalCacheKMS crypto.KMS
globalCacheKMS kms.KMS
// Allocated etcd endpoint for config and bucket DNS.
globalEtcdClient *etcd.Client
@ -246,7 +246,7 @@ var (
globalDNSConfig dns.Store
// GlobalKMS initialized KMS configuration
GlobalKMS crypto.KMS
GlobalKMS kms.KMS
// Auto-Encryption, if enabled, turns any non-SSE-C request
// into an SSE-S3 request. If enabled a valid, non-empty KMS

256
pkg/kms/context.go Normal file
View File

@ -0,0 +1,256 @@
// MinIO Cloud Storage, (C) 2021 MinIO, Inc.
//
// 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 kms
import (
"bytes"
"sort"
"unicode/utf8"
)
// Context is a set of key-value pairs that
// are associated with a generate data encryption
// key (DEK).
//
// A KMS implementation may bind the context to the
// generated DEK such that the same context must be
// provided when decrypting an encrypted DEK.
type Context map[string]string
// MarshalText returns a canonical text representation of
// the Context.
// MarshalText sorts the context keys and writes the sorted
// key-value pairs as canonical JSON object. The sort order
// is based on the un-escaped keys. It never returns an error.
func (c Context) MarshalText() ([]byte, error) {
if len(c) == 0 {
return []byte{'{', '}'}, nil
}
// Pre-allocate a buffer - 128 bytes is an arbitrary
// heuristic value that seems like a good starting size.
var b = bytes.NewBuffer(make([]byte, 0, 128))
if len(c) == 1 {
for k, v := range c {
b.WriteString(`{"`)
escapeStringJSON(b, k)
b.WriteString(`":"`)
escapeStringJSON(b, v)
b.WriteString(`"}`)
}
return b.Bytes(), nil
}
sortedKeys := make([]string, 0, len(c))
for k := range c {
sortedKeys = append(sortedKeys, k)
}
sort.Strings(sortedKeys)
b.WriteByte('{')
for i, k := range sortedKeys {
b.WriteByte('"')
escapeStringJSON(b, k)
b.WriteString(`":"`)
escapeStringJSON(b, c[k])
b.WriteByte('"')
if i < len(sortedKeys)-1 {
b.WriteByte(',')
}
}
b.WriteByte('}')
return b.Bytes(), nil
}
// Adapted from Go stdlib.
var hexTable = "0123456789abcdef"
// escapeStringJSON will escape a string for JSON and write it to dst.
func escapeStringJSON(dst *bytes.Buffer, s string) {
start := 0
for i := 0; i < len(s); {
if b := s[i]; b < utf8.RuneSelf {
if htmlSafeSet[b] {
i++
continue
}
if start < i {
dst.WriteString(s[start:i])
}
dst.WriteByte('\\')
switch b {
case '\\', '"':
dst.WriteByte(b)
case '\n':
dst.WriteByte('n')
case '\r':
dst.WriteByte('r')
case '\t':
dst.WriteByte('t')
default:
// This encodes bytes < 0x20 except for \t, \n and \r.
// If escapeHTML is set, it also escapes <, >, and &
// because they can lead to security holes when
// user-controlled strings are rendered into JSON
// and served to some browsers.
dst.WriteString(`u00`)
dst.WriteByte(hexTable[b>>4])
dst.WriteByte(hexTable[b&0xF])
}
i++
start = i
continue
}
c, size := utf8.DecodeRuneInString(s[i:])
if c == utf8.RuneError && size == 1 {
if start < i {
dst.WriteString(s[start:i])
}
dst.WriteString(`\ufffd`)
i += size
start = i
continue
}
// U+2028 is LINE SEPARATOR.
// U+2029 is PARAGRAPH SEPARATOR.
// They are both technically valid characters in JSON strings,
// but don't work in JSONP, which has to be evaluated as JavaScript,
// and can lead to security holes there. It is valid JSON to
// escape them, so we do so unconditionally.
// See http://timelessrepo.com/json-isnt-a-javascript-subset for discussion.
if c == '\u2028' || c == '\u2029' {
if start < i {
dst.WriteString(s[start:i])
}
dst.WriteString(`\u202`)
dst.WriteByte(hexTable[c&0xF])
i += size
start = i
continue
}
i += size
}
if start < len(s) {
dst.WriteString(s[start:])
}
}
// htmlSafeSet holds the value true if the ASCII character with the given
// array position can be safely represented inside a JSON string, embedded
// inside of HTML <script> tags, without any additional escaping.
//
// All values are true except for the ASCII control characters (0-31), the
// double quote ("), the backslash character ("\"), HTML opening and closing
// tags ("<" and ">"), and the ampersand ("&").
var htmlSafeSet = [utf8.RuneSelf]bool{
' ': true,
'!': true,
'"': false,
'#': true,
'$': true,
'%': true,
'&': false,
'\'': true,
'(': true,
')': true,
'*': true,
'+': true,
',': true,
'-': true,
'.': true,
'/': true,
'0': true,
'1': true,
'2': true,
'3': true,
'4': true,
'5': true,
'6': true,
'7': true,
'8': true,
'9': true,
':': true,
';': true,
'<': false,
'=': true,
'>': false,
'?': true,
'@': true,
'A': true,
'B': true,
'C': true,
'D': true,
'E': true,
'F': true,
'G': true,
'H': true,
'I': true,
'J': true,
'K': true,
'L': true,
'M': true,
'N': true,
'O': true,
'P': true,
'Q': true,
'R': true,
'S': true,
'T': true,
'U': true,
'V': true,
'W': true,
'X': true,
'Y': true,
'Z': true,
'[': true,
'\\': false,
']': true,
'^': true,
'_': true,
'`': true,
'a': true,
'b': true,
'c': true,
'd': true,
'e': true,
'f': true,
'g': true,
'h': true,
'i': true,
'j': true,
'k': true,
'l': true,
'm': true,
'n': true,
'o': true,
'p': true,
'q': true,
'r': true,
's': true,
't': true,
'u': true,
'v': true,
'w': true,
'x': true,
'y': true,
'z': true,
'{': true,
'|': true,
'}': true,
'~': true,
'\u007f': true,
}

69
pkg/kms/dek_test.go Normal file
View File

@ -0,0 +1,69 @@
// MinIO Cloud Storage, (C) 2021 MinIO, Inc.
//
// 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 kms
import (
"bytes"
"encoding/base64"
"testing"
)
var dekEncodeDecodeTests = []struct {
Key DEK
}{
{
Key: DEK{},
},
{
Key: DEK{
Plaintext: nil,
Ciphertext: mustDecodeB64("eyJhZWFkIjoiQUVTLTI1Ni1HQ00tSE1BQy1TSEEtMjU2IiwiaXYiOiJ3NmhLUFVNZXVtejZ5UlVZL29pTFVBPT0iLCJub25jZSI6IktMSEU3UE1jRGo2N2UweHkiLCJieXRlcyI6Ik1wUkhjQWJaTzZ1Sm5lUGJGcnpKTkxZOG9pdkxwTmlUcTNLZ0hWdWNGYkR2Y0RlbEh1c1lYT29zblJWVTZoSXIifQ=="),
},
},
{
Key: DEK{
Plaintext: mustDecodeB64("GM2UvLXp/X8lzqq0mibFC0LayDCGlmTHQhYLj7qAy7Q="),
Ciphertext: mustDecodeB64("eyJhZWFkIjoiQUVTLTI1Ni1HQ00tSE1BQy1TSEEtMjU2IiwiaXYiOiJ3NmhLUFVNZXVtejZ5UlVZL29pTFVBPT0iLCJub25jZSI6IktMSEU3UE1jRGo2N2UweHkiLCJieXRlcyI6Ik1wUkhjQWJaTzZ1Sm5lUGJGcnpKTkxZOG9pdkxwTmlUcTNLZ0hWdWNGYkR2Y0RlbEh1c1lYT29zblJWVTZoSXIifQ=="),
},
},
}
func TestEncodeDecodeDEK(t *testing.T) {
for i, test := range dekEncodeDecodeTests {
text, err := test.Key.MarshalText()
if err != nil {
t.Fatalf("Test %d: failed to marshal DEK: %v", i, err)
}
var key DEK
if err = key.UnmarshalText(text); err != nil {
t.Fatalf("Test %d: failed to unmarshal DEK: %v", i, err)
}
if key.Plaintext != nil {
t.Fatalf("Test %d: unmarshaled DEK contains non-nil plaintext", i)
}
if !bytes.Equal(key.Ciphertext, test.Key.Ciphertext) {
t.Fatalf("Test %d: ciphertext mismatch: got %x - want %x", i, key.Ciphertext, test.Key.Ciphertext)
}
}
}
func mustDecodeB64(s string) []byte {
b, err := base64.StdEncoding.DecodeString(s)
if err != nil {
panic(err)
}
return b
}

111
pkg/kms/kms.go Normal file
View File

@ -0,0 +1,111 @@
// MinIO Cloud Storage, (C) 2021 MinIO, Inc.
//
// 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 kms
import (
"encoding"
"encoding/json"
)
// KMS is the generic interface that abstracts over
// different KMS implementations.
type KMS interface {
// Stat returns the current KMS status.
Stat() (Status, error)
// CreateKey creates a new key at the KMS with the given key ID.
CreateKey(keyID string) error
// GenerateKey generates a new data encryption key using the
// key referenced by the key ID.
//
// The KMS may use a default key if the key ID is empty.
// GenerateKey returns an error if the referenced key does
// not exist.
//
// The context is associated and tied to the generated DEK.
// The same context must be provided when the generated key
// should be decrypted. Therefore, it is the callers
// responsibility to remember the corresponding context for
// a particular DEK. The context may be nil.
GenerateKey(keyID string, context Context) (DEK, error)
// DecryptKey decrypts the ciphertext with the key referenced
// by the key ID. The context must match the context value
// used to generate the ciphertext.
DecryptKey(keyID string, ciphertext []byte, context Context) ([]byte, error)
}
// Status describes the current state of a KMS.
type Status struct {
Name string // The name of the KMS
Endpoints []string // A set of the KMS endpoints
// DefaultKey is the key used when no explicit key ID
// is specified. It is empty if the KMS does not support
// a default key.
DefaultKey string
}
// DEK is a data encryption key. It consists of a
// plaintext-ciphertext pair and the ID of the key
// used to generate the ciphertext.
//
// The plaintext can be used for cryptographic
// operations - like encrypting some data. The
// ciphertext is the encrypted version of the
// plaintext data and can be stored on untrusted
// storage.
type DEK struct {
KeyID string
Plaintext []byte
Ciphertext []byte
}
var (
_ encoding.TextMarshaler = (*DEK)(nil)
_ encoding.TextUnmarshaler = (*DEK)(nil)
)
// MarshalText encodes the DEK's key ID and ciphertext
// as JSON.
func (d DEK) MarshalText() ([]byte, error) {
type JSON struct {
KeyID string `json:"keyid"`
Ciphertext []byte `json:"ciphertext"`
}
return json.Marshal(JSON{
KeyID: d.KeyID,
Ciphertext: d.Ciphertext,
})
}
// UnmarshalText tries to decode text as JSON representation
// of a DEK and sets DEK's key ID and ciphertext to the
// decoded values.
//
// It sets DEK's plaintext to nil.
func (d *DEK) UnmarshalText(text []byte) error {
type JSON struct {
KeyID string `json:"keyid"`
Ciphertext []byte `json:"ciphertext"`
}
var v JSON
if err := json.Unmarshal(text, &v); err != nil {
return err
}
d.KeyID, d.Plaintext, d.Ciphertext = v.KeyID, nil, v.Ciphertext
return nil
}

226
pkg/kms/single-key.go Normal file
View File

@ -0,0 +1,226 @@
// MinIO Cloud Storage, (C) 2021 MinIO, Inc.
//
// 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 kms
import (
"crypto/aes"
"crypto/cipher"
"crypto/hmac"
"crypto/sha256"
"encoding/base64"
"encoding/json"
"errors"
"fmt"
"strconv"
"strings"
"github.com/secure-io/sio-go/sioutil"
"golang.org/x/crypto/chacha20"
"golang.org/x/crypto/chacha20poly1305"
)
// Parse parses s as single-key KMS. The given string
// is expected to have the following format:
// <key-id>:<base64-key>
//
// The returned KMS implementation uses the parsed
// key ID and key to derive new DEKs and decrypt ciphertext.
func Parse(s string) (KMS, error) {
v := strings.SplitN(s, ":", 2)
if len(v) != 2 {
return nil, errors.New("kms: invalid master key format")
}
var keyID, b64Key = v[0], v[1]
key, err := base64.StdEncoding.DecodeString(b64Key)
if err != nil {
return nil, err
}
return New(keyID, key)
}
// New returns a single-key KMS that derives new DEKs from the
// given key.
func New(keyID string, key []byte) (KMS, error) {
if len(key) != 32 {
return nil, errors.New("kms: invalid key length " + strconv.Itoa(len(key)))
}
return secretKey{
keyID: keyID,
key: key,
}, nil
}
// secretKey is a KMS implementation that derives new DEKs
// from a single key.
type secretKey struct {
keyID string
key []byte
}
var _ KMS = secretKey{} // compiler check
const ( // algorithms used to derive and encrypt DEKs
algorithmAESGCM = "AES-256-GCM-HMAC-SHA-256"
algorithmChaCha20Poly1305 = "ChaCha20Poly1305"
)
func (kms secretKey) Stat() (Status, error) {
return Status{
Name: "SecretKey",
DefaultKey: kms.keyID,
}, nil
}
func (secretKey) CreateKey(string) error {
return errors.New("kms: creating keys is not supported")
}
func (kms secretKey) GenerateKey(keyID string, context Context) (DEK, error) {
if keyID == "" {
keyID = kms.keyID
}
if keyID != kms.keyID {
return DEK{}, fmt.Errorf("kms: key %q does not exist", keyID)
}
iv, err := sioutil.Random(16)
if err != nil {
return DEK{}, err
}
var algorithm string
if sioutil.NativeAES() {
algorithm = algorithmAESGCM
} else {
algorithm = algorithmChaCha20Poly1305
}
var aead cipher.AEAD
switch algorithm {
case algorithmAESGCM:
mac := hmac.New(sha256.New, kms.key)
mac.Write(iv)
sealingKey := mac.Sum(nil)
var block cipher.Block
block, err = aes.NewCipher(sealingKey)
if err != nil {
return DEK{}, err
}
aead, err = cipher.NewGCM(block)
if err != nil {
return DEK{}, err
}
case algorithmChaCha20Poly1305:
var sealingKey []byte
sealingKey, err = chacha20.HChaCha20(kms.key, iv)
if err != nil {
return DEK{}, err
}
aead, err = chacha20poly1305.New(sealingKey)
if err != nil {
return DEK{}, err
}
default:
return DEK{}, errors.New("invalid algorithm: " + algorithm)
}
nonce, err := sioutil.Random(aead.NonceSize())
if err != nil {
return DEK{}, err
}
plaintext, err := sioutil.Random(32)
if err != nil {
return DEK{}, err
}
associatedData, _ := context.MarshalText()
ciphertext := aead.Seal(nil, nonce, plaintext, associatedData)
ciphertext, err = json.Marshal(encryptedKey{
Algorithm: algorithm,
IV: iv,
Nonce: nonce,
Bytes: ciphertext,
})
if err != nil {
return DEK{}, err
}
return DEK{
KeyID: keyID,
Plaintext: plaintext,
Ciphertext: ciphertext,
}, nil
}
func (kms secretKey) DecryptKey(keyID string, ciphertext []byte, context Context) ([]byte, error) {
if keyID != kms.keyID {
return nil, fmt.Errorf("kms: key %q does not exist", keyID)
}
var encryptedKey encryptedKey
if err := json.Unmarshal(ciphertext, &encryptedKey); err != nil {
return nil, err
}
if n := len(encryptedKey.IV); n != 16 {
return nil, fmt.Errorf("kms: invalid iv size")
}
var aead cipher.AEAD
switch encryptedKey.Algorithm {
case algorithmAESGCM:
mac := hmac.New(sha256.New, kms.key)
mac.Write(encryptedKey.IV)
sealingKey := mac.Sum(nil)
block, err := aes.NewCipher(sealingKey[:])
if err != nil {
return nil, err
}
aead, err = cipher.NewGCM(block)
if err != nil {
return nil, err
}
case algorithmChaCha20Poly1305:
sealingKey, err := chacha20.HChaCha20(kms.key, encryptedKey.IV)
if err != nil {
return nil, err
}
aead, err = chacha20poly1305.New(sealingKey)
if err != nil {
return nil, err
}
default:
return nil, fmt.Errorf("kms: invalid algorithm: %q", encryptedKey.Algorithm)
}
if n := len(encryptedKey.Nonce); n != aead.NonceSize() {
return nil, fmt.Errorf("kms: invalid nonce size %d", n)
}
associatedData, _ := context.MarshalText()
plaintext, err := aead.Open(nil, encryptedKey.Nonce, encryptedKey.Bytes, associatedData)
if err != nil {
return nil, fmt.Errorf("kms: encrypted key is not authentic")
}
return plaintext, nil
}
type encryptedKey struct {
Algorithm string `json:"aead"`
IV []byte `json:"iv"`
Nonce []byte `json:"nonce"`
Bytes []byte `json:"bytes"`
}

View File

@ -0,0 +1,84 @@
// MinIO Cloud Storage, (C) 2021 MinIO, Inc.
//
// 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 kms
import (
"bytes"
"encoding/base64"
"testing"
)
func TestSingleKeyRoundtrip(t *testing.T) {
KMS, err := Parse("my-key:eEm+JI9/q4JhH8QwKvf3LKo4DEBl6QbfvAl1CAbMIv8=")
if err != nil {
t.Fatalf("Failed to initialize KMS: %v", err)
}
key, err := KMS.GenerateKey("my-key", Context{})
if err != nil {
t.Fatalf("Failed to generate key: %v", err)
}
plaintext, err := KMS.DecryptKey(key.KeyID, key.Ciphertext, Context{})
if err != nil {
t.Fatalf("Failed to decrypt key: %v", err)
}
if !bytes.Equal(key.Plaintext, plaintext) {
t.Fatalf("Decrypted key does not match generated one: got %x - want %x", key.Plaintext, plaintext)
}
}
func TestDecryptKey(t *testing.T) {
KMS, err := Parse("my-key:eEm+JI9/q4JhH8QwKvf3LKo4DEBl6QbfvAl1CAbMIv8=")
if err != nil {
t.Fatalf("Failed to initialize KMS: %v", err)
}
for i, test := range decryptKeyTests {
dataKey, err := base64.StdEncoding.DecodeString(test.Plaintext)
if err != nil {
t.Fatalf("Test %d: failed to decode plaintext key: %v", i, err)
}
ciphertext, err := base64.StdEncoding.DecodeString(test.Ciphertext)
if err != nil {
t.Fatalf("Test %d: failed to decode ciphertext key: %v", i, err)
}
plaintext, err := KMS.DecryptKey(test.KeyID, ciphertext, test.Context)
if err != nil {
t.Fatalf("Test %d: failed to decrypt key: %v", i, err)
}
if !bytes.Equal(plaintext, dataKey) {
t.Fatalf("Test %d: decrypted key does not generated one: got %x - want %x", i, plaintext, dataKey)
}
}
}
var decryptKeyTests = []struct {
KeyID string
Plaintext string
Ciphertext string
Context Context
}{
{
KeyID: "my-key",
Plaintext: "zmS7NrG765UZ0ZN85oPjybelxqVvpz01vxsSpOISy2M=",
Ciphertext: "eyJhZWFkIjoiQ2hhQ2hhMjBQb2x5MTMwNSIsIml2IjoiSmJJK3Z3dll3dzFsQ2I1VnBrQUZ1UT09Iiwibm9uY2UiOiJBUmpJakp4QlNENTQxR3o4IiwiYnl0ZXMiOiJLQ2JFYzJzQTBUTHZBN2FXVFdhMjNBZGNjVmZKTXBPeHdnRzhobSs0UGFOcnhZZnkxeEZXWmcyZ0VlblZyT2d2In0=",
},
{
KeyID: "my-key",
Plaintext: "UnPWsZgVI+T4L9WGNzFlP1PsP1Z6hn2Fx8ISeZfDGnA=",
Ciphertext: "eyJhZWFkIjoiQ2hhQ2hhMjBQb2x5MTMwNSIsIml2IjoicjQreWZpVmJWSVlSMFoySTlGcSs2Zz09Iiwibm9uY2UiOiIyWXB3R3dFNTlHY1ZyYUkzIiwiYnl0ZXMiOiJrL3N2TWdsT1U3L0tnd3Y3M2hlRzM4TldXNTc1WExjRnAzU2F4UUhETWpKR1l5UkkzRml5Z3UyT2V1dEdQWE5MIn0=",
Context: Context{"key": "value"},
},
}