mirror of
https://github.com/apple/foundationdb.git
synced 2025-05-15 02:18:39 +08:00
This PR add support for TLog encryption through commit proxy. The encryption is done on per-mutation basis. As CP writes mutations to TLog, it inserts encryption header alongside encrypted mutations. Storage server (and other consumers of TLog such as storage cache and backup worker) decrypts the mutations as they peek TLog.
353 lines
13 KiB
C++
353 lines
13 KiB
C++
/*
|
|
* SimEncryptKmsProxy.actor.cpp
|
|
*
|
|
* This source file is part of the FoundationDB open source project
|
|
*
|
|
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
|
*
|
|
* 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.
|
|
*/
|
|
|
|
#include "fdbserver/SimKmsConnector.h"
|
|
|
|
#include "fmt/format.h"
|
|
#include "fdbrpc/sim_validation.h"
|
|
#include "fdbserver/KmsConnectorInterface.h"
|
|
#include "fdbserver/Knobs.h"
|
|
#include "flow/ActorCollection.h"
|
|
#include "flow/Arena.h"
|
|
#include "flow/BlobCipher.h"
|
|
#include "flow/EncryptUtils.h"
|
|
#include "flow/Error.h"
|
|
#include "flow/FastRef.h"
|
|
#include "flow/IRandom.h"
|
|
#include "flow/ITrace.h"
|
|
#include "flow/Knobs.h"
|
|
#include "flow/Trace.h"
|
|
#include "flow/flow.h"
|
|
#include "flow/network.h"
|
|
#include "flow/UnitTest.h"
|
|
|
|
#include <memory>
|
|
#include <unordered_map>
|
|
#include <utility>
|
|
|
|
#include "flow/actorcompiler.h" // This must be the last #include.
|
|
|
|
using SimEncryptKey = std::string;
|
|
struct SimEncryptKeyCtx {
|
|
EncryptCipherBaseKeyId id;
|
|
SimEncryptKey key;
|
|
|
|
explicit SimEncryptKeyCtx(EncryptCipherBaseKeyId kId, const char* data) : id(kId), key(data, AES_256_KEY_LENGTH) {}
|
|
};
|
|
|
|
// The credentials may be allowed to change, but the storage locations and partitioning cannot change, even across
|
|
// restarts. Keep it as global static state in simulation.
|
|
static std::unordered_map<BlobMetadataDomainId, Standalone<BlobMetadataDetailsRef>> simBlobMetadataStore;
|
|
|
|
struct SimKmsConnectorContext : NonCopyable, ReferenceCounted<SimKmsConnectorContext> {
|
|
uint32_t maxEncryptionKeys;
|
|
std::unordered_map<EncryptCipherBaseKeyId, std::unique_ptr<SimEncryptKeyCtx>> simEncryptKeyStore;
|
|
|
|
explicit SimKmsConnectorContext(uint32_t keyCount) : maxEncryptionKeys(keyCount) {
|
|
const unsigned char SHA_KEY[] = "0c39e7906db6d51ac0573d328ce1b6be";
|
|
|
|
// Construct encryption keyStore.
|
|
// Note the keys generated must be the same after restart.
|
|
for (int i = 1; i <= maxEncryptionKeys; i++) {
|
|
Arena arena;
|
|
StringRef digest = computeAuthToken(
|
|
reinterpret_cast<const unsigned char*>(&i), sizeof(i), SHA_KEY, AES_256_KEY_LENGTH, arena);
|
|
simEncryptKeyStore[i] =
|
|
std::make_unique<SimEncryptKeyCtx>(i, reinterpret_cast<const char*>(digest.begin()));
|
|
}
|
|
}
|
|
};
|
|
|
|
namespace {
|
|
Optional<int64_t> getRefreshInterval(int64_t now, int64_t defaultTtl) {
|
|
if (BUGGIFY) {
|
|
return Optional<int64_t>(now + defaultTtl);
|
|
}
|
|
return Optional<int64_t>();
|
|
}
|
|
|
|
Optional<int64_t> getExpireInterval(Optional<int64_t> refTS) {
|
|
if (BUGGIFY) {
|
|
return Optional<int64_t>(-1);
|
|
}
|
|
return refTS;
|
|
}
|
|
} // namespace
|
|
|
|
ACTOR Future<Void> ekLookupByIds(Reference<SimKmsConnectorContext> ctx,
|
|
KmsConnectorInterface interf,
|
|
KmsConnLookupEKsByKeyIdsReq req) {
|
|
state KmsConnLookupEKsByKeyIdsRep rep;
|
|
state bool success = true;
|
|
state Optional<TraceEvent> dbgKIdTrace =
|
|
req.debugId.present() ? TraceEvent("SimKmsGetByKeyIds", interf.id()) : Optional<TraceEvent>();
|
|
|
|
if (dbgKIdTrace.present()) {
|
|
dbgKIdTrace.get().setMaxEventLength(100000);
|
|
dbgKIdTrace.get().detail("DbgId", req.debugId.get());
|
|
}
|
|
|
|
// Lookup corresponding EncryptKeyCtx for input keyId
|
|
for (const auto& item : req.encryptKeyInfos) {
|
|
const auto& itr = ctx->simEncryptKeyStore.find(item.baseCipherId);
|
|
if (itr != ctx->simEncryptKeyStore.end()) {
|
|
rep.cipherKeyDetails.emplace_back_deep(
|
|
rep.arena, item.domainId, itr->first, StringRef(itr->second.get()->key));
|
|
|
|
if (dbgKIdTrace.present()) {
|
|
// {encryptDomainId, baseCipherId} forms a unique tuple across encryption domains
|
|
dbgKIdTrace.get().detail(
|
|
getEncryptDbgTraceKey(ENCRYPT_DBG_TRACE_RESULT_PREFIX, item.domainId, item.domainName, itr->first),
|
|
"");
|
|
}
|
|
} else {
|
|
success = false;
|
|
break;
|
|
}
|
|
}
|
|
|
|
wait(delayJittered(1.0)); // simulate network delay
|
|
|
|
success ? req.reply.send(rep) : req.reply.sendError(encrypt_key_not_found());
|
|
|
|
return Void();
|
|
}
|
|
|
|
ACTOR Future<Void> ekLookupByDomainIds(Reference<SimKmsConnectorContext> ctx,
|
|
KmsConnectorInterface interf,
|
|
KmsConnLookupEKsByDomainIdsReq req) {
|
|
state KmsConnLookupEKsByDomainIdsRep rep;
|
|
state bool success = true;
|
|
state Optional<TraceEvent> dbgDIdTrace =
|
|
req.debugId.present() ? TraceEvent("SimKmsGetsByDomIds", interf.id()) : Optional<TraceEvent>();
|
|
|
|
if (dbgDIdTrace.present()) {
|
|
dbgDIdTrace.get().setMaxEventLength(16384).detail("DbgId", req.debugId.get());
|
|
}
|
|
|
|
// Map encryptionDomainId to corresponding EncryptKeyCtx element using a modulo operation. This
|
|
// would mean multiple domains gets mapped to the same encryption key which is fine, the
|
|
// EncryptKeyStore guarantees that keyId -> plaintext encryptKey mapping is idempotent.
|
|
int64_t currTS = (int64_t)now();
|
|
// Fetch default TTL to avoid BUGGIFY giving different value per invocation causing refTS > expTS
|
|
int64_t defaultTtl = FLOW_KNOBS->ENCRYPT_CIPHER_KEY_CACHE_TTL;
|
|
Optional<int64_t> refAtTS = getRefreshInterval(currTS, defaultTtl);
|
|
Optional<int64_t> expAtTS = getExpireInterval(refAtTS);
|
|
for (const auto& info : req.encryptDomainInfos) {
|
|
EncryptCipherBaseKeyId keyId = 1 + abs(info.domainId) % SERVER_KNOBS->SIM_KMS_MAX_KEYS;
|
|
const auto& itr = ctx->simEncryptKeyStore.find(keyId);
|
|
if (itr != ctx->simEncryptKeyStore.end()) {
|
|
rep.cipherKeyDetails.emplace_back_deep(
|
|
req.arena, info.domainId, keyId, StringRef(itr->second.get()->key), refAtTS, expAtTS);
|
|
if (dbgDIdTrace.present()) {
|
|
// {encryptId, baseCipherId} forms a unique tuple across encryption domains
|
|
dbgDIdTrace.get().detail(
|
|
getEncryptDbgTraceKey(ENCRYPT_DBG_TRACE_RESULT_PREFIX, info.domainId, info.domainName, keyId), "");
|
|
}
|
|
} else {
|
|
success = false;
|
|
break;
|
|
}
|
|
}
|
|
|
|
wait(delayJittered(1.0)); // simulate network delay
|
|
|
|
success ? req.reply.send(rep) : req.reply.sendError(encrypt_key_not_found());
|
|
|
|
return Void();
|
|
}
|
|
|
|
static Standalone<BlobMetadataDetailsRef> createBlobMetadata(BlobMetadataDomainId domainId) {
|
|
Standalone<BlobMetadataDetailsRef> metadata;
|
|
metadata.domainId = domainId;
|
|
// 0 == no partition, 1 == suffix partitioned, 2 == storage location partitioned
|
|
int type = deterministicRandom()->randomInt(0, 3);
|
|
int partitionCount = (type == 0) ? 0 : deterministicRandom()->randomInt(2, 12);
|
|
fmt::print("SimBlobMetadata ({})\n", domainId);
|
|
TraceEvent ev(SevDebug, "SimBlobMetadata");
|
|
ev.detail("DomainId", domainId).detail("TypeNum", type).detail("PartitionCount", partitionCount);
|
|
if (type == 0) {
|
|
// single storage location
|
|
metadata.base = StringRef(metadata.arena(), "file://fdbblob/" + std::to_string(domainId) + "/");
|
|
fmt::print(" {}\n", metadata.base.get().printable());
|
|
ev.detail("Base", metadata.base);
|
|
}
|
|
if (type == 1) {
|
|
// simulate hash prefixing in s3
|
|
metadata.base = StringRef(metadata.arena(), "file://fdbblob/"_sr);
|
|
ev.detail("Base", metadata.base);
|
|
fmt::print(" {} ({})\n", metadata.base.get().printable(), partitionCount);
|
|
for (int i = 0; i < partitionCount; i++) {
|
|
metadata.partitions.push_back_deep(metadata.arena(),
|
|
deterministicRandom()->randomUniqueID().shortString() + "-" +
|
|
std::to_string(domainId) + "/");
|
|
fmt::print(" {}\n", metadata.partitions.back().printable());
|
|
ev.detail("P" + std::to_string(i), metadata.partitions.back());
|
|
}
|
|
}
|
|
if (type == 2) {
|
|
// simulate separate storage location per partition
|
|
for (int i = 0; i < partitionCount; i++) {
|
|
metadata.partitions.push_back_deep(
|
|
metadata.arena(), "file://fdbblob" + std::to_string(domainId) + "_" + std::to_string(i) + "/");
|
|
fmt::print(" {}\n", metadata.partitions.back().printable());
|
|
ev.detail("P" + std::to_string(i), metadata.partitions.back());
|
|
}
|
|
}
|
|
return metadata;
|
|
}
|
|
|
|
ACTOR Future<Void> blobMetadataLookup(KmsConnectorInterface interf, KmsConnBlobMetadataReq req) {
|
|
state KmsConnBlobMetadataRep rep;
|
|
state Optional<TraceEvent> dbgDIdTrace =
|
|
req.debugId.present() ? TraceEvent("SimKmsBlobMetadataLookup", interf.id()) : Optional<TraceEvent>();
|
|
if (dbgDIdTrace.present()) {
|
|
dbgDIdTrace.get().detail("DbgId", req.debugId.get());
|
|
}
|
|
|
|
for (BlobMetadataDomainId domainId : req.domainIds) {
|
|
auto it = simBlobMetadataStore.find(domainId);
|
|
if (it == simBlobMetadataStore.end()) {
|
|
// construct new blob metadata
|
|
it = simBlobMetadataStore.insert({ domainId, createBlobMetadata(domainId) }).first;
|
|
}
|
|
rep.metadataDetails.arena().dependsOn(it->second.arena());
|
|
rep.metadataDetails.push_back(rep.metadataDetails.arena(), it->second);
|
|
}
|
|
|
|
wait(delayJittered(1.0)); // simulate network delay
|
|
|
|
req.reply.send(rep);
|
|
|
|
return Void();
|
|
}
|
|
|
|
ACTOR Future<Void> simKmsConnectorCore_impl(KmsConnectorInterface interf) {
|
|
TraceEvent("SimEncryptKmsProxy_Init", interf.id()).detail("MaxEncryptKeys", SERVER_KNOBS->SIM_KMS_MAX_KEYS);
|
|
|
|
state Reference<SimKmsConnectorContext> ctx = makeReference<SimKmsConnectorContext>(SERVER_KNOBS->SIM_KMS_MAX_KEYS);
|
|
|
|
ASSERT_EQ(ctx->simEncryptKeyStore.size(), SERVER_KNOBS->SIM_KMS_MAX_KEYS);
|
|
|
|
state PromiseStream<Future<Void>> addActor;
|
|
state Future<Void> collection = actorCollection(addActor.getFuture());
|
|
|
|
loop {
|
|
choose {
|
|
when(KmsConnLookupEKsByKeyIdsReq req = waitNext(interf.ekLookupByIds.getFuture())) {
|
|
addActor.send(ekLookupByIds(ctx, interf, req));
|
|
}
|
|
when(KmsConnLookupEKsByDomainIdsReq req = waitNext(interf.ekLookupByDomainIds.getFuture())) {
|
|
addActor.send(ekLookupByDomainIds(ctx, interf, req));
|
|
}
|
|
when(KmsConnBlobMetadataReq req = waitNext(interf.blobMetadataReq.getFuture())) {
|
|
addActor.send(blobMetadataLookup(interf, req));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
Future<Void> SimKmsConnector::connectorCore(KmsConnectorInterface interf) {
|
|
return simKmsConnectorCore_impl(interf);
|
|
}
|
|
void forceLinkSimKmsConnectorTests() {}
|
|
|
|
namespace {
|
|
|
|
ACTOR Future<Void> testRunWorkload(KmsConnectorInterface inf, uint32_t nEncryptionKeys) {
|
|
state uint32_t maxEncryptionKeys = nEncryptionKeys;
|
|
state int maxDomainIds = deterministicRandom()->randomInt(121, 295);
|
|
state int maxIterations = deterministicRandom()->randomInt(786, 1786);
|
|
state std::unordered_map<EncryptCipherDomainId, std::unique_ptr<SimEncryptKeyCtx>> domainIdKeyMap;
|
|
state int i = 0;
|
|
|
|
TraceEvent("RunWorkloadStart").detail("MaxDomainIds", maxDomainIds).detail("MaxIterations", maxIterations);
|
|
|
|
{
|
|
// construct domainId to EncryptKeyCtx map
|
|
KmsConnLookupEKsByDomainIdsReq domainIdsReq;
|
|
for (i = 0; i < maxDomainIds; i++) {
|
|
// domainIdsReq.encryptDomainIds.push_back(i);
|
|
EncryptCipherDomainId domainId = i;
|
|
EncryptCipherDomainName domainName = StringRef(domainIdsReq.arena, std::to_string(domainId));
|
|
domainIdsReq.encryptDomainInfos.emplace_back(domainIdsReq.arena, i, domainName);
|
|
}
|
|
KmsConnLookupEKsByDomainIdsRep domainIdsRep = wait(inf.ekLookupByDomainIds.getReply(domainIdsReq));
|
|
for (auto& element : domainIdsRep.cipherKeyDetails) {
|
|
domainIdKeyMap.emplace(
|
|
element.encryptDomainId,
|
|
std::make_unique<SimEncryptKeyCtx>(element.encryptKeyId, element.encryptKey.toString().c_str()));
|
|
}
|
|
|
|
// randomly pick any domainId and validate if lookupByKeyId result matches
|
|
state std::unordered_map<EncryptCipherBaseKeyId, StringRef> validationMap;
|
|
std::unordered_map<EncryptCipherBaseKeyId, EncryptCipherDomainId> idsToLookup;
|
|
for (i = 0; i < maxIterations; i++) {
|
|
state int idx = deterministicRandom()->randomInt(0, maxDomainIds);
|
|
state SimEncryptKeyCtx* ctx = domainIdKeyMap[idx].get();
|
|
validationMap[ctx->id] = StringRef(ctx->key);
|
|
idsToLookup.emplace(ctx->id, idx);
|
|
}
|
|
|
|
state KmsConnLookupEKsByKeyIdsReq keyIdsReq;
|
|
for (const auto& item : idsToLookup) {
|
|
keyIdsReq.encryptKeyInfos.emplace_back_deep(
|
|
keyIdsReq.arena, item.second, item.first, StringRef(std::to_string(item.second)));
|
|
}
|
|
state KmsConnLookupEKsByKeyIdsRep keyIdsReply = wait(inf.ekLookupByIds.getReply(keyIdsReq));
|
|
/* TraceEvent("Lookup")
|
|
.detail("KeyIdReqSize", keyIdsReq.encryptKeyIds.size())
|
|
.detail("KeyIdsRepSz", keyIdsReply.encryptKeyDetails.size())
|
|
.detail("ValSz", validationMap.size()); */
|
|
ASSERT(keyIdsReply.cipherKeyDetails.size() == validationMap.size());
|
|
for (const auto& element : keyIdsReply.cipherKeyDetails) {
|
|
ASSERT(validationMap[element.encryptKeyId].compare(element.encryptKey) == 0);
|
|
}
|
|
}
|
|
|
|
{
|
|
// Verify unknown key access returns the error
|
|
state KmsConnLookupEKsByKeyIdsReq req;
|
|
req.encryptKeyInfos.emplace_back_deep(
|
|
req.arena, 1, maxEncryptionKeys + 1, StringRef(req.arena, std::to_string(maxEncryptionKeys)));
|
|
try {
|
|
KmsConnLookupEKsByKeyIdsRep reply = wait(inf.ekLookupByIds.getReply(req));
|
|
} catch (Error& e) {
|
|
ASSERT(e.code() == error_code_encrypt_key_not_found);
|
|
}
|
|
}
|
|
|
|
TraceEvent("RunWorkloadDone").log();
|
|
return Void();
|
|
}
|
|
|
|
} // namespace
|
|
|
|
TEST_CASE("fdbserver/SimKmsConnector") {
|
|
state KmsConnectorInterface inf;
|
|
state uint32_t maxEncryptKeys = 64;
|
|
state SimKmsConnector connector;
|
|
|
|
loop choose {
|
|
when(wait(connector.connectorCore(inf))) { throw internal_error(); }
|
|
when(wait(testRunWorkload(inf, maxEncryptKeys))) { break; }
|
|
}
|
|
return Void();
|
|
} |