diff --git a/kvbc/CMakeLists.txt b/kvbc/CMakeLists.txt index 0d0a9a5188..3f5bf8c893 100644 --- a/kvbc/CMakeLists.txt +++ b/kvbc/CMakeLists.txt @@ -1,6 +1,7 @@ cmake_minimum_required (VERSION 3.2) project(libkvbc VERSION 0.1.0.0 LANGUAGES CXX) +find_package(Boost ${MIN_BOOST_VERSION} COMPONENTS filesystem REQUIRED) add_library(concord_block_update INTERFACE) target_include_directories(concord_block_update INTERFACE @@ -38,7 +39,8 @@ if (BUILD_ROCKSDB_STORAGE) src/categorization/kv_blockchain.cpp src/categorization/blocks.cpp src/categorization/blockchain.cpp - src/categorization/block_merkle_category.cpp) + src/categorization/block_merkle_category.cpp + src/migrations/block_merkle_latest_ver_cf_migration.cpp) endif (BUILD_ROCKSDB_STORAGE) target_link_libraries(kvbc PUBLIC corebft util) @@ -54,6 +56,7 @@ if(NOT BUILD_THIRDPARTY) find_package(OpenSSL REQUIRED) endif() target_link_libraries(kvbc PRIVATE OpenSSL::Crypto) +target_link_libraries(kvbc PRIVATE ${Boost_LIBRARIES}) if (BUILD_TESTING) add_subdirectory(test) diff --git a/kvbc/include/categorization/block_merkle_category.h b/kvbc/include/categorization/block_merkle_category.h index 37f6d979f3..cbaa8aaa79 100644 --- a/kvbc/include/categorization/block_merkle_category.h +++ b/kvbc/include/categorization/block_merkle_category.h @@ -51,7 +51,6 @@ class BlockMerkleCategory { // Returns the latest *block* version of a key. // Returns std::nullopt if the key doesn't exist. std::optional getLatestVersion(const std::string& key) const; - std::optional getLatestVersion(const Hash& key) const; // Get values for keys at specific versions. // `keys` and `versions` must be the same size. @@ -68,7 +67,6 @@ class BlockMerkleCategory { // If a key is missing, std::nullopt is returned for its version. void multiGetLatestVersion(const std::vector& keys, std::vector>& versions) const; - void multiGetLatestVersion(const std::vector& keys, std::vector>& versions) const; std::vector getBlockStaleKeys(BlockId, const BlockMerkleOutput&) const; // Delete the given block ID as a genesis one. @@ -172,8 +170,9 @@ class BlockMerkleCategory { // 3. Atomically write the batch to the database. void deleteStaleBatch(uint64_t start, uint64_t end); - // Retrieve the latest versions for all raw keys in a block and return them along with the hashed keys. - std::pair, std::vector>> getLatestVersions( + // Retrieve the latest versions for all raw keys in a block and return them along with the keys and the hashed keys. + // Returned tuple contains (list_of_key_hashes, list_of_keys, list_of_versions). + std::tuple, std::vector, std::vector>> getLatestVersions( const BlockMerkleOutput& out) const; // Return a map from block id to all hashed keys that were still active in previously pruned blocks. diff --git a/kvbc/include/migrations/block_merkle_latest_ver_cf_migration.h b/kvbc/include/migrations/block_merkle_latest_ver_cf_migration.h new file mode 100644 index 0000000000..cdb040b56f --- /dev/null +++ b/kvbc/include/migrations/block_merkle_latest_ver_cf_migration.h @@ -0,0 +1,77 @@ +// Concord +// +// Copyright (c) 2021 VMware, Inc. All Rights Reserved. +// +// This product is licensed to you under the Apache 2.0 license (the "License"). +// You may not use this product except in compliance with the Apache 2.0 License. +// +// This product may include a number of subcomponents with separate copyright +// notices and license terms. Your use of these subcomponents is subject to the +// terms and conditions of the sub-component's license, as noted in the +// LICENSE file. + +#pragma once + +#include "rocksdb/native_client.h" + +#include + +#include +#include +#include +#include +#include + +namespace concord::kvbc::migrations { + +// Migrates a RocksDB DB from using key hashes in the `block_merkle_latest_key_version` to using raw keys. +// Keeps track of what steps have been executed such that if the migration process crashes, it can start fresh or +// continue from where it left off. +class BlockMerkleLatestVerCfMigration { + public: + // Note: `export_path` must be on the same filesystem as `db_path`. + BlockMerkleLatestVerCfMigration(const std::string& db_path, const std::string& export_path); + + public: + static const std::string& temporaryColumnFamily(); + static const std::string& migrationKey(); + + // Migration states. + static inline const std::string kStateImportedTempCf{"imported-temp-cf"}; + static inline const std::string kStateMigrated{"migrated"}; + static inline const std::string kStateMigrationNotNeededOrCompleted{"migration-not-needed-or-completed"}; + + enum class ExecutionStatus { + kExecuted, // executed as part of this call + kNotNeededOrAlreadyExecuted, // migration is not needed or already executed and, therefore, nothing done in this + // call + }; + + public: + // Executes the migration, throwing on error. + // If execute() returns, it is always a success. The ExecutionStatus gives indication as to what the actual outcome + // is. + ExecutionStatus execute(); + + std::shared_ptr db() { return db_; } + + // Following methods are used for testing purposes only. Do not use in production. + public: + void removeExportDir(); + void checkpointDB(); + void exportLatestVerCf(); + void importTempLatestVerCf(); + void clearExistingLatestVerCf(); + void iterateAndMigrate(); + void dropTempLatestVerCf(); + void commitComplete(); + + private: + const std::string db_path_; + const std::string export_path_; + std::shared_ptr db_; + std::unique_ptr<::rocksdb::Checkpoint> checkpoint_; + std::unique_ptr<::rocksdb::ExportImportFilesMetaData> export_metadata_; +}; + +} // namespace concord::kvbc::migrations diff --git a/kvbc/src/categorization/block_merkle_category.cpp b/kvbc/src/categorization/block_merkle_category.cpp index cab4021ccb..8d59667a49 100644 --- a/kvbc/src/categorization/block_merkle_category.cpp +++ b/kvbc/src/categorization/block_merkle_category.cpp @@ -158,8 +158,8 @@ std::vector versionedKeys(const std::vector& keys, const st return versioned_keys; } -void putLatestKeyVersion(NativeWriteBatch& batch, const Hash& key_hash, TaggedVersion version) { - batch.put(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, key_hash, serializeThreadLocal(LatestKeyVersion{version.encode()})); +void putLatestKeyVersion(NativeWriteBatch& batch, const std::string& key, TaggedVersion version) { + batch.put(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, key, serializeThreadLocal(LatestKeyVersion{version.encode()})); } void putKeys(NativeWriteBatch& batch, @@ -167,7 +167,7 @@ void putKeys(NativeWriteBatch& batch, std::vector&& hashed_added_keys, std::vector&& hashed_deleted_keys, BlockMerkleInput& updates) { - auto kv_it = updates.kv.begin(); + auto kv_it = updates.kv.cbegin(); for (auto key_it = hashed_added_keys.begin(); key_it != hashed_added_keys.end(); key_it++) { // Only serialize the Header of a DBValue, to prevent the need to copy a potentially large value. auto header = toSlice(serializeThreadLocal(DbValueHeader{false, static_cast(kv_it->second.size())})); @@ -177,17 +177,19 @@ void putKeys(NativeWriteBatch& batch, batch.put(BLOCK_MERKLE_KEYS_CF, serialize(VersionedKey{*key_it, block_id}), val); // Put the latest version of the key - batch.put(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, key_it->value, serialize(LatestKeyVersion{block_id})); + batch.put(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, kv_it->first, serialize(LatestKeyVersion{block_id})); - kv_it++; + ++kv_it; } const bool deleted = true; + auto deletes_it = updates.deletes.cbegin(); for (auto key_it = hashed_deleted_keys.begin(); key_it != hashed_deleted_keys.end(); key_it++) { // Write a tombstone to the value. This is necessary for deleteLastReachable(). auto tombstone = serialize(DbValueHeader{true, 0}); batch.put(BLOCK_MERKLE_KEYS_CF, serialize(VersionedKey{*key_it, block_id}), tombstone); - putLatestKeyVersion(batch, key_it->value, TaggedVersion(deleted, block_id)); + putLatestKeyVersion(batch, *deletes_it, TaggedVersion(deleted, block_id)); + ++deletes_it; } } @@ -253,6 +255,7 @@ void removeMerkleNodes(NativeWriteBatch& batch, BlockId block_id, uint64_t tree_ // Return any active key hashes. std::vector deleteInactiveKeys(BlockId block_id, std::vector&& hashed_keys, + std::vector&& keys, const std::vector>& latest_versions, NativeWriteBatch& batch, size_t& deletes_counter) { @@ -260,6 +263,7 @@ std::vector deleteInactiveKeys(BlockId block_id, for (auto i = 0u; i < hashed_keys.size(); i++) { auto& tagged_version = latest_versions[i]; auto& hashed_key = hashed_keys[i]; + auto& key = keys[i]; ConcordAssert(tagged_version.has_value()); ConcordAssertLE(block_id, tagged_version->version); @@ -268,7 +272,7 @@ std::vector deleteInactiveKeys(BlockId block_id, // The latest version is a tombstone. We can delete the key and version. auto versioned_key = serialize(VersionedKey{KeyHash{hashed_key}, block_id}); batch.del(BLOCK_MERKLE_KEYS_CF, versioned_key); - batch.del(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, hashed_key); + batch.del(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, key); deletes_counter++; } else { active_keys.push_back(KeyHash{hashed_key}); @@ -356,21 +360,16 @@ std::optional BlockMerkleCategory::get(const Hash& hashed_key, BlockId bl } std::optional BlockMerkleCategory::getLatest(const std::string& key) const { - auto hashed_key = hash(key); - if (auto latest = getLatestVersion(hashed_key)) { + if (auto latest = getLatestVersion(key)) { if (!latest->deleted) { - return get(hashed_key, latest->version); + return get(hash(key), latest->version); } } return std::nullopt; } std::optional BlockMerkleCategory::getLatestVersion(const std::string& key) const { - return getLatestVersion(hash(key)); -} - -std::optional BlockMerkleCategory::getLatestVersion(const Hash& hashed_key) const { - const auto serialized = db_->getSlice(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, hashed_key); + const auto serialized = db_->getSlice(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, key); if (!serialized) { return std::nullopt; } @@ -418,16 +417,10 @@ void BlockMerkleCategory::multiGet(const std::vector& versioned_keys, void BlockMerkleCategory::multiGetLatestVersion(const std::vector& keys, std::vector>& versions) const { - auto hashed_keys = hashedKeys(keys); - multiGetLatestVersion(hashed_keys, versions); -} - -void BlockMerkleCategory::multiGetLatestVersion(const std::vector& hashed_keys, - std::vector>& versions) const { auto slices = std::vector<::rocksdb::PinnableSlice>{}; auto statuses = std::vector<::rocksdb::Status>{}; - db_->multiGet(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, hashed_keys, slices, statuses); + db_->multiGet(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, keys, slices, statuses); versions.clear(); for (auto i = 0ull; i < slices.size(); ++i) { const auto& status = statuses[i]; @@ -448,7 +441,7 @@ void BlockMerkleCategory::multiGetLatest(const std::vector& keys, std::vector>& values) const { auto hashed_keys = hashedKeys(keys); std::vector> versions; - multiGetLatestVersion(hashed_keys, versions); + multiGetLatestVersion(keys, versions); // Generate the set of versioned keys for all keys that have latest versions and are not deleted auto versioned_keys = std::vector{}; @@ -546,7 +539,8 @@ std::pair BlockMerkleCategory::rewriteAlreadyPru std::vector BlockMerkleCategory::getBlockStaleKeys(BlockId block_id, const BlockMerkleOutput& out) const { std::vector hash_stale_keys; - auto [hashed_keys, latest_versions] = getLatestVersions(out); + auto [hashed_keys, _, latest_versions] = getLatestVersions(out); + (void)_; for (auto i = 0u; i < hashed_keys.size(); i++) { auto& tagged_version = latest_versions[i]; auto& hashed_key = hashed_keys[i]; @@ -576,14 +570,15 @@ std::vector BlockMerkleCategory::getBlockStaleKeys(BlockId block_id size_t BlockMerkleCategory::deleteGenesisBlock(BlockId block_id, const BlockMerkleOutput& out, NativeWriteBatch& batch) { - auto [hashed_keys, latest_versions] = getLatestVersions(out); + auto [hashed_keys, keys, latest_versions] = getLatestVersions(out); auto overwritten_active_keys_from_pruned_blocks = findActiveKeysFromPrunedBlocks(hashed_keys); size_t num_of_deletes = 0; for (auto& kv : overwritten_active_keys_from_pruned_blocks) { num_of_deletes += kv.second.size(); } auto [block_adds, block_removes] = rewriteAlreadyPrunedBlocks(overwritten_active_keys_from_pruned_blocks, batch); - auto active_keys = deleteInactiveKeys(block_id, std::move(hashed_keys), latest_versions, batch, num_of_deletes); + auto active_keys = + deleteInactiveKeys(block_id, std::move(hashed_keys), std::move(keys), latest_versions, batch, num_of_deletes); if (active_keys.empty()) { block_removes.push_back(merkleKey(block_id)); } else { @@ -616,14 +611,14 @@ void BlockMerkleCategory::deleteLastReachableBlock(BlockId block_id, // Preserve the deleted flag from the value into the version index. auto deleted = Deleted{}; deserialize(iter.valueView(), deleted); - putLatestKeyVersion(batch, hashed_key.value, TaggedVersion(deleted.value, prev_key.version)); + putLatestKeyVersion(batch, key, TaggedVersion(deleted.value, prev_key.version)); } else { // This is the only version of the key - remove the latest version index too. - batch.del(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, hashed_key.value); + batch.del(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, key); } } else { // No previous keys means this is the only version of the key - remove the latest version index too. - batch.del(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, hashed_key.value); + batch.del(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, key); } // Remove the value for the key at `block_id`. batch.del(BLOCK_MERKLE_KEYS_CF, versioned_key); @@ -631,19 +626,22 @@ void BlockMerkleCategory::deleteLastReachableBlock(BlockId block_id, removeMerkleNodes(batch, block_id, out.state_root_version); } -std::pair, std::vector>> BlockMerkleCategory::getLatestVersions( - const BlockMerkleOutput& out) const { +std::tuple, std::vector, std::vector>> +BlockMerkleCategory::getLatestVersions(const BlockMerkleOutput& out) const { std::vector hashed_keys; + std::vector keys; hashed_keys.reserve(out.keys.size()); + keys.reserve(out.keys.size()); for (auto& [key, _] : out.keys) { (void)_; hashed_keys.push_back(hash(key)); + keys.push_back(key); } std::vector> latest_versions; - multiGetLatestVersion(hashed_keys, latest_versions); + multiGetLatestVersion(keys, latest_versions); - return std::make_pair(hashed_keys, latest_versions); + return std::make_tuple(hashed_keys, keys, latest_versions); } void BlockMerkleCategory::putLastDeletedTreeVersion(uint64_t tree_version, NativeWriteBatch& batch) { diff --git a/kvbc/src/categorization/kv_blockchain.cpp b/kvbc/src/categorization/kv_blockchain.cpp index 6b46d136b2..3311354194 100644 --- a/kvbc/src/categorization/kv_blockchain.cpp +++ b/kvbc/src/categorization/kv_blockchain.cpp @@ -20,6 +20,7 @@ #include "kvbc_key_types.hpp" #include "categorization/db_categories.h" #include "endianness.hpp" +#include "migrations/block_merkle_latest_ver_cf_migration.h" #include @@ -71,6 +72,11 @@ KeyValueBlockchain::KeyValueBlockchain(const std::shared_ptrput(migrations::BlockMerkleLatestVerCfMigration::migrationKey(), + migrations::BlockMerkleLatestVerCfMigration::kStateMigrationNotNeededOrCompleted); } void KeyValueBlockchain::initNewBlockchainCategories( diff --git a/kvbc/src/migrations/block_merkle_latest_ver_cf_migration.cpp b/kvbc/src/migrations/block_merkle_latest_ver_cf_migration.cpp new file mode 100644 index 0000000000..b193141eaf --- /dev/null +++ b/kvbc/src/migrations/block_merkle_latest_ver_cf_migration.cpp @@ -0,0 +1,193 @@ +// Concord +// +// Copyright (c) 2021 VMware, Inc. All Rights Reserved. +// +// This product is licensed to you under the Apache 2.0 license (the "License"). +// You may not use this product except in compliance with the Apache 2.0 License. +// +// This product may include a number of subcomponents with separate copyright +// notices and license terms. Your use of these subcomponents is subject to the +// terms and conditions of the sub-component's license, as noted in the +// LICENSE file. + +#include "migrations/block_merkle_latest_ver_cf_migration.h" + +#include "assertUtils.hpp" +#include "categorization/blockchain.h" +#include "categorization/column_families.h" +#include "categorization/db_categories.h" +#include "categorization/details.h" +#include "kv_types.hpp" +#include "storage/db_types.h" +#include "string.hpp" + +#include +#include + +#include +#include + +namespace concord::kvbc::migrations { + +using namespace concord::storage::rocksdb; + +using kvbc::INITIAL_GENESIS_BLOCK_ID; +using kvbc::categorization::BlockMerkleOutput; +using kvbc::categorization::kExecutionProvableCategory; +using kvbc::categorization::detail::Blockchain; +using kvbc::categorization::detail::BLOCK_MERKLE_LATEST_KEY_VERSION_CF; +using kvbc::categorization::detail::hash; +using kvbc::categorization::Hash; +using util::toChar; + +using ::rocksdb::Checkpoint; +using ::rocksdb::ColumnFamilyDescriptor; +using ::rocksdb::ColumnFamilyOptions; +using ::rocksdb::ExportImportFilesMetaData; +using ::rocksdb::ImportColumnFamilyOptions; + +// 2 bytes: EDBKeyType::Migration followed by EMigrationSubType::BlockMerkleLatestVerCfState . +const auto kMigrationKey = + std::string{toChar(storage::v2MerkleTree::detail::EDBKeyType::Migration), + toChar(storage::v2MerkleTree::detail::EMigrationSubType::BlockMerkleLatestVerCfState)}; +const std::string& BlockMerkleLatestVerCfMigration::migrationKey() { return kMigrationKey; } + +const auto kTempCf = BLOCK_MERKLE_LATEST_KEY_VERSION_CF + "_temp"; +const std::string& BlockMerkleLatestVerCfMigration::temporaryColumnFamily() { return kTempCf; } + +BlockMerkleLatestVerCfMigration::BlockMerkleLatestVerCfMigration(const std::string& db_path, + const std::string& export_path) + : db_path_{db_path}, export_path_{export_path} { + const auto read_only = false; + db_ = NativeClient::newClient(db_path, read_only, NativeClient::DefaultOptions{}); +} + +void BlockMerkleLatestVerCfMigration::removeExportDir() { boost::filesystem::remove_all(export_path_); } + +void BlockMerkleLatestVerCfMigration::checkpointDB() { + Checkpoint* checkpoint{nullptr}; + if (!Checkpoint::Create(&db_->rawDB(), &checkpoint).ok()) { + throw std::runtime_error{"Failed to create a RocksDB checkpoint for DB path = " + db_path_}; + } + checkpoint_.reset(checkpoint); +} + +void BlockMerkleLatestVerCfMigration::exportLatestVerCf() { + ExportImportFilesMetaData* export_metadata{nullptr}; + const auto status = checkpoint_->ExportColumnFamily( + db_->columnFamilyHandle(BLOCK_MERKLE_LATEST_KEY_VERSION_CF), export_path_, &export_metadata); + if (!status.ok()) { + throw std::runtime_error{"Failed to export " + BLOCK_MERKLE_LATEST_KEY_VERSION_CF + + " column family, reason: " + status.ToString()}; + } + export_metadata_.reset(export_metadata); +} + +void BlockMerkleLatestVerCfMigration::importTempLatestVerCf() { + auto import_opts = ImportColumnFamilyOptions{}; + import_opts.move_files = true; + db_->createColumnFamilyWithImport(kTempCf, import_opts, *export_metadata_); + db_->put(kMigrationKey, kStateImportedTempCf); +} + +void BlockMerkleLatestVerCfMigration::clearExistingLatestVerCf() { + // First, get the CF options from the CF descriptor. + auto cf = db_->columnFamilyHandle(BLOCK_MERKLE_LATEST_KEY_VERSION_CF); + auto cf_desc = ColumnFamilyDescriptor{}; + const auto status = cf->GetDescriptor(&cf_desc); + if (!status.ok()) { + throw std::runtime_error{"Failed to get CF descriptor, reason: " + status.ToString()}; + } + + // Then, drop and create the CF. + db_->dropColumnFamily(BLOCK_MERKLE_LATEST_KEY_VERSION_CF); + db_->createColumnFamily(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, cf_desc.options); +} + +void BlockMerkleLatestVerCfMigration::iterateAndMigrate() { + auto blockchain = Blockchain{db_}; + auto key_hashes = std::vector{}; + auto values = std::vector<::rocksdb::PinnableSlice>{}; + auto statuses = std::vector<::rocksdb::Status>{}; + for (auto block_id = INITIAL_GENESIS_BLOCK_ID; block_id <= blockchain.getLastReachableBlockId(); ++block_id) { + const auto block = blockchain.getBlock(block_id); + if (!block.has_value()) { + throw std::runtime_error{"Failed to load block ID = " + std::to_string(block_id)}; + } + auto it = block->data.categories_updates_info.find(kExecutionProvableCategory); + if (it != block->data.categories_updates_info.cend()) { + const auto block_merkle_output = std::get_if(&it->second); + ConcordAssertNE(block_merkle_output, nullptr); + key_hashes.clear(); + for (const auto& [key, _] : block_merkle_output->keys) { + (void)_; + key_hashes.push_back(hash(key)); + } + values.clear(); + statuses.clear(); + auto batch = NativeWriteBatch{db_}; + auto key_it = block_merkle_output->keys.cbegin(); + db_->multiGet(kTempCf, key_hashes, values, statuses); + ConcordAssertEQ(key_hashes.size(), values.size()); + ConcordAssertEQ(key_hashes.size(), statuses.size()); + for (auto i = 0ull; i < values.size(); ++i) { + const auto& value = values[i]; + const auto& status = statuses[i]; + if (status.ok()) { + batch.put(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, key_it->first, value); + } else { + // We expect that no pruning has occurred. Therefore, the latest version of the key cannot be missing. + throw std::runtime_error{"multiGet() failed, reason: " + status.ToString()}; + } + ++key_it; + } + db_->write(std::move(batch)); + } + } + db_->put(kMigrationKey, kStateMigrated); +} + +void BlockMerkleLatestVerCfMigration::dropTempLatestVerCf() { db_->dropColumnFamily(kTempCf); } + +void BlockMerkleLatestVerCfMigration::commitComplete() { db_->put(kMigrationKey, kStateMigrationNotNeededOrCompleted); } + +BlockMerkleLatestVerCfMigration::ExecutionStatus BlockMerkleLatestVerCfMigration::execute() { + auto migrate = [this]() { + clearExistingLatestVerCf(); + iterateAndMigrate(); + }; + + auto cleanup_and_complete = [this]() { + removeExportDir(); + dropTempLatestVerCf(); + commitComplete(); + }; + + const auto state = db_->get(kMigrationKey); + // If we've already completed or we are already at the new DB format, there's nothing for us to do. + if (state && *state == kStateMigrationNotNeededOrCompleted) { + return ExecutionStatus::kNotNeededOrAlreadyExecuted; + } + + // Start by removing the export dir. + removeExportDir(); + + if (state && *state == kStateImportedTempCf) { + // If we've successfully imported the temporary CF, we can migrate. + migrate(); + } else if (state && *state == kStateMigrated) { + // If we've already migrated, we can just cleanup and complete. + } else { + // We need to execute all the steps to migrate. + dropTempLatestVerCf(); + checkpointDB(); + exportLatestVerCf(); + importTempLatestVerCf(); + migrate(); + } + cleanup_and_complete(); + + return ExecutionStatus::kExecuted; +} + +} // namespace concord::kvbc::migrations diff --git a/kvbc/test/CMakeLists.txt b/kvbc/test/CMakeLists.txt index d826579312..744a24c043 100644 --- a/kvbc/test/CMakeLists.txt +++ b/kvbc/test/CMakeLists.txt @@ -256,4 +256,15 @@ if (BUILD_ROCKSDB_STORAGE) kvbc stdc++fs ) + + add_executable(block_merkle_latest_ver_cf_migration_test migrations/block_merkle_latest_ver_cf_migration_test.cpp) + add_test(block_merkle_latest_ver_cf_migration_test block_merkle_latest_ver_cf_migration_test) + target_link_libraries(block_merkle_latest_ver_cf_migration_test PUBLIC + GTest::Main + GTest::GTest + util + corebft + kvbc + stdc++fs + ) endif (BUILD_ROCKSDB_STORAGE) diff --git a/kvbc/test/categorization/block_merkle_category_unit_test.cpp b/kvbc/test/categorization/block_merkle_category_unit_test.cpp index 1aa35df172..04996ee438 100644 --- a/kvbc/test/categorization/block_merkle_category_unit_test.cpp +++ b/kvbc/test/categorization/block_merkle_category_unit_test.cpp @@ -202,9 +202,8 @@ TEST_F(block_merkle_category, put_and_get) { // Get by hash works ASSERT_EQ(expected, asMerkle(cat.get(hashed_key1, block_id).value())); - // Getting the latest version by key and hash works + // Getting the latest version by key works ASSERT_EQ(block_id, cat.getLatestVersion(key1)->encode()); - ASSERT_EQ(block_id, cat.getLatestVersion(hashed_key1)->encode()); // Getting the key at the wrong block fails ASSERT_EQ(false, cat.get(key1, block_id + 1).has_value()); diff --git a/kvbc/test/migrations/block_merkle_latest_ver_cf_migration_test.cpp b/kvbc/test/migrations/block_merkle_latest_ver_cf_migration_test.cpp new file mode 100644 index 0000000000..6f42753b83 --- /dev/null +++ b/kvbc/test/migrations/block_merkle_latest_ver_cf_migration_test.cpp @@ -0,0 +1,421 @@ +// Concord +// +// Copyright (c) 2021 VMware, Inc. All Rights Reserved. +// +// This product is licensed to you under the Apache 2.0 license (the "License"). +// You may not use this product except in compliance with the Apache 2.0 License. +// +// This product may include a number of subcomponents with separate copyright +// notices and license terms. Your use of these subcomponents is subject to the +// terms and conditions of the sub-component's license, as noted in the +// LICENSE file. + +#include "migrations/block_merkle_latest_ver_cf_migration.h" + +#include "gtest/gtest.h" +#include "gmock/gmock.h" + +#include "categorization/column_families.h" +#include "categorization/db_categories.h" +#include "categorization/kv_blockchain.h" +#include "hex_tools.h" +#include "sha_hash.hpp" +#include "storage/test/storage_test_common.h" + +#include + +#include +#include +#include +#include +#include +#include + +namespace { + +using namespace concord::kvbc::categorization; +using namespace concord::kvbc::migrations; +using namespace concord::storage::rocksdb; +using namespace concord::util; +using namespace ::testing; + +using concord::kvbc::categorization::detail::BLOCK_MERKLE_LATEST_KEY_VERSION_CF; + +class block_merkle_latest_ver_cf_migration_test : public Test { + void SetUp() override { + cleanupTestData(); + createKvbc(); + addBlocks(); + } + + void TearDown() override { cleanupTestData(); } + + protected: + auto createMigration() { + return BlockMerkleLatestVerCfMigration{rocksDbPath(db_path_id_), rocksDbPath(export_path_id_)}; + } + + void createKvbc() { + db_ = TestRocksDb::createNative(db_path_id_); + const auto link_st_chain = true; + kvbc_.emplace(db_, + link_st_chain, + std::map{{kExecutionProvableCategory, CATEGORY_TYPE::block_merkle}, + {kConcordInternalCategoryId, CATEGORY_TYPE::versioned_kv}, + {kVersionedCategoryId_, CATEGORY_TYPE::versioned_kv}}); + } + + void closeDb() { + kvbc_.reset(); + db_.reset(); + ASSERT_EQ(0, db_.use_count()); + } + + void cleanupTestData() { + closeDb(); + cleanup(db_path_id_); + cleanup(export_path_id_); + } + + void addBlocks() { + // Block 1: a -> va1, b -> vb1 + { + auto updates = Updates{}; + auto merkle_updates = BlockMerkleUpdates{}; + merkle_updates.addUpdate("a", "va1"); + merkle_updates.addUpdate("b", "vb1"); + updates.add(kExecutionProvableCategory, std::move(merkle_updates)); + kvbc_->addBlock(std::move(updates)); + } + + // Block 2: c -> vc2, a -> va2 + { + auto updates = Updates{}; + auto merkle_updates = BlockMerkleUpdates{}; + merkle_updates.addUpdate("c", "vc2"); + merkle_updates.addUpdate("a", "va2"); + updates.add(kExecutionProvableCategory, std::move(merkle_updates)); + kvbc_->addBlock(std::move(updates)); + } + + // Block 3: no updates + { + auto updates = Updates{}; + kvbc_->addBlock(std::move(updates)); + } + + // Block 4: delete a, d -> vd4 + { + auto updates = Updates{}; + auto merkle_updates = BlockMerkleUpdates{}; + merkle_updates.addDelete("a"); + merkle_updates.addUpdate("d", "vd4"); + updates.add(kExecutionProvableCategory, std::move(merkle_updates)); + kvbc_->addBlock(std::move(updates)); + } + + // Block 5: internal updates only + { + auto updates = Updates{}; + auto internal_updates = VersionedUpdates{}; + internal_updates.addUpdate("internal_key", "internal_value"); + updates.add(kVersionedCategoryId_, std::move(internal_updates)); + kvbc_->addBlock(std::move(updates)); + } + + // Block 6: b -> vb6 + { + auto updates = Updates{}; + auto merkle_updates = BlockMerkleUpdates{}; + merkle_updates.addUpdate("b", "vb6"); + updates.add(kExecutionProvableCategory, std::move(merkle_updates)); + kvbc_->addBlock(std::move(updates)); + } + } + + // Simulate previous format of BLOCK_MERKLE_LATEST_KEY_VERSION_CF, i.e. key_hash -> latest_version. Migration will + // then migrate from the previous version to raw_key -> latest_version. + void downgrade() { + auto batch = NativeWriteBatch{db_}; + auto it = db_->getIterator(BLOCK_MERKLE_LATEST_KEY_VERSION_CF); + + // Firstly, accumulate the key-values in a write batch, with the key hashes as keys. + it.first(); + while (it) { + const auto hash = SHA3_256{}.digest(it.keyView().data(), it.keyView().size()); + batch.put(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, hash, it.valueView()); + it.next(); + } + + // Secondly, remove all keys in the BLOCK_MERKLE_LATEST_KEY_VERSION_CF column family. + { + it.first(); + auto del_batch = NativeWriteBatch{db_}; + del_batch.delRange(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, it.keyView(), kKeyAfterLast); + db_->write(std::move(del_batch)); + } + + // Thirdly, remove the migration key as it is not supposed to be there in the previous DB format. + batch.del(BlockMerkleLatestVerCfMigration::migrationKey()); + + // Finally, write the key-values with the key hashes as keys. + db_->write(std::move(batch)); + + closeDb(); + } + + std::unordered_set columnFamilyKeys(const std::string& columnFamily) const { + auto ret = std::unordered_set{}; + auto iter = db_->getIterator(columnFamily); + iter.first(); + while (iter) { + ret.insert(iter.key()); + iter.next(); + } + return ret; + } + + void printColumnFamilyKeyValues(const std::string& columnFamily) const { + auto iter = db_->getIterator(columnFamily); + iter.first(); + while (iter) { + concordUtils::hexPrint(std::cout, iter.keyView().data(), iter.keyView().size()); + std::cout << ": "; + concordUtils::hexPrint(std::cout, iter.valueView().data(), iter.valueView().size()); + std::cout << std::endl; + iter.next(); + } + } + + std::optional currentState(BlockMerkleLatestVerCfMigration& migration) const { + return migration.db()->get(BlockMerkleLatestVerCfMigration::migrationKey()); + } + + void executeAndVerifyMigration() { + { + auto migration = createMigration(); + const auto status = migration.execute(); + ASSERT_EQ(BlockMerkleLatestVerCfMigration::ExecutionStatus::kExecuted, status); + } + + createKvbc(); + + ASSERT_FALSE(boost::filesystem::exists(rocksDbPath(export_path_id_))); + ASSERT_FALSE(db_->hasColumnFamily(BlockMerkleLatestVerCfMigration::temporaryColumnFamily())); + const auto state = db_->get(BlockMerkleLatestVerCfMigration::migrationKey()); + ASSERT_TRUE(state.has_value()); + ASSERT_EQ(*state, BlockMerkleLatestVerCfMigration::kStateMigrationNotNeededOrCompleted); + + const auto latest_ver_cf_keys = columnFamilyKeys(BLOCK_MERKLE_LATEST_KEY_VERSION_CF); + ASSERT_THAT(latest_ver_cf_keys, ContainerEq(std::unordered_set{"a", "b", "c", "d"})); + printColumnFamilyKeyValues(BLOCK_MERKLE_LATEST_KEY_VERSION_CF); + + // Key "a". + { + const auto a_ver = kvbc_->getLatestVersion(kExecutionProvableCategory, "a"); + ASSERT_TRUE(a_ver.has_value()); + ASSERT_EQ(4, a_ver->version); + ASSERT_TRUE(a_ver->deleted); + + const auto a_val = kvbc_->getLatest(kExecutionProvableCategory, "a"); + ASSERT_FALSE(a_val.has_value()); + } + + // Key "b". + { + const auto b_ver = kvbc_->getLatestVersion(kExecutionProvableCategory, "b"); + ASSERT_TRUE(b_ver.has_value()); + ASSERT_EQ(6, b_ver->version); + ASSERT_FALSE(b_ver->deleted); + + const auto b_val = kvbc_->getLatest(kExecutionProvableCategory, "b"); + ASSERT_TRUE(b_val.has_value()); + const auto& b_merkle_val = std::get(*b_val); + ASSERT_EQ(6, b_merkle_val.block_id); + ASSERT_EQ("vb6", b_merkle_val.data); + } + + // Key "c". + { + const auto c_ver = kvbc_->getLatestVersion(kExecutionProvableCategory, "c"); + ASSERT_TRUE(c_ver.has_value()); + ASSERT_EQ(2, c_ver->version); + ASSERT_FALSE(c_ver->deleted); + + const auto c_val = kvbc_->getLatest(kExecutionProvableCategory, "c"); + ASSERT_TRUE(c_val.has_value()); + const auto& c_merkle_val = std::get(*c_val); + ASSERT_EQ(2, c_merkle_val.block_id); + ASSERT_EQ("vc2", c_merkle_val.data); + } + + // Key "d". + { + const auto d_ver = kvbc_->getLatestVersion(kExecutionProvableCategory, "d"); + ASSERT_TRUE(d_ver.has_value()); + ASSERT_EQ(4, d_ver->version); + ASSERT_FALSE(d_ver->deleted); + + const auto d_val = kvbc_->getLatest(kExecutionProvableCategory, "d"); + ASSERT_TRUE(d_val.has_value()); + const auto& d_merkle_val = std::get(*d_val); + ASSERT_EQ(4, d_merkle_val.block_id); + ASSERT_EQ("vd4", d_merkle_val.data); + } + } + + protected: + const std::string kKeyAfterLast{"e"}; // as last key added is "d" + const std::string kVersionedCategoryId_{"ver"}; + const std::size_t db_path_id_{0}; + const std::size_t export_path_id_{1}; + std::shared_ptr db_; + std::optional kvbc_; +}; + +TEST_F(block_merkle_latest_ver_cf_migration_test, successful_migration) { + downgrade(); + executeAndVerifyMigration(); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, not_needed) { + closeDb(); + // We try to migrate after adding blocks only, before downgrading the DB. + auto migration = createMigration(); + const auto status = migration.execute(); + ASSERT_EQ(BlockMerkleLatestVerCfMigration::ExecutionStatus::kNotNeededOrAlreadyExecuted, status); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, already_executed) { + downgrade(); + executeAndVerifyMigration(); + closeDb(); + auto migration = createMigration(); + const auto status = migration.execute(); + ASSERT_EQ(BlockMerkleLatestVerCfMigration::ExecutionStatus::kNotNeededOrAlreadyExecuted, status); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, stop_before_checkpoint_db) { + downgrade(); + { + auto migration = createMigration(); + migration.removeExportDir(); + migration.dropTempLatestVerCf(); + const auto current_state = currentState(migration); + ASSERT_FALSE(current_state.has_value()); + } + executeAndVerifyMigration(); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, stop_after_checkpoint_db) { + downgrade(); + { + auto migration = createMigration(); + migration.removeExportDir(); + migration.dropTempLatestVerCf(); + migration.checkpointDB(); + const auto current_state = currentState(migration); + ASSERT_FALSE(current_state.has_value()); + } + executeAndVerifyMigration(); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, stop_after_imported_temp_latest_ver_cf) { + downgrade(); + { + auto migration = createMigration(); + migration.removeExportDir(); + migration.dropTempLatestVerCf(); + migration.checkpointDB(); + migration.exportLatestVerCf(); + migration.importTempLatestVerCf(); + const auto current_state = currentState(migration); + ASSERT_TRUE(current_state.has_value()); + ASSERT_EQ(BlockMerkleLatestVerCfMigration::kStateImportedTempCf, *current_state); + } + executeAndVerifyMigration(); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, stop_after_clear_existing_latest_ver_cf) { + downgrade(); + { + auto migration = createMigration(); + migration.removeExportDir(); + migration.dropTempLatestVerCf(); + migration.checkpointDB(); + migration.exportLatestVerCf(); + migration.importTempLatestVerCf(); + migration.clearExistingLatestVerCf(); + const auto current_state = currentState(migration); + ASSERT_TRUE(current_state.has_value()); + ASSERT_EQ(BlockMerkleLatestVerCfMigration::kStateImportedTempCf, *current_state); + } + executeAndVerifyMigration(); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, simulate_stop_in_the_middle_of_iterate_and_migrate) { + downgrade(); + auto dummy_key = std::string{}; + { + auto migration = createMigration(); + migration.removeExportDir(); + migration.dropTempLatestVerCf(); + migration.checkpointDB(); + migration.exportLatestVerCf(); + migration.importTempLatestVerCf(); + migration.clearExistingLatestVerCf(); + const auto current_state = currentState(migration); + ASSERT_TRUE(current_state.has_value()); + ASSERT_EQ(BlockMerkleLatestVerCfMigration::kStateImportedTempCf, *current_state); + + // Insert a dummy key in the BLOCK_MERKLE_LATEST_KEY_VERSION_CF column family, simulating a "partial" migration. + // We expect that this key is not there after migration. + auto iter = migration.db()->getIterator(BlockMerkleLatestVerCfMigration::temporaryColumnFamily()); + iter.first(); + dummy_key = iter.key(); + migration.db()->put(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, dummy_key, iter.value()); + } + executeAndVerifyMigration(); + const auto dummy_key_val = db_->get(BLOCK_MERKLE_LATEST_KEY_VERSION_CF, dummy_key); + ASSERT_FALSE(dummy_key_val.has_value()); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, stop_after_iterate_and_migrate_latest_ver_cf) { + downgrade(); + { + auto migration = createMigration(); + migration.removeExportDir(); + migration.dropTempLatestVerCf(); + migration.checkpointDB(); + migration.exportLatestVerCf(); + migration.importTempLatestVerCf(); + migration.clearExistingLatestVerCf(); + migration.iterateAndMigrate(); + const auto current_state = currentState(migration); + ASSERT_TRUE(current_state.has_value()); + ASSERT_EQ(BlockMerkleLatestVerCfMigration::kStateMigrated, *current_state); + } + executeAndVerifyMigration(); +} + +TEST_F(block_merkle_latest_ver_cf_migration_test, stop_after_commit_complete) { + downgrade(); + { + auto migration = createMigration(); + migration.removeExportDir(); + migration.dropTempLatestVerCf(); + migration.checkpointDB(); + migration.exportLatestVerCf(); + migration.importTempLatestVerCf(); + migration.clearExistingLatestVerCf(); + migration.iterateAndMigrate(); + migration.removeExportDir(); + migration.dropTempLatestVerCf(); + const auto current_state = currentState(migration); + ASSERT_TRUE(current_state.has_value()); + ASSERT_EQ(BlockMerkleLatestVerCfMigration::kStateMigrated, *current_state); + } + executeAndVerifyMigration(); +} + +} // namespace diff --git a/kvbc/tools/CMakeLists.txt b/kvbc/tools/CMakeLists.txt index b99e25f29c..170f334fbf 100644 --- a/kvbc/tools/CMakeLists.txt +++ b/kvbc/tools/CMakeLists.txt @@ -1 +1,2 @@ add_subdirectory(db_editor) +add_subdirectory(migrations) diff --git a/kvbc/tools/migrations/CMakeLists.txt b/kvbc/tools/migrations/CMakeLists.txt new file mode 100644 index 0000000000..4e8a37831b --- /dev/null +++ b/kvbc/tools/migrations/CMakeLists.txt @@ -0,0 +1,8 @@ +find_package(Boost ${MIN_BOOST_VERSION} COMPONENTS program_options REQUIRED) + +add_executable(block_merkle_latest_ver_cf_migration_tool block_merkle_latest_ver_cf_migration_tool.cpp) +target_link_libraries(block_merkle_latest_ver_cf_migration_tool PUBLIC + kvbc + util + ${Boost_LIBRARIES} +) diff --git a/kvbc/tools/migrations/block_merkle_latest_ver_cf_migration_tool.cpp b/kvbc/tools/migrations/block_merkle_latest_ver_cf_migration_tool.cpp new file mode 100644 index 0000000000..3bd823dd9f --- /dev/null +++ b/kvbc/tools/migrations/block_merkle_latest_ver_cf_migration_tool.cpp @@ -0,0 +1,101 @@ +#include "migrations/block_merkle_latest_ver_cf_migration.h" + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace { + +namespace po = boost::program_options; +namespace fs = boost::filesystem; + +std::pair parseArgs(int argc, char* argv[]) { + auto desc = po::options_description( + "Migrates a RocksDB DB from using key hashes in the `block_merkle_latest_key_version` to using raw keys.\n" + "Allowed options"); + + // clang-format off + desc.add_options() + ("help", "Show help.") + + ("rocksdb-path", + po::value(), + "The path to the RocksDB data directory.") + + ("temp-export-path", + po::value(), + "The path to a temporary export directory. Must be on the same filesystem as rocksdb-path."); + // clang-format on + + auto config = po::variables_map{}; + po::store(po::parse_command_line(argc, argv, desc), config); + po::notify(config); + return std::make_pair(desc, config); +} + +int run(int argc, char* argv[]) { + using namespace concord::kvbc::migrations; + + const auto [desc, config] = parseArgs(argc, argv); + + if (config.count("help")) { + std::cout << desc << std::endl; + return EXIT_SUCCESS; + } + + if (config["rocksdb-path"].empty()) { + std::cerr << desc << std::endl; + return EXIT_FAILURE; + } + const auto rocksdb_path = config["rocksdb-path"].as(); + + if (!fs::exists(rocksdb_path)) { + std::cerr << "RocksDB database doesn't exist at " << rocksdb_path << std::endl; + return EXIT_FAILURE; + } + + if (config["temp-export-path"].empty()) { + std::cerr << desc << std::endl; + return EXIT_FAILURE; + } + const auto temp_export_path = config["temp-export-path"].as(); + + if (fs::equivalent(rocksdb_path, temp_export_path)) { + std::cerr << "Error! 'rocksdb-path path' cannot be the same as 'temp-export-path'." << std::endl; + return EXIT_FAILURE; + } + + auto migration = BlockMerkleLatestVerCfMigration{rocksdb_path, temp_export_path}; + const auto status = migration.execute(); + switch (status) { + case BlockMerkleLatestVerCfMigration::ExecutionStatus::kExecuted: + std::cout << "Success! Migration executed successfully!" << std::endl; + break; + + case BlockMerkleLatestVerCfMigration::ExecutionStatus::kNotNeededOrAlreadyExecuted: + std::cout << "Success! Migration not needed or already executed!" << std::endl; + break; + } + + return EXIT_SUCCESS; +} + +} // namespace + +int main(int argc, char* argv[]) { + try { + return run(argc, argv); + } catch (const std::exception& e) { + std::cerr << "Error: " << e.what() << std::endl; + } catch (...) { + std::cerr << "Unknown error" << std::endl; + } + return EXIT_FAILURE; +} diff --git a/storage/include/rocksdb/native_client.h b/storage/include/rocksdb/native_client.h index d9b6ef37aa..df40321f3a 100644 --- a/storage/include/rocksdb/native_client.h +++ b/storage/include/rocksdb/native_client.h @@ -154,6 +154,11 @@ class NativeClient : public std::enable_shared_from_this { // Throws if the column family already exists. void createColumnFamily(const std::string &cFamily, const ::rocksdb::ColumnFamilyOptions &options = ::rocksdb::ColumnFamilyOptions{}); + // Create a column family by importing previously exported SST files. + void createColumnFamilyWithImport(const std::string &cFamily, + const ::rocksdb::ImportColumnFamilyOptions &importOpts, + const ::rocksdb::ExportImportFilesMetaData &metadata, + const ::rocksdb::ColumnFamilyOptions &cfOpts = ::rocksdb::ColumnFamilyOptions{}); // Return the column family options for an existing column family in this client. ::rocksdb::ColumnFamilyOptions columnFamilyOptions(const std::string &cFamily) const; // Drops a column family and its data. It is not an error if the column family doesn't exist or if the client is not diff --git a/storage/include/rocksdb/native_client.ipp b/storage/include/rocksdb/native_client.ipp index 65e75dd3f1..0f6c099ff8 100644 --- a/storage/include/rocksdb/native_client.ipp +++ b/storage/include/rocksdb/native_client.ipp @@ -231,6 +231,17 @@ inline void NativeClient::createColumnFamily(const std::string &cFamily, client_->cf_handles_[cFamily] = std::move(handle); } +inline void NativeClient::createColumnFamilyWithImport(const std::string &cFamily, + const ::rocksdb::ImportColumnFamilyOptions &importOpts, + const ::rocksdb::ExportImportFilesMetaData &metadata, + const ::rocksdb::ColumnFamilyOptions &cfOpts) { + ::rocksdb::ColumnFamilyHandle *handlePtr{nullptr}; + auto s = rawDB().CreateColumnFamilyWithImport(cfOpts, cFamily, importOpts, metadata, &handlePtr); + auto handleUniquePtr = Client::CfUniquePtr{handlePtr, Client::CfDeleter{client_.get()}}; + detail::throwOnError("failed to import column family"sv, cFamily, std::move(s)); + client_->cf_handles_[cFamily] = std::move(handleUniquePtr); +} + inline ::rocksdb::ColumnFamilyOptions NativeClient::columnFamilyOptions(const std::string &cFamily) const { auto family = columnFamilyHandle(cFamily); auto descriptor = ::rocksdb::ColumnFamilyDescriptor{}; diff --git a/storage/include/storage/db_types.h b/storage/include/storage/db_types.h index 7d3453d33d..b770283645 100644 --- a/storage/include/storage/db_types.h +++ b/storage/include/storage/db_types.h @@ -40,11 +40,7 @@ namespace v2MerkleTree::detail { // Top-level DB key types used when saving the blockchain in the form of a merkle tree. // Key types might have subtypes so that the top-level enum is not quickly exhausted and keys are structured in // a clearer way. A note is that there is an overhead of 1 byte in the key length when using subtypes. -enum class EDBKeyType : std::uint8_t { - Block, - BFT, - Key, -}; +enum class EDBKeyType : std::uint8_t { Block, BFT, Key, Migration }; // Key subtypes. Internal and ProvableStale are used internally by the merkle tree implementation. The Leaf type is the // one containing actual application data. @@ -66,6 +62,11 @@ enum class EBFTSubtype : std::uint8_t { STCheckpointDescriptor, STTempBlock, }; + +enum class EMigrationSubType : std::uint8_t { + BlockMerkleLatestVerCfState, +}; + } // namespace v2MerkleTree::detail typedef std::uint32_t ObjectId;