Support encrypting TxnStateStore (#7253)
Adding encryption support for TxnStateStore. It is done by supporting encryption. for KeyValueStoreMemory. The encryption is currently done on operation level when the operations are being write to the underlying log file. See inline comment for the encrypted data format. This PR depends on #7252. It is part of the effort to support TLog encryption #6942.
This commit is contained in:
parent
21ee76a44d
commit
6246664006
|
@ -1048,8 +1048,14 @@ ACTOR Future<Void> readTransactionSystemState(Reference<ClusterRecoveryData> sel
|
|||
if (self->txnStateStore)
|
||||
self->txnStateStore->close();
|
||||
self->txnStateLogAdapter = openDiskQueueAdapter(oldLogSystem, myLocality, txsPoppedVersion);
|
||||
self->txnStateStore =
|
||||
keyValueStoreLogSystem(self->txnStateLogAdapter, self->dbgid, self->memoryLimit, false, false, true);
|
||||
self->txnStateStore = keyValueStoreLogSystem(self->txnStateLogAdapter,
|
||||
self->dbInfo,
|
||||
self->dbgid,
|
||||
self->memoryLimit,
|
||||
false,
|
||||
false,
|
||||
true,
|
||||
SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION);
|
||||
|
||||
// Version 0 occurs at the version epoch. The version epoch is the number
|
||||
// of microseconds since the Unix epoch. It can be set through fdbcli.
|
||||
|
|
|
@ -2421,7 +2421,8 @@ ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
|
|||
commitData.logSystem = ILogSystem::fromServerDBInfo(proxy.id(), commitData.db->get(), false, addActor);
|
||||
commitData.logAdapter =
|
||||
new LogSystemDiskQueueAdapter(commitData.logSystem, Reference<AsyncVar<PeekTxsInfo>>(), 1, false);
|
||||
commitData.txnStateStore = keyValueStoreLogSystem(commitData.logAdapter, proxy.id(), 2e9, true, true, true);
|
||||
commitData.txnStateStore = keyValueStoreLogSystem(
|
||||
commitData.logAdapter, commitData.db, proxy.id(), 2e9, true, true, true, SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION);
|
||||
createWhitelistBinPathVec(whitelistBinPaths, commitData.whitelistedBinPathVec);
|
||||
|
||||
commitData.updateLatencyBandConfig(commitData.db->get().latencyBandConfig);
|
||||
|
|
|
@ -57,7 +57,7 @@ ACTOR Future<EKPGetLatestBaseCipherKeysReply> getUncachedLatestEncryptCipherKeys
|
|||
try {
|
||||
EKPGetLatestBaseCipherKeysReply reply = wait(proxy.get().getLatestBaseCipherKeys.getReply(request));
|
||||
if (reply.error.present()) {
|
||||
TraceEvent("GetLatestCipherKeys_RequestFailed").error(reply.error.get());
|
||||
TraceEvent(SevWarn, "GetLatestCipherKeys_RequestFailed").error(reply.error.get());
|
||||
throw encrypt_keys_fetch_failed();
|
||||
}
|
||||
return reply;
|
||||
|
|
|
@ -24,6 +24,7 @@
|
|||
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include "fdbserver/ServerDBInfo.h"
|
||||
#include "fdbclient/StorageCheckpoint.h"
|
||||
|
||||
struct CheckpointRequest {
|
||||
|
@ -162,11 +163,13 @@ extern IKeyValueStore* keyValueStoreMemory(std::string const& basename,
|
|||
std::string ext = "fdq",
|
||||
KeyValueStoreType storeType = KeyValueStoreType::MEMORY);
|
||||
extern IKeyValueStore* keyValueStoreLogSystem(class IDiskQueue* queue,
|
||||
Reference<AsyncVar<ServerDBInfo> const> db,
|
||||
UID logID,
|
||||
int64_t memoryLimit,
|
||||
bool disableSnapshot,
|
||||
bool replaceContent,
|
||||
bool exactRecovery);
|
||||
bool exactRecovery,
|
||||
bool enableEncryption);
|
||||
|
||||
extern IKeyValueStore* openRemoteKVStore(KeyValueStoreType storeType,
|
||||
std::string const& filename,
|
||||
|
|
|
@ -22,11 +22,13 @@
|
|||
#include "fdbclient/Notified.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbserver/DeltaTree.h"
|
||||
#include "fdbserver/GetEncryptCipherKeys.h"
|
||||
#include "fdbserver/IDiskQueue.h"
|
||||
#include "fdbserver/IKeyValueContainer.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
#include "fdbserver/RadixTree.h"
|
||||
#include "flow/ActorCollection.h"
|
||||
#include "flow/Knobs.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
#define OP_DISK_OVERHEAD (sizeof(OpHeader) + 1)
|
||||
|
@ -35,12 +37,14 @@ template <typename Container>
|
|||
class KeyValueStoreMemory final : public IKeyValueStore, NonCopyable {
|
||||
public:
|
||||
KeyValueStoreMemory(IDiskQueue* log,
|
||||
Reference<AsyncVar<ServerDBInfo> const> db,
|
||||
UID id,
|
||||
int64_t memoryLimit,
|
||||
KeyValueStoreType storeType,
|
||||
bool disableSnapshot,
|
||||
bool replaceContent,
|
||||
bool exactRecovery);
|
||||
bool exactRecovery,
|
||||
bool enableEncryption);
|
||||
|
||||
// IClosable
|
||||
Future<Void> getError() const override { return log->getError(); }
|
||||
|
@ -292,7 +296,8 @@ private:
|
|||
OpSnapshotAbort, // terminate an in progress snapshot in order to start a full snapshot
|
||||
OpCommit, // only in log, not in queue
|
||||
OpRollback, // only in log, not in queue
|
||||
OpSnapshotItemDelta
|
||||
OpSnapshotItemDelta,
|
||||
OpEncrypted
|
||||
};
|
||||
|
||||
struct OpRef {
|
||||
|
@ -365,6 +370,7 @@ private:
|
|||
|
||||
OpQueue queue; // mutations not yet commit()ted
|
||||
IDiskQueue* log;
|
||||
Reference<AsyncVar<ServerDBInfo> const> db;
|
||||
Future<Void> recovering, snapshotting;
|
||||
int64_t committedWriteBytes;
|
||||
int64_t overheadWriteBytes;
|
||||
|
@ -391,6 +397,10 @@ private:
|
|||
int64_t memoryLimit; // The upper limit on the memory used by the store (excluding, possibly, some clear operations)
|
||||
std::vector<std::pair<KeyValueMapPair, uint64_t>> dataSets;
|
||||
|
||||
bool enableEncryption;
|
||||
TextAndHeaderCipherKeys cipherKeys;
|
||||
Future<Void> refreshCipherKeysActor;
|
||||
|
||||
int64_t commit_queue(OpQueue& ops, bool log, bool sequential = false) {
|
||||
int64_t total = 0, count = 0;
|
||||
IDiskQueue::location log_location = 0;
|
||||
|
@ -441,14 +451,85 @@ private:
|
|||
return total;
|
||||
}
|
||||
|
||||
// Data format for normal operation:
|
||||
// +-------------+-------------+-------------+--------+--------+
|
||||
// | opType | len1 | len2 | param2 | param2 |
|
||||
// | sizeof(int) | sizeof(int) | sizeof(int) | len1 | len2 |
|
||||
// +-------------+-------------+-------------+--------+--------+
|
||||
//
|
||||
// However, if the operation is encrypted:
|
||||
// +-------------+-------------+-------------+---------------------------------+-------------+--------+--------+
|
||||
// | OpEncrypted | len1 | len2 | BlobCipherEncryptHeader | opType | param1 | param2 |
|
||||
// | sizeof(int) | sizeof(int) | sizeof(int) | sizeof(BlobCipherEncryptHeader) | sizeof(int) | len1 | len2 |
|
||||
// +-------------+-------------+-------------+---------------------------------+-------------+--------+--------+
|
||||
// | plaintext | encrypted |
|
||||
// +-----------------------------------------------------------------------------------------------------------+
|
||||
//
|
||||
IDiskQueue::location log_op(OpType op, StringRef v1, StringRef v2) {
|
||||
OpHeader h = { (int)op, v1.size(), v2.size() };
|
||||
log->push(StringRef((const uint8_t*)&h, sizeof(h)));
|
||||
log->push(v1);
|
||||
log->push(v2);
|
||||
// Metadata op types to be excluded from encryption.
|
||||
static std::unordered_set<OpType> metaOps = { OpSnapshotEnd, OpSnapshotAbort, OpCommit, OpRollback };
|
||||
if (!enableEncryption || metaOps.count(op) > 0) {
|
||||
OpHeader h = { (int)op, v1.size(), v2.size() };
|
||||
log->push(StringRef((const uint8_t*)&h, sizeof(h)));
|
||||
log->push(v1);
|
||||
log->push(v2);
|
||||
} else {
|
||||
OpHeader h = { (int)OpEncrypted, v1.size(), v2.size() };
|
||||
log->push(StringRef((const uint8_t*)&h, sizeof(h)));
|
||||
|
||||
uint8_t* plaintext = new uint8_t[sizeof(int) + v1.size() + v2.size()];
|
||||
*(int*)plaintext = op;
|
||||
memcpy(plaintext + sizeof(int), v1.begin(), v1.size());
|
||||
memcpy(plaintext + sizeof(int) + v1.size(), v2.begin(), v2.size());
|
||||
|
||||
ASSERT(cipherKeys.cipherTextKey.isValid());
|
||||
ASSERT(cipherKeys.cipherHeaderKey.isValid());
|
||||
EncryptBlobCipherAes265Ctr cipher(
|
||||
cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE);
|
||||
BlobCipherEncryptHeader cipherHeader;
|
||||
Arena arena;
|
||||
StringRef ciphertext =
|
||||
cipher.encrypt(plaintext, sizeof(int) + v1.size() + v2.size(), &cipherHeader, arena)->toStringRef();
|
||||
log->push(StringRef((const uint8_t*)&cipherHeader, BlobCipherEncryptHeader::headerSize));
|
||||
log->push(ciphertext);
|
||||
}
|
||||
return log->push(LiteralStringRef("\x01")); // Changes here should be reflected in OP_DISK_OVERHEAD
|
||||
}
|
||||
|
||||
// In case the op data is not encrypted, simply read the operands and the zero fill flag.
|
||||
// Otherwise, decrypt the op type and data.
|
||||
ACTOR static Future<Standalone<StringRef>> readOpData(KeyValueStoreMemory* self,
|
||||
OpHeader* h,
|
||||
bool* isZeroFilled,
|
||||
int* zeroFillSize) {
|
||||
state int remainingBytes = h->len1 + h->len2 + 1;
|
||||
if (h->op == OpEncrypted) {
|
||||
// encryption header, plus the real (encrypted) op type
|
||||
remainingBytes += BlobCipherEncryptHeader::headerSize + sizeof(int);
|
||||
}
|
||||
state Standalone<StringRef> data = wait(self->log->readNext(remainingBytes));
|
||||
ASSERT(data.size() <= remainingBytes);
|
||||
*zeroFillSize = remainingBytes - data.size();
|
||||
if (*zeroFillSize == 0) {
|
||||
*isZeroFilled = (data[data.size() - 1] == 0);
|
||||
}
|
||||
if (h->op != OpEncrypted || *zeroFillSize > 0 || *isZeroFilled) {
|
||||
return data;
|
||||
}
|
||||
state BlobCipherEncryptHeader cipherHeader = *(BlobCipherEncryptHeader*)data.begin();
|
||||
TextAndHeaderCipherKeys cipherKeys = wait(getEncryptCipherKeys(self->db, cipherHeader));
|
||||
DecryptBlobCipherAes256Ctr cipher(cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, cipherHeader.iv);
|
||||
Arena arena;
|
||||
StringRef plaintext = cipher
|
||||
.decrypt(data.begin() + BlobCipherEncryptHeader::headerSize,
|
||||
sizeof(int) + h->len1 + h->len2,
|
||||
cipherHeader,
|
||||
arena)
|
||||
->toStringRef();
|
||||
h->op = *(int*)plaintext.begin();
|
||||
return Standalone<StringRef>(plaintext.substr(sizeof(int)), arena);
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> recover(KeyValueStoreMemory* self, bool exactRecovery) {
|
||||
loop {
|
||||
// 'uncommitted' variables track something that might be rolled back by an OpRollback, and are copied into
|
||||
|
@ -473,6 +554,7 @@ private:
|
|||
state OpQueue recoveryQueue;
|
||||
state OpHeader h;
|
||||
state Standalone<StringRef> lastSnapshotKey;
|
||||
state bool isZeroFilled;
|
||||
|
||||
TraceEvent("KVSMemRecoveryStarted", self->id).detail("SnapshotEndLocation", uncommittedSnapshotEnd);
|
||||
|
||||
|
@ -486,6 +568,10 @@ private:
|
|||
memset(&h, 0, sizeof(OpHeader));
|
||||
memcpy(&h, data.begin(), data.size());
|
||||
zeroFillSize = sizeof(OpHeader) - data.size() + h.len1 + h.len2 + 1;
|
||||
if (h.op == OpEncrypted) {
|
||||
// encryption header, plus the real (encrypted) op type
|
||||
zeroFillSize += BlobCipherEncryptHeader::headerSize + sizeof(int);
|
||||
}
|
||||
}
|
||||
TraceEvent("KVSMemRecoveryComplete", self->id)
|
||||
.detail("Reason", "Non-header sized data read")
|
||||
|
@ -497,9 +583,8 @@ private:
|
|||
}
|
||||
h = *(OpHeader*)data.begin();
|
||||
}
|
||||
Standalone<StringRef> data = wait(self->log->readNext(h.len1 + h.len2 + 1));
|
||||
if (data.size() != h.len1 + h.len2 + 1) {
|
||||
zeroFillSize = h.len1 + h.len2 + 1 - data.size();
|
||||
state Standalone<StringRef> data = wait(readOpData(self, &h, &isZeroFilled, &zeroFillSize));
|
||||
if (zeroFillSize > 0) {
|
||||
TraceEvent("KVSMemRecoveryComplete", self->id)
|
||||
.detail("Reason", "data specified by header does not exist")
|
||||
.detail("DataSize", data.size())
|
||||
|
@ -510,7 +595,7 @@ private:
|
|||
break;
|
||||
}
|
||||
|
||||
if (data[data.size() - 1]) {
|
||||
if (!isZeroFilled) {
|
||||
StringRef p1 = data.substr(0, h.len1);
|
||||
StringRef p2 = data.substr(h.len1, h.len2);
|
||||
|
||||
|
@ -630,6 +715,12 @@ private:
|
|||
.detail("TimeTaken", now() - startt);
|
||||
|
||||
self->semiCommit();
|
||||
|
||||
// Make sure cipher keys are ready before recovery finishes.
|
||||
if (self->enableEncryption) {
|
||||
wait(updateCipherKeys(self));
|
||||
}
|
||||
|
||||
return Void();
|
||||
} catch (Error& e) {
|
||||
bool ok = e.code() == error_code_operation_cancelled || e.code() == error_code_file_not_found ||
|
||||
|
@ -852,20 +943,36 @@ private:
|
|||
self->log->pop(location);
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> updateCipherKeys(KeyValueStoreMemory* self) {
|
||||
TextAndHeaderCipherKeys cipherKeys = wait(getLatestSystemEncryptCipherKeys(self->db));
|
||||
self->cipherKeys = cipherKeys;
|
||||
return Void();
|
||||
}
|
||||
|
||||
// TODO(yiwu): Implement background refresh mechanism for BlobCipher and use that mechanism to refresh cipher key.
|
||||
ACTOR static Future<Void> refreshCipherKeys(KeyValueStoreMemory* self) {
|
||||
loop {
|
||||
wait(updateCipherKeys(self));
|
||||
wait(delay(FLOW_KNOBS->ENCRYPT_KEY_REFRESH_INTERVAL));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Container>
|
||||
KeyValueStoreMemory<Container>::KeyValueStoreMemory(IDiskQueue* log,
|
||||
Reference<AsyncVar<ServerDBInfo> const> db,
|
||||
UID id,
|
||||
int64_t memoryLimit,
|
||||
KeyValueStoreType storeType,
|
||||
bool disableSnapshot,
|
||||
bool replaceContent,
|
||||
bool exactRecovery)
|
||||
: type(storeType), id(id), log(log), committedWriteBytes(0), overheadWriteBytes(0), currentSnapshotEnd(-1),
|
||||
bool exactRecovery,
|
||||
bool enableEncryption)
|
||||
: type(storeType), id(id), log(log), db(db), committedWriteBytes(0), overheadWriteBytes(0), currentSnapshotEnd(-1),
|
||||
previousSnapshotEnd(-1), committedDataSize(0), transactionSize(0), transactionIsLarge(false), resetSnapshot(false),
|
||||
disableSnapshot(disableSnapshot), replaceContent(replaceContent), firstCommitWithSnapshot(true), snapshotCount(0),
|
||||
memoryLimit(memoryLimit) {
|
||||
memoryLimit(memoryLimit), enableEncryption(enableEncryption) {
|
||||
// create reserved buffer for radixtree store type
|
||||
this->reserved_buffer =
|
||||
(storeType == KeyValueStoreType::MEMORY) ? nullptr : new uint8_t[CLIENT_KNOBS->SYSTEM_KEY_SIZE_LIMIT];
|
||||
|
@ -875,6 +982,9 @@ KeyValueStoreMemory<Container>::KeyValueStoreMemory(IDiskQueue* log,
|
|||
recovering = recover(this, exactRecovery);
|
||||
snapshotting = snapshot(this);
|
||||
commitActors = actorCollection(addActor.getFuture());
|
||||
if (enableEncryption) {
|
||||
refreshCipherKeysActor = refreshCipherKeys(this);
|
||||
}
|
||||
}
|
||||
|
||||
IKeyValueStore* keyValueStoreMemory(std::string const& basename,
|
||||
|
@ -890,18 +1000,31 @@ IKeyValueStore* keyValueStoreMemory(std::string const& basename,
|
|||
// SOMEDAY: update to use DiskQueueVersion::V2 with xxhash3 checksum for FDB >= 7.2
|
||||
IDiskQueue* log = openDiskQueue(basename, ext, logID, DiskQueueVersion::V1);
|
||||
if (storeType == KeyValueStoreType::MEMORY_RADIXTREE) {
|
||||
return new KeyValueStoreMemory<radix_tree>(log, logID, memoryLimit, storeType, false, false, false);
|
||||
return new KeyValueStoreMemory<radix_tree>(
|
||||
log, Reference<AsyncVar<ServerDBInfo> const>(), logID, memoryLimit, storeType, false, false, false, false);
|
||||
} else {
|
||||
return new KeyValueStoreMemory<IKeyValueContainer>(log, logID, memoryLimit, storeType, false, false, false);
|
||||
return new KeyValueStoreMemory<IKeyValueContainer>(
|
||||
log, Reference<AsyncVar<ServerDBInfo> const>(), logID, memoryLimit, storeType, false, false, false, false);
|
||||
}
|
||||
}
|
||||
|
||||
IKeyValueStore* keyValueStoreLogSystem(class IDiskQueue* queue,
|
||||
Reference<AsyncVar<ServerDBInfo> const> db,
|
||||
UID logID,
|
||||
int64_t memoryLimit,
|
||||
bool disableSnapshot,
|
||||
bool replaceContent,
|
||||
bool exactRecovery) {
|
||||
return new KeyValueStoreMemory<IKeyValueContainer>(
|
||||
queue, logID, memoryLimit, KeyValueStoreType::MEMORY, disableSnapshot, replaceContent, exactRecovery);
|
||||
bool exactRecovery,
|
||||
bool enableEncryption) {
|
||||
// ServerDBInfo is required if encryption is to be enabled, or the KV store instance have been encrypted.
|
||||
ASSERT(!enableEncryption || db.isValid());
|
||||
return new KeyValueStoreMemory<IKeyValueContainer>(queue,
|
||||
db,
|
||||
logID,
|
||||
memoryLimit,
|
||||
KeyValueStoreType::MEMORY,
|
||||
disableSnapshot,
|
||||
replaceContent,
|
||||
exactRecovery,
|
||||
enableEncryption);
|
||||
}
|
||||
|
|
|
@ -655,7 +655,8 @@ ACTOR Future<Void> resolverCore(ResolverInterface resolver,
|
|||
state TransactionStateResolveContext transactionStateResolveContext;
|
||||
if (SERVER_KNOBS->PROXY_USE_RESOLVER_PRIVATE_MUTATIONS) {
|
||||
self->logAdapter = new LogSystemDiskQueueAdapter(self->logSystem, Reference<AsyncVar<PeekTxsInfo>>(), 1, false);
|
||||
self->txnStateStore = keyValueStoreLogSystem(self->logAdapter, resolver.id(), 2e9, true, true, true);
|
||||
self->txnStateStore = keyValueStoreLogSystem(
|
||||
self->logAdapter, db, resolver.id(), 2e9, true, true, true, SERVER_KNOBS->ENABLE_TLOG_ENCRYPTION);
|
||||
|
||||
// wait for txnStateStore recovery
|
||||
wait(success(self->txnStateStore->readValue(StringRef())));
|
||||
|
|
Loading…
Reference in New Issue